Merge branch 'cassandra-2.1' into cassandra-2.2
diff --git a/.gitignore b/.gitignore
index c7cf9fd..acaa51a 100644
--- a/.gitignore
+++ b/.gitignore
@@ -7,6 +7,7 @@
 logs/
 data/
 conf/hotspot_compiler
+doc/cql3/CQL.html
 
 # C* debs
 build-stamp
@@ -24,6 +25,7 @@
 
 # IntelliJ
 .idea/
+*.eml
 *.iml
 *.ipr
 *.iws
@@ -57,3 +59,15 @@
 *.tmp
 .DS_Store
 Thumbs.db
+
+# JSR223
+lib/jsr223/clojure/*.jar
+lib/jsr223/groovy/*.jar
+lib/jsr223/jaskell/*.jar
+lib/jsr223/jruby/*.jar
+lib/jsr223/jruby/jni
+lib/jsr223/jruby/ruby
+lib/jsr223/jython/*.jar
+lib/jsr223/jython/cachedir
+lib/jsr223/scala/*.jar
+
diff --git a/CHANGES.txt b/CHANGES.txt
index f62f162..56890c9 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,12 +1,66 @@
-2.1.18
+2.2.10
+ * Nodes started with join_ring=False should be able to serve requests when authentication is enabled (CASSANDRA-11381)
+ * cqlsh COPY FROM: increment error count only for failures, not for attempts (CASSANDRA-13209)
+ * nodetool upgradesstables should upgrade system tables (CASSANDRA-13119)
+ * Avoid starting gossiper in RemoveTest (CASSANDRA-13407)
+ * Fix weightedSize() for row-cache reported by JMX and NodeTool (CASSANDRA-13393)
+ * Fix JVM metric paths (CASSANDRA-13103)
+ * Honor truststore-password parameter in cassandra-stress (CASSANDRA-12773)
+ * Discard in-flight shadow round responses (CASSANDRA-12653)
+ * Don't anti-compact repaired data to avoid inconsistencies (CASSANDRA-13153)
+ * Wrong logger name in AnticompactionTask (CASSANDRA-13343)
+ * Fix queries updating multiple time the same list (CASSANDRA-13130)
+ * Fix GRANT/REVOKE when keyspace isn't specified (CASSANDRA-13053)
+ * Avoid race on receiver by starting streaming sender thread after sending init message (CASSANDRA-12886)
+ * Fix "multiple versions of ant detected..." when running ant test (CASSANDRA-13232)
+ * Coalescing strategy sleeps too much (CASSANDRA-13090)
+ * Make sure compaction stats are updated when compaction is interrupted (Backport from 3.0, CASSANDRA-12100)
+ * Fix flaky LongLeveledCompactionStrategyTest (CASSANDRA-12202)
+ * Fix failing COPY TO STDOUT (CASSANDRA-12497)
+ * Fix ColumnCounter::countAll behaviour for reverse queries (CASSANDRA-13222)
+ * Exceptions encountered calling getSeeds() breaks OTC thread (CASSANDRA-13018)
+ * Commitlog replay may fail if last mutation is within 4 bytes of end of segment (CASSANDRA-13282)
+Merged from 2.1:
  * Fix 2ndary indexes on primary key columns to don't create expiring entries (CASSANDRA-13412)
  * Set javac encoding to utf-8 (CASSANDRA-13466)
  * Fix 2ndary index queries on partition keys for tables with static columns (CASSANDRA-13147)
  * Fix ParseError unhashable type list in cqlsh copy from (CASSANDRA-13364)
- * Log stacktrace of uncaught exceptions (CASSANDRA-13108)
  * Remove unused repositories (CASSANDRA-13278)
+ * Log stacktrace of uncaught exceptions (CASSANDRA-13108)
 
-2.1.17
+
+2.2.9
+ * Fix negative mean latency metric (CASSANDRA-12876)
+ * Use only one file pointer when creating commitlog segments (CASSANDRA-12539)
+ * Fix speculative retry bugs (CASSANDRA-13009)
+ * Fix handling of nulls and unsets in IN conditions (CASSANDRA-12981) 
+ * Remove support for non-JavaScript UDFs (CASSANDRA-12883)
+ * Fix DynamicEndpointSnitch noop in multi-datacenter situations (CASSANDRA-13074)
+ * cqlsh copy-from: encode column names to avoid primary key parsing errors (CASSANDRA-12909)
+ * Temporarily fix bug that creates commit log when running offline tools (CASSANDRA-8616)
+ * Reduce granuality of OpOrder.Group during index build (CASSANDRA-12796)
+ * Test bind parameters and unset parameters in InsertUpdateIfConditionTest (CASSANDRA-12980)
+ * Do not specify local address on outgoing connection when listen_on_broadcast_address is set (CASSANDRA-12673)
+ * Use saved tokens when setting local tokens on StorageService.joinRing (CASSANDRA-12935)
+ * cqlsh: fix DESC TYPES errors (CASSANDRA-12914)
+ * Fix leak on skipped SSTables in sstableupgrade (CASSANDRA-12899)
+ * Avoid blocking gossip during pending range calculation (CASSANDRA-12281)
+ * Fix purgeability of tombstones with max timestamp (CASSANDRA-12792)
+ * Fail repair if participant dies during sync or anticompaction (CASSANDRA-12901)
+ * cqlsh COPY: unprotected pk values before converting them if not using prepared statements (CASSANDRA-12863)
+ * Fix Util.spinAssertEquals (CASSANDRA-12283)
+ * Fix potential NPE for compactionstats (CASSANDRA-12462)
+ * Prepare legacy authenticate statement if credentials table initialised after node startup (CASSANDRA-12813)
+ * Change cassandra.wait_for_tracing_events_timeout_secs default to 0 (CASSANDRA-12754)
+ * Clean up permissions when a UDA is dropped (CASSANDRA-12720)
+ * Limit colUpdateTimeDelta histogram updates to reasonable deltas (CASSANDRA-11117)
+ * Fix leak errors and execution rejected exceptions when draining (CASSANDRA-12457)
+ * Fix merkle tree depth calculation (CASSANDRA-12580)
+ * Make Collections deserialization more robust (CASSANDRA-12618)
+ * Better handle invalid system roles table (CASSANDRA-12700)
+ * Split consistent range movement flag correction (CASSANDRA-12786)
+ * CompactionTasks now correctly drops sstables out of compaction when not enough disk space is available (CASSANDRA-12979)
+Merged from 2.1:
  * Use portable stderr for java error in startup (CASSANDRA-13211)
  * Fix Thread Leak in OutboundTcpConnection (CASSANDRA-13204)
  * Coalescing strategy can enter infinite loop (CASSANDRA-13159)
@@ -14,27 +68,83 @@
  * Fix paging for DISTINCT queries on partition keys and static columns (CASSANDRA-13017)
  * Fix race causing infinite loop if Thrift server is stopped before it starts listening (CASSANDRA-12856)
  * cqlsh copy-from: sort user type fields in csv (CASSANDRA-12959)
- * Fix missed signal when commit log segments are recycled (CASSANDRA-13037)
- * Fix RecoveryManagerTruncateTest (CASSANDRA-12802)
  * Don't skip sstables based on maxLocalDeletionTime (CASSANDRA-12765)
 
 
-2.1.16
- * Avoid infinitely looping result set when paging SELECT queries with
-   an IN clause with duplicate keys by treating the IN values as a set instead
-   of a list (CASSANDRA-12420)
+2.2.8
+ * Fix exceptions when enabling gossip on nodes that haven't joined the ring (CASSANDRA-12253)
+ * Fix authentication problem when invoking cqlsh copy from a SOURCE command (CASSANDRA-12642)
+ * Decrement pending range calculator jobs counter in finally block
+  (CASSANDRA-12554)
+ * Add local address entry in PropertyFileSnitch (CASSANDRA-11332)
+ * cqlshlib tests: increase default execute timeout (CASSANDRA-12481)
+ * Forward writes to replacement node when replace_address != broadcast_address (CASSANDRA-8523)
+ * Enable repair -pr and -local together (fix regression of CASSANDRA-7450) (CASSANDRA-12522)
+ * Fail repair on non-existing table (CASSANDRA-12279)
+ * cqlsh copy: fix missing counter values (CASSANDRA-12476)
+ * Move migration tasks to non-periodic queue, assure flush executor shutdown after non-periodic executor (CASSANDRA-12251)
+ * cqlsh copy: fixed possible race in initializing feeding thread (CASSANDRA-11701)
+ * Only set broadcast_rpc_address on Ec2MultiRegionSnitch if it's not set (CASSANDRA-11357)
+ * Update StorageProxy range metrics for timeouts, failures and unavailables (CASSANDRA-9507)
+ * Add Sigar to classes included in clientutil.jar (CASSANDRA-11635)
+ * Add decay to histograms and timers used for metrics (CASSANDRA-11752)
+ * Fix hanging stream session (CASSANDRA-10992)
+ * Add byteman support for testing (CASSANDRA-12377)
+ * Fix INSERT JSON, fromJson() support of smallint, tinyint types (CASSANDRA-12371)
+ * Restore JVM metric export for metric reporters (CASSANDRA-12312)
+ * Release sstables of failed stream sessions only when outgoing transfers are finished (CASSANDRA-11345)
+ * Revert CASSANDRA-11427 (CASSANDRA-12351)
+ * Wait for tracing events before returning response and query at same consistency level client side (CASSANDRA-11465)
+ * cqlsh copyutil should get host metadata by connected address (CASSANDRA-11979)
+ * Fixed cqlshlib.test.remove_test_db (CASSANDRA-12214)
+ * Synchronize ThriftServer::stop() (CASSANDRA-12105)
+ * Use dedicated thread for JMX notifications (CASSANDRA-12146)
+ * NPE when trying to remove purgable tombstones from result (CASSANDRA-12143)
+ * Improve streaming synchronization and fault tolerance (CASSANDRA-11414)
+ * MemoryUtil.getShort() should return an unsigned short also for architectures not supporting unaligned memory accesses (CASSANDRA-11973)
+ * Don't write shadowed range tombstone (CASSANDRA-12030)
+Merged from 2.1:
  * Add system property to set the max number of native transport requests in queue (CASSANDRA-11363)
- * Include column family parameter when -st and -et are provided (CASSANDRA-11866)
- * Fix queries with empty ByteBuffer values in clustering column restrictions (CASSANDRA-12127) 
  * Disable passing control to post-flush after flush failure to prevent data loss (CASSANDRA-11828)
  * Allow STCS-in-L0 compactions to reduce scope with LCS (CASSANDRA-12040)
  * cannot use cql since upgrading python to 2.7.11+ (CASSANDRA-11850)
- * Fix filtering on clustering columns when 2i is used (CASSANDRA-11907)
  * Improve digest calculation in the presence of overlapping tombstones (CASSANDRA-11349)
-
-
-2.1.15
+ * Fix filtering on clustering columns when 2i is used (CASSANDRA-11907)
  * Account for partition deletions in tombstone histogram (CASSANDRA-12112)
+
+
+2.2.7
+ * Allow nodetool info to run with readonly JMX access (CASSANDRA-11755)
+ * Validate bloom_filter_fp_chance against lowest supported
+   value when the table is created (CASSANDRA-11920)
+ * RandomAccessReader: call isEOF() only when rebuffering, not for every read operation (CASSANDRA-12013)
+ * Don't send erroneous NEW_NODE notifications on restart (CASSANDRA-11038)
+ * StorageService shutdown hook should use a volatile variable (CASSANDRA-11984)
+ * Persist local metadata earlier in startup sequence (CASSANDRA-11742)
+ * Run CommitLog tests with different compression settings (CASSANDRA-9039)
+ * cqlsh: fix tab completion for case-sensitive identifiers (CASSANDRA-11664)
+ * Avoid showing estimated key as -1 in tablestats (CASSANDRA-11587)
+ * Fix possible race condition in CommitLog.recover (CASSANDRA-11743)
+ * Enable client encryption in sstableloader with cli options (CASSANDRA-11708)
+ * Possible memory leak in NIODataInputStream (CASSANDRA-11867)
+ * Fix commit log replay after out-of-order flush completion (CASSANDRA-9669)
+ * Add seconds to cqlsh tracing session duration (CASSANDRA-11753)
+ * Prohibit Reverse Counter type as part of the PK (CASSANDRA-9395)
+ * cqlsh: correctly handle non-ascii chars in error messages (CASSANDRA-11626)
+ * Exit JVM if JMX server fails to startup (CASSANDRA-11540)
+ * Produce a heap dump when exiting on OOM (CASSANDRA-9861)
+ * Avoid read repairing purgeable tombstones on range slices (CASSANDRA-11427)
+ * Restore ability to filter on clustering columns when using a 2i (CASSANDRA-11510)
+ * JSON datetime formatting needs timezone (CASSANDRA-11137)
+ * Fix is_dense recalculation for Thrift-updated tables (CASSANDRA-11502)
+ * Remove unnescessary file existence check during anticompaction (CASSANDRA-11660)
+ * Add missing files to debian packages (CASSANDRA-11642)
+ * Avoid calling Iterables::concat in loops during ModificationStatement::getFunctions (CASSANDRA-11621)
+ * cqlsh: COPY FROM should use regular inserts for single statement batches and
+   report errors correctly if workers processes crash on initialization (CASSANDRA-11474)
+ * Always close cluster with connection in CqlRecordWriter (CASSANDRA-11553)
+ * Fix slice queries on ordered COMPACT tables (CASSANDRA-10988)
+Merged from 2.1:
  * Avoid stalling paxos when the paxos state expires (CASSANDRA-12043)
  * Remove finished incoming streaming connections from MessagingService (CASSANDRA-11854)
  * Don't try to get sstables for non-repairing column families (CASSANDRA-12077)
@@ -47,7 +157,6 @@
  * Allow LWT operation on static column with only partition keys (CASSANDRA-10532)
  * Create interval tree over canonical sstables to avoid missing sstables during streaming (CASSANDRA-11886)
  * cqlsh COPY FROM: shutdown parent cluster after forking, to avoid corrupting SSL connections (CASSANDRA-11749)
- * Updated cqlsh Python driver to fix DESCRIBE problem for legacy tables (CASSANDRA-11055)
  * cqlsh: apply current keyspace to source command (CASSANDRA-11152)
  * Backport CASSANDRA-11578 (CASSANDRA-11750)
  * Clear out parent repair session if repair coordinator dies (CASSANDRA-11824)
@@ -57,8 +166,6 @@
  * Add message dropped tasks to nodetool netstats (CASSANDRA-11855)
  * Don't compute expensive MaxPurgeableTimestamp until we've verified there's an 
    expired tombstone (CASSANDRA-11834)
- * Fix paging on DISTINCT queries repeats result when first row in partition changes 
-   (CASSANDRA-11679)
  * Add option to disable use of severity in DynamicEndpointSnitch (CASSANDRA-11737)
  * cqlsh COPY FROM fails for null values with non-prepared statements (CASSANDRA-11631)
  * Make cython optional in pylib/setup.py (CASSANDRA-11630)
@@ -67,21 +174,63 @@
  * cqlsh COPY FROM fails with []{} chars in UDT/tuple fields/values (CASSANDRA-11633)
  * clqsh: COPY FROM throws TypeError with Cython extensions enabled (CASSANDRA-11574)
  * cqlsh: COPY FROM ignores NULL values in conversion (CASSANDRA-11549)
+ * (cqlsh) Fix potential COPY deadlock when parent process is terminating child
+   processes (CASSANDRA-11505)
  * Validate levels when building LeveledScanner to avoid overlaps with orphaned 
    sstables (CASSANDRA-9935)
 
 
-2.1.14
- * Start L0 STCS-compactions even if there is a L0 -> L1 compaction
-   going (CASSANDRA-10979)
- * (cqlsh) Fix potential COPY deadlock when parent process is terminating child
-   processes (CASSANDRA-11505)
- * Replace sstables on DataTracker before marking them as non-compacting during anti-compaction (CASSANDRA-11548)
+2.2.6
+ * Allow only DISTINCT queries with partition keys restrictions (CASSANDRA-11339)
+ * CqlConfigHelper no longer requires both a keystore and truststore to work (CASSANDRA-11532)
+ * Make deprecated repair methods backward-compatible with previous notification service (CASSANDRA-11430)
+ * IncomingStreamingConnection version check message wrong (CASSANDRA-11462)
+ * DatabaseDescriptor should log stacktrace in case of Eception during seed provider creation (CASSANDRA-11312)
+ * Use canonical path for directory in SSTable descriptor (CASSANDRA-10587)
+ * Add cassandra-stress keystore option (CASSANDRA-9325)
+ * Fix out-of-space error treatment in memtable flushing (CASSANDRA-11448).
+ * Dont mark sstables as repairing with sub range repairs (CASSANDRA-11451)
+ * Fix use of NullUpdater for 2i during compaction (CASSANDRA-11450)
+ * Notify when sstables change after cancelling compaction (CASSANDRA-11373)
+ * cqlsh: COPY FROM should check that explicit column names are valid (CASSANDRA-11333)
+ * Add -Dcassandra.start_gossip startup option (CASSANDRA-10809)
+ * Fix UTF8Validator.validate() for modified UTF-8 (CASSANDRA-10748)
+ * Clarify that now() function is calculated on the coordinator node in CQL documentation (CASSANDRA-10900)
+ * Fix bloom filter sizing with LCS (CASSANDRA-11344)
+ * (cqlsh) Fix error when result is 0 rows with EXPAND ON (CASSANDRA-11092)
+ * Fix intra-node serialization issue for multicolumn-restrictions (CASSANDRA-11196)
+ * Non-obsoleting compaction operations over compressed files can impose rate limit on normal reads (CASSANDRA-11301)
+ * Add missing newline at end of bin/cqlsh (CASSANDRA-11325)
+ * Fix AE in nodetool cfstats (backport CASSANDRA-10859) (CASSANDRA-11297)
+ * Unresolved hostname leads to replace being ignored (CASSANDRA-11210)
+ * Fix filtering on non-primary key columns for thrift static column families
+   (CASSANDRA-6377)
+ * Only log yaml config once, at startup (CASSANDRA-11217)
+ * Preserve order for preferred SSL cipher suites (CASSANDRA-11164)
+ * Reference leak with parallel repairs on the same table (CASSANDRA-11215)
+ * Range.compareTo() violates the contract of Comparable (CASSANDRA-11216)
+ * Avoid NPE when serializing ErrorMessage with null message (CASSANDRA-11167)
+ * Replacing an aggregate with a new version doesn't reset INITCOND (CASSANDRA-10840)
+ * (cqlsh) cqlsh cannot be called through symlink (CASSANDRA-11037)
+ * fix ohc and java-driver pom dependencies in build.xml (CASSANDRA-10793)
+ * Protect from keyspace dropped during repair (CASSANDRA-11065)
+ * Handle adding fields to a UDT in SELECT JSON and toJson() (CASSANDRA-11146)
+ * Better error message for cleanup (CASSANDRA-10991)
+ * cqlsh pg-style-strings broken if line ends with ';' (CASSANDRA-11123)
+ * Use cloned TokenMetadata in size estimates to avoid race against membership check
+   (CASSANDRA-10736)
+ * Always persist upsampled index summaries (CASSANDRA-10512)
+ * (cqlsh) Fix inconsistent auto-complete (CASSANDRA-10733)
+ * Make SELECT JSON and toJson() threadsafe (CASSANDRA-11048)
+ * Fix SELECT on tuple relations for mixed ASC/DESC clustering order (CASSANDRA-7281)
+ * (cqlsh) Support utf-8/cp65001 encoding on Windows (CASSANDRA-11030)
+ * Fix paging on DISTINCT queries repeats result when first row in partition changes
+   (CASSANDRA-10010)
+Merged from 2.1:
  * Checking if an unlogged batch is local is inefficient (CASSANDRA-11529)
  * Fix paging for COMPACT tables without clustering columns (CASSANDRA-11467)
- * Fix out-of-space error treatment in memtable flushing (CASSANDRA-11448)
- * Backport CASSANDRA-10859 (CASSANDRA-11415)
- * COPY FROM fails when importing blob (CASSANDRA-11375)
+ * Add a -j parameter to scrub/cleanup/upgradesstables to state how
+   many threads to use (CASSANDRA-11179)
  * Backport CASSANDRA-10679 (CASSANDRA-9598)
  * Don't do defragmentation if reading from repaired sstables (CASSANDRA-10342)
  * Fix streaming_socket_timeout_in_ms not enforced (CASSANDRA-11286)
@@ -96,24 +245,50 @@
  * Fix incorrect warning in 'nodetool status' (CASSANDRA-10176)
  * Properly release sstable ref when doing offline scrub (CASSANDRA-10697)
  * Improve nodetool status performance for large cluster (CASSANDRA-7238)
- * Make it clear what DTCS timestamp_resolution is used for (CASSANDRA-11041)
  * Gossiper#isEnabled is not thread safe (CASSANDRA-11116)
  * Avoid major compaction mixing repaired and unrepaired sstables in DTCS (CASSANDRA-11113)
+ * Make it clear what DTCS timestamp_resolution is used for (CASSANDRA-11041)
  * test_bulk_round_trip_blogposts is failing occasionally (CASSANDRA-10938)
- * Add a -j parameter to scrub/cleanup/upgradesstables to state how
-   many threads to use (CASSANDRA-11179)
+ * (cqlsh) Support timezone conversion using pytz (CASSANDRA-10397)
+ * cqlsh: change default encoding to UTF-8 (CASSANDRA-11124)
 
 
-2.1.13
+2.2.5
+ * maxPurgeableTimestamp needs to check memtables too (CASSANDRA-9949)
+ * Apply change to compaction throughput in real time (CASSANDRA-10025)
+ * Fix potential NPE on ORDER BY queries with IN (CASSANDRA-10955)
+ * Avoid over-fetching during the page of range queries (CASSANDRA-8521)
+ * Start L0 STCS-compactions even if there is a L0 -> L1 compaction
+   going (CASSANDRA-10979)
+ * Make UUID LSB unique per process (CASSANDRA-7925)
+ * Avoid NPE when performing sstable tasks (scrub etc.) (CASSANDRA-10980)
+ * Make sure client gets tombstone overwhelmed warning (CASSANDRA-9465)
+ * Fix error streaming section more than 2GB (CASSANDRA-10961)
+ * (cqlsh) Also apply --connect-timeout to control connection
+   timeout (CASSANDRA-10959)
+ * Histogram buckets exposed in jmx are sorted incorrectly (CASSANDRA-10975)
+ * Enable GC logging by default (CASSANDRA-10140)
+ * Optimize pending range computation (CASSANDRA-9258)
+ * Skip commit log and saved cache directories in SSTable version startup check (CASSANDRA-10902)
+ * drop/alter user should be case sensitive (CASSANDRA-10817)
+ * jemalloc detection fails due to quoting issues in regexv (CASSANDRA-10946)
+ * Support counter-columns for native aggregates (sum,avg,max,min) (CASSANDRA-9977)
+ * (cqlsh) show correct column names for empty result sets (CASSANDRA-9813)
+ * Add new types to Stress (CASSANDRA-9556)
+ * Add property to allow listening on broadcast interface (CASSANDRA-9748)
+ * Fix regression in split size on CqlInputFormat (CASSANDRA-10835)
+ * Better handling of SSL connection errors inter-node (CASSANDRA-10816)
+ * Disable reloading of GossipingPropertyFileSnitch (CASSANDRA-9474)
+ * Verify tables in pseudo-system keyspaces at startup (CASSANDRA-10761)
+ * (cqlsh) encode input correctly when saving history
+Merged from 2.1:
  * Fix isJoined return true only after becoming cluster member (CASANDRA-11007)
  * Fix bad gossip generation seen in long-running clusters (CASSANDRA-10969)
  * Avoid NPE when incremental repair fails (CASSANDRA-10909)
  * Unmark sstables compacting once they are done in cleanup/scrub/upgradesstables (CASSANDRA-10829)
- * Revert CASSANDRA-10012 and add more logging (CASSANDRA-10961)
  * Allow simultaneous bootstrapping with strict consistency when no vnodes are used (CASSANDRA-11005)
  * Log a message when major compaction does not result in a single file (CASSANDRA-10847)
  * (cqlsh) fix cqlsh_copy_tests when vnodes are disabled (CASSANDRA-10997)
- * (cqlsh) fix formatting bytearray values (CASSANDRA-10839)
  * (cqlsh) Add request timeout option to cqlsh (CASSANDRA-10686)
  * Avoid AssertionError while submitting hint with LWT (CASSANDRA-10477)
  * If CompactionMetadata is not in stats file, use index summary instead (CASSANDRA-10676)
@@ -122,7 +297,6 @@
  * Sane default (200Mbps) for inter-DC streaming througput (CASSANDRA-8708)
  * Match cassandra-loader options in COPY FROM (CASSANDRA-9303)
  * Fix binding to any address in CqlBulkRecordWriter (CASSANDRA-9309)
- * Fix the way we replace sstables after anticompaction (CASSANDRA-10831)
  * cqlsh fails to decode utf-8 characters for text typed columns (CASSANDRA-10875)
  * Log error when stream session fails (CASSANDRA-9294)
  * Fix bugs in commit log archiving startup behavior (CASSANDRA-10593)
@@ -136,7 +310,26 @@
  * Fix Stress profile parsing on Windows (CASSANDRA-10808)
 
 
-2.1.12
+2.2.4
+ * Show CQL help in cqlsh in web browser (CASSANDRA-7225)
+ * Serialize on disk the proper SSTable compression ratio (CASSANDRA-10775)
+ * Reject index queries while the index is building (CASSANDRA-8505)
+ * CQL.textile syntax incorrectly includes optional keyspace for aggregate SFUNC and FINALFUNC (CASSANDRA-10747)
+ * Fix JSON update with prepared statements (CASSANDRA-10631)
+ * Don't do anticompaction after subrange repair (CASSANDRA-10422)
+ * Fix SimpleDateType type compatibility (CASSANDRA-10027)
+ * (Hadoop) fix splits calculation (CASSANDRA-10640)
+ * (Hadoop) ensure that Cluster instances are always closed (CASSANDRA-10058)
+ * (cqlsh) show partial trace if incomplete after max_trace_wait (CASSANDRA-7645)
+ * Use most up-to-date version of schema for system tables (CASSANDRA-10652)
+ * Deprecate memory_allocator in cassandra.yaml (CASSANDRA-10581,10628)
+ * Expose phi values from failure detector via JMX and tweak debug
+   and trace logging (CASSANDRA-9526)
+ * Fix RangeNamesQueryPager (CASSANDRA-10509)
+ * Deprecate Pig support (CASSANDRA-10542)
+ * Reduce contention getting instances of CompositeType (CASSANDRA-10433)
+ * Fix IllegalArgumentException in DataOutputBuffer.reallocate for large buffers (CASSANDRA-10592)
+Merged from 2.1:
  * Fix incremental repair hang when replica is down (CASSANDRA-10288)
  * Avoid writing range tombstones after END_OF_ROW marker (CASSANDRA-10791)
  * Optimize the way we check if a token is repaired in anticompaction (CASSANDRA-10768)
@@ -149,7 +342,6 @@
  * Properly reject counters as map keys (CASSANDRA-10760)
  * Fix the sstable-needs-cleanup check (CASSANDRA-10740)
  * (cqlsh) Print column names before COPY operation (CASSANDRA-8935)
- * Add Native-Transport-Requests back to tpstats (CASSANDRA-10044)
  * Make paging logic consistent between searcher impls (CASSANDRA-10683)
  * Fix CompressedInputStream for proper cleanup (CASSANDRA-10012)
  * (cqlsh) Support counters in COPY commands (CASSANDRA-9043)
@@ -181,25 +373,41 @@
  * Don't try to get ancestors from half-renamed sstables (CASSANDRA-10501)
  * Avoid repetition of JVM_OPTS in debian package (CASSANDRA-10251)
  * Fix potential NPE from handling result of SIM.highestSelectivityIndex (CASSANDRA-10550)
- * Fix paging issues with partitions containing only static columns data
-   (CASSANDRA-10381)
+ * Fix paging issues with partitions containing only static columns data (CASSANDRA-10381)
  * Fix conditions on static columns (CASSANDRA-10264)
  * AssertionError: attempted to delete non-existing file CommitLog (CASSANDRA-10377)
- * Merge range tombstones during compaction (CASSANDRA-7953)
  * (cqlsh) Distinguish negative and positive infinity in output (CASSANDRA-10523)
  * (cqlsh) allow custom time_format for COPY TO (CASSANDRA-8970)
  * Don't allow startup if the node's rack has changed (CASSANDRA-10242)
  * Fix sorting for queries with an IN condition on partition key columns (CASSANDRA-10363)
 
 
-2.1.11
+2.2.3
+ * Avoid NoClassDefFoundError during DataDescriptor initialization on windows (CASSANDRA-10412)
+ * Preserve case of quoted Role & User names (CASSANDRA-10394)
+ * cqlsh pg-style-strings broken (CASSANDRA-10484)
+ * Make Hadoop CF splits more polite to custom orderered partitioners (CASSANDRA-10400)
+ * Fix the regression when using LIMIT with aggregates (CASSANDRA-10487)
+Merged from 2.1:
  * Fix mmap file segment seeking to EOF (CASSANDRA-10478)
  * Allow LOCAL_JMX to be easily overridden (CASSANDRA-10275)
  * Mark nodes as dead even if they've already left (CASSANDRA-10205)
  * Update internal python driver used by cqlsh (CASSANDRA-10161, CASSANDRA-10507)
 
 
-2.1.10
+2.2.2
+ * cqlsh prompt includes name of keyspace after failed `use` statement (CASSANDRA-10369)
+ * Configurable page size in cqlsh (CASSANDRA-9855)
+ * Defer default role manager setup until all nodes are on 2.2+ (CASSANDRA-9761)
+ * Cancel transaction for sstables we wont redistribute index summary
+   for (CASSANDRA-10270)
+ * Handle missing RoleManager in config after upgrade to 2.2 (CASSANDRA-10209) 
+ * Retry snapshot deletion after compaction and gc on Windows (CASSANDRA-10222)
+ * Fix failure to start with space in directory path on Windows (CASSANDRA-10239)
+ * Fix repair hang when snapshot failed (CASSANDRA-10057)
+ * Fall back to 1/4 commitlog volume for commitlog_total_space on small disks
+   (CASSANDRA-10199)
+Merged from 2.1:
  * Bulk Loader API could not tolerate even node failure (CASSANDRA-10347)
  * Avoid misleading pushed notifications when multiple nodes
    share an rpc_address (CASSANDRA-10052)
@@ -211,7 +419,7 @@
  * (Pig) support BulkOutputFormat as a URL parameter (CASSANDRA-7410)
  * BATCH statement is broken in cqlsh (CASSANDRA-10272)
  * Added configurable warning threshold for GC duration (CASSANDRA-8907)
- * (cqlsh) Make cqlsh PEP8 compliant (CASSANDRA-10066)
+ * (cqlsh) Make cqlsh PEP8 Compliant (CASSANDRA-10066)
  * (cqlsh) Fix error when starting cqlsh with --debug (CASSANDRA-10282)
  * Scrub, Cleanup and Upgrade do not unmark compacting until all operations
    have completed, regardless of the occurence of exceptions (CASSANDRA-10274)
@@ -219,13 +427,32 @@
  * Only check KeyCache when it is enabled
  * Change streaming_socket_timeout_in_ms default to 1 hour (CASSANDRA-8611)
  * (cqlsh) update list of CQL keywords (CASSANDRA-9232)
+ * Add nodetool gettraceprobability command (CASSANDRA-10234)
 Merged from 2.0:
  * Fix rare race where older gossip states can be shadowed (CASSANDRA-10366)
  * Fix consolidating racks violating the RF contract (CASSANDRA-10238)
  * Disallow decommission when node is in drained state (CASSANDRA-8741)
 
 
-2.1.9
+2.2.1
+ * Fix race during construction of commit log (CASSANDRA-10049)
+ * Fix LeveledCompactionStrategyTest (CASSANDRA-9757)
+ * Fix broken UnbufferedDataOutputStreamPlus.writeUTF (CASSANDRA-10203)
+ * (cqlsh) add CLEAR command (CASSANDRA-10086)
+ * Support string literals as Role names for compatibility (CASSANDRA-10135)
+ * Allow count(*) and count(1) to be use as normal aggregation (CASSANDRA-10114)
+ * An NPE is thrown if the column name is unknown for an IN relation (CASSANDRA-10043)
+ * Apply commit_failure_policy to more errors on startup (CASSANDRA-9749)
+ * Fix histogram overflow exception (CASSANDRA-9973)
+ * Route gossip messages over dedicated socket (CASSANDRA-9237)
+ * Add checksum to saved cache files (CASSANDRA-9265)
+ * Log warning when using an aggregate without partition key (CASSANDRA-9737)
+ * Avoid grouping sstables for anticompaction with DTCS (CASSANDRA-9900)
+ * UDF / UDA execution time in trace (CASSANDRA-9723)
+ * Fix broken internode SSL (CASSANDRA-9884)
+Merged from 2.1:
+ * Change streaming_socket_timeout_in_ms default to 1 hour (CASSANDRA-8611)
+ * (cqlsh) update list of CQL keywords (CASSANDRA-9232)
  * Avoid race condition during read repair (CASSANDRA-9460)
  * (cqlsh) default load-from-file encoding to utf-8 (CASSANDRA-9898)
  * Avoid returning Permission.NONE when failing to query users table (CASSANDRA-10168)
@@ -239,53 +466,85 @@
  * Release snapshot selfRef when doing snapshot repair (CASSANDRA-9998)
  * Cannot replace token does not exist - DN node removed as Fat Client (CASSANDRA-9871)
  * 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)
- * 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)
- * Fix clientutil jar and tests (CASSANDRA-9760)
- * (cqlsh) Allow the SSL protocol version to be specified through the
-   config file or environment variables (CASSANDRA-9544)
  * 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)
  * Fix nodetool info error when the node is not joined (CASSANDRA-9031)
 Merged from 2.0:
  * Make getFullyExpiredSSTables less expensive (CASSANDRA-9882)
+ * 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)
+ * Fix clientutil jar and tests (CASSANDRA-9760)
+ * (cqlsh) Allow the SSL protocol version to be specified through the
+   config file or environment variables (CASSANDRA-9544)
+Merged from 2.0:
  * Add tool to find why expired sstables are not getting dropped (CASSANDRA-10015)
  * Remove erroneous pending HH tasks from tpstats/jmx (CASSANDRA-9129)
  * Don't cast expected bf size to an int (CASSANDRA-9959)
- * Log when messages are dropped due to cross_node_timeout (CASSANDRA-9793)
  * checkForEndpointCollision fails for legitimate collisions (CASSANDRA-9765)
  * Complete CASSANDRA-8448 fix (CASSANDRA-9519)
  * Don't include auth credentials in debug log (CASSANDRA-9682)
  * Can't transition from write survey to normal mode (CASSANDRA-9740)
- * Scrub (recover) sstables even when -Index.db is missing, (CASSANDRA-9591)
+ * Scrub (recover) sstables even when -Index.db is missing (CASSANDRA-9591)
  * Fix growing pending background compaction (CASSANDRA-9662)
- * Don't track hotness when opening from snapshot for validation (CASSANDRA-9382)
 
 
-2.1.8
+2.2.0-rc2
+ * Re-enable memory-mapped I/O on Windows (CASSANDRA-9658)
+ * Warn when an extra-large partition is compacted (CASSANDRA-9643)
+ * (cqlsh) Allow setting the initial connection timeout (CASSANDRA-9601)
+ * BulkLoader has --transport-factory option but does not use it (CASSANDRA-9675)
+ * Allow JMX over SSL directly from nodetool (CASSANDRA-9090)
+ * Update cqlsh for UDFs (CASSANDRA-7556)
+ * Change Windows kernel default timer resolution (CASSANDRA-9634)
+ * Deprected sstable2json and json2sstable (CASSANDRA-9618)
+ * Allow native functions in user-defined aggregates (CASSANDRA-9542)
+ * Don't repair system_distributed by default (CASSANDRA-9621)
+ * Fix mixing min, max, and count aggregates for blob type (CASSANRA-9622)
+ * Rename class for DATE type in Java driver (CASSANDRA-9563)
+ * Duplicate compilation of UDFs on coordinator (CASSANDRA-9475)
+ * Fix connection leak in CqlRecordWriter (CASSANDRA-9576)
+ * Mlockall before opening system sstables & remove boot_without_jna option (CASSANDRA-9573)
+ * Add functions to convert timeuuid to date or time, deprecate dateOf and unixTimestampOf (CASSANDRA-9229)
+ * Make sure we cancel non-compacting sstables from LifecycleTransaction (CASSANDRA-9566)
+ * Fix deprecated repair JMX API (CASSANDRA-9570)
+ * Add logback metrics (CASSANDRA-9378)
+ * Update and refactor ant test/test-compression to run the tests in parallel (CASSANDRA-9583)
+ * Fix upgrading to new directory for secondary index (CASSANDRA-9687)
+Merged from 2.1:
  * (cqlsh) Fix bad check for CQL compatibility when DESCRIBE'ing
    COMPACT STORAGE tables with no clustering columns
- * Warn when an extra-large partition is compacted (CASSANDRA-9643)
  * Eliminate strong self-reference chains in sstable ref tidiers (CASSANDRA-9656)
  * Ensure StreamSession uses canonical sstable reader instances (CASSANDRA-9700) 
  * Ensure memtable book keeping is not corrupted in the event we shrink usage (CASSANDRA-9681)
  * Update internal python driver for cqlsh (CASSANDRA-9064)
  * Fix IndexOutOfBoundsException when inserting tuple with too many
    elements using the string literal notation (CASSANDRA-9559)
- * Allow JMX over SSL directly from nodetool (CASSANDRA-9090)
- * Fix incorrect result for IN queries where column not found (CASSANDRA-9540)
  * Enable describe on indices (CASSANDRA-7814)
+ * Fix incorrect result for IN queries where column not found (CASSANDRA-9540)
  * ColumnFamilyStore.selectAndReference may block during compaction (CASSANDRA-9637)
+ * Fix bug in cardinality check when compacting (CASSANDRA-9580)
+ * Fix memory leak in Ref due to ConcurrentLinkedQueue.remove() behaviour (CASSANDRA-9549)
+ * Make rebuild only run one at a time (CASSANDRA-9119)
 Merged from 2.0:
  * Avoid NPE in AuthSuccess#decode (CASSANDRA-9727)
  * Add listen_address to system.local (CASSANDRA-9603)
  * Bug fixes to resultset metadata construction (CASSANDRA-9636)
  * Fix setting 'durable_writes' in ALTER KEYSPACE (CASSANDRA-9560)
- * Avoid ballot clash in Paxos (CASSANDRA-9649)
+ * Avoids ballot clash in Paxos (CASSANDRA-9649)
  * Improve trace messages for RR (CASSANDRA-9479)
  * Fix suboptimal secondary index selection when restricted
    clustering column is also indexed (CASSANDRA-9631)
@@ -293,20 +552,35 @@
  * Fix error message when attempting to create an index on a column
    in a COMPACT STORAGE table with clustering columns (CASSANDRA-9527)
  * 'WITH WITH' in alter keyspace statements causes NPE (CASSANDRA-9565)
-
-
-2.1.7
- * Fix bug in cardinality check when compacting (CASSANDRA-9580)
- * Fix memory leak in Ref due to ConcurrentLinkedQueue.remove() behaviour (CASSANDRA-9549)
-Merged from 2.0:
  * Expose some internals of SelectStatement for inspection (CASSANDRA-9532)
  * ArrivalWindow should use primitives (CASSANDRA-9496)
  * Periodically submit background compaction tasks (CASSANDRA-9592)
  * Set HAS_MORE_PAGES flag to false when PagingState is null (CASSANDRA-9571)
- * Make rebuild only run one at a time (CASSANDRA-9119)
 
 
-2.1.6
+2.2.0-rc1
+ * Compressed commit log should measure compressed space used (CASSANDRA-9095)
+ * Fix comparison bug in CassandraRoleManager#collectRoles (CASSANDRA-9551)
+ * Add tinyint,smallint,time,date support for UDFs (CASSANDRA-9400)
+ * Deprecates SSTableSimpleWriter and SSTableSimpleUnsortedWriter (CASSANDRA-9546)
+ * Empty INITCOND treated as null in aggregate (CASSANDRA-9457)
+ * Remove use of Cell in Thrift MapReduce classes (CASSANDRA-8609)
+ * Integrate pre-release Java Driver 2.2-rc1, custom build (CASSANDRA-9493)
+ * Clean up gossiper logic for old versions (CASSANDRA-9370)
+ * Fix custom payload coding/decoding to match the spec (CASSANDRA-9515)
+ * ant test-all results incomplete when parsed (CASSANDRA-9463)
+ * Disallow frozen<> types in function arguments and return types for
+   clarity (CASSANDRA-9411)
+ * Static Analysis to warn on unsafe use of Autocloseable instances (CASSANDRA-9431)
+ * Update commitlog archiving examples now that commitlog segments are
+   not recycled (CASSANDRA-9350)
+ * Extend Transactional API to sstable lifecycle management (CASSANDRA-8568)
+ * (cqlsh) Add support for native protocol 4 (CASSANDRA-9399)
+ * Ensure that UDF and UDAs are keyspace-isolated (CASSANDRA-9409)
+ * Revert CASSANDRA-7807 (tracing completion client notifications) (CASSANDRA-9429)
+ * Add ability to stop compaction by ID (CASSANDRA-7207)
+ * Let CassandraVersion handle SNAPSHOT version (CASSANDRA-9438)
+Merged from 2.1:
  * (cqlsh) Fix using COPY through SOURCE or -f (CASSANDRA-9083)
  * Fix occasional lack of `system` keyspace in schema tables (CASSANDRA-8487)
  * Use ProtocolError code instead of ServerError code for native protocol
@@ -321,6 +595,133 @@
  * Improve estimated row count (CASSANDRA-9107)
  * Optimize range tombstone memory footprint (CASSANDRA-8603)
  * Use configured gcgs in anticompaction (CASSANDRA-9397)
+Merged from 2.0:
+ * Don't accumulate more range than necessary in RangeTombstone.Tracker (CASSANDRA-9486)
+ * Add broadcast and rpc addresses to system.local (CASSANDRA-9436)
+ * Always mark sstable suspect when corrupted (CASSANDRA-9478)
+ * Add database users and permissions to CQL3 documentation (CASSANDRA-7558)
+ * Allow JVM_OPTS to be passed to standalone tools (CASSANDRA-5969)
+ * Fix bad condition in RangeTombstoneList (CASSANDRA-9485)
+ * Fix potential StackOverflow when setting CrcCheckChance over JMX (CASSANDRA-9488)
+ * Fix null static columns in pages after the first, paged reversed
+   queries (CASSANDRA-8502)
+ * Fix counting cache serialization in request metrics (CASSANDRA-9466)
+ * Add option not to validate atoms during scrub (CASSANDRA-9406)
+
+
+2.2.0-beta1
+ * Introduce Transactional API for internal state changes (CASSANDRA-8984)
+ * Add a flag in cassandra.yaml to enable UDFs (CASSANDRA-9404)
+ * Better support of null for UDF (CASSANDRA-8374)
+ * Use ecj instead of javassist for UDFs (CASSANDRA-8241)
+ * faster async logback configuration for tests (CASSANDRA-9376)
+ * Add `smallint` and `tinyint` data types (CASSANDRA-8951)
+ * Avoid thrift schema creation when native driver is used in stress tool (CASSANDRA-9374)
+ * Make Functions.declared thread-safe
+ * Add client warnings to native protocol v4 (CASSANDRA-8930)
+ * Allow roles cache to be invalidated (CASSANDRA-8967)
+ * Upgrade Snappy (CASSANDRA-9063)
+ * Don't start Thrift rpc by default (CASSANDRA-9319)
+ * Only stream from unrepaired sstables with incremental repair (CASSANDRA-8267)
+ * Aggregate UDFs allow SFUNC return type to differ from STYPE if FFUNC specified (CASSANDRA-9321)
+ * Remove Thrift dependencies in bundled tools (CASSANDRA-8358)
+ * Disable memory mapping of hsperfdata file for JVM statistics (CASSANDRA-9242)
+ * Add pre-startup checks to detect potential incompatibilities (CASSANDRA-8049)
+ * Distinguish between null and unset in protocol v4 (CASSANDRA-7304)
+ * Add user/role permissions for user-defined functions (CASSANDRA-7557)
+ * Allow cassandra config to be updated to restart daemon without unloading classes (CASSANDRA-9046)
+ * Don't initialize compaction writer before checking if iter is empty (CASSANDRA-9117)
+ * Don't execute any functions at prepare-time (CASSANDRA-9037)
+ * Share file handles between all instances of a SegmentedFile (CASSANDRA-8893)
+ * Make it possible to major compact LCS (CASSANDRA-7272)
+ * Make FunctionExecutionException extend RequestExecutionException
+   (CASSANDRA-9055)
+ * Add support for SELECT JSON, INSERT JSON syntax and new toJson(), fromJson()
+   functions (CASSANDRA-7970)
+ * Optimise max purgeable timestamp calculation in compaction (CASSANDRA-8920)
+ * Constrain internode message buffer sizes, and improve IO class hierarchy (CASSANDRA-8670) 
+ * New tool added to validate all sstables in a node (CASSANDRA-5791)
+ * Push notification when tracing completes for an operation (CASSANDRA-7807)
+ * Delay "node up" and "node added" notifications until native protocol server is started (CASSANDRA-8236)
+ * Compressed Commit Log (CASSANDRA-6809)
+ * Optimise IntervalTree (CASSANDRA-8988)
+ * Add a key-value payload for third party usage (CASSANDRA-8553, 9212)
+ * Bump metrics-reporter-config dependency for metrics 3.0 (CASSANDRA-8149)
+ * Partition intra-cluster message streams by size, not type (CASSANDRA-8789)
+ * Add WriteFailureException to native protocol, notify coordinator of
+   write failures (CASSANDRA-8592)
+ * Convert SequentialWriter to nio (CASSANDRA-8709)
+ * Add role based access control (CASSANDRA-7653, 8650, 7216, 8760, 8849, 8761, 8850)
+ * Record client ip address in tracing sessions (CASSANDRA-8162)
+ * Indicate partition key columns in response metadata for prepared
+   statements (CASSANDRA-7660)
+ * Merge UUIDType and TimeUUIDType parse logic (CASSANDRA-8759)
+ * Avoid memory allocation when searching index summary (CASSANDRA-8793)
+ * Optimise (Time)?UUIDType Comparisons (CASSANDRA-8730)
+ * Make CRC32Ex into a separate maven dependency (CASSANDRA-8836)
+ * Use preloaded jemalloc w/ Unsafe (CASSANDRA-8714, 9197)
+ * Avoid accessing partitioner through StorageProxy (CASSANDRA-8244, 8268)
+ * Upgrade Metrics library and remove depricated metrics (CASSANDRA-5657)
+ * Serializing Row cache alternative, fully off heap (CASSANDRA-7438)
+ * Duplicate rows returned when in clause has repeated values (CASSANDRA-6706)
+ * Make CassandraException unchecked, extend RuntimeException (CASSANDRA-8560)
+ * Support direct buffer decompression for reads (CASSANDRA-8464)
+ * DirectByteBuffer compatible LZ4 methods (CASSANDRA-7039)
+ * Group sstables for anticompaction correctly (CASSANDRA-8578)
+ * Add ReadFailureException to native protocol, respond
+   immediately when replicas encounter errors while handling
+   a read request (CASSANDRA-7886)
+ * Switch CommitLogSegment from RandomAccessFile to nio (CASSANDRA-8308)
+ * Allow mixing token and partition key restrictions (CASSANDRA-7016)
+ * Support index key/value entries on map collections (CASSANDRA-8473)
+ * Modernize schema tables (CASSANDRA-8261)
+ * Support for user-defined aggregation functions (CASSANDRA-8053)
+ * Fix NPE in SelectStatement with empty IN values (CASSANDRA-8419)
+ * Refactor SelectStatement, return IN results in natural order instead
+   of IN value list order and ignore duplicate values in partition key IN restrictions (CASSANDRA-7981)
+ * Support UDTs, tuples, and collections in user-defined
+   functions (CASSANDRA-7563)
+ * Fix aggregate fn results on empty selection, result column name,
+   and cqlsh parsing (CASSANDRA-8229)
+ * Mark sstables as repaired after full repair (CASSANDRA-7586)
+ * Extend Descriptor to include a format value and refactor reader/writer
+   APIs (CASSANDRA-7443)
+ * Integrate JMH for microbenchmarks (CASSANDRA-8151)
+ * Keep sstable levels when bootstrapping (CASSANDRA-7460)
+ * Add Sigar library and perform basic OS settings check on startup (CASSANDRA-7838)
+ * Support for aggregation functions (CASSANDRA-4914)
+ * Remove cassandra-cli (CASSANDRA-7920)
+ * Accept dollar quoted strings in CQL (CASSANDRA-7769)
+ * Make assassinate a first class command (CASSANDRA-7935)
+ * Support IN clause on any partition key column (CASSANDRA-7855)
+ * Support IN clause on any clustering column (CASSANDRA-4762)
+ * Improve compaction logging (CASSANDRA-7818)
+ * Remove YamlFileNetworkTopologySnitch (CASSANDRA-7917)
+ * Do anticompaction in groups (CASSANDRA-6851)
+ * Support user-defined functions (CASSANDRA-7395, 7526, 7562, 7740, 7781, 7929,
+   7924, 7812, 8063, 7813, 7708)
+ * Permit configurable timestamps with cassandra-stress (CASSANDRA-7416)
+ * Move sstable RandomAccessReader to nio2, which allows using the
+   FILE_SHARE_DELETE flag on Windows (CASSANDRA-4050)
+ * Remove CQL2 (CASSANDRA-5918)
+ * Optimize fetching multiple cells by name (CASSANDRA-6933)
+ * Allow compilation in java 8 (CASSANDRA-7028)
+ * Make incremental repair default (CASSANDRA-7250)
+ * Enable code coverage thru JaCoCo (CASSANDRA-7226)
+ * Switch external naming of 'column families' to 'tables' (CASSANDRA-4369) 
+ * Shorten SSTable path (CASSANDRA-6962)
+ * Use unsafe mutations for most unit tests (CASSANDRA-6969)
+ * Fix race condition during calculation of pending ranges (CASSANDRA-7390)
+ * Fail on very large batch sizes (CASSANDRA-8011)
+ * Improve concurrency of repair (CASSANDRA-6455, 8208, 9145)
+ * Select optimal CRC32 implementation at runtime (CASSANDRA-8614)
+ * Evaluate MurmurHash of Token once per query (CASSANDRA-7096)
+ * Generalize progress reporting (CASSANDRA-8901)
+ * Resumable bootstrap streaming (CASSANDRA-8838, CASSANDRA-8942)
+ * Allow scrub for secondary index (CASSANDRA-5174)
+ * Save repair data to system table (CASSANDRA-5839)
+ * fix nodetool names that reference column families (CASSANDRA-8872)
+ Merged from 2.1:
  * Warn on misuse of unlogged batches (CASSANDRA-9282)
  * Failure detector detects and ignores local pauses (CASSANDRA-9183)
  * Add utility class to support for rate limiting a given log statement (CASSANDRA-9029)
@@ -347,21 +748,10 @@
  * Fix streaming not holding ref when stream error (CASSANDRA-9295)
  * Fix canonical view returning early opened SSTables (CASSANDRA-9396)
 Merged from 2.0:
- * Don't accumulate more range than necessary in RangeTombstone.Tracker (CASSANDRA-9486)
- * Add broadcast and rpc addresses to system.local (CASSANDRA-9436)
- * Always mark sstable suspect when corrupted (CASSANDRA-9478)
- * Add database users and permissions to CQL3 documentation (CASSANDRA-7558)
- * Allow JVM_OPTS to be passed to standalone tools (CASSANDRA-5969)
- * Fix bad condition in RangeTombstoneList (CASSANDRA-9485)
- * Fix potential StackOverflow when setting CrcCheckChance over JMX (CASSANDRA-9488)
- * Fix null static columns in pages after the first, paged reversed
-   queries (CASSANDRA-8502)
- * Fix counting cache serialization in request metrics (CASSANDRA-9466)
  * (cqlsh) Add LOGIN command to switch users (CASSANDRA-7212)
  * Clone SliceQueryFilter in AbstractReadCommand implementations (CASSANDRA-8940)
  * Push correct protocol notification for DROP INDEX (CASSANDRA-9310)
  * token-generator - generated tokens too long (CASSANDRA-9300)
- * Add option not to validate atoms during scrub (CASSANDRA-9406)
  * Fix counting of tombstones for TombstoneOverwhelmingException (CASSANDRA-9299)
  * Fix ReconnectableSnitch reconnecting to peers during upgrade (CASSANDRA-6702)
  * Include keyspace and table name in error log for collections over the size
@@ -388,11 +778,11 @@
    key parameter (CASSANDRA-6458)
  * Don't check other keyspaces for schema changes when an user-defined
    type is altered (CASSANDRA-9187)
+ * Add generate-idea-files target to build.xml (CASSANDRA-9123)
  * Allow takeColumnFamilySnapshot to take a list of tables (CASSANDRA-8348)
  * Limit major sstable operations to their canonical representation (CASSANDRA-8669)
  * cqlsh: Add tests for INSERT and UPDATE tab completion (CASSANDRA-9125)
  * cqlsh: quote column names when needed in COPY FROM inserts (CASSANDRA-9080)
- * Add generate-idea-files target to build.xml (CASSANDRA-9123)
  * Do not load read meter for offline operations (CASSANDRA-9082)
  * cqlsh: Make CompositeType data readable (CASSANDRA-8919)
  * cqlsh: Fix display of triggers (CASSANDRA-9081)
@@ -420,12 +810,10 @@
  * Don't check if an sstable is live for offline compactions (CASSANDRA-8841)
  * Don't set clientMode in SSTableLoader (CASSANDRA-8238)
  * Fix SSTableRewriter with disabled early open (CASSANDRA-8535)
- * Allow invalidating permissions and cache time (CASSANDRA-8722)
- * Log warning when queries that will require ALLOW FILTERING in Cassandra 3.0
-   are executed (CASSANDRA-8418)
  * Fix cassandra-stress so it respects the CL passed in user mode (CASSANDRA-8948)
  * Fix rare NPE in ColumnDefinition#hasIndexOption() (CASSANDRA-8786)
  * cassandra-stress reports per-operation statistics, plus misc (CASSANDRA-8769)
+ * Add SimpleDate (cql date) and Time (cql time) types (CASSANDRA-7523)
  * Use long for key count in cfstats (CASSANDRA-8913)
  * Make SSTableRewriter.abort() more robust to failure (CASSANDRA-8832)
  * Remove cold_reads_to_omit from STCS (CASSANDRA-8860)
@@ -603,7 +991,6 @@
  * BTree updates may call provided update function twice (CASSANDRA-8018)
  * Release sstable references after anticompaction (CASSANDRA-8386)
  * Handle abort() in SSTableRewriter properly (CASSANDRA-8320)
- * Fix high size calculations for prepared statements (CASSANDRA-8231)
  * Centralize shared executors (CASSANDRA-8055)
  * Fix filtering for CONTAINS (KEY) relations on frozen collection
    clustering columns when the query is restricted to a single
@@ -630,6 +1017,7 @@
    (CASSANDRA-8619)
  * Round up time deltas lower than 1ms in BulkLoader (CASSANDRA-8645)
  * Add batch remove iterator to ABSC (CASSANDRA-8414, 8666)
+ * Round up time deltas lower than 1ms in BulkLoader (CASSANDRA-8645)
  * Fix isClientMode check in Keyspace (CASSANDRA-8687)
  * Use more efficient slice size for querying internal secondary
    index tables (CASSANDRA-8550)
@@ -803,6 +1191,7 @@
  * Avoid NPEs when receiving type changes for an unknown keyspace (CASSANDRA-7689)
  * Add support for custom 2i validation (CASSANDRA-7575)
  * Pig support for hadoop CqlInputFormat (CASSANDRA-6454)
+ * Add duration mode to cassandra-stress (CASSANDRA-7468)
  * Add listen_interface and rpc_interface options (CASSANDRA-7417)
  * Improve schema merge performance (CASSANDRA-7444)
  * Adjust MT depth based on # of partition validating (CASSANDRA-5263)
@@ -947,6 +1336,8 @@
  * Bogus deserialization of static cells from sstable (CASSANDRA-7684)
  * Fix NPE on compaction leftover cleanup for dropped table (CASSANDRA-7770)
 Merged from 2.0:
+ * Fix race condition in StreamTransferTask that could lead to
+   infinite loops and premature sstable deletion (CASSANDRA-7704)
  * (cqlsh) Wait up to 10 sec for a tracing session (CASSANDRA-7222)
  * Fix NPE in FileCacheService.sizeInBytes (CASSANDRA-7756)
  * Remove duplicates from StorageService.getJoiningNodes (CASSANDRA-7478)
@@ -1015,7 +1406,7 @@
  * Updated memtable_cleanup_threshold and memtable_flush_writers defaults 
    (CASSANDRA-7551)
  * (Windows) fix startup when WMI memory query fails (CASSANDRA-7505)
- * Anti-compaction proceeds if any part of the repair failed (CASANDRA-7521)
+ * Anti-compaction proceeds if any part of the repair failed (CASSANDRA-7521)
  * Add missing table name to DROP INDEX responses and notifications (CASSANDRA-7539)
  * Bump CQL version to 3.2.0 and update CQL documentation (CASSANDRA-7527)
  * Fix configuration error message when running nodetool ring (CASSANDRA-7508)
diff --git a/NEWS.txt b/NEWS.txt
index c5c798c..abc1bf5 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -13,61 +13,95 @@
 'sstableloader' tool. You can upgrade the file format of your snapshots
 using the provided 'sstableupgrade' tool.
 
-2.1.18
+2.2.10
 ======
 
 Upgrading
 ---------
-    - Nothing specific to this release, but please see 2.1 if you are upgrading
+    - Nothing specific to this release, but please see 2.2 if you are upgrading
       from a previous version.
 
-2.1.17
-======
-
-Upgrading
----------
-    - Nothing specific to this release, but please see 2.1 if you are upgrading
-      from a previous version.
-
-2.1.16
-======
-
-Upgrading
----------
-    - Duplicate partition keys in SELECT statement IN clauses will now be
-      filtered out, meaning that duplicate results will no longer be returned.
-      Futhermore, the partitions will be returned in the order of the sorted
-      partition keys instead of the order of the IN values; this matches the
-      behavior of Cassandra 2.2+.  This was necessary to avoid an infinitely
-      looping result set when combined with paging under some circumstances.
-      See CASSANDRA-12420 for details.
-    - The ReversedType behaviour has been corrected for clustering columns of
-      BYTES type containing empty value. Scrub should be run on the existing
-      SSTables containing a descending clustering column of BYTES type to correct
-      their ordering. See CASSANDRA-12127 for details.
-
-2.1.15
-======
-
-Upgrading
----------
-    - Nothing specific to this release, but please see 2.1 if you are upgrading
-      from a previous version.
-
-2.1.14
+2.2.9
 =====
 
 Upgrading
 ---------
-    - Nothing specific to this release, but please see 2.1 if you are upgrading
-      from a previous version.
+   - Compaction now correctly drops sstables out of CompactionTask when there
+     isn't enough disk space to perform the full compaction.  This should reduce
+     pending compaction tasks on systems with little remaining disk space.
 
-2.1.13
-======
+Deprecation
+-----------
+
+(See note about the new feature User-Defined-Functions in 2.2.0.)
+
+Since the security manager added in 3.0 only allows Java and JavaScript
+UDFs to be run, UDFs for other languages are deprecated and support for
+non-Java and non-JavaScript UDFs is deprecated in 2.2 and has been removed
+in version 3.0.11.
+
+2.2.8
+=====
+
+Upgrading
+---------
+    - The ReversedType behaviour has been corrected for clustering columns of
+      BYTES type containing empty value. Scrub should be run on the existing
+      SSTables containing a descending clustering column of BYTES type to correct
+      their ordering. See CASSANDRA-12127 for more details.
+
+2.2.7
+=====
 
 New features
 ------------
-    - New options for cqlsh COPY FROM and COPY TO, see CASSANDRA-9303 for details.
+    - JSON timestamps are now in UTC and contain the timezone information, see
+      CASSANDRA-11137 for more details.
+
+Upgrading
+---------
+    - Ec2MultiRegionSnitch will no longer automatically set broadcast_rpc_address
+      to the public instance IP if this property is defined on cassandra.yaml.
+
+
+2.2.6
+=====
+
+Upgrading
+---------
+    - Nothing specific to this release, but please see 2.2 if you are upgrading
+      from a previous version.
+
+2.2.5
+=====
+
+Upgrading
+---------
+    - Nothing specific to this release, but please see 2.2 if you are upgrading
+      from a previous version.
+
+2.2.4
+=====
+
+Deprecation
+-----------
+    - Pig support has been deprecated, and will be removed in 3.0.
+      Please see CASSANDRA-10542 for more details.
+    - Configuration parameter memory_allocator in cassandra.yaml has been deprecated
+      and will be removed in 3.0.0. As mentioned below for 2.2.0, jemalloc is
+      automatically preloaded on Unix platforms.
+
+Operations
+----------
+    - Switching data center or racks is no longer an allowed operation on a node
+      which has data. Instead, the node will need to be decommissioned and
+      rebootstrapped. If moving from the SimpleSnitch, make sure that the data
+      center and rack containing all current nodes is named "datacenter1" and
+      "rack1". To override this behaviour use -Dcassandra.ignore_rack=true and/or
+      -Dcassandra.ignore_dc=true.
+    - Reloading the configuration file of GossipingPropertyFileSnitch has been disabled.
+    - GC logging is now enabled by default (but you can disable it if you want by
+      commenting the relevant lines of the cassandra-env file).
 
 Upgrading
 ---------
@@ -76,9 +110,6 @@
       the same than the one for intra-DC one (200Mbps) instead of being unlimited.
       Having it unlimited was never intended and was a bug.
 
-2.1.12
-======
-
 New features
 ------------
     - Time windows in DTCS are now limited to 1 day by default to be able to
@@ -87,28 +118,241 @@
     - DTCS option max_sstable_age_days is now deprecated and defaults to 1000 days.
     - Native protocol server now allows both SSL and non-SSL connections on
       the same port.
-    - a new validate(key, cf) method is added to PerRowSecondaryIndex. A default
-      implementation is provided, so no changes are required to custom implementations.
 
-Operations
-------------
-    - Switching data center or racks is no longer an allowed operation on a node
-      which has data. Instead, the node will need to be decommissioned and
-      rebootstrapped. If moving from the SimpleSnitch, make sure that the data
-      center and rack containing all current nodes is named "datacenter1" and
-      "rack1". To override this behaviour use -Dcassandra.ignore_rack=true and/or
-      -Dcassandra.ignore_dc=true.
-    - Reloading the configuration file of GossipingPropertyFileSnitch has been disabled.
-
-2.1.11
+2.2.3
 =====
 
 Upgrading
 ---------
-    - Nothing specific to this release, but please see 2.1 if you are upgrading
+    - Nothing specific to this release, but please see 2.2 if you are upgrading
       from a previous version.
 
 
+2.2.2
+=====
+
+Upgrading
+---------
+    - Version 1 and 2 of the native protocol are now deprecated and support
+      will be removed in Cassandra 3.0. You are encouraged to upgrade to a
+      client driver using version 3 of the native protocol.
+
+Changed Defaults
+----------------
+   - commitlog_total_space_in_mb will use the smaller of 8192, and 1/4
+     of the total space of the commitlog volume. (Before: always used
+     8192)
+   - Incremental repair is on by default since 2.2.0, run full repairs by
+     providing the '-full' parameter to nodetool repair.
+   - Parallel repairs are the default since 2.2.0, run sequential repairs
+     by providing the '-seq' parameter to nodetool repair.
+   - The following INFO logs were reduced to DEBUG level and will now show
+     on debug.log instead of system.log:
+      - Memtable flushing actions
+      - Commit log replayed files
+      - Compacted sstables
+      - SStable opening (SSTableReader)
+
+New features
+------------
+   - Custom QueryHandlers can retrieve the column specifications for the bound
+     variables from QueryOptions by using the hasColumnSpecifications()
+     and getColumnSpecifications() methods.
+   - A new default assynchronous log appender debug.log was created in addition
+     to  the system.log appender in order to provide more detailed log debugging.
+     In order to disable debug logging, you must comment-out the ASYNCDEBUGLOG
+     appender on conf/logback.xml. See CASSANDRA-10241 for more information.
+
+
+2.2.1
+=====
+
+Upgrading
+---------
+    - Nothing specific to this release, but please see 2.2 if you are upgrading
+      from a previous version.
+
+New features
+------------
+   - COUNT(*) and COUNT(1) can be selected with other columns or functions
+
+
+2.2
+===
+
+Upgrading
+---------
+   - The authentication & authorization subsystems have been redesigned to
+     support role based access control (RBAC), resulting in a change to the
+     schema of the system_auth keyspace. See below for more detail.
+     For systems already using the internal auth implementations, the process
+     for converting existing data during a rolling upgrade is straightforward.
+     As each node is restarted, it will attempt to convert any data in the
+     legacy tables into the new schema. Until enough nodes to satisfy the
+     replication strategy for the system_auth keyspace are upgraded and so have
+     the new schema, this conversion will fail with the failure being reported
+     in the system log.
+     During the upgrade, Cassandra's internal auth classes will continue to use
+     the legacy tables, so clients experience no disruption. Issuing DCL
+     statements during an upgrade is not supported.
+     Once all nodes are upgraded, an operator with superuser privileges should
+     drop the legacy tables, system_auth.users, system_auth.credentials and 
+     system_auth.permissions. Doing so will prompt Cassandra to switch over to 
+     the new tables without requiring any further intervention.
+     While the legacy tables are present a restarted node will re-run the data
+     conversion and report the outcome so that operators can verify that it is
+     safe to drop them.
+
+New features
+------------
+   - The LIMIT clause applies now only to the number of rows returned to the user,
+     not to the number of row queried. By consequence, queries using aggregates will not
+     be impacted by the LIMIT clause anymore.
+   - Very large batches will now be rejected (defaults to 50kb). This
+     can be customized by modifying batch_size_fail_threshold_in_kb.
+   - Selecting columns,scalar functions, UDT fields, writetime or ttl together
+     with aggregated is now possible. The value returned for the columns,
+     scalar functions, UDT fields, writetime and ttl will be the ones for
+     the first row matching the query.
+   - Windows is now a supported platform. Powershell execution for startup scripts
+     is highly recommended and can be enabled via an administrator command-prompt
+     with: 'powershell set-executionpolicy unrestricted'
+   - It is now possible to do major compactions when using leveled compaction.
+     Doing that will take all sstables and compact them out in levels. The
+     levels will be non overlapping so doing this will still not be something
+     you want to do very often since it might cause more compactions for a while.
+     It is also possible to split output when doing a major compaction with
+     STCS - files will be split in sizes 50%, 25%, 12.5% etc of the total size.
+     This might be a bit better than old major compactions which created one big
+     file on disk.
+   - A new tool has been added bin/sstableverify that checks for errors/bitrot
+     in all sstables.  Unlike scrub, this is a non-invasive tool.
+   - Authentication & Authorization APIs have been updated to introduce
+     roles. Roles and Permissions granted to them are inherited, supporting
+     role based access control. The role concept supercedes that of users
+     and CQL constructs such as CREATE USER are deprecated but retained for
+     compatibility. The requirement to explicitly create Roles in Cassandra
+     even when auth is handled by an external system has been removed, so
+     authentication & authorization can be delegated to such systems in their
+     entirety.
+   - In addition to the above, Roles are also first class resources and can be the
+     subject of permissions. Users (roles) can now be granted permissions on other
+     roles, including CREATE, ALTER, DROP & AUTHORIZE, which removesthe need for
+     superuser privileges in order to perform user/role management operations.
+   - Creators of database resources (Keyspaces, Tables, Roles) are now automatically
+     granted all permissions on them (if the IAuthorizer implementation supports
+     this).
+   - SSTable file name is changed. Now you don't have Keyspace/CF name
+     in file name. Also, secondary index has its own directory under parent's
+     directory.
+   - Support for user-defined functions and user-defined aggregates have
+     been added to CQL.
+     ************************************************************************
+     IMPORTANT NOTE: user-defined functions can be used to execute
+     arbitrary and possibly evil code in Cassandra 2.2, and are
+     therefore disabled by default.  To enable UDFs edit
+     cassandra.yaml and set enable_user_defined_functions to true.
+
+     CASSANDRA-9402 will add a security manager for UDFs in Cassandra
+     3.0.  This will inherently be backwards-incompatible with any 2.2
+     UDF that perform insecure operations such as opening a socket or
+     writing to the filesystem.
+
+     Per the previous note about adding a security manager in 3.0, this security manager
+     means that non JavaScipt UDF's won't run, there for their use is deprecated.
+     ************************************************************************
+   - Row-cache is now fully off-heap.
+   - jemalloc is now automatically preloaded and used on Linux and OS-X if
+     installed.
+   - Please ensure on Unix platforms that there is no libjnadispath.so
+     installed which is accessible by Cassandra. Old versions of
+     libjna packages (< 4.0.0) will cause problems - e.g. Debian Wheezy
+     contains libjna versin 3.2.x.
+   - The node now keeps up when streaming is failed during bootstrapping. You can
+     use new `nodetool bootstrap resume` command to continue streaming after resolving
+     an issue.
+   - Protocol version 4 specifies that bind variables do not require having a
+     value when executing a statement. Bind variables without a value are
+     called 'unset'. The 'unset' bind variable is serialized as the int
+     value '-2' without following bytes.
+     In an EXECUTE or BATCH request an unset bind value does not modify the value and
+     does not create a tombstone, an unset bind ttl is treated as 'unlimited',
+     an unset bind timestamp is treated as 'now', an unset bind counter operation
+     does not change the counter value.
+     Unset tuple field, UDT field and map key are not allowed.
+     In a QUERY request an unset limit is treated as 'unlimited'.
+     Unset WHERE clauses with unset partition column, clustering column
+     or index column are not allowed.
+   - New `ByteType` (cql tinyint). 1-byte signed integer
+   - New `ShortType` (cql smallint). 2-byte signed integer
+   - New `SimpleDateType` (cql date). 4-byte unsigned integer
+   - New `TimeType` (cql time). 8-byte long
+   - The toDate(timeuuid), toTimestamp(timeuuid) and toUnixTimestamp(timeuuid) functions have been added to allow
+     to convert from timeuuid into date type, timestamp type and bigint raw value.
+     The functions unixTimestampOf(timeuuid) and dateOf(timeuuid) have been deprecated.
+   - The toDate(timestamp) and toUnixTimestamp(timestamp) functions have been added to allow
+     to convert from timestamp into date type and bigint raw value.
+   - The toTimestamp(date) and toUnixTimestamp(date) functions have been added to allow
+     to convert from date into timestamp type and bigint raw value.
+   - SizeTieredCompactionStrategy parameter cold_reads_to_omit has been removed.
+   - The default JVM flag -XX:+PerfDisableSharedMem will cause the following tools JVM
+     to stop working: jps, jstack, jinfo, jmc, jcmd as well as 3rd party tools like Jolokia.
+     If you wish to use these tools you can comment this flag out in cassandra-env.{sh,ps1}
+
+Upgrading
+---------
+   - Thrift rpc is no longer being started by default.
+     Set `start_rpc` parameter to `true` to enable it.
+   - Pig's CqlStorage has been removed, use CqlNativeStorage instead
+   - Pig's CassandraStorage has been deprecated. CassandraStorage
+     should only be used against tables created via thrift.
+     Use CqlNativeStorage for all other tables.
+   - IAuthenticator been updated to remove responsibility for user/role
+     maintenance and is now solely responsible for validating credentials,
+     This is primarily done via SASL, though an optional method exists for
+     systems which need support for the Thrift login() method.
+   - IRoleManager interface has been added which takes over the maintenance
+     functions from IAuthenticator. IAuthorizer is mainly unchanged. Auth data
+     in systems using the stock internal implementations PasswordAuthenticator
+     & CassandraAuthorizer will be automatically converted during upgrade,
+     with minimal operator intervention required. Custom implementations will
+     require modification, though these can be used in conjunction with the
+     stock CassandraRoleManager so providing an IRoleManager implementation
+     should not usually be necessary.
+   - Fat client support has been removed since we have push notifications to clients
+   - cassandra-cli has been removed. Please use cqlsh instead.
+   - YamlFileNetworkTopologySnitch has been removed; switch to
+     GossipingPropertyFileSnitch instead.
+   - CQL2 has been removed entirely in this release (previously deprecated
+     in 2.0.0). Please switch to CQL3 if you haven't already done so.
+   - The results of CQL3 queries containing an IN restriction will be ordered
+     in the normal order and not anymore in the order in which the column values were
+     specified in the IN restriction.
+   - Some secondary index queries with restrictions on non-indexed clustering
+     columns were not requiring ALLOW FILTERING as they should. This has been
+     fixed, and those queries now require ALLOW FILTERING (see CASSANDRA-8418
+     for details).
+   - The SSTableSimpleWriter and SSTableSimpleUnsortedWriter classes have been
+     deprecated and will be removed in the next major Cassandra release. You
+     should use the CQLSSTableWriter class instead.
+   - The sstable2json and json2sstable tools have been deprecated and will be
+     removed in the next major Cassandra release. See CASSANDRA-9618
+     (https://issues.apache.org/jira/browse/CASSANDRA-9618) for details.
+   - nodetool enablehandoff will no longer support a list of data centers starting
+     with the next major release. Two new commands will be added, enablehintsfordc and disablehintsfordc,
+     to exclude data centers from using hinted handoff when the global status is enabled.
+     In cassandra.yaml, hinted_handoff_enabled will no longer support a list of data centers starting
+     with the next major release. A new setting will be added, hinted_handoff_disabled_datacenters,
+     to exclude data centers when the global status is enabled, see CASSANDRA-9035 for details.
+
+
+2.1.13
+======
+
+New features
+------------
+    - New options for cqlsh COPY FROM and COPY TO, see CASSANDRA-9303 for details.
+
 2.1.10
 =====
 
@@ -142,10 +386,6 @@
 2.1.7
 =====
 
-Upgrading
----------
-    - Nothing specific to this release, but please see 2.1 if you are upgrading
-      from a previous version.
 
 
 2.1.6
@@ -164,15 +404,14 @@
 ---------
     - The option to omit cold sstables with size tiered compaction has been
       removed - it is almost always better to use date tiered compaction for
-      workloads that have cold data. 
-
+      workloads that have cold data.
 
 2.1.4
 =====
 
 Upgrading
 ---------
-The default JMX config now listens to localhost only. You must enable 
+The default JMX config now listens to localhost only. You must enable
 the other JMX flags in cassandra-env.sh manually.
 
 
@@ -242,12 +481,12 @@
      using LCS, or any data not repaired in your first incremental repair
      will be put back in L0.
    - Bootstrapping now ensures that range movements are consistent,
-     meaning the data for the new node is taken from the node that is no 
+     meaning the data for the new node is taken from the node that is no
      longer a responsible for that range of keys.
      If you want the old behavior (due to a lost node perhaps)
      you can set the following property (-Dcassandra.consistent.rangemovement=false)
-   - It is now possible to use quoted identifiers in triggers' names. 
-     WARNING: if you previously used triggers with capital letters in their 
+   - It is now possible to use quoted identifiers in triggers' names.
+     WARNING: if you previously used triggers with capital letters in their
      names, then you must quote them from now on.
    - Improved stress tool (http://goo.gl/OTNqiQ)
    - New incremental repair option (http://goo.gl/MjohJp, http://goo.gl/f8jSme)
@@ -310,7 +549,7 @@
     - If you are using Leveled Compaction, you can now disable doing size-tiered
       compaction in L0 by starting Cassandra with -Dcassandra.disable_stcs_in_l0
       (see CASSANDRA-6621 for details).
-    - Shuffle and taketoken have been removed.  For clusters that choose to 
+    - Shuffle and taketoken have been removed.  For clusters that choose to
       upgrade to vnodes, creating a new datacenter with vnodes and migrating is
       recommended. See http://goo.gl/Sna2S1 for further information.
 
@@ -487,7 +726,7 @@
     - VNodes are enabled by default in cassandra.yaml.  initial_token
       for non-vnode deployments has been removed from the example
       yaml, but is still respected if specified.
-    - Major compactions, cleanup, scrub, and upgradesstables will interrupt 
+    - Major compactions, cleanup, scrub, and upgradesstables will interrupt
       any in-progress compactions (but not repair validations) when invoked.
     - Disabling autocompactions by setting min/max compaction threshold to 0
       has been deprecated, instead, use the nodetool commands 'disableautocompaction'
@@ -504,7 +743,7 @@
       use DESCRIBE FULL SCHEMA if you need the schema of system_* keyspaces.
     - CQL2 has been deprecated, and will be removed entirely in 2.2. See
       CASSANDRA-5918 for details.
-    - Commit log archiver now assumes the client time stamp to be in microsecond 
+    - Commit log archiver now assumes the client time stamp to be in microsecond
       precision, during restore. Please refer to commitlog_archiving.properties.
 
 
diff --git a/NOTICE.txt b/NOTICE.txt
index cf7b8dc..a71d822 100644
--- a/NOTICE.txt
+++ b/NOTICE.txt
@@ -60,3 +60,17 @@
 
 HLL++ support provided by stream-lib
 (https://github.com/addthis/stream-lib)
+
+Eclipse JDT
+Java compilation software for user-defined-functions is provided by Eclipse,
+which is open source software.  The original software and
+related information is available at http://www.eclipse.org/
+(http://www.eclipse.org/jdt/)
+
+SIGAR
+http://sigar.hyperic.com/
+
+OHC
+(https://github.com/snazy/ohc)
+Java Off-Heap-Cache, licensed under APLv2
+Copyright 2014-2015 Robert Stupp, Germany.
diff --git a/bin/cassandra b/bin/cassandra
index 957cc7d..2dd0fe1 100755
--- a/bin/cassandra
+++ b/bin/cassandra
@@ -132,6 +132,55 @@
     ;;
 esac
 
+# Cassandra uses an installed jemalloc via LD_PRELOAD / DYLD_INSERT_LIBRARIES by default to improve off-heap
+# memory allocation performance. The following code searches for an installed libjemalloc.dylib/.so/.1.so using
+# Linux and OS-X specific approaches.
+# To specify your own libjemalloc in a different path, configure the fully qualified path in CASSANDRA_LIBJEMALLOC.
+# To disable jemalloc preload at all, set CASSANDRA_LIBJEMALLOC=-
+#
+#CASSANDRA_LIBJEMALLOC=
+#
+find_library()
+{
+    pattern=$1
+    path=$(echo ${2} | tr ":" " ")
+
+    find $path -regex "$pattern" -print 2>/dev/null | head -n 1
+}
+case "`uname -s`" in
+    Linux)
+        if [ -z $CASSANDRA_LIBJEMALLOC ] ; then
+            which ldconfig > /dev/null 2>&1
+            if [ $? = 0 ] ; then
+                # e.g. for CentOS
+                dirs="/lib64 /lib /usr/lib64 /usr/lib `ldconfig -v 2>/dev/null | grep -v '^\s' | sed 's/^\([^:]*\):.*$/\1/'`"
+            else
+                # e.g. for Debian, OpenSUSE
+                dirs="/lib64 /lib /usr/lib64 /usr/lib `cat /etc/ld.so.conf /etc/ld.so.conf.d/*.conf | grep '^/'`"
+            fi
+            dirs=`echo $dirs | tr " " ":"`
+            CASSANDRA_LIBJEMALLOC=$(find_library '.*/libjemalloc\.so\(\.1\)*' $dirs)
+        fi
+        if [ ! -z $CASSANDRA_LIBJEMALLOC ] ; then
+            export JVM_OPTS="$JVM_OPTS -Dcassandra.libjemalloc=$CASSANDRA_LIBJEMALLOC"
+            if [ "-" != "$CASSANDRA_LIBJEMALLOC" ] ; then
+                export LD_PRELOAD=$CASSANDRA_LIBJEMALLOC
+            fi
+        fi
+    ;;
+    Darwin)
+        if [ -z $CASSANDRA_LIBJEMALLOC ] ; then
+            CASSANDRA_LIBJEMALLOC=$(find_library '.*/libjemalloc\.dylib' $DYLD_LIBRARY_PATH:${DYLD_FALLBACK_LIBRARY_PATH-$HOME/lib:/usr/local/lib:/lib:/usr/lib})
+        fi
+        if [ ! -z $CASSANDRA_LIBJEMALLOC ] ; then
+            export JVM_OPTS="$JVM_OPTS -Dcassandra.libjemalloc=$CASSANDRA_LIBJEMALLOC"
+            if [ "-" != "$CASSANDRA_LIBJEMALLOC" ] ; then
+                export DYLD_INSERT_LIBRARIES=$CASSANDRA_LIBJEMALLOC
+            fi
+        fi
+    ;;
+esac
+
 launch_service()
 {
     pidpath="$1"
diff --git a/bin/cassandra-cli b/bin/cassandra-cli
deleted file mode 100755
index 88c9d2f..0000000
--- a/bin/cassandra-cli
+++ /dev/null
@@ -1,61 +0,0 @@
-#!/bin/sh
-
-# 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.
-
-if [ "x$CASSANDRA_INCLUDE" = "x" ]; then
-    for include in /usr/share/cassandra/cassandra.in.sh \
-                   /usr/local/share/cassandra/cassandra.in.sh \
-                   /opt/cassandra/cassandra.in.sh \
-                   "$HOME/.cassandra.in.sh" \
-                   "`dirname "$0"`/cassandra.in.sh"; do
-        if [ -r "$include" ]; then
-            . "$include"
-            break
-        fi
-    done
-elif [ -r "$CASSANDRA_INCLUDE" ]; then
-    . "$CASSANDRA_INCLUDE"
-fi
-
-# Use JAVA_HOME if set, otherwise look for java in PATH
-if [ -x "$JAVA_HOME/bin/java" ]; then
-    JAVA="$JAVA_HOME/bin/java"
-else
-    JAVA="`which java`"
-fi
-
-if [ "x$JAVA" = "x" ]; then
-    echo "Java executable not found (hint: set JAVA_HOME)" >&2
-    exit 1
-fi
-
-if [ -z "$CLASSPATH" ]; then
-    echo "You must set the CLASSPATH var" >&2
-    exit 1
-fi
-
-if [ "x$MAX_HEAP_SIZE" = "x" ]; then
-    MAX_HEAP_SIZE="256M"
-fi
-
-"$JAVA" $JAVA_AGENT -ea -cp "$CLASSPATH" $JVM_OPTS -Xmx$MAX_HEAP_SIZE \
-        -Dcassandra.storagedir="$cassandra_storagedir" \
-        -Dlogback.configurationFile=logback-tools.xml \
-        $JVM_ARGS \
-        org.apache.cassandra.cli.CliMain "$@"
-
-# vi:ai sw=4 ts=4 tw=0 et
diff --git a/bin/cassandra-cli.bat b/bin/cassandra-cli.bat
deleted file mode 100644
index 6211371..0000000
--- a/bin/cassandra-cli.bat
+++ /dev/null
@@ -1,36 +0,0 @@
-@REM

-@REM  Licensed to the Apache Software Foundation (ASF) under one or more

-@REM  contributor license agreements.  See the NOTICE file distributed with

-@REM  this work for additional information regarding copyright ownership.

-@REM  The ASF licenses this file to You under the Apache License, Version 2.0

-@REM  (the "License"); you may not use this file except in compliance with

-@REM  the License.  You may obtain a copy of the License at

-@REM

-@REM      http://www.apache.org/licenses/LICENSE-2.0

-@REM

-@REM  Unless required by applicable law or agreed to in writing, software

-@REM  distributed under the License is distributed on an "AS IS" BASIS,

-@REM  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.

-@REM  See the License for the specific language governing permissions and

-@REM  limitations under the License.

-

-@echo off

-if "%OS%" == "Windows_NT" setlocal

-

-pushd "%~dp0"

-call cassandra.in.bat

-

-if NOT DEFINED CASSANDRA_HOME set CASSANDRA_HOME=%~dp0..

-if NOT DEFINED JAVA_HOME goto :err

-

-echo Starting Cassandra Client

-"%JAVA_HOME%\bin\java" -cp %CASSANDRA_CLASSPATH% org.apache.cassandra.cli.CliMain %*

-goto finally

-

-:err

-echo The JAVA_HOME environment variable must be set to run this program!

-pause

-

-:finally

-

-ENDLOCAL

diff --git a/bin/cassandra.bat b/bin/cassandra.bat
index fefd3fc..0d207cd 100644
--- a/bin/cassandra.bat
+++ b/bin/cassandra.bat
@@ -18,7 +18,6 @@
 if "%OS%" == "Windows_NT" setlocal

 

 set ARG=%1

-if /i "%ARG%" == "LEGACY" goto runLegacy

 set INSTALL="INSTALL"

 set UNINSTALL="UNINSTALL"

 

@@ -26,6 +25,7 @@
 if NOT DEFINED CASSANDRA_HOME set CASSANDRA_HOME=%CD%

 popd

 

+if /i "%ARG%" == "LEGACY" goto runLegacy

 REM -----------------------------------------------------------------------------

 REM See if we have access to run unsigned powershell scripts

 for /F "delims=" %%i in ('powershell Get-ExecutionPolicy') do set PERMISSION=%%i

@@ -66,6 +66,7 @@
  -XX:CMSInitiatingOccupancyFraction=75^

  -XX:+UseCMSInitiatingOccupancyOnly^

  -Dlogback.configurationFile=logback.xml^

+ -Djava.library.path="%CASSANDRA_HOME%\lib\sigar-bin"^

  -Dcassandra.jmx.local.port=7199

 REM **** JMX REMOTE ACCESS SETTINGS SEE: https://wiki.apache.org/cassandra/JmxSecurity ***

 REM -Dcom.sun.management.jmxremote.port=7199^

@@ -87,6 +88,29 @@
 

 REM -----------------------------------------------------------------------------

 :okClasspath

+

+REM JSR223 - collect all JSR223 engines' jars

+for /D %%P in ("%CASSANDRA_HOME%\lib\jsr223\*.*") do (

+	for %%i in ("%%P\*.jar") do call :append "%%i"

+)

+

+REM JSR223/JRuby - set ruby lib directory

+if EXIST "%CASSANDRA_HOME%\lib\jsr223\jruby\ruby" (

+    set JAVA_OPTS=%JAVA_OPTS% "-Djruby.lib=%CASSANDRA_HOME%\lib\jsr223\jruby"

+)

+REM JSR223/JRuby - set ruby JNI libraries root directory

+if EXIST "%CASSANDRA_HOME%\lib\jsr223\jruby\jni" (

+    set JAVA_OPTS=%JAVA_OPTS% "-Djffi.boot.library.path=%CASSANDRA_HOME%\lib\jsr223\jruby\jni"

+)

+REM JSR223/Jython - set python.home system property

+if EXIST "%CASSANDRA_HOME%\lib\jsr223\jython\jython.jar" (

+    set JAVA_OPTS=%JAVA_OPTS% "-Dpython.home=%CASSANDRA_HOME%\lib\jsr223\jython"

+)

+REM JSR223/Scala - necessary system property

+if EXIST "%CASSANDRA_HOME%\lib\jsr223\scala\scala-compiler.jar" (

+    set JAVA_OPTS=%JAVA_OPTS% "-Dscala.usejavacp=true"

+)

+

 REM Include the build\classes\main directory so it works in development

 set CASSANDRA_CLASSPATH=%CLASSPATH%;"%CASSANDRA_HOME%\build\classes\main";"%CASSANDRA_HOME%\build\classes\thrift"

 set CASSANDRA_PARAMS=-Dcassandra -Dcassandra-foreground=yes

diff --git a/bin/cassandra.in.bat b/bin/cassandra.in.bat
index e3304e7..5682f9d 100644
--- a/bin/cassandra.in.bat
+++ b/bin/cassandra.in.bat
@@ -31,7 +31,7 @@
 REM ***** CLASSPATH library setting *****

 

 REM Ensure that any user defined CLASSPATH variables are not used on startup

-set CLASSPATH="%CASSANDRA_HOME%\conf"

+set CLASSPATH=%CASSANDRA_CONF%

 

 REM For each jar in the CASSANDRA_HOME lib directory call append to build the CLASSPATH variable.

 for %%i in ("%CASSANDRA_HOME%\lib\*.jar") do call :append "%%i"

@@ -48,3 +48,27 @@
 

 REM Add the default storage location.  Can be overridden in conf\cassandra.yaml

 set CASSANDRA_PARAMS=%CASSANDRA_PARAMS% "-Dcassandra.storagedir=%CASSANDRA_HOME%\data"

+

+REM JSR223 - collect all JSR223 engines' jars

+for /r %%P in ("%CASSANDRA_HOME%\lib\jsr223\*.jar") do (

+    set CLASSPATH=%CLASSPATH%;%%~fP

+)

+REM JSR223/JRuby - set ruby lib directory

+if EXIST "%CASSANDRA_HOME%\lib\jsr223\jruby\ruby" (

+    set JAVA_OPTS=%JAVA_OPTS% "-Djruby.lib=%CASSANDRA_HOME%\lib\jsr223\jruby"

+)

+REM JSR223/JRuby - set ruby JNI libraries root directory

+if EXIST "%CASSANDRA_HOME%\lib\jsr223\jruby\jni" (

+    set JAVA_OPTS=%JAVA_OPTS% "-Djffi.boot.library.path=%CASSANDRA_HOME%\lib\jsr223\jruby\jni"

+)

+REM JSR223/Jython - set python.home system property

+if EXIST "%$CASSANDRA_HOME%\lib\jsr223\jython\jython.jar" (

+    set JAVA_OPTS=%JAVA_OPTS% "-Dpython.home=%CASSANDRA_HOME%\lib\jsr223\jython"

+)

+REM JSR223/Scala - necessary system property

+if EXIST "$CASSANDRA_HOME\lib\jsr223\scala\scala-compiler.jar" (

+    set JAVA_OPTS=%JAVA_OPTS% "-Dscala.usejavacp=true"

+)

+

+REM Add the sigar-bin path to the java.library.path CASSANDRA-7838

+set JAVA_OPTS=%JAVA_OPTS% -Djava.library.path=%CASSANDRA_HOME%\lib\sigar-bin"

diff --git a/bin/cassandra.in.sh b/bin/cassandra.in.sh
index b6a53f3..92f79c7 100644
--- a/bin/cassandra.in.sh
+++ b/bin/cassandra.in.sh
@@ -28,7 +28,7 @@
 # it's just used here in constructing the classpath.
 cassandra_bin="$CASSANDRA_HOME/build/classes/main"
 cassandra_bin="$cassandra_bin:$CASSANDRA_HOME/build/classes/thrift"
-#cassandra_bin="$cassandra_home/build/cassandra.jar"
+#cassandra_bin="$CASSANDRA_HOME/build/cassandra.jar"
 
 # the default location for commitlogs, sstables, and saved caches
 # if not set in cassandra.yaml
@@ -44,9 +44,33 @@
     CLASSPATH="$CLASSPATH:$jar"
 done
 
+# JSR223 - collect all JSR223 engines' jars
+for jsr223jar in "$CASSANDRA_HOME"/lib/jsr223/*/*.jar; do
+    CLASSPATH="$CLASSPATH:$jsr223jar"
+done
+# JSR223/JRuby - set ruby lib directory
+if [ -d "$CASSANDRA_HOME"/lib/jsr223/jruby/ruby ] ; then
+    export JVM_OPTS="$JVM_OPTS -Djruby.lib=$CASSANDRA_HOME/lib/jsr223/jruby"
+fi
+# JSR223/JRuby - set ruby JNI libraries root directory
+if [ -d "$CASSANDRA_HOME"/lib/jsr223/jruby/jni ] ; then
+    export JVM_OPTS="$JVM_OPTS -Djffi.boot.library.path=$CASSANDRA_HOME/lib/jsr223/jruby/jni"
+fi
+# JSR223/Jython - set python.home system property
+if [ -f "$CASSANDRA_HOME"/lib/jsr223/jython/jython.jar ] ; then
+    export JVM_OPTS="$JVM_OPTS -Dpython.home=$CASSANDRA_HOME/lib/jsr223/jython"
+fi
+# JSR223/Scala - necessary system property
+if [ -f "$CASSANDRA_HOME"/lib/jsr223/scala/scala-compiler.jar ] ; then
+    export JVM_OPTS="$JVM_OPTS -Dscala.usejavacp=true"
+fi
+
 # set JVM javaagent opts to avoid warnings/errors
 if [ "$JVM_VENDOR" != "OpenJDK" -o "$JVM_VERSION" \> "1.6.0" ] \
       || [ "$JVM_VERSION" = "1.6.0" -a "$JVM_PATCH_VERSION" -ge 23 ]
 then
     JAVA_AGENT="$JAVA_AGENT -javaagent:$CASSANDRA_HOME/lib/jamm-0.3.0.jar"
 fi
+
+# Added sigar-bin to the java.library.path CASSANDRA-7838
+JAVA_OPTS="$JAVA_OPTS:-Djava.library.path=$CASSANDRA_HOME/lib/sigar-bin"
diff --git a/bin/cassandra.ps1 b/bin/cassandra.ps1
index d9af6e6..5d10994 100644
--- a/bin/cassandra.ps1
+++ b/bin/cassandra.ps1
@@ -17,7 +17,7 @@
 Function PrintUsage

 {

     echo @"

-usage: cassandra.ps1 [-f] [-h] [-p pidfile] [-H dumpfile] [-D arg] [-E errorfile] [-install | -uninstall] [-help]

+usage: cassandra.ps1 [-f] [-h] [-q] [-a] [-p pidfile] [-H dumpfile] [-D arg] [-E errorfile] [-install | -uninstall] [-help]

     -f              Run cassandra in foreground

     -install        install cassandra as a service

     -uninstall      remove cassandra service

@@ -27,6 +27,8 @@
     -E              change JVM ErrorFile

     -v              Print cassandra version and exit

     -s              Show detailed jvm environment information during launch

+    -a              Aggressive startup. Skip VerifyPorts check. For use in dev environments.

+    -q              Quiet output. Does not print stdout/stderr to console (when run without -f)

     -help           print this message

 

     NOTE: installing cassandra as a service requires Commons Daemon Service Runner

@@ -36,8 +38,6 @@
 }

 

 #-----------------------------------------------------------------------------

-# Note: throughout these scripts we're replacing \ with /.  This allows clean

-# operation on both command-prompt and cygwin-based environments.

 Function Main

 {

     ValidateArguments

@@ -194,7 +194,6 @@
 #-----------------------------------------------------------------------------

 Function RunCassandra([string]$foreground)

 {

-    echo "Starting cassandra server"

     $cmd = @"

 $env:JAVA_BIN

 "@

@@ -255,7 +254,14 @@
     }

     else

     {

-        $proc = Start-Process -FilePath "$cmd" -ArgumentList $arg1,$arg2,$arg3,$arg4 -PassThru -WindowStyle Hidden

+        if ($q)

+        {

+            $proc = Start-Process -FilePath "$cmd" -ArgumentList $arg1,$arg2,$arg3,$arg4 -PassThru -WindowStyle Hidden

+        }

+        else

+        {

+            $proc = Start-Process -FilePath "$cmd" -ArgumentList $arg1,$arg2,$arg3,$arg4 -PassThru -NoNewWindow

+        }

 

         $exitCode = $?

 

@@ -283,6 +289,10 @@
 #-----------------------------------------------------------------------------

 Function VerifyPortsAreAvailable

 {

+    if ($a)

+    {

+        return

+    }

     # Need to confirm 5 different ports are available or die if any are currently bound

     # From cassandra.yaml:

     #   storage_port

@@ -290,89 +300,28 @@
     #   native_transport_port

     #   rpc_port, which we'll match to rpc_address

     # and from env: JMX_PORT which we cache in our environment during SetCassandraEnvironment for this check

-    $toMatch = @("storage_port:","ssl_storage_port:","native_transport_port:","rpc_port")

+    $yamlRegex = "storage_port:|ssl_storage_port:|native_transport_port:|rpc_port"

     $yaml = Get-Content "$env:CASSANDRA_CONF\cassandra.yaml"

-

-    $listenAddress = ""

-    $rpcAddress = ""

-    foreach ($line in $yaml)

-    {

-        if ($line -match "^listen_address:")

-        {

-            $args = $line -Split ": "

-            $listenAddress = $args[1] -replace " ", ""

-        }

-        if ($line -match "^rpc_address:")

-        {

-            $args = $line -Split ": "

-            $rpcAddress = $args[1] -replace " ", ""

-        }

-    }

-    if ([string]::IsNullOrEmpty($listenAddress))

-    {

-        Write-Error "Failed to parse listen_address from cassandra.yaml to check open ports.  Aborting startup."

-        Exit

-    }

-    if ([string]::IsNullOrEmpty($rpcAddress))

-    {

-        Write-Error "Failed to parse rpc_address from cassandra.yaml to check open ports.  Aborting startup."

-        Exit

-    }

+    $portRegex = ":$env:JMX_PORT |"

 

     foreach ($line in $yaml)

     {

-        foreach ($match in $toMatch)

+        if ($line -match $yamlRegex)

         {

-            if ($line -match "^$match")

-            {

-                if ($line.contains("rpc"))

-                {

-                    CheckPort $rpcAddress $line

-                }

-                else

-                {

-                    CheckPort $listenAddress $line

-                }

-            }

+            $sa = $line.Split(":")

+            $portRegex = $portRegex + ":" + ($sa[1] -replace " ","") + " |"

         }

     }

-    if ([string]::IsNullOrEmpty($env:JMX_PORT))

-    {

-        Write-Error "No JMX_PORT is set in environment.  Aborting startup."

-        Exit

-    }

-    CheckPort $listenAddress "jmx_port: $env:JMX_PORT"

-}

+    $portRegex = $portRegex.Substring(0, $portRegex.Length - 2)

 

-#-----------------------------------------------------------------------------

-Function CheckPort([string]$listenAddress, [string]$configLine)

-{

-    $split = $configLine -Split ":"

-    if ($split.Length -ne 2)

-    {

-        echo "Invalid cassandra.yaml config line parsed while checking for available ports:"

-        echo "$configLine"

-        echo "Aborting startup"

-        Exit

-    }

-    else

-    {

-        $port = $split[1] -replace " ", ""

+    $netstat = netstat -an

 

-        # start an async connect to the ip/port combo, give it 25ms, and error out if it succeeded

-        $tcpobject = new-Object system.Net.Sockets.TcpClient

-        $connect = $tcpobject.BeginConnect($listenAddress, $port, $null, $null)

-        $wait = $connect.AsyncWaitHandle.WaitOne(25, $false)

-

-        if (!$wait)

+    foreach ($line in $netstat)

+    {

+        if ($line -match "TCP" -and $line -match $portRegex)

         {

-            # still trying to connect, if it's not serviced in 25ms we'll assume it's not open

-            $tcpobject.Close()

-        }

-        else

-        {

-            $tcpobject.EndConnect($connect) | out-Null

-            echo "Cassandra port already in use ($configLine).  Aborting"

+            Write-Error "Found a port already in use. Aborting startup"

+            Write-Error $line

             Exit

         }

     }

@@ -392,6 +341,7 @@
     }

 }

 

+#-----------------------------------------------------------------------------

 Function CheckEmptyParam($param)

 {

     if ([String]::IsNullOrEmpty($param))

@@ -401,6 +351,8 @@
     }

 }

 

+#-----------------------------------------------------------------------------

+# Populate arguments

 for ($i = 0; $i -lt $args.count; $i++)

 {

     # Skip JVM args

@@ -413,12 +365,16 @@
         "-install"          { $install = $True }

         "-uninstall"        { $uninstall = $True }

         "-help"             { PrintUsage }

+        "-?"                { PrintUsage }

+        "--help"            { PrintUsage }

         "-v"                { $v = $True }

         "-f"                { $f = $True }

         "-s"                { $s = $True }

         "-p"                { $p = $args[++$i]; CheckEmptyParam($p) }

         "-H"                { $H = $args[++$i]; CheckEmptyParam($H) }

         "-E"                { $E = $args[++$i]; CheckEmptyParam($E) }

+        "-a"                { $a = $True }

+        "-q"                { $q = $True }

         default

         {

             "Invalid argument: " + $args[$i];

diff --git a/bin/cqlsh b/bin/cqlsh
index 6317ec9..82a4a53 100755
--- a/bin/cqlsh
+++ b/bin/cqlsh
@@ -1,6 +1,4 @@
 #!/bin/sh
-# -*- mode: Python -*-
-
 # 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
@@ -17,2207 +15,12 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-""":"
 # bash code here; finds a suitable python interpreter and execs this file.
 # prefer unqualified "python" if suitable:
-python -c 'import sys; sys.exit(not (0x020500b0 < sys.hexversion < 0x03000000))' 2>/dev/null \
-    && exec python "$0" "$@"
-for pyver in 2.6 2.7 2.5; do
-    which python$pyver > /dev/null 2>&1 && exec python$pyver "$0" "$@"
+python -c 'import sys; sys.exit(not (0x020700b0 < sys.hexversion < 0x03000000))' 2>/dev/null \
+    && exec python "`python -c "import os;print(os.path.dirname(os.path.realpath('$0')))"`/cqlsh.py" "$@"
+for pyver in 2.7; do
+    which python$pyver > /dev/null 2>&1 && exec python$pyver "`python$pyver -c "import os;print(os.path.dirname(os.path.realpath('$0')))"`/cqlsh.py" "$@"
 done
 echo "No appropriate python interpreter found." >&2
 exit 1
-":"""
-
-from __future__ import with_statement
-
-import cmd
-import codecs
-import ConfigParser
-import csv
-import getpass
-import locale
-import optparse
-import os
-import platform
-import sys
-import traceback
-import warnings
-
-from StringIO import StringIO
-from contextlib import contextmanager
-from glob import glob
-from uuid import UUID
-
-description = "CQL Shell for Apache Cassandra"
-version = "5.0.1"
-
-readline = None
-try:
-    # check if tty first, cause readline doesn't check, and only cares
-    # about $TERM. we don't want the funky escape code stuff to be
-    # output if not a tty.
-    if sys.stdin.isatty():
-        import readline
-except ImportError:
-    pass
-
-CQL_LIB_PREFIX = 'cassandra-driver-internal-only-'
-
-CASSANDRA_PATH = os.path.join(os.path.dirname(os.path.realpath(__file__)), '..')
-
-# use bundled libs for python-cql and thrift, if available. if there
-# is a ../lib dir, use bundled libs there preferentially.
-ZIPLIB_DIRS = [os.path.join(CASSANDRA_PATH, 'lib')]
-myplatform = platform.system()
-if myplatform == 'Linux':
-    ZIPLIB_DIRS.append('/usr/share/cassandra/lib')
-
-if os.environ.get('CQLSH_NO_BUNDLED', ''):
-    ZIPLIB_DIRS = ()
-
-
-def find_zip(libprefix):
-    for ziplibdir in ZIPLIB_DIRS:
-        zips = glob(os.path.join(ziplibdir, libprefix + '*.zip'))
-        if zips:
-            return max(zips)   # probably the highest version, if multiple
-
-cql_zip = find_zip(CQL_LIB_PREFIX)
-if cql_zip:
-    ver = os.path.splitext(os.path.basename(cql_zip))[0][len(CQL_LIB_PREFIX):]
-    sys.path.insert(0, os.path.join(cql_zip, 'cassandra-driver-' + ver))
-
-third_parties = ('futures-', 'six-')
-
-for lib in third_parties:
-    lib_zip = find_zip(lib)
-    if lib_zip:
-        sys.path.insert(0, lib_zip)
-
-warnings.filterwarnings("ignore", r".*blist.*")
-try:
-    import cassandra
-except ImportError, e:
-    sys.exit("\nPython Cassandra driver not installed, or not on PYTHONPATH.\n"
-             'You might try "pip install cassandra-driver".\n\n'
-             'Python: %s\n'
-             'Module load path: %r\n\n'
-             'Error: %s\n' % (sys.executable, sys.path, e))
-
-from cassandra.auth import PlainTextAuthProvider
-from cassandra.cluster import Cluster, PagedResult
-from cassandra.metadata import protect_name, protect_names
-from cassandra.policies import WhiteListRoundRobinPolicy
-from cassandra.protocol import ResultMessage
-from cassandra.query import SimpleStatement, ordered_dict_factory, tuple_factory
-
-# cqlsh should run correctly when run out of a Cassandra source tree,
-# out of an unpacked Cassandra tarball, and after a proper package install.
-cqlshlibdir = os.path.join(CASSANDRA_PATH, 'pylib')
-if os.path.isdir(cqlshlibdir):
-    sys.path.insert(0, cqlshlibdir)
-
-from cqlshlib import cql3handling, cqlhandling, pylexotron, sslhandling
-from cqlshlib.copyutil import ExportTask, ImportTask
-from cqlshlib.displaying import (ANSI_RESET, BLUE, COLUMN_NAME_COLORS, CYAN,
-                                 RED, FormattedValue, colorme)
-from cqlshlib.formatting import (format_by_type, format_value_utype,
-                                 formatter_for)
-from cqlshlib.tracing import print_trace, print_trace_session
-from cqlshlib.util import get_file_encoding_bomsize, trim_if_present
-
-DEFAULT_HOST = '127.0.0.1'
-DEFAULT_PORT = 9042
-DEFAULT_CQLVER = '3.2.1'
-DEFAULT_PROTOCOL_VERSION = 3
-DEFAULT_CONNECT_TIMEOUT_SECONDS = 5
-DEFAULT_REQUEST_TIMEOUT_SECONDS = 10
-
-DEFAULT_TIME_FORMAT = '%Y-%m-%d %H:%M:%S%z'
-DEFAULT_FLOAT_PRECISION = 5
-DEFAULT_MAX_TRACE_WAIT = 10
-
-if readline is not None and readline.__doc__ is not None and 'libedit' in readline.__doc__:
-    DEFAULT_COMPLETEKEY = '\t'
-else:
-    DEFAULT_COMPLETEKEY = 'tab'
-
-cqldocs = None
-cqlruleset = None
-
-epilog = """Connects to %(DEFAULT_HOST)s:%(DEFAULT_PORT)d by default. These
-defaults can be changed by setting $CQLSH_HOST and/or $CQLSH_PORT. When a
-host (and optional port number) are given on the command line, they take
-precedence over any defaults.""" % globals()
-
-parser = optparse.OptionParser(description=description, epilog=epilog,
-                               usage="Usage: %prog [options] [host [port]]",
-                               version='cqlsh ' + version)
-parser.add_option("-C", "--color", action='store_true', dest='color',
-                  help='Always use color output')
-parser.add_option("--no-color", action='store_false', dest='color',
-                  help='Never use color output')
-parser.add_option('--ssl', action='store_true', help='Use SSL', default=False)
-parser.add_option("-u", "--username", help="Authenticate as user.")
-parser.add_option("-p", "--password", help="Authenticate using password.")
-parser.add_option('-k', '--keyspace', help='Authenticate to the given keyspace.')
-parser.add_option("-f", "--file", help="Execute commands from FILE, then exit")
-parser.add_option('--debug', action='store_true',
-                  help='Show additional debugging information')
-parser.add_option("--encoding", help="Specify a non-default encoding for output.  If you are " +
-                  "experiencing problems with unicode characters, using utf8 may fix the problem." +
-                  " (Default from system preferences: %s)" % (locale.getpreferredencoding(),))
-parser.add_option("--cqlshrc", help="Specify an alternative cqlshrc file location.")
-parser.add_option('--cqlversion', default=DEFAULT_CQLVER,
-                  help='Specify a particular CQL version (default: %default).'
-                       ' Examples: "3.0.3", "3.1.0"')
-parser.add_option("-e", "--execute", help='Execute the statement and quit.')
-parser.add_option("--connect-timeout", default=DEFAULT_CONNECT_TIMEOUT_SECONDS, dest='connect_timeout',
-                  help='Specify the connection timeout in seconds (default: %default seconds).')
-parser.add_option("--request-timeout", default=DEFAULT_REQUEST_TIMEOUT_SECONDS, dest='request_timeout',
-                  help='Specify the default request timeout in seconds (default: %default seconds).')
-
-optvalues = optparse.Values()
-(options, arguments) = parser.parse_args(sys.argv[1:], values=optvalues)
-
-# BEGIN history/config definition
-HISTORY_DIR = os.path.expanduser(os.path.join('~', '.cassandra'))
-
-if hasattr(options, 'cqlshrc'):
-    CONFIG_FILE = options.cqlshrc
-    if not os.path.exists(CONFIG_FILE):
-        print '\nWarning: Specified cqlshrc location `%s` does not exist.  Using `%s` instead.\n' % (CONFIG_FILE, HISTORY_DIR)
-        CONFIG_FILE = os.path.join(HISTORY_DIR, 'cqlshrc')
-else:
-    CONFIG_FILE = os.path.join(HISTORY_DIR, 'cqlshrc')
-
-HISTORY = os.path.join(HISTORY_DIR, 'cqlsh_history')
-if not os.path.exists(HISTORY_DIR):
-    try:
-        os.mkdir(HISTORY_DIR)
-    except OSError:
-        print '\nWarning: Cannot create directory at `%s`. Command history will not be saved.\n' % HISTORY_DIR
-
-OLD_CONFIG_FILE = os.path.expanduser(os.path.join('~', '.cqlshrc'))
-if os.path.exists(OLD_CONFIG_FILE):
-    if os.path.exists(CONFIG_FILE):
-        print '\nWarning: cqlshrc config files were found at both the old location (%s) and \
-               the new location (%s), the old config file will not be migrated to the new \
-               location, and the new location will be used for now.  You should manually \
-               consolidate the config files at the new location and remove the old file.' \
-               % (OLD_CONFIG_FILE, CONFIG_FILE)
-    else:
-        os.rename(OLD_CONFIG_FILE, CONFIG_FILE)
-OLD_HISTORY = os.path.expanduser(os.path.join('~', '.cqlsh_history'))
-if os.path.exists(OLD_HISTORY):
-    os.rename(OLD_HISTORY, HISTORY)
-# END history/config definition
-
-CQL_ERRORS = (
-    cassandra.AlreadyExists, cassandra.AuthenticationFailed, cassandra.InvalidRequest,
-    cassandra.Timeout, cassandra.Unauthorized, cassandra.OperationTimedOut,
-    cassandra.cluster.NoHostAvailable,
-    cassandra.connection.ConnectionBusy, cassandra.connection.ProtocolError, cassandra.connection.ConnectionException,
-    cassandra.protocol.ErrorMessage, cassandra.protocol.InternalError, cassandra.query.TraceUnavailable
-)
-
-debug_completion = bool(os.environ.get('CQLSH_DEBUG_COMPLETION', '') == 'YES')
-
-SYSTEM_KEYSPACES = ('system', 'system_traces', 'system_auth')
-
-# we want the cql parser to understand our cqlsh-specific commands too
-my_commands_ending_with_newline = (
-    'help',
-    '?',
-    'consistency',
-    'serial',
-    'describe',
-    'desc',
-    'show',
-    'source',
-    'capture',
-    'login',
-    'debug',
-    'tracing',
-    'expand',
-    'paging',
-    'exit',
-    'quit'
-)
-
-
-cqlsh_syntax_completers = []
-
-
-def cqlsh_syntax_completer(rulename, termname):
-    def registrator(f):
-        cqlsh_syntax_completers.append((rulename, termname, f))
-        return f
-    return registrator
-
-
-cqlsh_extra_syntax_rules = r'''
-<cqlshCommand> ::= <CQL_Statement>
-                 | <specialCommand> ( ";" | "\n" )
-                 ;
-
-<specialCommand> ::= <describeCommand>
-                   | <consistencyCommand>
-                   | <serialConsistencyCommand>
-                   | <showCommand>
-                   | <sourceCommand>
-                   | <captureCommand>
-                   | <copyCommand>
-                   | <loginCommand>
-                   | <debugCommand>
-                   | <helpCommand>
-                   | <tracingCommand>
-                   | <expandCommand>
-                   | <exitCommand>
-                   | <pagingCommand>
-                   ;
-
-<describeCommand> ::= ( "DESCRIBE" | "DESC" )
-                                  ( "KEYSPACES"
-                                  | "KEYSPACE" ksname=<keyspaceName>?
-                                  | ( "COLUMNFAMILY" | "TABLE" ) cf=<columnFamilyName>
-                                  | "INDEX" idx=<indexName>
-                                  | ( "COLUMNFAMILIES" | "TABLES" )
-                                  | "FULL"? "SCHEMA"
-                                  | "CLUSTER"
-                                  | "TYPES"
-                                  | "TYPE" ut=<userTypeName>
-                                  | (ksname=<keyspaceName> | cf=<columnFamilyName> | idx=<indexName>))
-                    ;
-
-<consistencyCommand> ::= "CONSISTENCY" ( level=<consistencyLevel> )?
-                       ;
-
-<consistencyLevel> ::= "ANY"
-                     | "ONE"
-                     | "TWO"
-                     | "THREE"
-                     | "QUORUM"
-                     | "ALL"
-                     | "LOCAL_QUORUM"
-                     | "EACH_QUORUM"
-                     | "SERIAL"
-                     | "LOCAL_SERIAL"
-                     | "LOCAL_ONE"
-                     ;
-
-<serialConsistencyCommand> ::= "SERIAL" "CONSISTENCY" ( level=<serialConsistencyLevel> )?
-                             ;
-
-<serialConsistencyLevel> ::= "SERIAL"
-                           | "LOCAL_SERIAL"
-                           ;
-
-<showCommand> ::= "SHOW" what=( "VERSION" | "HOST" | "SESSION" sessionid=<uuid> )
-                ;
-
-<sourceCommand> ::= "SOURCE" fname=<stringLiteral>
-                  ;
-
-<captureCommand> ::= "CAPTURE" ( fname=( <stringLiteral> | "OFF" ) )?
-                   ;
-
-<copyCommand> ::= "COPY" cf=<columnFamilyName>
-                         ( "(" [colnames]=<colname> ( "," [colnames]=<colname> )* ")" )?
-                         ( dir="FROM" ( fname=<stringLiteral> | "STDIN" )
-                         | dir="TO"   ( fname=<stringLiteral> | "STDOUT" ) )
-                         ( "WITH" <copyOption> ( "AND" <copyOption> )* )?
-                ;
-
-<copyOption> ::= [optnames]=(<identifier>|<reserved_identifier>) "=" [optvals]=<copyOptionVal>
-               ;
-
-<copyOptionVal> ::= <identifier>
-                  | <reserved_identifier>
-                  | <term>
-                  ;
-
-# avoiding just "DEBUG" so that this rule doesn't get treated as a terminal
-<debugCommand> ::= "DEBUG" "THINGS"?
-                 ;
-
-<helpCommand> ::= ( "HELP" | "?" ) [topic]=( /[a-z_]*/ )*
-                ;
-
-<tracingCommand> ::= "TRACING" ( switch=( "ON" | "OFF" ) )?
-                   ;
-
-<expandCommand> ::= "EXPAND" ( switch=( "ON" | "OFF" ) )?
-                   ;
-
-<pagingCommand> ::= "PAGING" ( switch=( "ON" | "OFF" ) )?
-                  ;
-
-<loginCommand> ::= "LOGIN" username=<username> (password=<stringLiteral>)?
-                 ;
-
-<exitCommand> ::= "exit" | "quit"
-                ;
-
-<qmark> ::= "?" ;
-'''
-
-
-@cqlsh_syntax_completer('helpCommand', 'topic')
-def complete_help(ctxt, cqlsh):
-    return sorted([t.upper() for t in cqldocs.get_help_topics() + cqlsh.get_help_topics()])
-
-
-def complete_source_quoted_filename(ctxt, cqlsh):
-    partial_path = ctxt.get_binding('partial', '')
-    head, tail = os.path.split(partial_path)
-    exhead = os.path.expanduser(head)
-    try:
-        contents = os.listdir(exhead or '.')
-    except OSError:
-        return ()
-    matches = filter(lambda f: f.startswith(tail), contents)
-    annotated = []
-    for f in matches:
-        match = os.path.join(head, f)
-        if os.path.isdir(os.path.join(exhead, f)):
-            match += '/'
-        annotated.append(match)
-    return annotated
-
-
-cqlsh_syntax_completer('sourceCommand', 'fname')(complete_source_quoted_filename)
-cqlsh_syntax_completer('captureCommand', 'fname')(complete_source_quoted_filename)
-
-
-@cqlsh_syntax_completer('copyCommand', 'fname')
-def copy_fname_completer(ctxt, cqlsh):
-    lasttype = ctxt.get_binding('*LASTTYPE*')
-    if lasttype == 'unclosedString':
-        return complete_source_quoted_filename(ctxt, cqlsh)
-    partial_path = ctxt.get_binding('partial')
-    if partial_path == '':
-        return ["'"]
-    return ()
-
-
-@cqlsh_syntax_completer('copyCommand', 'colnames')
-def complete_copy_column_names(ctxt, cqlsh):
-    existcols = map(cqlsh.cql_unprotect_name, ctxt.get_binding('colnames', ()))
-    ks = cqlsh.cql_unprotect_name(ctxt.get_binding('ksname', None))
-    cf = cqlsh.cql_unprotect_name(ctxt.get_binding('cfname'))
-    colnames = cqlsh.get_column_names(ks, cf)
-    if len(existcols) == 0:
-        return [colnames[0]]
-    return set(colnames[1:]) - set(existcols)
-
-
-COPY_COMMON_OPTIONS = ['DELIMITER', 'QUOTE', 'ESCAPE', 'HEADER', 'NULL', 'DATETIMEFORMAT',
-                       'MAXATTEMPTS', 'REPORTFREQUENCY', 'DECIMALSEP', 'THOUSANDSSEP', 'BOOLSTYLE',
-                       'NUMPROCESSES', 'CONFIGFILE', 'RATEFILE']
-COPY_FROM_OPTIONS = ['CHUNKSIZE', 'INGESTRATE', 'MAXBATCHSIZE', 'MINBATCHSIZE', 'MAXROWS',
-                     'SKIPROWS', 'SKIPCOLS', 'MAXPARSEERRORS', 'MAXINSERTERRORS', 'ERRFILE', 'PREPAREDSTATEMENTS']
-COPY_TO_OPTIONS = ['ENCODING', 'PAGESIZE', 'PAGETIMEOUT', 'BEGINTOKEN', 'ENDTOKEN', 'MAXOUTPUTSIZE', 'MAXREQUESTS']
-
-
-@cqlsh_syntax_completer('copyOption', 'optnames')
-def complete_copy_options(ctxt, cqlsh):
-    optnames = map(str.upper, ctxt.get_binding('optnames', ()))
-    direction = ctxt.get_binding('dir').upper()
-    if direction == 'FROM':
-        opts = set(COPY_COMMON_OPTIONS + COPY_FROM_OPTIONS) - set(optnames)
-    elif direction == 'TO':
-        opts = set(COPY_COMMON_OPTIONS + COPY_TO_OPTIONS) - set(optnames)
-    return opts
-
-
-@cqlsh_syntax_completer('copyOption', 'optvals')
-def complete_copy_opt_values(ctxt, cqlsh):
-    optnames = ctxt.get_binding('optnames', ())
-    lastopt = optnames[-1].lower()
-    if lastopt == 'header':
-        return ['true', 'false']
-    return [cqlhandling.Hint('<single_character_string>')]
-
-
-class NoKeyspaceError(Exception):
-    pass
-
-
-class KeyspaceNotFound(Exception):
-    pass
-
-
-class ColumnFamilyNotFound(Exception):
-    pass
-
-
-class IndexNotFound(Exception):
-    pass
-
-
-class ObjectNotFound(Exception):
-    pass
-
-
-class VersionNotSupported(Exception):
-    pass
-
-
-class UserTypeNotFound(Exception):
-    pass
-
-
-class DecodeError(Exception):
-    verb = 'decode'
-
-    def __init__(self, thebytes, err, colname=None):
-        self.thebytes = thebytes
-        self.err = err
-        self.colname = colname
-
-    def __str__(self):
-        return str(self.thebytes)
-
-    def message(self):
-        what = 'value %r' % (self.thebytes,)
-        if self.colname is not None:
-            what = 'value %r (for column %r)' % (self.thebytes, self.colname)
-        return 'Failed to %s %s : %s' \
-               % (self.verb, what, self.err)
-
-    def __repr__(self):
-        return '<%s %s>' % (self.__class__.__name__, self.message())
-
-
-class FormatError(DecodeError):
-    verb = 'format'
-
-
-def full_cql_version(ver):
-    while ver.count('.') < 2:
-        ver += '.0'
-    ver_parts = ver.split('-', 1) + ['']
-    vertuple = tuple(map(int, ver_parts[0].split('.')) + [ver_parts[1]])
-    return ver, vertuple
-
-
-def format_value(val, output_encoding, addcolor=False, time_format=None,
-                 float_precision=None, colormap=None, nullval=None):
-    if isinstance(val, DecodeError):
-        if addcolor:
-            return colorme(repr(val.thebytes), colormap, 'error')
-        else:
-            return FormattedValue(repr(val.thebytes))
-    return format_by_type(type(val), val, output_encoding, colormap=colormap,
-                          addcolor=addcolor, nullval=nullval, time_format=time_format,
-                          float_precision=float_precision)
-
-
-def show_warning_without_quoting_line(message, category, filename, lineno, file=None, line=None):
-    if file is None:
-        file = sys.stderr
-    try:
-        file.write(warnings.formatwarning(message, category, filename, lineno, line=''))
-    except IOError:
-        pass
-warnings.showwarning = show_warning_without_quoting_line
-warnings.filterwarnings('always', category=cql3handling.UnexpectedTableStructure)
-
-
-def insert_driver_hooks():
-    extend_cql_deserialization()
-    auto_format_udts()
-
-
-def extend_cql_deserialization():
-    """
-    The python driver returns BLOBs as string, but we expect them as bytearrays; therefore we change
-    the implementation of cassandra.cqltypes.BytesType.deserialize.
-
-    The deserializers package exists only when the driver has been compiled with cython extensions and
-    cassandra.deserializers.DesBytesType replaces cassandra.cqltypes.BytesType.deserialize.
-
-    DesBytesTypeByteArray is a fast deserializer that converts blobs into bytearrays but it was
-    only introduced recently (3.1.0). If it is available we use it, otherwise we remove
-    cassandra.deserializers.DesBytesType so that we fall back onto cassandra.cqltypes.BytesType.deserialize
-    just like in the case where no cython extensions are present.
-    """
-    if hasattr(cassandra, 'deserializers'):
-        if hasattr(cassandra.deserializers, 'DesBytesTypeByteArray'):
-            cassandra.deserializers.DesBytesType = cassandra.deserializers.DesBytesTypeByteArray
-        else:
-            del cassandra.deserializers.DesBytesType
-
-    cassandra.cqltypes.BytesType.deserialize = staticmethod(lambda byts, protocol_version: bytearray(byts))
-    cassandra.cqltypes.CassandraType.support_empty_values = True
-
-
-def auto_format_udts():
-    # when we see a new user defined type, set up the shell formatting for it
-    udt_apply_params = cassandra.cqltypes.UserType.apply_parameters
-
-    def new_apply_params(cls, *args, **kwargs):
-        udt_class = udt_apply_params(*args, **kwargs)
-        formatter_for(udt_class.typename)(format_value_utype)
-        return udt_class
-
-    cassandra.cqltypes.UserType.udt_apply_parameters = classmethod(new_apply_params)
-
-    make_udt_class = cassandra.cqltypes.UserType.make_udt_class
-
-    def new_make_udt_class(cls, *args, **kwargs):
-        udt_class = make_udt_class(*args, **kwargs)
-        formatter_for(udt_class.tuple_type.__name__)(format_value_utype)
-        return udt_class
-
-    cassandra.cqltypes.UserType.make_udt_class = classmethod(new_make_udt_class)
-
-
-class FrozenType(cassandra.cqltypes._ParameterizedType):
-    """
-    Needed until the bundled python driver adds FrozenType.
-    """
-    typename = "frozen"
-    num_subtypes = 1
-
-    @classmethod
-    def deserialize_safe(cls, byts, protocol_version):
-        subtype, = cls.subtypes
-        return subtype.from_binary(byts)
-
-    @classmethod
-    def serialize_safe(cls, val, protocol_version):
-        subtype, = cls.subtypes
-        return subtype.to_binary(val, protocol_version)
-
-
-class Shell(cmd.Cmd):
-    custom_prompt = os.getenv('CQLSH_PROMPT', '')
-    if custom_prompt is not '':
-        custom_prompt += "\n"
-    default_prompt = custom_prompt + "cqlsh> "
-    continue_prompt = "   ... "
-    keyspace_prompt = custom_prompt + "cqlsh:%s> "
-    keyspace_continue_prompt = "%s    ... "
-    show_line_nums = False
-    debug = False
-    stop = False
-    last_hist = None
-    shunted_query_out = None
-    use_paging = True
-
-    default_page_size = 100
-
-    def __init__(self, hostname, port, color=False,
-                 username=None, password=None, encoding=None, stdin=None, tty=True,
-                 completekey=DEFAULT_COMPLETEKEY, use_conn=None,
-                 cqlver=DEFAULT_CQLVER, keyspace=None,
-                 tracing_enabled=False, expand_enabled=False,
-                 display_time_format=DEFAULT_TIME_FORMAT,
-                 display_float_precision=DEFAULT_FLOAT_PRECISION,
-                 max_trace_wait=DEFAULT_MAX_TRACE_WAIT,
-                 ssl=False,
-                 single_statement=None,
-                 request_timeout=DEFAULT_REQUEST_TIMEOUT_SECONDS,
-                 connect_timeout=DEFAULT_CONNECT_TIMEOUT_SECONDS):
-        cmd.Cmd.__init__(self, completekey=completekey)
-        self.hostname = hostname
-        self.port = port
-        self.auth_provider = None
-        if username:
-            if not password:
-                password = getpass.getpass()
-            self.auth_provider = PlainTextAuthProvider(username=username, password=password)
-        self.username = username
-        self.keyspace = keyspace
-        self.ssl = ssl
-        self.tracing_enabled = tracing_enabled
-        self.expand_enabled = expand_enabled
-        if use_conn:
-            self.conn = use_conn
-        else:
-            self.conn = Cluster(contact_points=(self.hostname,), port=self.port, cql_version=cqlver,
-                                protocol_version=DEFAULT_PROTOCOL_VERSION,
-                                auth_provider=self.auth_provider,
-                                ssl_options=sslhandling.ssl_settings(hostname, CONFIG_FILE) if ssl else None,
-                                load_balancing_policy=WhiteListRoundRobinPolicy([self.hostname]),
-                                connect_timeout=connect_timeout)
-        self.owns_connection = not use_conn
-        self.set_expanded_cql_version(cqlver)
-
-        if keyspace:
-            self.session = self.conn.connect(keyspace)
-        else:
-            self.session = self.conn.connect()
-
-        self.color = color
-        self.display_time_format = display_time_format
-        self.display_float_precision = display_float_precision
-
-        # If there is no schema metadata present (due to a schema mismatch), force schema refresh
-        if not self.conn.metadata.keyspaces:
-            self.refresh_schema_metadata_best_effort()
-
-        self.session.default_timeout = request_timeout
-        self.session.row_factory = ordered_dict_factory
-        self.get_connection_versions()
-
-        self.current_keyspace = keyspace
-
-        self.max_trace_wait = max_trace_wait
-        self.session.max_trace_wait = max_trace_wait
-        if encoding is None:
-            encoding = locale.getpreferredencoding()
-            if encoding is None:
-                encoding = 'utf-8'
-        self.encoding = encoding
-        self.output_codec = codecs.lookup(encoding)
-
-        self.statement = StringIO()
-        self.lineno = 1
-        self.in_comment = False
-
-        self.prompt = ''
-        if stdin is None:
-            stdin = sys.stdin
-        self.tty = tty
-        if tty:
-            self.reset_prompt()
-            self.report_connection()
-            print 'Use HELP for help.'
-        else:
-            self.show_line_nums = True
-        self.stdin = stdin
-        self.query_out = sys.stdout
-        self.consistency_level = cassandra.ConsistencyLevel.ONE
-        self.serial_consistency_level = cassandra.ConsistencyLevel.SERIAL
-
-        self.empty_lines = 0
-        self.statement_error = False
-        self.single_statement = single_statement
-
-    def refresh_schema_metadata_best_effort(self):
-        try:
-            self.conn.refresh_schema_metadata(5)  # will throw exception if there is a schema mismatch
-        except Exception:
-            self.printerr("Warning: schema version mismatch detected, which might be caused by DOWN nodes; if "
-                          "this is not the case, check the schema versions of your nodes in system.local and "
-                          "system.peers.")
-            self.conn.refresh_schema_metadata(0)
-
-    def set_expanded_cql_version(self, ver):
-        ver, vertuple = full_cql_version(ver)
-        self.cql_version = ver
-        self.cql_ver_tuple = vertuple
-
-    def cqlver_atleast(self, major, minor=0, patch=0):
-        return self.cql_ver_tuple[:3] >= (major, minor, patch)
-
-    def myformat_value(self, val, **kwargs):
-        if isinstance(val, DecodeError):
-            self.decoding_errors.append(val)
-        try:
-            return format_value(val, self.output_codec.name,
-                                addcolor=self.color, time_format=self.display_time_format,
-                                float_precision=self.display_float_precision, **kwargs)
-        except Exception, e:
-            err = FormatError(val, e)
-            self.decoding_errors.append(err)
-            return format_value(err, self.output_codec.name, addcolor=self.color)
-
-    def myformat_colname(self, name, table_meta=None):
-        column_colors = COLUMN_NAME_COLORS.copy()
-        # check column role and color appropriately
-        if table_meta:
-            if name in [col.name for col in table_meta.partition_key]:
-                column_colors.default_factory = lambda: RED
-            elif name in [col.name for col in table_meta.clustering_key]:
-                column_colors.default_factory = lambda: CYAN
-        return self.myformat_value(name, colormap=column_colors)
-
-    def report_connection(self):
-        self.show_host()
-        self.show_version()
-
-    def show_host(self):
-        print "Connected to %s at %s:%d." % \
-            (self.applycolor(self.get_cluster_name(), BLUE),
-              self.hostname,
-              self.port)
-
-    def show_version(self):
-        vers = self.connection_versions.copy()
-        vers['shver'] = version
-        # system.Versions['cql'] apparently does not reflect changes with
-        # set_cql_version.
-        vers['cql'] = self.cql_version
-        print "[cqlsh %(shver)s | Cassandra %(build)s | CQL spec %(cql)s | Native protocol v%(protocol)s]" % vers
-
-    def show_session(self, sessionid):
-        print_trace_session(self, self.session, sessionid)
-
-    def get_connection_versions(self):
-        result, = self.session.execute("select * from system.local where key = 'local'")
-        vers = {
-            'build': result['release_version'],
-            'protocol': result['native_protocol_version'],
-            'cql': result['cql_version'],
-        }
-        self.connection_versions = vers
-
-    def get_keyspace_names(self):
-        return map(str, self.conn.metadata.keyspaces.keys())
-
-    def get_columnfamily_names(self, ksname=None):
-        if ksname is None:
-            ksname = self.current_keyspace
-
-        return map(str, self.get_keyspace_meta(ksname).tables.keys())
-
-    def get_index_names(self, ksname=None):
-        if ksname is None:
-            ksname = self.current_keyspace
-
-        return map(str, self.get_keyspace_meta(ksname).indexes.keys())
-
-    def get_column_names(self, ksname, cfname):
-        if ksname is None:
-            ksname = self.current_keyspace
-        layout = self.get_table_meta(ksname, cfname)
-        return [str(col) for col in layout.columns]
-
-    def get_usertype_names(self, ksname=None):
-        if ksname is None:
-            ksname = self.current_keyspace
-
-        return self.get_keyspace_meta(ksname).user_types.keys()
-
-    def get_usertype_layout(self, ksname, typename):
-        if ksname is None:
-            ksname = self.current_keyspace
-
-        ks_meta = self.get_keyspace_meta(ksname)
-
-        try:
-            user_type = ks_meta.user_types[typename]
-        except KeyError:
-            raise UserTypeNotFound("User type %r not found" % typename)
-
-        return [(field_name, field_type.cql_parameterized_type())
-                for field_name, field_type in zip(user_type.field_names, user_type.field_types)]
-
-    def get_cluster_name(self):
-        return self.conn.metadata.cluster_name
-
-    def get_partitioner(self):
-        return self.conn.metadata.partitioner
-
-    def get_keyspace_meta(self, ksname):
-        if ksname not in self.conn.metadata.keyspaces:
-            raise KeyspaceNotFound('Keyspace %r not found.' % ksname)
-        return self.conn.metadata.keyspaces[ksname]
-
-    def get_keyspaces(self):
-        return self.conn.metadata.keyspaces.values()
-
-    def get_ring(self, ks):
-        self.conn.metadata.token_map.rebuild_keyspace(ks, build_if_absent=True)
-        return self.conn.metadata.token_map.tokens_to_hosts_by_ks[ks]
-
-    def get_table_meta(self, ksname, tablename):
-        if ksname is None:
-            ksname = self.current_keyspace
-        ksmeta = self.get_keyspace_meta(ksname)
-
-        if tablename not in ksmeta.tables:
-            raise ColumnFamilyNotFound("Column family %r not found" % tablename)
-
-        return ksmeta.tables[tablename]
-
-    def get_index_meta(self, ksname, idxname):
-        if ksname is None:
-            ksname = self.current_keyspace
-        ksmeta = self.get_keyspace_meta(ksname)
-
-        if idxname not in ksmeta.indexes:
-            raise IndexNotFound("Index %r not found" % idxname)
-
-        return ksmeta.indexes[idxname]
-
-    def get_object_meta(self, ks, name):
-        if name is None:
-            if ks and ks in self.conn.metadata.keyspaces:
-                return self.conn.metadata.keyspaces[ks]
-            elif self.current_keyspace is None:
-                raise ObjectNotFound("%r not found in keyspaces" % (ks))
-            else:
-                name = ks
-                ks = self.current_keyspace
-
-        if ks is None:
-            ks = self.current_keyspace
-
-        ksmeta = self.get_keyspace_meta(ks)
-
-        if name in ksmeta.tables:
-            return ksmeta.tables[name]
-        elif name in ksmeta.indexes:
-            return ksmeta.indexes[name]
-
-        raise ObjectNotFound("%r not found in keyspace %r" % (name, ks))
-
-    def get_usertypes_meta(self):
-        data = self.session.execute("select * from system.schema_usertypes")
-        if not data:
-            return cql3handling.UserTypesMeta({})
-
-        return cql3handling.UserTypesMeta.from_layout(data)
-
-    def get_trigger_names(self, ksname=None):
-        if ksname is None:
-            ksname = self.current_keyspace
-
-        return [trigger.name
-                for table in self.get_keyspace_meta(ksname).tables.values()
-                for trigger in table.triggers.values()]
-
-    def reset_statement(self):
-        self.reset_prompt()
-        self.statement.truncate(0)
-        self.empty_lines = 0
-
-    def reset_prompt(self):
-        if self.current_keyspace is None:
-            self.set_prompt(self.default_prompt, True)
-        else:
-            self.set_prompt(self.keyspace_prompt % self.current_keyspace, True)
-
-    def set_continue_prompt(self):
-        if self.empty_lines >= 3:
-            self.set_prompt("Statements are terminated with a ';'.  You can press CTRL-C to cancel an incomplete statement.")
-            self.empty_lines = 0
-            return
-        if self.current_keyspace is None:
-            self.set_prompt(self.continue_prompt)
-        else:
-            spaces = ' ' * len(str(self.current_keyspace))
-            self.set_prompt(self.keyspace_continue_prompt % spaces)
-        self.empty_lines = self.empty_lines + 1 if not self.lastcmd else 0
-
-    @contextmanager
-    def prepare_loop(self):
-        readline = None
-        if self.tty and self.completekey:
-            try:
-                import readline
-            except ImportError:
-                if platform.system() == 'Windows':
-                    print "WARNING: pyreadline dependency missing.  Install to enable tab completion."
-                pass
-            else:
-                old_completer = readline.get_completer()
-                readline.set_completer(self.complete)
-                if readline.__doc__ is not None and 'libedit' in readline.__doc__:
-                    readline.parse_and_bind("bind -e")
-                    readline.parse_and_bind("bind '" + self.completekey + "' rl_complete")
-                    readline.parse_and_bind("bind ^R em-inc-search-prev")
-                else:
-                    readline.parse_and_bind(self.completekey + ": complete")
-        try:
-            yield
-        finally:
-            if readline is not None:
-                readline.set_completer(old_completer)
-
-    def get_input_line(self, prompt=''):
-        if self.tty:
-            self.lastcmd = raw_input(prompt).decode(self.encoding)
-            line = self.lastcmd + '\n'
-        else:
-            self.lastcmd = self.stdin.readline()
-            line = self.lastcmd
-            if not len(line):
-                raise EOFError
-        self.lineno += 1
-        return line
-
-    def use_stdin_reader(self, until='', prompt=''):
-        until += '\n'
-        while True:
-            try:
-                newline = self.get_input_line(prompt=prompt)
-            except EOFError:
-                return
-            if newline == until:
-                return
-            yield newline
-
-    def cmdloop(self):
-        """
-        Adapted from cmd.Cmd's version, because there is literally no way with
-        cmd.Cmd.cmdloop() to tell the difference between "EOF" showing up in
-        input and an actual EOF.
-        """
-        with self.prepare_loop():
-            while not self.stop:
-                try:
-                    if self.single_statement:
-                        line = self.single_statement
-                        self.stop = True
-                    else:
-                        line = self.get_input_line(self.prompt)
-                    self.statement.write(line)
-                    if self.onecmd(self.statement.getvalue()):
-                        self.reset_statement()
-                except EOFError:
-                    self.handle_eof()
-                except CQL_ERRORS, cqlerr:
-                    self.printerr(str(cqlerr))
-                except KeyboardInterrupt:
-                    self.reset_statement()
-                    print
-
-    def onecmd(self, statementtext):
-        """
-        Returns true if the statement is complete and was handled (meaning it
-        can be reset).
-        """
-
-        try:
-            statements, in_batch = cqlruleset.cql_split_statements(statementtext)
-        except pylexotron.LexingError, e:
-            if self.show_line_nums:
-                self.printerr('Invalid syntax at char %d' % (e.charnum,))
-            else:
-                self.printerr('Invalid syntax at line %d, char %d'
-                              % (e.linenum, e.charnum))
-            statementline = statementtext.split('\n')[e.linenum - 1]
-            self.printerr('  %s' % statementline)
-            self.printerr(' %s^' % (' ' * e.charnum))
-            return True
-
-        while statements and not statements[-1]:
-            statements = statements[:-1]
-        if not statements:
-            return True
-        if in_batch or statements[-1][-1][0] != 'endtoken':
-            self.set_continue_prompt()
-            return
-        for st in statements:
-            try:
-                self.handle_statement(st, statementtext)
-            except Exception, e:
-                if self.debug:
-                    traceback.print_exc()
-                else:
-                    self.printerr(e)
-        return True
-
-    def handle_eof(self):
-        if self.tty:
-            print
-        statement = self.statement.getvalue()
-        if statement.strip():
-            if not self.onecmd(statement):
-                self.printerr('Incomplete statement at end of file')
-        self.do_exit()
-
-    def handle_statement(self, tokens, srcstr):
-        # Concat multi-line statements and insert into history
-        if readline is not None:
-            nl_count = srcstr.count("\n")
-
-            new_hist = srcstr.replace("\n", " ").rstrip()
-
-            if nl_count > 1 and self.last_hist != new_hist:
-                readline.add_history(new_hist)
-
-            self.last_hist = new_hist
-        cmdword = tokens[0][1]
-        if cmdword == '?':
-            cmdword = 'help'
-        custom_handler = getattr(self, 'do_' + cmdword.lower(), None)
-        if custom_handler:
-            parsed = cqlruleset.cql_whole_parse_tokens(tokens, srcstr=srcstr,
-                                                       startsymbol='cqlshCommand')
-            if parsed and not parsed.remainder:
-                # successful complete parse
-                return custom_handler(parsed)
-            else:
-                return self.handle_parse_error(cmdword, tokens, parsed, srcstr)
-        return self.perform_statement(cqlruleset.cql_extract_orig(tokens, srcstr))
-
-    def handle_parse_error(self, cmdword, tokens, parsed, srcstr):
-        if cmdword.lower() in ('select', 'insert', 'update', 'delete', 'truncate',
-                               'create', 'drop', 'alter', 'grant', 'revoke',
-                               'batch', 'list'):
-            # hey, maybe they know about some new syntax we don't. type
-            # assumptions won't work, but maybe the query will.
-            return self.perform_statement(cqlruleset.cql_extract_orig(tokens, srcstr))
-        if parsed:
-            self.printerr('Improper %s command (problem at %r).' % (cmdword, parsed.remainder[0]))
-        else:
-            self.printerr('Improper %s command.' % cmdword)
-
-    def do_use(self, parsed):
-        ksname = parsed.get_binding('ksname')
-        if self.perform_simple_statement(SimpleStatement(parsed.extract_orig())):
-            if ksname[0] == '"' and ksname[-1] == '"':
-                self.current_keyspace = self.cql_unprotect_name(ksname)
-            else:
-                self.current_keyspace = ksname.lower()
-
-    def do_select(self, parsed):
-        tracing_was_enabled = self.tracing_enabled
-        ksname = parsed.get_binding('ksname')
-        stop_tracing = ksname == 'system_traces' or (ksname is None and self.current_keyspace == 'system_traces')
-        self.tracing_enabled = self.tracing_enabled and not stop_tracing
-        statement = parsed.extract_orig()
-        self.perform_statement(statement)
-        self.tracing_enabled = tracing_was_enabled
-
-    def perform_statement(self, statement):
-        stmt = SimpleStatement(statement, consistency_level=self.consistency_level, serial_consistency_level=self.serial_consistency_level, fetch_size=self.default_page_size if self.use_paging else None)
-        result = self.perform_simple_statement(stmt)
-        if self.tracing_enabled:
-            if stmt.trace:
-                print_trace(self, stmt.trace)
-            else:
-                msg = "Statement trace did not complete within %d seconds" % (self.session.max_trace_wait)
-                self.writeresult(msg, color=RED)
-
-        return result
-
-    def parse_for_table_meta(self, query_string):
-        try:
-            parsed = cqlruleset.cql_parse(query_string)[1]
-        except IndexError:
-            return None
-        ks = self.cql_unprotect_name(parsed.get_binding('ksname', None))
-        cf = self.cql_unprotect_name(parsed.get_binding('cfname'))
-        return self.get_table_meta(ks, cf)
-
-    def perform_simple_statement(self, statement):
-        if not statement:
-            return False
-        rows = None
-        while True:
-            try:
-                rows = self.session.execute(statement, trace=self.tracing_enabled)
-                break
-            except cassandra.OperationTimedOut, err:
-                self.refresh_schema_metadata_best_effort()
-                self.printerr(str(err.__class__.__name__) + ": " + str(err))
-                return False
-            except CQL_ERRORS, err:
-                self.printerr(str(err.__class__.__name__) + ": " + str(err))
-                return False
-            except Exception, err:
-                import traceback
-                self.printerr(traceback.format_exc())
-                return False
-
-        if statement.query_string[:6].lower() == 'select':
-            self.print_result(rows, self.parse_for_table_meta(statement.query_string))
-        elif statement.query_string.lower().startswith("list users"):
-            self.print_result(rows, self.get_table_meta('system_auth', 'users'))
-        elif statement.query_string.lower().startswith("list"):
-            self.print_result(rows, self.get_table_meta('system_auth', 'permissions'))
-        elif rows:
-            # CAS INSERT/UPDATE
-            self.writeresult("")
-            self.print_static_result(rows, self.parse_for_table_meta(statement.query_string))
-        self.flush_output()
-        return True
-
-    def print_result(self, rows, table_meta):
-        self.decoding_errors = []
-
-        self.writeresult("")
-        if isinstance(rows, PagedResult) and self.tty:
-            num_rows = 0
-            while True:
-                page = list(rows.current_response)
-                if not page:
-                    break
-                num_rows += len(page)
-                self.print_static_result(page, table_meta)
-                if not rows.response_future.has_more_pages:
-                    break
-                raw_input("---MORE---")
-
-                rows.response_future.start_fetching_next_page()
-                result = rows.response_future.result()
-                if rows.response_future.has_more_pages:
-                    rows.current_response = result.current_response
-                else:
-                    rows.current_response = iter(result)
-        else:
-            rows = list(rows or [])
-            num_rows = len(rows)
-            self.print_static_result(rows, table_meta)
-        self.writeresult("(%d rows)" % num_rows)
-
-        if self.decoding_errors:
-            for err in self.decoding_errors[:2]:
-                self.writeresult(err.message(), color=RED)
-            if len(self.decoding_errors) > 2:
-                self.writeresult('%d more decoding errors suppressed.'
-                                 % (len(self.decoding_errors) - 2), color=RED)
-
-    def print_static_result(self, rows, table_meta):
-        if not rows:
-            if not table_meta:
-                return
-            # print header only
-            colnames = table_meta.columns.keys()  # full header
-            formatted_names = [self.myformat_colname(name, table_meta) for name in colnames]
-            self.print_formatted_result(formatted_names, None)
-            return
-
-        colnames = rows[0].keys()
-        formatted_names = [self.myformat_colname(name, table_meta) for name in colnames]
-        formatted_values = [map(self.myformat_value, row.values()) for row in rows]
-
-        if self.expand_enabled:
-            self.print_formatted_result_vertically(formatted_names, formatted_values)
-        else:
-            self.print_formatted_result(formatted_names, formatted_values)
-
-    def print_formatted_result(self, formatted_names, formatted_values):
-        # determine column widths
-        widths = [n.displaywidth for n in formatted_names]
-        if formatted_values is not None:
-            for fmtrow in formatted_values:
-                for num, col in enumerate(fmtrow):
-                    widths[num] = max(widths[num], col.displaywidth)
-
-        # print header
-        header = ' | '.join(hdr.ljust(w, color=self.color) for (hdr, w) in zip(formatted_names, widths))
-        self.writeresult(' ' + header.rstrip())
-        self.writeresult('-%s-' % '-+-'.join('-' * w for w in widths))
-
-        # stop if there are no rows
-        if formatted_values is None:
-            self.writeresult("")
-            return
-
-        # print row data
-        for row in formatted_values:
-            line = ' | '.join(col.rjust(w, color=self.color) for (col, w) in zip(row, widths))
-            self.writeresult(' ' + line)
-
-        self.writeresult("")
-
-    def print_formatted_result_vertically(self, formatted_names, formatted_values):
-        max_col_width = max([n.displaywidth for n in formatted_names])
-        max_val_width = max([n.displaywidth for row in formatted_values for n in row])
-
-        # for each row returned, list all the column-value pairs
-        for row_id, row in enumerate(formatted_values):
-            self.writeresult("@ Row %d" % (row_id + 1))
-            self.writeresult('-%s-' % '-+-'.join(['-' * max_col_width, '-' * max_val_width]))
-            for field_id, field in enumerate(row):
-                column = formatted_names[field_id].ljust(max_col_width, color=self.color)
-                value = field.ljust(field.displaywidth, color=self.color)
-                self.writeresult(' ' + " | ".join([column, value]))
-            self.writeresult('')
-
-    def emptyline(self):
-        pass
-
-    def parseline(self, line):
-        # this shouldn't be needed
-        raise NotImplementedError
-
-    def complete(self, text, state):
-        if readline is None:
-            return
-        if state == 0:
-            try:
-                self.completion_matches = self.find_completions(text)
-            except Exception:
-                if debug_completion:
-                    import traceback
-                    traceback.print_exc()
-                else:
-                    raise
-        try:
-            return self.completion_matches[state]
-        except IndexError:
-            return None
-
-    def find_completions(self, text):
-        curline = readline.get_line_buffer()
-        prevlines = self.statement.getvalue()
-        wholestmt = prevlines + curline
-        begidx = readline.get_begidx() + len(prevlines)
-        stuff_to_complete = wholestmt[:begidx]
-        return cqlruleset.cql_complete(stuff_to_complete, text, cassandra_conn=self,
-                                       debug=debug_completion, startsymbol='cqlshCommand')
-
-    def set_prompt(self, prompt, prepend_user=False):
-        if prepend_user and self.username:
-            self.prompt = "%s@%s" % (self.username, prompt)
-            return
-        self.prompt = prompt
-
-    def cql_unprotect_name(self, namestr):
-        if namestr is None:
-            return
-        return cqlruleset.dequote_name(namestr)
-
-    def cql_unprotect_value(self, valstr):
-        if valstr is not None:
-            return cqlruleset.dequote_value(valstr)
-
-    def print_recreate_keyspace(self, ksdef, out):
-        out.write(ksdef.export_as_string())
-        out.write("\n")
-
-    def print_recreate_columnfamily(self, ksname, cfname, out):
-        """
-        Output CQL commands which should be pasteable back into a CQL session
-        to recreate the given table.
-
-        Writes output to the given out stream.
-        """
-        out.write(self.get_table_meta(ksname, cfname).export_as_string())
-        out.write("\n")
-
-    def print_recreate_index(self, ksname, idxname, out):
-        """
-        Output CQL commands which should be pasteable back into a CQL session
-        to recreate the given index.
-
-        Writes output to the given out stream.
-        """
-        out.write(self.get_index_meta(ksname, idxname).export_as_string())
-        out.write("\n")
-
-    def print_recreate_object(self, ks, name, out):
-        """
-        Output CQL commands which should be pasteable back into a CQL session
-        to recreate the given object (ks, table or index).
-
-        Writes output to the given out stream.
-        """
-        out.write(self.get_object_meta(ks, name).export_as_string())
-        out.write("\n")
-
-    def describe_keyspaces(self):
-        print
-        cmd.Cmd.columnize(self, protect_names(self.get_keyspace_names()))
-        print
-
-    def describe_keyspace(self, ksname):
-        print
-        self.print_recreate_keyspace(self.get_keyspace_meta(ksname), sys.stdout)
-        print
-
-    def describe_columnfamily(self, ksname, cfname):
-        if ksname is None:
-            ksname = self.current_keyspace
-        print
-        self.print_recreate_columnfamily(ksname, cfname, sys.stdout)
-        print
-
-    def describe_index(self, ksname, idxname):
-        print
-        self.print_recreate_index(ksname, idxname, sys.stdout)
-        print
-
-    def describe_object(self, ks, name):
-        print
-        self.print_recreate_object(ks, name, sys.stdout)
-        print
-
-    def describe_columnfamilies(self, ksname):
-        print
-        if ksname is None:
-            for k in self.get_keyspaces():
-                name = protect_name(k.name)
-                print 'Keyspace %s' % (name,)
-                print '---------%s' % ('-' * len(name))
-                cmd.Cmd.columnize(self, protect_names(self.get_columnfamily_names(k.name)))
-                print
-        else:
-            cmd.Cmd.columnize(self, protect_names(self.get_columnfamily_names(ksname)))
-            print
-
-    def describe_usertypes(self, ksname):
-        print
-        if ksname is None:
-            for ksmeta in self.get_keyspaces():
-                name = protect_name(ksmeta.name)
-                print 'Keyspace %s' % (name,)
-                print '---------%s' % ('-' * len(name))
-                cmd.Cmd.columnize(self, protect_names(ksmeta.user_types.keys()))
-                print
-        else:
-            ksmeta = self.get_keyspace_meta(ksname)
-            cmd.Cmd.columnize(self, protect_names(ksmeta.user_types.keys()))
-            print
-
-    def describe_usertype(self, ksname, typename):
-        if ksname is None:
-            ksname = self.current_keyspace
-        print
-        ksmeta = self.get_keyspace_meta(ksname)
-        try:
-            usertype = ksmeta.user_types[typename]
-        except KeyError:
-            raise UserTypeNotFound("User type %r not found" % typename)
-        print usertype.as_cql_query(formatted=True)
-        print
-
-    def describe_cluster(self):
-        print '\nCluster: %s' % self.get_cluster_name()
-        p = trim_if_present(self.get_partitioner(), 'org.apache.cassandra.dht.')
-        print 'Partitioner: %s\n' % p
-        # TODO: snitch?
-        # snitch = trim_if_present(self.get_snitch(), 'org.apache.cassandra.locator.')
-        # print 'Snitch: %s\n' % snitch
-        if self.current_keyspace is not None and self.current_keyspace != 'system':
-            print "Range ownership:"
-            ring = self.get_ring(self.current_keyspace)
-            for entry in ring.items():
-                print ' %39s  [%s]' % (str(entry[0].value), ', '.join([host.address for host in entry[1]]))
-            print
-
-    def describe_schema(self, include_system=False):
-        print
-        for k in self.get_keyspaces():
-            if include_system or k.name not in SYSTEM_KEYSPACES:
-                self.print_recreate_keyspace(k, sys.stdout)
-                print
-
-    def do_describe(self, parsed):
-        """
-        DESCRIBE [cqlsh only]
-
-        (DESC may be used as a shorthand.)
-
-          Outputs information about the connected Cassandra cluster, or about
-          the data objects stored in the cluster. Use in one of the following ways:
-
-        DESCRIBE KEYSPACES
-
-          Output the names of all keyspaces.
-
-        DESCRIBE KEYSPACE [<keyspacename>]
-
-          Output CQL commands that could be used to recreate the given keyspace,
-          and the objects in it (such as tables, types, etc.).
-          In some cases, as the CQL interface matures, there will be some metadata
-          about a keyspace that is not representable with CQL. That metadata will not be shown.
-
-          The '<keyspacename>' argument may be omitted, in which case the current
-          keyspace will be described.
-
-        DESCRIBE TABLES
-
-          Output the names of all tables in the current keyspace, or in all
-          keyspaces if there is no current keyspace.
-
-        DESCRIBE TABLE [<keyspace>.]<tablename>
-
-          Output CQL commands that could be used to recreate the given table.
-          In some cases, as above, there may be table metadata which is not
-          representable and which will not be shown.
-
-        DESCRIBE INDEX <indexname>
-
-          Output the CQL command that could be used to recreate the given index.
-          In some cases, there may be index metadata which is not representable
-          and which will not be shown.
-
-        DESCRIBE CLUSTER
-
-          Output information about the connected Cassandra cluster, such as the
-          cluster name, and the partitioner and snitch in use. When you are
-          connected to a non-system keyspace, also shows endpoint-range
-          ownership information for the Cassandra ring.
-
-        DESCRIBE [FULL] SCHEMA
-
-          Output CQL commands that could be used to recreate the entire (non-system) schema.
-          Works as though "DESCRIBE KEYSPACE k" was invoked for each non-system keyspace
-          k. Use DESCRIBE FULL SCHEMA to include the system keyspaces.
-
-        DESCRIBE TYPES
-
-          Output the names of all user-defined-types in the current keyspace, or in all
-          keyspaces if there is no current keyspace.
-
-        DESCRIBE TYPE [<keyspace>.]<type>
-
-          Output the CQL command that could be used to recreate the given user-defined-type.
-
-        DESCRIBE <objname>
-
-          Output CQL commands that could be used to recreate the entire object schema,
-          where object can be either a keyspace or a table or an index (in this order).
-
-        """
-        what = parsed.matched[1][1].lower()
-        if what == 'keyspaces':
-            self.describe_keyspaces()
-        elif what == 'keyspace':
-            ksname = self.cql_unprotect_name(parsed.get_binding('ksname', ''))
-            if not ksname:
-                ksname = self.current_keyspace
-                if ksname is None:
-                    self.printerr('Not in any keyspace.')
-                    return
-            self.describe_keyspace(ksname)
-        elif what in ('columnfamily', 'table'):
-            ks = self.cql_unprotect_name(parsed.get_binding('ksname', None))
-            cf = self.cql_unprotect_name(parsed.get_binding('cfname'))
-            self.describe_columnfamily(ks, cf)
-        elif what == 'index':
-            ks = self.cql_unprotect_name(parsed.get_binding('ksname', None))
-            idx = self.cql_unprotect_name(parsed.get_binding('idxname', None))
-            self.describe_index(ks, idx)
-        elif what in ('columnfamilies', 'tables'):
-            self.describe_columnfamilies(self.current_keyspace)
-        elif what == 'types':
-            self.describe_usertypes(self.current_keyspace)
-        elif what == 'type':
-            ks = self.cql_unprotect_name(parsed.get_binding('ksname', None))
-            ut = self.cql_unprotect_name(parsed.get_binding('utname'))
-            self.describe_usertype(ks, ut)
-        elif what == 'cluster':
-            self.describe_cluster()
-        elif what == 'schema':
-            self.describe_schema(False)
-        elif what == 'full' and parsed.matched[2][1].lower() == 'schema':
-            self.describe_schema(True)
-        elif what:
-            ks = self.cql_unprotect_name(parsed.get_binding('ksname', None))
-            name = self.cql_unprotect_name(parsed.get_binding('cfname'))
-            if not name:
-                name = self.cql_unprotect_name(parsed.get_binding('idxname', None))
-            self.describe_object(ks, name)
-    do_desc = do_describe
-
-    def do_copy(self, parsed):
-        r"""
-        COPY [cqlsh only]
-
-          COPY x FROM: Imports CSV data into a Cassandra table
-          COPY x TO: Exports data from a Cassandra table in CSV format.
-
-        COPY <table_name> [ ( column [, ...] ) ]
-             FROM ( '<file_pattern_1, file_pattern_2, ... file_pattern_n>' | STDIN )
-             [ WITH <option>='value' [AND ...] ];
-
-        File patterns are either file names or valid python glob expressions, e.g. *.csv or folder/*.csv.
-
-        COPY <table_name> [ ( column [, ...] ) ]
-             TO ( '<filename>' | STDOUT )
-             [ WITH <option>='value' [AND ...] ];
-
-        Available common COPY options and defaults:
-
-          DELIMITER=','           - character that appears between records
-          QUOTE='"'               - quoting character to be used to quote fields
-          ESCAPE='\'              - character to appear before the QUOTE char when quoted
-          HEADER=false            - whether to ignore the first line
-          NULL=''                 - string that represents a null value
-          DATETIMEFORMAT=         - timestamp strftime format
-            '%Y-%m-%d %H:%M:%S%z'   defaults to time_format value in cqlshrc
-          MAXATTEMPTS=5           - the maximum number of attempts per batch or range
-          REPORTFREQUENCY=0.25    - the frequency with which we display status updates in seconds
-          DECIMALSEP='.'          - the separator for decimal values
-          THOUSANDSSEP=''         - the separator for thousands digit groups
-          BOOLSTYLE='True,False'  - the representation for booleans, case insensitive, specify true followed by false,
-                                    for example yes,no or 1,0
-          NUMPROCESSES=n          - the number of worker processes, by default the number of cores minus one
-                                    capped at 16
-          CONFIGFILE=''           - a configuration file with the same format as .cqlshrc (see the Python ConfigParser
-                                    documentation) where you can specify WITH options under the following optional
-                                    sections: [copy], [copy-to], [copy-from], [copy:ks.table], [copy-to:ks.table],
-                                    [copy-from:ks.table], where <ks> is your keyspace name and <table> is your table
-                                    name. Options are read from these sections, in the order specified
-                                    above, and command line options always override options in configuration files.
-                                    Depending on the COPY direction, only the relevant copy-from or copy-to sections
-                                    are used. If no configfile is specified then .cqlshrc is searched instead.
-          RATEFILE=''             - an optional file where to print the output statistics
-
-        Available COPY FROM options and defaults:
-
-          CHUNKSIZE=5000          - the size of chunks passed to worker processes
-          INGESTRATE=100000       - an approximate ingest rate in rows per second
-          MINBATCHSIZE=10         - the minimum size of an import batch
-          MAXBATCHSIZE=20         - the maximum size of an import batch
-          MAXROWS=-1              - the maximum number of rows, -1 means no maximum
-          SKIPROWS=0              - the number of rows to skip
-          SKIPCOLS=''             - a comma separated list of column names to skip
-          MAXPARSEERRORS=-1       - the maximum global number of parsing errors, -1 means no maximum
-          MAXINSERTERRORS=-1      - the maximum global number of insert errors, -1 means no maximum
-          ERRFILE=''              - a file where to store all rows that could not be imported, by default this is
-                                    import_ks_table.err where <ks> is your keyspace and <table> is your table name.
-          PREPAREDSTATEMENTS=True - whether to use prepared statements when importing, by default True. Set this to
-                                    False if you don't mind shifting data parsing to the cluster. The cluster will also
-                                    have to compile every batch statement. For large and oversized clusters
-                                    this will result in a faster import but for smaller clusters it may generate
-                                    timeouts.
-
-        Available COPY TO options and defaults:
-
-          ENCODING='utf8'          - encoding for CSV output
-          PAGESIZE='1000'          - the page size for fetching results
-          PAGETIMEOUT=10           - the page timeout in seconds for fetching results
-          BEGINTOKEN=''            - the minimum token string to consider when exporting data
-          ENDTOKEN=''              - the maximum token string to consider when exporting data
-          MAXREQUESTS=6            - the maximum number of requests each worker process can work on in parallel
-          MAXOUTPUTSIZE='-1'       - the maximum size of the output file measured in number of lines,
-                                     beyond this maximum the output file will be split into segments,
-                                     -1 means unlimited.
-
-        When entering CSV data on STDIN, you can use the sequence "\."
-        on a line by itself to end the data input.
-        """
-
-        ks = self.cql_unprotect_name(parsed.get_binding('ksname', None))
-        if ks is None:
-            ks = self.current_keyspace
-            if ks is None:
-                raise NoKeyspaceError("Not in any keyspace.")
-        table = self.cql_unprotect_name(parsed.get_binding('cfname'))
-        columns = parsed.get_binding('colnames', None)
-        if columns is not None:
-            columns = map(self.cql_unprotect_name, columns)
-        else:
-            # default to all known columns
-            columns = self.get_column_names(ks, table)
-        fname = parsed.get_binding('fname', None)
-        if fname is not None:
-            fname = self.cql_unprotect_value(fname)
-        copyoptnames = map(str.lower, parsed.get_binding('optnames', ()))
-        copyoptvals = map(self.cql_unprotect_value, parsed.get_binding('optvals', ()))
-        opts = dict(zip(copyoptnames, copyoptvals))
-
-        direction = parsed.get_binding('dir').upper()
-        if direction == 'FROM':
-            task = ImportTask(self, ks, table, columns, fname, opts, DEFAULT_PROTOCOL_VERSION, CONFIG_FILE)
-        elif direction == 'TO':
-            task = ExportTask(self, ks, table, columns, fname, opts, DEFAULT_PROTOCOL_VERSION, CONFIG_FILE)
-        else:
-            raise SyntaxError("Unknown direction %s" % direction)
-
-        task.run()
-
-    def do_show(self, parsed):
-        """
-        SHOW [cqlsh only]
-
-          Displays information about the current cqlsh session. Can be called in
-          the following ways:
-
-        SHOW VERSION
-
-          Shows the version and build of the connected Cassandra instance, as
-          well as the versions of the CQL spec and the Thrift protocol that
-          the connected Cassandra instance understands.
-
-        SHOW HOST
-
-          Shows where cqlsh is currently connected.
-
-        SHOW SESSION <sessionid>
-
-          Pretty-prints the requested tracing session.
-        """
-        showwhat = parsed.get_binding('what').lower()
-        if showwhat == 'version':
-            self.get_connection_versions()
-            self.show_version()
-        elif showwhat == 'host':
-            self.show_host()
-        elif showwhat.startswith('session'):
-            session_id = parsed.get_binding('sessionid').lower()
-            self.show_session(UUID(session_id))
-        else:
-            self.printerr('Wait, how do I show %r?' % (showwhat,))
-
-    def do_source(self, parsed):
-        """
-        SOURCE [cqlsh only]
-
-        Executes a file containing CQL statements. Gives the output for each
-        statement in turn, if any, or any errors that occur along the way.
-
-        Errors do NOT abort execution of the CQL source file.
-
-        Usage:
-
-          SOURCE '<file>';
-
-        That is, the path to the file to be executed must be given inside a
-        string literal. The path is interpreted relative to the current working
-        directory. The tilde shorthand notation ('~/mydir') is supported for
-        referring to $HOME.
-
-        See also the --file option to cqlsh.
-        """
-        fname = parsed.get_binding('fname')
-        fname = os.path.expanduser(self.cql_unprotect_value(fname))
-        try:
-            encoding, bom_size = get_file_encoding_bomsize(fname)
-            f = codecs.open(fname, 'r', encoding)
-            f.seek(bom_size)
-        except IOError, e:
-            self.printerr('Could not open %r: %s' % (fname, e))
-            return
-        subshell = Shell(self.hostname, self.port,
-                         color=self.color, encoding=self.encoding, stdin=f,
-                         tty=False, use_conn=self.conn, cqlver=self.cql_version,
-                         keyspace=self.current_keyspace,
-                         display_time_format=self.display_time_format,
-                         display_float_precision=self.display_float_precision,
-                         max_trace_wait=self.max_trace_wait)
-        subshell.cmdloop()
-        f.close()
-
-    def do_capture(self, parsed):
-        """
-        CAPTURE [cqlsh only]
-
-        Begins capturing command output and appending it to a specified file.
-        Output will not be shown at the console while it is captured.
-
-        Usage:
-
-          CAPTURE '<file>';
-          CAPTURE OFF;
-          CAPTURE;
-
-        That is, the path to the file to be appended to must be given inside a
-        string literal. The path is interpreted relative to the current working
-        directory. The tilde shorthand notation ('~/mydir') is supported for
-        referring to $HOME.
-
-        Only query result output is captured. Errors and output from cqlsh-only
-        commands will still be shown in the cqlsh session.
-
-        To stop capturing output and show it in the cqlsh session again, use
-        CAPTURE OFF.
-
-        To inspect the current capture configuration, use CAPTURE with no
-        arguments.
-        """
-        fname = parsed.get_binding('fname')
-        if fname is None:
-            if self.shunted_query_out is not None:
-                print "Currently capturing query output to %r." % (self.query_out.name,)
-            else:
-                print "Currently not capturing query output."
-            return
-
-        if fname.upper() == 'OFF':
-            if self.shunted_query_out is None:
-                self.printerr('Not currently capturing output.')
-                return
-            self.query_out.close()
-            self.query_out = self.shunted_query_out
-            self.color = self.shunted_color
-            self.shunted_query_out = None
-            del self.shunted_color
-            return
-
-        if self.shunted_query_out is not None:
-            self.printerr('Already capturing output to %s. Use CAPTURE OFF'
-                          ' to disable.' % (self.query_out.name,))
-            return
-
-        fname = os.path.expanduser(self.cql_unprotect_value(fname))
-        try:
-            f = open(fname, 'a')
-        except IOError, e:
-            self.printerr('Could not open %r for append: %s' % (fname, e))
-            return
-        self.shunted_query_out = self.query_out
-        self.shunted_color = self.color
-        self.query_out = f
-        self.color = False
-        print 'Now capturing query output to %r.' % (fname,)
-
-    def do_tracing(self, parsed):
-        """
-        TRACING [cqlsh]
-
-          Enables or disables request tracing.
-
-        TRACING ON
-
-          Enables tracing for all further requests.
-
-        TRACING OFF
-
-          Disables tracing.
-
-        TRACING
-
-          TRACING with no arguments shows the current tracing status.
-        """
-        self.tracing_enabled = SwitchCommand("TRACING", "Tracing").execute(self.tracing_enabled, parsed, self.printerr)
-
-    def do_expand(self, parsed):
-        """
-        EXPAND [cqlsh]
-
-          Enables or disables expanded (vertical) output.
-
-        EXPAND ON
-
-          Enables expanded (vertical) output.
-
-        EXPAND OFF
-
-          Disables expanded (vertical) output.
-
-        EXPAND
-
-          EXPAND with no arguments shows the current value of expand setting.
-        """
-        self.expand_enabled = SwitchCommand("EXPAND", "Expanded output").execute(self.expand_enabled, parsed, self.printerr)
-
-    def do_consistency(self, parsed):
-        """
-        CONSISTENCY [cqlsh only]
-
-           Overrides default consistency level (default level is ONE).
-
-        CONSISTENCY <level>
-
-           Sets consistency level for future requests.
-
-           Valid consistency levels:
-
-           ANY, ONE, TWO, THREE, QUORUM, ALL, LOCAL_ONE, LOCAL_QUORUM, EACH_QUORUM, SERIAL and LOCAL_SERIAL.
-
-           SERIAL and LOCAL_SERIAL may be used only for SELECTs; will be rejected with updates.
-
-        CONSISTENCY
-
-           CONSISTENCY with no arguments shows the current consistency level.
-        """
-        level = parsed.get_binding('level')
-        if level is None:
-            print 'Current consistency level is %s.' % (cassandra.ConsistencyLevel.value_to_name[self.consistency_level])
-            return
-
-        self.consistency_level = cassandra.ConsistencyLevel.name_to_value[level.upper()]
-        print 'Consistency level set to %s.' % (level.upper(),)
-
-    def do_serial(self, parsed):
-        """
-        SERIAL CONSISTENCY [cqlsh only]
-
-           Overrides serial consistency level (default level is SERIAL).
-
-        SERIAL CONSISTENCY <level>
-
-           Sets consistency level for future conditional updates.
-
-           Valid consistency levels:
-
-           SERIAL, LOCAL_SERIAL.
-
-        SERIAL CONSISTENCY
-
-           SERIAL CONSISTENCY with no arguments shows the current consistency level.
-        """
-        level = parsed.get_binding('level')
-        if level is None:
-            print 'Current serial consistency level is %s.' % (cassandra.ConsistencyLevel.value_to_name[self.serial_consistency_level])
-            return
-
-        self.serial_consistency_level = cassandra.ConsistencyLevel.name_to_value[level.upper()]
-        print 'Serial consistency level set to %s.' % (level.upper(),)
-
-    def do_login(self, parsed):
-        """
-        LOGIN [cqlsh only]
-
-           Changes login information without requiring restart.
-
-        LOGIN <username> (<password>)
-
-           Login using the specified username. If password is specified, it will be used
-           otherwise, you will be prompted to enter.
-        """
-        username = parsed.get_binding('username')
-        password = parsed.get_binding('password')
-        if password is None:
-            password = getpass.getpass()
-        else:
-            password = password[1:-1]
-
-        auth_provider = PlainTextAuthProvider(username=username, password=password)
-
-        conn = Cluster(contact_points=(self.hostname,), port=self.port, cql_version=self.conn.cql_version,
-                       protocol_version=DEFAULT_PROTOCOL_VERSION,
-                       auth_provider=auth_provider,
-                       ssl_options=self.conn.ssl_options,
-                       load_balancing_policy=WhiteListRoundRobinPolicy([self.hostname]),
-                       connect_timeout=self.conn.connect_timeout)
-
-        if self.current_keyspace:
-            session = conn.connect(self.current_keyspace)
-        else:
-            session = conn.connect()
-
-        # Update after we've connected in case we fail to authenticate
-        self.conn = conn
-        self.auth_provider = auth_provider
-        self.username = username
-        self.session = session
-
-    def do_exit(self, parsed=None):
-        """
-        EXIT/QUIT [cqlsh only]
-
-        Exits cqlsh.
-        """
-        self.stop = True
-        if self.owns_connection:
-            self.conn.shutdown()
-    do_quit = do_exit
-
-    def do_debug(self, parsed):
-        import pdb
-        pdb.set_trace()
-
-    def get_help_topics(self):
-        topics = [t[3:] for t in dir(self) if t.startswith('do_') and getattr(self, t, None).__doc__]
-        for hide_from_help in ('quit',):
-            topics.remove(hide_from_help)
-        return topics
-
-    def columnize(self, slist, *a, **kw):
-        return cmd.Cmd.columnize(self, sorted([u.upper() for u in slist]), *a, **kw)
-
-    def do_help(self, parsed):
-        """
-        HELP [cqlsh only]
-
-        Gives information about cqlsh commands. To see available topics,
-        enter "HELP" without any arguments. To see help on a topic,
-        use "HELP <topic>".
-        """
-        topics = parsed.get_binding('topic', ())
-        if not topics:
-            shell_topics = [t.upper() for t in self.get_help_topics()]
-            self.print_topics("\nDocumented shell commands:", shell_topics, 15, 80)
-            cql_topics = [t.upper() for t in cqldocs.get_help_topics()]
-            self.print_topics("CQL help topics:", cql_topics, 15, 80)
-            return
-        for t in topics:
-            if t.lower() in self.get_help_topics():
-                doc = getattr(self, 'do_' + t.lower()).__doc__
-                self.stdout.write(doc + "\n")
-            elif t.lower() in cqldocs.get_help_topics():
-                cqldocs.print_help_topic(t)
-            else:
-                self.printerr("*** No help on %s" % (t,))
-
-    def do_paging(self, parsed):
-        """
-        PAGING [cqlsh]
-
-          Enables or disables query paging.
-
-        PAGING ON
-
-          Enables query paging for all further queries.
-
-        PAGING OFF
-
-          Disables paging.
-
-        PAGING
-
-          PAGING with no arguments shows the current query paging status.
-        """
-        self.use_paging = SwitchCommand("PAGING", "Query paging").execute(self.use_paging, parsed, self.printerr)
-
-    def applycolor(self, text, color=None):
-        if not color or not self.color:
-            return text
-        return color + text + ANSI_RESET
-
-    def writeresult(self, text, color=None, newline=True, out=None):
-        if out is None:
-            out = self.query_out
-        out.write(self.applycolor(str(text), color) + ('\n' if newline else ''))
-
-    def flush_output(self):
-        self.query_out.flush()
-
-    def printerr(self, text, color=RED, newline=True, shownum=None):
-        self.statement_error = True
-        if shownum is None:
-            shownum = self.show_line_nums
-        if shownum:
-            text = '%s:%d:%s' % (self.stdin.name, self.lineno, text)
-        self.writeresult(text, color, newline=newline, out=sys.stderr)
-
-
-class SwitchCommand(object):
-    command = None
-    description = None
-
-    def __init__(self, command, desc):
-        self.command = command
-        self.description = desc
-
-    def execute(self, state, parsed, printerr):
-        switch = parsed.get_binding('switch')
-        if switch is None:
-            if state:
-                print "%s is currently enabled. Use %s OFF to disable" \
-                      % (self.description, self.command)
-            else:
-                print "%s is currently disabled. Use %s ON to enable." \
-                      % (self.description, self.command)
-            return state
-
-        if switch.upper() == 'ON':
-            if state:
-                printerr('%s is already enabled. Use %s OFF to disable.'
-                         % (self.description, self.command))
-                return state
-            print 'Now %s is enabled' % (self.description,)
-            return True
-
-        if switch.upper() == 'OFF':
-            if not state:
-                printerr('%s is not enabled.' % (self.description,))
-                return state
-            print 'Disabled %s.' % (self.description,)
-            return False
-
-
-def option_with_default(cparser_getter, section, option, default=None):
-    try:
-        return cparser_getter(section, option)
-    except ConfigParser.Error:
-        return default
-
-
-def raw_option_with_default(configs, section, option, default=None):
-    """
-    Same (almost) as option_with_default() but won't do any string interpolation.
-    Useful for config values that include '%' symbol, e.g. time format string.
-    """
-    try:
-        return configs.get(section, option, raw=True)
-    except ConfigParser.Error:
-        return default
-
-
-def should_use_color():
-    if not sys.stdout.isatty():
-        return False
-    if os.environ.get('TERM', '') in ('dumb', ''):
-        return False
-    try:
-        import subprocess
-        p = subprocess.Popen(['tput', 'colors'], stdout=subprocess.PIPE)
-        stdout, _ = p.communicate()
-        if int(stdout.strip()) < 8:
-            return False
-    except (OSError, ImportError, ValueError):
-        # oh well, we tried. at least we know there's a $TERM and it's
-        # not "dumb".
-        pass
-    return True
-
-
-def read_options(cmdlineargs, environment):
-    configs = ConfigParser.SafeConfigParser()
-    configs.read(CONFIG_FILE)
-
-    rawconfigs = ConfigParser.RawConfigParser()
-    rawconfigs.read(CONFIG_FILE)
-
-    optvalues = optparse.Values()
-    optvalues.username = option_with_default(configs.get, 'authentication', 'username')
-    optvalues.password = option_with_default(rawconfigs.get, 'authentication', 'password')
-    optvalues.keyspace = option_with_default(configs.get, 'authentication', 'keyspace')
-    optvalues.completekey = option_with_default(configs.get, 'ui', 'completekey',
-                                                DEFAULT_COMPLETEKEY)
-    optvalues.color = option_with_default(configs.getboolean, 'ui', 'color')
-    optvalues.time_format = raw_option_with_default(configs, 'ui', 'time_format',
-                                                    DEFAULT_TIME_FORMAT)
-    optvalues.float_precision = option_with_default(configs.getint, 'ui', 'float_precision',
-                                                    DEFAULT_FLOAT_PRECISION)
-    optvalues.field_size_limit = option_with_default(configs.getint, 'csv', 'field_size_limit', csv.field_size_limit())
-    optvalues.max_trace_wait = option_with_default(configs.getfloat, 'tracing', 'max_trace_wait',
-                                                   DEFAULT_MAX_TRACE_WAIT)
-
-    optvalues.debug = False
-    optvalues.file = None
-    optvalues.ssl = False
-    optvalues.encoding = None
-
-    optvalues.tty = sys.stdin.isatty()
-    optvalues.cqlversion = option_with_default(configs.get, 'cql', 'version', DEFAULT_CQLVER)
-    optvalues.connect_timeout = option_with_default(configs.getint, 'connection', 'timeout', DEFAULT_CONNECT_TIMEOUT_SECONDS)
-    optvalues.request_timeout = option_with_default(configs.getint, 'connection', 'request_timeout', DEFAULT_CONNECT_TIMEOUT_SECONDS)
-    optvalues.execute = None
-
-    (options, arguments) = parser.parse_args(cmdlineargs, values=optvalues)
-
-    hostname = option_with_default(configs.get, 'connection', 'hostname', DEFAULT_HOST)
-    port = option_with_default(configs.get, 'connection', 'port', DEFAULT_PORT)
-
-    try:
-        options.connect_timeout = int(options.connect_timeout)
-    except ValueError:
-        parser.error('"%s" is not a valid connect timeout.' % (options.connect_timeout,))
-        options.connect_timeout = DEFAULT_CONNECT_TIMEOUT_SECONDS
-
-    try:
-        options.request_timeout = int(options.request_timeout)
-    except ValueError:
-        parser.error('"%s" is not a valid request timeout.' % (options.request_timeout,))
-        options.request_timeout = DEFAULT_REQUEST_TIMEOUT_SECONDS
-
-    hostname = environment.get('CQLSH_HOST', hostname)
-    port = environment.get('CQLSH_PORT', port)
-
-    if len(arguments) > 0:
-        hostname = arguments[0]
-    if len(arguments) > 1:
-        port = arguments[1]
-
-    if options.file or options.execute:
-        options.tty = False
-
-    if options.execute and not options.execute.endswith(';'):
-        options.execute += ';'
-
-    if optvalues.color in (True, False):
-        options.color = optvalues.color
-    else:
-        if options.file is not None:
-            options.color = False
-        else:
-            options.color = should_use_color()
-
-    options.cqlversion, cqlvertup = full_cql_version(options.cqlversion)
-    if cqlvertup[0] < 3:
-        parser.error('%r is not a supported CQL version.' % options.cqlversion)
-    else:
-        options.cqlmodule = cql3handling
-
-    try:
-        port = int(port)
-    except ValueError:
-        parser.error('%r is not a valid port number.' % port)
-
-    return options, hostname, port
-
-
-def setup_cqlruleset(cqlmodule):
-    global cqlruleset
-    cqlruleset = cqlmodule.CqlRuleSet
-    cqlruleset.append_rules(cqlsh_extra_syntax_rules)
-    for rulename, termname, func in cqlsh_syntax_completers:
-        cqlruleset.completer_for(rulename, termname)(func)
-    cqlruleset.commands_end_with_newline.update(my_commands_ending_with_newline)
-
-
-def setup_cqldocs(cqlmodule):
-    global cqldocs
-    cqldocs = cqlmodule.cqldocs
-
-
-def init_history():
-    if readline is not None:
-        try:
-            readline.read_history_file(HISTORY)
-        except IOError:
-            pass
-        delims = readline.get_completer_delims()
-        delims.replace("'", "")
-        delims += '.'
-        readline.set_completer_delims(delims)
-
-
-def save_history():
-    if readline is not None:
-        try:
-            readline.write_history_file(HISTORY)
-        except IOError:
-            pass
-
-
-def main(options, hostname, port):
-    setup_cqlruleset(options.cqlmodule)
-    setup_cqldocs(options.cqlmodule)
-    init_history()
-    csv.field_size_limit(options.field_size_limit)
-
-    if options.file is None:
-        stdin = None
-    else:
-        try:
-            encoding, bom_size = get_file_encoding_bomsize(options.file)
-            stdin = codecs.open(options.file, 'r', encoding)
-            stdin.seek(bom_size)
-        except IOError, e:
-            sys.exit("Can't open %r: %s" % (options.file, e))
-
-    if options.debug:
-        sys.stderr.write("Using CQL driver: %s\n" % (cassandra,))
-        sys.stderr.write("Using connect timeout: %s seconds\n" % (options.connect_timeout,))
-
-    try:
-        shell = Shell(hostname,
-                      port,
-                      color=options.color,
-                      username=options.username,
-                      password=options.password,
-                      stdin=stdin,
-                      tty=options.tty,
-                      completekey=options.completekey,
-                      cqlver=options.cqlversion,
-                      keyspace=options.keyspace,
-                      display_time_format=options.time_format,
-                      display_float_precision=options.float_precision,
-                      max_trace_wait=options.max_trace_wait,
-                      ssl=options.ssl,
-                      single_statement=options.execute,
-                      request_timeout=options.request_timeout,
-                      connect_timeout=options.connect_timeout,
-                      encoding=options.encoding)
-    except KeyboardInterrupt:
-        sys.exit('Connection aborted.')
-    except CQL_ERRORS, e:
-        sys.exit('Connection error: %s' % (e,))
-    except VersionNotSupported, e:
-        sys.exit('Unsupported CQL version: %s' % (e,))
-    if options.debug:
-        shell.debug = True
-
-    shell.cmdloop()
-    save_history()
-    batch_mode = options.file or options.execute
-    if batch_mode and shell.statement_error:
-        sys.exit(2)
-
-# always call this regardless of module name: when a sub-process is spawned
-# on Windows then the module name is not __main__, see CASSANDRA-9304
-insert_driver_hooks()
-
-if __name__ == '__main__':
-    main(*read_options(sys.argv[1:], os.environ))
-
-# vim: set ft=python et ts=4 sw=4 :
diff --git a/bin/cqlsh.bat b/bin/cqlsh.bat
index 066b1d0..6170373 100644
--- a/bin/cqlsh.bat
+++ b/bin/cqlsh.bat
@@ -22,7 +22,7 @@
 python -V >nul 2>&1

 if ERRORLEVEL 1 goto err

 

-python "%~dp0\cqlsh" %*

+python "%~dp0\cqlsh.py" %*

 goto finally

 

 :err

diff --git a/bin/cqlsh.py b/bin/cqlsh.py
new file mode 100644
index 0000000..a9c5ff1
--- /dev/null
+++ b/bin/cqlsh.py
@@ -0,0 +1,2582 @@
+#!/bin/sh
+# -*- mode: Python -*-
+
+# 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.
+
+""":"
+# bash code here; finds a suitable python interpreter and execs this file.
+# prefer unqualified "python" if suitable:
+python -c 'import sys; sys.exit(not (0x020700b0 < sys.hexversion < 0x03000000))' 2>/dev/null \
+    && exec python "$0" "$@"
+for pyver in 2.7; do
+    which python$pyver > /dev/null 2>&1 && exec python$pyver "$0" "$@"
+done
+echo "No appropriate python interpreter found." >&2
+exit 1
+":"""
+
+from __future__ import with_statement
+
+import cmd
+import codecs
+import ConfigParser
+import csv
+import getpass
+import optparse
+import os
+import platform
+import sys
+import traceback
+import warnings
+import webbrowser
+from StringIO import StringIO
+from contextlib import contextmanager
+from glob import glob
+from uuid import UUID
+
+if sys.version_info[0] != 2 or sys.version_info[1] != 7:
+    sys.exit("\nCQL Shell supports only Python 2.7\n")
+
+UTF8 = 'utf-8'
+CP65001 = 'cp65001'  # Win utf-8 variant
+
+description = "CQL Shell for Apache Cassandra"
+version = "5.0.1"
+
+readline = None
+try:
+    # check if tty first, cause readline doesn't check, and only cares
+    # about $TERM. we don't want the funky escape code stuff to be
+    # output if not a tty.
+    if sys.stdin.isatty():
+        import readline
+except ImportError:
+    pass
+
+CQL_LIB_PREFIX = 'cassandra-driver-internal-only-'
+
+CASSANDRA_PATH = os.path.join(os.path.dirname(os.path.realpath(__file__)), '..')
+CASSANDRA_CQL_HTML_FALLBACK = 'https://cassandra.apache.org/doc/cql3/CQL-2.2.html'
+
+if os.path.exists(CASSANDRA_PATH + '/doc/cql3/CQL.html'):
+    # default location of local CQL.html
+    CASSANDRA_CQL_HTML = 'file://' + CASSANDRA_PATH + '/doc/cql3/CQL.html'
+elif os.path.exists('/usr/share/doc/cassandra/CQL.html'):
+    # fallback to package file
+    CASSANDRA_CQL_HTML = 'file:///usr/share/doc/cassandra/CQL.html'
+else:
+    # fallback to online version
+    CASSANDRA_CQL_HTML = CASSANDRA_CQL_HTML_FALLBACK
+
+# On Linux, the Python webbrowser module uses the 'xdg-open' executable
+# to open a file/URL. But that only works, if the current session has been
+# opened from _within_ a desktop environment. I.e. 'xdg-open' will fail,
+# if the session's been opened via ssh to a remote box.
+#
+# Use 'python' to get some information about the detected browsers.
+# >>> import webbrowser
+# >>> webbrowser._tryorder
+# >>> webbrowser._browser
+#
+if len(webbrowser._tryorder) == 0:
+    CASSANDRA_CQL_HTML = CASSANDRA_CQL_HTML_FALLBACK
+elif webbrowser._tryorder[0] == 'xdg-open' and os.environ.get('XDG_DATA_DIRS', '') == '':
+    # only on Linux (some OS with xdg-open)
+    webbrowser._tryorder.remove('xdg-open')
+    webbrowser._tryorder.append('xdg-open')
+
+# use bundled libs for python-cql and thrift, if available. if there
+# is a ../lib dir, use bundled libs there preferentially.
+ZIPLIB_DIRS = [os.path.join(CASSANDRA_PATH, 'lib')]
+myplatform = platform.system()
+is_win = myplatform == 'Windows'
+
+# Workaround for supporting CP65001 encoding on python < 3.3 (https://bugs.python.org/issue13216)
+if is_win and sys.version_info < (3, 3):
+    codecs.register(lambda name: codecs.lookup(UTF8) if name == CP65001 else None)
+
+if myplatform == 'Linux':
+    ZIPLIB_DIRS.append('/usr/share/cassandra/lib')
+
+if os.environ.get('CQLSH_NO_BUNDLED', ''):
+    ZIPLIB_DIRS = ()
+
+
+def find_zip(libprefix):
+    for ziplibdir in ZIPLIB_DIRS:
+        zips = glob(os.path.join(ziplibdir, libprefix + '*.zip'))
+        if zips:
+            return max(zips)   # probably the highest version, if multiple
+
+cql_zip = find_zip(CQL_LIB_PREFIX)
+if cql_zip:
+    ver = os.path.splitext(os.path.basename(cql_zip))[0][len(CQL_LIB_PREFIX):]
+    sys.path.insert(0, os.path.join(cql_zip, 'cassandra-driver-' + ver))
+
+third_parties = ('futures-', 'six-')
+
+for lib in third_parties:
+    lib_zip = find_zip(lib)
+    if lib_zip:
+        sys.path.insert(0, lib_zip)
+
+warnings.filterwarnings("ignore", r".*blist.*")
+try:
+    import cassandra
+except ImportError, e:
+    sys.exit("\nPython Cassandra driver not installed, or not on PYTHONPATH.\n"
+             'You might try "pip install cassandra-driver".\n\n'
+             'Python: %s\n'
+             'Module load path: %r\n\n'
+             'Error: %s\n' % (sys.executable, sys.path, e))
+
+from cassandra.auth import PlainTextAuthProvider
+from cassandra.cluster import Cluster
+from cassandra.metadata import (ColumnMetadata, KeyspaceMetadata,
+                                TableMetadata, protect_name, protect_names)
+from cassandra.policies import WhiteListRoundRobinPolicy
+from cassandra.query import SimpleStatement, ordered_dict_factory, TraceUnavailable
+
+# cqlsh should run correctly when run out of a Cassandra source tree,
+# out of an unpacked Cassandra tarball, and after a proper package install.
+cqlshlibdir = os.path.join(CASSANDRA_PATH, 'pylib')
+if os.path.isdir(cqlshlibdir):
+    sys.path.insert(0, cqlshlibdir)
+
+from cqlshlib import cql3handling, cqlhandling, pylexotron, sslhandling
+from cqlshlib.copyutil import ExportTask, ImportTask
+from cqlshlib.displaying import (ANSI_RESET, BLUE, COLUMN_NAME_COLORS, CYAN,
+                                 RED, FormattedValue, colorme)
+from cqlshlib.formatting import (DEFAULT_DATE_FORMAT, DEFAULT_NANOTIME_FORMAT,
+                                 DEFAULT_TIMESTAMP_FORMAT, DateTimeFormat,
+                                 format_by_type, format_value_utype,
+                                 formatter_for)
+from cqlshlib.tracing import print_trace, print_trace_session
+from cqlshlib.util import get_file_encoding_bomsize, trim_if_present
+
+DEFAULT_HOST = '127.0.0.1'
+DEFAULT_PORT = 9042
+DEFAULT_CQLVER = '3.3.1'
+DEFAULT_PROTOCOL_VERSION = 4
+DEFAULT_CONNECT_TIMEOUT_SECONDS = 5
+DEFAULT_REQUEST_TIMEOUT_SECONDS = 10
+
+DEFAULT_FLOAT_PRECISION = 5
+DEFAULT_MAX_TRACE_WAIT = 10
+
+if readline is not None and readline.__doc__ is not None and 'libedit' in readline.__doc__:
+    DEFAULT_COMPLETEKEY = '\t'
+else:
+    DEFAULT_COMPLETEKEY = 'tab'
+
+cqldocs = None
+cqlruleset = None
+
+epilog = """Connects to %(DEFAULT_HOST)s:%(DEFAULT_PORT)d by default. These
+defaults can be changed by setting $CQLSH_HOST and/or $CQLSH_PORT. When a
+host (and optional port number) are given on the command line, they take
+precedence over any defaults.""" % globals()
+
+parser = optparse.OptionParser(description=description, epilog=epilog,
+                               usage="Usage: %prog [options] [host [port]]",
+                               version='cqlsh ' + version)
+parser.add_option("-C", "--color", action='store_true', dest='color',
+                  help='Always use color output')
+parser.add_option("--no-color", action='store_false', dest='color',
+                  help='Never use color output')
+parser.add_option("--browser", dest='browser', help="""The browser to use to display CQL help, where BROWSER can be:
+                                                    - one of the supported browsers in https://docs.python.org/2/library/webbrowser.html.
+                                                    - browser path followed by %s, example: /usr/bin/google-chrome-stable %s""")
+parser.add_option('--ssl', action='store_true', help='Use SSL', default=False)
+parser.add_option("-u", "--username", help="Authenticate as user.")
+parser.add_option("-p", "--password", help="Authenticate using password.")
+parser.add_option('-k', '--keyspace', help='Authenticate to the given keyspace.')
+parser.add_option("-f", "--file", help="Execute commands from FILE, then exit")
+parser.add_option('--debug', action='store_true',
+                  help='Show additional debugging information')
+parser.add_option("--encoding", help="Specify a non-default encoding for output." +
+                  " (Default: %s)" % (UTF8,))
+parser.add_option("--cqlshrc", help="Specify an alternative cqlshrc file location.")
+parser.add_option('--cqlversion', default=DEFAULT_CQLVER,
+                  help='Specify a particular CQL version (default: %default).'
+                       ' Examples: "3.0.3", "3.1.0"')
+parser.add_option("-e", "--execute", help='Execute the statement and quit.')
+parser.add_option("--connect-timeout", default=DEFAULT_CONNECT_TIMEOUT_SECONDS, dest='connect_timeout',
+                  help='Specify the connection timeout in seconds (default: %default seconds).')
+parser.add_option("--request-timeout", default=DEFAULT_REQUEST_TIMEOUT_SECONDS, dest='request_timeout',
+                  help='Specify the default request timeout in seconds (default: %default seconds).')
+parser.add_option("-t", "--tty", action='store_true', dest='tty',
+                  help='Force tty mode (command prompt).')
+
+optvalues = optparse.Values()
+(options, arguments) = parser.parse_args(sys.argv[1:], values=optvalues)
+
+# BEGIN history/config definition
+HISTORY_DIR = os.path.expanduser(os.path.join('~', '.cassandra'))
+
+if hasattr(options, 'cqlshrc'):
+    CONFIG_FILE = options.cqlshrc
+    if not os.path.exists(CONFIG_FILE):
+        print '\nWarning: Specified cqlshrc location `%s` does not exist.  Using `%s` instead.\n' % (CONFIG_FILE, HISTORY_DIR)
+        CONFIG_FILE = os.path.join(HISTORY_DIR, 'cqlshrc')
+else:
+    CONFIG_FILE = os.path.join(HISTORY_DIR, 'cqlshrc')
+
+HISTORY = os.path.join(HISTORY_DIR, 'cqlsh_history')
+if not os.path.exists(HISTORY_DIR):
+    try:
+        os.mkdir(HISTORY_DIR)
+    except OSError:
+        print '\nWarning: Cannot create directory at `%s`. Command history will not be saved.\n' % HISTORY_DIR
+
+OLD_CONFIG_FILE = os.path.expanduser(os.path.join('~', '.cqlshrc'))
+if os.path.exists(OLD_CONFIG_FILE):
+    if os.path.exists(CONFIG_FILE):
+        print '\nWarning: cqlshrc config files were found at both the old location (%s) and \
+                the new location (%s), the old config file will not be migrated to the new \
+                location, and the new location will be used for now.  You should manually \
+                consolidate the config files at the new location and remove the old file.' \
+                % (OLD_CONFIG_FILE, CONFIG_FILE)
+    else:
+        os.rename(OLD_CONFIG_FILE, CONFIG_FILE)
+OLD_HISTORY = os.path.expanduser(os.path.join('~', '.cqlsh_history'))
+if os.path.exists(OLD_HISTORY):
+    os.rename(OLD_HISTORY, HISTORY)
+# END history/config definition
+
+CQL_ERRORS = (
+    cassandra.AlreadyExists, cassandra.AuthenticationFailed, cassandra.InvalidRequest,
+    cassandra.Timeout, cassandra.Unauthorized, cassandra.OperationTimedOut,
+    cassandra.cluster.NoHostAvailable,
+    cassandra.connection.ConnectionBusy, cassandra.connection.ProtocolError, cassandra.connection.ConnectionException,
+    cassandra.protocol.ErrorMessage, cassandra.protocol.InternalError, cassandra.query.TraceUnavailable
+)
+
+debug_completion = bool(os.environ.get('CQLSH_DEBUG_COMPLETION', '') == 'YES')
+
+# we want the cql parser to understand our cqlsh-specific commands too
+my_commands_ending_with_newline = (
+    'help',
+    '?',
+    'consistency',
+    'serial',
+    'describe',
+    'desc',
+    'show',
+    'source',
+    'capture',
+    'login',
+    'debug',
+    'tracing',
+    'expand',
+    'paging',
+    'exit',
+    'quit',
+    'clear',
+    'cls'
+)
+
+
+cqlsh_syntax_completers = []
+
+
+def cqlsh_syntax_completer(rulename, termname):
+    def registrator(f):
+        cqlsh_syntax_completers.append((rulename, termname, f))
+        return f
+    return registrator
+
+
+cqlsh_extra_syntax_rules = r'''
+<cqlshCommand> ::= <CQL_Statement>
+                 | <specialCommand> ( ";" | "\n" )
+                 ;
+
+<specialCommand> ::= <describeCommand>
+                   | <consistencyCommand>
+                   | <serialConsistencyCommand>
+                   | <showCommand>
+                   | <sourceCommand>
+                   | <captureCommand>
+                   | <copyCommand>
+                   | <loginCommand>
+                   | <debugCommand>
+                   | <helpCommand>
+                   | <tracingCommand>
+                   | <expandCommand>
+                   | <exitCommand>
+                   | <pagingCommand>
+                   | <clearCommand>
+                   ;
+
+<describeCommand> ::= ( "DESCRIBE" | "DESC" )
+                                  ( "FUNCTIONS"
+                                  | "FUNCTION" udf=<anyFunctionName>
+                                  | "AGGREGATES"
+                                  | "AGGREGATE" uda=<userAggregateName>
+                                  | "KEYSPACES"
+                                  | "KEYSPACE" ksname=<keyspaceName>?
+                                  | ( "COLUMNFAMILY" | "TABLE" ) cf=<columnFamilyName>
+                                  | "INDEX" idx=<indexName>
+                                  | ( "COLUMNFAMILIES" | "TABLES" )
+                                  | "FULL"? "SCHEMA"
+                                  | "CLUSTER"
+                                  | "TYPES"
+                                  | "TYPE" ut=<userTypeName>
+                                  | (ksname=<keyspaceName> | cf=<columnFamilyName> | idx=<indexName>))
+                    ;
+
+<consistencyCommand> ::= "CONSISTENCY" ( level=<consistencyLevel> )?
+                       ;
+
+<consistencyLevel> ::= "ANY"
+                     | "ONE"
+                     | "TWO"
+                     | "THREE"
+                     | "QUORUM"
+                     | "ALL"
+                     | "LOCAL_QUORUM"
+                     | "EACH_QUORUM"
+                     | "SERIAL"
+                     | "LOCAL_SERIAL"
+                     | "LOCAL_ONE"
+                     ;
+
+<serialConsistencyCommand> ::= "SERIAL" "CONSISTENCY" ( level=<serialConsistencyLevel> )?
+                             ;
+
+<serialConsistencyLevel> ::= "SERIAL"
+                           | "LOCAL_SERIAL"
+                           ;
+
+<showCommand> ::= "SHOW" what=( "VERSION" | "HOST" | "SESSION" sessionid=<uuid> )
+                ;
+
+<sourceCommand> ::= "SOURCE" fname=<stringLiteral>
+                  ;
+
+<captureCommand> ::= "CAPTURE" ( fname=( <stringLiteral> | "OFF" ) )?
+                   ;
+
+<copyCommand> ::= "COPY" cf=<columnFamilyName>
+                         ( "(" [colnames]=<colname> ( "," [colnames]=<colname> )* ")" )?
+                         ( dir="FROM" ( fname=<stringLiteral> | "STDIN" )
+                         | dir="TO"   ( fname=<stringLiteral> | "STDOUT" ) )
+                         ( "WITH" <copyOption> ( "AND" <copyOption> )* )?
+                ;
+
+<copyOption> ::= [optnames]=(<identifier>|<reserved_identifier>) "=" [optvals]=<copyOptionVal>
+               ;
+
+<copyOptionVal> ::= <identifier>
+                  | <reserved_identifier>
+                  | <term>
+                  ;
+
+# avoiding just "DEBUG" so that this rule doesn't get treated as a terminal
+<debugCommand> ::= "DEBUG" "THINGS"?
+                 ;
+
+<helpCommand> ::= ( "HELP" | "?" ) [topic]=( /[a-z_]*/ )*
+                ;
+
+<tracingCommand> ::= "TRACING" ( switch=( "ON" | "OFF" ) )?
+                   ;
+
+<expandCommand> ::= "EXPAND" ( switch=( "ON" | "OFF" ) )?
+                   ;
+
+<pagingCommand> ::= "PAGING" ( switch=( "ON" | "OFF" | /[0-9]+/) )?
+                  ;
+
+<loginCommand> ::= "LOGIN" username=<username> (password=<stringLiteral>)?
+                 ;
+
+<exitCommand> ::= "exit" | "quit"
+                ;
+
+<clearCommand> ::= "CLEAR" | "CLS"
+                 ;
+
+<qmark> ::= "?" ;
+'''
+
+
+@cqlsh_syntax_completer('helpCommand', 'topic')
+def complete_help(ctxt, cqlsh):
+    return sorted([t.upper() for t in cqldocs.get_help_topics() + cqlsh.get_help_topics()])
+
+
+def complete_source_quoted_filename(ctxt, cqlsh):
+    partial_path = ctxt.get_binding('partial', '')
+    head, tail = os.path.split(partial_path)
+    exhead = os.path.expanduser(head)
+    try:
+        contents = os.listdir(exhead or '.')
+    except OSError:
+        return ()
+    matches = filter(lambda f: f.startswith(tail), contents)
+    annotated = []
+    for f in matches:
+        match = os.path.join(head, f)
+        if os.path.isdir(os.path.join(exhead, f)):
+            match += '/'
+        annotated.append(match)
+    return annotated
+
+
+cqlsh_syntax_completer('sourceCommand', 'fname')(complete_source_quoted_filename)
+cqlsh_syntax_completer('captureCommand', 'fname')(complete_source_quoted_filename)
+
+
+@cqlsh_syntax_completer('copyCommand', 'fname')
+def copy_fname_completer(ctxt, cqlsh):
+    lasttype = ctxt.get_binding('*LASTTYPE*')
+    if lasttype == 'unclosedString':
+        return complete_source_quoted_filename(ctxt, cqlsh)
+    partial_path = ctxt.get_binding('partial')
+    if partial_path == '':
+        return ["'"]
+    return ()
+
+
+@cqlsh_syntax_completer('copyCommand', 'colnames')
+def complete_copy_column_names(ctxt, cqlsh):
+    existcols = map(cqlsh.cql_unprotect_name, ctxt.get_binding('colnames', ()))
+    ks = cqlsh.cql_unprotect_name(ctxt.get_binding('ksname', None))
+    cf = cqlsh.cql_unprotect_name(ctxt.get_binding('cfname'))
+    colnames = cqlsh.get_column_names(ks, cf)
+    if len(existcols) == 0:
+        return [colnames[0]]
+    return set(colnames[1:]) - set(existcols)
+
+
+COPY_COMMON_OPTIONS = ['DELIMITER', 'QUOTE', 'ESCAPE', 'HEADER', 'NULL', 'DATETIMEFORMAT',
+                       'MAXATTEMPTS', 'REPORTFREQUENCY', 'DECIMALSEP', 'THOUSANDSSEP', 'BOOLSTYLE',
+                       'NUMPROCESSES', 'CONFIGFILE', 'RATEFILE']
+COPY_FROM_OPTIONS = ['CHUNKSIZE', 'INGESTRATE', 'MAXBATCHSIZE', 'MINBATCHSIZE', 'MAXROWS',
+                     'SKIPROWS', 'SKIPCOLS', 'MAXPARSEERRORS', 'MAXINSERTERRORS', 'ERRFILE', 'PREPAREDSTATEMENTS']
+COPY_TO_OPTIONS = ['ENCODING', 'PAGESIZE', 'PAGETIMEOUT', 'BEGINTOKEN', 'ENDTOKEN', 'MAXOUTPUTSIZE', 'MAXREQUESTS']
+
+
+@cqlsh_syntax_completer('copyOption', 'optnames')
+def complete_copy_options(ctxt, cqlsh):
+    optnames = map(str.upper, ctxt.get_binding('optnames', ()))
+    direction = ctxt.get_binding('dir').upper()
+    if direction == 'FROM':
+        opts = set(COPY_COMMON_OPTIONS + COPY_FROM_OPTIONS) - set(optnames)
+    elif direction == 'TO':
+        opts = set(COPY_COMMON_OPTIONS + COPY_TO_OPTIONS) - set(optnames)
+    return opts
+
+
+@cqlsh_syntax_completer('copyOption', 'optvals')
+def complete_copy_opt_values(ctxt, cqlsh):
+    optnames = ctxt.get_binding('optnames', ())
+    lastopt = optnames[-1].lower()
+    if lastopt == 'header':
+        return ['true', 'false']
+    return [cqlhandling.Hint('<single_character_string>')]
+
+
+class NoKeyspaceError(Exception):
+    pass
+
+
+class KeyspaceNotFound(Exception):
+    pass
+
+
+class ColumnFamilyNotFound(Exception):
+    pass
+
+
+class IndexNotFound(Exception):
+    pass
+
+
+class ObjectNotFound(Exception):
+    pass
+
+
+class VersionNotSupported(Exception):
+    pass
+
+
+class UserTypeNotFound(Exception):
+    pass
+
+
+class FunctionNotFound(Exception):
+    pass
+
+
+class AggregateNotFound(Exception):
+    pass
+
+
+class DecodeError(Exception):
+    verb = 'decode'
+
+    def __init__(self, thebytes, err, colname=None):
+        self.thebytes = thebytes
+        self.err = err
+        self.colname = colname
+
+    def __str__(self):
+        return str(self.thebytes)
+
+    def message(self):
+        what = 'value %r' % (self.thebytes,)
+        if self.colname is not None:
+            what = 'value %r (for column %r)' % (self.thebytes, self.colname)
+        return 'Failed to %s %s : %s' \
+               % (self.verb, what, self.err)
+
+    def __repr__(self):
+        return '<%s %s>' % (self.__class__.__name__, self.message())
+
+
+class FormatError(DecodeError):
+    verb = 'format'
+
+
+def full_cql_version(ver):
+    while ver.count('.') < 2:
+        ver += '.0'
+    ver_parts = ver.split('-', 1) + ['']
+    vertuple = tuple(map(int, ver_parts[0].split('.')) + [ver_parts[1]])
+    return ver, vertuple
+
+
+def format_value(val, output_encoding, addcolor=False, date_time_format=None,
+                 float_precision=None, colormap=None, nullval=None):
+    if isinstance(val, DecodeError):
+        if addcolor:
+            return colorme(repr(val.thebytes), colormap, 'error')
+        else:
+            return FormattedValue(repr(val.thebytes))
+    return format_by_type(type(val), val, output_encoding, colormap=colormap,
+                          addcolor=addcolor, nullval=nullval, date_time_format=date_time_format,
+                          float_precision=float_precision)
+
+
+def show_warning_without_quoting_line(message, category, filename, lineno, file=None, line=None):
+    if file is None:
+        file = sys.stderr
+    try:
+        file.write(warnings.formatwarning(message, category, filename, lineno, line=''))
+    except IOError:
+        pass
+warnings.showwarning = show_warning_without_quoting_line
+warnings.filterwarnings('always', category=cql3handling.UnexpectedTableStructure)
+
+
+def insert_driver_hooks():
+    extend_cql_deserialization()
+    auto_format_udts()
+
+
+def extend_cql_deserialization():
+    """
+    The python driver returns BLOBs as string, but we expect them as bytearrays; therefore we change
+    the implementation of cassandra.cqltypes.BytesType.deserialize.
+
+    The deserializers package exists only when the driver has been compiled with cython extensions and
+    cassandra.deserializers.DesBytesType replaces cassandra.cqltypes.BytesType.deserialize.
+
+    DesBytesTypeByteArray is a fast deserializer that converts blobs into bytearrays but it was
+    only introduced recently (3.1.0). If it is available we use it, otherwise we remove
+    cassandra.deserializers.DesBytesType so that we fall back onto cassandra.cqltypes.BytesType.deserialize
+    just like in the case where no cython extensions are present.
+    """
+    if hasattr(cassandra, 'deserializers'):
+        if hasattr(cassandra.deserializers, 'DesBytesTypeByteArray'):
+            cassandra.deserializers.DesBytesType = cassandra.deserializers.DesBytesTypeByteArray
+        else:
+            del cassandra.deserializers.DesBytesType
+
+    cassandra.cqltypes.BytesType.deserialize = staticmethod(lambda byts, protocol_version: bytearray(byts))
+    cassandra.cqltypes.CassandraType.support_empty_values = True
+
+
+def auto_format_udts():
+    # when we see a new user defined type, set up the shell formatting for it
+    udt_apply_params = cassandra.cqltypes.UserType.apply_parameters
+
+    def new_apply_params(cls, *args, **kwargs):
+        udt_class = udt_apply_params(*args, **kwargs)
+        formatter_for(udt_class.typename)(format_value_utype)
+        return udt_class
+
+    cassandra.cqltypes.UserType.udt_apply_parameters = classmethod(new_apply_params)
+
+    make_udt_class = cassandra.cqltypes.UserType.make_udt_class
+
+    def new_make_udt_class(cls, *args, **kwargs):
+        udt_class = make_udt_class(*args, **kwargs)
+        formatter_for(udt_class.tuple_type.__name__)(format_value_utype)
+        return udt_class
+
+    cassandra.cqltypes.UserType.make_udt_class = classmethod(new_make_udt_class)
+
+
+class FrozenType(cassandra.cqltypes._ParameterizedType):
+    """
+    Needed until the bundled python driver adds FrozenType.
+    """
+    typename = "frozen"
+    num_subtypes = 1
+
+    @classmethod
+    def deserialize_safe(cls, byts, protocol_version):
+        subtype, = cls.subtypes
+        return subtype.from_binary(byts)
+
+    @classmethod
+    def serialize_safe(cls, val, protocol_version):
+        subtype, = cls.subtypes
+        return subtype.to_binary(val, protocol_version)
+
+
+class Shell(cmd.Cmd):
+    custom_prompt = os.getenv('CQLSH_PROMPT', '')
+    if custom_prompt is not '':
+        custom_prompt += "\n"
+    default_prompt = custom_prompt + "cqlsh> "
+    continue_prompt = "   ... "
+    keyspace_prompt = custom_prompt + "cqlsh:%s> "
+    keyspace_continue_prompt = "%s    ... "
+    show_line_nums = False
+    debug = False
+    stop = False
+    last_hist = None
+    shunted_query_out = None
+    use_paging = True
+
+    default_page_size = 100
+
+    def __init__(self, hostname, port, color=False,
+                 username=None, password=None, encoding=None, stdin=None, tty=True,
+                 completekey=DEFAULT_COMPLETEKEY, browser=None, use_conn=None,
+                 cqlver=DEFAULT_CQLVER, keyspace=None,
+                 tracing_enabled=False, expand_enabled=False,
+                 display_nanotime_format=DEFAULT_NANOTIME_FORMAT,
+                 display_timestamp_format=DEFAULT_TIMESTAMP_FORMAT,
+                 display_date_format=DEFAULT_DATE_FORMAT,
+                 display_float_precision=DEFAULT_FLOAT_PRECISION,
+                 display_timezone=None,
+                 max_trace_wait=DEFAULT_MAX_TRACE_WAIT,
+                 ssl=False,
+                 single_statement=None,
+                 request_timeout=DEFAULT_REQUEST_TIMEOUT_SECONDS,
+                 protocol_version=DEFAULT_PROTOCOL_VERSION,
+                 connect_timeout=DEFAULT_CONNECT_TIMEOUT_SECONDS):
+        cmd.Cmd.__init__(self, completekey=completekey)
+        self.hostname = hostname
+        self.port = port
+        self.auth_provider = None
+        if username:
+            if not password:
+                password = getpass.getpass()
+            self.auth_provider = PlainTextAuthProvider(username=username, password=password)
+        self.username = username
+        self.keyspace = keyspace
+        self.ssl = ssl
+        self.tracing_enabled = tracing_enabled
+        self.page_size = self.default_page_size
+        self.expand_enabled = expand_enabled
+        if use_conn:
+            self.conn = use_conn
+        else:
+            self.conn = Cluster(contact_points=(self.hostname,), port=self.port, cql_version=cqlver,
+                                protocol_version=protocol_version,
+                                auth_provider=self.auth_provider,
+                                ssl_options=sslhandling.ssl_settings(hostname, CONFIG_FILE) if ssl else None,
+                                load_balancing_policy=WhiteListRoundRobinPolicy([self.hostname]),
+                                control_connection_timeout=connect_timeout,
+                                connect_timeout=connect_timeout)
+        self.owns_connection = not use_conn
+        self.set_expanded_cql_version(cqlver)
+
+        if keyspace:
+            self.session = self.conn.connect(keyspace)
+        else:
+            self.session = self.conn.connect()
+
+        if browser == "":
+            browser = None
+        self.browser = browser
+        self.color = color
+
+        self.display_nanotime_format = display_nanotime_format
+        self.display_timestamp_format = display_timestamp_format
+        self.display_date_format = display_date_format
+
+        self.display_float_precision = display_float_precision
+
+        self.display_timezone = display_timezone
+
+        self.session.default_timeout = request_timeout
+        self.session.row_factory = ordered_dict_factory
+        self.session.default_consistency_level = cassandra.ConsistencyLevel.ONE
+        self.get_connection_versions()
+
+        self.current_keyspace = keyspace
+
+        self.display_timestamp_format = display_timestamp_format
+        self.display_nanotime_format = display_nanotime_format
+        self.display_date_format = display_date_format
+
+        self.max_trace_wait = max_trace_wait
+        self.session.max_trace_wait = max_trace_wait
+
+        self.tty = tty
+        self.encoding = encoding
+        self.check_windows_encoding()
+
+        self.output_codec = codecs.lookup(encoding)
+
+        self.statement = StringIO()
+        self.lineno = 1
+        self.in_comment = False
+
+        self.prompt = ''
+        if stdin is None:
+            stdin = sys.stdin
+
+        if tty:
+            self.reset_prompt()
+            self.report_connection()
+            print 'Use HELP for help.'
+        else:
+            self.show_line_nums = True
+        self.stdin = stdin
+        self.query_out = sys.stdout
+        self.consistency_level = cassandra.ConsistencyLevel.ONE
+        self.serial_consistency_level = cassandra.ConsistencyLevel.SERIAL
+
+        self.empty_lines = 0
+        self.statement_error = False
+        self.single_statement = single_statement
+
+    @property
+    def is_using_utf8(self):
+        # utf8 encodings from https://docs.python.org/{2,3}/library/codecs.html
+        return self.encoding.replace('-', '_').lower() in ['utf', 'utf_8', 'u8', 'utf8', CP65001]
+
+    def check_windows_encoding(self):
+        if is_win and os.name == 'nt' and self.tty and \
+           self.is_using_utf8 and sys.stdout.encoding != CP65001:
+            self.printerr("\nWARNING: console codepage must be set to cp65001 "
+                          "to support {} encoding on Windows platforms.\n"
+                          "If you experience encoding problems, change your console"
+                          " codepage with 'chcp 65001' before starting cqlsh.\n".format(self.encoding))
+
+    def set_expanded_cql_version(self, ver):
+        ver, vertuple = full_cql_version(ver)
+        self.cql_version = ver
+        self.cql_ver_tuple = vertuple
+
+    def cqlver_atleast(self, major, minor=0, patch=0):
+        return self.cql_ver_tuple[:3] >= (major, minor, patch)
+
+    def myformat_value(self, val, **kwargs):
+        if isinstance(val, DecodeError):
+            self.decoding_errors.append(val)
+        try:
+            dtformats = DateTimeFormat(timestamp_format=self.display_timestamp_format,
+                                       date_format=self.display_date_format, nanotime_format=self.display_nanotime_format,
+                                       timezone=self.display_timezone)
+            return format_value(val, self.output_codec.name,
+                                addcolor=self.color, date_time_format=dtformats,
+                                float_precision=self.display_float_precision, **kwargs)
+        except Exception, e:
+            err = FormatError(val, e)
+            self.decoding_errors.append(err)
+            return format_value(err, self.output_codec.name, addcolor=self.color)
+
+    def myformat_colname(self, name, table_meta=None):
+        column_colors = COLUMN_NAME_COLORS.copy()
+        # check column role and color appropriately
+        if table_meta:
+            if name in [col.name for col in table_meta.partition_key]:
+                column_colors.default_factory = lambda: RED
+            elif name in [col.name for col in table_meta.clustering_key]:
+                column_colors.default_factory = lambda: CYAN
+        return self.myformat_value(name, colormap=column_colors)
+
+    def report_connection(self):
+        self.show_host()
+        self.show_version()
+
+    def show_host(self):
+        print "Connected to %s at %s:%d." % \
+            (self.applycolor(self.get_cluster_name(), BLUE),
+              self.hostname,
+              self.port)
+
+    def show_version(self):
+        vers = self.connection_versions.copy()
+        vers['shver'] = version
+        # system.Versions['cql'] apparently does not reflect changes with
+        # set_cql_version.
+        vers['cql'] = self.cql_version
+        print "[cqlsh %(shver)s | Cassandra %(build)s | CQL spec %(cql)s | Native protocol v%(protocol)s]" % vers
+
+    def show_session(self, sessionid, partial_session=False):
+        print_trace_session(self, self.session, sessionid, partial_session)
+
+    def get_connection_versions(self):
+        result, = self.session.execute("select * from system.local where key = 'local'")
+        vers = {
+            'build': result['release_version'],
+            'protocol': result['native_protocol_version'],
+            'cql': result['cql_version'],
+        }
+        self.connection_versions = vers
+
+    def get_keyspace_names(self):
+        return map(str, self.conn.metadata.keyspaces.keys())
+
+    def get_columnfamily_names(self, ksname=None):
+        if ksname is None:
+            ksname = self.current_keyspace
+
+        return map(str, self.get_keyspace_meta(ksname).tables.keys())
+
+    def get_index_names(self, ksname=None):
+        if ksname is None:
+            ksname = self.current_keyspace
+
+        return map(str, self.get_keyspace_meta(ksname).indexes.keys())
+
+    def get_column_names(self, ksname, cfname):
+        if ksname is None:
+            ksname = self.current_keyspace
+        layout = self.get_table_meta(ksname, cfname)
+        return [unicode(col) for col in layout.columns]
+
+    def get_usertype_names(self, ksname=None):
+        if ksname is None:
+            ksname = self.current_keyspace
+
+        return self.get_keyspace_meta(ksname).user_types.keys()
+
+    def get_usertype_layout(self, ksname, typename):
+        if ksname is None:
+            ksname = self.current_keyspace
+
+        ks_meta = self.get_keyspace_meta(ksname)
+
+        try:
+            user_type = ks_meta.user_types[typename]
+        except KeyError:
+            raise UserTypeNotFound("User type %r not found" % typename)
+
+        return [(field_name, field_type.cql_parameterized_type())
+                for field_name, field_type in zip(user_type.field_names, user_type.field_types)]
+
+    def get_userfunction_names(self, ksname=None):
+        if ksname is None:
+            ksname = self.current_keyspace
+
+        return map(lambda f: f.name, self.get_keyspace_meta(ksname).functions.values())
+
+    def get_useraggregate_names(self, ksname=None):
+        if ksname is None:
+            ksname = self.current_keyspace
+
+        return map(lambda f: f.name, self.get_keyspace_meta(ksname).aggregates.values())
+
+    def get_cluster_name(self):
+        return self.conn.metadata.cluster_name
+
+    def get_partitioner(self):
+        return self.conn.metadata.partitioner
+
+    def get_keyspace_meta(self, ksname):
+        if ksname not in self.conn.metadata.keyspaces:
+            raise KeyspaceNotFound('Keyspace %r not found.' % ksname)
+        return self.conn.metadata.keyspaces[ksname]
+
+    def get_keyspaces(self):
+        return self.conn.metadata.keyspaces.values()
+
+    def get_ring(self, ks):
+        self.conn.metadata.token_map.rebuild_keyspace(ks, build_if_absent=True)
+        return self.conn.metadata.token_map.tokens_to_hosts_by_ks[ks]
+
+    def get_table_meta(self, ksname, tablename):
+        if ksname is None:
+            ksname = self.current_keyspace
+        ksmeta = self.get_keyspace_meta(ksname)
+
+        if tablename not in ksmeta.tables:
+            if ksname == 'system_auth' and tablename in ['roles', 'role_permissions']:
+                self.get_fake_auth_table_meta(ksname, tablename)
+            else:
+                raise ColumnFamilyNotFound("Column family %r not found" % tablename)
+        else:
+            return ksmeta.tables[tablename]
+
+    def get_fake_auth_table_meta(self, ksname, tablename):
+        # may be using external auth implementation so internal tables
+        # aren't actually defined in schema. In this case, we'll fake
+        # them up
+        if tablename == 'roles':
+            ks_meta = KeyspaceMetadata(ksname, True, None, None)
+            table_meta = TableMetadata(ks_meta, 'roles')
+            table_meta.columns['role'] = ColumnMetadata(table_meta, 'role', cassandra.cqltypes.UTF8Type)
+            table_meta.columns['is_superuser'] = ColumnMetadata(table_meta, 'is_superuser', cassandra.cqltypes.BooleanType)
+            table_meta.columns['can_login'] = ColumnMetadata(table_meta, 'can_login', cassandra.cqltypes.BooleanType)
+        elif tablename == 'role_permissions':
+            ks_meta = KeyspaceMetadata(ksname, True, None, None)
+            table_meta = TableMetadata(ks_meta, 'role_permissions')
+            table_meta.columns['role'] = ColumnMetadata(table_meta, 'role', cassandra.cqltypes.UTF8Type)
+            table_meta.columns['resource'] = ColumnMetadata(table_meta, 'resource', cassandra.cqltypes.UTF8Type)
+            table_meta.columns['permission'] = ColumnMetadata(table_meta, 'permission', cassandra.cqltypes.UTF8Type)
+        else:
+            raise ColumnFamilyNotFound("Column family %r not found" % tablename)
+
+    def get_index_meta(self, ksname, idxname):
+        if ksname is None:
+            ksname = self.current_keyspace
+        ksmeta = self.get_keyspace_meta(ksname)
+
+        if idxname not in ksmeta.indexes:
+            raise IndexNotFound("Index %r not found" % idxname)
+
+        return ksmeta.indexes[idxname]
+
+    def get_object_meta(self, ks, name):
+        if name is None:
+            if ks and ks in self.conn.metadata.keyspaces:
+                return self.conn.metadata.keyspaces[ks]
+            elif self.current_keyspace is None:
+                raise ObjectNotFound("%r not found in keyspaces" % (ks))
+            else:
+                name = ks
+                ks = self.current_keyspace
+
+        if ks is None:
+            ks = self.current_keyspace
+
+        ksmeta = self.get_keyspace_meta(ks)
+
+        if name in ksmeta.tables:
+            return ksmeta.tables[name]
+        elif name in ksmeta.indexes:
+            return ksmeta.indexes[name]
+
+        raise ObjectNotFound("%r not found in keyspace %r" % (name, ks))
+
+    def get_usertypes_meta(self):
+        data = self.session.execute("select * from system.schema_usertypes")
+        if not data:
+            return cql3handling.UserTypesMeta({})
+
+        return cql3handling.UserTypesMeta.from_layout(data)
+
+    def get_trigger_names(self, ksname=None):
+        if ksname is None:
+            ksname = self.current_keyspace
+
+        return [trigger.name
+                for table in self.get_keyspace_meta(ksname).tables.values()
+                for trigger in table.triggers.values()]
+
+    def reset_statement(self):
+        self.reset_prompt()
+        self.statement.truncate(0)
+        self.empty_lines = 0
+
+    def reset_prompt(self):
+        if self.current_keyspace is None:
+            self.set_prompt(self.default_prompt, True)
+        else:
+            self.set_prompt(self.keyspace_prompt % self.current_keyspace, True)
+
+    def set_continue_prompt(self):
+        if self.empty_lines >= 3:
+            self.set_prompt("Statements are terminated with a ';'.  You can press CTRL-C to cancel an incomplete statement.")
+            self.empty_lines = 0
+            return
+        if self.current_keyspace is None:
+            self.set_prompt(self.continue_prompt)
+        else:
+            spaces = ' ' * len(str(self.current_keyspace))
+            self.set_prompt(self.keyspace_continue_prompt % spaces)
+        self.empty_lines = self.empty_lines + 1 if not self.lastcmd else 0
+
+    @contextmanager
+    def prepare_loop(self):
+        readline = None
+        if self.tty and self.completekey:
+            try:
+                import readline
+            except ImportError:
+                if is_win:
+                    print "WARNING: pyreadline dependency missing.  Install to enable tab completion."
+                pass
+            else:
+                old_completer = readline.get_completer()
+                readline.set_completer(self.complete)
+                if readline.__doc__ is not None and 'libedit' in readline.__doc__:
+                    readline.parse_and_bind("bind -e")
+                    readline.parse_and_bind("bind '" + self.completekey + "' rl_complete")
+                    readline.parse_and_bind("bind ^R em-inc-search-prev")
+                else:
+                    readline.parse_and_bind(self.completekey + ": complete")
+        try:
+            yield
+        finally:
+            if readline is not None:
+                readline.set_completer(old_completer)
+
+    def get_input_line(self, prompt=''):
+        if self.tty:
+            try:
+                self.lastcmd = raw_input(prompt).decode(self.encoding)
+            except UnicodeDecodeError:
+                self.lastcmd = ''
+                traceback.print_exc()
+                self.check_windows_encoding()
+            line = self.lastcmd + '\n'
+        else:
+            self.lastcmd = self.stdin.readline()
+            line = self.lastcmd
+            if not len(line):
+                raise EOFError
+        self.lineno += 1
+        return line
+
+    def use_stdin_reader(self, until='', prompt=''):
+        until += '\n'
+        while True:
+            try:
+                newline = self.get_input_line(prompt=prompt)
+            except EOFError:
+                return
+            if newline == until:
+                return
+            yield newline
+
+    def cmdloop(self):
+        """
+        Adapted from cmd.Cmd's version, because there is literally no way with
+        cmd.Cmd.cmdloop() to tell the difference between "EOF" showing up in
+        input and an actual EOF.
+        """
+        with self.prepare_loop():
+            while not self.stop:
+                try:
+                    if self.single_statement:
+                        line = self.single_statement
+                        self.stop = True
+                    else:
+                        line = self.get_input_line(self.prompt)
+                    self.statement.write(line)
+                    if self.onecmd(self.statement.getvalue()):
+                        self.reset_statement()
+                except EOFError:
+                    self.handle_eof()
+                except CQL_ERRORS, cqlerr:
+                    self.printerr(cqlerr.message.decode(encoding='utf-8'))
+                except KeyboardInterrupt:
+                    self.reset_statement()
+                    print
+
+    def onecmd(self, statementtext):
+        """
+        Returns true if the statement is complete and was handled (meaning it
+        can be reset).
+        """
+
+        try:
+            statements, endtoken_escaped = cqlruleset.cql_split_statements(statementtext)
+        except pylexotron.LexingError, e:
+            if self.show_line_nums:
+                self.printerr('Invalid syntax at char %d' % (e.charnum,))
+            else:
+                self.printerr('Invalid syntax at line %d, char %d'
+                              % (e.linenum, e.charnum))
+            statementline = statementtext.split('\n')[e.linenum - 1]
+            self.printerr('  %s' % statementline)
+            self.printerr(' %s^' % (' ' * e.charnum))
+            return True
+
+        while statements and not statements[-1]:
+            statements = statements[:-1]
+        if not statements:
+            return True
+        if endtoken_escaped or statements[-1][-1][0] != 'endtoken':
+            self.set_continue_prompt()
+            return
+        for st in statements:
+            try:
+                self.handle_statement(st, statementtext)
+            except Exception, e:
+                if self.debug:
+                    traceback.print_exc()
+                else:
+                    self.printerr(e)
+        return True
+
+    def handle_eof(self):
+        if self.tty:
+            print
+        statement = self.statement.getvalue()
+        if statement.strip():
+            if not self.onecmd(statement):
+                self.printerr('Incomplete statement at end of file')
+        self.do_exit()
+
+    def handle_statement(self, tokens, srcstr):
+        # Concat multi-line statements and insert into history
+        if readline is not None:
+            nl_count = srcstr.count("\n")
+
+            new_hist = srcstr.replace("\n", " ").rstrip()
+
+            if nl_count > 1 and self.last_hist != new_hist:
+                readline.add_history(new_hist.encode(self.encoding))
+
+            self.last_hist = new_hist
+        cmdword = tokens[0][1]
+        if cmdword == '?':
+            cmdword = 'help'
+        custom_handler = getattr(self, 'do_' + cmdword.lower(), None)
+        if custom_handler:
+            parsed = cqlruleset.cql_whole_parse_tokens(tokens, srcstr=srcstr,
+                                                       startsymbol='cqlshCommand')
+            if parsed and not parsed.remainder:
+                # successful complete parse
+                return custom_handler(parsed)
+            else:
+                return self.handle_parse_error(cmdword, tokens, parsed, srcstr)
+        return self.perform_statement(cqlruleset.cql_extract_orig(tokens, srcstr))
+
+    def handle_parse_error(self, cmdword, tokens, parsed, srcstr):
+        if cmdword.lower() in ('select', 'insert', 'update', 'delete', 'truncate',
+                               'create', 'drop', 'alter', 'grant', 'revoke',
+                               'batch', 'list'):
+            # hey, maybe they know about some new syntax we don't. type
+            # assumptions won't work, but maybe the query will.
+            return self.perform_statement(cqlruleset.cql_extract_orig(tokens, srcstr))
+        if parsed:
+            self.printerr('Improper %s command (problem at %r).' % (cmdword, parsed.remainder[0]))
+        else:
+            self.printerr('Improper %s command.' % cmdword)
+
+    def do_use(self, parsed):
+        ksname = parsed.get_binding('ksname')
+        success, _ = self.perform_simple_statement(SimpleStatement(parsed.extract_orig()))
+        if success:
+            if ksname[0] == '"' and ksname[-1] == '"':
+                self.current_keyspace = self.cql_unprotect_name(ksname)
+            else:
+                self.current_keyspace = ksname.lower()
+
+    def do_select(self, parsed):
+        tracing_was_enabled = self.tracing_enabled
+        ksname = parsed.get_binding('ksname')
+        stop_tracing = ksname == 'system_traces' or (ksname is None and self.current_keyspace == 'system_traces')
+        self.tracing_enabled = self.tracing_enabled and not stop_tracing
+        statement = parsed.extract_orig()
+        self.perform_statement(statement)
+        self.tracing_enabled = tracing_was_enabled
+
+    def perform_statement(self, statement):
+        stmt = SimpleStatement(statement, consistency_level=self.consistency_level, serial_consistency_level=self.serial_consistency_level, fetch_size=self.page_size if self.use_paging else None)
+        success, future = self.perform_simple_statement(stmt)
+
+        if future:
+            if future.warnings:
+                self.print_warnings(future.warnings)
+
+            if self.tracing_enabled:
+                try:
+                    for trace in future.get_all_query_traces(max_wait_per=self.max_trace_wait, query_cl=self.consistency_level):
+                        print_trace(self, trace)
+                except TraceUnavailable:
+                    msg = "Statement trace did not complete within %d seconds; trace data may be incomplete." % (self.session.max_trace_wait,)
+                    self.writeresult(msg, color=RED)
+                    for trace_id in future.get_query_trace_ids():
+                        self.show_session(trace_id, partial_session=True)
+                except Exception, err:
+                    self.printerr("Unable to fetch query trace: %s" % (str(err),))
+
+        return success
+
+    def parse_for_table_meta(self, query_string):
+        try:
+            parsed = cqlruleset.cql_parse(query_string)[1]
+        except IndexError:
+            return None
+        ks = self.cql_unprotect_name(parsed.get_binding('ksname', None))
+        cf = self.cql_unprotect_name(parsed.get_binding('cfname'))
+        return self.get_table_meta(ks, cf)
+
+    def perform_simple_statement(self, statement):
+        if not statement:
+            return False, None
+
+        future = self.session.execute_async(statement, trace=self.tracing_enabled)
+        result = None
+        try:
+            result = future.result()
+        except CQL_ERRORS, err:
+            self.printerr(unicode(err.__class__.__name__) + u": " + err.message.decode(encoding='utf-8'))
+        except Exception:
+            import traceback
+            self.printerr(traceback.format_exc())
+
+        # Even if statement failed we try to refresh schema if not agreed (see CASSANDRA-9689)
+        if not future.is_schema_agreed:
+            try:
+                self.conn.refresh_schema_metadata(5)  # will throw exception if there is a schema mismatch
+            except Exception:
+                self.printerr("Warning: schema version mismatch detected; check the schema versions of your "
+                              "nodes in system.local and system.peers.")
+                self.conn.refresh_schema_metadata(-1)
+
+        if result is None:
+            return False, None
+
+        if statement.query_string[:6].lower() == 'select':
+            self.print_result(result, self.parse_for_table_meta(statement.query_string))
+        elif statement.query_string.lower().startswith("list users") or statement.query_string.lower().startswith("list roles"):
+            self.print_result(result, self.get_table_meta('system_auth', 'roles'))
+        elif statement.query_string.lower().startswith("list"):
+            self.print_result(result, self.get_table_meta('system_auth', 'role_permissions'))
+        elif result:
+            # CAS INSERT/UPDATE
+            self.writeresult("")
+            self.print_static_result(result.column_names, list(result), self.parse_for_table_meta(statement.query_string))
+        self.flush_output()
+        return True, future
+
+    def print_result(self, result, table_meta):
+        self.decoding_errors = []
+
+        self.writeresult("")
+        if result.has_more_pages and self.tty:
+            num_rows = 0
+            while True:
+                page = result.current_rows
+                if page:
+                    num_rows += len(page)
+                    self.print_static_result(result.column_names, page, table_meta)
+                if result.has_more_pages:
+                    raw_input("---MORE---")
+                    result.fetch_next_page()
+                else:
+                    break
+        else:
+            rows = list(result)
+            num_rows = len(rows)
+            self.print_static_result(result.column_names, rows, table_meta)
+        self.writeresult("(%d rows)" % num_rows)
+
+        if self.decoding_errors:
+            for err in self.decoding_errors[:2]:
+                self.writeresult(err.message(), color=RED)
+            if len(self.decoding_errors) > 2:
+                self.writeresult('%d more decoding errors suppressed.'
+                                 % (len(self.decoding_errors) - 2), color=RED)
+
+    def print_static_result(self, column_names, rows, table_meta):
+        if not column_names and not table_meta:
+            return
+
+        column_names = column_names or table_meta.columns.keys()
+        formatted_names = [self.myformat_colname(name, table_meta) for name in column_names]
+        if not rows:
+            # print header only
+            self.print_formatted_result(formatted_names, None)
+            return
+        formatted_values = [map(self.myformat_value, row.values()) for row in rows]
+
+        if self.expand_enabled:
+            self.print_formatted_result_vertically(formatted_names, formatted_values)
+        else:
+            self.print_formatted_result(formatted_names, formatted_values)
+
+    def print_formatted_result(self, formatted_names, formatted_values):
+        # determine column widths
+        widths = [n.displaywidth for n in formatted_names]
+        if formatted_values is not None:
+            for fmtrow in formatted_values:
+                for num, col in enumerate(fmtrow):
+                    widths[num] = max(widths[num], col.displaywidth)
+
+        # print header
+        header = ' | '.join(hdr.ljust(w, color=self.color) for (hdr, w) in zip(formatted_names, widths))
+        self.writeresult(' ' + header.rstrip())
+        self.writeresult('-%s-' % '-+-'.join('-' * w for w in widths))
+
+        # stop if there are no rows
+        if formatted_values is None:
+            self.writeresult("")
+            return
+
+        # print row data
+        for row in formatted_values:
+            line = ' | '.join(col.rjust(w, color=self.color) for (col, w) in zip(row, widths))
+            self.writeresult(' ' + line)
+
+        self.writeresult("")
+
+    def print_formatted_result_vertically(self, formatted_names, formatted_values):
+        max_col_width = max([n.displaywidth for n in formatted_names])
+        max_val_width = max([n.displaywidth for row in formatted_values for n in row])
+
+        # for each row returned, list all the column-value pairs
+        for row_id, row in enumerate(formatted_values):
+            self.writeresult("@ Row %d" % (row_id + 1))
+            self.writeresult('-%s-' % '-+-'.join(['-' * max_col_width, '-' * max_val_width]))
+            for field_id, field in enumerate(row):
+                column = formatted_names[field_id].ljust(max_col_width, color=self.color)
+                value = field.ljust(field.displaywidth, color=self.color)
+                self.writeresult(' ' + " | ".join([column, value]))
+            self.writeresult('')
+
+    def print_warnings(self, warnings):
+        if warnings is None or len(warnings) == 0:
+            return
+
+        self.writeresult('')
+        self.writeresult('Warnings :')
+        for warning in warnings:
+            self.writeresult(warning)
+            self.writeresult('')
+
+    def emptyline(self):
+        pass
+
+    def parseline(self, line):
+        # this shouldn't be needed
+        raise NotImplementedError
+
+    def complete(self, text, state):
+        if readline is None:
+            return
+        if state == 0:
+            try:
+                self.completion_matches = self.find_completions(text)
+            except Exception:
+                if debug_completion:
+                    import traceback
+                    traceback.print_exc()
+                else:
+                    raise
+        try:
+            return self.completion_matches[state]
+        except IndexError:
+            return None
+
+    def find_completions(self, text):
+        curline = readline.get_line_buffer()
+        prevlines = self.statement.getvalue()
+        wholestmt = prevlines + curline
+        begidx = readline.get_begidx() + len(prevlines)
+        stuff_to_complete = wholestmt[:begidx]
+        return cqlruleset.cql_complete(stuff_to_complete, text, cassandra_conn=self,
+                                       debug=debug_completion, startsymbol='cqlshCommand')
+
+    def set_prompt(self, prompt, prepend_user=False):
+        if prepend_user and self.username:
+            self.prompt = "%s@%s" % (self.username, prompt)
+            return
+        self.prompt = prompt
+
+    def cql_unprotect_name(self, namestr):
+        if namestr is None:
+            return
+        return cqlruleset.dequote_name(namestr)
+
+    def cql_unprotect_value(self, valstr):
+        if valstr is not None:
+            return cqlruleset.dequote_value(valstr)
+
+    def print_recreate_keyspace(self, ksdef, out):
+        out.write(ksdef.export_as_string())
+        out.write("\n")
+
+    def print_recreate_columnfamily(self, ksname, cfname, out):
+        """
+        Output CQL commands which should be pasteable back into a CQL session
+        to recreate the given table.
+
+        Writes output to the given out stream.
+        """
+        out.write(self.get_table_meta(ksname, cfname).export_as_string())
+        out.write("\n")
+
+    def print_recreate_index(self, ksname, idxname, out):
+        """
+        Output CQL commands which should be pasteable back into a CQL session
+        to recreate the given index.
+
+        Writes output to the given out stream.
+        """
+        out.write(self.get_index_meta(ksname, idxname).export_as_string())
+        out.write("\n")
+
+    def print_recreate_object(self, ks, name, out):
+        """
+        Output CQL commands which should be pasteable back into a CQL session
+        to recreate the given object (ks, table or index).
+
+        Writes output to the given out stream.
+        """
+        out.write(self.get_object_meta(ks, name).export_as_string())
+        out.write("\n")
+
+    def describe_keyspaces(self):
+        print
+        cmd.Cmd.columnize(self, protect_names(self.get_keyspace_names()))
+        print
+
+    def describe_keyspace(self, ksname):
+        print
+        self.print_recreate_keyspace(self.get_keyspace_meta(ksname), sys.stdout)
+        print
+
+    def describe_columnfamily(self, ksname, cfname):
+        if ksname is None:
+            ksname = self.current_keyspace
+        if ksname is None:
+            raise NoKeyspaceError("No keyspace specified and no current keyspace")
+        print
+        self.print_recreate_columnfamily(ksname, cfname, sys.stdout)
+        print
+
+    def describe_index(self, ksname, idxname):
+        print
+        self.print_recreate_index(ksname, idxname, sys.stdout)
+        print
+
+    def describe_object(self, ks, name):
+        print
+        self.print_recreate_object(ks, name, sys.stdout)
+        print
+
+    def describe_columnfamilies(self, ksname):
+        print
+        if ksname is None:
+            for k in self.get_keyspaces():
+                name = protect_name(k.name)
+                print 'Keyspace %s' % (name,)
+                print '---------%s' % ('-' * len(name))
+                cmd.Cmd.columnize(self, protect_names(self.get_columnfamily_names(k.name)))
+                print
+        else:
+            cmd.Cmd.columnize(self, protect_names(self.get_columnfamily_names(ksname)))
+            print
+
+    def describe_functions(self, ksname):
+        print
+        if ksname is None:
+            for ksmeta in self.get_keyspaces():
+                name = protect_name(ksmeta.name)
+                print 'Keyspace %s' % (name,)
+                print '---------%s' % ('-' * len(name))
+                self._columnize_unicode(ksmeta.functions.keys())
+        else:
+            ksmeta = self.get_keyspace_meta(ksname)
+            self._columnize_unicode(ksmeta.functions.keys())
+
+    def describe_function(self, ksname, functionname):
+        if ksname is None:
+            ksname = self.current_keyspace
+        if ksname is None:
+            raise NoKeyspaceError("No keyspace specified and no current keyspace")
+        print
+        ksmeta = self.get_keyspace_meta(ksname)
+        functions = filter(lambda f: f.name == functionname, ksmeta.functions.values())
+        if len(functions) == 0:
+            raise FunctionNotFound("User defined function %r not found" % functionname)
+        print "\n\n".join(func.as_cql_query(formatted=True) for func in functions)
+        print
+
+    def describe_aggregates(self, ksname):
+        print
+        if ksname is None:
+            for ksmeta in self.get_keyspaces():
+                name = protect_name(ksmeta.name)
+                print 'Keyspace %s' % (name,)
+                print '---------%s' % ('-' * len(name))
+                self._columnize_unicode(ksmeta.aggregates.keys())
+        else:
+            ksmeta = self.get_keyspace_meta(ksname)
+            self._columnize_unicode(ksmeta.aggregates.keys())
+
+    def describe_aggregate(self, ksname, aggregatename):
+        if ksname is None:
+            ksname = self.current_keyspace
+        if ksname is None:
+            raise NoKeyspaceError("No keyspace specified and no current keyspace")
+        print
+        ksmeta = self.get_keyspace_meta(ksname)
+        aggregates = filter(lambda f: f.name == aggregatename, ksmeta.aggregates.values())
+        if len(aggregates) == 0:
+            raise FunctionNotFound("User defined aggregate %r not found" % aggregatename)
+        print "\n\n".join(aggr.as_cql_query(formatted=True) for aggr in aggregates)
+        print
+
+    def describe_usertypes(self, ksname):
+        print
+        if ksname is None:
+            for ksmeta in self.get_keyspaces():
+                name = protect_name(ksmeta.name)
+                print 'Keyspace %s' % (name,)
+                print '---------%s' % ('-' * len(name))
+                self._columnize_unicode(ksmeta.user_types.keys(), quote=True)
+        else:
+            ksmeta = self.get_keyspace_meta(ksname)
+            self._columnize_unicode(ksmeta.user_types.keys(), quote=True)
+
+    def describe_usertype(self, ksname, typename):
+        if ksname is None:
+            ksname = self.current_keyspace
+        if ksname is None:
+            raise NoKeyspaceError("No keyspace specified and no current keyspace")
+        print
+        ksmeta = self.get_keyspace_meta(ksname)
+        try:
+            usertype = ksmeta.user_types[typename]
+        except KeyError:
+            raise UserTypeNotFound("User type %r not found" % typename)
+        print usertype.as_cql_query(formatted=True)
+        print
+
+    def _columnize_unicode(self, name_list, quote=False):
+        """
+        Used when columnizing identifiers that may contain unicode
+        """
+        names = [n.encode('utf-8') for n in name_list]
+        if quote:
+            names = protect_names(names)
+        cmd.Cmd.columnize(self, names)
+        print
+
+    def describe_cluster(self):
+        print '\nCluster: %s' % self.get_cluster_name()
+        p = trim_if_present(self.get_partitioner(), 'org.apache.cassandra.dht.')
+        print 'Partitioner: %s\n' % p
+        # TODO: snitch?
+        # snitch = trim_if_present(self.get_snitch(), 'org.apache.cassandra.locator.')
+        # print 'Snitch: %s\n' % snitch
+        if self.current_keyspace is not None and self.current_keyspace != 'system':
+            print "Range ownership:"
+            ring = self.get_ring(self.current_keyspace)
+            for entry in ring.items():
+                print ' %39s  [%s]' % (str(entry[0].value), ', '.join([host.address for host in entry[1]]))
+            print
+
+    def describe_schema(self, include_system=False):
+        print
+        for k in self.get_keyspaces():
+            if include_system or k.name not in cql3handling.SYSTEM_KEYSPACES:
+                self.print_recreate_keyspace(k, sys.stdout)
+                print
+
+    def do_describe(self, parsed):
+        """
+        DESCRIBE [cqlsh only]
+
+        (DESC may be used as a shorthand.)
+
+          Outputs information about the connected Cassandra cluster, or about
+          the data objects stored in the cluster. Use in one of the following ways:
+
+        DESCRIBE KEYSPACES
+
+          Output the names of all keyspaces.
+
+        DESCRIBE KEYSPACE [<keyspacename>]
+
+          Output CQL commands that could be used to recreate the given keyspace,
+          and the objects in it (such as tables, types, functions, etc.).
+          In some cases, as the CQL interface matures, there will be some metadata
+          about a keyspace that is not representable with CQL. That metadata will not be shown.
+
+          The '<keyspacename>' argument may be omitted, in which case the current
+          keyspace will be described.
+
+        DESCRIBE TABLES
+
+          Output the names of all tables in the current keyspace, or in all
+          keyspaces if there is no current keyspace.
+
+        DESCRIBE TABLE [<keyspace>.]<tablename>
+
+          Output CQL commands that could be used to recreate the given table.
+          In some cases, as above, there may be table metadata which is not
+          representable and which will not be shown.
+
+        DESCRIBE INDEX <indexname>
+
+          Output the CQL command that could be used to recreate the given index.
+          In some cases, there may be index metadata which is not representable
+          and which will not be shown.
+
+        DESCRIBE CLUSTER
+
+          Output information about the connected Cassandra cluster, such as the
+          cluster name, and the partitioner and snitch in use. When you are
+          connected to a non-system keyspace, also shows endpoint-range
+          ownership information for the Cassandra ring.
+
+        DESCRIBE [FULL] SCHEMA
+
+          Output CQL commands that could be used to recreate the entire (non-system) schema.
+          Works as though "DESCRIBE KEYSPACE k" was invoked for each non-system keyspace
+          k. Use DESCRIBE FULL SCHEMA to include the system keyspaces.
+
+        DESCRIBE TYPES
+
+          Output the names of all user-defined-types in the current keyspace, or in all
+          keyspaces if there is no current keyspace.
+
+        DESCRIBE TYPE [<keyspace>.]<type>
+
+          Output the CQL command that could be used to recreate the given user-defined-type.
+
+        DESCRIBE FUNCTIONS
+
+          Output the names of all user-defined-functions in the current keyspace, or in all
+          keyspaces if there is no current keyspace.
+
+        DESCRIBE FUNCTION [<keyspace>.]<function>
+
+          Output the CQL command that could be used to recreate the given user-defined-function.
+
+        DESCRIBE AGGREGATES
+
+          Output the names of all user-defined-aggregates in the current keyspace, or in all
+          keyspaces if there is no current keyspace.
+
+        DESCRIBE AGGREGATE [<keyspace>.]<aggregate>
+
+          Output the CQL command that could be used to recreate the given user-defined-aggregate.
+
+        DESCRIBE <objname>
+
+          Output CQL commands that could be used to recreate the entire object schema,
+          where object can be either a keyspace or a table or an index (in this order).
+  """
+        what = parsed.matched[1][1].lower()
+        if what == 'functions':
+            self.describe_functions(self.current_keyspace)
+        elif what == 'function':
+            ksname = self.cql_unprotect_name(parsed.get_binding('ksname', None))
+            functionname = self.cql_unprotect_name(parsed.get_binding('udfname'))
+            self.describe_function(ksname, functionname)
+        elif what == 'aggregates':
+            self.describe_aggregates(self.current_keyspace)
+        elif what == 'aggregate':
+            ksname = self.cql_unprotect_name(parsed.get_binding('ksname', None))
+            aggregatename = self.cql_unprotect_name(parsed.get_binding('udaname'))
+            self.describe_aggregate(ksname, aggregatename)
+        elif what == 'keyspaces':
+            self.describe_keyspaces()
+        elif what == 'keyspace':
+            ksname = self.cql_unprotect_name(parsed.get_binding('ksname', ''))
+            if not ksname:
+                ksname = self.current_keyspace
+                if ksname is None:
+                    self.printerr('Not in any keyspace.')
+                    return
+            self.describe_keyspace(ksname)
+        elif what in ('columnfamily', 'table'):
+            ks = self.cql_unprotect_name(parsed.get_binding('ksname', None))
+            cf = self.cql_unprotect_name(parsed.get_binding('cfname'))
+            self.describe_columnfamily(ks, cf)
+        elif what == 'index':
+            ks = self.cql_unprotect_name(parsed.get_binding('ksname', None))
+            idx = self.cql_unprotect_name(parsed.get_binding('idxname', None))
+            self.describe_index(ks, idx)
+        elif what in ('columnfamilies', 'tables'):
+            self.describe_columnfamilies(self.current_keyspace)
+        elif what == 'types':
+            self.describe_usertypes(self.current_keyspace)
+        elif what == 'type':
+            ks = self.cql_unprotect_name(parsed.get_binding('ksname', None))
+            ut = self.cql_unprotect_name(parsed.get_binding('utname'))
+            self.describe_usertype(ks, ut)
+        elif what == 'cluster':
+            self.describe_cluster()
+        elif what == 'schema':
+            self.describe_schema(False)
+        elif what == 'full' and parsed.matched[2][1].lower() == 'schema':
+            self.describe_schema(True)
+        elif what:
+            ks = self.cql_unprotect_name(parsed.get_binding('ksname', None))
+            name = self.cql_unprotect_name(parsed.get_binding('cfname'))
+            if not name:
+                name = self.cql_unprotect_name(parsed.get_binding('idxname', None))
+            self.describe_object(ks, name)
+    do_desc = do_describe
+
+    def do_copy(self, parsed):
+        r"""
+        COPY [cqlsh only]
+
+          COPY x FROM: Imports CSV data into a Cassandra table
+          COPY x TO: Exports data from a Cassandra table in CSV format.
+
+        COPY <table_name> [ ( column [, ...] ) ]
+             FROM ( '<file_pattern_1, file_pattern_2, ... file_pattern_n>' | STDIN )
+             [ WITH <option>='value' [AND ...] ];
+
+        File patterns are either file names or valid python glob expressions, e.g. *.csv or folder/*.csv.
+
+        COPY <table_name> [ ( column [, ...] ) ]
+             TO ( '<filename>' | STDOUT )
+             [ WITH <option>='value' [AND ...] ];
+
+        Available common COPY options and defaults:
+
+          DELIMITER=','           - character that appears between records
+          QUOTE='"'               - quoting character to be used to quote fields
+          ESCAPE='\'              - character to appear before the QUOTE char when quoted
+          HEADER=false            - whether to ignore the first line
+          NULL=''                 - string that represents a null value
+          DATETIMEFORMAT=         - timestamp strftime format
+            '%Y-%m-%d %H:%M:%S%z'   defaults to time_format value in cqlshrc
+          MAXATTEMPTS=5           - the maximum number of attempts per batch or range
+          REPORTFREQUENCY=0.25    - the frequency with which we display status updates in seconds
+          DECIMALSEP='.'          - the separator for decimal values
+          THOUSANDSSEP=''         - the separator for thousands digit groups
+          BOOLSTYLE='True,False'  - the representation for booleans, case insensitive, specify true followed by false,
+                                    for example yes,no or 1,0
+          NUMPROCESSES=n          - the number of worker processes, by default the number of cores minus one
+                                    capped at 16
+          CONFIGFILE=''           - a configuration file with the same format as .cqlshrc (see the Python ConfigParser
+                                    documentation) where you can specify WITH options under the following optional
+                                    sections: [copy], [copy-to], [copy-from], [copy:ks.table], [copy-to:ks.table],
+                                    [copy-from:ks.table], where <ks> is your keyspace name and <table> is your table
+                                    name. Options are read from these sections, in the order specified
+                                    above, and command line options always override options in configuration files.
+                                    Depending on the COPY direction, only the relevant copy-from or copy-to sections
+                                    are used. If no configfile is specified then .cqlshrc is searched instead.
+          RATEFILE=''             - an optional file where to print the output statistics
+
+        Available COPY FROM options and defaults:
+
+          CHUNKSIZE=5000          - the size of chunks passed to worker processes
+          INGESTRATE=100000       - an approximate ingest rate in rows per second
+          MINBATCHSIZE=10         - the minimum size of an import batch
+          MAXBATCHSIZE=20         - the maximum size of an import batch
+          MAXROWS=-1              - the maximum number of rows, -1 means no maximum
+          SKIPROWS=0              - the number of rows to skip
+          SKIPCOLS=''             - a comma separated list of column names to skip
+          MAXPARSEERRORS=-1       - the maximum global number of parsing errors, -1 means no maximum
+          MAXINSERTERRORS=-1      - the maximum global number of insert errors, -1 means no maximum
+          ERRFILE=''              - a file where to store all rows that could not be imported, by default this is
+                                    import_ks_table.err where <ks> is your keyspace and <table> is your table name.
+          PREPAREDSTATEMENTS=True - whether to use prepared statements when importing, by default True. Set this to
+                                    False if you don't mind shifting data parsing to the cluster. The cluster will also
+                                    have to compile every batch statement. For large and oversized clusters
+                                    this will result in a faster import but for smaller clusters it may generate
+                                    timeouts.
+
+        Available COPY TO options and defaults:
+
+          ENCODING='utf8'          - encoding for CSV output
+          PAGESIZE='1000'          - the page size for fetching results
+          PAGETIMEOUT=10           - the page timeout in seconds for fetching results
+          BEGINTOKEN=''            - the minimum token string to consider when exporting data
+          ENDTOKEN=''              - the maximum token string to consider when exporting data
+          MAXREQUESTS=6            - the maximum number of requests each worker process can work on in parallel
+          MAXOUTPUTSIZE='-1'       - the maximum size of the output file measured in number of lines,
+                                     beyond this maximum the output file will be split into segments,
+                                     -1 means unlimited.
+
+        When entering CSV data on STDIN, you can use the sequence "\."
+        on a line by itself to end the data input.
+        """
+
+        ks = self.cql_unprotect_name(parsed.get_binding('ksname', None))
+        if ks is None:
+            ks = self.current_keyspace
+            if ks is None:
+                raise NoKeyspaceError("Not in any keyspace.")
+        table = self.cql_unprotect_name(parsed.get_binding('cfname'))
+        columns = parsed.get_binding('colnames', None)
+        if columns is not None:
+            columns = map(self.cql_unprotect_name, columns)
+        else:
+            # default to all known columns
+            columns = self.get_column_names(ks, table)
+
+        fname = parsed.get_binding('fname', None)
+        if fname is not None:
+            fname = self.cql_unprotect_value(fname)
+
+        copyoptnames = map(str.lower, parsed.get_binding('optnames', ()))
+        copyoptvals = map(self.cql_unprotect_value, parsed.get_binding('optvals', ()))
+        opts = dict(zip(copyoptnames, copyoptvals))
+
+        direction = parsed.get_binding('dir').upper()
+        if direction == 'FROM':
+            task = ImportTask(self, ks, table, columns, fname, opts, DEFAULT_PROTOCOL_VERSION, CONFIG_FILE)
+        elif direction == 'TO':
+            task = ExportTask(self, ks, table, columns, fname, opts, DEFAULT_PROTOCOL_VERSION, CONFIG_FILE)
+        else:
+            raise SyntaxError("Unknown direction %s" % direction)
+
+        task.run()
+
+    def do_show(self, parsed):
+        """
+        SHOW [cqlsh only]
+
+          Displays information about the current cqlsh session. Can be called in
+          the following ways:
+
+        SHOW VERSION
+
+          Shows the version and build of the connected Cassandra instance, as
+          well as the versions of the CQL spec and the Thrift protocol that
+          the connected Cassandra instance understands.
+
+        SHOW HOST
+
+          Shows where cqlsh is currently connected.
+
+        SHOW SESSION <sessionid>
+
+          Pretty-prints the requested tracing session.
+        """
+        showwhat = parsed.get_binding('what').lower()
+        if showwhat == 'version':
+            self.get_connection_versions()
+            self.show_version()
+        elif showwhat == 'host':
+            self.show_host()
+        elif showwhat.startswith('session'):
+            session_id = parsed.get_binding('sessionid').lower()
+            self.show_session(UUID(session_id))
+        else:
+            self.printerr('Wait, how do I show %r?' % (showwhat,))
+
+    def do_source(self, parsed):
+        """
+        SOURCE [cqlsh only]
+
+        Executes a file containing CQL statements. Gives the output for each
+        statement in turn, if any, or any errors that occur along the way.
+
+        Errors do NOT abort execution of the CQL source file.
+
+        Usage:
+
+          SOURCE '<file>';
+
+        That is, the path to the file to be executed must be given inside a
+        string literal. The path is interpreted relative to the current working
+        directory. The tilde shorthand notation ('~/mydir') is supported for
+        referring to $HOME.
+
+        See also the --file option to cqlsh.
+        """
+        fname = parsed.get_binding('fname')
+        fname = os.path.expanduser(self.cql_unprotect_value(fname))
+        try:
+            encoding, bom_size = get_file_encoding_bomsize(fname)
+            f = codecs.open(fname, 'r', encoding)
+            f.seek(bom_size)
+        except IOError, e:
+            self.printerr('Could not open %r: %s' % (fname, e))
+            return
+        username = self.auth_provider.username if self.auth_provider else None
+        password = self.auth_provider.password if self.auth_provider else None
+        subshell = Shell(self.hostname, self.port, color=self.color,
+                         username=username, password=password,
+                         encoding=self.encoding, stdin=f, tty=False, use_conn=self.conn,
+                         cqlver=self.cql_version, keyspace=self.current_keyspace,
+                         tracing_enabled=self.tracing_enabled,
+                         display_nanotime_format=self.display_nanotime_format,
+                         display_timestamp_format=self.display_timestamp_format,
+                         display_date_format=self.display_date_format,
+                         display_float_precision=self.display_float_precision,
+                         display_timezone=self.display_timezone,
+                         max_trace_wait=self.max_trace_wait, ssl=self.ssl,
+                         request_timeout=self.session.default_timeout,
+                         connect_timeout=self.conn.connect_timeout)
+        subshell.cmdloop()
+        f.close()
+
+    def do_capture(self, parsed):
+        """
+        CAPTURE [cqlsh only]
+
+        Begins capturing command output and appending it to a specified file.
+        Output will not be shown at the console while it is captured.
+
+        Usage:
+
+          CAPTURE '<file>';
+          CAPTURE OFF;
+          CAPTURE;
+
+        That is, the path to the file to be appended to must be given inside a
+        string literal. The path is interpreted relative to the current working
+        directory. The tilde shorthand notation ('~/mydir') is supported for
+        referring to $HOME.
+
+        Only query result output is captured. Errors and output from cqlsh-only
+        commands will still be shown in the cqlsh session.
+
+        To stop capturing output and show it in the cqlsh session again, use
+        CAPTURE OFF.
+
+        To inspect the current capture configuration, use CAPTURE with no
+        arguments.
+        """
+        fname = parsed.get_binding('fname')
+        if fname is None:
+            if self.shunted_query_out is not None:
+                print "Currently capturing query output to %r." % (self.query_out.name,)
+            else:
+                print "Currently not capturing query output."
+            return
+
+        if fname.upper() == 'OFF':
+            if self.shunted_query_out is None:
+                self.printerr('Not currently capturing output.')
+                return
+            self.query_out.close()
+            self.query_out = self.shunted_query_out
+            self.color = self.shunted_color
+            self.shunted_query_out = None
+            del self.shunted_color
+            return
+
+        if self.shunted_query_out is not None:
+            self.printerr('Already capturing output to %s. Use CAPTURE OFF'
+                          ' to disable.' % (self.query_out.name,))
+            return
+
+        fname = os.path.expanduser(self.cql_unprotect_value(fname))
+        try:
+            f = open(fname, 'a')
+        except IOError, e:
+            self.printerr('Could not open %r for append: %s' % (fname, e))
+            return
+        self.shunted_query_out = self.query_out
+        self.shunted_color = self.color
+        self.query_out = f
+        self.color = False
+        print 'Now capturing query output to %r.' % (fname,)
+
+    def do_tracing(self, parsed):
+        """
+        TRACING [cqlsh]
+
+          Enables or disables request tracing.
+
+        TRACING ON
+
+          Enables tracing for all further requests.
+
+        TRACING OFF
+
+          Disables tracing.
+
+        TRACING
+
+          TRACING with no arguments shows the current tracing status.
+        """
+        self.tracing_enabled = SwitchCommand("TRACING", "Tracing").execute(self.tracing_enabled, parsed, self.printerr)
+
+    def do_expand(self, parsed):
+        """
+        EXPAND [cqlsh]
+
+          Enables or disables expanded (vertical) output.
+
+        EXPAND ON
+
+          Enables expanded (vertical) output.
+
+        EXPAND OFF
+
+          Disables expanded (vertical) output.
+
+        EXPAND
+
+          EXPAND with no arguments shows the current value of expand setting.
+        """
+        self.expand_enabled = SwitchCommand("EXPAND", "Expanded output").execute(self.expand_enabled, parsed, self.printerr)
+
+    def do_consistency(self, parsed):
+        """
+        CONSISTENCY [cqlsh only]
+
+           Overrides default consistency level (default level is ONE).
+
+        CONSISTENCY <level>
+
+           Sets consistency level for future requests.
+
+           Valid consistency levels:
+
+           ANY, ONE, TWO, THREE, QUORUM, ALL, LOCAL_ONE, LOCAL_QUORUM, EACH_QUORUM, SERIAL and LOCAL_SERIAL.
+
+           SERIAL and LOCAL_SERIAL may be used only for SELECTs; will be rejected with updates.
+
+        CONSISTENCY
+
+           CONSISTENCY with no arguments shows the current consistency level.
+        """
+        level = parsed.get_binding('level')
+        if level is None:
+            print 'Current consistency level is %s.' % (cassandra.ConsistencyLevel.value_to_name[self.consistency_level])
+            return
+
+        self.consistency_level = cassandra.ConsistencyLevel.name_to_value[level.upper()]
+        print 'Consistency level set to %s.' % (level.upper(),)
+
+    def do_serial(self, parsed):
+        """
+        SERIAL CONSISTENCY [cqlsh only]
+
+           Overrides serial consistency level (default level is SERIAL).
+
+        SERIAL CONSISTENCY <level>
+
+           Sets consistency level for future conditional updates.
+
+           Valid consistency levels:
+
+           SERIAL, LOCAL_SERIAL.
+
+        SERIAL CONSISTENCY
+
+           SERIAL CONSISTENCY with no arguments shows the current consistency level.
+        """
+        level = parsed.get_binding('level')
+        if level is None:
+            print 'Current serial consistency level is %s.' % (cassandra.ConsistencyLevel.value_to_name[self.serial_consistency_level])
+            return
+
+        self.serial_consistency_level = cassandra.ConsistencyLevel.name_to_value[level.upper()]
+        print 'Serial consistency level set to %s.' % (level.upper(),)
+
+    def do_login(self, parsed):
+        """
+        LOGIN [cqlsh only]
+
+           Changes login information without requiring restart.
+
+        LOGIN <username> (<password>)
+
+           Login using the specified username. If password is specified, it will be used
+           otherwise, you will be prompted to enter.
+        """
+        username = parsed.get_binding('username')
+        password = parsed.get_binding('password')
+        if password is None:
+            password = getpass.getpass()
+        else:
+            password = password[1:-1]
+
+        auth_provider = PlainTextAuthProvider(username=username, password=password)
+
+        conn = Cluster(contact_points=(self.hostname,), port=self.port, cql_version=self.conn.cql_version,
+                       protocol_version=self.conn.protocol_version,
+                       auth_provider=auth_provider,
+                       ssl_options=self.conn.ssl_options,
+                       load_balancing_policy=WhiteListRoundRobinPolicy([self.hostname]),
+                       control_connection_timeout=self.conn.connect_timeout,
+                       connect_timeout=self.conn.connect_timeout)
+
+        if self.current_keyspace:
+            session = conn.connect(self.current_keyspace)
+        else:
+            session = conn.connect()
+
+        # Update after we've connected in case we fail to authenticate
+        self.conn = conn
+        self.auth_provider = auth_provider
+        self.username = username
+        self.session = session
+
+    def do_exit(self, parsed=None):
+        """
+        EXIT/QUIT [cqlsh only]
+
+        Exits cqlsh.
+        """
+        self.stop = True
+        if self.owns_connection:
+            self.conn.shutdown()
+    do_quit = do_exit
+
+    def do_clear(self, parsed):
+        """
+        CLEAR/CLS [cqlsh only]
+
+        Clears the console.
+        """
+        import subprocess
+        subprocess.call(['clear', 'cls'][is_win], shell=True)
+    do_cls = do_clear
+
+    def do_debug(self, parsed):
+        import pdb
+        pdb.set_trace()
+
+    def get_help_topics(self):
+        topics = [t[3:] for t in dir(self) if t.startswith('do_') and getattr(self, t, None).__doc__]
+        for hide_from_help in ('quit',):
+            topics.remove(hide_from_help)
+        return topics
+
+    def columnize(self, slist, *a, **kw):
+        return cmd.Cmd.columnize(self, sorted([u.upper() for u in slist]), *a, **kw)
+
+    def do_help(self, parsed):
+        """
+        HELP [cqlsh only]
+
+        Gives information about cqlsh commands. To see available topics,
+        enter "HELP" without any arguments. To see help on a topic,
+        use "HELP <topic>".
+        """
+        topics = parsed.get_binding('topic', ())
+        if not topics:
+            shell_topics = [t.upper() for t in self.get_help_topics()]
+            self.print_topics("\nDocumented shell commands:", shell_topics, 15, 80)
+            cql_topics = [t.upper() for t in cqldocs.get_help_topics()]
+            self.print_topics("CQL help topics:", cql_topics, 15, 80)
+            return
+        for t in topics:
+            if t.lower() in self.get_help_topics():
+                doc = getattr(self, 'do_' + t.lower()).__doc__
+                self.stdout.write(doc + "\n")
+            elif t.lower() in cqldocs.get_help_topics():
+                urlpart = cqldocs.get_help_topic(t)
+                if urlpart is not None:
+                    url = "%s#%s" % (CASSANDRA_CQL_HTML, urlpart)
+                    if len(webbrowser._tryorder) == 0:
+                        self.printerr("*** No browser to display CQL help. URL for help topic %s : %s" % (t, url))
+                    elif self.browser is not None:
+                        webbrowser.get(self.browser).open_new_tab(url)
+                    else:
+                        webbrowser.open_new_tab(url)
+            else:
+                self.printerr("*** No help on %s" % (t,))
+
+    def do_unicode(self, parsed):
+        """
+        Textual input/output
+
+        When control characters, or other characters which can't be encoded
+        in your current locale, are found in values of 'text' or 'ascii'
+        types, it will be shown as a backslash escape. If color is enabled,
+        any such backslash escapes will be shown in a different color from
+        the surrounding text.
+
+        Unicode code points in your data will be output intact, if the
+        encoding for your locale is capable of decoding them. If you prefer
+        that non-ascii characters be shown with Python-style "\\uABCD"
+        escape sequences, invoke cqlsh with an ASCII locale (for example,
+        by setting the $LANG environment variable to "C").
+        """
+
+    def do_paging(self, parsed):
+        """
+        PAGING [cqlsh]
+
+          Enables or disables query paging.
+
+        PAGING ON
+
+          Enables query paging for all further queries.
+
+        PAGING OFF
+
+          Disables paging.
+
+        PAGING
+
+          PAGING with no arguments shows the current query paging status.
+        """
+        (self.use_paging, requested_page_size) = SwitchCommandWithValue(
+            "PAGING", "Query paging", value_type=int).execute(self.use_paging, parsed, self.printerr)
+        if self.use_paging and requested_page_size is not None:
+            self.page_size = requested_page_size
+        if self.use_paging:
+            print("Page size: {}".format(self.page_size))
+        else:
+            self.page_size = self.default_page_size
+
+    def applycolor(self, text, color=None):
+        if not color or not self.color:
+            return text
+        return color + text + ANSI_RESET
+
+    def writeresult(self, text, color=None, newline=True, out=None):
+        if out is None:
+            out = self.query_out
+
+        # convert Exceptions, etc to text
+        if not isinstance(text, (unicode, str)):
+            text = unicode(text)
+
+        if isinstance(text, unicode):
+            text = text.encode(self.encoding)
+
+        to_write = self.applycolor(text, color) + ('\n' if newline else '')
+        out.write(to_write)
+
+    def flush_output(self):
+        self.query_out.flush()
+
+    def printerr(self, text, color=RED, newline=True, shownum=None):
+        self.statement_error = True
+        if shownum is None:
+            shownum = self.show_line_nums
+        if shownum:
+            text = '%s:%d:%s' % (self.stdin.name, self.lineno, text)
+        self.writeresult(text, color, newline=newline, out=sys.stderr)
+
+
+class SwitchCommand(object):
+    command = None
+    description = None
+
+    def __init__(self, command, desc):
+        self.command = command
+        self.description = desc
+
+    def execute(self, state, parsed, printerr):
+        switch = parsed.get_binding('switch')
+        if switch is None:
+            if state:
+                print "%s is currently enabled. Use %s OFF to disable" \
+                      % (self.description, self.command)
+            else:
+                print "%s is currently disabled. Use %s ON to enable." \
+                      % (self.description, self.command)
+            return state
+
+        if switch.upper() == 'ON':
+            if state:
+                printerr('%s is already enabled. Use %s OFF to disable.'
+                         % (self.description, self.command))
+                return state
+            print 'Now %s is enabled' % (self.description,)
+            return True
+
+        if switch.upper() == 'OFF':
+            if not state:
+                printerr('%s is not enabled.' % (self.description,))
+                return state
+            print 'Disabled %s.' % (self.description,)
+            return False
+
+
+class SwitchCommandWithValue(SwitchCommand):
+    """The same as SwitchCommand except it also accepts a value in place of ON.
+
+    This returns a tuple of the form: (SWITCH_VALUE, PASSED_VALUE)
+    eg: PAGING 50 returns (True, 50)
+        PAGING OFF returns (False, None)
+        PAGING ON returns (True, None)
+
+    The value_type must match for the PASSED_VALUE, otherwise it will return None.
+    """
+    def __init__(self, command, desc, value_type=int):
+        SwitchCommand.__init__(self, command, desc)
+        self.value_type = value_type
+
+    def execute(self, state, parsed, printerr):
+        binary_switch_value = SwitchCommand.execute(self, state, parsed, printerr)
+        switch = parsed.get_binding('switch')
+        try:
+            value = self.value_type(switch)
+            binary_switch_value = True
+        except (ValueError, TypeError):
+            value = None
+        return (binary_switch_value, value)
+
+
+def option_with_default(cparser_getter, section, option, default=None):
+    try:
+        return cparser_getter(section, option)
+    except ConfigParser.Error:
+        return default
+
+
+def raw_option_with_default(configs, section, option, default=None):
+    """
+    Same (almost) as option_with_default() but won't do any string interpolation.
+    Useful for config values that include '%' symbol, e.g. time format string.
+    """
+    try:
+        return configs.get(section, option, raw=True)
+    except ConfigParser.Error:
+        return default
+
+
+def should_use_color():
+    if not sys.stdout.isatty():
+        return False
+    if os.environ.get('TERM', '') in ('dumb', ''):
+        return False
+    try:
+        import subprocess
+        p = subprocess.Popen(['tput', 'colors'], stdout=subprocess.PIPE)
+        stdout, _ = p.communicate()
+        if int(stdout.strip()) < 8:
+            return False
+    except (OSError, ImportError, ValueError):
+        # oh well, we tried. at least we know there's a $TERM and it's
+        # not "dumb".
+        pass
+    return True
+
+
+def read_options(cmdlineargs, environment):
+    configs = ConfigParser.SafeConfigParser()
+    configs.read(CONFIG_FILE)
+
+    rawconfigs = ConfigParser.RawConfigParser()
+    rawconfigs.read(CONFIG_FILE)
+
+    optvalues = optparse.Values()
+    optvalues.username = option_with_default(configs.get, 'authentication', 'username')
+    optvalues.password = option_with_default(rawconfigs.get, 'authentication', 'password')
+    optvalues.keyspace = option_with_default(configs.get, 'authentication', 'keyspace')
+    optvalues.browser = option_with_default(configs.get, 'ui', 'browser', None)
+    optvalues.completekey = option_with_default(configs.get, 'ui', 'completekey',
+                                                DEFAULT_COMPLETEKEY)
+    optvalues.color = option_with_default(configs.getboolean, 'ui', 'color')
+    optvalues.time_format = raw_option_with_default(configs, 'ui', 'time_format',
+                                                    DEFAULT_TIMESTAMP_FORMAT)
+    optvalues.nanotime_format = raw_option_with_default(configs, 'ui', 'nanotime_format',
+                                                        DEFAULT_NANOTIME_FORMAT)
+    optvalues.date_format = raw_option_with_default(configs, 'ui', 'date_format',
+                                                    DEFAULT_DATE_FORMAT)
+    optvalues.float_precision = option_with_default(configs.getint, 'ui', 'float_precision',
+                                                    DEFAULT_FLOAT_PRECISION)
+    optvalues.field_size_limit = option_with_default(configs.getint, 'csv', 'field_size_limit', csv.field_size_limit())
+    optvalues.max_trace_wait = option_with_default(configs.getfloat, 'tracing', 'max_trace_wait',
+                                                   DEFAULT_MAX_TRACE_WAIT)
+    optvalues.timezone = option_with_default(configs.get, 'ui', 'timezone', None)
+
+    optvalues.debug = False
+    optvalues.file = None
+    optvalues.ssl = False
+    optvalues.encoding = option_with_default(configs.get, 'ui', 'encoding', UTF8)
+
+    optvalues.tty = option_with_default(configs.getboolean, 'ui', 'tty', sys.stdin.isatty())
+    optvalues.cqlversion = option_with_default(configs.get, 'cql', 'version', DEFAULT_CQLVER)
+    optvalues.connect_timeout = option_with_default(configs.getint, 'connection', 'timeout', DEFAULT_CONNECT_TIMEOUT_SECONDS)
+    optvalues.request_timeout = option_with_default(configs.getint, 'connection', 'request_timeout', DEFAULT_REQUEST_TIMEOUT_SECONDS)
+    optvalues.execute = None
+
+    (options, arguments) = parser.parse_args(cmdlineargs, values=optvalues)
+
+    hostname = option_with_default(configs.get, 'connection', 'hostname', DEFAULT_HOST)
+    port = option_with_default(configs.get, 'connection', 'port', DEFAULT_PORT)
+
+    try:
+        options.connect_timeout = int(options.connect_timeout)
+    except ValueError:
+        parser.error('"%s" is not a valid connect timeout.' % (options.connect_timeout,))
+        options.connect_timeout = DEFAULT_CONNECT_TIMEOUT_SECONDS
+
+    try:
+        options.request_timeout = int(options.request_timeout)
+    except ValueError:
+        parser.error('"%s" is not a valid request timeout.' % (options.request_timeout,))
+        options.request_timeout = DEFAULT_REQUEST_TIMEOUT_SECONDS
+
+    hostname = environment.get('CQLSH_HOST', hostname)
+    port = environment.get('CQLSH_PORT', port)
+
+    if len(arguments) > 0:
+        hostname = arguments[0]
+    if len(arguments) > 1:
+        port = arguments[1]
+
+    if options.file or options.execute:
+        options.tty = False
+
+    if options.execute and not options.execute.endswith(';'):
+        options.execute += ';'
+
+    if optvalues.color in (True, False):
+        options.color = optvalues.color
+    else:
+        if options.file is not None:
+            options.color = False
+        else:
+            options.color = should_use_color()
+
+    options.cqlversion, cqlvertup = full_cql_version(options.cqlversion)
+    if cqlvertup[0] < 3:
+        parser.error('%r is not a supported CQL version.' % options.cqlversion)
+    else:
+        options.cqlmodule = cql3handling
+
+    try:
+        port = int(port)
+    except ValueError:
+        parser.error('%r is not a valid port number.' % port)
+    return options, hostname, port
+
+
+def setup_cqlruleset(cqlmodule):
+    global cqlruleset
+    cqlruleset = cqlmodule.CqlRuleSet
+    cqlruleset.append_rules(cqlsh_extra_syntax_rules)
+    for rulename, termname, func in cqlsh_syntax_completers:
+        cqlruleset.completer_for(rulename, termname)(func)
+    cqlruleset.commands_end_with_newline.update(my_commands_ending_with_newline)
+
+
+def setup_cqldocs(cqlmodule):
+    global cqldocs
+    cqldocs = cqlmodule.cqldocs
+
+
+def init_history():
+    if readline is not None:
+        try:
+            readline.read_history_file(HISTORY)
+        except IOError:
+            pass
+        delims = readline.get_completer_delims()
+        delims.replace("'", "")
+        delims += '.'
+        readline.set_completer_delims(delims)
+
+
+def save_history():
+    if readline is not None:
+        try:
+            readline.write_history_file(HISTORY)
+        except IOError:
+            pass
+
+
+def main(options, hostname, port):
+    setup_cqlruleset(options.cqlmodule)
+    setup_cqldocs(options.cqlmodule)
+    init_history()
+    csv.field_size_limit(options.field_size_limit)
+
+    if options.file is None:
+        stdin = None
+    else:
+        try:
+            encoding, bom_size = get_file_encoding_bomsize(options.file)
+            stdin = codecs.open(options.file, 'r', encoding)
+            stdin.seek(bom_size)
+        except IOError, e:
+            sys.exit("Can't open %r: %s" % (options.file, e))
+
+    if options.debug:
+        sys.stderr.write("Using CQL driver: %s\n" % (cassandra,))
+        sys.stderr.write("Using connect timeout: %s seconds\n" % (options.connect_timeout,))
+        sys.stderr.write("Using '%s' encoding\n" % (options.encoding,))
+
+    # create timezone based on settings, environment or auto-detection
+    timezone = None
+    if options.timezone or 'TZ' in os.environ:
+        try:
+            import pytz
+            if options.timezone:
+                try:
+                    timezone = pytz.timezone(options.timezone)
+                except:
+                    sys.stderr.write("Warning: could not recognize timezone '%s' specified in cqlshrc\n\n" % (options.timezone))
+            if 'TZ' in os.environ:
+                try:
+                    timezone = pytz.timezone(os.environ['TZ'])
+                except:
+                    sys.stderr.write("Warning: could not recognize timezone '%s' from environment value TZ\n\n" % (os.environ['TZ']))
+        except ImportError:
+            sys.stderr.write("Warning: Timezone defined and 'pytz' module for timezone conversion not installed. Timestamps will be displayed in UTC timezone.\n\n")
+
+    # try auto-detect timezone if tzlocal is installed
+    if not timezone:
+        try:
+            from tzlocal import get_localzone
+            timezone = get_localzone()
+        except ImportError:
+            # we silently ignore and fallback to UTC unless a custom timestamp format (which likely
+            # does contain a TZ part) was specified
+            if options.time_format != DEFAULT_TIMESTAMP_FORMAT:
+                sys.stderr.write("Warning: custom timestamp format specified in cqlshrc, but local timezone could not be detected.\n" +
+                                 "Either install Python 'tzlocal' module for auto-detection or specify client timezone in your cqlshrc.\n\n")
+
+    try:
+        shell = Shell(hostname,
+                      port,
+                      color=options.color,
+                      username=options.username,
+                      password=options.password,
+                      stdin=stdin,
+                      tty=options.tty,
+                      completekey=options.completekey,
+                      browser=options.browser,
+                      cqlver=options.cqlversion,
+                      keyspace=options.keyspace,
+                      display_timestamp_format=options.time_format,
+                      display_nanotime_format=options.nanotime_format,
+                      display_date_format=options.date_format,
+                      display_float_precision=options.float_precision,
+                      display_timezone=timezone,
+                      max_trace_wait=options.max_trace_wait,
+                      ssl=options.ssl,
+                      single_statement=options.execute,
+                      request_timeout=options.request_timeout,
+                      connect_timeout=options.connect_timeout,
+                      encoding=options.encoding)
+    except KeyboardInterrupt:
+        sys.exit('Connection aborted.')
+    except CQL_ERRORS, e:
+        sys.exit('Connection error: %s' % (e,))
+    except VersionNotSupported, e:
+        sys.exit('Unsupported CQL version: %s' % (e,))
+    if options.debug:
+        shell.debug = True
+
+    shell.cmdloop()
+    save_history()
+    batch_mode = options.file or options.execute
+    if batch_mode and shell.statement_error:
+        sys.exit(2)
+
+# always call this regardless of module name: when a sub-process is spawned
+# on Windows then the module name is not __main__, see CASSANDRA-9304
+insert_driver_hooks()
+
+if __name__ == '__main__':
+    main(*read_options(sys.argv[1:], os.environ))
+
+# vim: set ft=python et ts=4 sw=4 :
diff --git a/bin/nodetool.bat b/bin/nodetool.bat
index ec64db0..416aca5 100644
--- a/bin/nodetool.bat
+++ b/bin/nodetool.bat
@@ -23,8 +23,11 @@
 if NOT DEFINED CASSANDRA_HOME set CASSANDRA_HOME=%~dp0..

 if NOT DEFINED JAVA_HOME goto :err

 

+set CASSANDRA_PARAMS=%CASSANDRA_PARAMS% -Dcassandra.logdir="%CASSANDRA_HOME%\logs"

+set CASSANDRA_PARAMS=%CASSANDRA_PARAMS% -Dcassandra.storagedir="%CASSANDRA_HOME%\data"

+

 echo Starting NodeTool

-"%JAVA_HOME%\bin\java" -cp %CASSANDRA_CLASSPATH% -Dlogback.configurationFile=logback-tools.xml org.apache.cassandra.tools.NodeTool %*

+"%JAVA_HOME%\bin\java" -cp %CASSANDRA_CLASSPATH% %CASSANDRA_PARAMS% -Dlogback.configurationFile=logback-tools.xml org.apache.cassandra.tools.NodeTool %*

 goto finally

 

 :err

@@ -32,5 +35,5 @@
 pause

 

 :finally

-

-ENDLOCAL

+ENDLOCAL & set RC=%ERRORLEVEL%

+exit /B %RC%

diff --git a/bin/sstableverify b/bin/sstableverify
new file mode 100755
index 0000000..892750b
--- /dev/null
+++ b/bin/sstableverify
@@ -0,0 +1,60 @@
+#!/bin/sh
+
+# 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.
+
+if [ "x$CASSANDRA_INCLUDE" = "x" ]; then
+    for include in /usr/share/cassandra/cassandra.in.sh \
+                   /usr/local/share/cassandra/cassandra.in.sh \
+                   /opt/cassandra/cassandra.in.sh \
+                   "$HOME/.cassandra.in.sh" \
+                   "`dirname "$0"`/cassandra.in.sh"; do
+        if [ -r "$include" ]; then
+            . "$include"
+            break
+        fi
+    done
+elif [ -r "$CASSANDRA_INCLUDE" ]; then
+    . "$CASSANDRA_INCLUDE"
+fi
+
+# Use JAVA_HOME if set, otherwise look for java in PATH
+if [ -x "$JAVA_HOME/bin/java" ]; then
+    JAVA="$JAVA_HOME/bin/java"
+else
+    JAVA="`which java`"
+fi
+
+if [ "x$JAVA" = "x" ]; then
+    echo "Java executable not found (hint: set JAVA_HOME)" >&2
+    exit 1
+fi
+
+if [ -z "$CLASSPATH" ]; then
+    echo "You must set the CLASSPATH var" >&2
+    exit 1
+fi
+
+if [ "x$MAX_HEAP_SIZE" = "x" ]; then
+    MAX_HEAP_SIZE="256M"
+fi
+
+"$JAVA" $JAVA_AGENT -ea -cp "$CLASSPATH" $JVM_OPTS -Xmx$MAX_HEAP_SIZE \
+        -Dcassandra.storagedir="$cassandra_storagedir" \
+        -Dlogback.configurationFile=logback-tools.xml \
+        org.apache.cassandra.tools.StandaloneVerifier "$@"
+
+# vi:ai sw=4 ts=4 tw=0 et
diff --git a/bin/sstableverify.bat b/bin/sstableverify.bat
new file mode 100644
index 0000000..eb2629f
--- /dev/null
+++ b/bin/sstableverify.bat
@@ -0,0 +1,48 @@
+@REM

+@REM  Licensed to the Apache Software Foundation (ASF) under one or more

+@REM  contributor license agreements.  See the NOTICE file distributed with

+@REM  this work for additional information regarding copyright ownership.

+@REM  The ASF licenses this file to You under the Apache License, Version 2.0

+@REM  (the "License"); you may not use this file except in compliance with

+@REM  the License.  You may obtain a copy of the License at

+@REM

+@REM      http://www.apache.org/licenses/LICENSE-2.0

+@REM

+@REM  Unless required by applicable law or agreed to in writing, software

+@REM  distributed under the License is distributed on an "AS IS" BASIS,

+@REM  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.

+@REM  See the License for the specific language governing permissions and

+@REM  limitations under the License.

+

+@echo off

+if "%OS%" == "Windows_NT" setlocal

+

+pushd "%~dp0"

+call cassandra.in.bat

+

+if NOT DEFINED CASSANDRA_MAIN set CASSANDRA_MAIN=org.apache.cassandra.tools.StandaloneVerifier

+if NOT DEFINED JAVA_HOME goto :err

+

+REM ***** JAVA options *****

+set JAVA_OPTS=^

+ -Dlogback.configurationFile=logback-tools.xml

+

+set TOOLS_PARAMS=

+

+"%JAVA_HOME%\bin\java" %JAVA_OPTS% %CASSANDRA_PARAMS% -cp %CASSANDRA_CLASSPATH% "%CASSANDRA_MAIN%" %*

+goto finally

+

+:err

+echo JAVA_HOME environment variable must be set!

+set ERRORLEVEL=1

+pause

+

+:finally

+ENDLOCAL & set RC=%ERRORLEVEL%

+goto :exit_with_code

+

+:returncode

+exit /B %RC%

+

+:exit_with_code

+call :returncode %RC%

diff --git a/bin/stop-server.ps1 b/bin/stop-server.ps1
index 0d125dc..d1bc952 100644
--- a/bin/stop-server.ps1
+++ b/bin/stop-server.ps1
@@ -125,7 +125,10 @@
 

                     // Must wait here. If we don't and re-enable Ctrl-C

                     // handling below too fast, we might terminate ourselves.

-                    proc.WaitForExit(2000);

+                    bool exited = proc.WaitForExit(30000);

+                    if(!exited)

+                        proc.Kill();

+

                     FreeConsole();

 

                     // Re-attach to current console to write output

@@ -137,7 +140,12 @@
                     SetConsoleCtrlHandler(null, false);

 

                     if (!silent)

-                        System.Console.WriteLine("Successfully sent ctrl+c to process with id: " + pidToKill + ".");

+                    {

+                        if(exited)

+                            System.Console.WriteLine("Successfully sent ctrl+c to process with id: " + pidToKill + ".");

+                        else

+                            System.Console.WriteLine("Process with id: " + pidToKill + " did not exit after 30 seconds, killed.");

+                    }

                 }

                 else

                 {

diff --git a/build.xml b/build.xml
index 6619bfd..b302b5b 100644
--- a/build.xml
+++ b/build.xml
@@ -25,7 +25,7 @@
     <property name="debuglevel" value="source,lines,vars"/>
 
     <!-- default version and SCM information -->
-    <property name="base.version" value="2.1.18"/>
+    <property name="base.version" value="2.2.10"/>
     <property name="scm.connection" value="scm:git://git.apache.org/cassandra.git"/>
     <property name="scm.developerConnection" value="scm:git://git.apache.org/cassandra.git"/>
     <property name="scm.url" value="http://git-wip-us.apache.org/repos/asf?p=cassandra.git;a=tree"/>
@@ -34,10 +34,10 @@
     <property name="basedir" value="."/>
     <property name="build.src" value="${basedir}/src"/>
     <property name="build.src.java" value="${basedir}/src/java"/>
+    <property name="build.src.jdkoverride" value="${basedir}/src/jdkoverride" />
     <property name="build.src.resources" value="${basedir}/src/resources"/>
     <property name="build.src.gen-java" value="${basedir}/src/gen-java"/>
     <property name="build.lib" value="${basedir}/lib"/>
-    <property name="build.tools.lib" value="${basedir}/tools/lib"/>
     <property name="build.dir" value="${basedir}/build"/>
     <property name="build.dir.lib" value="${basedir}/build/lib"/>
     <property name="build.test.dir" value="${build.dir}/test"/>
@@ -56,16 +56,20 @@
     <property name="test.conf" value="${test.dir}/conf"/>
     <property name="test.data" value="${test.dir}/data"/>
     <property name="test.name" value="*Test"/>
+    <property name="test.classlistfile" value="testlist.txt"/>
+    <property name="benchmark.name" value=""/>
     <property name="test.methods" value=""/>
     <property name="test.runners" value="1"/>
     <property name="test.unit.src" value="${test.dir}/unit"/>
     <property name="test.long.src" value="${test.dir}/long"/>
+    <property name="test.burn.src" value="${test.dir}/burn"/>
+    <property name="test.microbench.src" value="${test.dir}/microbench"/>
     <property name="test.pig.src" value="${test.dir}/pig"/>
     <property name="dist.dir" value="${build.dir}/dist"/>
     <property name="tmp.dir" value="${java.io.tmpdir}"/>
 	
-	<property name="source.version" value="1.7"/>
-	<property name="target.version" value="1.7"/>
+    <property name="source.version" value="1.7"/>
+    <property name="target.version" value="1.7"/>
 	
     <condition property="version" value="${base.version}">
       <isset property="release"/>
@@ -104,6 +108,17 @@
     <property name="cobertura.report.dir" value="${cobertura.build.dir}/report"/>
     <property name="cobertura.classes.dir" value="${cobertura.build.dir}/classes"/>
     <property name="cobertura.datafile" value="${cobertura.build.dir}/cobertura.ser"/>
+    
+    <!-- http://www.eclemma.org/jacoco/ -->
+    <property name="jacoco.export.dir" value="${build.dir}/jacoco/" />
+    <property name="jacoco.partials.dir" value="${jacoco.export.dir}/partials" />
+    <property name="jacoco.partialexecfile" value="${jacoco.partials.dir}/partial.exec" />
+    <property name="jacoco.finalexecfile" value="${jacoco.export.dir}/jacoco.exec" />
+    <property name="jacoco.version" value="0.7.5.201505241946"/>
+
+    <property name="byteman.version" value="3.0.3"/>
+
+    <property name="ecj.version" value="4.4.2"/>
 
     <condition property="maven-ant-tasks.jar.exists">
       <available file="${build.dir}/maven-ant-tasks-${maven-ant-tasks.version}.jar" />
@@ -132,10 +147,12 @@
         <fileset dir="${build.lib}">
           <include name="**/*.jar" />
           <exclude name="**/*-sources.jar"/>
+          <exclude name="**/ant-*.jar"/>
         </fileset>
         <fileset dir="${build.dir.lib}">
           <include name="**/*.jar" />
           <exclude name="**/*-sources.jar"/>
+          <exclude name="**/ant-*.jar"/>
         </fileset>
     </path>
 	
@@ -169,6 +186,8 @@
         <mkdir dir="${test.classes}"/>
         <mkdir dir="${build.src.gen-java}"/>
         <mkdir dir="${build.dir.lib}"/>
+        <mkdir dir="${jacoco.export.dir}"/>
+        <mkdir dir="${jacoco.partials.dir}"/>
     </target>
 
     <target name="clean" description="Remove all locally created artifacts">
@@ -177,6 +196,8 @@
         <delete dir="${cobertura.classes.dir}" />
         <delete dir="${build.src.gen-java}" />
         <delete dir="${version.properties.dir}" />
+        <delete dir="${jacoco.export.dir}" />
+        <delete dir="${jacoco.partials.dir}"/>
     </target>
     <target depends="clean" name="cleanall"/>
 
@@ -185,53 +206,14 @@
     </target>
 
     <!--
-       This generates the CLI grammar files from Cli.g
-    -->
-    <target name="check-gen-cli-grammar">
-        <uptodate property="cliUpToDate"
-                srcfile="${build.src.java}/org/apache/cassandra/cli/Cli.g"
-                targetfile="${build.src.gen-java}/org/apache/cassandra/cli/Cli.tokens"/>
-    </target>
-
-    <target name="gen-cli-grammar" depends="check-gen-cli-grammar" unless="cliUpToDate">
-      <echo>Building Grammar ${build.src.java}/org/apache/cassandra/cli/Cli.g  ....</echo>
-      <java classname="org.antlr.Tool"
-        classpath="${build.dir.lib}/jars/antlr-3.5.2.jar;${build.lib}/antlr-runtime-3.5.2.jar;${build.lib}/ST4-4.0.8.jar"
-            fork="true"
-            failonerror="true">
-         <jvmarg value="-Xmx512M" />
-         <arg value="${build.src.java}/org/apache/cassandra/cli/Cli.g" />
-         <arg value="-fo" />
-         <arg value="${build.src.gen-java}/org/apache/cassandra/cli/" />
-      </java>
-    </target>
-
-    <!--
        This generates the CQL grammar files from Cql.g
     -->
-    <target name="check-gen-cql2-grammar">
-        <uptodate property="cql2current"
-                srcfile="${build.src.java}/org/apache/cassandra/cql/Cql.g"
-                targetfile="${build.src.gen-java}/org/apache/cassandra/cql/Cql.tokens"/>
-    </target>
     <target name="check-gen-cql3-grammar">
         <uptodate property="cql3current"
                 srcfile="${build.src.java}/org/apache/cassandra/cql3/Cql.g"
                 targetfile="${build.src.gen-java}/org/apache/cassandra/cql3/Cql.tokens"/>
     </target>
  
-    <target name="gen-cql2-grammar" depends="check-gen-cql2-grammar" unless="cql2current">
-      <echo>Building Grammar ${build.src.java}/org/apache/cassandra/cql/Cql.g  ...</echo>
-      <java classname="org.antlr.Tool"
-            classpath="${build.dir.lib}/jars/antlr-3.5.2.jar;${build.lib}/antlr-runtime-3.5.2.jar;${build.lib}/ST4-4.0.8.jar"
-            fork="true"
-            failonerror="true">
-         <jvmarg value="-Xmx512M" />
-         <arg value="${build.src.java}/org/apache/cassandra/cql/Cql.g" />
-         <arg value="-fo" />
-         <arg value="${build.src.gen-java}/org/apache/cassandra/cql/" />
-      </java>
-    </target>
     <target name="gen-cql3-grammar" depends="check-gen-cql3-grammar" unless="cql3current">
       <echo>Building Grammar ${build.src.java}/org/apache/cassandra/cql3/Cql.g  ...</echo>
       <java classname="org.antlr.Tool"
@@ -244,6 +226,8 @@
          <arg value="${build.src.java}/org/apache/cassandra/cql3/Cql.g" />
          <arg value="-fo" />
          <arg value="${build.src.gen-java}/org/apache/cassandra/cql3/" />
+         <arg value="-Xmaxinlinedfastates"/>
+         <arg value="10"/> <!-- default is 60 -->
       </java>
     </target>
 
@@ -257,9 +241,6 @@
         <taskdef classpathref="wikitext.classpath" resource="wikitexttasks.properties" />
         <wikitext-to-html markupLanguage="Textile">
             <fileset dir="${basedir}">
-                <include name="doc/cql/*.textile"/>
-            </fileset>
-            <fileset dir="${basedir}">
                 <include name="doc/cql3/*.textile"/>
             </fileset>
         </wikitext-to-html>
@@ -282,6 +263,8 @@
       <echo>Downloading Maven ANT Tasks...</echo>
       <get src="${maven-ant-tasks.url}/${maven-ant-tasks.version}/maven-ant-tasks-${maven-ant-tasks.version}.jar"
            dest="${build.dir}/maven-ant-tasks-${maven-ant-tasks.version}.jar" usetimestamp="true" />
+      <copy file="${build.dir}/maven-ant-tasks-${maven-ant-tasks.version}.jar"
+            tofile="${maven-ant-tasks.local}/${maven-ant-tasks.version}/maven-ant-tasks-${maven-ant-tasks.version}.jar"/>
     </target>
 
     <target name="maven-ant-tasks-init" depends="init,maven-ant-tasks-download" unless="maven-ant-tasks.initialized"
@@ -350,94 +333,143 @@
         <license name="The Apache Software License, Version 2.0" url="http://www.apache.org/licenses/LICENSE-2.0.txt"/>
         <scm connection="${scm.connection}" developerConnection="${scm.developerConnection}" url="${scm.url}"/>
         <dependencyManagement>
-          <dependency groupId="org.xerial.snappy" artifactId="snappy-java" version="1.0.5"/>
-          <dependency groupId="net.jpountz.lz4" artifactId="lz4" version="1.2.0"/>
+          <dependency groupId="org.xerial.snappy" artifactId="snappy-java" version="1.1.1.7"/>
+          <dependency groupId="net.jpountz.lz4" artifactId="lz4" version="1.3.0"/>
           <dependency groupId="com.ning" artifactId="compress-lzf" version="0.8.4"/>
           <dependency groupId="com.google.guava" artifactId="guava" version="16.0"/>
           <dependency groupId="commons-cli" artifactId="commons-cli" version="1.1"/>
           <dependency groupId="commons-codec" artifactId="commons-codec" version="1.2"/>
           <dependency groupId="org.apache.commons" artifactId="commons-lang3" version="3.1"/>
           <dependency groupId="org.apache.commons" artifactId="commons-math3" version="3.2"/>
-          <dependency groupId="com.googlecode.concurrentlinkedhashmap" artifactId="concurrentlinkedhashmap-lru" version="1.3"/>
+          <dependency groupId="com.googlecode.concurrentlinkedhashmap" artifactId="concurrentlinkedhashmap-lru" version="1.4"/>
           <dependency groupId="org.antlr" artifactId="antlr" version="3.5.2">
             <exclusion groupId="org.antlr" artifactId="stringtemplate"/>
           </dependency>
           <dependency groupId="org.antlr" artifactId="antlr-runtime" version="3.5.2">
             <exclusion groupId="org.antlr" artifactId="stringtemplate"/>
           </dependency>
-          <dependency groupId="org.slf4j" artifactId="slf4j-api" version="1.7.2"/>
-          <dependency groupId="ch.qos.logback" artifactId="logback-core" version="1.1.2"/>
-          <dependency groupId="ch.qos.logback" artifactId="logback-classic" version="1.1.2"/>
+          <dependency groupId="org.slf4j" artifactId="slf4j-api" version="1.7.7"/>
+          <dependency groupId="org.slf4j" artifactId="log4j-over-slf4j" version="1.7.7"/>
+          <dependency groupId="org.slf4j" artifactId="jcl-over-slf4j" version="1.7.7" />
+          <dependency groupId="ch.qos.logback" artifactId="logback-core" version="1.1.3"/>
+          <dependency groupId="ch.qos.logback" artifactId="logback-classic" version="1.1.3"/>
           <dependency groupId="org.codehaus.jackson" artifactId="jackson-core-asl" version="1.9.2"/>
           <dependency groupId="org.codehaus.jackson" artifactId="jackson-mapper-asl" version="1.9.2"/>
-          <dependency groupId="jline" artifactId="jline" version="1.0">
-            <exclusion groupId="junit" artifactId="junit"/>
-          </dependency>
           <dependency groupId="com.googlecode.json-simple" artifactId="json-simple" version="1.1"/>
           <dependency groupId="com.boundary" artifactId="high-scale-lib" version="1.0.6"/>
           <dependency groupId="com.github.jbellis" artifactId="jamm" version="0.3.0"/>
+	  	  <dependency groupId="com.github.tjake" artifactId="crc32ex" version="0.1.1"/>
           <dependency groupId="com.thinkaurelius.thrift" artifactId="thrift-server" version="0.3.7">
-	      	<exclusion groupId="org.slf4j" artifactId="slf4j-log4j12"/>
+            <exclusion groupId="org.slf4j" artifactId="slf4j-log4j12"/>
           </dependency>
           <dependency groupId="org.yaml" artifactId="snakeyaml" version="1.11"/>
-          <dependency groupId="org.apache.thrift" artifactId="libthrift" version="0.9.2"/>
-
+          <dependency groupId="org.apache.thrift" artifactId="libthrift" version="0.9.2">
+	         <exclusion groupId="commons-logging" artifactId="commons-logging"/>
+          </dependency>
           <dependency groupId="junit" artifactId="junit" version="4.6" />
-          <dependency groupId="commons-logging" artifactId="commons-logging" version="1.1.1"/>
           <dependency groupId="org.apache.rat" artifactId="apache-rat" version="0.10">
              <exclusion groupId="commons-lang" artifactId="commons-lang"/>
           </dependency>
           <dependency groupId="org.apache.hadoop" artifactId="hadoop-core" version="1.0.3">
           	<exclusion groupId="org.mortbay.jetty" artifactId="servlet-api"/>
-		<exclusion groupId="ant" artifactId="ant"/>
+          	<exclusion groupId="commons-logging" artifactId="commons-logging"/>
+          	<exclusion groupId="org.eclipse.jdt" artifactId="core"/>
+		    <exclusion groupId="ant" artifactId="ant"/>
           </dependency>
-          <dependency groupId="org.apache.hadoop" artifactId="hadoop-minicluster" version="1.0.3"/>
-          <dependency groupId="org.apache.pig" artifactId="pig" version="0.12.1"/>
+          <dependency groupId="org.apache.hadoop" artifactId="hadoop-minicluster" version="1.0.3">
+		    <exclusion groupId="asm" artifactId="asm"/> <!-- this is the outdated version 3.1 -->
+          </dependency>
+          <dependency groupId="org.apache.pig" artifactId="pig" version="0.12.1">
+          	<exclusion groupId="org.mortbay.jetty" artifactId="jetty"/>
+            <exclusion groupId="hsqldb" artifactId="hsqldb"/> <!-- outdated version -->
+            <exclusion groupId="antlr" artifactId="antlr"/> <!-- outdated version -->
+          </dependency>
           <dependency groupId="net.java.dev.jna" artifactId="jna" version="4.0.0"/>
 
           <dependency groupId="net.sourceforge.cobertura" artifactId="cobertura" version="${cobertura.version}">
             <exclusion groupId="xerces" artifactId="xercesImpl"/>
+          	<exclusion groupId="org.mortbay.jetty" artifactId="jetty"/> <!-- older version, also via hadoop-core + pig -->
+          	<exclusion groupId="org.mortbay.jetty" artifactId="jetty-util"/> <!-- older version, also via hadoop-core + pig -->
+            <exclusion groupId="org.apache.ant" artifactId="ant"/> <!-- older version 1.8.3 -->
           </dependency>
+          <dependency groupId="org.jacoco" artifactId="org.jacoco.agent" version="${jacoco.version}"/>
+          <dependency groupId="org.jacoco" artifactId="org.jacoco.ant" version="${jacoco.version}"/>
+
+          <dependency groupId="org.jboss.byteman" artifactId="byteman-install" version="${byteman.version}" scope="test"/>
+          <dependency groupId="org.jboss.byteman" artifactId="byteman" version="${byteman.version}" scope="test"/>
+          <dependency groupId="org.jboss.byteman" artifactId="byteman-submit" version="${byteman.version}" scope="test"/>
+          <dependency groupId="org.jboss.byteman" artifactId="byteman-bmunit" version="${byteman.version}" scope="test"/>
+
+
+          <dependency groupId="org.openjdk.jmh" artifactId="jmh-core" version="1.1.1"/>
+          <dependency groupId="org.openjdk.jmh" artifactId="jmh-generator-annprocess" version="1.1.1"/>
 
           <dependency groupId="org.apache.cassandra" artifactId="cassandra-all" version="${version}" />
           <dependency groupId="org.apache.cassandra" artifactId="cassandra-thrift" version="${version}" />
-          <dependency groupId="com.yammer.metrics" artifactId="metrics-core" version="2.2.0" />
-          <dependency groupId="com.addthis.metrics" artifactId="reporter-config" version="2.1.0" />
+          <dependency groupId="io.dropwizard.metrics" artifactId="metrics-core" version="3.1.0" />
+          <dependency groupId="io.dropwizard.metrics" artifactId="metrics-jvm" version="3.1.0" />
+          <dependency groupId="com.addthis.metrics" artifactId="reporter-config3" version="3.0.0" />
           <dependency groupId="org.mindrot" artifactId="jbcrypt" version="0.3m" />
           <dependency groupId="io.airlift" artifactId="airline" version="0.6" />
           <dependency groupId="io.netty" artifactId="netty-all" version="4.0.44.Final" />
           <dependency groupId="com.google.code.findbugs" artifactId="jsr305" version="2.0.2" />
           <dependency groupId="com.clearspring.analytics" artifactId="stream" version="2.5.2" />
-          <dependency groupId="com.datastax.cassandra" artifactId="cassandra-driver-core" version="2.0.9.2" />
+          <!-- TODO CASSANDRA-9543
+          <dependency groupId="com.datastax.cassandra" artifactId="cassandra-driver-core" version="2.1.5" classifier="shaded" />
+          -->
+          <dependency groupId="org.eclipse.jdt.core.compiler" artifactId="ecj" version="4.4.2" />
+          <dependency groupId="org.caffinitas.ohc" artifactId="ohc-core" version="0.3.4" />
           <dependency groupId="net.sf.supercsv" artifactId="super-csv" version="2.1.0" />
 	      <dependency groupId="net.ju-n.compile-command-annotations" artifactId="compile-command-annotations" version="1.2.0" />
+          <dependency groupId="org.fusesource" artifactId="sigar" version="1.6.4">
+          	<exclusion groupId="log4j" artifactId="log4j"/>
+          </dependency>
+          <dependency groupId="joda-time" artifactId="joda-time" version="2.4" />
+        	
         </dependencyManagement>
+        <developer id="adelapena" name="Andres de la Peña"/>
         <developer id="alakshman" name="Avinash Lakshman"/>
         <developer id="aleksey" name="Aleksey Yeschenko"/>
         <developer id="amorton" name="Aaron Morton"/>
+        <developer id="aweisberg" name="Ariel Weisberg"/>
+        <developer id="bdeggleston" name="Blake Eggleston"/>
         <developer id="benedict" name="Benedict Elliott Smith"/>
         <developer id="benjamin" name="Benjamin Lerer"/>
+        <developer id="blambov" name="Branimir Lambov"/>
         <developer id="brandonwilliams" name="Brandon Williams"/>
-        <developer id="dbrosius" name="David Brosius"/>
+        <developer id="carl" name="Carl Yeksigian"/>
+        <developer id="dbrosius" name="David Brosiusd"/>
+        <developer id="dikang" name="Dikang Gu"/>
         <developer id="eevans" name="Eric Evans"/>
         <developer id="gdusbabek" name="Gary Dusbabek"/>
         <developer id="goffinet" name="Chris Goffinet"/>
+        <developer id="ifesdjeen" name="Alex Petrov"/>
         <developer id="jaakko" name="Laine Jaakko Olavi"/>
         <developer id="jake" name="T Jake Luciani"/>
         <developer id="jasonbrown" name="Jason Brown"/>
         <developer id="jbellis" name="Jonathan Ellis"/>
+        <developer id="jfarrell" name="Jake Farrell"/>
+        <developer id="jjirsa" name="Jeff Jirsa"/>
+        <developer id="jkni" name="Joel Knighton"/>
         <developer id="jmckenzie" name="Josh McKenzie"/>
         <developer id="johan" name="Johan Oskarsson"/>
         <developer id="junrao" name="Jun Rao"/>
+        <developer id="kohlisankalp" name="Sankalp Kohli"/>
         <developer id="marcuse" name="Marcus Eriksson"/>
+        <developer id="mck" name="Michael Semb Wever"/>
         <developer id="mishail" name="Mikhail Stepura"/>
+        <developer id="mshuler" name="Michael Shuler"/>
+        <developer id="paulo" name="Paulo Motta"/>
         <developer id="pmalik" name="Prashant Malik"/>
+        <developer id="rstupp" name="Robert Stupp"/>
         <developer id="scode" name="Peter Schuller"/>
         <developer id="slebresne" name="Sylvain Lebresne"/>
+        <developer id="stefania" name="Stefania Alborghetti"/>
         <developer id="tylerhobbs" name="Tyler Hobbs"/>
         <developer id="vijay" name="Vijay Parthasarathy"/>
         <developer id="xedin" name="Pavel Yaskevich"/>
         <developer id="yukim" name="Yuki Morishita"/>
+        <developer id="zznate" name="Nate McCall"/>
       </artifact:pom>
 
       <!-- each dependency set then defines the subset of the dependencies for that dependency set -->
@@ -447,15 +479,48 @@
                 artifactId="cassandra-parent"
                 version="${version}"/>
         <dependency groupId="junit" artifactId="junit"/>
-        <dependency groupId="commons-logging" artifactId="commons-logging"/>
         <dependency groupId="org.apache.rat" artifactId="apache-rat"/>
         <dependency groupId="org.apache.hadoop" artifactId="hadoop-core"/>
       	<dependency groupId="org.apache.hadoop" artifactId="hadoop-minicluster"/>
         <dependency groupId="org.apache.pig" artifactId="pig"/>
       	<dependency groupId="com.google.code.findbugs" artifactId="jsr305"/>
         <dependency groupId="org.antlr" artifactId="antlr"/>
-        <dependency groupId="com.datastax.cassandra" artifactId="cassandra-driver-core"/>
-	<dependency groupId="net.ju-n.compile-command-annotations" artifactId="compile-command-annotations"/>
+        <!-- TODO CASSANDRA-9543
+        <dependency groupId="com.datastax.cassandra" artifactId="cassandra-driver-core" classifier="shaded"/>
+        -->
+        <dependency groupId="org.eclipse.jdt.core.compiler" artifactId="ecj"/>
+        <dependency groupId="org.caffinitas.ohc" artifactId="ohc-core"/>
+        <dependency groupId="org.openjdk.jmh" artifactId="jmh-core"/>
+        <dependency groupId="org.openjdk.jmh" artifactId="jmh-generator-annprocess"/>
+        <dependency groupId="net.ju-n.compile-command-annotations" artifactId="compile-command-annotations"/>
+        <dependency groupId="org.apache.ant" artifactId="ant-junit" version="1.9.4" />
+      </artifact:pom>
+      <!-- this build-deps-pom-sources "artifact" is the same as build-deps-pom but only with those
+           artifacts that have "-source.jar" files -->
+      <artifact:pom id="build-deps-pom-sources"
+                    artifactId="cassandra-build-deps">
+        <parent groupId="org.apache.cassandra"
+                artifactId="cassandra-parent"
+                version="${version}"/>
+        <dependency groupId="junit" artifactId="junit"/>
+        <dependency groupId="org.apache.pig" artifactId="pig">
+          <exclusion groupId="xmlenc" artifactId="xmlenc"/>
+          <exclusion groupId="tomcat" artifactId="jasper-runtime"/>
+          <exclusion groupId="tomcat" artifactId="jasper-compiler"/>
+          <exclusion groupId="org.eclipse.jdt" artifactId="core"/>
+          <exclusion groupId="net.sf.kosmosfs" artifactId="kfs"/>
+          <exclusion groupId="hsqldb" artifactId="hsqldb"/>
+          <exclusion groupId="antlr" artifactId="antlr"/>
+        </dependency>
+        <!-- TODO CASSANDRA-9543
+        <dependency groupId="com.datastax.cassandra" artifactId="cassandra-driver-core" classifier="shaded"/>
+        -->
+        <dependency groupId="org.eclipse.jdt.core.compiler" artifactId="ecj"/>
+        <dependency groupId="org.caffinitas.ohc" artifactId="ohc-core"/>
+        <dependency groupId="org.openjdk.jmh" artifactId="jmh-core"/>
+        <dependency groupId="org.openjdk.jmh" artifactId="jmh-generator-annprocess"/>
+        <dependency groupId="net.ju-n.compile-command-annotations" artifactId="compile-command-annotations"/>
+        <dependency groupId="org.apache.ant" artifactId="ant-junit" version="1.9.4" />
       </artifact:pom>
 
       <artifact:pom id="coverage-deps-pom"
@@ -464,6 +529,12 @@
                 artifactId="cassandra-parent"
                 version="${version}"/>
         <dependency groupId="net.sourceforge.cobertura" artifactId="cobertura"/>
+        <dependency groupId="org.jacoco" artifactId="org.jacoco.agent"/>
+        <dependency groupId="org.jacoco" artifactId="org.jacoco.ant"/>
+        <dependency groupId="org.jboss.byteman" artifactId="byteman-install" scope="test"/>
+        <dependency groupId="org.jboss.byteman" artifactId="byteman" scope="test"/>
+        <dependency groupId="org.jboss.byteman" artifactId="byteman-submit" scope="test"/>
+        <dependency groupId="org.jboss.byteman" artifactId="byteman-bmunit" scope="test"/>
       </artifact:pom>
 
       <artifact:pom id="test-deps-pom"
@@ -472,8 +543,7 @@
                 artifactId="cassandra-parent"
                 version="${version}"/>
         <!-- do NOT remove this, it breaks pig-test -->
-        <dependency groupId="org.slf4j" artifactId="slf4j-log4j12" version="1.7.2"/>
-        <dependency groupId="joda-time" artifactId="joda-time" version="2.3" />
+        <dependency groupId="joda-time" artifactId="joda-time"/>
       </artifact:pom>
 
       <!-- now the pom's for artifacts being deployed to Maven Central -->
@@ -498,37 +568,49 @@
         <dependency groupId="org.antlr" artifactId="antlr"/>
         <dependency groupId="org.antlr" artifactId="antlr-runtime"/>
         <dependency groupId="org.slf4j" artifactId="slf4j-api"/>
+        <dependency groupId="org.slf4j" artifactId="log4j-over-slf4j"/>
+        <dependency groupId="org.slf4j" artifactId="jcl-over-slf4j"/>
         <dependency groupId="org.codehaus.jackson" artifactId="jackson-core-asl"/>
         <dependency groupId="org.codehaus.jackson" artifactId="jackson-mapper-asl"/>
-        <dependency groupId="jline" artifactId="jline"/>
         <dependency groupId="com.googlecode.json-simple" artifactId="json-simple"/>
         <dependency groupId="com.boundary" artifactId="high-scale-lib"/>
         <dependency groupId="org.yaml" artifactId="snakeyaml"/>
         <dependency groupId="org.mindrot" artifactId="jbcrypt"/>
-        <dependency groupId="com.yammer.metrics" artifactId="metrics-core"/>
-        <dependency groupId="com.addthis.metrics" artifactId="reporter-config"/>
-        <dependency groupId="com.thinkaurelius.thrift" artifactId="thrift-server" version="0.3.7"/>
-        <dependency groupId="com.clearspring.analytics" artifactId="stream" version="2.5.2" />
-        <dependency groupId="net.sf.supercsv" artifactId="super-csv" version="2.1.0" />
+        <dependency groupId="io.dropwizard.metrics" artifactId="metrics-core"/>
+        <dependency groupId="io.dropwizard.metrics" artifactId="metrics-jvm"/>
+        <dependency groupId="com.addthis.metrics" artifactId="reporter-config3"/>
+        <dependency groupId="com.thinkaurelius.thrift" artifactId="thrift-server"/>
+        <dependency groupId="com.clearspring.analytics" artifactId="stream"/>
+        <dependency groupId="net.sf.supercsv" artifactId="super-csv"/>
 
         <dependency groupId="ch.qos.logback" artifactId="logback-core"/>
         <dependency groupId="ch.qos.logback" artifactId="logback-classic"/>
 
         <dependency groupId="org.apache.thrift" artifactId="libthrift"/>
         <dependency groupId="org.apache.cassandra" artifactId="cassandra-thrift"/>
-        
+
         <!-- don't need hadoop classes to run, but if you use the hadoop stuff -->
         <dependency groupId="org.apache.hadoop" artifactId="hadoop-core" optional="true"/>
         <dependency groupId="org.apache.hadoop" artifactId="hadoop-minicluster" optional="true"/>
         <dependency groupId="org.apache.pig" artifactId="pig" optional="true"/>
-      	<dependency groupId="com.datastax.cassandra" artifactId="cassandra-driver-core" optional="true"/>
+        <!-- TODO CASSANDRA-9543
+        <dependency groupId="com.datastax.cassandra" artifactId="cassandra-driver-core" classifier="shaded" optional="true"/>
+        -->
 
         <!-- don't need jna to run, but nice to have -->
-        <dependency groupId="net.java.dev.jna" artifactId="jna" version="4.0.0"/>
+        <dependency groupId="net.java.dev.jna" artifactId="jna"/>
         
         <!-- don't need jamm unless running a server in which case it needs to be a -javagent to be used anyway -->
         <dependency groupId="com.github.jbellis" artifactId="jamm"/>
+		<dependency groupId="com.github.tjake" artifactId="crc32ex"/>
         <dependency groupId="io.netty" artifactId="netty-all"/>
+      	
+      	<dependency groupId="joda-time" artifactId="joda-time"/>
+
+        <dependency groupId="org.fusesource" artifactId="sigar"/>
+
+        <dependency groupId="org.eclipse.jdt.core.compiler" artifactId="ecj"/>
+        <dependency groupId="org.caffinitas.ohc" artifactId="ohc-core"/>
       </artifact:pom>
       <artifact:pom id="thrift-pom"
                     artifactId="cassandra-thrift"
@@ -540,6 +622,8 @@
         <scm connection="${scm.connection}" developerConnection="${scm.developerConnection}" url="${scm.url}"/>
         <dependency groupId="org.apache.commons" artifactId="commons-lang3"/>
         <dependency groupId="org.slf4j" artifactId="slf4j-api"/>
+        <dependency groupId="org.slf4j" artifactId="log4j-over-slf4j"/>
+        <dependency groupId="org.slf4j" artifactId="jcl-over-slf4j"/>
         <dependency groupId="org.apache.thrift" artifactId="libthrift"/>
       </artifact:pom>
       <artifact:pom id="clientutil-pom"
@@ -566,19 +650,22 @@
     </target>
 
     <target name="maven-ant-tasks-retrieve-build" depends="maven-declare-dependencies" unless="without.maven">
+      <!-- retrieve artifacts -->
       <artifact:dependencies pomRefId="build-deps-pom"
-                             filesetId="build-dependency-jars" 
-                             sourcesFilesetId="build-dependency-sources" 
-                             cacheDependencyRefs="true" 
+                             filesetId="build-dependency-jars"
+                             cacheDependencyRefs="true"
                              dependencyRefsBuildFile="${build.dir}/build-dependencies.xml">
           <remoteRepository refid="central"/>
           <remoteRepository refid="apache"/>
       </artifact:dependencies>
-      <artifact:dependencies pomRefId="coverage-deps-pom"
-                             pathId="cobertura.classpath">
+      <!-- retrieve -source.jar artifacts using the reference-pom with the artifacts that have these -->
+      <artifact:dependencies pomRefId="build-deps-pom-sources"
+                             sourcesFilesetId="build-dependency-sources"
+                             cacheDependencyRefs="true"
+                             dependencyRefsBuildFile="${build.dir}/build-dependencies-sources.xml">
           <remoteRepository refid="central"/>
+          <remoteRepository refid="apache"/>
       </artifact:dependencies>
-
       <copy todir="${build.dir.lib}/jars">
           <fileset refid="build-dependency-jars"/>
           <mapper type="flatten"/>
@@ -587,6 +674,23 @@
           <fileset refid="build-dependency-sources"/>
           <mapper type="flatten"/>
       </copy>
+      <!-- code coverage tools -->
+      <artifact:dependencies pomRefId="coverage-deps-pom"
+                             filesetId="coverage-dependency-jars"
+                             pathId="cobertura.classpath">
+          <remoteRepository refid="central"/>
+      </artifact:dependencies>
+      <copy todir="${build.dir.lib}/jars">
+          <fileset refid="coverage-dependency-jars"/>
+          <mapper type="flatten"/>
+      </copy>
+      <!-- jacoco agent jar comes wrapped in a jar -->
+      <unzip src="${build.dir.lib}/jars/org.jacoco.agent-${jacoco.version}.jar" dest="${build.dir.lib}/jars">
+        <patternset>
+            <include name="*.jar"/>
+        </patternset>
+        <mapper type="flatten"/>
+      </unzip>
     </target>
 
     <target name="maven-ant-tasks-retrieve-test" depends="maven-ant-tasks-init">
@@ -702,9 +806,9 @@
     -->
     <target name="build"
         depends="maven-ant-tasks-retrieve-build,build-project" description="Compile Cassandra classes"/>
-    <target name="codecoverage" depends="cobertura-instrument,test,cobertura-report" description="Create code coverage report"/>
+    <target name="codecoverage" depends="jacoco-run,jacoco-report" description="Create code coverage report"/>
 
-    <target depends="init,gen-cli-grammar,gen-cql2-grammar,gen-cql3-grammar"
+    <target depends="init,gen-cql3-grammar,generate-cql-html"
             name="build-project">
         <echo message="${ant.project.name}: ${ant.file}"/>
         <!-- Order matters! -->
@@ -723,8 +827,9 @@
                memorymaximumsize="512M">
             <src path="${build.src.java}"/>
             <src path="${build.src.gen-java}"/>
+        	<compilerarg value="-XDignore.symbol.file"/>
+                <compilerarg value="-Xbootclasspath/p:${build.src.jdkoverride}"/>
             <classpath refid="cassandra.classpath"/>
-            <compilerarg value="-XDignore.symbol.file"/>
         </javac>
         <antcall target="createVersionPropFile"/>
         <copy todir="${build.classes.main}">
@@ -751,9 +856,6 @@
                     <fileset dir="${build.lib}">
                         <include name="**/*.jar" />
                     </fileset>
-                    <fileset dir="${build.tools.lib}">
-                        <include name="**/*.jar" />
-                    </fileset>
                 </path>
             </classpath>
         </javac>
@@ -833,7 +935,8 @@
           <include name="org/apache/cassandra/utils/FBUtilities*.class" />
           <include name="org/apache/cassandra/exceptions/*.class" />
           <include name="org/apache/cassandra/utils/CloseableIterator.class" />
-          <include name="org/apache/cassandra/io/util/DataOutputPlus.class" />
+          <include name="org/apache/cassandra/io/util/*.class" />
+          <include name="org/apache/cassandra/utils/SigarLibrary.class" />
         </fileset>
         <manifest>
           <attribute name="Implementation-Title" value="Cassandra"/>
@@ -945,6 +1048,11 @@
       <copy todir="${dist.dir}/javadoc">
         <fileset dir="${javadoc.dir}"/>
       </copy>
+      <copy todir="${dist.dir}/doc">
+        <fileset dir="doc">
+          <exclude name="cql3/CQL.textile"/>
+        </fileset>
+      </copy>
       <copy todir="${dist.dir}/bin">
         <fileset dir="bin"/>
       </copy>
@@ -979,9 +1087,6 @@
         <fileset dir="${build.dir}/tools/lib/">
             <include name="*.jar" />
         </fileset>
-        <fileset dir="${build.tools.lib}">
-            <include name="*.jar" />
-        </fileset>
       </copy>
       <artifact:writepom pomRefId="dist-pom" 
             file="${build.dir}/${final.name}-dist.pom"/>
@@ -1038,7 +1143,7 @@
       </tar>
     </target>
 
-    <target name="release" depends="artifacts,rat-init"
+    <target name="release" depends="eclipse-warnings,artifacts,rat-init"
             description="Create and QC release artifacts">
       <checksum forceOverwrite="yes" todir="${build.dir}" fileext=".md5"
                 algorithm="MD5">
@@ -1078,15 +1183,18 @@
      debuglevel="${debuglevel}"
      encoding="utf-8"
      destdir="${test.classes}"
-     includeantruntime="false"
+     includeantruntime="true"
      source="${source.version}" 
      target="${target.version}">
-      <classpath>
+     <classpath>
         <path refid="cassandra.classpath"/>
-      </classpath>
-      <src path="${test.unit.src}"/>
-      <src path="${test.long.src}"/>
-      <src path="${test.pig.src}"/>
+     </classpath>
+	 <compilerarg value="-XDignore.symbol.file"/>
+     <src path="${test.unit.src}"/>
+     <src path="${test.long.src}"/>
+     <src path="${test.burn.src}"/>
+     <src path="${test.pig.src}"/>
+     <src path="${test.microbench.src}"/>
     </javac>
 
     <!-- Non-java resources needed by the test suite -->
@@ -1095,8 +1203,12 @@
     </copy>
   </target>
 
-  <macrodef name="testmacro">
-    <attribute name="suitename" />
+  <!-- Defines how to run a set of tests. If you change the defaults for attributes
+       you should also update them in testmacro.,
+       The two are split because the helper doesn't generate
+       a junit report or fail on errors, since this is called in parallel to run tests
+       when we choose to run tests in parallel -->
+  <macrodef name="testmacrohelper">
     <attribute name="inputdir" />
     <attribute name="timeout" default="${test.timeout}" />
     <attribute name="forkmode" default="perTest"/>
@@ -1105,24 +1217,36 @@
     <attribute name="exclude" default="" />
     <attribute name="filelist" default="" />
     <attribute name="poffset" default="0"/>
+    <attribute name="testtag" default=""/>
+    
+    <attribute name="usejacoco" default="no"/>
     <sequential>
-      <echo message="running @{suitename} tests"/>
+      <condition property="additionalagent"
+                 value="-javaagent:${build.dir.lib}/jars/jacocoagent.jar=destfile=${jacoco.partialexecfile}"
+                 else="">
+        <istrue value="${usejacoco}"/>
+      </condition>
       <mkdir dir="${build.test.dir}/cassandra"/>
       <mkdir dir="${build.test.dir}/output"/>
+      <mkdir dir="${build.test.dir}/output/@{testtag}"/>
       <junit fork="on" forkmode="@{forkmode}" failureproperty="testfailed" maxmemory="1024m" timeout="@{timeout}">
         <sysproperty key="net.sourceforge.cobertura.datafile" file="${cobertura.datafile}"/>
-        <formatter type="xml" usefile="true"/>
-        <formatter type="brief" usefile="false"/>
+        <formatter classname="org.apache.cassandra.CassandraXMLJUnitResultFormatter" extension=".xml" usefile="true"/>
+        <formatter classname="org.apache.cassandra.CassandraBriefJUnitResultFormatter" usefile="false"/>
         <jvmarg value="-Dstorage-config=${test.conf}"/>
         <jvmarg value="-Djava.awt.headless=true"/>
-        <jvmarg value="-javaagent:${basedir}/lib/jamm-0.3.0.jar" />
+        <!-- Cassandra 3.0+ needs <jvmarg line="... ${additionalagent}" /> here! (not value=) -->
+        <jvmarg line="-javaagent:${basedir}/lib/jamm-0.3.0.jar ${additionalagent}" />
         <jvmarg value="-ea"/>
         <jvmarg value="-Dcassandra.debugrefcount=true"/>
         <jvmarg value="-Xss256k"/>
         <jvmarg value="-Dcassandra.memtable_row_overhead_computation_step=100"/>
         <jvmarg value="-Dcassandra.test.use_prepared=${cassandra.test.use_prepared}"/>
-	<jvmarg value="-Dcassandra.jmx.local.port=7199" />
-	<jvmarg value="-Dcassandra.test.offsetseed=@{poffset}"/>        
+	    <jvmarg value="-Dcassandra.test.offsetseed=@{poffset}"/>
+        <jvmarg value="-Dcassandra.test.sstableformatdevelopment=true"/>
+        <jvmarg value="-Dcassandra.testtag=@{testtag}"/>
+        <!-- The first time SecureRandom initializes can be slow if it blocks on /dev/random -->
+        <jvmarg value="-Djava.security.egd=file:/dev/urandom" />
 	<optjvmargs/>
         <classpath>
           <path refid="cassandra.classpath" />
@@ -1133,27 +1257,14 @@
             <include name="**/*.jar" />
           </fileset>
         </classpath>
-        <batchtest todir="${build.test.dir}/output">
+        <batchtest todir="${build.test.dir}/output/@{testtag}">
             <fileset dir="@{inputdir}" includes="@{filter}" excludes="@{exclude}"/>
             <filelist dir="@{inputdir}" files="@{filelist}"/>
         </batchtest>
       </junit>
-      <junitreport todir="${build.test.dir}">
-        <fileset dir="${build.test.dir}/output">
-          <include name="TEST-*.xml"/>
-        </fileset>
-        <report format="frames" todir="${build.test.dir}/junitreport"/>
-      </junitreport>
-      <fail message="Some @{suitename} test(s) failed.">
-        <condition>
-            <and>
-            <isset property="testfailed"/>
-            <not>
-              <isset property="ant.test.failure.ignore"/>
-            </not>
-          </and>
-        </condition>
-      </fail>
+      <delete quiet="true" failonerror="false" dir="${build.test.dir}/cassandra/commitlog:@{poffset}"/>
+      <delete quiet="true" failonerror="false" dir="${build.test.dir}/cassandra/data:@{poffset}"/>
+      <delete quiet="true" failonerror="false" dir="${build.test.dir}/cassandra/saved_caches:@{poffset}"/>
     </sequential>
   </macrodef>
 
@@ -1165,6 +1276,8 @@
   -->
   <target name="test-clientutil-jar" depends="build-test,jar" description="Test clientutil jar">
     <junit fork="on" forkmode="perTest" failureproperty="testfailed" maxmemory="1024m" timeout="${test.timeout}">
+      <!-- Note that the test pass without that next line, but it prints an ugly error message -->
+      <jvmarg value="-Djava.library.path=${build.lib}/sigar-bin"/>
       <test name="org.apache.cassandra.serializers.ClientUtilsTest" />
       <formatter type="brief" usefile="false" />
       <classpath>
@@ -1172,11 +1285,13 @@
         <pathelement location="${build.dir}/${ant.project.name}-clientutil-${version}.jar" />
         <pathelement location="${build.dir}/${ant.project.name}-thrift-${version}.jar" />
         <pathelement location="${build.lib}/libthrift-0.9.0.jar" />
-        <pathelement location="${build.lib}/slf4j-api-1.7.2.jar" />
-        <pathelement location="${build.lib}/logback-core-1.1.2.jar" />
-        <pathelement location="${build.lib}/logback-classic-1.1.2.jar" />
+        <pathelement location="${build.lib}/slf4j-api-1.7.7.jar" />
+        <pathelement location="${build.lib}/log4j-over-slf4j.jar" />
+        <pathelement location="${build.lib}/logback-core-1.1.3.jar" />
+        <pathelement location="${build.lib}/logback-classic-1.1.3.jar" />
         <pathelement location="${build.lib}/jackson-core-asl-1.9.2.jar" />
         <pathelement location="${build.lib}/jackson-mapper-asl-1.9.2.jar" />
+        <pathelement location="${build.lib}/sigar-1.6.4.jar" />
         <fileset dir="${build.dir.lib}">
           <include name="**/junit*.jar" />
         </fileset>
@@ -1195,92 +1310,138 @@
   </target>
 
   <target name="testold" depends="build-test" description="Execute unit tests">
-    <testmacro suitename="unit" inputdir="${test.unit.src}" exclude="**/pig/*.java" timeout="${test.timeout}">
+    <testmacro inputdir="${test.unit.src}" exclude="**/pig/*.java" timeout="${test.timeout}">
       <jvmarg value="-Dlegacy-sstable-root=${test.data}/legacy-sstables"/>
+      <jvmarg value="-Dinvalid-legacy-sstable-root=${test.data}/invalid-legacy-sstables"/>
       <jvmarg value="-Dcorrupt-sstable-root=${test.data}/corrupt-sstables"/>
       <jvmarg value="-Dmigration-sstable-root=${test.data}/migration-sstables"/>
       <jvmarg value="-Dcassandra.ring_delay_ms=1000"/>
       <jvmarg value="-Dcassandra.tolerate_sstable_size=true"/>
       <jvmarg value="-Djava.io.tmpdir=${tmp.dir}"/>
+      <jvmarg value="-Dcassandra.skip_sync=true" />
     </testmacro>
     <fileset dir="${test.unit.src}">
         <exclude name="**/pig/*.java" />
     </fileset>
   </target>
   
-  <target name="testlist">
-    <testmacro suitename="${testlist.name}" inputdir="${test.unit.src}" filelist="${test.file.list}" poffset="${testlist.offset}" exclude="**/*.java" timeout="${test.timeout}">
-      <jvmarg value="-Dlegacy-sstable-root=${test.data}/legacy-sstables"/>
-      <jvmarg value="-Dcorrupt-sstable-root=${test.data}/corrupt-sstables"/>
-      <jvmarg value="-Dmigration-sstable-root=${test.data}/migration-sstables"/>
-      <jvmarg value="-Dcassandra.ring_delay_ms=1000"/>
-      <jvmarg value="-Dcassandra.tolerate_sstable_size=true"/>
-      <jvmarg value="-Dcassandra.config.loader=org.apache.cassandra.OffsetAwareConfigurationLoader"/>
-    </testmacro>
+  <!-- Will not generate a junit report or fail on error since it is called in parallel for test-compression
+       That is taken care of by testparallel -->
+  <macrodef name="testlist">
+    <attribute name="test.file.list"/>
+    <attribute name="testlist.offset"/>
+    <sequential>
+      <testmacrohelper inputdir="${test.unit.src}" filelist="@{test.file.list}" poffset="@{testlist.offset}" exclude="**/*.java" timeout="${test.timeout}">
+        <jvmarg value="-Dlegacy-sstable-root=${test.data}/legacy-sstables"/>
+        <jvmarg value="-Dinvalid-legacy-sstable-root=${test.data}/invalid-legacy-sstables"/>
+        <jvmarg value="-Dcorrupt-sstable-root=${test.data}/corrupt-sstables"/>
+        <jvmarg value="-Dmigration-sstable-root=${test.data}/migration-sstables"/>
+        <jvmarg value="-Dcassandra.ring_delay_ms=1000"/>
+        <jvmarg value="-Dcassandra.tolerate_sstable_size=true"/>
+        <jvmarg value="-Dcassandra.config.loader=org.apache.cassandra.OffsetAwareConfigurationLoader"/>
+        <jvmarg value="-Dcassandra.skip_sync=true" />
+      </testmacrohelper>
+    </sequential>
+  </macrodef>
+
+  <!-- Will not generate a junit report or fail on error since it is called in parallel for test-compression
+       That is taken care of by testparallel -->
+  <macrodef name="testlist-compression">
+    <attribute name="test.file.list" />
+    <attribute name="testlist.offset" />
+    <sequential>
+      <property name="compressed_yaml" value="${build.test.dir}/cassandra.compressed.yaml"/>
+      <testmacrohelper inputdir="${test.unit.src}" filelist="@{test.file.list}" poffset="@{testlist.offset}"
+                       exclude="**/*.java" timeout="${test.timeout}" testtag="compression">
+        <jvmarg value="-Dlegacy-sstable-root=${test.data}/legacy-sstables"/>
+        <jvmarg value="-Dinvalid-legacy-sstable-root=${test.data}/invalid-legacy-sstables"/>
+        <jvmarg value="-Dcorrupt-sstable-root=${test.data}/corrupt-sstables"/>
+        <jvmarg value="-Dmigration-sstable-root=${test.data}/migration-sstables"/>
+        <jvmarg value="-Dcassandra.test.compression=true"/>
+        <jvmarg value="-Dcassandra.ring_delay_ms=1000"/>
+        <jvmarg value="-Dcassandra.tolerate_sstable_size=true"/>
+        <jvmarg value="-Dcassandra.config=file:///${compressed_yaml}"/>
+        <jvmarg value="-Dcassandra.skip_sync=true" />
+        <jvmarg value="-Dcassandra.config.loader=org.apache.cassandra.OffsetAwareConfigurationLoader"/>
+      </testmacrohelper>
+    </sequential>
+  </macrodef>
+
+  <!--
+    Run named ant task with jacoco, such as "ant jacoco-run -Dtaskname=pig-test"
+    the target run must enable the jacoco agent if usejacoco is 'yes' -->
+  <target name="jacoco-run" description="run named task with jacoco instrumentation">
+    <condition property="runtask" value="${taskname}" else="test">
+      <isset property="taskname"/>
+    </condition>
+    <antcall target="${runtask}">
+      <param name="usejacoco" value="yes"/>
+    </antcall>
   </target>
 
   <!-- Use this with an FQDN for test class, and a csv list of methods like this:
     ant testsome -Dtest.name=org.apache.cassandra.service.StorageServiceServerTest -Dtest.methods=testRegularMode,testGetAllRangesEmpty
   -->
   <target name="testsome" depends="build-test" description="Execute specific unit tests" >
-    <testmacro suitename="unit" inputdir="${test.unit.src}" exclude="**/pig/*.java" timeout="${test.timeout}">
+    <testmacro inputdir="${test.unit.src}" exclude="**/pig/*.java" timeout="${test.timeout}">
       <test name="${test.name}" methods="${test.methods}"/>
       <jvmarg value="-Dlegacy-sstable-root=${test.data}/legacy-sstables"/>
+      <jvmarg value="-Dinvalid-legacy-sstable-root=${test.data}/invalid-legacy-sstables"/>
       <jvmarg value="-Dcorrupt-sstable-root=${test.data}/corrupt-sstables"/>
       <jvmarg value="-Dmigration-sstable-root=${test.data}/migration-sstables"/>
       <jvmarg value="-Dcassandra.ring_delay_ms=1000"/>
       <jvmarg value="-Dcassandra.tolerate_sstable_size=true"/>
+      <jvmarg value="-Dcassandra.skip_sync=true" />
     </testmacro>
   </target>
     
   <target name="test-compression" depends="build-test" description="Execute unit tests with sstable compression enabled">
-      <testmacro suitename="unit" inputdir="${test.unit.src}" exclude="**/pig/*.java" timeout="${test.timeout}">
-      <jvmarg value="-Dlegacy-sstable-root=${test.data}/legacy-sstables"/>
-      <jvmarg value="-Dcorrupt-sstable-root=${test.data}/corrupt-sstables"/>
-      <jvmarg value="-Dmigration-sstable-root=${test.data}/migration-sstables"/>
-      <jvmarg value="-Dcassandra.test.compression=true"/>
-      <jvmarg value="-Dcassandra.ring_delay_ms=1000"/>
-      <jvmarg value="-Dcassandra.tolerate_sstable_size=true"/>
-    </testmacro>
-    <fileset dir="${test.unit.src}">
-        <exclude name="**/pig/*.java" />
-    </fileset>
+    <property name="compressed_yaml" value="${build.test.dir}/cassandra.compressed.yaml"/>
+    <concat destfile="${compressed_yaml}">
+      <fileset file="${test.conf}/cassandra.yaml"/>
+      <fileset file="${test.conf}/commitlog_compression.yaml"/>
+    </concat>
+    <path id="all-test-classes-path">
+      <fileset dir="${test.unit.src}" excludes="**/pig/*.java" includes="**/${test.name}.java" />
+    </path>
+    <property name="all-test-classes" refid="all-test-classes-path"/>
+    <testparallel testdelegate="testlist-compression" />
   </target>
 
   <target name="msg-ser-gen-test" depends="build-test" description="Generates message serializations">
-    <testmacro suitename="unit" inputdir="${test.unit.src}" 
+    <testmacro inputdir="${test.unit.src}"
         timeout="${test.timeout}" filter="**/SerializationsTest.java">
       <jvmarg value="-Dcassandra.test-serialization-writes=True"/>
     </testmacro>
   </target>
   
   <target name="msg-ser-test" depends="build-test" description="Tests message serializations">
-      <testmacro suitename="unit" inputdir="${test.unit.src}" timeout="${test.timeout}"
+      <testmacro inputdir="${test.unit.src}" timeout="${test.timeout}"
                filter="**/SerializationsTest.java"/>
   </target>
   
   <target name="msg-ser-test-7" depends="build-test" description="Generates message serializations">
-    <testmacro suitename="unit" inputdir="${test.unit.src}" 
+    <testmacro inputdir="${test.unit.src}"
         timeout="${test.timeout}" filter="**/SerializationsTest.java">
       <jvmarg value="-Dcassandra.version=0.7"/>
     </testmacro>
   </target>
 
   <target name="msg-ser-test-10" depends="build-test" description="Tests message serializations on 1.0 messages">
-    <testmacro suitename="unit" inputdir="${test.unit.src}" 
+    <testmacro inputdir="${test.unit.src}"
         timeout="${test.timeout}" filter="**/SerializationsTest.java">
       <jvmarg value="-Dcassandra.version=1.0"/>
     </testmacro>
   </target>
 
   <target name="test-burn" depends="build-test" description="Execute functional tests">
-    <testmacro suitename="burn" inputdir="${test.burn.src}"
+    <testmacro inputdir="${test.burn.src}"
                timeout="${test.burn.timeout}">
     </testmacro>
   </target>
 
   <target name="long-test" depends="build-test" description="Execute functional tests">
-    <testmacro suitename="long" inputdir="${test.long.src}"
+    <testmacro inputdir="${test.long.src}"
                timeout="${test.long.timeout}">
       <jvmarg value="-Dcassandra.ring_delay_ms=1000"/>
       <jvmarg value="-Dcassandra.tolerate_sstable_size=true"/>
@@ -1301,6 +1462,7 @@
         <jvmarg value="-Xss256k"/>
         <jvmarg value="-Dcassandra.memtable_row_overhead_computation_step=100"/>
         <jvmarg value="-Dcassandra.test.use_prepared=${cassandra.test.use_prepared}"/>
+        <jvmarg value="-Dcassandra.skip_sync=true" />
         <classpath>
           <path refid="cassandra.classpath" />
           <pathelement location="${test.classes}"/>
@@ -1344,6 +1506,7 @@
         <jvmarg value="-Xss256k"/>
         <jvmarg value="-Dcassandra.test.use_prepared=${cassandra.test.use_prepared}"/>
         <jvmarg value="-Dcassandra.memtable_row_overhead_computation_step=100"/>
+        <jvmarg value="-Dcassandra.skip_sync=true" />
         <classpath>
           <path refid="cassandra.classpath" />
           <pathelement location="${test.classes}"/>
@@ -1359,14 +1522,58 @@
   </target>
 
   <target name="pig-test" depends="build-test,maven-ant-tasks-retrieve-pig-test" description="Excute Pig tests">
-    <testmacro suitename="pig" inputdir="${test.pig.src}" 
+    <testmacro inputdir="${test.pig.src}"
                timeout="1200000">
     </testmacro>
   </target>
 
   <target name="test-all" 
-          depends="test,long-test,test-compression,pig-test,test-clientutil-jar" 
-          description="Run all tests" />
+          depends="eclipse-warnings,test,long-test,test-compression,pig-test,test-clientutil-jar" 
+          description="Run all tests except for those under test-burn" />
+  
+  <!-- Use JaCoCo ant extension without needing externally saved lib -->
+  <target name="jacoco-init" depends="maven-ant-tasks-init">
+    <artifact:dependencies pathId="jacocoant.classpath">
+      <dependency groupId="org.jacoco" artifactId="org.jacoco.ant" version="${jacoco.version}" />
+    </artifact:dependencies>
+    <typedef uri="antlib:org.jacoco.ant" classpathref="jacocoant.classpath"/>
+  </target>
+
+  <target name="jacoco-merge" depends="jacoco-init">
+    <jacoco:merge destfile="${jacoco.finalexecfile}" xmlns:jacoco="antlib:org.jacoco.ant">
+        <fileset dir="${jacoco.export.dir}" includes="*.exec,**/*.exec"/>
+    </jacoco:merge>
+  </target>
+
+  <target name="jacoco-report" depends="jacoco-merge">
+    <jacoco:report xmlns:jacoco="antlib:org.jacoco.ant">
+      <executiondata>
+        <file file="${jacoco.finalexecfile}" />
+      </executiondata>
+      <structure name="JaCoCo Cassandara Coverage Report">
+        <classfiles>
+          <fileset dir="${build.classes.main}">
+            <include name="**/*.class"/>
+          </fileset>
+        </classfiles>
+        <sourcefiles encoding="UTF-8">
+          <dirset dir="${build.src}">
+            <include name="java"/>
+            <include name="gen-java"/>
+          </dirset>
+        </sourcefiles>
+      </structure>
+      <!-- to produce reports in different formats. -->
+      <html destdir="${jacoco.export.dir}" />
+      <csv destfile="${jacoco.export.dir}/report.csv" />
+      <xml destfile="${jacoco.export.dir}/report.xml" />
+    </jacoco:report>
+  </target>
+
+  <target name="jacoco-cleanup" description="Destroy JaCoCo exec data and reports">
+    <delete file="${jacoco.partialexecfile}"/>
+    <delete dir="${jacoco.export.dir}"/>
+  </target>
 
   <!-- instruments the classes to later create code coverage reports -->
   <target name="cobertura-instrument" depends="build,build-test">
@@ -1437,7 +1644,7 @@
     </java>
   </target>
 
-  <target name="javadoc" depends="init" description="Create javadoc">
+  <target name="javadoc" depends="init" description="Create javadoc" unless="no-javadoc">
     <create-javadoc destdir="${javadoc.dir}">
       <filesets>
       <fileset dir="${build.src.java}" defaultexcludes="yes">
@@ -1450,163 +1657,146 @@
     </create-javadoc>
    </target>
 
-  <!-- Split test classes into n buckets and run across processes -->
+  <!-- Run tests not in parallel and reports errors and generates a junit report after -->
+  <macrodef name="testmacro">
+    <attribute name="inputdir" />
+    <attribute name="timeout" default="${test.timeout}" />
+    <attribute name="forkmode" default="perTest"/>
+    <element name="optjvmargs" implicit="true" optional="true" />
+    <attribute name="filter" default="**/${test.name}.java"/>
+    <attribute name="exclude" default="" />
+    <attribute name="filelist" default="" />
+    <attribute name="poffset" default="0"/>
+    <attribute name="testtag" default=""/>
+
+    <sequential>
+      <testmacrohelper inputdir="@{inputdir}" timeout="@{timeout}"
+                       forkmode="@{forkmode}" filter="@{filter}"
+                       exclude="@{exclude}" filelist="@{filelist}" poffset="@{poffset}"
+                       testtag="@{testtag}" >
+          <optjvmargs/>
+      </testmacrohelper>
+      <junitreport todir="${build.test.dir}">
+        <fileset dir="${build.test.dir}/output">
+          <include name="**/TEST-*.xml"/>
+        </fileset>
+        <report format="frames" todir="${build.test.dir}/junitreport"/>
+      </junitreport>
+      <fail message="Some test(s) failed.">
+        <condition>
+            <and>
+            <isset property="testfailed"/>
+            <not>
+              <isset property="ant.test.failure.ignore"/>
+            </not>
+          </and>
+        </condition>
+      </fail>
+    </sequential>
+  </macrodef>
+
+  <!-- Run tests in parallel and report errors after and generate a junit report -->
+  <macrodef name="testparallel">
+    <attribute name="testdelegate"/>
+    <sequential>
+      <testparallelhelper testdelegate="@{testdelegate}"/>
+      <junitreport todir="${build.test.dir}">
+        <fileset dir="${build.test.dir}/output">
+          <include name="**/TEST-*.xml"/>
+        </fileset>
+        <report format="frames" todir="${build.test.dir}/junitreport"/>
+      </junitreport>
+      <fail message="Some test(s) failed.">
+        <condition>
+            <and>
+            <isset property="testfailed"/>
+            <not>
+              <isset property="ant.test.failure.ignore"/>
+            </not>
+          </and>
+        </condition>
+      </fail>
+    </sequential>
+  </macrodef>
+
+  <!-- Run multiple junit tasks in parallel, but don't track errors or generate a report after
+       If a test fails the testfailed property will be set. All the tests are run using te testdelegate
+       macro that is specified as an attribute and they will be run concurrently in this ant process -->
+  <scriptdef name="testparallelhelper" language="javascript">
+    <attribute name="testdelegate"/>
+    <![CDATA[
+        var Integer = java.lang.Integer;
+        sep = project.getProperty("path.separator");
+        all = project.getProperty("all-test-classes").split(sep);
+        dir = project.getProperty("test.unit.src");
+
+        numRunners = parseInt(project.getProperty("test.runners"));
+
+        var p = project.createTask('parallel');
+        p.setThreadCount(numRunners);
+
+        for (i = 0; i < all.length; i++) {
+
+            if (all[i] == undefined) continue;
+
+            task = project.createTask( attributes.get("testdelegate") );
+
+            task.setDynamicAttribute( "test.file.list", "" + all[i]);
+
+            task.setDynamicAttribute( "testlist.offset", "" + i );
+
+            p.addTask(task);
+        }
+
+        p.perform();
+    ]]>
+  </scriptdef>
+
   <target name="test" depends="build-test" description="Parallel Test Runner">
     <path id="all-test-classes-path">
-      <fileset dir="${test.unit.src}" excludes="**/pig/*.java" includes="**/${test.name}.java" />   
+      <fileset dir="${test.unit.src}" excludes="**/pig/*.java" includes="**/${test.name}.java" />
     </path>
     <property name="all-test-classes" refid="all-test-classes-path"/>
-    <script language="javascript"> <![CDATA[
-	var Integer = java.lang.Integer;
-	sep = project.getProperty("path.separator");
-	all = project.getProperty("all-test-classes").split(sep);
-	dir = project.getProperty("test.unit.src");
-
-	numRunners = parseInt(project.getProperty("test.runners"));  	
-	
-	buckets = new Array(numRunners);
-	for (i = 0; i < all.length; i++) {
-	    bucketNum = i % numRunners;
-	    if (buckets[bucketNum] == undefined) 
-		buckets[bucketNum] = "";
-	    else
-		buckets[bucketNum] += ",";
-	
-	    buckets[bucketNum] += all[i];
-	}
-
-
-	var p = project.createTask('parallel');
-	p.setThreadCount(numRunners);
-
-  	for (i = 0; i < buckets.length; i++) {
-
-	    if (buckets[i] == undefined) continue;
-
-	    task = project.createTask( 'antcall' );
-
-	    task.setTarget("testlist");
-	    param = task.createParam();
-	    param.setName("test.file.list");
-	    param.setValue("" + buckets[i]);
-
-	    param = task.createParam();
-	    param.setName("testlist.name");
-	    param.setValue("test bucket "+i);	  
-
-	    param = task.createParam();
-	    param.setName("testlist.offset");
-	    param.setValue("" + i);
-
-	    p.addTask(task); 
-  	}
-	
-	p.perform();
-	  			    
-]]> </script>
+    <testparallel testdelegate="testlist"/>
   </target>
 
-  <pathconvert property="eclipse.project.name">
-    <path path="${basedir}" />
-    <regexpmapper from="^.*/([^/]+)$$" to="\1" handledirsep="yes" />
-  </pathconvert>
+  <!-- run a list of tests as provided in -Dtest.classlistfile (or default of 'testnames.txt')
+  The class list file should be one test class per line, with the path starting after test/unit
+  e.g. org/apache/cassandra/hints/HintMessageTest.java -->
+  <target name="testclasslist" depends="build-test" description="Parallel-run tests given in file -Dtest.classlistfile (one-class-per-line, e.g. org/apache/cassandra/db/SomeTest.java)">
+    <path id="all-test-classes-path">
+      <fileset dir="${test.unit.src}" includesfile="${test.classlistfile}"/>
+    </path>
+    <property name="all-test-classes" refid="all-test-classes-path"/>
+    <testparallel testdelegate="testlist"/>
+  </target>
+
+  <!-- run microbenchmarks suite -->
+  <target name="microbench" depends="build-test">
+      <java classname="org.openjdk.jmh.Main"
+            fork="true"
+            failonerror="true">
+          <classpath>
+              <path refid="cassandra.classpath" />
+              <pathelement location="${test.classes}"/>
+              <path refid="cobertura.classpath"/>
+              <pathelement location="${test.conf}"/>
+              <fileset dir="${test.lib}">
+                  <include name="**/*.jar" />
+              </fileset>
+          </classpath>
+          <arg value=".*microbench.*${benchmark.name}"/>
+      </java>
+  </target>
 
   <!-- Generate IDEA project description files -->
   <target name="generate-idea-files" depends="build-test" description="Generate IDEA files">
-    <echo file="${eclipse.project.name}.iml"><![CDATA[<?xml version="1.0" encoding="UTF-8"?>
-<module type="JAVA_MODULE" version="4">
-  <component name="NewModuleRootManager" LANGUAGE_LEVEL="JDK_1_7" inherit-compiler-output="false">
-    <output url="file://$MODULE_DIR$/.idea/out/main" />
-    <output-test url="file://$MODULE_DIR$/.idea/out/test" />
-    <exclude-output />
-    <content url="file://$MODULE_DIR$">
-      <sourceFolder url="file://$MODULE_DIR$/src/java" isTestSource="false" />
-      <sourceFolder url="file://$MODULE_DIR$/src/gen-java" isTestSource="false" />
-      <sourceFolder url="file://$MODULE_DIR$/src/resources" type="java-resource" />
-      <sourceFolder url="file://$MODULE_DIR$/interface/thrift/gen-java" isTestSource="false" />
-      <sourceFolder url="file://$MODULE_DIR$/tools/stress/src" isTestSource="false" />
-      <sourceFolder url="file://$MODULE_DIR$/test/unit" isTestSource="true" />
-      <sourceFolder url="file://$MODULE_DIR$/test/long" isTestSource="true" />
-      <sourceFolder url="file://$MODULE_DIR$/test/pig" isTestSource="true" />
-      <sourceFolder url="file://$MODULE_DIR$/test/microbench" isTestSource="true" />
-      <sourceFolder url="file://$MODULE_DIR$/test/conf" type="java-test-resource" />
-      <sourceFolder url="file://$MODULE_DIR$/test/resources" type="java-test-resource" />
-      <excludeFolder url="file://$MODULE_DIR$/.idea" />
-      <excludeFolder url="file://$MODULE_DIR$/.settings" />
-      <excludeFolder url="file://$MODULE_DIR$/build" />
-      <excludeFolder url="file://$MODULE_DIR$/data"/>
-      <excludeFolder url="file://$MODULE_DIR$/logs"/>
-    </content>
-    <orderEntry type="inheritedJdk" />
-    <orderEntry type="sourceFolder" forTests="false" />]]>
-    </echo>
-
-  	<path id="idea-project-libs-path">
-  	 <fileset dir="lib">
-  	    <include name="**/*.jar" />
-     </fileset>
- 	 <fileset dir="build/lib/jars">
-  	    <include name="**/*.jar" />
-  	 </fileset>
-  	 <fileset dir="tools/lib">
-  	     <include name="**/*.jar" />
-  	 </fileset>
-  	</path>
     <mkdir dir=".idea"/>
     <mkdir dir=".idea/libraries"/>
     <copy todir=".idea">
         <fileset dir="ide/idea"/>
     </copy>
-  	<property name="idea-project-libs" refid="idea-project-libs-path"/>
-  	<script language="javascript" classpathref="cassandra.classpath"> <![CDATA[
-  		var File = java.io.File;
-  		var FilenameUtils = Packages.org.apache.commons.io.FilenameUtils;
-  		var basedir = project.getProperty("basedir");
-  		jars = project.getProperty("idea-project-libs").split(project.getProperty("path.separator"));
-
-  		cp = "\n";
-  	    for (i=0; i< jars.length; i++) {
-  	       basename = FilenameUtils.getBaseName(jars[i]);
-  	       srcjar = basename + '-sources.jar';
-  		   srcdir = FilenameUtils.concat(project.getProperty("build.dir.lib"), 'sources');
-  		   srcfile = new File(FilenameUtils.concat(srcdir, srcjar));
-
-  		   cp += '    <orderEntry type="library" name="' + basename + '" level="project" />\n';
-
-  		   lib = '';
-           lib += '<component name="libraryTable">\n';
-           lib += '  <library name="' + basename + '">\n';
-           lib += '    <CLASSES>\n';
-           lib += '      <root url="jar://' + jars[i].replace(basedir,"$PROJECT_DIR$") + '!/" />\n';
-           lib += '    </CLASSES>\n';
-           lib += '    <JAVADOC />\n';
-  		   if (srcfile.exists()) {
-               lib += '    <SOURCES>\n';
-               lib += '      <root url="jar://' + srcfile.getAbsolutePath().replace(basedir,"$PROJECT_DIR$") + '!/" />\n';
-               lib += '    </SOURCES>\n';
-           } else {
-               lib += '    <SOURCES />\n';
-           }
-           lib += '  </library>\n';
-           lib += '</component>';
-
-           echo = project.createTask("echo");
-           echo.setMessage(lib);
-           echo.setFile(new File(".idea/libraries/" + basename.replace('-','_').replace('.','_') + ".xml"));
-           echo.setAppend(false);
-           echo.perform();
-  		}
-
-  		echo = project.createTask("echo");
-  	    echo.setMessage(cp);
-  		echo.setFile(new File(project.getProperty("eclipse.project.name") + ".iml"));
-  		echo.setAppend(true);
-  	    echo.perform();
-  	]]> </script>
-    <echo file="${eclipse.project.name}.iml" append="true"><![CDATA[
-    </component>
-</module>]]>
-    </echo>
+    <copy tofile="${eclipse.project.name}.iml" file="ide/idea-iml-file.xml"/>
     <echo file=".idea/.name">Apache Cassandra ${eclipse.project.name}</echo>
     <echo file=".idea/modules.xml"><![CDATA[<?xml version="1.0" encoding="UTF-8"?>
 <project version="4">
@@ -1616,101 +1806,6 @@
     </modules>
   </component>
 </project>]]></echo>
-    <echo file=".idea/misc.xml"><![CDATA[<?xml version="1.0" encoding="UTF-8"?>
-<project version="4">
-<component name="ProjectRootManager" version="2" languageLevel="JDK_1_7" default="false" assert-keyword="true" jdk-15="true" project-jdk-name="1.7" project-jdk-type="JavaSDK" />
-</project>]]></echo>
-    <echo file=".idea/vcs.xml"><![CDATA[<?xml version="1.0" encoding="UTF-8"?>
-<project version="4">
-  <component name="VcsDirectoryMappings">
-    <mapping directory="$PROJECT_DIR$" vcs="Git" />
-  </component>
-</project>]]></echo>
-    <echo file=".idea/workspace.xml"><![CDATA[<?xml version="1.0" encoding="UTF-8"?>
-<project version="4">
-    <component name="RunManager" selected="Application.Cassandra">
-        <configuration default="true" type="Application" factoryName="Application">
-          <extension name="coverage" enabled="false" merge="false" sample_coverage="true" runner="idea" />
-          <option name="MAIN_CLASS_NAME" value="" />
-          <option name="VM_PARAMETERS" value="-Dcassandra.config=file://$PROJECT_DIR$/conf/cassandra.yaml -Dcassandra.storagedir=$PROJECT_DIR$/data -Dlogback.configurationFile=file://$PROJECT_DIR$/conf/logback.xml -Dcassandra.logdir=$PROJECT_DIR$/data/logs -ea" />
-          <option name="PROGRAM_PARAMETERS" value="" />
-          <option name="WORKING_DIRECTORY" value="" />
-          <option name="ALTERNATIVE_JRE_PATH_ENABLED" value="false" />
-          <option name="ALTERNATIVE_JRE_PATH" value="" />
-          <option name="ENABLE_SWING_INSPECTOR" value="false" />
-          <option name="ENV_VARIABLES" />
-          <option name="PASS_PARENT_ENVS" value="true" />
-          <module name="" />
-          <envs />
-          <method>
-            <option name="AntTarget" enabled="true" antfile="file://$PROJECT_DIR$/build.xml" target="gen-cql3-grammar" />
-            <option name="AntTarget" enabled="true" antfile="file://$PROJECT_DIR$/build.xml" target="gen-thrift-java" />
-            <option name="Make" enabled="true" />
-          </method>
-        </configuration>
-        <configuration default="true" type="JUnit" factoryName="JUnit">
-          <extension name="coverage" enabled="false" merge="false" sample_coverage="true" runner="idea" />
-          <module name="" />
-          <option name="ALTERNATIVE_JRE_PATH_ENABLED" value="false" />
-          <option name="ALTERNATIVE_JRE_PATH" value="" />
-          <option name="PACKAGE_NAME" />
-          <option name="MAIN_CLASS_NAME" value="" />
-          <option name="METHOD_NAME" value="" />
-          <option name="TEST_OBJECT" value="class" />
-          <option name="VM_PARAMETERS" value="-Dcassandra.config=file://$PROJECT_DIR$/test/conf/cassandra.yaml -Dlogback.configurationFile=file://$PROJECT_DIR$/test/conf/logback-test.xml -Dcassandra.logdir=$PROJECT_DIR$/build/test/logs -ea" />
-          <option name="PARAMETERS" value="" />
-          <option name="WORKING_DIRECTORY" value="" />
-          <option name="ENV_VARIABLES" />
-          <option name="PASS_PARENT_ENVS" value="true" />
-          <option name="TEST_SEARCH_SCOPE">
-            <value defaultName="singleModule" />
-          </option>
-          <envs />
-          <patterns />
-          <method>
-            <option name="AntTarget" enabled="true" antfile="file://$PROJECT_DIR$/build.xml" target="gen-cql3-grammar" />
-            <option name="AntTarget" enabled="true" antfile="file://$PROJECT_DIR$/build.xml" target="gen-thrift-java" />
-            <option name="Make" enabled="true" />
-          </method>
-        </configuration>
-        <configuration default="false" name="Cassandra" type="Application" factoryName="Application">
-          <extension name="coverage" enabled="false" merge="false" sample_coverage="true" runner="idea" />
-          <option name="MAIN_CLASS_NAME" value="org.apache.cassandra.service.CassandraDaemon" />
-          <option name="VM_PARAMETERS" value="-Dcassandra-foreground=yes -Dcassandra.config=file://$PROJECT_DIR$/conf/cassandra.yaml -Dcassandra.storagedir=$PROJECT_DIR$/data -Dlogback.configurationFile=file://$PROJECT_DIR$/conf/logback.xml -Dcassandra.logdir=$PROJECT_DIR$/data/logs -Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.port=7199 -Dcom.sun.management.jmxremote.local.only=false -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.ssl=false -ea -Xmx1G" />
-          <option name="PROGRAM_PARAMETERS" value="" />
-          <option name="WORKING_DIRECTORY" value="file://$PROJECT_DIR$" />
-          <option name="ALTERNATIVE_JRE_PATH_ENABLED" value="false" />
-          <option name="ALTERNATIVE_JRE_PATH" value="" />
-          <option name="ENABLE_SWING_INSPECTOR" value="false" />
-          <option name="ENV_VARIABLES" />
-          <option name="PASS_PARENT_ENVS" value="true" />
-          <module name="]]>${eclipse.project.name}<![CDATA[" />
-          <envs />
-          <RunnerSettings RunnerId="Debug">
-            <option name="DEBUG_PORT" value="" />
-            <option name="TRANSPORT" value="0" />
-            <option name="LOCAL" value="true" />
-          </RunnerSettings>
-          <RunnerSettings RunnerId="Run" />
-          <ConfigurationWrapper RunnerId="Debug" />
-          <ConfigurationWrapper RunnerId="Run" />
-          <method>
-            <option name="AntTarget" enabled="true" antfile="file://$PROJECT_DIR$/build.xml" target="gen-cql3-grammar" />
-            <option name="AntTarget" enabled="true" antfile="file://$PROJECT_DIR$/build.xml" target="gen-thrift-java" />
-            <option name="Make" enabled="true" />
-          </method>
-        </configuration>
-    </component>
-    <component name="antWorkspaceConfiguration">
-        <option name="IS_AUTOSCROLL_TO_SOURCE" value="false" />
-        <option name="FILTER_TARGETS" value="false" />
-        <buildFile url="file://$PROJECT_DIR$/build.xml">
-            <expanded value="true" />
-        </buildFile>
-  </component>
-</project>
-    ]]>
-    </echo>
   </target>
 
   <!-- Generate Eclipse project description files -->
@@ -1736,6 +1831,7 @@
   <classpathentry kind="src" path="src/java"/>
   <classpathentry kind="src" path="src/resources"/>
   <classpathentry kind="src" path="src/gen-java"/>
+  <classpathentry kind="src" path="interface/thrift/gen-java"/>
   <classpathentry kind="src" output="build/test/classes" path="test/unit"/>
   <classpathentry kind="src" output="build/test/classes" path="test/long"/>
   <classpathentry kind="src" output="build/test/classes" path="test/pig"/>
@@ -1754,9 +1850,6 @@
  	 <fileset dir="build/lib/jars">
   	    <include name="**/*.jar" />
   	 </fileset>
-  	 <fileset dir="tools/lib">
-  	     <include name="**/*.jar" />
-  	 </fileset>
   	</path>
   	<property name="eclipse-project-libs" refid="eclipse-project-libs-path"/>
   	<script language="javascript" classpathref="cassandra.classpath"> <![CDATA[
@@ -1802,6 +1895,40 @@
   	<delete dir="build/eclipse-classes" />
   </target>
 
+
+  <target name="eclipse-warnings" depends="build" description="Run eclipse compiler code analysis">        
+        <property name="ecj.log.dir" value="${build.dir}/ecj" />
+        <property name="ecj.warnings.file" value="${ecj.log.dir}/eclipse_compiler_checks.txt"/>
+        <delete dir="${ecj.log.dir}" />
+        <mkdir  dir="${ecj.log.dir}" />
+
+        <property name="ecj.properties" value="${basedir}/eclipse_compiler.properties" />                
+
+        <echo message="Running Eclipse Code Analysis.  Output logged to ${ecj.warnings.file}" />        
+        
+	<java 
+	    jar="${build.dir.lib}/jars/ecj-${ecj.version}.jar"
+            fork="true"
+	    failonerror="true"
+            maxmemory="512m"> 
+            <arg value="-source"/>
+	    <arg value="${source.version}" /> 
+	    <arg value="-target"/>
+	    <arg value="${target.version}" /> 
+	    <arg value="-d" />
+            <arg value="none" />
+	    <arg value="-proc:none" /> 
+            <arg value="-log" />
+            <arg value="${ecj.warnings.file}" /> 
+            <arg value="-properties" />
+            <arg value="${ecj.properties}" />
+            <arg value="-cp" />
+            <arg value="${toString:cassandra.classpath}" />
+            <arg value="${build.src.java}" />
+        </java>
+  </target>
+  
+
   <!-- Publish artifacts to Maven repositories -->
   <target name="mvn-install"
           depends="maven-declare-dependencies,artifacts,jar,sources-jar,javadoc-jar"
diff --git a/conf/cassandra-env.ps1 b/conf/cassandra-env.ps1
index 35da856..321a9ca 100644
--- a/conf/cassandra-env.ps1
+++ b/conf/cassandra-env.ps1
@@ -197,6 +197,42 @@
 }

 

 #-----------------------------------------------------------------------------

+Function SetJsr223Env

+{

+    $cp = $env:CLASSPATH

+    foreach ($jsrDir in Get-ChildItem -Path "$env:CASSANDRA_HOME\lib\jsr223")

+    {

+        foreach ($file in Get-ChildItem -Path "$env:CASSANDRA_HOME\lib\jsr223\$jsrDir\*.jar")

+        {

+            $file = $file -replace "\\", "/"

+			$cp = $cp + ";" + """$file"""

+        }

+    }

+    $env:CLASSPATH=$cp

+

+	# JSR223/JRuby - set ruby lib directory

+	if (Test-Path "$env:CASSANDRA_HOME\lib\jsr223\jruby\ruby")

+	{

+		$env:CASSANDRA_PARAMS=$env:CASSANDRA_PARAMS + " -Djruby.lib=$env:CASSANDRA_HOME\lib\jsr223\jruby"

+	}

+	# JSR223/JRuby - set ruby JNI libraries root directory

+	if (Test-Path "$env:CASSANDRA_HOME\lib\jsr223\jruby\jni")

+	{

+		$env:CASSANDRA_PARAMS=$env:CASSANDRA_PARAMS + " -Djffi.boot.library.path=$env:CASSANDRA_HOME\lib\jsr223\jruby\jni"

+	}

+	# JSR223/Jython - set python.home system property

+	if (Test-Path "$env:CASSANDRA_HOME\lib\jsr223\jython\jython.jar")

+	{

+		$env:CASSANDRA_PARAMS=$env:CASSANDRA_PARAMS + " -Dpython.home=$env:CASSANDRA_HOME\lib\jsr223\jython"

+	}

+	# JSR223/Scala - necessary system property

+	if (Test-Path "$env:CASSANDRA_HOME\lib\jsr223\scala\scala-compiler.jar")

+	{

+		$env:CASSANDRA_PARAMS=$env:CASSANDRA_PARAMS + " -Dscala.usejavacp=true"

+	}

+}

+

+#-----------------------------------------------------------------------------

 Function ParseJVMInfo

 {

     # grab info about the JVM

@@ -205,13 +241,28 @@
     $pinfo.RedirectStandardError = $true

     $pinfo.RedirectStandardOutput = $true

     $pinfo.UseShellExecute = $false

-    $pinfo.Arguments = "-version"

+    $pinfo.Arguments = "-d64 -version"

     $p = New-Object System.Diagnostics.Process

     $p.StartInfo = $pinfo

     $p.Start() | Out-Null

     $p.WaitForExit()

     $stderr = $p.StandardError.ReadToEnd()

 

+    $env:JVM_ARCH = "64-bit"

+

+    if ($stderr.Contains("Error"))

+    {

+        # 32-bit JVM. re-run w/out -d64

+        echo "Failed 64-bit check. Re-running to get version from 32-bit"

+        $pinfo.Arguments = "-version"

+        $p = New-Object System.Diagnostics.Process

+        $p.StartInfo = $pinfo

+        $p.Start() | Out-Null

+        $p.WaitForExit()

+        $stderr = $p.StandardError.ReadToEnd()

+        $env:JVM_ARCH = "32-bit"

+    }

+

     $sa = $stderr.Split("""")

     $env:JVM_VERSION = $sa[1]

 

@@ -230,29 +281,11 @@
 

     $pa = $sa[1].Split("_")

     $env:JVM_PATCH_VERSION=$pa[1]

-

-    # get 64-bit vs. 32-bit

-    $pinfo.Arguments = "-d64 -version"

-    $pArch = New-Object System.Diagnostics.Process

-    $p.StartInfo = $pinfo

-    $p.Start() | Out-Null

-    $p.WaitForExit()

-    $stderr = $p.StandardError.ReadToEnd()

-

-    if ($stderr.Contains("Error"))

-    {

-        $env:JVM_ARCH = "32-bit"

-    }

-    else

-    {

-        $env:JVM_ARCH = "64-bit"

-    }

 }

 

 #-----------------------------------------------------------------------------

 Function SetCassandraEnvironment

 {

-    echo "Setting up Cassandra environment"

     if (Test-Path Env:\JAVA_HOME)

     {

         $env:JAVA_BIN = "$env:JAVA_HOME\bin\java.exe"

@@ -276,6 +309,7 @@
 

     SetCassandraMain

     BuildClassPath

+    SetJsr223Env

 

     # Override these to set the amount of memory to allocate to the JVM at

     # start-up. For production use you may wish to adjust this for your

@@ -297,6 +331,29 @@
     CalculateHeapSizes

 

     ParseJVMInfo

+    # Add sigar env - see Cassandra-7838

+    $env:JVM_OPTS = "$env:JVM_OPTS -Djava.library.path=""$env:CASSANDRA_HOME\lib\sigar-bin"""

+

+    # Confirm we're on high performance power plan, warn if not

+    # Change to $true to suppress this warning

+    $suppressPowerWarning = $false

+    if (!$suppressPowerWarning)

+    {

+        $currentProfile = powercfg /GETACTIVESCHEME

+        if (!$currentProfile.Contains("High performance"))

+        {

+            echo "*---------------------------------------------------------------------*"

+            echo "*---------------------------------------------------------------------*"

+            echo ""

+            echo "    WARNING! Detected a power profile other than High Performance."

+            echo "    Performance of this node will suffer."

+            echo "    Modify conf\cassandra.env.ps1 to suppress this warning."

+            echo ""

+            echo "*---------------------------------------------------------------------*"

+            echo "*---------------------------------------------------------------------*"

+        }

+    }

+

     # add the jamm javaagent

     if (($env:JVM_VENDOR -ne "OpenJDK") -or ($env:JVM_VERSION.CompareTo("1.6.0") -eq 1) -or

         (($env:JVM_VERSION -eq "1.6.0") -and ($env:JVM_PATCH_VERSION.CompareTo("22") -eq 1)))

@@ -315,8 +372,6 @@
     # store in env to check if it's avail in verification

     $env:JMX_PORT=$JMX_PORT

 

-    $env:JVM_OPTS = "$env:JVM_OPTS -Dlog4j.defaultInitOverride=true"

-

     # some JVMs will fill up their heap when accessed via JMX, see CASSANDRA-6541

     $env:JVM_OPTS="$env:JVM_OPTS -XX:+CMSClassUnloadingEnabled"

 

@@ -349,6 +404,7 @@
     $env:JVM_OPTS="$env:JVM_OPTS -XX:MaxTenuringThreshold=1"

     $env:JVM_OPTS="$env:JVM_OPTS -XX:CMSInitiatingOccupancyFraction=75"

     $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseCMSInitiatingOccupancyOnly"

+    $env:JVM_OPTS="$env:JVM_OPTS -XX:+PerfDisableSharedMem"

     $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseTLAB"

     if (($env:JVM_VERSION.CompareTo("1.7") -eq 1) -and ($env:JVM_ARCH -eq "64-Bit"))

     {

@@ -360,24 +416,23 @@
         $env:JVM_OPTS="$env:JVM_OPTS -XX:+CMSParallelInitialMarkEnabled -XX:+CMSEdenChunksRecordAlways"

     }

 

-    # GC logging options -- uncomment to enable

-    # $env:JVM_OPTS="$env:JVM_OPTS -XX:+PrintGCDetails"

-    # $env:JVM_OPTS="$env:JVM_OPTS -XX:+PrintGCDateStamps"

-    # $env:JVM_OPTS="$env:JVM_OPTS -XX:+PrintHeapAtGC"

-    # $env:JVM_OPTS="$env:JVM_OPTS -XX:+PrintTenuringDistribution"

-    # $env:JVM_OPTS="$env:JVM_OPTS -XX:+PrintGCApplicationStoppedTime"

-    # $env:JVM_OPTS="$env:JVM_OPTS -XX:+PrintPromotionFailure"

+    # GC logging options

+    $env:JVM_OPTS="$env:JVM_OPTS -XX:+PrintGCDetails"

+    $env:JVM_OPTS="$env:JVM_OPTS -XX:+PrintGCDateStamps"

+    $env:JVM_OPTS="$env:JVM_OPTS -XX:+PrintHeapAtGC"

+    $env:JVM_OPTS="$env:JVM_OPTS -XX:+PrintTenuringDistribution"

+    $env:JVM_OPTS="$env:JVM_OPTS -XX:+PrintGCApplicationStoppedTime"

+    $env:JVM_OPTS="$env:JVM_OPTS -XX:+PrintPromotionFailure"

     # $env:JVM_OPTS="$env:JVM_OPTS -XX:PrintFLSStatistics=1"

+

+    $env:JVM_OPTS="$env:JVM_OPTS -Xloggc:""$env:CASSANDRA_HOME/logs/gc.log"""

+    $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseGCLogFileRotation"

+    $env:JVM_OPTS="$env:JVM_OPTS -XX:NumberOfGCLogFiles=10"

+    $env:JVM_OPTS="$env:JVM_OPTS -XX:GCLogFileSize=10M"

+    # if using version before JDK 6u34 or 7u2 use this instead of log rotation

     # $currentDate = (Get-Date).ToString('yyyy.MM.dd')

     # $env:JVM_OPTS="$env:JVM_OPTS -Xloggc:$env:CASSANDRA_HOME/logs/gc-$currentDate.log"

 

-    # If you are using JDK 6u34 7u2 or later you can enable GC log rotation

-    # don't stick the date in the log name if rotation is on.

-    # $env:JVM_OPTS="$env:JVM_OPTS -Xloggc:$env:CASSANDRA_HOME/logs/gc.log"

-    # $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseGCLogFileRotation"

-    # $env:JVM_OPTS="$env:JVM_OPTS -XX:NumberOfGCLogFiles=10"

-    # $env:JVM_OPTS="$env:JVM_OPTS -XX:GCLogFileSize=10M"

-

     # Configure the following for JEMallocAllocator and if jemalloc is not available in the system

     # library path.

     # set LD_LIBRARY_PATH=<JEMALLOC_HOME>/lib/

@@ -413,5 +468,5 @@
 

     $env:JVM_OPTS="$env:JVM_OPTS $env:JVM_EXTRA_OPTS"

 

-    $env:JVM_OPTS = "$env:JVM_OPTS -Dlog4j.configuration=log4j-server.properties"

+    #$env:JVM_OPTS="$env:JVM_OPTS -XX:+UnlockCommercialFeatures -XX:+FlightRecorder"

 }

diff --git a/conf/cassandra-env.sh b/conf/cassandra-env.sh
index 13e9cc7..b519b76 100644
--- a/conf/cassandra-env.sh
+++ b/conf/cassandra-env.sh
@@ -160,6 +160,7 @@
     export MALLOC_ARENA_MAX=4
 fi
 
+
 # Specifies the default port over which Cassandra will be available for
 # JMX connections.
 # For security reasons, you should not expose this port to the internet.  Firewall it if needed.
@@ -218,6 +219,7 @@
 JVM_OPTS="$JVM_OPTS -XX:CMSInitiatingOccupancyFraction=75"
 JVM_OPTS="$JVM_OPTS -XX:+UseCMSInitiatingOccupancyOnly"
 JVM_OPTS="$JVM_OPTS -XX:+UseTLAB"
+JVM_OPTS="$JVM_OPTS -XX:+PerfDisableSharedMem"
 JVM_OPTS="$JVM_OPTS -XX:CompileCommandFile=$CASSANDRA_CONF/hotspot_compiler"
 JVM_OPTS="$JVM_OPTS -XX:CMSWaitDuration=10000"
 
@@ -230,32 +232,28 @@
     JVM_OPTS="$JVM_OPTS -XX:+UseCondCardMark"
 fi
 
-# GC logging options -- uncomment to enable
-# JVM_OPTS="$JVM_OPTS -XX:+PrintGCDetails"
-# JVM_OPTS="$JVM_OPTS -XX:+PrintGCDateStamps"
-# JVM_OPTS="$JVM_OPTS -XX:+PrintHeapAtGC"
-# JVM_OPTS="$JVM_OPTS -XX:+PrintTenuringDistribution"
-# JVM_OPTS="$JVM_OPTS -XX:+PrintGCApplicationStoppedTime"
-# JVM_OPTS="$JVM_OPTS -XX:+PrintPromotionFailure"
-# JVM_OPTS="$JVM_OPTS -XX:PrintFLSStatistics=1"
-# JVM_OPTS="$JVM_OPTS -Xloggc:/var/log/cassandra/gc-`date +%s`.log"
-# If you are using JDK 6u34 7u2 or later you can enable GC log rotation
-# don't stick the date in the log name if rotation is on.
-# JVM_OPTS="$JVM_OPTS -Xloggc:/var/log/cassandra/gc.log"
-# JVM_OPTS="$JVM_OPTS -XX:+UseGCLogFileRotation"
-# JVM_OPTS="$JVM_OPTS -XX:NumberOfGCLogFiles=10"
-# JVM_OPTS="$JVM_OPTS -XX:GCLogFileSize=10M"
+# GC logging options
+JVM_OPTS="$JVM_OPTS -XX:+PrintGCDetails"
+JVM_OPTS="$JVM_OPTS -XX:+PrintGCDateStamps"
+JVM_OPTS="$JVM_OPTS -XX:+PrintHeapAtGC"
+JVM_OPTS="$JVM_OPTS -XX:+PrintTenuringDistribution"
+JVM_OPTS="$JVM_OPTS -XX:+PrintGCApplicationStoppedTime"
+JVM_OPTS="$JVM_OPTS -XX:+PrintPromotionFailure"
+#JVM_OPTS="$JVM_OPTS -XX:PrintFLSStatistics=1"
 
-# Configure the following for JEMallocAllocator and if jemalloc is not available in the system 
-# library path (Example: /usr/local/lib/). Usually "make install" will do the right thing. 
-# export LD_LIBRARY_PATH=<JEMALLOC_HOME>/lib/
-# JVM_OPTS="$JVM_OPTS -Djava.library.path=<JEMALLOC_HOME>/lib/"
+JVM_OPTS="$JVM_OPTS -Xloggc:${CASSANDRA_HOME}/logs/gc.log"
+JVM_OPTS="$JVM_OPTS -XX:+UseGCLogFileRotation"
+JVM_OPTS="$JVM_OPTS -XX:NumberOfGCLogFiles=10"
+JVM_OPTS="$JVM_OPTS -XX:GCLogFileSize=10M"
+# if using version before JDK 6u34 or 7u2 use this instead of log rotation
+# JVM_OPTS="$JVM_OPTS -Xloggc:/var/log/cassandra/gc-`date +%s`.log"
 
 # uncomment to have Cassandra JVM listen for remote debuggers/profilers on port 1414
 # JVM_OPTS="$JVM_OPTS -agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=1414"
 
 # uncomment to have Cassandra JVM log internal method compilation (developers only)
 # JVM_OPTS="$JVM_OPTS -XX:+UnlockDiagnosticVMOptions -XX:+LogCompilation"
+# JVM_OPTS="$JVM_OPTS -XX:+UnlockCommercialFeatures -XX:+FlightRecorder"
 
 # Prefer binding to IPv4 network intefaces (when net.ipv6.bindv6only=1). See
 # http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=6342561 (short version:
@@ -306,6 +304,11 @@
 #MX4J_ADDRESS="-Dmx4jaddress=127.0.0.1"
 #MX4J_PORT="-Dmx4jport=8081"
 
+# Cassandra uses SIGAR to capture OS metrics CASSANDRA-7838
+# for SIGAR we have to set the java.library.path
+# to the location of the native libraries.
+JVM_OPTS="$JVM_OPTS -Djava.library.path=$CASSANDRA_HOME/lib/sigar-bin"
+
 JVM_OPTS="$JVM_OPTS $MX4J_ADDRESS"
 JVM_OPTS="$JVM_OPTS $MX4J_PORT"
 JVM_OPTS="$JVM_OPTS $JVM_EXTRA_OPTS"
diff --git a/conf/cassandra-rackdc.properties b/conf/cassandra-rackdc.properties
index f85646e..2ea6043 100644
--- a/conf/cassandra-rackdc.properties
+++ b/conf/cassandra-rackdc.properties
@@ -16,11 +16,8 @@
 
 # These properties are used with GossipingPropertyFileSnitch and will
 # indicate the rack and dc for this node
-#
-# When upgrading from SimpleSnitch, you will need to set your initial machines
-# to have rack=rack1
-dc=DC1
-rack=RAC1
+dc=dc1
+rack=rack1
 
 # Add a suffix to a datacenter name. Used by the Ec2Snitch and Ec2MultiRegionSnitch
 # to append a string to the EC2 region name.
diff --git a/conf/cassandra-topology.yaml b/conf/cassandra-topology.yaml
deleted file mode 100644
index 11c37d0..0000000
--- a/conf/cassandra-topology.yaml
+++ /dev/null
@@ -1,25 +0,0 @@
-# YAML topology configuration file for Cassandra,
-# to be used with YamlFileNetworkTopologySnitch.
-
-# The topology, as a list of data centers.
-topology:
-    # Each data center has a name and a list of racks.
-    - dc_name: DC1
-      racks:
-          # Each rack has a name and a list of nodes.
-          - rack_name: c1
-            nodes:
-                # Each node has a broadcast address (required)
-                # and a data-center-local address (optional).
-                # If dc_local_address is specified, its peers
-                # in the same data center will attempt to
-                # reconnect over that address instead.
-                - broadcast_address: 1.2.3.4
-                  dc_local_address: 5.6.7.8
-
-# Default data center name for unknown nodes; defaults to "UNKNOWN"
-# default_dc_name: UNKNOWN
-
-# Default rack name for unknown nodes
-# default_rack_name: UNKNOWN; defaults to "UNKNOWN"
-
diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml
index db4cd37..41c1fb1 100644
--- a/conf/cassandra.yaml
+++ b/conf/cassandra.yaml
@@ -62,6 +62,7 @@
 # - PasswordAuthenticator relies on username/password pairs to authenticate
 #   users. It keeps usernames and hashed passwords in system_auth.credentials table.
 #   Please increase system_auth keyspace replication factor if you use this authenticator.
+#   If using PasswordAuthenticator, CassandraRoleManager must also be used (see below)
 authenticator: AllowAllAuthenticator
 
 # Authorization backend, implementing IAuthorizer; used to limit access/provide permissions
@@ -73,6 +74,33 @@
 #   increase system_auth keyspace replication factor if you use this authorizer.
 authorizer: AllowAllAuthorizer
 
+# Part of the Authentication & Authorization backend, implementing IRoleManager; used
+# to maintain grants and memberships between roles.
+# Out of the box, Cassandra provides org.apache.cassandra.auth.CassandraRoleManager,
+# which stores role information in the system_auth keyspace. Most functions of the
+# IRoleManager require an authenticated login, so unless the configured IAuthenticator
+# actually implements authentication, most of this functionality will be unavailable.
+#
+# - CassandraRoleManager stores role data in the system_auth keyspace. Please
+#   increase system_auth keyspace replication factor if you use this role manager.
+role_manager: CassandraRoleManager
+
+# Validity period for roles cache (fetching permissions can be an
+# expensive operation depending on the authorizer). Granted roles are cached for
+# authenticated sessions in AuthenticatedUser and after the period specified
+# here, become eligible for (async) reload.
+# Defaults to 2000, set to 0 to disable.
+# Will be disabled automatically for AllowAllAuthenticator.
+roles_validity_in_ms: 2000
+
+# Refresh interval for roles cache (if enabled).
+# After this interval, cache entries become eligible for refresh. Upon next
+# access, an async reload is scheduled and the old value returned until it
+# completes. If roles_validity_in_ms is non-zero, then this must be
+# also.
+# Defaults to the same value as roles_validity_in_ms.
+# roles_update_interval_in_ms: 1000
+
 # Validity period for permissions cache (fetching permissions can be an
 # expensive operation depending on the authorizer, CassandraAuthorizer is
 # one example). Defaults to 2000, set to 0 to disable.
@@ -162,15 +190,25 @@
 # Disabled by default, meaning all keys are going to be saved
 # key_cache_keys_to_save: 100
 
+# Row cache implementation class name.
+# Available implementations:
+#   org.apache.cassandra.cache.OHCProvider                Fully off-heap row cache implementation (default).
+#   org.apache.cassandra.cache.SerializingCacheProvider   This is the row cache implementation availabile
+#                                                         in previous releases of Cassandra.
+# row_cache_class_name: org.apache.cassandra.cache.OHCProvider
+
 # Maximum size of the row cache in memory.
-# NOTE: if you reduce the size, you may not get you hottest keys loaded on startup.
+# Please note that OHC cache implementation requires some additional off-heap memory to manage
+# the map structures and some in-flight memory during operations before/after cache entries can be
+# accounted against the cache capacity. This overhead is usually small compared to the whole capacity.
+# Do not specify more memory that the system can afford in the worst usual situation and leave some
+# headroom for OS block level cache. Do never allow your system to swap.
 #
 # Default value is 0, to disable row caching.
 row_cache_size_in_mb: 0
 
-# Duration in seconds after which Cassandra should
-# save the row cache. Caches are saved to saved_caches_directory as specified
-# in this configuration file.
+# Duration in seconds after which Cassandra should save the row cache.
+# Caches are saved to saved_caches_directory as specified in this configuration file.
 #
 # Saved caches greatly improve cold-start speeds, and is relatively cheap in
 # terms of I/O for the key cache. Row cache saving is much more expensive and
@@ -179,8 +217,8 @@
 # Default is 0 to disable saving the row cache.
 row_cache_save_period: 0
 
-# Number of keys from the row cache to save
-# Disabled by default, meaning all keys are going to be saved
+# Number of keys from the row cache to save.
+# Specify 0 (which is the default), meaning all keys are going to be saved
 # row_cache_keys_to_save: 100
 
 # Maximum size of the counter cache in memory.
@@ -209,19 +247,6 @@
 # Disabled by default, meaning all keys are going to be saved
 # counter_cache_keys_to_save: 100
 
-# The off-heap memory allocator.  Affects storage engine metadata as
-# well as caches.  Experiments show that JEMAlloc saves some memory
-# than the native GCC allocator (i.e., JEMalloc is more
-# fragmentation-resistant).
-# 
-# Supported values are: NativeAllocator, JEMallocAllocator
-#
-# If you intend to use JEMallocAllocator you have to install JEMalloc as library and
-# modify cassandra-env.sh as directed in the file.
-#
-# Defaults to NativeAllocator
-# memory_allocator: NativeAllocator
-
 # saved caches
 # If not set, the default directory is $CASSANDRA_HOME/data/saved_caches.
 # saved_caches_directory: /var/lib/cassandra/saved_caches
@@ -255,9 +280,13 @@
 # is reasonable.
 commitlog_segment_size_in_mb: 32
 
-# Reuse commit log files when possible. The default is false, and this
-# feature will be removed entirely in future versions of Cassandra.
-#commitlog_segment_recycling: false
+# Compression to apply to the commit log. If omitted, the commit log
+# will be written uncompressed.  LZ4, Snappy, and Deflate compressors
+# are supported.
+#commitlog_compression:
+#   - class_name: LZ4Compressor
+#     parameters:
+#         -
 
 # any class that implements the SeedProvider interface and has a
 # constructor that takes a Map<String, String> of parameters will do.
@@ -299,7 +328,7 @@
 # memtable_offheap_space_in_mb: 2048
 
 # Ratio of occupied non-flushing memtable size to total permitted size
-# that will trigger a flush of the largest memtable.  Lager mct will
+# that will trigger a flush of the largest memtable. Larger mct will
 # mean larger flushes and hence less compaction, but also less concurrent
 # flush activity which can make it difficult to keep your disks fed
 # under heavy write load.
@@ -314,14 +343,15 @@
 #   offheap_objects: native memory, eliminating nio buffer heap overhead
 memtable_allocation_type: heap_buffers
 
-# Total space to use for commitlogs.  Since commitlog segments are
-# mmapped, and hence use up address space, the default size is 32
-# on 32-bit JVMs, and 8192 on 64-bit JVMs.
+# Total space to use for commit logs on disk.
 #
-# If space gets above this value (it will round up to the next nearest
-# segment multiple), Cassandra will flush every dirty CF in the oldest
-# segment and remove it.  So a small total commitlog space will tend
-# to cause more flush activity on less-active columnfamilies.
+# If space gets above this value, Cassandra will flush every dirty CF
+# in the oldest segment and remove it.  So a small total commitlog space
+# will tend to cause more flush activity on less-active columnfamilies.
+#
+# The default value is the smaller of 8192, and 1/4 of the total space
+# of the commitlog volume.
+#
 # commitlog_total_space_in_mb: 8192
 
 # This sets the amount of memtable flush writer threads.  These will
@@ -391,6 +421,14 @@
 # Leaving this blank will set it to the same value as listen_address
 # broadcast_address: 1.2.3.4
 
+# When using multiple physical network interfaces, set this
+# to true to listen on broadcast_address in addition to
+# the listen_address, allowing nodes to communicate in both
+# interfaces.
+# Ignore this property if the network configuration automatically
+# routes  between the public and private networks such as EC2.
+# listen_on_broadcast_address: false
+
 # Internode authentication backend, implementing IInternodeAuthenticator;
 # used to allow/disallow connections from peer nodes.
 # internode_authenticator: org.apache.cassandra.auth.AllowAllInternodeAuthenticator
@@ -421,7 +459,7 @@
 # native_transport_max_concurrent_connections_per_ip: -1
 
 # Whether to start the thrift rpc server.
-start_rpc: true
+start_rpc: false
 
 # The address or interface to bind the Thrift RPC service and native transport
 # server to.
@@ -555,6 +593,8 @@
 # Caution should be taken on increasing the size of this threshold as it can lead to node instability.
 batch_size_warn_threshold_in_kb: 5
 
+# Fail any batch exceeding this value. 50kb (10x warn threshold) by default.
+batch_size_fail_threshold_in_kb: 50
 
 # Log WARN on any batches not of type LOGGED than span across more partitions than this limit
 unlogged_batch_across_partitions_warn_threshold: 10
@@ -817,7 +857,62 @@
 # latency if you block for cross-datacenter responses.
 inter_dc_tcp_nodelay: false
 
-# GC Pauses greater than gc_warn_threshold_in_ms will be logged at WARN level
-# Adjust the threshold based on your application throughput requirement
+# TTL for different trace types used during logging of the repair process.
+tracetype_query_ttl: 86400
+tracetype_repair_ttl: 604800
+
 # By default, Cassandra logs GC Pauses greater than 200 ms at INFO level
+# This threshold can be adjusted to minimize logging if necessary
+# gc_log_threshold_in_ms: 200
+
+# GC Pauses greater than gc_warn_threshold_in_ms will be logged at WARN level
+# If unset, all GC Pauses greater than gc_log_threshold_in_ms will log at
+# INFO level
+# Adjust the threshold based on your application throughput requirement
 # gc_warn_threshold_in_ms: 1000
+
+# UDFs (user defined functions) are disabled by default.
+# As of Cassandra 2.2, there is no security manager or anything else in place that
+# prevents execution of evil code. CASSANDRA-9402 will fix this issue for Cassandra 3.0.
+# This will inherently be backwards-incompatible with any 2.2 UDF that perform insecure
+# operations such as opening a socket or writing to the filesystem.
+enable_user_defined_functions: false
+
+# The default Windows kernel timer and scheduling resolution is 15.6ms for power conservation.
+# Lowering this value on Windows can provide much tighter latency and better throughput, however
+# some virtualized environments may see a negative performance impact from changing this setting
+# below their system default. The sysinternals 'clockres' tool can confirm your system's default
+# setting.
+windows_timer_interval: 1
+
+# Maximum size of any value in SSTables. Safety measure to detect SSTable corruption
+# early. Any value size larger than this threshold will result into marking an SSTable
+# as corrupted.
+# max_value_size_in_mb: 256
+
+# Coalescing Strategies #
+# Coalescing multiples messages turns out to significantly boost message processing throughput (think doubling or more).
+# On bare metal, the floor for packet processing throughput is high enough that many applications won't notice, but in
+# virtualized environments, the point at which an application can be bound by network packet processing can be
+# surprisingly low compared to the throughput of task processing that is possible inside a VM. It's not that bare metal
+# doesn't benefit from coalescing messages, it's that the number of packets a bare metal network interface can process
+# is sufficient for many applications such that no load starvation is experienced even without coalescing.
+# There are other benefits to coalescing network messages that are harder to isolate with a simple metric like messages
+# per second. By coalescing multiple tasks together, a network thread can process multiple messages for the cost of one
+# trip to read from a socket, and all the task submission work can be done at the same time reducing context switching
+# and increasing cache friendliness of network message processing.
+# See CASSANDRA-8692 for details.
+
+# Strategy to use for coalescing messages in OutboundTcpConnection.
+# Can be fixed, movingaverage, timehorizon (default), disabled.
+# You can also specify a subclass of CoalescingStrategies.CoalescingStrategy by name.
+# otc_coalescing_strategy: TIMEHORIZON
+
+# How many microseconds to wait for coalescing. For fixed strategy this is the amount of time after the first
+# message is received before it will be sent with any accompanying messages. For moving average this is the
+# maximum amount of time that will be waited as well as the interval at which messages must arrive on average
+# for coalescing to be enabled.
+# otc_coalescing_window_us: 200
+
+# Do not try to coalesce messages if we already got that many messages. This should be more than 2 and less than 128.
+# otc_coalescing_enough_coalesced_messages: 8
diff --git a/conf/commitlog_archiving.properties b/conf/commitlog_archiving.properties
index 109a50b..393259c 100644
--- a/conf/commitlog_archiving.properties
+++ b/conf/commitlog_archiving.properties
@@ -22,13 +22,6 @@
 #             %name => Name of the commit log.
 # Example: archive_command=/bin/ln %path /backup/%name
 #
-# commitlog archiving configuration.  Leave blank to disable.
-
-# Command to execute to archive a commitlog segment
-# Parameters: %path => Fully qualified path of the segment to archive
-#             %name => Name of the commit log.
-# Example: archive_command=/bin/cp -f %path /backup/%name
-#
 # Limitation: *_command= expects one command with arguments. STDOUT
 # and STDIN or multiple commands cannot be executed.  You might want
 # to script multiple commands and add a pointer here.
diff --git a/conf/cqlshrc.sample b/conf/cqlshrc.sample
index 5a33a2f..462dcc6 100644
--- a/conf/cqlshrc.sample
+++ b/conf/cqlshrc.sample
@@ -32,6 +32,9 @@
 ;; Used for displaying timestamps (and reading them with COPY)
 ; datetimeformat = %Y-%m-%d %H:%M:%S%z
 
+;; Display timezone
+;timezone = Etc/UTC
+
 ;; The number of digits displayed after the decimal point
 ;; (note that increasing this to large numbers can result in unusual values)
 ; float_precision = 5
@@ -39,7 +42,25 @@
 ;; Used for automatic completion and suggestions
 ; completekey = tab
 
+;; The encoding used for characters
+; encoding = utf8
 
+; To use another than the system default browser for cqlsh HELP to open
+; the CQL doc HTML, use the 'browser' preference.
+; If the field value is empty or not specified, cqlsh will use the
+; default browser (specifying 'browser = default' does not work).
+;
+; Supported browsers are those supported by the Python webbrowser module.
+; (https://docs.python.org/2/library/webbrowser.html).
+;
+; Hint: to use Google Chome, use
+; 'browser = open -a /Applications/Google\ Chrome.app %s' on Mac OS X and
+; 'browser = /usr/bin/google-chrome-stable %s' on Linux and
+; 'browser = C:/Program Files (x86)/Google/Chrome/Application/chrome.exe %s' on Windows.
+;
+; This setting can be overridden with the --browser command line option.
+;
+;browser =
 
 [cql]
 ;; A version of CQL to use (this should almost never be set)
diff --git a/conf/logback.xml b/conf/logback.xml
index e170d41..a47740d 100644
--- a/conf/logback.xml
+++ b/conf/logback.xml
@@ -17,37 +17,85 @@
  under the License.
 -->
 
+<!--
+In order to disable debug.log, comment-out the ASYNCDEBUGLOG
+appender reference in the root level section below.
+-->
+
 <configuration scan="true">
   <jmxConfigurator />
-  <appender name="FILE" class="ch.qos.logback.core.rolling.RollingFileAppender">
+  <shutdownHook class="ch.qos.logback.core.hook.DelayingShutdownHook"/>
+
+  <!-- SYSTEMLOG rolling file appender to system.log (INFO level) -->
+
+  <appender name="SYSTEMLOG" class="ch.qos.logback.core.rolling.RollingFileAppender">
+    <filter class="ch.qos.logback.classic.filter.ThresholdFilter">
+      <level>INFO</level>
+    </filter>
     <file>${cassandra.logdir}/system.log</file>
     <rollingPolicy class="ch.qos.logback.core.rolling.FixedWindowRollingPolicy">
       <fileNamePattern>${cassandra.logdir}/system.log.%i.zip</fileNamePattern>
       <minIndex>1</minIndex>
       <maxIndex>20</maxIndex>
     </rollingPolicy>
-
     <triggeringPolicy class="ch.qos.logback.core.rolling.SizeBasedTriggeringPolicy">
       <maxFileSize>20MB</maxFileSize>
     </triggeringPolicy>
     <encoder>
       <pattern>%-5level [%thread] %date{ISO8601} %F:%L - %msg%n</pattern>
-      <!-- old-style log format
-      <pattern>%5level [%thread] %date{ISO8601} %F (line %L) %msg%n</pattern>
-      -->
     </encoder>
   </appender>
-  
+
+  <!-- DEBUGLOG rolling file appender to debug.log (all levels) -->
+
+  <appender name="DEBUGLOG" class="ch.qos.logback.core.rolling.RollingFileAppender">
+    <file>${cassandra.logdir}/debug.log</file>
+    <rollingPolicy class="ch.qos.logback.core.rolling.FixedWindowRollingPolicy">
+      <fileNamePattern>${cassandra.logdir}/debug.log.%i.zip</fileNamePattern>
+      <minIndex>1</minIndex>
+      <maxIndex>20</maxIndex>
+    </rollingPolicy>
+    <triggeringPolicy class="ch.qos.logback.core.rolling.SizeBasedTriggeringPolicy">
+      <maxFileSize>20MB</maxFileSize>
+    </triggeringPolicy>
+    <encoder>
+      <pattern>%-5level [%thread] %date{ISO8601} %F:%L - %msg%n</pattern>
+    </encoder>
+  </appender>
+
+  <!-- ASYNCLOG assynchronous appender to debug.log (all levels) -->
+
+  <appender name="ASYNCDEBUGLOG" class="ch.qos.logback.classic.AsyncAppender">
+    <queueSize>1024</queueSize>
+    <discardingThreshold>0</discardingThreshold>
+    <includeCallerData>true</includeCallerData>
+    <appender-ref ref="DEBUGLOG" />
+  </appender>
+
+  <!-- STDOUT console appender to stdout (INFO level) -->
+
   <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
+    <filter class="ch.qos.logback.classic.filter.ThresholdFilter">
+      <level>INFO</level>
+    </filter>
     <encoder>
       <pattern>%-5level %date{HH:mm:ss,SSS} %msg%n</pattern>
     </encoder>
   </appender>
-        
+
+  <!-- Uncomment bellow and corresponding appender-ref to activate logback metrics
+  <appender name="LogbackMetrics" class="com.codahale.metrics.logback.InstrumentedAppender" />
+   -->
+
   <root level="INFO">
-    <appender-ref ref="FILE" />
+    <appender-ref ref="SYSTEMLOG" />
     <appender-ref ref="STDOUT" />
+    <appender-ref ref="ASYNCDEBUGLOG" /> <!-- Comment this line to disable debug.log -->
+    <!--
+    <appender-ref ref="LogbackMetrics" />
+    -->
   </root>
-  
+
+  <logger name="org.apache.cassandra" level="DEBUG"/>
   <logger name="com.thinkaurelius.thrift" level="ERROR"/>
 </configuration>
diff --git a/debian/cassandra.install b/debian/cassandra.install
index 9420949..e8da5e9 100644
--- a/debian/cassandra.install
+++ b/debian/cassandra.install
@@ -1,4 +1,3 @@
-conf/cassandra-topology.yaml etc/cassandra
 conf/cassandra.yaml etc/cassandra
 conf/cassandra-env.sh etc/cassandra
 conf/cassandra-rackdc.properties etc/cassandra
@@ -12,14 +11,16 @@
 debian/cassandra.conf etc/security/limits.d
 debian/cassandra-sysctl.conf etc/sysctl.d
 bin/cassandra usr/sbin
-bin/cassandra-cli usr/bin
 bin/nodetool usr/bin
 bin/sstablekeys usr/bin
 bin/sstableloader usr/bin
 bin/cqlsh usr/bin
+bin/cqlsh.py usr/bin
 bin/sstablescrub usr/bin
 bin/sstableupgrade usr/bin
+bin/sstableverify usr/bin
 tools/bin/cassandra-stress usr/bin
 lib/*.jar usr/share/cassandra/lib
 lib/*.zip usr/share/cassandra/lib
+lib/sigar-bin/* usr/share/cassandra/lib/sigar-bin
 lib/licenses usr/share/doc/cassandra
diff --git a/debian/changelog b/debian/changelog
index a0ee850..a291da3 100644
--- a/debian/changelog
+++ b/debian/changelog
@@ -1,86 +1,86 @@
-cassandra (2.1.18) unstable; urgency=medium
+cassandra (2.2.10) unstable; urgency=medium
 
   * New release
 
- -- Michael Shuler <michael@pbandjelly.org>  Wed, 21 Jun 2017 13:57:08 -0500
+ -- Michael Shuler <michael@pbandjelly.org>  Wed, 21 Jun 2017 14:07:47 -0500
 
-cassandra (2.1.17) unstable; urgency=medium
+cassandra (2.2.9) unstable; urgency=medium
 
   * New release
 
- -- Michael Shuler <michael@pbandjelly.org>  Wed, 15 Feb 2017 18:09:07 -0600
+ -- Michael Shuler <michael@pbandjelly.org>  Wed, 15 Feb 2017 18:12:32 -0600
 
-cassandra (2.1.16) unstable; urgency=medium
-
-  * New release
-
- -- Michael Shuler <mshuler@apache.org>  Wed, 05 Oct 2016 17:16:47 -0500
-
-cassandra (2.1.15) unstable; urgency=medium
-
-  * New release
-
- -- Jake Luciani <jake@apache.org>  Tue, 28 Jun 2016 20:06:09 -0400
-
-cassandra (2.1.14) unstable; urgency=medium
-
-  * New release
-
- -- Jake Luciani <jake@apache.org>  Fri, 15 Apr 2016 10:29:30 -0400
-
-cassandra (2.1.13) unstable; urgency=medium
+cassandra (2.2.8) unstable; urgency=medium
 
   * New release 
 
- -- Jake Luciani <jake@apache.org>  Tue, 26 Jan 2016 09:11:55 -0500
+ -- Michael Shuler <michael@pbandjelly.org>  Fri, 23 Sep 2016 16:35:25 -0500
 
-cassandra (2.1.12) unstable; urgency=medium
+cassandra (2.2.7) unstable; urgency=medium
 
   * New release 
 
- -- Jake Luciani <jake@apache.org>  Wed, 02 Dec 2015 10:38:36 -0500
+ -- Jake Luciani <jake@apache.org>  Tue, 28 Jun 2016 20:08:44 -0400
 
-cassandra (2.1.11) unstable; urgency=medium
+cassandra (2.2.6) unstable; urgency=medium
+
+  * New release
+
+ -- Jake Luciani <jake@apache.org>  Fri, 15 Apr 2016 09:47:38 -0400
+
+cassandra (2.2.5) unstable; urgency=medium
 
   * New release 
 
- -- Jake Luciani <jake@apache.org>  Mon, 12 Oct 2015 09:46:10 -0400
+ -- Jake Luciani <jake@apache.org>  Tue, 26 Jan 2016 09:15:04 -0500
 
-cassandra (2.1.10) unstable; urgency=medium
+cassandra (2.2.4) unstable; urgency=medium
 
   * New release
 
- -- Jake Luciani <jake@apache.org>  Thu, 01 Oct 2015 09:35:19 -0400
+ -- Jake Luciani <jake@apache.org>  Wed, 02 Dec 2015 10:52:18 -0500
 
-cassandra (2.1.9) unstable; urgency=medium
-
-  * New release
-
- -- Jake Luciani <jake@apache.org>  Tue, 25 Aug 2015 09:32:27 -0400
-
-cassandra (2.1.8) unstable; urgency=medium
+cassandra (2.2.3) unstable; urgency=medium
 
   * New release 
 
- -- Jake Luciani <jake@apache.org>  Mon, 06 Jul 2015 11:18:23 -0400
+ -- Jake Luciani <jake@apache.org>  Mon, 12 Oct 2015 09:49:15 -0400
 
-cassandra (2.1.7) unstable; urgency=medium
+cassandra (2.2.2) unstable; urgency=medium
 
   * New release 
 
- -- Jake Luciani <jake@apache.org>  Thu, 18 Jun 2015 14:52:26 -0400
+ -- Jake Luciani <jake@apache.org>  Thu, 01 Oct 2015 09:37:57 -0400
 
-cassandra (2.1.6) unstable; urgency=medium
+cassandra (2.2.1) unstable; urgency=medium
 
   * New release
 
- -- Jake Luciani <jake@apache.org>  Fri, 05 Jun 2015 10:21:39 -0400
+ -- Jake Luciani <jake@apache.org>  Tue, 25 Aug 2015 13:22:45 -0400
 
-cassandra (2.1.5) unstable; urgency=medium
+cassandra (2.2.0) unstable; urgency=medium
 
   * New release
 
- -- Jake Luciani <jake@apache.org>  Mon, 27 Apr 2015 10:24:48 -0400
+ -- Jake Luciani <jake@apache.org>  Fri, 17 Jul 2015 13:06:22 -0400
+
+cassandra (2.2.0~rc2) unstable; urgency=medium
+
+  * New RC release
+
+ -- Jake Luciani <jake@apache.org>  Tue, 30 Jun 2015 09:43:38 -0400
+
+cassandra (2.2.0~rc1) unstable; urgency=medium
+
+  * New RC release 
+
+ -- Jake Luciani <jake@apache.org>  Fri, 05 Jun 2015 11:11:50 -0400
+
+cassandra (2.2.0~beta1) unstable; urgency=medium
+
+  * New Beta Release 
+
+ -- Jake Luciani <jake@apache.org>  Sun, 17 May 2015 21:33:49 -0400
 
 cassandra (2.1.4) unstable; urgency=medium
 
diff --git a/debian/control b/debian/control
index cc5928c..d45a875 100644
--- a/debian/control
+++ b/debian/control
@@ -3,7 +3,7 @@
 Priority: extra
 Maintainer: Eric Evans <eevans@apache.org>
 Uploaders: Sylvain Lebresne <slebresne@apache.org>
-Build-Depends: debhelper (>= 5), openjdk-7-jdk | java7-jdk, ant (>= 1.7), ant-optional (>= 1.7), dh-python, python-dev (>= 2.7), dpatch, bash-completion
+Build-Depends: debhelper (>= 5), openjdk-7-jdk | java7-jdk, ant (>= 1.9), ant-optional (>= 1.9), dh-python, python-dev (>= 2.7), dpatch, bash-completion
 Homepage: http://cassandra.apache.org
 Vcs-Git: http://git-wip-us.apache.org/repos/asf/cassandra.git
 Vcs-Browser: https://git-wip-us.apache.org/repos/asf?p=cassandra.git
diff --git a/debian/patches/002cassandra_logdir_fix.dpatch b/debian/patches/002cassandra_logdir_fix.dpatch
index 8836eb4..cca337c 100644
--- a/debian/patches/002cassandra_logdir_fix.dpatch
+++ b/debian/patches/002cassandra_logdir_fix.dpatch
@@ -6,9 +6,9 @@
 
 @DPATCH@
 diff -urNad '--exclude=CVS' '--exclude=.svn' '--exclude=.git' '--exclude=.arch' '--exclude=.hg' '--exclude=_darcs' '--exclude=.bzr' cassandra~/bin/cassandra cassandra/bin/cassandra
---- cassandra~/bin/cassandra	2014-09-15 19:42:28.000000000 -0500
-+++ cassandra/bin/cassandra	2014-09-15 21:15:15.627505503 -0500
-@@ -134,7 +134,7 @@
+--- cassandra~/bin/cassandra	2015-10-27 14:15:10.718076265 -0500
++++ cassandra/bin/cassandra	2015-10-27 14:23:10.000000000 -0500
+@@ -139,7 +139,7 @@
      props="$3"
      class="$4"
      cassandra_parms="-Dlogback.configurationFile=logback.xml"
@@ -17,3 +17,15 @@
      cassandra_parms="$cassandra_parms -Dcassandra.storagedir=$cassandra_storagedir"
  
      if [ "x$pidpath" != "x" ]; then
+diff -urNad '--exclude=CVS' '--exclude=.svn' '--exclude=.git' '--exclude=.arch' '--exclude=.hg' '--exclude=_darcs' '--exclude=.bzr' cassandra~/conf/cassandra-env.sh cassandra/conf/cassandra-env.sh
+--- cassandra~/conf/cassandra-env.sh	2015-10-27 14:20:22.990840135 -0500
++++ cassandra/conf/cassandra-env.sh	2015-10-27 14:24:03.210202234 -0500
+@@ -288,7 +288,7 @@
+ JVM_OPTS="$JVM_OPTS -XX:+PrintPromotionFailure"
+ #JVM_OPTS="$JVM_OPTS -XX:PrintFLSStatistics=1"
+ 
+-JVM_OPTS="$JVM_OPTS -Xloggc:${CASSANDRA_HOME}/logs/gc.log"
++JVM_OPTS="$JVM_OPTS -Xloggc:/var/log/cassandra/gc.log"
+ JVM_OPTS="$JVM_OPTS -XX:+UseGCLogFileRotation"
+ JVM_OPTS="$JVM_OPTS -XX:NumberOfGCLogFiles=10"
+ JVM_OPTS="$JVM_OPTS -XX:GCLogFileSize=10M"
diff --git a/debian/rules b/debian/rules
index 70db61c..35f5a51 100755
--- a/debian/rules
+++ b/debian/rules
@@ -28,6 +28,7 @@
 	dh_testdir
 	printf "version=%s" $(VERSION) > build.properties
 
+	$(ANT) generate-cql-html
 	$(ANT) jar
 	cd pylib && python setup.py install --no-compile --install-layout deb \
 		--root $(CURDIR)/debian/cassandra
@@ -50,8 +51,6 @@
 	# Copy stress jars
 	dh_install build/tools/lib/stress.jar \
 		usr/share/cassandra
-	dh_install tools/lib/*.jar \
-		usr/share/cassandra
 
 	dh_link usr/share/cassandra/apache-cassandra-$(VERSION).jar \
 		usr/share/cassandra/apache-cassandra.jar
@@ -64,7 +63,7 @@
 	dh_testroot
 	dh_installchangelogs
 	dh_installinit -u'start 50 2 3 4 5 . stop 50 0 1 6 .'
-	dh_installdocs README.asc CHANGES.txt NEWS.txt
+	dh_installdocs README.asc CHANGES.txt NEWS.txt doc/cql3/CQL.css doc/cql3/CQL.html
 	dh_installexamples tools/*.yaml
 	dh_bash-completion
 	dh_compress
diff --git a/doc/cql/CQL.css b/doc/cql/CQL.css
deleted file mode 100644
index 2e4a41c..0000000
--- a/doc/cql/CQL.css
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 
- * 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.
- * 
- */
-
-/* Just a sample of some things that might be helpful in rendering CQL.html */
-
-pre.sample {
-  padding: 1ex;
-  border: 1px solid black;
-  background-color: #ffe0e0;
-}
-pre.syntax {
-  padding: 1ex;
-  border: 1px solid black;
-  background-color: #e0e0ff;
-}
-table {
-  width: 80%;
-  border-collapse: collapse;
-  border: 1px solid black;
-}
-td {
-  padding: 2px 1ex;
-  border: 1px solid black;
-}
-body {
-  background-color: white;
-}
-h2 {
-  margin-top: 3em;
-}
diff --git a/doc/cql/CQL.textile b/doc/cql/CQL.textile
deleted file mode 100644
index 519f8d2..0000000
--- a/doc/cql/CQL.textile
+++ /dev/null
@@ -1,713 +0,0 @@
-==<!-- This is only a suggestion, to demonstrate that the pre.syntax and pre.sample classes should probably be rendered differently. Render as you like. --><link rel="StyleSheet" href="CQL.css" type="text/css" media="screen">==
-
-h1. Cassandra Query Language (CQL) v2.0
-
-<b>%{color:red}NOTICE%</b>: CQL 2 has been deprecated since Cassandra 2.0, and will be removed in Cassandra 3.0.  It is recommended that you upgrade to CQL 3, which has much better driver support and many new features.  For further reading, see the "CQL 3 documentation":http://cassandra.apache.org/doc/cql3/CQL.html and the "CQL 3 drivers":http://wiki.apache.org/cassandra/ClientOptions.
-
-h2. Table of Contents
-
-{toc}
-
-h2. Syntax conventions
-
-To aid in specifying the CQL syntax, we will use the following conventions in this document:
-
-* Language rules will be given in a BNF(Backus-Naur Form)-like notation, looking like this:
-
-bc(syntax). 
-<Start> ::= <CQL_Statement>*
-          ;
-
-* Nonterminal symbols in syntax rules will have @<angle brackets>@.
-* Terminal symbols will be shown in @"single quotes"@.
-* As an additional shortcut notation to BNF, optional symbols (that can occur zero or one times) will be followed by a @?@ marker. Optional symbols that can occur zero or any number of times will be followed by a @*@ marker. Multiple symbols may be grouped together in @(parentheses)@ to signify that they are all optional or repeatable together.
-* In a few cases where meaning is more easily conveyed through prose, we explain a symbol and its expansion without BNF.
-* Sample code will also be shown in a code block:
-
-bc(sample). 
-SELECT sample_usage FROM cql;
-
-* References to keywords or pieces of CQL code in running text will be shown in a @fixed-width font@.
-
-h2. Overall syntax
-
-CQL consists of statements. As in SQL, some statements directly make changes to data, some look up data, and some change the way data is stored.
-
-All statements end with a semicolon.
-
-bc(syntax). 
-<CQL_Statement> ::= <statementBody> ";"
-                  ;
-<statementBody> ::= <useStatement>
-                  | <selectStatement>
-                  | <dataChangeStatement>
-                  | <schemaChangeStatement>
-                  ;
-<dataChangeStatement> ::= <insertStatement>
-                        | <updateStatement>
-                        | <batchStatement>
-                        | <deleteStatement>
-                        | <truncateStatement>
-                        ;
-<schemaChangeStatement> ::= <createKeyspaceStatement>
-                          | <createColumnFamilyStatement>
-                          | <createIndexStatement>
-                          | <dropKeyspaceStatement>
-                          | <dropColumnFamilyStatement>
-                          | <dropIndexStatement>
-                          | <alterTableStatement>
-                          ;
-
-String literals and identifiers (including keyspace and column family names) are case-sensitive, but CQL keywords are not. We show CQL keywords in this document in @UPPERCASE@ merely as a convention to aid readability.
-
-Literal values can be expressed in several ways in CQL.
-
-bc(syntax). 
-<term> ::= "KEY"
-         | <identifier>
-         | <stringLiteral>
-         | <integer>
-         | <float>
-         | <uuid>
-         ;
-
-* An @<identifier>@ is a letter followed by any sequence of letters, digits, or the underscore (@_@).
-* A @<stringLiteral>@ is encased in @'single quotes'@. A single quote itself can be represented in a string literal by doubling it, as in SQL: @'Single quote -> '' <-'@.
-* An @<integer>@ consists of an optional minus sign (@-@) followed by one or more digits (@0-9@).
-* A @<uuid>@ can be expressed in the canonical UUID form: 32 hex digits (@0-9@ or @a-f@, case insensitive), separated by dashes (@-@) after the 8th, 12th, 16th, and 20th digits. Example: @01234567-0123-0123-0123-0123456789ab@
-* A @<float>@ is a series of one or more decimal digits, followed by a period (@.@), and one or more decimal digits following. Note that there is no provision for "e" notation, no optional @+@ sign, and the forms @.42@ and @42.@ are not accepted. Use @0.42@ and @42.0@.
-* Whitespace is not significant except to separate terms, and inside string literals.
-
-Comments in CQL can begin with a double dash (@--@) or a double slash (@//@) and extend to the end of the line. Multiline comments are enclosed in @/* ... */@.
-
-h2(#storageTypes). Data Storage Types
-
-__Syntax:__
-
-bc(syntax). 
-<storageType> ::= "ascii"
-                | "bigint"
-                | "blob"
-                | "boolean"
-                | "counter"
-                | "decimal"
-                | "double"
-                | "float"
-                | "int"
-                | "text"
-                | "timestamp"
-                | "uuid"
-                | "varchar"
-                | "varint"
-                ;
-
-The following table gives additional information on the available data types.
-
-|_. type|_. description|
-|ascii|ASCII character string|
-|bigint|64-bit signed long|
-|blob|Arbitrary bytes (no validation)|
-|boolean|true or false|
-|counter|Counter column (64-bit long)|
-|decimal|Variable-precision decimal|
-|double|64-bit IEEE-754 floating point|
-|float|32-bit IEEE-754 floating point|
-|int|32-bit signed int|
-|text|UTF8 encoded string|
-|timestamp|A timestamp. See "Working with dates":#usingdates below for more information.|
-|uuid|Type 1 or type 4 UUID|
-|varchar|UTF8 encoded string|
-|varint|Arbitrary-precision integer|
-
-_Note: In addition to the recognized types listed above, it is also possible to supply a string containing the name of a class (a sub-class of @AbstractType@ loadable by Cassandra). The class name should either be fully qualified, or relative to the @org.apache.cassandra.db.marshal@ package._
-
-h3(#usingdates). Working with dates
-
-Values serialized with the @timestamp@ type are encoded as 64-bit signed integers representing a number of milliseconds since the standard base time known as "the epoch": January 1 1970 at 00:00:00 GMT.
-
-Timestamp types can be input in CQL as simple long integers, giving the number of milliseconds since the epoch, as defined above.
-
-Timestamp types can also be input as string literals in any of the following ISO 8601 formats, each representing the time and date Jan 2, 2003, at 04:05:00 AM, GMT.:
-
-* @2011-02-03 04:05+0000@
-* @2011-02-03 04:05:00+0000@
-* @2011-02-03T04:05+0000@
-* @2011-02-03T04:05:00+0000@
-
-The @+0000@ above is an RFC 822 4-digit time zone specification; @+0000@ refers to GMT. US Pacific Standard Time is @-0800@. The time zone may be omitted if desired-- the date will be interpreted as being in the time zone under which the coordinating Cassandra node is configured.
-
-* @2011-02-03 04:05@
-* @2011-02-03 04:05:00@
-* @2011-02-03T04:05@
-* @2011-02-03T04:05:00@
-
-There are clear difficulties inherent in relying on the time zone configuration being as expected, though, so it is recommended that the time zone always be specified for timestamps when feasible.
-
-The time of day may also be omitted, if the date is the only piece that matters:
-
-* @2011-02-03@
-* @2011-02-03+0000@
-
-In that case, the time of day will default to 00:00:00, in the specified or default time zone.
-
-h2. USE
-
-__Syntax:__
-
-bc(syntax). 
-<useStatement> ::= "USE" <term>
-                 ;
-
-__Sample:__
-
-bc(sample). 
-USE myApp;
-
-A @USE@ statement consists of the @USE@ keyword, followed by a valid keyspace name.  Its purpose is to assign the per-connection, current working keyspace.  All subsequent keyspace-specific actions will be performed in the context of the keyspace selected, unless otherwise specified, until another USE statement is issued or the connection terminates.
-
-h2. SELECT
-
-__Syntax:__
-
-bc(syntax). 
-<selectStatement> ::= "SELECT" <whatToSelect>
-                        "FROM" ( <name> "." )? <name>
-                               ( "USING" "CONSISTENCY" <consistencylevel> )?
-                               ( "WHERE" <selectWhereClause> )?
-                               ( "LIMIT" <integer> )?
-                    ;
-<whatToSelect> ::= <term> ( "," <term> )*
-                 | ("FIRST" <integer> )? "REVERSED"? <columnRange>
-                 | "COUNT" "(" <countTarget> ")"
-                 ;
-<columnRange> ::= <term> ".." <term>
-                | "*"
-                ;
-<countTarget> ::= "*"
-                | "1"
-                ;
-<name> ::= <identifier>
-         | <stringLiteral>
-         | <integer>
-         ;
-<selectWhereClause> ::= <relation> ( "AND" <relation> )*
-                      | <term> "IN" "(" <term> ( "," <term> )* ")"
-                      ;
-<relation> ::= <term> <relationOperator> <term>
-             ;
-<relationOperator> ::= "=" | "<" | ">" | "<=" | ">="
-                     ;
-
-__Sample:__
-
-bc(sample). 
-SELECT Name, Occupation FROM People WHERE key IN (199, 200, 207);
-SELECT FIRST 3 REVERSED 'time199'..'time100' FROM Events;
-SELECT COUNT(*) FROM system.Migrations;
-
-A @SELECT@ is used to read one or more records from a Cassandra column family. It returns a result-set of rows, where each row consists of a key and a collection of columns corresponding to the query.
-
-h3. Specifying Columns
-
-bc(sample). 
-SELECT col1, col2 FROM ...
-SELECT range_lo..range_hi FROM ...
-SELECT * FROM ...
-SELECT FIRST 4 REVERSED range_hi..range_lo FROM ...
-
-The @SELECT@ expression determines which columns will appear in the results and can take a few different forms, as shown above. The simplest is a comma-separated list of column names. Note that column names in Cassandra can be specified with string literals or integers, in addition to identifiers.
-
-It is also possible to specify a range of column names. The range notation consists of start and end column names, separated by two periods (@..@). The set of columns returned for a range is start and end inclusive. A single star (@*@) may be used as a range to request "all columns".
-
-When using a range, it is sometimes useful to limit the number of columns that can be returned as part of each row (since Cassandra is schemaless, it is not necessarily possible to determine ahead of time how many columns will be in the result set). To accomplish this, use the @FIRST@ clause with an integer to specify an upper limit on the number of columns returned per row. The default limit is 10,000 columns.
-
-The @REVERSED@ option causes the sort order of the columns returned to be reversed. This affects the @FIRST@ clause; when limiting the columns returned, the columns at the end of the range will be selected instead of the ones at the beginning of the range.
-
-A @SELECT@ expression may also be @COUNT(*)@. In this case, the result will be only one value: the number of rows which matched the query.
-
-It is worth noting that unlike the projection in a SQL SELECT, there is no guarantee that the results will contain all of the columns specified, because Cassandra is schemaless.
-
-h3. Column Family
-
-bc(sample). 
-SELECT ... FROM MyApp.LocationSnapshots ...;
-SELECT ... FROM EventTimeline ...;
-
-The @FROM@ clause is used to specify the Cassandra column family applicable to a @SELECT@ query. Unlike other operations on column families, the keyspace in which the column family exists may also be specified by giving its name before the column family name, and separating them by a dot (@.@). If the keyspace is not specified, the current keyspace will be used, as per normal.
-
-h3. Consistency Level
-
-bc(sample). 
-SELECT ... USING CONSISTENCY QUORUM;
-
-Following the column family clause is an optional "consistency level specification":#consistency.
-
-h3. Filtering rows
-
-bc(sample). 
-SELECT ... WHERE KEY = 11194251 AND startdate = '2011-10-08-0500';
-SELECT ... WHERE KEY >= 'AM' and KEY =< 'AZ' AND module = 17;
-SELECT ... WHERE keyalias IN ('key1', 'key2', 'key3', ...);
-
-The @WHERE@ clause provides for filtering the rows that appear in results.  The clause can filter on a key name, or range of keys, and in the case of indexed columns, on column values.  Key filters are specified using the @KEY@ keyword or key alias name, followed by a relational operator (one of @=@, @>@, @>=@, @<@, and @<=@), and then a term value.  When terms appear on both sides of a relational operator it is assumed the filter applies to an indexed column. With column index filters, the term on the left of the operator must be the name of the indexed column, and the term on the right is the value to filter __on__.
-
-__Note: The greater-than and less-than operators (@>@ and @<@) result in key ranges that are inclusive of the terms. There is no supported notion of "strictly" greater-than or less-than; these operators are merely supported as aliases to @>=@ and @<=@.__
-
-h3. Limits
-
-bc(sample). 
-SELECT ... WHERE favoriteArtist = 'The Mighty Mighty Bosstones' LIMIT 90000;
-
-The @LIMIT@ option to a @SELECT@ expression limits the number of rows returned by a query. @LIMIT@ defaults to 10,000 when left unset.
-
-h2. INSERT
-
-__Syntax:__
-
-bc(syntax). 
-<insertStatement> ::= "INSERT" "INTO" <name>
-                               "(" <term> "," <term> ( "," <term> )* ")"
-                      "VALUES" "(" <term> "," <term> ( "," <term> )* ")"
-                      ( "USING" <usingOption> ( "AND" <usingOption> )* )?
-                    ;
-<usingOption> ::= "CONSISTENCY" <consistencylevel>
-                | "TIMESTAMP" <integer>
-                | "TTL" <integer>
-                ;
-
-__Sample:__
-
-bc(sample). 
-INSERT INTO NerdMovies (KEY, 11924)
-                VALUES ('Serenity', 'Nathan Fillion')
-      USING CONSISTENCY LOCAL_QUORUM AND TTL 86400;
-
-An @INSERT@ is used to write one or more columns to a record in a Cassandra column family. No results are returned.
-
-The first column name in the @INSERT@ list must be the name of the column family key. Also, there must be more than one column name specified (Cassandra rows are not considered to exist with only a key and no associated columns).
-
-Unlike in SQL, the semantics of @INSERT@ and @UPDATE@ are identical. In either case a record is created if none existed before, and updated when it does.  For information on query modifiers and types, see the "@UPDATE@":#update section below.
-
-h2(#update). UPDATE
-
-__Syntax:__
-
-bc(syntax). 
-<updateStatement> ::= "UPDATE" <name>
-                        ( "USING" <usingOption> ( "AND" <usingOption> )* )?
-                        "SET" <assignment> ( "," <assignment> )*
-                        "WHERE" <updateWhereClause>
-                    ;
-<assignment> ::= <term> "=" <term>
-               | <term> "=" <term> "+" <term>
-               | <term> "=" <term> "-" <term>
-               ;
-<updateWhereClause> ::= <term> "=" <term>
-                      | <term> "IN" "(" <term> ( "," <term> )* ")"
-                      ;
-
-__Sample:__
-
-bc(sample). 
-UPDATE NerdMovies USING CONSISTENCY ALL AND TTL 400
-       SET 'A 1194' = 'The Empire Strikes Back',
-           'B 1194' = 'Han Solo'
-     WHERE KEY = B70DE1D0-9908-4AE3-BE34-5573E5B09F14;
-UPDATE UserActionCounts SET total = total + 2 WHERE keyalias = 523;
-
-An @UPDATE@ is used to write one or more columns to a record in a Cassandra column family. No results are returned. The row key can be specified using the @KEY@ keyword or by a key alias set per column family.
-
-h3. Column Family
-
-Statements begin with the @UPDATE@ keyword followed by a Cassandra column family name.
-
-h3. Consistency Level
-
-bc(sample). 
-UPDATE Foo USING CONSISTENCY EACH_QUORUM ...
-
-Following the column family identifier is an optional @USING@ clause, which can specify the "consistency level":#consistency for the update, or the timestamp and/or the TTL for the new columns.
-
-h3. Timestamp
-
-bc(sample). 
-UPDATE Foo USING TIMESTAMP=1318452291034 ...
-
-@UPDATE@ supports setting client-supplied optional timestamp for modification.
-
-h3. TTL
-
-bc(sample). 
-UPDATE Foo USING TTL=6800 ...
-
-@UPDATE@ supports setting a time to live (TTL), in seconds, for each of the added columns.
-
-h3. Specifying Columns and Row
-
-bc(sample). 
-UPDATE ... SET col1 = val1, col2 = val2 WHERE KEY = key1;
-UPDATE ... SET col3 = val3 WHERE KEY IN (key1, key2, key3);
-UPDATE ... SET col4 = 22 WHERE keyalias = key4;
-
-Rows are created or updated by supplying column names and values, after the @SET@ keyword, in term assignment format. Multiple columns can be set by separating the name/value pairs using commas.  Each update statement requires a precise set of row keys to be specified using a @WHERE@ clause and the @KEY@ keyword or key alias.
-
-h3. Updating Counter Columns
-
-bc(sample). 
-UPDATE ... SET name1 = name1 + <value> ...;
-UPDATE ... SET name1 = name1 - <value> ...;
-
-Counter columns can be incremented or decremented by an arbitrary numeric value though the assignment of an expression that adds or substracts the value.
-
-h2. DELETE
-
-__Syntax:__
-
-bc(syntax). 
-<deleteStatement> ::= "DELETE" ( <term> ( "," <term> )* )?
-                        "FROM" <name>
-                        ( "USING" <deleteOption> ( "AND" <deleteOption> )* )?
-                        "WHERE" <updateWhereClause>
-                    ;
-<deleteOption> ::= "CONSISTENCY" <consistencylevel>
-                 | "TIMESTAMP" <integer>
-                 ;
-
-__Sample:__
-
-bc(sample). 
-DELETE col1, col2, col3 FROM Planeteers USING CONSISTENCY ONE WHERE KEY = 'Captain';
-DELETE FROM MastersOfTheUniverse WHERE KEY IN ('Man-At-Arms', 'Teela');
-
-A @DELETE@ is used to perform the removal of one or more columns from one or more rows. The key can be given using the @KEY@ keyword or by the key alias set per column family.
-
-h3. Specifying Columns
-
-Following the @DELETE@ keyword is an optional comma-delimited list of column name terms. When no column names are specified, the remove applies to the entire row(s) matched by the "WHERE clause":#deleterows.
-
-h3. Column Family
-
-The column family name follows the list of column names and the keyword @FROM@.
-
-h3. Consistency Level
-
-Following the column family identifier is an optional "consistency level specification":#consistency.
-
-h3(#deleterows). Specifying Rows
-
-bc(sample). 
-DELETE ... WHERE KEY = 'some_key_value';
-DELETE ... WHERE keyalias IN (key1, key2);
-
-The @WHERE@ clause is used to determine to which row(s) a @DELETE@ applies. The first form allows the specification of a single keyname using the @KEY@ keyword (or by key alias) and the @=@ operator.  The second form allows a list of keyname terms to be specified using the @IN@ notation and a parenthesized list of comma-delimited keyname terms.
-
-h2. TRUNCATE
-
-__Syntax:__
-
-bc(syntax). 
-<truncateStatement> ::= "TRUNCATE" <name>
-                      ;
-
-__Sample:__
-
-bc(sample). 
-TRUNCATE super_important_data;
-
-@TRUNCATE@ accepts a single argument for the column family name, and permanently removes all data from said column family.
-
-h2. BATCH
-
-__Syntax:__
-
-bc(syntax). 
-<batchStatement> ::= "BEGIN" "BATCH"
-                        ( "USING" <usingOption> ( "AND" <usingOption> )* )?
-                        <batchStatementMember> ( ";" <batchStatementMember> )*
-                     "APPLY" "BATCH"
-                   ;
-<batchStatementMember> ::= <insertStatement>
-                         | <updateStatement>
-                         | <deleteStatement>
-                         ;
-
-__Sample:__
-
-bc(sample). 
-BEGIN BATCH USING CONSISTENCY QUORUM AND TTL 8640000
-  INSERT INTO users (KEY, password, name) VALUES ('user2', 'ch@ngem3b', 'second user')
-  UPDATE users SET password = 'ps22dhds' WHERE KEY = 'user2'
-  INSERT INTO users (KEY, password) VALUES ('user3', 'ch@ngem3c')
-  DELETE name FROM users WHERE key = 'user2'
-  INSERT INTO users (KEY, password, name) VALUES ('user4', 'ch@ngem3c', 'Andrew')
-APPLY BATCH;
-
-@BATCH@ supports setting a client-supplied optional global timestamp which will be used for each of the operations included in the batch.
-
-A single consistency level is used for the entire batch. It appears after the @BEGIN BATCH@ statement, and uses the standard "consistency level specification":#consistency. Batched statements default to @CONSISTENCY.ONE@ when left unspecified.
-
-Only data modification statements (specifically, @UPDATE@, @INSERT@, and @DELETE@) are allowed in a @BATCH@ statement. @BATCH@ is _not_ an analogue for SQL transactions.
-
-_NOTE: While there are no isolation guarantees, @UPDATE@ queries are atomic within a given record._
-
-h2. CREATE KEYSPACE
-
-__Syntax:__
-
-bc(syntax). 
-<createKeyspaceStatement> ::= "CREATE" "KEYSPACE" <name>
-                                 "WITH" <optionName> "=" <optionVal>
-                                 ( "AND" <optionName> "=" <optionVal> )*
-                            ;
-<optionName> ::= <identifier>
-               | <optionName> ":" <identifier>
-               | <optionName> ":" <integer>
-               ;
-<optionVal> ::= <stringLiteral>
-              | <identifier>
-              | <integer>
-              ;
-
-__Sample:__
-
-bc(sample). 
-CREATE KEYSPACE Excelsior WITH strategy_class = 'SimpleStrategy'
-    AND strategy_options:replication_factor = 1;
-CREATE KEYSPACE Excalibur WITH strategy_class = 'NetworkTopologyStrategy'
-    AND strategy_options:DC1 = 1 AND strategy_options:DC2 = 3;
-
-The @CREATE KEYSPACE@ statement creates a new top-level namespace (aka "keyspace"). Valid names are any string constructed of alphanumeric characters and underscores. Names which do not work as valid identifiers or integers should be quoted as string literals. Properties such as replication strategy and count are specified during creation using the following accepted keyword arguments:
-
-|_. keyword|_. required|_. description|
-|strategy_class|yes|The name of the replication strategy class which should be used for the new keyspace. Some often-used classes are @SimpleStrategy@ and @NetworkTopologyStrategy@.|
-|strategy_options|no|Most strategies require additional arguments which can be supplied by appending the option name to the @strategy_options@ keyword, separated by a colon (@:@).  For example, a strategy option of "DC1" with a value of "1" would be specified as @strategy_options:DC1 = 1@; replication_factor for SimpleStrategy could be @strategy_options:replication_factor=3@.|
-
-h2. CREATE COLUMNFAMILY
-
-__Syntax:__
-
-bc(syntax). 
-<createColumnFamilyStatement> ::= "CREATE" "COLUMNFAMILY" <name>
-                                    "(" <term> <storageType> "PRIMARY" "KEY"
-                                        ( "," <term> <storageType> )* ")"
-                                   ( "WITH" <optionName> "=" <cfOptionVal>
-                                     ( "AND" <optionName> "=" <cfOptionVal> )* )?
-                                ;
-<optionName> ::= <identifier>
-               | <optionName> ":" <identifier>
-               | <optionName> ":" <integer>
-               ;
-
-<cfOptionVal> ::= <storageType>
-                | <identifier>
-                | <stringLiteral>
-                | <integer>
-                | <float>
-                ;
-
-__Sample:__
-
-bc(sample). 
-CREATE COLUMNFAMILY Fish (KEY blob PRIMARY KEY);
-CREATE COLUMNFAMILY FastFoodEatings (user text PRIMARY KEY)
-    WITH comparator=timestamp AND default_validation=int;
-CREATE COLUMNFAMILY MonkeyTypes (
-    KEY uuid PRIMARY KEY,
-    species text,
-    alias text,
-    population varint
-) WITH comment='Important biological records'
-   AND read_repair_chance = 1.0;
-
-@CREATE COLUMNFAMILY@ statements create new column family namespaces under the current keyspace. Valid column family names are strings of alphanumeric characters and underscores, which begin with a letter.
-
-h3(#keytypes). Specifying Key Type
-
-bc(sample). 
-CREATE ... (KEY ascii PRIMARY KEY, ... ) ...
-
-When creating a new column family, you must specify the key type. The list of possible types is identical to column comparators/validators (see "Data Storage Types":#storageTypes), except it probably does not make sense to use @counter@ for a key. It's important to note that the key type you use must be compatible with the partitioner in use. For example, @OrderPreservingPartitioner@ and @CollatingOrderPreservingPartitioner@ both require UTF-8 keys. If you use an identifier for the primary key name, instead of the @KEY@ keyword, a key alias will be set automatically.
-
-h3. Specifying Column Types (optional)
-
-bc(sample). 
-CREATE ... ( ... , name1 type1, name2 type2, ... ) ...
-
-It is possible to assign columns a type during column family creation. Columns configured with a type are validated accordingly when a write occurs, and intelligent CQL drivers and interfaces will be able to decode the column values correctly when receiving them. Column types are specified as a parenthesized, comma-separated list of column term and type pairs. See "Data Storage Types":#storageTypes for the list of recognized types.
-
-h3(#cfopts). Column Family Options (optional)
-
-bc(sample). 
-CREATE COLUMNFAMILY ... WITH keyword1 = arg1 AND keyword2 = arg2;
-
-A number of optional keyword arguments can be supplied to control the configuration of a new column family.
-
-|_. keyword|_. default|_. description|
-|comparator|text|Determines the storage type of column names (which itself determines the sorting and validation of column names). Valid values are listed in the "Data Storage Types":#storageTypes table above.|
-|comment|none|A free-form, human-readable comment.|
-|read_repair_chance|1.0|The probability with which read repairs should be invoked on non-quorum reads.|
-|gc_grace_seconds|864000|Time to wait before garbage collecting tombstones (deletion markers).|
-|default_validation|text|Determines the default storage type of column values (which itself determines the validation for column values). This option does not affect the types of columns which were defined in a @CREATE COLUMNFAMILY@ statement-- only new columns. Valid values are listed in the "Data Storage Types":#storageTypes table above.|
-|min_compaction_threshold|4|Minimum number of SSTables needed to start a minor compaction.|
-|max_compaction_threshold|32|Maximum number of SSTables allowed before a minor compaction is forced.|
-|compaction_strategy_options|none|CompactionStrategy specific options such as "sstable_size_in_mb" for LeveledCompactionStrategy and "min_sstable_size" for SizeTieredCompactionStrategy|
-|compression_parameters|none|Compression parameters such as "sstable_compressor" and "chunk_length_kb"|
-
-h2. CREATE INDEX
-
-__Syntax:__
-
-bc(syntax). 
-<createIndexStatement> ::= "CREATE" "INDEX" <identifier>? "ON"
-                               <name> "(" <term> ")"
-                         ;
-
-__Sample:__
-
-bc(sample). 
-CREATE INDEX userIndex ON NerdMovies (user);
-CREATE INDEX ON Mutants (abilityId);
-
-A @CREATE INDEX@ statement is used to create a new, automatic secondary index on the given column family, for the named column. A name for the index itself can be specified before the @ON@ keyword, if desired. A single column name must be specified inside the parentheses. It is not necessary for the column to exist on any current rows (Cassandra is schemaless), but the column must already have a type (specified during the @CREATE COLUMNFAMILY@, or added afterwards with @ALTER COLUMNFAMILY@.
-
-h2. DROP KEYSPACE
-
-__Syntax:__
-
-bc(syntax). 
-<dropKeyspaceStatement> ::= "DROP" "KEYSPACE" <name>
-                          ;
-
-__Sample:__
-
-bc(sample). DROP KEYSPACE MyTwitterClone;
-
-A @DROP KEYSPACE@ statement results in the immediate, irreversible removal of a keyspace, including all column families in it, and all data contained in those column families.
-
-h2. DROP COLUMNFAMILY
-
-__Syntax:__
-
-bc(syntax). 
-<dropColumnFamilyStatement> ::= "DROP" "COLUMNFAMILY" <name>
-                              ;
-
-__Sample:__
-
-bc(sample). DROP COLUMNFAMILY worldSeriesAttendees;
-
-A @DROP COLUMNFAMILY@ statement results in the immediate, irreversible removal of a column family, including all data contained in it.
-
-h2. DROP INDEX
-
-__Syntax:__
-
-bc(syntax). 
-<dropIndexStatement> ::= "DROP" "INDEX" <name>
-                       ;
-
-__Sample:__
-
-bc(sample). DROP INDEX cf_col_idx;
-
-A @DROP INDEX@ statement is used to drop an existing secondary index.
-
-h2. ALTER COLUMNFAMILY
-
-__Syntax:__
-
-bc(syntax). 
-<alterTableStatement> ::= "ALTER" "COLUMNFAMILY" <name> <alterInstructions>
-                        ;
-<alterInstructions> ::= "ALTER" <name> "TYPE" <storageType>
-                      | "ADD" <name> <storageType>
-                      | "DROP" <name>
-                      | "WITH" <optionName> "=" <cfOptionVal>
-                               ( "AND" <optionName> "=" <cfOptionVal> )*
-                      ;
-
-An @ALTER@ statement is used to manipulate column family column metadata. It allows you to add new columns, drop existing columns, data storage type of existing columns, or change the column family properties. No results are returned.
-
-Specify the name of the column family to be changed after the @ALTER COLUMNFAMILY@ keywords, followed by the type of change desired (@ALTER@, @ADD@, @DROP@, or @WITH@), and provide the rest of the needed information, as explained below.
-
-h3. Changing the type of a typed column
-
-bc(sample). 
-ALTER COLUMNFAMILY addamsFamily ALTER lastKnownLocation TYPE uuid;
-
-@ALTER COLUMNFAMILY ... ALTER@ changes the expected storage type for a column. The column must either be the key alias or already have a type in the column family metadata. The column may or may not already exist in current rows-- but be aware that no validation of existing data is done. The bytes stored in values for that column will remain unchanged, and if existing data is not deserializable according to the new type, this may cause your CQL driver or interface to report errors.
-
-h3. Adding a typed column
-
-bc(sample). 
-ALTER COLUMNFAMILY addamsFamily ADD gravesite varchar;
-
-The @ALTER COLUMNFAMILY ... ADD@ variant adds a typed column to a column family. The column must not already have a type in the column family metadata. The same warnings from the above @ALTER@ section, about there being no validation of existing data, apply here as well.
-
-h3. Dropping a typed column
-
-bc(sample). 
-ALTER COLUMNFAMILY addamsFamily DROP gender;
-
-An @ALTER COLUMNFAMILY ... DROP@ statement removes the type of a column from the column family metadata. Note that this does _not_ remove the column from current rows; it just removes the metadata saying that the bytes stored under that column are expected to be deserializable according to a certain type.
-
-h3. Modifying column family properties
-
-bc(sample). 
-ALTER COLUMNFAMILY addamsFamily WITH comment = 'A most excellent and useful column family' AND read_repair_chance = 0.2;
-
-An @ALTER COLUMNFAMILY ... WITH@ statement makes adjustments to the column family properties, as defined when the column family was created (see "CREATE COLUMNFAMILY options":#cfopts for information about the supported options and values).
-
-Note that setting any @compaction_strategy_options:*@ parameters has the effect of erasing all previous @compaction_strategy_options:*@ parameters, so you will need to re-specify any such parameters which have already been set, if you want to keep them. The same note applies to the set of @compression_parameters:*@ parameters.
-
-h2. Common Idioms
-
-h3(#consistency). Specifying Consistency
-
-bc(syntax). 
-<consistency> ::= "ANY"
-                | "ONE"
-                | "QUORUM"
-                | "ALL"
-                | "LOCAL_QUORUM"
-                | "EACH_QUORUM"
-                ;
-
-bc(sample). 
-... USING CONSISTENCY LOCAL_QUORUM ...
-
-Consistency level specifications are made up the keywords @USING CONSISTENCY@, followed by a consistency level identifier. Valid consistency level identifiers are as listed above. When not specified, @USING CONSISTENCY ONE@ is the default.
-
-Consult your Cassandra documentation for information about how consistency levels work.
-
-h1. Versioning
-
-Versioning of the CQL language adheres to the "Semantic Versioning":http://semver.org guidelines.  Versions take the form X.Y.Z where X, Y, and Z are integer values representing major, minor, and patch level respectively.  There is no correlation between Cassandra release versions and the CQL language version.
-
-|_. version|_. description|
-|Major|The major version _must_ be bumped when backward incompatible changes are introduced.  This should rarely (if ever) occur.|
-|Minor|Minor version increments occur when new, but backward compatible, functionality is introduced.|
-|Patch|The patch version is incremented when bugs are fixed.|
-
-h1. Changes
-
-pre.. 
-Wed, 12 Oct 2011 16:53:00 -0500 - paul cannon
- * Rework whole doc, adding syntax specifics and additional explanations
-
-Fri, 09 Sep 2011 11:43:00 -0500 - Jonathan Ellis
- * add int data type
-
-pre. Wed, 07 Sep 2011 09:01:00 -0500 - Jonathan Ellis
- * Updated version to 2.0; Documented row-based count()
- * Updated list of supported data types
-
-pre. Wed, 10 Aug 2011 11:22:00 -0500 - Eric Evans
- * Improved INSERT vs. UPDATE wording.
- * Documented counter column incr/descr.
-
-pre. Sat, 01 Jun 2011 15:58:00 -0600 - Pavel Yaskevich
- * Updated to support ALTER (CASSANDRA-1709)
-
-pre. Tue, 22 Mar 2011 18:10:28 -0700 - Eric Evans <eevans@rackspace.com>
- * Initial version, 1.0.0
diff --git a/doc/cql3/CQL.textile b/doc/cql3/CQL.textile
index 87a4be9..af584d0 100644
--- a/doc/cql3/CQL.textile
+++ b/doc/cql3/CQL.textile
@@ -1,6 +1,7 @@
 <link rel="StyleSheet" href="CQL.css" type="text/css" media="screen">
 
-h1. Cassandra Query Language (CQL) v3.2.1
+h1. Cassandra Query Language (CQL) v3.3.1
+
 
 
  <span id="tableOfContents">
@@ -45,6 +46,8 @@
 
 p. There is a second kind of identifiers called _quoted identifiers_ defined by enclosing an arbitrary sequence of characters in double-quotes(@"@). Quoted identifiers are never keywords. Thus @"select"@ is not a reserved keyword and can be used to refer to a column, while @select@ would raise a parse error. Also, contrarily to unquoted identifiers and keywords, quoted identifiers are case sensitive (@"My Quoted Id"@ is _different_ from @"my quoted id"@). A fully lowercase quoted identifier that matches @[a-zA-Z]@@[a-zA-Z0-9_]@@*@ is equivalent to the unquoted identifier obtained by removing the double-quote (so @"myid"@ is equivalent to @myid@ and to @myId@ but different from @"myId"@). Inside a quoted identifier, the double-quote character can be repeated to escape it, so @"foo "" bar"@ is a valid identifier.
 
+p. *Warning*: _quoted identifiers_ allows to declare columns with arbitrary names, and those can sometime clash with specific names used by the server. For instance, when using conditional update, the server will respond with a result-set containing a special result named @"[applied]"@. If you've declared a column with such a name, this could potentially confuse some tools and should be avoided. In general, unquoted identifiers should be preferred but if you use quoted identifiers, it is strongly advised to avoid any name enclosed by squared brackets (like @"[applied]"@) and any name that looks like a function call (like @"f(x)"@).
+
 h3(#constants). Constants
 
 CQL defines the following kind of _constants_: strings, integers, floats, booleans, uuids and blobs:
@@ -120,10 +123,19 @@
 
 p. The @<properties>@ production is use by statement that create and alter keyspaces and tables. Each @<property>@ is either a _simple_ one, in which case it just has a value, or a _map_ one, in which case it's value is a map grouping sub-options. The following will refer to one or the other as the _kind_ (_simple_ or _map_) of the property.
 
-p. A @<tablename>@ will be used to identify a table. This is an identifier representing the table name that can be preceded by a keyspace name. The keyspace name, if provided, allow to identify a table in another keyspace than the currently active one (the currently active keyspace is set through the <a href="#useStmt"><tt>USE</tt></a> statement).
+p. A @<tablename>@ will be used to identify a table. This is an identifier representing the table name that can be preceded by a keyspace name. The keyspace name, if provided, allow to identify a table in another keyspace than the currently active one (the currently active keyspace is set through the <a href=":#useStmt"><tt>USE</tt></a> statement).
 
 p. For supported @<function>@, see the section on "functions":#functions.
 
+p. Strings can be either enclosed with single quotes or two dollar characters. The second syntax has been introduced to allow strings that contain single quotes. Typical candidates for such strings are source code fragments for user-defined functions.
+
+__Sample:__
+
+bc(sample).. 
+  'some string value'
+
+  $$double-dollar string can contain single ' quotes$$
+p. 
 
 h3(#preparedStatement). Prepared Statement
 
@@ -371,7 +383,7 @@
 
 h4. Other considerations:
 
-* When "inserting":#insertStmt/"updating":#updateStmt a given row, not all columns needs to be defined (except for those part of the key), and missing columns occupy no space on disk. Furthermore, adding new columns (see <a href=#alterStmt><tt>ALTER TABLE</tt></a>) is a constant time operation. There is thus no need to try to anticipate future usage (or to cry when you haven't) when creating a table.
+* When "inserting":#insertStmt / "updating":#updateStmt a given row, not all columns needs to be defined (except for those part of the key), and missing columns occupy no space on disk. Furthermore, adding new columns (see <a href=#alterStmt><tt>ALTER TABLE</tt></a>) is a constant time operation. There is thus no need to try to anticipate future usage (or to cry when you haven't) when creating a table.
 
 
 h3(#alterTableStmt). ALTER TABLE
@@ -578,7 +590,6 @@
 bc(syntax).. 
 <drop-trigger-stmt> ::= DROP TRIGGER ( IF EXISTS )? ( <triggername> )?
                             ON <tablename>
-
 p. 
 __Sample:__
 
@@ -587,6 +598,162 @@
 
 @DROP TRIGGER@ statement removes the registration of a trigger created using @CREATE TRIGGER@.
 
+h3(#createFunctionStmt). CREATE FUNCTION
+
+__Syntax:__
+
+bc(syntax).. 
+<create-function-stmt> ::= CREATE ( OR REPLACE )? 
+                            FUNCTION ( IF NOT EXISTS )?
+                            ( <keyspace> '.' )? <function-name>
+                            '(' <arg-name> <arg-type> ( ',' <arg-name> <arg-type> )* ')'
+                            ( CALLED | RETURNS NULL ) ON NULL INPUT
+                            RETURNS <type>
+                            LANGUAGE <language>
+                            AS <body>
+p. 
+__Sample:__
+
+bc(sample). 
+CREATE OR REPLACE FUNCTION somefunction
+    ( somearg int, anotherarg text, complexarg frozen<someUDT>, listarg list<bigint> )
+    RETURNS NULL ON NULL INPUT
+    RETURNS text
+    LANGUAGE java
+    AS $$
+       // some Java code
+    $$;
+CREATE FUNCTION akeyspace.fname IF NOT EXISTS
+    ( someArg int )
+    CALLED ON NULL INPUT
+    RETURNS text
+    LANGUAGE java
+    AS $$
+       // some Java code
+    $$;
+
+@CREATE FUNCTION@ creates or replaces a user-defined function.
+
+h4(#functionSignature). Function Signature
+
+Signatures are used to distinguish individual functions. The signature consists of:
+
+# The fully qualified function name - i.e _keyspace_ plus _function-name_
+# The concatenated list of all argument types
+
+Note that keyspace names, function names and argument types are subject to the default naming conventions and case-sensitivity rules.
+
+@CREATE FUNCTION@ with the optional @OR REPLACE@ keywords either creates a function or replaces an existing one with the same signature. A @CREATE FUNCTION@ without @OR REPLACE@ fails if a function with the same signature already exists.
+
+Behavior on invocation with @null@ values must be defined for each function. There are two options:
+
+# @RETURNS NULL ON NULL INPUT@ declares that the function will always return @null@ if any of the input arguments is @null@.
+# @CALLED ON NULL INPUT@ declares that the function will always be executed.
+
+If the optional @IF NOT EXISTS@ keywords are used, the function will only be created if another function with the same signature does not exist.
+
+@OR REPLACE@ and @IF NOT EXIST@ cannot be used together.
+
+Functions belong to a keyspace. If no keyspace is specified in @<function-name>@, the current keyspace is used (i.e. the keyspace specified using the "@USE@":#useStmt statement). It is not possible to create a user-defined function in one of the system keyspaces.
+
+See the section on "user-defined functions":#udfs for more information.
+
+h3(#dropFunctionStmt). DROP FUNCTION
+
+__Syntax:__
+
+bc(syntax).. 
+<drop-function-stmt> ::= DROP FUNCTION ( IF EXISTS )?
+                         ( <keyspace> '.' )? <function-name>
+                         ( '(' <arg-type> ( ',' <arg-type> )* ')' )?
+
+p. 
+__Sample:__
+
+bc(sample). 
+DROP FUNCTION myfunction;
+DROP FUNCTION mykeyspace.afunction;
+DROP FUNCTION afunction ( int );
+DROP FUNCTION afunction ( text );
+
+@DROP FUNCTION@ statement removes a function created using @CREATE FUNCTION@.
+You must specify the argument types ("signature":#functionSignature ) of the function to drop if there are multiple functions with the same name but a different signature (overloaded functions).
+
+@DROP FUNCTION@ with the optional @IF EXISTS@ keywords drops a function if it exists.
+
+h3(#createAggregateStmt). CREATE AGGREGATE
+
+__Syntax:__
+
+bc(syntax).. 
+<create-aggregate-stmt> ::= CREATE ( OR REPLACE )? 
+                            AGGREGATE ( IF NOT EXISTS )?
+                            ( <keyspace> '.' )? <aggregate-name>
+                            '(' <arg-type> ( ',' <arg-type> )* ')'
+                            SFUNC <state-functionname>
+                            STYPE <state-type>
+                            ( FINALFUNC <final-functionname> )?
+                            ( INITCOND <init-cond> )?
+p. 
+__Sample:__
+
+bc(sample). 
+CREATE AGGREGATE myaggregate ( val text )
+  SFUNC myaggregate_state
+  STYPE text
+  FINALFUNC myaggregate_final
+  INITCOND 'foo';
+
+See the section on "user-defined aggregates":#udas for a complete example.
+
+@CREATE AGGREGATE@ creates or replaces a user-defined aggregate.
+
+@CREATE AGGREGATE@ with the optional @OR REPLACE@ keywords either creates an aggregate or replaces an existing one with the same signature. A @CREATE AGGREGATE@ without @OR REPLACE@ fails if an aggregate with the same signature already exists.
+
+@CREATE AGGREGATE@ with the optional @IF NOT EXISTS@ keywords either creates an aggregate if it does not already exist.
+
+@OR REPLACE@ and @IF NOT EXIST@ cannot be used together.
+
+Aggregates belong to a keyspace. If no keyspace is specified in @<aggregate-name>@, the current keyspace is used (i.e. the keyspace specified using the "@USE@":#useStmt statement). It is not possible to create a user-defined aggregate in one of the system keyspaces.
+
+Signatures for user-defined aggregates follow the "same rules":#functionSignature as for user-defined functions.
+
+@STYPE@ defines the type of the state value and must be specified.
+
+The optional @INITCOND@ defines the initial state value for the aggregate. It defaults to @null@. A non-@null@ @INITCOND@ must be specified for state functions that are declared with @RETURNS NULL ON NULL INPUT@.
+
+@SFUNC@ references an existing function to be used as the state modifying function. The type of first argument of the state function must match @STYPE@. The remaining argument types of the state function must match the argument types of the aggregate function. State is not updated for state functions declared with @RETURNS NULL ON NULL INPUT@ and called with @null@.
+
+The optional @FINALFUNC@ is called just before the aggregate result is returned. It must take only one argument with type @STYPE@. The return type of the @FINALFUNC@ may be a different type. A final function declared with @RETURNS NULL ON NULL INPUT@ means that the aggregate's return value will be @null@, if the last state is @null@.
+
+If no @FINALFUNC@ is defined, the overall return type of the aggregate function is @STYPE@.  If a @FINALFUNC@ is defined, it is the return type of that function.
+
+See the section on "user-defined aggregates":#udas for more information.
+
+h3(#dropAggregateStmt). DROP AGGREGATE
+
+__Syntax:__
+
+bc(syntax).. 
+<drop-aggregate-stmt> ::= DROP AGGREGATE ( IF EXISTS )?
+                         ( <keyspace> '.' )? <aggregate-name>
+                         ( '(' <arg-type> ( ',' <arg-type> )* ')' )?
+p. 
+
+__Sample:__
+
+bc(sample). 
+DROP AGGREGATE myAggregate;
+DROP AGGREGATE myKeyspace.anAggregate;
+DROP AGGREGATE someAggregate ( int );
+DROP AGGREGATE someAggregate ( text );
+
+The @DROP AGGREGATE@ statement removes an aggregate created using @CREATE AGGREGATE@.  You must specify the argument types of the aggregate to drop if there are multiple aggregates with the same name but a different signature (overloaded aggregates).
+
+@DROP AGGREGATE@ with the optional @IF EXISTS@ keywords drops an aggregate if it exists, and does nothing if a function with the signature does not exist.
+
+Signatures for user-defined aggregates follow the "same rules":#functionSignature as for user-defined functions.
+
 h2(#dataManipulation). Data Manipulation
 
 h3(#insertStmt). INSERT
@@ -595,11 +762,15 @@
 
 bc(syntax).. 
 <insertStatement> ::= INSERT INTO <tablename>
-                             '(' <identifier> ( ',' <identifier> )* ')'
-                      VALUES '(' <term-or-literal> ( ',' <term-or-literal> )* ')'
+                      ( ( <name-list> VALUES <value-list> )
+                      | ( JSON <string> ))
                       ( IF NOT EXISTS )?
                       ( USING <option> ( AND <option> )* )?
 
+<names-list> ::= '(' <identifier> ( ',' <identifier> )* ')'
+
+<value-list> ::= '(' <term-or-literal> ( ',' <term-or-literal> )* ')'
+
 <term-or-literal> ::= <term>
                     | <collection-literal>
 
@@ -608,12 +779,14 @@
 p. 
 __Sample:__
 
-bc(sample). 
+bc(sample).. 
 INSERT INTO NerdMovies (movie, director, main_actor, year)
                 VALUES ('Serenity', 'Joss Whedon', 'Nathan Fillion', 2005)
 USING TTL 86400;
 
-The @INSERT@ statement writes one or more columns for a given row in a table. Note that since a row is identified by its @PRIMARY KEY@, at least the columns composing it must be specified.
+INSERT INTO NerdMovies JSON '{"movie": "Serenity", "director": "Joss Whedon", "year": 2005}'
+p. 
+The @INSERT@ statement writes one or more columns for a given row in a table. Note that since a row is identified by its @PRIMARY KEY@, at least the columns composing it must be specified.  The list of columns to insert to must be supplied when using the @VALUES@ syntax.  When using the @JSON@ syntax, they are optional.  See the section on "@INSERT JSON@":#insertJson for more details.
 
 Note that unlike in SQL, @INSERT@ does not check the prior existence of the row by default: the row is created if none existed before, and updated otherwise. Furthermore, there is no mean to know which of creation or update happened.
 
@@ -786,7 +959,7 @@
 __Syntax:__
 
 bc(syntax).. 
-<select-stmt> ::= SELECT <select-clause>
+<select-stmt> ::= SELECT ( JSON )? <select-clause>
                   FROM <tablename>
                   ( WHERE <where-clause> )?
                   ( ORDER BY <order-by> )?
@@ -822,6 +995,8 @@
 bc(sample).. 
 SELECT name, occupation FROM users WHERE userid IN (199, 200, 207);
 
+SELECT JSON name, occupation FROM users WHERE userid = 199;
+
 SELECT name AS user_name, occupation AS user_occupation FROM users;
 
 SELECT time, value
@@ -835,13 +1010,13 @@
 SELECT COUNT(*) AS user_count FROM users;
 
 p. 
-The @SELECT@ statements reads one or more columns for one or more rows in a table. It returns a result-set of rows, where each row contains the collection of columns corresponding to the query.
+The @SELECT@ statements reads one or more columns for one or more rows in a table. It returns a result-set of rows, where each row contains the collection of columns corresponding to the query.  If the @JSON@ keyword is used, the results for each row will contain only a single column named "json".  See the section on "@SELECT JSON@":#selectJson for more details.
 
 h4(#selectSelection). @<select-clause>@
 
 The @<select-clause>@ determines which columns needs to be queried and returned in the result-set. It consists of either the comma-separated list of <selector> or the wildcard character (@*@) to select all the columns defined for the table.
 
-A @<selector>@ is either a column name to retrieve, or a @<function>@ of one or multiple column names. The functions allows are the same that for @<term>@ and are describe in the "function section":#function. In addition to these generic functions, the @WRITETIME@ (resp. @TTL@) function allows to select the timestamp of when the column was inserted (resp. the time to live (in seconds) for the column (or null if the column has no expiration set)).
+A @<selector>@ is either a column name to retrieve or a @<function>@ of one or more @<term>@s. The function allowed are the same as for @<term>@ and are described in the "function section":#functions. In addition to these generic functions, the @WRITETIME@ (resp. @TTL@) function allows to select the timestamp of when the column was inserted (resp. the time to live (in seconds) for the column (or null if the column has no expiration set)).
 
 Any @<selector>@ can be aliased using @AS@ keyword (see examples). Please note that @<where-clause>@ and @<order-by>@ clause should refer to the columns by their original names and not by their aliases.
 
@@ -901,7 +1076,7 @@
 
 h4(#selectOrderBy). @<order-by>@
 
-The @ORDER BY@ option allows to select the order of the returned results. It takes as argument a list of column names along with the order for the column (@ASC@ for ascendant and @DESC@ for descendant, omitting the order being equivalent to @ASC@). Currently the possible orderings are limited (which depends on the table "@CLUSTERING ORDER@":#createTableOptions):
+The @ORDER BY@ option allows to select the order of the returned results. It takes as argument a list of column names along with the order for the column (@ASC@ for ascendant and @DESC@ for descendant, omitting the order being equivalent to @ASC@). Currently the possible orderings are limited (which depends on the table "@CLUSTERING ORDER@":#createTableOptions ):
 * if the table has been defined without any specific @CLUSTERING ORDER@, then then allowed orderings are the order induced by the clustering columns and the reverse of that one.
 * otherwise, the orderings allowed are the order of the @CLUSTERING ORDER@ option and the reversed one.
 
@@ -947,10 +1122,165 @@
 bc(sample). 
 SELECT firstname, lastname FROM users WHERE birth_year = 1981 AND country = 'FR' ALLOW FILTERING;
 
-h2(#databaseUsers). Database Users
+h2(#databaseRoles). Database Roles
+
+h3(#createRoleStmt). CREATE ROLE
+
+__Syntax:__
+
+bc(syntax).. 
+<create-role-stmt> ::= CREATE ROLE ( IF NOT EXISTS )? <identifier> ( WITH <option> ( AND <option> )* )?
+
+<option> ::= PASSWORD = <string>
+           | LOGIN = <boolean>
+           | SUPERUSER = <boolean>
+           | OPTIONS = <map_literal>
+p. 
+
+__Sample:__
+
+bc(sample). 
+CREATE ROLE new_role;
+CREATE ROLE alice WITH PASSWORD = 'password_a' AND LOGIN = true;
+CREATE ROLE bob WITH PASSWORD = 'password_b' AND LOGIN = true AND SUPERUSER = true;
+CREATE ROLE carlos WITH OPTIONS = { 'custom_option1' : 'option1_value', 'custom_option2' : 99 };
+
+By default roles do not possess @LOGIN@ privileges or @SUPERUSER@ status.
+
+"Permissions":#permissions on database resources are granted to roles; types of resources include keyspaces, tables, functions and roles themselves. Roles may be granted to other roles to create hierarchical permissions structures; in these hierarchies, permissions and @SUPERUSER@ status are inherited, but the @LOGIN@ privilege is not. 
+
+If a role has the @LOGIN@ privilege, clients may identify as that role when connecting. For the duration of that connection, the client will acquire any roles and privileges granted to that role.
+
+Only a client with with the @CREATE@ permission on the database roles resource may issue @CREATE ROLE@ requests (see the "relevant section":#permissions below), unless the client is a @SUPERUSER@. Role management in Cassandra is pluggable and custom implementations may support only a subset of the listed options.
+
+Role names should be quoted if they contain non-alphanumeric characters. 
+
+h4(#createRolePwd). Setting credentials for internal authentication
+
+Use the @WITH PASSWORD@ clause to set a password for internal authentication, enclosing the password in single quotation marks.
+If internal authentication has not been set up or the role does not have @LOGIN@ privileges, the @WITH PASSWORD@ clause is not necessary.
+
+h4(#createRoleConditional). Creating a role conditionally
+
+Attempting to create an existing role results in an invalid query condition unless the @IF NOT EXISTS@ option is used. If the option is used and the role exists, the statement is a no-op.
+
+bc(sample). 
+CREATE ROLE other_role;
+CREATE ROLE IF NOT EXISTS other_role;
+
+h3(#alterRoleStmt). ALTER ROLE
+
+__Syntax:__
+
+bc(syntax).. 
+<alter-role-stmt> ::= ALTER ROLE <identifier> ( WITH <option> ( AND <option> )* )?
+
+<option> ::= PASSWORD = <string>
+           | LOGIN = <boolean>
+           | SUPERUSER = <boolean>
+           | OPTIONS = <map_literal>
+p. 
+
+__Sample:__
+
+bc(sample). 
+ALTER ROLE bob WITH PASSWORD = 'PASSWORD_B' AND SUPERUSER = false;
+
+Conditions on executing @ALTER ROLE@ statements:
+
+* A client must have @SUPERUSER@ status to alter the @SUPERUSER@ status of another role
+* A client cannot alter the @SUPERUSER@ status of any role it currently holds
+* A client can only modify certain properties of the role with which it identified at login (e.g. @PASSWORD@)
+* To modify properties of a role, the client must be granted @ALTER@ "permission":#permissions on that role
+
+
+h3(#dropRoleStmt). DROP ROLE
+
+__Syntax:__
+
+bc(syntax).. 
+<drop-role-stmt> ::= DROP ROLE ( IF EXISTS )? <identifier>
+p. 
+
+__Sample:__
+
+bc(sample). 
+DROP ROLE alice;
+DROP ROLE IF EXISTS bob;
+
+
+@DROP ROLE@ requires the client to have @DROP@ "permission":#permissions on the role in question. In addition, client may not @DROP@ the role with which it identified at login. Finaly, only a client with @SUPERUSER@ status may @DROP@ another @SUPERUSER@ role.
+Attempting to drop a role which does not exist results in an invalid query condition unless the @IF EXISTS@ option is used. If the option is used and the role does not exist the statement is a no-op. 
+
+
+h3(#grantRoleStmt). GRANT ROLE
+
+__Syntax:__
+
+bc(syntax). 
+<grant-role-stmt> ::= GRANT <identifier> TO <identifier>
+
+
+__Sample:__
+
+bc(sample). 
+GRANT report_writer TO alice;
+
+This statement grants the @report_writer@ role to @alice@. Any permissions granted to @report_writer@ are also acquired by @alice@.
+Roles are modelled as a directed acyclic graph, so circular grants are not permitted. The following examples result in error conditions:
+
+bc(sample). 
+GRANT role_a TO role_b;
+GRANT role_b TO role_a;
+
+bc(sample). 
+GRANT role_a TO role_b;
+GRANT role_b TO role_c;
+GRANT role_c TO role_a;
+
+
+h3(#revokeRoleStmt). REVOKE ROLE
+
+__Syntax:__
+
+bc(syntax). 
+<revoke-role-stmt> ::= REVOKE <identifier> FROM <identifier>
+
+__Sample:__
+
+bc(sample). 
+REVOKE report_writer FROM alice;
+
+This statement revokes the @report_writer@ role from @alice@. Any permissions that @alice@ has acquired via the @report_writer@ role are also revoked. 
+
+h4(#listRolesStmt). LIST ROLES
+
+__Syntax:__
+
+bc(syntax). 
+<list-roles-stmt> ::= LIST ROLES ( OF <identifier> )? ( NORECURSIVE )?
+
+__Sample:__ 
+
+bc(sample). 
+LIST ROLES;
+
+Return all known roles in the system, this requires @DESCRIBE@ permission on the database roles resource.
+
+bc(sample). 
+LIST ROLES OF @alice@;
+
+Enumerate all roles granted to @alice@, including those transitively aquired.
+
+bc(sample). 
+LIST ROLES OF @bob@ NORECURSIVE
+
+List all roles directly granted to @bob@.
 
 h3(#createUserStmt). CREATE USER 
 
+Prior to the introduction of roles in Cassandra 2.2, authentication and authorization were based around the concept of a @USER@. For backward compatibility, the legacy syntax has been preserved with @USER@ centric statments becoming synonyms for the @ROLE@ based equivalents.
+
 __Syntax:__ 
 
 bc(syntax).. 
@@ -966,23 +1296,24 @@
 CREATE USER alice WITH PASSWORD 'password_a' SUPERUSER;
 CREATE USER bob WITH PASSWORD 'password_b' NOSUPERUSER;
 
-By default users do not possess @SUPERUSER@ status.
+@CREATE USER@ is equivalent to @CREATE ROLE@ where the @LOGIN@ option is @true@. So, the following pairs of statements are equivalent:
 
-"Permissions":#permissions on database resources (keyspaces and tables) are granted to users.
-USer names should be quoted if they contain non-alphanumeric characters. 
+bc(sample).. 
+CREATE USER alice WITH PASSWORD 'password_a' SUPERUSER;
+CREATE ROLE alice WITH PASSWORD = 'password_a' AND LOGIN = true AND SUPERUSER = true;
 
-h4(#createUserPwd). Setting credentials for internal authentication
+CREATE USER IF EXISTS alice WITH PASSWORD 'password_a' SUPERUSER;
+CREATE ROLE IF EXISTS alice WITH PASSWORD = 'password_a' AND LOGIN = true AND SUPERUSER = true;
 
-Use the @WITH PASSWORD@ clause to set a password for internal authentication, enclosing the password in single quotation marks.
-If internal authentication has not been set up the @WITH PASSWORD@ clause is not necessary.
+CREATE USER alice WITH PASSWORD 'password_a' NOSUPERUSER;
+CREATE ROLE alice WITH PASSWORD = 'password_a' AND LOGIN = true AND SUPERUSER = false;
 
-h4(#createUserConditional). Creating a user conditionally
+CREATE USER alice WITH PASSWORD 'password_a' NOSUPERUSER;
+CREATE ROLE alice WITH PASSWORD = 'password_a' WITH LOGIN = true;
 
-Attempting to create an existing user results in an invalid query condition unless the @IF NOT EXISTS@ option is used. If the option is used and the user exists, the statement is a no-op.
-
-bc(sample). 
-CREATE USER carlos;
-CREATE USER IF NOT EXISTS carlos;
+CREATE USER alice WITH PASSWORD 'password_a';
+CREATE ROLE alice WITH PASSWORD = 'password_a' WITH LOGIN = true;
+p. 
 
 h3(#alterUserStmt). ALTER USER 
 
@@ -999,11 +1330,6 @@
 ALTER USER alice WITH PASSWORD 'PASSWORD_A';
 ALTER USER bob SUPERUSER;
 
-@ALTER USER@ requires @SUPERUSER@ status, with two caveats:
-
-* A user cannot alter its own @SUPERUSER@ status
-* A user without @SUPERUSER@ status is permitted to modify a subset of it's own properties (e.g. its @PASSWORD@)
-
 h3(#dropUserStmt). DROP USER 
 
 __Syntax:__ 
@@ -1018,9 +1344,6 @@
 DROP USER alice;
 DROP USER IF EXISTS bob;
 
-@DROP USER@ requires @SUPERUSER@ status, and users are not permitted to @DROP@ themselves.
-Attempting to drop a user which does not exist results in an invalid query condition unless the @IF EXISTS@ option is used. If the option is used and the user does not exist the statement is a no-op. 
-
 h3(#listUsersStmt). LIST USERS
 
 __Syntax:__
@@ -1033,15 +1356,25 @@
 bc(sample). 
 LIST USERS;
 
-Return all known users in the system.
+This statement is equivalent to
+
+bc(sample). 
+LIST ROLES;
+
+but only roles with the @LOGIN@ privilege are included in the output.
+
 
 h2(#dataControl). Data Control
 
 h3(#permissions). Permissions 
 
-Permissions on resources are granted to users and data resources in Cassandra are organized hierarchically, like so: @ALL KEYSPACES@ -> @KEYSPACE@ -> @TABLE@
+Permissions on resources are granted to roles; there are several different types of resources in Cassandra and each type is modelled hierarchically:
 
-Permissions can be granted at any level of the hierarchy and they flow downwards. So granting a permission on a resource higher up the chain automatically grants that same permission on all resources lower down. For example, granting @SELECT@ on a @KEYSPACE@ automatically grants it on all @TABLES@ in that @KEYSPACE@. 
+* The hierarchy of Data resources, Keyspaces and Tables has the structure @ALL KEYSPACES@ -> @KEYSPACE@ -> @TABLE@
+* Function resources have the structure @ALL FUNCTIONS@ -> @KEYSPACE@ -> @FUNCTION@
+* Resources representing roles have the structure @ALL ROLES@ -> @ROLE@
+
+Permissions can be granted at any level of these hierarchies and they flow downwards. So granting a permission on a resource higher up the chain automatically grants that same permission on all resources lower down. For example, granting @SELECT@ on a @KEYSPACE@ automatically grants it on all @TABLES@ in that @KEYSPACE@. Likewise, granting a permission on @ALL FUNCTIONS@ grants it on every defined function, regardless of which keyspace it is scoped in. It is also possible to grant permissions on all functions scoped to a particular keyspace. 
 
 Modifications to permissions are visible to existing client sessions; that is, connections need not be re-established following permissions changes.
 
@@ -1052,26 +1385,52 @@
 * @SELECT@
 * @MODIFY@
 * @AUTHORIZE@
+* @DESCRIBE@
+* @EXECUTE@
 
+Not all permissions are applicable to every type of resource. For instance, @EXECUTE@ is only relevant in the context of functions; granting @EXECUTE@ on a resource representing a table is nonsensical. Attempting to @GRANT@ a permission on resource to which it cannot be applied results in an error response. The following illustrates which permissions can be granted on which types of resource, and which statements are enabled by that permission.
 
 |_. permission |_. resource                   |_. operations        |
 | @CREATE@     | @ALL KEYSPACES@              |@CREATE KEYSPACE@ ==<br>== @CREATE TABLE@ in any keyspace|
 | @CREATE@     | @KEYSPACE@                   |@CREATE TABLE@ in specified keyspace|
+| @CREATE@     | @ALL FUNCTIONS@              |@CREATE FUNCTION@ in any keyspace ==<br>== @CREATE AGGREGATE@ in any keyspace|
+| @CREATE@     | @ALL FUNCTIONS IN KEYSPACE@  |@CREATE FUNCTION@ in keyspace ==<br>== @CREATE AGGREGATE@ in keyspace|
+| @CREATE@     | @ALL ROLES@                  |@CREATE ROLE@|
 | @ALTER@      | @ALL KEYSPACES@              |@ALTER KEYSPACE@ ==<br>== @ALTER TABLE@ in any keyspace|
 | @ALTER@      | @KEYSPACE@                   |@ALTER KEYSPACE@ ==<br>== @ALTER TABLE@ in keyspace|
-| @ALTER@      | @TABLE@                      |@ALTER TABLE@
+| @ALTER@      | @TABLE@                      |@ALTER TABLE@|
+| @ALTER@      | @ALL FUNCTIONS@              |@CREATE FUNCTION@ replacing any existing ==<br>== @CREATE AGGREGATE@ replacing any existing|
+| @ALTER@      | @ALL FUNCTIONS IN KEYSPACE@  |@CREATE FUNCTION@ replacing existing in keyspace ==<br>== @CREATE AGGREGATE@ replacing any existing in keyspace|
+| @ALTER@      | @FUNCTION@                   |@CREATE FUNCTION@ replacing existing ==<br>== @CREATE AGGREGATE@ replacing existing|
+| @ALTER@      | @ALL ROLES@                  |@ALTER ROLE@ on any role|
+| @ALTER@      | @ROLE@                       |@ALTER ROLE@|
 | @DROP@       | @ALL KEYSPACES@              |@DROP KEYSPACE@ ==<br>== @DROP TABLE@ in any keyspace|
 | @DROP@       | @KEYSPACE@                   |@DROP TABLE@ in specified keyspace|
 | @DROP@       | @TABLE@                      |@DROP TABLE@|
+| @DROP@       | @ALL FUNCTIONS@              |@DROP FUNCTION@ in any keyspace ==<br>== @DROP AGGREGATE@ in any existing|
+| @DROP@       | @ALL FUNCTIONS IN KEYSPACE@  |@DROP FUNCTION@ in keyspace ==<br>== @DROP AGGREGATE@ in existing|
+| @DROP@       | @FUNCTION@                   |@DROP FUNCTION@|
+| @DROP@       | @ALL ROLES@                  |@DROP ROLE@ on any role|
+| @DROP@       | @ROLE@                       |@DROP ROLE@|
 | @SELECT@     | @ALL KEYSPACES@              |@SELECT@ on any table|
 | @SELECT@     | @KEYSPACE@                   |@SELECT@ on any table in keyspace|
 | @SELECT@     | @TABLE@                      |@SELECT@ on specified table|
 | @MODIFY@     | @ALL KEYSPACES@              |@INSERT@ on any table ==<br>== @UPDATE@ on any table ==<br>== @DELETE@ on any table ==<br>== @TRUNCATE@ on any table|
-| @MODIFY@     | @KEYSPACE@                   |@INSERT@ on any table in keyspace ==<br>== @UPDATE@ on any table in keyspace ==<br>== @DELETE@ on any table in keyspace ==<br>== @TRUNCATE@ on any table in keyspace
+| @MODIFY@     | @KEYSPACE@                  |@INSERT@ on any table in keyspace ==<br>== @UPDATE@ on any table in keyspace ==<br>  == @DELETE@ on any table in keyspace ==<br>== @TRUNCATE@ on any table in keyspace
 | @MODIFY@     | @TABLE@                      |@INSERT@ ==<br>== @UPDATE@ ==<br>== @DELETE@ ==<br>== @TRUNCATE@|
 | @AUTHORIZE@  | @ALL KEYSPACES@              |@GRANT PERMISSION@ on any table ==<br>== @REVOKE PERMISSION@ on any table|
 | @AUTHORIZE@  | @KEYSPACE@                   |@GRANT PERMISSION@ on table in keyspace ==<br>== @REVOKE PERMISSION@ on table in keyspace|
 | @AUTHORIZE@  | @TABLE@                      |@GRANT PERMISSION@ ==<br>== @REVOKE PERMISSION@ |
+| @AUTHORIZE@  | @ALL FUNCTIONS@              |@GRANT PERMISSION@ on any function ==<br>== @REVOKE PERMISSION@ on any function|
+| @AUTHORIZE@  | @ALL FUNCTIONS IN KEYSPACE@  |@GRANT PERMISSION@ in keyspace ==<br>== @REVOKE PERMISSION@ in keyspace|
+| @AUTHORIZE@  | @ALL FUNCTIONS IN KEYSPACE@  |@GRANT PERMISSION@ in keyspace ==<br>== @REVOKE PERMISSION@ in keyspace|
+| @AUTHORIZE@  | @FUNCTION@                   |@GRANT PERMISSION@ ==<br>== @REVOKE PERMISSION@|
+| @AUTHORIZE@  | @ALL ROLES@                  |@GRANT ROLE@ grant any role ==<br>== @REVOKE ROLE@ revoke any role|
+| @AUTHORIZE@  | @ROLES@                      |@GRANT ROLE@ grant role ==<br>== @REVOKE ROLE@ revoke role|
+| @DESCRIBE@   | @ALL ROLES@                  |@LIST ROLES@ all roles or only roles granted to another, specified role|
+| @EXECUTE@    | @ALL FUNCTIONS@              |@SELECT@, @INSERT@, @UPDATE@ using any function ==<br>== use of any function in @CREATE AGGREGATE@|
+| @EXECUTE@    | @ALL FUNCTIONS IN KEYSPACE@  |@SELECT@, @INSERT@, @UPDATE@ using any function in keyspace ==<br>== use of any function in keyspace in @CREATE AGGREGATE@|
+| @EXECUTE@    | @FUNCTION@                   |@SELECT@, @INSERT@, @UPDATE@ using function ==<br>== use of function in @CREATE AGGREGATE@|
 
 
 h3(#grantPermissionsStmt). GRANT PERMISSION
@@ -1081,29 +1440,51 @@
 bc(syntax).. 
 <grant-permission-stmt> ::= GRANT ( ALL ( PERMISSIONS )? | <permission> ( PERMISSION )? ) ON <resource> TO <identifier>
 
-<permission> ::= CREATE | ALTER | DROP | SELECT | MODIFY | AUTHORIZE 
+<permission> ::= CREATE | ALTER | DROP | SELECT | MODIFY | AUTHORIZE | DESRIBE | EXECUTE
 
 <resource> ::= ALL KEYSPACES
              | KEYSPACE <identifier>
              | ( TABLE )? <tablename>
+             | ALL ROLES
+             | ROLE <identifier>
+             | ALL FUNCTIONS ( IN KEYSPACE <identifier> )?
+             | FUNCTION <functionname>
 p. 
 
 __Sample:__ 
 
 bc(sample). 
-GRANT SELECT ON ALL KEYSPACES TO alice;
+GRANT SELECT ON ALL KEYSPACES TO data_reader;
 
-This gives @alice@ permissions to execute @SELECT@ statements on any table across all keyspaces
+This gives any user with the role @data_reader@ permission to execute @SELECT@ statements on any table across all keyspaces
 
 bc(sample). 
-GRANT MODIFY ON KEYSPACE keyspace1 TO bob;
+GRANT MODIFY ON KEYSPACE keyspace1 TO data_writer;
 
-This gives @bob@ permissions to perform @UPDATE@, @INSERT@, @UPDATE@, @DELETE@ and @TRUNCATE@ queries on all tables in the @keyspace1@ keyspace
+This give any user with the role @data_writer@ permission to perform @UPDATE@, @INSERT@, @UPDATE@, @DELETE@ and @TRUNCATE@ queries on all tables in the @keyspace1@ keyspace
 
 bc(sample). 
-GRANT DROP ON keyspace1.table1 TO carlos;
+GRANT DROP ON keyspace1.table1 TO schema_owner;
 
-This gives @carlos@ permissions to @DROP@ @keyspace1.table1@.
+This gives any user with the @schema_owner@ role permissions to @DROP@ @keyspace1.table1@.
+
+bc(sample). 
+GRANT EXECUTE ON FUNCTION keyspace1.user_function( int ) TO report_writer;
+
+This grants any user with the @report_writer@ role permission to execute @SELECT@, @INSERT@ and @UPDATE@ queries which use the function @keyspace1.user_function( int )@
+
+bc(sample). 
+GRANT DESCRIBE ON ALL ROLES TO role_admin;
+
+This grants any user with the @role_admin@ role permission to view any and all roles in the system with a @LIST ROLES@ statement
+
+h4(#grantAll). GRANT ALL 
+
+When the @GRANT ALL@ form is used, the appropriate set of permissions is determined automatically based on the target resource.
+
+h4(#autoGrantPermissions). Automatic Granting
+
+When a resource is created, via a @CREATE KEYSPACE@, @CREATE TABLE@, @CREATE FUNCTION@, @CREATE AGGREGATE@ or @CREATE ROLE@ statement, the creator (the role the database user who issues the statement is identified as), is automatically granted all applicable permissions on the new resource.
 
 
 h3(#revokePermissionsStmt). REVOKE PERMISSION
@@ -1113,19 +1494,25 @@
 bc(syntax).. 
 <revoke-permission-stmt> ::= REVOKE ( ALL ( PERMISSIONS )? | <permission> ( PERMISSION )? ) ON <resource> FROM <identifier>
 
-<permission> ::= CREATE | ALTER | DROP | SELECT | MODIFY | AUTHORIZE 
+<permission> ::= CREATE | ALTER | DROP | SELECT | MODIFY | AUTHORIZE | DESRIBE | EXECUTE
 
 <resource> ::= ALL KEYSPACES
              | KEYSPACE <identifier>
              | ( TABLE )? <tablename>
+             | ALL ROLES
+             | ROLE <identifier>
+             | ALL FUNCTIONS ( IN KEYSPACE <identifier> )?
+             | FUNCTION <functionname>
 p. 
 
 __Sample:__ 
 
 bc(sample).. 
-REVOKE SELECT ON ALL KEYSPACES FROM alice;
-REVOKE MODIFY ON KEYSPACE keyspace1 FROM bob;
-REVOKE DROP ON keyspace1.table1 FROM carlos;
+REVOKE SELECT ON ALL KEYSPACES FROM data_reader;
+REVOKE MODIFY ON KEYSPACE keyspace1 FROM data_writer;
+REVOKE DROP ON keyspace1.table1 FROM schema_owner;
+REVOKE EXECUTE ON FUNCTION keyspace1.user_function( int ) FROM report_writer;
+REVOKE DESCRIBE ON ALL ROLES FROM role_admin;
 p. 
 
 h4(#listPermissionsStmt). LIST PERMISSIONS
@@ -1140,6 +1527,10 @@
 <resource> ::= ALL KEYSPACES
              | KEYSPACE <identifier>
              | ( TABLE )? <tablename>
+             | ALL ROLES
+             | ROLE <identifier>
+             | ALL FUNCTIONS ( IN KEYSPACE <identifier> )?
+             | FUNCTION <functionname>
 p. 
 
 __Sample:__
@@ -1147,17 +1538,17 @@
 bc(sample). 
 LIST ALL PERMISSIONS OF alice;
 
-Show all permissions granted to @alice@. 
+Show all permissions granted to @alice@, including those acquired transitively from any other roles. 
 
 bc(sample). 
 LIST ALL PERMISSIONS ON keyspace1.table1 OF bob;
 
-Show all permissions on @keyspace1.table1@ granted to @bob@. This also includes any permissions higher up the resource hierarchy which can be applied to @keyspace1.table1@. For example, should @bob@ have @ALTER@ permission on @keyspace1@, that would be included in the results of this query. Adding the @NORECURSIVE@ switch restricts the results to only those permissions which were directly granted to @bob@.
+Show all permissions on @keyspace1.table1@ granted to @bob@, including those acquired transitively from any other roles. This also includes any permissions higher up the resource hierarchy which can be applied to @keyspace1.table1@. For example, should @bob@ have @ALTER@ permission on @keyspace1@, that would be included in the results of this query. Adding the @NORECURSIVE@ switch restricts the results to only those permissions which were directly granted to @bob@ or one of @bob@'s roles.
 
 bc(sample). 
 LIST SELECT PERMISSIONS OF carlos;
 
-Show any permissions granted to @carlos@, limited to @SELECT@ permissions on any resource.
+Show any permissions granted to @carlos@ or any of @carlos@'s roles, limited to @SELECT@ permissions on any resource.
 
 h2(#types). Data Types
 
@@ -1174,14 +1565,18 @@
                 | blob
                 | boolean
                 | counter
+                | date
                 | decimal
                 | double
                 | float
                 | inet
                 | int
+                | smallint
                 | text
+                | time
                 | timestamp
                 | timeuuid
+                | tinyint
                 | uuid
                 | varchar
                 | varint
@@ -1200,21 +1595,25 @@
 |@blob@     |   blobs              |Arbitrary bytes (no validation)|
 |@boolean@  |   booleans           |true or false|
 |@counter@  |   integers           |Counter column (64-bit signed value). See "Counters":#counters for details|
+|@date@     |   integers, strings  |A date (with no corresponding time value).  See "Working with dates":#usingdates below for more information.|
 |@decimal@  |   integers, floats   |Variable-precision decimal|
 |@double@   |   integers           |64-bit IEEE-754 floating point|
 |@float@    |   integers, floats   |32-bit IEEE-754 floating point|
 |@inet@     |   strings            |An IP address. It can be either 4 bytes long (IPv4) or 16 bytes long (IPv6). There is no @inet@ constant, IP address should be inputed as strings|
 |@int@      |   integers           |32-bit signed int|
+|@smallint@ |   integers           |16-bit signed int|
 |@text@     |   strings            |UTF8 encoded string|
-|@timestamp@|   integers, strings  |A timestamp. Strings constant are allow to input timestamps as dates, see "Working with dates":#usingdates below for more information.|
+|@time@     |   integers, strings  |A time with nanosecond precision.  See "Working with time":#usingtime below for more information.|
+|@timestamp@|   integers, strings  |A timestamp. Strings constant are allow to input timestamps as dates, see "Working with timestamps":#usingtimestamps below for more information.|
 |@timeuuid@ |   uuids              |Type 1 UUID. This is generally used as a "conflict-free" timestamp. Also see the "functions on Timeuuid":#timeuuidFun|
+|@tinyint@  |   integers           |8-bit signed int|
 |@uuid@     |   uuids              |Type 1 or type 4 UUID|
 |@varchar@  |   strings            |UTF8 encoded string|
 |@varint@   |   integers           |Arbitrary-precision integer|
 
 For more information on how to use the collection types, see the "Working with collections":#collections section below.
 
-h3(#usingdates). Working with dates
+h3(#usingtimestamps). Working with timestamps
 
 Values of the @timestamp@ type are encoded as 64-bit signed integers representing a number of milliseconds since the standard base time known as "the epoch": January 1 1970 at 00:00:00 GMT.
 
@@ -1248,6 +1647,31 @@
 In that case, the time of day will default to 00:00:00, in the specified or default time zone.
 
 
+h3(#usingdates). Working with dates
+
+Values of the @date@ type are encoded as 32-bit unsigned integers representing a number of days with "the epoch" at the center of the range (2^31). Epoch is January 1st, 1970
+
+A date can be input in CQL as an unsigned integer as defined above.
+
+They can also be input as string literals in the following format:
+
+* @2014-01-01@
+
+
+h3(#usingtime). Working with time
+
+Values of the @time@ type are encoded as 64-bit signed integers representing the number of nanoseconds since midnight.
+
+A time can be input in CQL as simple long integers, giving the number of nanoseconds since midnight.
+
+They can also be input as string literals in any of the following formats:
+
+* @08:12:54@
+* @08:12:54.123@
+* @08:12:54.123456@
+* @08:12:54.123456789@
+
+
 h3(#counters). Counters
 
 The @counter@ type is used to define _counter columns_. A counter column is a column whose value is a 64-bit signed integer and on which 2 operations are supported: incrementation and decrementation (see "@UPDATE@":#updateStmt for syntax).  Note the value of a counter cannot be set. A counter doesn't exist until first incremented/decremented, and the first incrementation/decrementation is made as if the previous value was 0. Deletion of counter columns is supported but have some limitations (see the "Cassandra Wiki":http://wiki.apache.org/cassandra/Counters for more information).
@@ -1373,7 +1797,7 @@
 
 h2(#functions). Functions
 
-CQL3 supports a few functions (more to come). Currently, it only support functions on values (functions that transform one or more column values into a new value) and in particular aggregation functions are not supported. The functions supported are described below:
+CQL3 distinguishes between built-in functions (so called 'native functions') and "user-defined functions":#udfs.  CQL3 includes several native functions, described below:
 
 h3(#tokenFun). Token
 
@@ -1403,32 +1827,245 @@
 
 h4. @now@
 
-The @now@ function takes no arguments and generates a new unique timeuuid (at the time where the statement using it is executed). Note that this method is useful for insertion but is largely non-sensical in @WHERE@ clauses. For instance, a query of the form
+The @now@ function takes no arguments and generates, on the coordinator node, a new unique timeuuid (at the time where the statement using it is executed). Note that this method is useful for insertion but is largely non-sensical in @WHERE@ clauses. For instance, a query of the form
 
 bc(sample). 
 SELECT * FROM myTable WHERE t = now()
 
+
 will never return any result by design, since the value returned by @now()@ is guaranteed to be unique.
 
 h4. @minTimeuuid@ and @maxTimeuuid@
 
-The @minTimeuuid@ (resp. @maxTimeuuid@) function takes a @timestamp@ value @t@ (which can be "either a timestamp or a date string":#usingdates) and return a _fake_ @timeuuid@ corresponding to the _smallest_ (resp. _biggest_) possible @timeuuid@ having for timestamp @t@. So for instance:
- 
+The @minTimeuuid@ (resp. @maxTimeuuid@) function takes a @timestamp@ value @t@ (which can be "either a timestamp or a date string":#usingtimestamps ) and return a _fake_ @timeuuid@ corresponding to the _smallest_ (resp. _biggest_) possible @timeuuid@ having for timestamp @t@. So for instance:
+
+
 bc(sample). 
 SELECT * FROM myTable WHERE t > maxTimeuuid('2013-01-01 00:05+0000') AND t < minTimeuuid('2013-02-02 10:00+0000')
- 
+
+
 will select all rows where the @timeuuid@ column @t@ is strictly older than '2013-01-01 00:05+0000' but strictly younger than '2013-02-02 10:00+0000'.  Please note that @t >= maxTimeuuid('2013-01-01 00:05+0000')@ would still _not_ select a @timeuuid@ generated exactly at '2013-01-01 00:05+0000' and is essentially equivalent to @t > maxTimeuuid('2013-01-01 00:05+0000')@.
 
 _Warning_: We called the values generated by @minTimeuuid@ and @maxTimeuuid@ _fake_ UUID because they do no respect the Time-Based UUID generation process specified by the "RFC 4122":http://www.ietf.org/rfc/rfc4122.txt. In particular, the value returned by these 2 methods will not be unique. This means you should only use those methods for querying (as in the example above). Inserting the result of those methods is almost certainly _a bad idea_.
 
-h4. @dateOf@ and @unixTimestampOf@
+h3(#timeFun). Time conversion functions
 
-The @dateOf@ and @unixTimestampOf@ functions take a @timeuuid@ argument and extract the embedded timestamp. However, while the @dateof@ function return it with the @timestamp@ type (that most client, including cqlsh, interpret as a date), the @unixTimestampOf@ function returns it as a @bigint@ raw value.
+A number of functions are provided to "convert" a @timeuuid@, a @timestamp@ or a @date@ into another @native@ type.
+
+|_. function name    |_. input type   |_. description|
+|@toDate@            |@timeuuid@      |Converts the @timeuuid@ argument into a @date@ type|
+|@toDate@            |@timestamp@     |Converts the @timestamp@ argument into a @date@ type|
+|@toTimestamp@       |@timeuuid@      |Converts the @timeuuid@ argument into a @timestamp@ type|
+|@toTimestamp@       |@date@          |Converts the @date@ argument into a @timestamp@ type|
+|@toUnixTimestamp@   |@timeuuid@      |Converts the @timeuuid@ argument into a @bigInt@ raw value|
+|@toUnixTimestamp@   |@timestamp@     |Converts the @timestamp@ argument into a @bigInt@ raw value|
+|@toUnixTimestamp@   |@date@          |Converts the @date@ argument into a @bigInt@ raw value|
+|@dateOf@            |@timeuuid@      |Similar to @toTimestamp(timeuuid)@ (DEPRECATED)|
+|@unixTimestampOf@   |@timeuuid@      |Similar to @toUnixTimestamp(timeuuid)@ (DEPRECATED)|
 
 h3(#blobFun). Blob conversion functions
 
 A number of functions are provided to "convert" the native types into binary data (@blob@). For every @<native-type>@ @type@ supported by CQL3 (a notable exceptions is @blob@, for obvious reasons), the function @typeAsBlob@ takes a argument of type @type@ and return it as a @blob@.  Conversely, the function @blobAsType@ takes a 64-bit @blob@ argument and convert it to a @bigint@ value.  And so for instance, @bigintAsBlob(3)@ is @0x0000000000000003@ and @blobAsBigint(0x0000000000000003)@ is @3@.
 
+h2(#aggregates). Aggregates
+
+Aggregate functions work on a set of rows. They receive values for each row and returns one value for the whole set.
+If @normal@ columns, @scalar functions@, @UDT@ fields, @writetime@ or @ttl@ are selected together with aggregate functions, the values returned for them will be the ones of the first row matching the query.
+
+CQL3 distinguishes between built-in aggregates (so called 'native aggregates') and "user-defined aggregates":#udas. CQL3 includes several native aggregates, described below:
+
+h3(#countFct). Count
+
+The @count@ function can be used to count the rows returned by a query. Example:
+
+bc(sample). 
+SELECT COUNT(*) FROM plays;
+SELECT COUNT(1) FROM plays;
+
+It also can be used to count the non null value of a given column. Example:
+
+bc(sample). 
+SELECT COUNT(scores) FROM plays;
+
+h3(#maxMinFcts). Max and Min
+
+The @max@ and @min@ functions can be used to compute the maximum and the minimum value returned by a query for a given column.
+
+bc(sample). 
+SELECT MIN(players), MAX(players) FROM plays WHERE game = 'quake';
+
+h3(#sumFct). Sum
+
+The @sum@ function can be used to sum up all the values returned by a query for a given column.
+
+bc(sample). 
+SELECT SUM(players) FROM plays;
+
+h3(#avgFct). Avg
+
+The @avg@ function can be used to compute the average of all the values returned by a query for a given column.
+
+bc(sample). 
+SELECT AVG(players) FROM plays;
+
+h2(#udfs). User-Defined Functions
+
+User-defined functions allow execution of user-provided code in Cassandra. By default, Cassandra supports defining functions in _Java_ and _JavaScript_. Support for other JSR 223 compliant scripting languages (such as Python, Ruby, and Scala) can be added by adding a JAR to the classpath.
+
+UDFs are part of the Cassandra schema.  As such, they are automatically propagated to all nodes in the cluster.
+
+UDFs can be _overloaded_ - i.e. multiple UDFs with different argument types but the same function name. Example:
+
+bc(sample). 
+CREATE FUNCTION sample ( arg int ) ...;
+CREATE FUNCTION sample ( arg text ) ...;
+
+User-defined functions are susceptible to all of the normal problems with the chosen programming language.  Accordingly, implementations should be safe against null pointer exceptions, illegal arguments, or any other potential source of exceptions.  An exception during function execution will result in the entire statement failing.
+
+It is valid to use _complex_ types like collections, tuple types and user-defined types as argument and return types. Tuple types and user-defined types are handled by the conversion functions of the DataStax Java Driver. Please see the documentation of the Java Driver for details on handling tuple types and user-defined types.
+
+Arguments for functions can be literals or terms. Prepared statement placeholders can be used, too.
+
+Note that you can use the double-quoted string syntax to enclose the UDF source code. For example:
+
+bc(sample).. 
+CREATE FUNCTION some_function ( arg int )
+  RETURNS NULL ON NULL INPUT
+  RETURNS int
+  LANGUAGE java
+  AS $$ return arg; $$;
+
+SELECT some_function(column) FROM atable ...;
+UPDATE atable SET col = some_function(?) ...;
+p. 
+
+bc(sample). 
+CREATE TYPE custom_type (txt text, i int);
+CREATE FUNCTION fct_using_udt ( udtarg frozen<custom_type> )
+  RETURNS NULL ON NULL INPUT
+  RETURNS text
+  LANGUAGE java
+  AS $$ return udtarg.getString("txt"); $$;
+
+User-defined functions can be used in "@SELECT@":#selectStmt, "@INSERT@":#insertStmt and "@UPDATE@":#updateStmt statements.
+
+See "@CREATE FUNCTION@":#createFunctionStmt and "@DROP FUNCTION@":#dropFunctionStmt.
+
+h2(#udas). User-Defined Aggregates
+
+User-defined aggregates allow creation of custom aggregate functions using "UDFs":#udfs. Common examples of aggregate functions are _count_, _min_, and _max_.
+
+Each aggregate requires an _initial state_ (@INITCOND@, which defaults to @null@) of type @STYPE@. The first argument of the state function must have type @STYPE@. The remaining arguments of the state function must match the types of the user-defined aggregate arguments. The state function is called once for each row, and the value returned by the state function becomes the new state. After all rows are processed, the optional @FINALFUNC@ is executed with last state value as its argument.
+
+@STYPE@ is mandatory in order to be able to distinguish possibly overloaded versions of the state and/or final function (since the overload can appear after creation of the aggregate).
+
+User-defined aggregates can be used in "@SELECT@":#selectStmt statement.
+
+A complete working example for user-defined aggregates (assuming that a keyspace has been selected using the "@USE@":#useStmt statement):
+
+bc(sample).. 
+CREATE OR REPLACE FUNCTION averageState ( state tuple<int,bigint>, val int )
+  CALLED ON NULL INPUT
+  RETURNS tuple<int,bigint>
+  LANGUAGE java
+  AS '
+    if (val != null) {
+      state.setInt(0, state.getInt(0)+1);
+      state.setLong(1, state.getLong(1)+val.intValue());
+    }
+    return state;
+  ';
+
+CREATE OR REPLACE FUNCTION averageFinal ( state tuple<int,bigint> )
+  CALLED ON NULL INPUT
+  RETURNS double
+  LANGUAGE java
+  AS '
+    double r = 0;
+    if (state.getInt(0) == 0) return null;
+    r = state.getLong(1);
+    r /= state.getInt(0);
+    return Double.valueOf(r);
+  ';
+
+CREATE OR REPLACE AGGREGATE average ( int )
+  SFUNC averageState
+  STYPE tuple<int,bigint>
+  FINALFUNC averageFinal
+  INITCOND (0, 0);
+
+CREATE TABLE atable (
+  pk int PRIMARY KEY,
+  val int);
+INSERT INTO atable (pk, val) VALUES (1,1);
+INSERT INTO atable (pk, val) VALUES (2,2);
+INSERT INTO atable (pk, val) VALUES (3,3);
+INSERT INTO atable (pk, val) VALUES (4,4);
+SELECT average(val) FROM atable;
+p. 
+
+See "@CREATE AGGREGATE@":#createAggregateStmt and "@DROP AGGREGATE@":#dropAggregateStmt.
+
+h2(#json). JSON Support
+
+Cassandra 2.2 introduces JSON support to "@SELECT@":#selectStmt and "@INSERT@":#insertStmt statements.  This support does not fundamentally alter the CQL API (for example, the schema is still enforced), it simply provides a convenient way to work with JSON documents.
+
+h3(#selectJson). SELECT JSON
+
+With @SELECT@ statements, the new @JSON@ keyword can be used to return each row as a single @JSON@ encoded map.  The remainder of the @SELECT@ statment behavior is the same.
+
+The result map keys are the same as the column names in a normal result set.  For example, a statement like "@SELECT JSON a, ttl(b) FROM ...@" would result in a map with keys @"a"@ and @"ttl(b)"@.  However, this is one notable exception: for symmetry with @INSERT JSON@ behavior, case-sensitive column names with upper-case letters will be surrounded with double quotes.  For example, "@SELECT JSON myColumn FROM ...@" would result in a map key @"\"myColumn\""@ (note the escaped quotes).
+
+The map values will @JSON@-encoded representations (as described below) of the result set values.
+
+h3(#insertJson). INSERT JSON
+
+With @INSERT@ statements, the new @JSON@ keyword can be used to enable inserting a @JSON@ encoded map as a single row.  The format of the @JSON@ map should generally match that returned by a @SELECT JSON@ statement on the same table.  In particular, case-sensitive column names should be surrounded with double quotes.  For example, to insert into a table with two columns named "myKey" and "value", you would do the following:
+
+bc(sample). 
+INSERT INTO mytable JSON '{"\"myKey\"": 0, "value": 0}'
+
+Any columns which are ommitted from the @JSON@ map will be defaulted to a @NULL@ value (which will result in a tombstone being created).
+
+h3(#jsonEncoding). JSON Encoding of Cassandra Data Types
+
+Where possible, Cassandra will represent and accept data types in their native @JSON@ representation.  Cassandra will also accept string representations matching the CQL literal format for all single-field types.  For example, floats, ints, UUIDs, and dates can be represented by CQL literal strings.  However, compound types, such as collections, tuples, and user-defined types must be represented by native @JSON@ collections (maps and lists) or a JSON-encoded string representation of the collection.
+
+The following table describes the encodings that Cassandra will accept in @INSERT JSON@ values (and @fromJson()@ arguments) as well as the format Cassandra will use when returning data for @SELECT JSON@ statements (and @fromJson()@):
+
+|_. type    |_. formats accepted   |_. return format |_. notes|
+|@ascii@    |string                |string           |Uses JSON's @\u@ character escape|
+|@bigint@   |integer, string       |integer          |String must be valid 64 bit integer|
+|@blob@     |string                |string           |String should be 0x followed by an even number of hex digits|
+|@boolean@  |boolean, string       |boolean          |String must be "true" or "false"|
+|@date@     |string                |string           |Date in format @YYYY-MM-DD@, timezone UTC|
+|@decimal@  |integer, float, string|float            |May exceed 32 or 64-bit IEEE-754 floating point precision in client-side decoder|
+|@double@   |integer, float, string|float            |String must be valid integer or float|
+|@float@    |integer, float, string|float            |String must be valid integer or float|
+|@inet@     |string                |string           |IPv4 or IPv6 address|
+|@int@      |integer, string       |integer          |String must be valid 32 bit integer|
+|@list@     |list, string          |list             |Uses JSON's native list representation|
+|@map@      |map, string           |map              |Uses JSON's native map representation|
+|@smallint@ |integer, string       |integer          |String must be valid 16 bit integer|
+|@set@      |list, string          |list             |Uses JSON's native list representation|
+|@text@     |string                |string           |Uses JSON's @\u@ character escape|
+|@time@     |string                |string           |Time of day in format @HH-MM-SS[.fffffffff]@|
+|@timestamp@|integer, string       |string           |A timestamp. Strings constant are allow to input timestamps as dates, see "Working with dates":#usingdates below for more information.  Datestamps with format @YYYY-MM-DD HH:MM:SS.SSS@ are returned.|
+|@timeuuid@ |string                |string           |Type 1 UUID. See "Constants":#constants for the UUID format|
+|@tinyint@  |integer, string       |integer          |String must be valid 8 bit integer|
+|@tuple@    |list, string          |list             |Uses JSON's native list representation|
+|@UDT@      |map, string           |map              |Uses JSON's native map representation with field names as keys|
+|@uuid@     |string                |string           |See "Constants":#constants for the UUID format|
+|@varchar@  |string                |string           |Uses JSON's @\u@ character escape|
+|@varint@   |integer, string       |integer          |Variable length; may overflow 32 or 64 bit integers in client-side decoder|
+
+h3(#fromJson). The fromJson() Function
+
+The @fromJson()@ function may be used similarly to @INSERT JSON@, but for a single column value.  It may only be used in the @VALUES@ clause of an @INSERT@ statement or as one of the column values in an @UPDATE@, @DELETE@, or @SELECT@ statement.  For example, it cannot be used in the selection clause of a @SELECT@ statement.
+
+h3(#toJson). The toJson() Function
+
+The @toJson()@ function may be used similarly to @SELECT JSON@, but for a single column value.  It may only be used in the selection clause of a @SELECT@ statement.
 
 h2(#appendixA). Appendix A: CQL Keywords
 
@@ -1436,10 +2073,11 @@
 
 |_. Keyword      |_. Reserved? |
 | @ADD@          | yes |
+| @AGGREGATE@    | no  |
 | @ALL@          | no  |
+| @ALLOW@        | yes |
 | @ALTER@        | yes |
 | @AND@          | yes |
-| @ANY@          | yes |
 | @APPLY@        | yes |
 | @AS@           | no  |
 | @ASC@          | yes |
@@ -1451,62 +2089,100 @@
 | @BLOB@         | no  |
 | @BOOLEAN@      | no  |
 | @BY@           | yes |
+| @CALLED@       | no  |
 | @CLUSTERING@   | no  |
 | @COLUMNFAMILY@ | yes |
 | @COMPACT@      | no  |
-| @CONSISTENCY@  | no  |
+| @CONTAINS@     | no  |
 | @COUNT@        | no  |
 | @COUNTER@      | no  |
 | @CREATE@       | yes |
+| @CUSTOM@       | no  |
+| @DATE@         | no  |
 | @DECIMAL@      | no  |
 | @DELETE@       | yes |
 | @DESC@         | yes |
+| @DESCRIBE@     | yes |
+| @DISTINCT@     | no  |
 | @DOUBLE@       | no  |
 | @DROP@         | yes |
-| @EACH_QUORUM@  | yes |
+| @ENTRIES@      | yes |
+| @EXECUTE@      | yes |
+| @EXISTS@       | no  |
+| @FILTERING@    | no  |
+| @FINALFUNC@    | no  |
 | @FLOAT@        | no  |
 | @FROM@         | yes |
+| @FROZEN@       | no  |
+| @FULL@         | yes |
+| @FUNCTION@     | no  |
+| @FUNCTIONS@    | no  |
 | @GRANT@        | yes |
+| @IF@           | yes |
 | @IN@           | yes |
 | @INDEX@        | yes |
-| @CUSTOM@       | no  |
+| @INET@         | no  |
+| @INFINITY@     | yes |
+| @INITCOND@     | no  |
+| @INPUT@        | no  |
 | @INSERT@       | yes |
 | @INT@          | no  |
 | @INTO@         | yes |
+| @JSON@         | no  |
 | @KEY@          | no  |
+| @KEYS@         | no  |
 | @KEYSPACE@     | yes |
-| @LEVEL@        | no  |
+| @KEYSPACES@    | no  |
+| @LANGUAGE@     | no  |
 | @LIMIT@        | yes |
-| @LOCAL_ONE@    | yes |
-| @LOCAL_QUORUM@ | yes |
+| @LIST@         | no  |
+| @LOGIN@        | no  |
+| @MAP@          | no  |
 | @MODIFY@       | yes |
+| @NAN@          | yes |
+| @NOLOGIN@      | no  |
 | @NORECURSIVE@  | yes |
 | @NOSUPERUSER@  | no  |
+| @NOT@          | yes |
+| @NULL@         | yes |
 | @OF@           | yes |
 | @ON@           | yes |
-| @ONE@          | yes |
+| @OPTIONS@      | no  |
+| @OR@           | yes |
 | @ORDER@        | yes |
 | @PASSWORD@     | no  |
 | @PERMISSION@   | no  |
 | @PERMISSIONS@  | no  |
 | @PRIMARY@      | yes |
-| @QUORUM@       | yes |
+| @RENAME@       | yes |
+| @REPLACE@      | yes |
+| @RETURNS@      | no  |
 | @REVOKE@       | yes |
+| @ROLE@         | no  |
+| @ROLES@        | no  |
 | @SCHEMA@       | yes |
 | @SELECT@       | yes |
 | @SET@          | yes |
+| @SFUNC@        | no  |
+| @SMALLINT@     | no  |
+| @STATIC@       | no  |
 | @STORAGE@      | no  |
+| @STYPE@        | no  |
 | @SUPERUSER@    | no  |
 | @TABLE@        | yes |
 | @TEXT@         | no  |
+| @TIME@         | no  |
 | @TIMESTAMP@    | no  |
 | @TIMEUUID@     | no  |
-| @THREE@        | yes |
+| @TINYINT@      | no  |
+| @TO@           | yes |
 | @TOKEN@        | yes |
+| @TRIGGER@      | no  |
 | @TRUNCATE@     | yes |
 | @TTL@          | no  |
-| @TWO@          | yes |
+| @TUPLE@        | no  |
 | @TYPE@         | no  |
+| @UNLOGGED@     | yes |
 | @UPDATE@       | yes |
 | @USE@          | yes |
 | @USER@         | no  |
@@ -1519,36 +2195,48 @@
 | @WHERE@        | yes |
 | @WITH@         | yes |
 | @WRITETIME@    | no  |
-| @DISTINCT@     | no  |
 
 h2(#appendixB). Appendix B: CQL Reserved Types
 
 The following type names are not currently used by CQL, but are reserved for potential future use.  User-defined types may not use reserved type names as their name.
 
 |_. type      |
+| @bitstring@ |
 | @byte@      |
-| @smallint@  |
 | @complex@   |
-| @enum@      |
 | @date@      |
+| @enum@      |
 | @interval@  |
 | @macaddr@   |
-| @bitstring@ |
+| @smallint@  |
 
 h2(#changes). Changes
 
 The following describes the changes in each version of CQL.
 
-h3. 3.2.1
+
+h3. 3.3.1
 
 * The syntax @TRUNCATE TABLE X@ is now accepted as an alias for @TRUNCATE X@
 
+h3. 3.3.0
+
+* Adds new "aggregates":#aggregates
+* User-defined functions are now supported through "@CREATE FUNCTION@":#createFunctionStmt and "@DROP FUNCTION@":#dropFunctionStmt.
+* User-defined aggregates are now supported through "@CREATE AGGREGATE@":#createAggregateStmt and "@DROP AGGREGATE@":#dropAggregateStmt.
+* Allows double-dollar enclosed strings literals as an alternative to single-quote enclosed strings.
+* Introduces Roles to supercede user based authentication and access control
+* "@Date@":#usingdates and "@Time@":usingtime data types have been added
+* "@JSON@":#json support has been added
+* @Tinyint@ and @Smallint@ data types have been added
+* Adds new time conversion functions and deprecate @dateOf@ and @unixTimestampOf@. See "@Time conversion functions@":#timeFun
+
 h3. 3.2.0
 
 * User-defined types are now supported through "@CREATE TYPE@":#createTypeStmt, "@ALTER TYPE@":#alterTypeStmt, and "@DROP TYPE@":#dropTypeStmt
 * "@CREATE INDEX@":#createIndexStmt now supports indexing collection columns, including indexing the keys of map collections through the @keys()@ function
 * Indexes on collections may be queried using the new @CONTAINS@ and @CONTAINS KEY@ operators
-* Tuple types were added to hold fixed-length sets of typed positional fields (see the section on "types":#types)
+* Tuple types were added to hold fixed-length sets of typed positional fields (see the section on "types":#types )
 * "@DROP INDEX@":#dropIndexStmt now supports optionally specifying a keyspace
 
 h3. 3.1.7
@@ -1568,15 +2256,15 @@
 
 h3. 3.1.4
 
-* @CREATE INDEX@ now allows specifying options when creating CUSTOM indexes (see "CREATE INDEX reference":#createIndexStmt).
+* @CREATE INDEX@ now allows specifying options when creating CUSTOM indexes (see "CREATE INDEX reference":#createIndexStmt ).
 
 h3. 3.1.3
 
-* Millisecond precision formats have been added to the timestamp parser (see "working with dates":#usingdates).
+* Millisecond precision formats have been added to the timestamp parser (see "working with dates":#usingtimestamps ).
 
 h3. 3.1.2
 
-* @NaN@ and @Infinity@ has been added as valid float contants. They are now reserved keywords. In the unlikely case you we using them as a column identifier (or keyspace/table one), you will noew need to double quote them (see "quote identifiers":#identifiers).
+* @NaN@ and @Infinity@ has been added as valid float contants. They are now reserved keywords. In the unlikely case you we using them as a column identifier (or keyspace/table one), you will noew need to double quote them (see "quote identifiers":#identifiers ).
 
 h3. 3.1.1
 
@@ -1587,7 +2275,7 @@
 h3. 3.1.0
 
 * "ALTER TABLE":#alterTableStmt @DROP@ option has been reenabled for CQL3 tables and has new semantics now: the space formerly used by dropped columns will now be eventually reclaimed (post-compaction). You should not readd previously dropped columns unless you use timestamps with microsecond precision (see "CASSANDRA-3919":https://issues.apache.org/jira/browse/CASSANDRA-3919 for more details).
-* @SELECT@ statement now supports aliases in select clause. Aliases in WHERE and ORDER BY clauses are not supported. See the "section on select"#selectStmt for details.
+* @SELECT@ statement now supports aliases in select clause. Aliases in WHERE and ORDER BY clauses are not supported. See the "section on select":#selectStmt for details.
 * @CREATE@ statements for @KEYSPACE@, @TABLE@ and @INDEX@ now supports an @IF NOT EXISTS@ condition. Similarly, @DROP@ statements support a @IF EXISTS@ condition.
 * @INSERT@ statements optionally supports a @IF NOT EXISTS@ condition and @UPDATE@ supports @IF@ conditions.
 
@@ -1598,7 +2286,7 @@
 h3. 3.0.4
 
 * Updated the syntax for custom "secondary indexes":#createIndexStmt.
-* Non-equal condition on the partition key are now never supported, even for ordering partitioner as this was not correct (the order was *not* the one of the type of the partition key). Instead, the @token@ method should always be used for range queries on the partition key (see "WHERE clauses":#selectWhere).
+* Non-equal condition on the partition key are now never supported, even for ordering partitioner as this was not correct (the order was *not* the one of the type of the partition key). Instead, the @token@ method should always be used for range queries on the partition key (see "WHERE clauses":#selectWhere ).
 
 h3. 3.0.3
 
@@ -1612,10 +2300,9 @@
 
 h3. 3.0.1
 
-* "Date strings":#usingdates (and timestamps) are no longer accepted as valid @timeuuid@ values. Doing so was a bug in the sense that date string are not valid @timeuuid@, and it was thus resulting in "confusing behaviors":https://issues.apache.org/jira/browse/CASSANDRA-4936.  However, the following new methods have been added to help working with @timeuuid@: @now@, @minTimeuuid@, @maxTimeuuid@ , @dateOf@ and @unixTimestampOf@. See the "section dedicated to these methods":#usingtimeuuid for more detail.
+* "Date strings":#usingtimestamps (and timestamps) are no longer accepted as valid @timeuuid@ values. Doing so was a bug in the sense that date string are not valid @timeuuid@, and it was thus resulting in "confusing behaviors":https://issues.apache.org/jira/browse/CASSANDRA-4936.  However, the following new methods have been added to help working with @timeuuid@: @now@, @minTimeuuid@, @maxTimeuuid@ , @dateOf@ and @unixTimestampOf@. See the "section dedicated to these methods":#timeuuidFun for more detail.
 * "Float constants"#constants now support the exponent notation. In other words, @4.2E10@ is now a valid floating point value.
 
-
 h2. Versioning
 
 Versioning of the CQL language adheres to the "Semantic Versioning":http://semver.org guidelines. Versions take the form X.Y.Z where X, Y, and Z are integer values representing major, minor, and patch level respectively. There is no correlation between Cassandra release versions and the CQL language version.
diff --git a/doc/native_protocol_v4.spec b/doc/native_protocol_v4.spec
new file mode 100644
index 0000000..187ff80
--- /dev/null
+++ b/doc/native_protocol_v4.spec
@@ -0,0 +1,1171 @@
+
+                             CQL BINARY PROTOCOL v4
+
+
+Table of Contents
+
+  1. Overview
+  2. Frame header
+    2.1. version
+    2.2. flags
+    2.3. stream
+    2.4. opcode
+    2.5. length
+  3. Notations
+  4. Messages
+    4.1. Requests
+      4.1.1. STARTUP
+      4.1.2. AUTH_RESPONSE
+      4.1.3. OPTIONS
+      4.1.4. QUERY
+      4.1.5. PREPARE
+      4.1.6. EXECUTE
+      4.1.7. BATCH
+      4.1.8. REGISTER
+    4.2. Responses
+      4.2.1. ERROR
+      4.2.2. READY
+      4.2.3. AUTHENTICATE
+      4.2.4. SUPPORTED
+      4.2.5. RESULT
+        4.2.5.1. Void
+        4.2.5.2. Rows
+        4.2.5.3. Set_keyspace
+        4.2.5.4. Prepared
+        4.2.5.5. Schema_change
+      4.2.6. EVENT
+      4.2.7. AUTH_CHALLENGE
+      4.2.8. AUTH_SUCCESS
+  5. Compression
+  6. Data Type Serialization Formats
+  7. User Defined Type Serialization
+  8. Result paging
+  9. Error codes
+  10. Changes from v3
+
+
+1. Overview
+
+  The CQL binary protocol is a frame based protocol. Frames are defined as:
+
+      0         8        16        24        32         40
+      +---------+---------+---------+---------+---------+
+      | version |  flags  |      stream       | opcode  |
+      +---------+---------+---------+---------+---------+
+      |                length                 |
+      +---------+---------+---------+---------+
+      |                                       |
+      .            ...  body ...              .
+      .                                       .
+      .                                       .
+      +----------------------------------------
+
+  The protocol is big-endian (network byte order).
+
+  Each frame contains a fixed size header (9 bytes) followed by a variable size
+  body. The header is described in Section 2. The content of the body depends
+  on the header opcode value (the body can in particular be empty for some
+  opcode values). The list of allowed opcodes is defined in Section 2.3 and the
+  details of each corresponding message are described Section 4.
+
+  The protocol distinguishes two types of frames: requests and responses. Requests
+  are those frames sent by the client to the server. Responses are those frames sent
+  by the server to the client. Note, however, that the protocol supports server pushes
+  (events) so a response does not necessarily come right after a client request.
+
+  Note to client implementors: client libraries should always assume that the
+  body of a given frame may contain more data than what is described in this
+  document. It will however always be safe to ignore the remainder of the frame
+  body in such cases. The reason is that this may enable extending the protocol
+  with optional features without needing to change the protocol version.
+
+
+
+2. Frame header
+
+2.1. version
+
+  The version is a single byte that indicates both the direction of the message
+  (request or response) and the version of the protocol in use. The most
+  significant bit of version is used to define the direction of the message:
+  0 indicates a request, 1 indicates a response. This can be useful for protocol
+  analyzers to distinguish the nature of the packet from the direction in which
+  it is moving. The rest of that byte is the protocol version (4 for the protocol
+  defined in this document). In other words, for this version of the protocol,
+  version will be one of:
+    0x04    Request frame for this protocol version
+    0x84    Response frame for this protocol version
+
+  Please note that while every message ships with the version, only one version
+  of messages is accepted on a given connection. In other words, the first message
+  exchanged (STARTUP) sets the version for the connection for the lifetime of this
+  connection.
+
+  This document describes version 4 of the protocol. For the changes made since
+  version 3, see Section 10.
+
+
+2.2. flags
+
+  Flags applying to this frame. The flags have the following meaning (described
+  by the mask that allows selecting them):
+    0x01: Compression flag. If set, the frame body is compressed. The actual
+          compression to use should have been set up beforehand through the
+          Startup message (which thus cannot be compressed; Section 4.1.1).
+    0x02: Tracing flag. For a request frame, this indicates the client requires
+          tracing of the request. Note that only QUERY, PREPARE and EXECUTE queries
+          support tracing. Other requests will simply ignore the tracing flag if 
+          set. If a request supports tracing and the tracing flag is set, the response
+          to this request will have the tracing flag set and contain tracing
+          information.
+          If a response frame has the tracing flag set, its body contains
+          a tracing ID. The tracing ID is a [uuid] and is the first thing in
+          the frame body. The rest of the body will then be the usual body
+          corresponding to the response opcode.
+    0x04: Custom payload flag. For a request or response frame, this indicates
+          that a generic key-value custom payload for a custom QueryHandler
+          implementation is present in the frame. Such a custom payload is simply
+          ignored by the default QueryHandler implementation.
+          Currently, only QUERY, PREPARE, EXECUTE and BATCH requests support
+          payload.
+          Type of custom payload is [bytes map] (see below).
+    0x08: Warning flag. The response contains warnings which were generated by the
+          server to go along with this response.
+          If a response frame has the warning flag set, its body will contain the
+          text of the warnings. The warnings are a [string list] and will be the
+          first value in the frame body if the tracing flag is not set, or directly
+          after the tracing ID if it is.
+
+  The rest of flags is currently unused and ignored.
+
+2.3. stream
+
+  A frame has a stream id (a [short] value). When sending request messages, this
+  stream id must be set by the client to a non-negative value (negative stream id
+  are reserved for streams initiated by the server; currently all EVENT messages
+  (section 4.2.6) have a streamId of -1). If a client sends a request message
+  with the stream id X, it is guaranteed that the stream id of the response to
+  that message will be X.
+
+  This helps to enable the asynchronous nature of the protocol. If a client
+  sends multiple messages simultaneously (without waiting for responses), there
+  is no guarantee on the order of the responses. For instance, if the client
+  writes REQ_1, REQ_2, REQ_3 on the wire (in that order), the server might
+  respond to REQ_3 (or REQ_2) first. Assigning different stream ids to these 3
+  requests allows the client to distinguish to which request a received answer
+  responds to. As there can only be 32768 different simultaneous streams, it is up
+  to the client to reuse stream id.
+
+  Note that clients are free to use the protocol synchronously (i.e. wait for
+  the response to REQ_N before sending REQ_N+1). In that case, the stream id
+  can be safely set to 0. Clients should also feel free to use only a subset of
+  the 32768 maximum possible stream ids if it is simpler for its implementation.
+
+2.4. opcode
+
+  An integer byte that distinguishes the actual message:
+    0x00    ERROR
+    0x01    STARTUP
+    0x02    READY
+    0x03    AUTHENTICATE
+    0x05    OPTIONS
+    0x06    SUPPORTED
+    0x07    QUERY
+    0x08    RESULT
+    0x09    PREPARE
+    0x0A    EXECUTE
+    0x0B    REGISTER
+    0x0C    EVENT
+    0x0D    BATCH
+    0x0E    AUTH_CHALLENGE
+    0x0F    AUTH_RESPONSE
+    0x10    AUTH_SUCCESS
+
+  Messages are described in Section 4.
+
+  (Note that there is no 0x04 message in this version of the protocol)
+
+
+2.5. length
+
+  A 4 byte integer representing the length of the body of the frame (note:
+  currently a frame is limited to 256MB in length).
+
+
+3. Notations
+
+  To describe the layout of the frame body for the messages in Section 4, we
+  define the following:
+
+    [int]          A 4 bytes integer
+    [long]         A 8 bytes integer
+    [short]        A 2 bytes unsigned integer
+    [string]       A [short] n, followed by n bytes representing an UTF-8
+                   string.
+    [long string]  An [int] n, followed by n bytes representing an UTF-8 string.
+    [uuid]         A 16 bytes long uuid.
+    [string list]  A [short] n, followed by n [string].
+    [bytes]        A [int] n, followed by n bytes if n >= 0. If n < 0,
+                   no byte should follow and the value represented is `null`.
+    [value]        A [int] n, followed by n bytes if n >= 0.
+                   If n == -1 no byte should follow and the value represented is `null`.
+                   If n == -2 no byte should follow and the value represented is
+                   `not set` not resulting in any change to the existing value.
+                   n < -2 is an invalid value and results in an error.
+    [short bytes]  A [short] n, followed by n bytes if n >= 0.
+
+    [option]       A pair of <id><value> where <id> is a [short] representing
+                   the option id and <value> depends on that option (and can be
+                   of size 0). The supported id (and the corresponding <value>)
+                   will be described when this is used.
+    [option list]  A [short] n, followed by n [option].
+    [inet]         An address (ip and port) to a node. It consists of one
+                   [byte] n, that represents the address size, followed by n
+                   [byte] representing the IP address (in practice n can only be
+                   either 4 (IPv4) or 16 (IPv6)), following by one [int]
+                   representing the port.
+    [consistency]  A consistency level specification. This is a [short]
+                   representing a consistency level with the following
+                   correspondance:
+                     0x0000    ANY
+                     0x0001    ONE
+                     0x0002    TWO
+                     0x0003    THREE
+                     0x0004    QUORUM
+                     0x0005    ALL
+                     0x0006    LOCAL_QUORUM
+                     0x0007    EACH_QUORUM
+                     0x0008    SERIAL
+                     0x0009    LOCAL_SERIAL
+                     0x000A    LOCAL_ONE
+
+    [string map]      A [short] n, followed by n pair <k><v> where <k> and <v>
+                      are [string].
+    [string multimap] A [short] n, followed by n pair <k><v> where <k> is a
+                      [string] and <v> is a [string list].
+    [bytes map]       A [short] n, followed by n pair <k><v> where <k> is a
+                      [string] and <v> is a [bytes].
+
+
+4. Messages
+
+4.1. Requests
+
+  Note that outside of their normal responses (described below), all requests
+  can get an ERROR message (Section 4.2.1) as response.
+
+4.1.1. STARTUP
+
+  Initialize the connection. The server will respond by either a READY message
+  (in which case the connection is ready for queries) or an AUTHENTICATE message
+  (in which case credentials will need to be provided using AUTH_RESPONSE).
+
+  This must be the first message of the connection, except for OPTIONS that can
+  be sent before to find out the options supported by the server. Once the
+  connection has been initialized, a client should not send any more STARTUP
+  messages.
+
+  The body is a [string map] of options. Possible options are:
+    - "CQL_VERSION": the version of CQL to use. This option is mandatory and
+      currently the only version supported is "3.0.0". Note that this is
+      different from the protocol version.
+    - "COMPRESSION": the compression algorithm to use for frames (See section 5).
+      This is optional; if not specified no compression will be used.
+
+
+4.1.2. AUTH_RESPONSE
+
+  Answers a server authentication challenge.
+
+  Authentication in the protocol is SASL based. The server sends authentication
+  challenges (a bytes token) to which the client answers with this message. Those
+  exchanges continue until the server accepts the authentication by sending a
+  AUTH_SUCCESS message after a client AUTH_RESPONSE. Note that the exchange
+  begins with the client sending an initial AUTH_RESPONSE in response to a
+  server AUTHENTICATE request.
+
+  The body of this message is a single [bytes] token. The details of what this
+  token contains (and when it can be null/empty, if ever) depends on the actual
+  authenticator used.
+
+  The response to a AUTH_RESPONSE is either a follow-up AUTH_CHALLENGE message,
+  an AUTH_SUCCESS message or an ERROR message.
+
+
+4.1.3. OPTIONS
+
+  Asks the server to return which STARTUP options are supported. The body of an
+  OPTIONS message should be empty and the server will respond with a SUPPORTED
+  message.
+
+
+4.1.4. QUERY
+
+  Performs a CQL query. The body of the message must be:
+    <query><query_parameters>
+  where <query> is a [long string] representing the query and
+  <query_parameters> must be
+    <consistency><flags>[<n>[name_1]<value_1>...[name_n]<value_n>][<result_page_size>][<paging_state>][<serial_consistency>][<timestamp>]
+  where:
+    - <consistency> is the [consistency] level for the operation.
+    - <flags> is a [byte] whose bits define the options for this query and
+      in particular influence what the remainder of the message contains.
+      A flag is set if the bit corresponding to its `mask` is set. Supported
+      flags are, given their mask:
+        0x01: Values. If set, a [short] <n> followed by <n> [value]
+              values are provided. Those values are used for bound variables in
+              the query. Optionally, if the 0x40 flag is present, each value
+              will be preceded by a [string] name, representing the name of
+              the marker the value must be bound to.
+        0x02: Skip_metadata. If set, the Result Set returned as a response
+              to the query (if any) will have the NO_METADATA flag (see
+              Section 4.2.5.2).
+        0x04: Page_size. If set, <result_page_size> is an [int]
+              controlling the desired page size of the result (in CQL3 rows).
+              See the section on paging (Section 8) for more details.
+        0x08: With_paging_state. If set, <paging_state> should be present.
+              <paging_state> is a [bytes] value that should have been returned
+              in a result set (Section 4.2.5.2). The query will be
+              executed but starting from a given paging state. This is also to
+              continue paging on a different node than the one where it
+              started (See Section 8 for more details).
+        0x10: With serial consistency. If set, <serial_consistency> should be
+              present. <serial_consistency> is the [consistency] level for the
+              serial phase of conditional updates. That consitency can only be
+              either SERIAL or LOCAL_SERIAL and if not present, it defaults to
+              SERIAL. This option will be ignored for anything else other than a
+              conditional update/insert.
+        0x20: With default timestamp. If set, <timestamp> should be present.
+              <timestamp> is a [long] representing the default timestamp for the query
+              in microseconds (negative values are forbidden). This will
+              replace the server side assigned timestamp as default timestamp.
+              Note that a timestamp in the query itself will still override
+              this timestamp. This is entirely optional.
+        0x40: With names for values. This only makes sense if the 0x01 flag is set and
+              is ignored otherwise. If present, the values from the 0x01 flag will
+              be preceded by a name (see above). Note that this is only useful for
+              QUERY requests where named bind markers are used; for EXECUTE statements,
+              since the names for the expected values was returned during preparation,
+              a client can always provide values in the right order without any names
+              and using this flag, while supported, is almost surely inefficient.
+
+  Note that the consistency is ignored by some queries (USE, CREATE, ALTER,
+  TRUNCATE, ...).
+
+  The server will respond to a QUERY message with a RESULT message, the content
+  of which depends on the query.
+
+
+4.1.5. PREPARE
+
+  Prepare a query for later execution (through EXECUTE). The body consists of
+  the CQL query to prepare as a [long string].
+
+  The server will respond with a RESULT message with a `prepared` kind (0x0004,
+  see Section 4.2.5).
+
+
+4.1.6. EXECUTE
+
+  Executes a prepared query. The body of the message must be:
+    <id><query_parameters>
+  where <id> is the prepared query ID. It's the [short bytes] returned as a
+  response to a PREPARE message. As for <query_parameters>, it has the exact
+  same definition as in QUERY (see Section 4.1.4).
+
+  The response from the server will be a RESULT message.
+
+
+4.1.7. BATCH
+
+  Allows executing a list of queries (prepared or not) as a batch (note that
+  only DML statements are accepted in a batch). The body of the message must
+  be:
+    <type><n><query_1>...<query_n><consistency><flags>[<serial_consistency>][<timestamp>]
+  where:
+    - <type> is a [byte] indicating the type of batch to use:
+        - If <type> == 0, the batch will be "logged". This is equivalent to a
+          normal CQL3 batch statement.
+        - If <type> == 1, the batch will be "unlogged".
+        - If <type> == 2, the batch will be a "counter" batch (and non-counter
+          statements will be rejected).
+    - <flags> is a [byte] whose bits define the options for this query and
+      in particular influence what the remainder of the message contains. It is similar
+      to the <flags> from QUERY and EXECUTE methods, except that the 4 rightmost
+      bits must always be 0 as their corresponding options do not make sense for
+      Batch. A flag is set if the bit corresponding to its `mask` is set. Supported
+      flags are, given their mask:
+        0x10: With serial consistency. If set, <serial_consistency> should be
+              present. <serial_consistency> is the [consistency] level for the
+              serial phase of conditional updates. That consistency can only be
+              either SERIAL or LOCAL_SERIAL and if not present, it defaults to
+              SERIAL. This option will be ignored for anything else other than a
+              conditional update/insert.
+        0x20: With default timestamp. If set, <timestamp> should be present.
+              <timestamp> is a [long] representing the default timestamp for the query
+              in microseconds. This will replace the server side assigned
+              timestamp as default timestamp. Note that a timestamp in the query itself
+              will still override this timestamp. This is entirely optional.
+        0x40: With names for values. If set, then all values for all <query_i> must be
+              preceded by a [string] <name_i> that have the same meaning as in QUERY
+              requests [IMPORTANT NOTE: this feature does not work and should not be
+              used. It is specified in a way that makes it impossible for the server
+              to implement. This will be fixed in a future version of the native
+              protocol. See https://issues.apache.org/jira/browse/CASSANDRA-10246 for
+              more details].
+    - <n> is a [short] indicating the number of following queries.
+    - <query_1>...<query_n> are the queries to execute. A <query_i> must be of the
+      form:
+        <kind><string_or_id><n>[<name_1>]<value_1>...[<name_n>]<value_n>
+      where:
+       - <kind> is a [byte] indicating whether the following query is a prepared
+         one or not. <kind> value must be either 0 or 1.
+       - <string_or_id> depends on the value of <kind>. If <kind> == 0, it should be
+         a [long string] query string (as in QUERY, the query string might contain
+         bind markers). Otherwise (that is, if <kind> == 1), it should be a
+         [short bytes] representing a prepared query ID.
+       - <n> is a [short] indicating the number (possibly 0) of following values.
+       - <name_i> is the optional name of the following <value_i>. It must be present
+         if and only if the 0x40 flag is provided for the batch.
+       - <value_i> is the [value] to use for bound variable i (of bound variable <name_i>
+         if the 0x40 flag is used).
+    - <consistency> is the [consistency] level for the operation.
+    - <serial_consistency> is only present if the 0x10 flag is set. In that case,
+      <serial_consistency> is the [consistency] level for the serial phase of
+      conditional updates. That consitency can only be either SERIAL or
+      LOCAL_SERIAL and if not present will defaults to SERIAL. This option will
+      be ignored for anything else other than a conditional update/insert.
+
+  The server will respond with a RESULT message.
+
+
+4.1.8. REGISTER
+
+  Register this connection to receive some types of events. The body of the
+  message is a [string list] representing the event types to register for. See
+  section 4.2.6 for the list of valid event types.
+
+  The response to a REGISTER message will be a READY message.
+
+  Please note that if a client driver maintains multiple connections to a
+  Cassandra node and/or connections to multiple nodes, it is advised to
+  dedicate a handful of connections to receive events, but to *not* register
+  for events on all connections, as this would only result in receiving
+  multiple times the same event messages, wasting bandwidth.
+
+
+4.2. Responses
+
+  This section describes the content of the frame body for the different
+  responses. Please note that to make room for future evolution, clients should
+  support extra informations (that they should simply discard) to the one
+  described in this document at the end of the frame body.
+
+4.2.1. ERROR
+
+  Indicates an error processing a request. The body of the message will be an
+  error code ([int]) followed by a [string] error message. Then, depending on
+  the exception, more content may follow. The error codes are defined in
+  Section 9, along with their additional content if any.
+
+
+4.2.2. READY
+
+  Indicates that the server is ready to process queries. This message will be
+  sent by the server either after a STARTUP message if no authentication is
+  required (if authentication is required, the server indicates readiness by
+  sending a AUTH_RESPONSE message).
+
+  The body of a READY message is empty.
+
+
+4.2.3. AUTHENTICATE
+
+  Indicates that the server requires authentication, and which authentication
+  mechanism to use.
+
+  The authentication is SASL based and thus consists of a number of server
+  challenges (AUTH_CHALLENGE, Section 4.2.7) followed by client responses
+  (AUTH_RESPONSE, Section 4.1.2). The initial exchange is however boostrapped
+  by an initial client response. The details of that exchange (including how
+  many challenge-response pairs are required) are specific to the authenticator
+  in use. The exchange ends when the server sends an AUTH_SUCCESS message or
+  an ERROR message.
+
+  This message will be sent following a STARTUP message if authentication is
+  required and must be answered by a AUTH_RESPONSE message from the client.
+
+  The body consists of a single [string] indicating the full class name of the
+  IAuthenticator in use.
+
+
+4.2.4. SUPPORTED
+
+  Indicates which startup options are supported by the server. This message
+  comes as a response to an OPTIONS message.
+
+  The body of a SUPPORTED message is a [string multimap]. This multimap gives
+  for each of the supported STARTUP options, the list of supported values.
+
+
+4.2.5. RESULT
+
+  The result to a query (QUERY, PREPARE, EXECUTE or BATCH messages).
+
+  The first element of the body of a RESULT message is an [int] representing the
+  `kind` of result. The rest of the body depends on the kind. The kind can be
+  one of:
+    0x0001    Void: for results carrying no information.
+    0x0002    Rows: for results to select queries, returning a set of rows.
+    0x0003    Set_keyspace: the result to a `use` query.
+    0x0004    Prepared: result to a PREPARE message.
+    0x0005    Schema_change: the result to a schema altering query.
+
+  The body for each kind (after the [int] kind) is defined below.
+
+
+4.2.5.1. Void
+
+  The rest of the body for a Void result is empty. It indicates that a query was
+  successful without providing more information.
+
+
+4.2.5.2. Rows
+
+  Indicates a set of rows. The rest of the body of a Rows result is:
+    <metadata><rows_count><rows_content>
+  where:
+    - <metadata> is composed of:
+        <flags><columns_count>[<paging_state>][<global_table_spec>?<col_spec_1>...<col_spec_n>]
+      where:
+        - <flags> is an [int]. The bits of <flags> provides information on the
+          formatting of the remaining information. A flag is set if the bit
+          corresponding to its `mask` is set. Supported flags are, given their
+          mask:
+            0x0001    Global_tables_spec: if set, only one table spec (keyspace
+                      and table name) is provided as <global_table_spec>. If not
+                      set, <global_table_spec> is not present.
+            0x0002    Has_more_pages: indicates whether this is not the last
+                      page of results and more should be retrieved. If set, the
+                      <paging_state> will be present. The <paging_state> is a
+                      [bytes] value that should be used in QUERY/EXECUTE to
+                      continue paging and retrieve the remainder of the result for
+                      this query (See Section 8 for more details).
+            0x0004    No_metadata: if set, the <metadata> is only composed of
+                      these <flags>, the <column_count> and optionally the
+                      <paging_state> (depending on the Has_more_pages flag) but
+                      no other information (so no <global_table_spec> nor <col_spec_i>).
+                      This will only ever be the case if this was requested
+                      during the query (see QUERY and RESULT messages).
+        - <columns_count> is an [int] representing the number of columns selected
+          by the query that produced this result. It defines the number of <col_spec_i>
+          elements in and the number of elements for each row in <rows_content>.
+        - <global_table_spec> is present if the Global_tables_spec is set in
+          <flags>. It is composed of two [string] representing the
+          (unique) keyspace name and table name the columns belong to.
+        - <col_spec_i> specifies the columns returned in the query. There are
+          <column_count> such column specifications that are composed of:
+            (<ksname><tablename>)?<name><type>
+          The initial <ksname> and <tablename> are two [string] and are only present
+          if the Global_tables_spec flag is not set. The <column_name> is a
+          [string] and <type> is an [option] that corresponds to the description
+          (what this description is depends a bit on the context: in results to
+          selects, this will be either the user chosen alias or the selection used
+          (often a colum name, but it can be a function call too). In results to
+          a PREPARE, this will be either the name of the corresponding bind variable
+          or the column name for the variable if it is "anonymous") and type of
+          the corresponding result. The option for <type> is either a native
+          type (see below), in which case the option has no value, or a
+          'custom' type, in which case the value is a [string] representing
+          the fully qualified class name of the type represented. Valid option
+          ids are:
+            0x0000    Custom: the value is a [string], see above.
+            0x0001    Ascii
+            0x0002    Bigint
+            0x0003    Blob
+            0x0004    Boolean
+            0x0005    Counter
+            0x0006    Decimal
+            0x0007    Double
+            0x0008    Float
+            0x0009    Int
+            0x000B    Timestamp
+            0x000C    Uuid
+            0x000D    Varchar
+            0x000E    Varint
+            0x000F    Timeuuid
+            0x0010    Inet
+            0x0011    Date
+            0x0012    Time
+            0x0013    Smallint
+            0x0014    Tinyint
+            0x0020    List: the value is an [option], representing the type
+                            of the elements of the list.
+            0x0021    Map: the value is two [option], representing the types of the
+                           keys and values of the map
+            0x0022    Set: the value is an [option], representing the type
+                            of the elements of the set
+            0x0030    UDT: the value is <ks><udt_name><n><name_1><type_1>...<name_n><type_n>
+                           where:
+                              - <ks> is a [string] representing the keyspace name this
+                                UDT is part of.
+                              - <udt_name> is a [string] representing the UDT name.
+                              - <n> is a [short] representing the number of fields of
+                                the UDT, and thus the number of <name_i><type_i> pairs
+                                following
+                              - <name_i> is a [string] representing the name of the
+                                i_th field of the UDT.
+                              - <type_i> is an [option] representing the type of the
+                                i_th field of the UDT.
+            0x0031    Tuple: the value is <n><type_1>...<type_n> where <n> is a [short]
+                             representing the number of values in the type, and <type_i>
+                             are [option] representing the type of the i_th component
+                             of the tuple
+
+    - <rows_count> is an [int] representing the number of rows present in this
+      result. Those rows are serialized in the <rows_content> part.
+    - <rows_content> is composed of <row_1>...<row_m> where m is <rows_count>.
+      Each <row_i> is composed of <value_1>...<value_n> where n is
+      <columns_count> and where <value_j> is a [bytes] representing the value
+      returned for the jth column of the ith row. In other words, <rows_content>
+      is composed of (<rows_count> * <columns_count>) [bytes].
+
+
+4.2.5.3. Set_keyspace
+
+  The result to a `use` query. The body (after the kind [int]) is a single
+  [string] indicating the name of the keyspace that has been set.
+
+
+4.2.5.4. Prepared
+
+  The result to a PREPARE message. The body of a Prepared result is:
+    <id><metadata><result_metadata>
+  where:
+    - <id> is [short bytes] representing the prepared query ID.
+    - <metadata> is composed of:
+        <flags><columns_count><pk_count>[<pk_index_1>...<pk_index_n>][<global_table_spec>?<col_spec_1>...<col_spec_n>]
+      where:
+        - <flags> is an [int]. The bits of <flags> provides information on the
+          formatting of the remaining information. A flag is set if the bit
+          corresponding to its `mask` is set. Supported masks and their flags
+          are:
+            0x0001    Global_tables_spec: if set, only one table spec (keyspace
+                      and table name) is provided as <global_table_spec>. If not
+                      set, <global_table_spec> is not present.
+        - <columns_count> is an [int] representing the number of bind markers
+          in the prepared statement.  It defines the number of <col_spec_i>
+          elements.
+        - <pk_count> is an [int] representing the number of <pk_index_i>
+          elements to follow. If this value is zero, at least one of the
+          partition key columns in the table that the statement acts on
+          did not have a corresponding bind marker (or the bind marker
+          was wrapped in a function call).
+        - <pk_index_i> is a short that represents the index of the bind marker
+          that corresponds to the partition key column in position i.
+          For example, a <pk_index> sequence of [2, 0, 1] indicates that the
+          table has three partition key columns; the full partition key
+          can be constructed by creating a composite of the values for
+          the bind markers at index 2, at index 0, and at index 1.
+          This allows implementations with token-aware routing to correctly
+          construct the partition key without needing to inspect table
+          metadata.
+        - <global_table_spec> is present if the Global_tables_spec is set in
+          <flags>. If present, it is composed of two [string]s. The first
+          [string] is the name of the keyspace that the statement acts on.
+          The second [string] is the name of the table that the columns
+          represented by the bind markers belong to.
+        - <col_spec_i> specifies the bind markers in the prepared statement.
+          There are <column_count> such column specifications, each with the
+          following format:
+            (<ksname><tablename>)?<name><type>
+          The initial <ksname> and <tablename> are two [string] that are only
+          present if the Global_tables_spec flag is not set. The <name> field
+          is a [string] that holds the name of the bind marker (if named),
+          or the name of the column, field, or expression that the bind marker
+          corresponds to (if the bind marker is "anonymous").  The <type>
+          field is an [option] that represents the expected type of values for
+          the bind marker.  See the Rows documentation (section 4.2.5.2) for
+          full details on the <type> field.
+
+    - <result_metadata> is defined exactly the same as <metadata> in the Rows
+      documentation (section 4.2.5.2).  This describes the metadata for the
+      result set that will be returned when this prepared statement is executed.
+      Note that <result_metadata> may be empty (have the No_metadata flag and
+      0 columns, See section 4.2.5.2) and will be for any query that is not a
+      Select. In fact, there is never a guarantee that this will be non-empty, so
+      implementations should protect themselves accordingly. This result metadata
+      is an optimization that allows implementations to later execute the
+      prepared statement without requesting the metadata (see the Skip_metadata
+      flag in EXECUTE).  Clients can safely discard this metadata if they do not
+      want to take advantage of that optimization.
+
+  Note that the prepared query ID returned is global to the node on which the query
+  has been prepared. It can be used on any connection to that node
+  until the node is restarted (after which the query must be reprepared).
+
+4.2.5.5. Schema_change
+
+  The result to a schema altering query (creation/update/drop of a
+  keyspace/table/index). The body (after the kind [int]) is the same
+  as the body for a "SCHEMA_CHANGE" event, so 3 strings:
+    <change_type><target><options>
+  Please refer to section 4.2.6 below for the meaning of those fields.
+
+  Note that a query to create or drop an index is considered to be a change
+  to the table the index is on.
+
+
+4.2.6. EVENT
+
+  An event pushed by the server. A client will only receive events for the
+  types it has REGISTERed to. The body of an EVENT message will start with a
+  [string] representing the event type. The rest of the message depends on the
+  event type. The valid event types are:
+    - "TOPOLOGY_CHANGE": events related to change in the cluster topology.
+      Currently, events are sent when new nodes are added to the cluster, and
+      when nodes are removed. The body of the message (after the event type)
+      consists of a [string] and an [inet], corresponding respectively to the
+      type of change ("NEW_NODE" or "REMOVED_NODE") followed by the address of
+      the new/removed node.
+    - "STATUS_CHANGE": events related to change of node status. Currently,
+      up/down events are sent. The body of the message (after the event type)
+      consists of a [string] and an [inet], corresponding respectively to the
+      type of status change ("UP" or "DOWN") followed by the address of the
+      concerned node.
+    - "SCHEMA_CHANGE": events related to schema change. After the event type,
+      the rest of the message will be <change_type><target><options> where:
+        - <change_type> is a [string] representing the type of changed involved.
+          It will be one of "CREATED", "UPDATED" or "DROPPED".
+        - <target> is a [string] that can be one of "KEYSPACE", "TABLE", "TYPE",
+          "FUNCTION" or "AGGREGATE" and describes what has been modified
+          ("TYPE" stands for modifications related to user types, "FUNCTION"
+          for modifications related to user defined functions, "AGGREGATE"
+          for modifications related to user defined aggregates).
+        - <options> depends on the preceding <target>:
+          - If <target> is "KEYSPACE", then <options> will be a single [string]
+            representing the keyspace changed.
+          - If <target> is "TABLE" or "TYPE", then
+            <options> will be 2 [string]: the first one will be the keyspace
+            containing the affected object, and the second one will be the name
+            of said affected object (either the table, user type, function, or
+            aggregate name).
+          - If <target> is "FUNCTION" or "AGGREGATE", multiple arguments follow:
+            - [string] keyspace containing the user defined function / aggregate
+            - [string] the function/aggregate name
+            - [string list] one string for each argument type (as CQL type)
+
+  All EVENT messages have a streamId of -1 (Section 2.3).
+
+  Please note that "NEW_NODE" and "UP" events are sent based on internal Gossip
+  communication and as such may be sent a short delay before the binary
+  protocol server on the newly up node is fully started. Clients are thus
+  advised to wait a short time before trying to connect to the node (1 second
+  should be enough), otherwise they may experience a connection refusal at
+  first.
+
+4.2.7. AUTH_CHALLENGE
+
+  A server authentication challenge (see AUTH_RESPONSE (Section 4.1.2) for more
+  details).
+
+  The body of this message is a single [bytes] token. The details of what this
+  token contains (and when it can be null/empty, if ever) depends on the actual
+  authenticator used.
+
+  Clients are expected to answer the server challenge with an AUTH_RESPONSE
+  message.
+
+4.2.8. AUTH_SUCCESS
+
+  Indicates the success of the authentication phase. See Section 4.2.3 for more
+  details.
+
+  The body of this message is a single [bytes] token holding final information
+  from the server that the client may require to finish the authentication
+  process. What that token contains and whether it can be null depends on the
+  actual authenticator used.
+
+
+5. Compression
+
+  Frame compression is supported by the protocol, but then only the frame body
+  is compressed (the frame header should never be compressed).
+
+  Before being used, client and server must agree on a compression algorithm to
+  use, which is done in the STARTUP message. As a consequence, a STARTUP message
+  must never be compressed.  However, once the STARTUP frame has been received
+  by the server, messages can be compressed (including the response to the STARTUP
+  request). Frames do not have to be compressed, however, even if compression has
+  been agreed upon (a server may only compress frames above a certain size at its
+  discretion). A frame body should be compressed if and only if the compressed
+  flag (see Section 2.2) is set.
+
+  As of version 2 of the protocol, the following compressions are available:
+    - lz4 (https://code.google.com/p/lz4/). In that, note that the first four bytes
+      of the body will be the uncompressed length (followed by the compressed
+      bytes).
+    - snappy (https://code.google.com/p/snappy/). This compression might not be
+      available as it depends on a native lib (server-side) that might not be
+      avaivable on some installations.
+
+
+6. Data Type Serialization Formats
+
+  This sections describes the serialization formats for all CQL data types
+  supported by Cassandra through the native protocol.  These serialization
+  formats should be used by client drivers to encode values for EXECUTE
+  messages.  Cassandra will use these formats when returning values in
+  RESULT messages.
+
+  All values are represented as [bytes] in EXECUTE and RESULT messages.
+  The [bytes] format includes an int prefix denoting the length of the value.
+  For that reason, the serialization formats described here will not include
+  a length component.
+
+  For legacy compatibility reasons, note that most non-string types support
+  "empty" values (i.e. a value with zero length).  An empty value is distinct
+  from NULL, which is encoded with a negative length.
+
+  As with the rest of the native protocol, all encodings are big-endian.
+
+6.1. ascii
+
+  A sequence of bytes in the ASCII range [0, 127].  Bytes with values outside of
+  this range will result in a validation error.
+
+6.2 bigint
+
+  An eight-byte two's complement integer.
+
+6.3 blob
+
+  Any sequence of bytes.
+
+6.4 boolean
+
+  A single byte.  A value of 0 denotes "false"; any other value denotes "true".
+  (However, it is recommended that a value of 1 be used to represent "true".)
+
+6.5 date
+
+  An unsigned integer representing days with epoch centered at 2^31.
+  (unix epoch January 1st, 1970).
+  A few examples:
+    0:    -5877641-06-23
+    2^31: 1970-1-1
+    2^32: 5881580-07-11
+
+6.6 decimal
+
+  The decimal format represents an arbitrary-precision number.  It contains an
+  [int] "scale" component followed by a varint encoding (see section 6.17)
+  of the unscaled value.  The encoded value represents "<unscaled>E<-scale>".
+  In other words, "<unscaled> * 10 ^ (-1 * <scale>)".
+
+6.7 double
+
+  An 8 byte floating point number in the IEEE 754 binary64 format.
+
+6.8 float
+
+  A 4 byte floating point number in the IEEE 754 binary32 format.
+
+6.9 inet
+
+  A 4 byte or 16 byte sequence denoting an IPv4 or IPv6 address, respectively.
+
+6.10 int
+
+  A 4 byte two's complement integer.
+
+6.11 list
+
+  A [int] n indicating the number of elements in the list, followed by n
+  elements.  Each element is [bytes] representing the serialized value.
+
+6.12 map
+
+  A [int] n indicating the number of key/value pairs in the map, followed by
+  n entries.  Each entry is composed of two [bytes] representing the key
+  and value.
+
+6.13 set
+
+  A [int] n indicating the number of elements in the set, followed by n
+  elements.  Each element is [bytes] representing the serialized value.
+
+6.14 smallint
+
+  A 2 byte two's complement integer.
+
+6.15 text
+
+  A sequence of bytes conforming to the UTF-8 specifications.
+
+6.16 time
+
+  An 8 byte two's complement long representing nanoseconds since midnight.
+  Valid values are in the range 0 to 86399999999999
+
+6.17 timestamp
+
+  An 8 byte two's complement integer representing a millisecond-precision
+  offset from the unix epoch (00:00:00, January 1st, 1970).  Negative values
+  represent a negative offset from the epoch.
+
+6.18 timeuuid
+
+  A 16 byte sequence representing a version 1 UUID as defined by RFC 4122.
+
+6.19 tinyint
+
+  A 1 byte two's complement integer.
+
+6.20 tuple
+
+  A sequence of [bytes] values representing the items in a tuple.  The encoding
+  of each element depends on the data type for that position in the tuple.
+  Null values may be represented by using length -1 for the [bytes]
+  representation of an element.
+
+6.21 uuid
+
+  A 16 byte sequence representing any valid UUID as defined by RFC 4122.
+
+6.22 varchar
+
+  An alias of the "text" type.
+
+6.23 varint
+
+  A variable-length two's complement encoding of a signed integer.
+
+  The following examples may help implementors of this spec:
+
+  Value | Encoding
+  ------|---------
+      0 |     0x00
+      1 |     0x01
+    127 |     0x7F
+    128 |   0x0080
+    129 |   0x0081
+     -1 |     0xFF
+   -128 |     0x80
+   -129 |   0xFF7F
+
+  Note that positive numbers must use a most-significant byte with a value
+  less than 0x80, because a most-significant bit of 1 indicates a negative
+  value.  Implementors should pad positive values that have a MSB >= 0x80
+  with a leading 0x00 byte.
+
+
+7. User Defined Types
+
+  This section describes the serialization format for User defined types (UDT),
+  as described in section 4.2.5.2.
+
+  A UDT value is composed of successive [bytes] values, one for each field of the UDT
+  value (in the order defined by the type). A UDT value will generally have one value
+  for each field of the type it represents, but it is allowed to have less values than
+  the type has fields.
+
+
+8. Result paging
+
+  The protocol allows for paging the result of queries. For that, the QUERY and
+  EXECUTE messages have a <result_page_size> value that indicate the desired
+  page size in CQL3 rows.
+
+  If a positive value is provided for <result_page_size>, the result set of the
+  RESULT message returned for the query will contain at most the
+  <result_page_size> first rows of the query result. If that first page of results
+  contains the full result set for the query, the RESULT message (of kind `Rows`)
+  will have the Has_more_pages flag *not* set. However, if some results are not
+  part of the first response, the Has_more_pages flag will be set and the result
+  will contain a <paging_state> value. In that case, the <paging_state> value
+  should be used in a QUERY or EXECUTE message (that has the *same* query as
+  the original one or the behavior is undefined) to retrieve the next page of
+  results.
+
+  Only CQL3 queries that return a result set (RESULT message with a Rows `kind`)
+  support paging. For other type of queries, the <result_page_size> value is
+  ignored.
+
+  Note to client implementors:
+  - While <result_page_size> can be as low as 1, it will likely be detrimental
+    to performance to pick a value too low. A value below 100 is probably too
+    low for most use cases.
+  - Clients should not rely on the actual size of the result set returned to
+    decide if there are more results to fetch or not. Instead, they should always
+    check the Has_more_pages flag (unless they did not enable paging for the query
+    obviously). Clients should also not assert that no result will have more than
+    <result_page_size> results. While the current implementation always respects
+    the exact value of <result_page_size>, we reserve the right to return
+    slightly smaller or bigger pages in the future for performance reasons.
+  - The <paging_state> is specific to a protocol version and drivers should not
+    send a <paging_state> returned by a node using protocol v3 to query a node
+    using protocol v4 for instance.
+
+
+9. Error codes
+
+  Let us recall that an ERROR message is composed of <code><message>[...]
+  (see 4.2.1 for details). The supported error codes, as well as any additional
+  information the message may contain after the <message> are described below:
+    0x0000    Server error: something unexpected happened. This indicates a
+              server-side bug.
+    0x000A    Protocol error: some client message triggered a protocol
+              violation (for instance a QUERY message is sent before a STARTUP
+              one has been sent)
+    0x0100    Authentication error: authentication was required and failed. The
+              possible reason for failing depends on the authenticator in use,
+              which may or may not include more detail in the accompanying
+              error message.
+    0x1000    Unavailable exception. The rest of the ERROR message body will be
+                <cl><required><alive>
+              where:
+                <cl> is the [consistency] level of the query that triggered
+                     the exception.
+                <required> is an [int] representing the number of nodes that
+                           should be alive to respect <cl>
+                <alive> is an [int] representing the number of replicas that
+                        were known to be alive when the request had been
+                        processed (since an unavailable exception has been
+                        triggered, there will be <alive> < <required>)
+    0x1001    Overloaded: the request cannot be processed because the
+              coordinator node is overloaded
+    0x1002    Is_bootstrapping: the request was a read request but the
+              coordinator node is bootstrapping
+    0x1003    Truncate_error: error during a truncation error.
+    0x1100    Write_timeout: Timeout exception during a write request. The rest
+              of the ERROR message body will be
+                <cl><received><blockfor><writeType>
+              where:
+                <cl> is the [consistency] level of the query having triggered
+                     the exception.
+                <received> is an [int] representing the number of nodes having
+                           acknowledged the request.
+                <blockfor> is an [int] representing the number of replicas whose
+                           acknowledgement is required to achieve <cl>.
+                <writeType> is a [string] that describe the type of the write
+                            that timed out. The value of that string can be one
+                            of:
+                             - "SIMPLE": the write was a non-batched
+                               non-counter write.
+                             - "BATCH": the write was a (logged) batch write.
+                               If this type is received, it means the batch log
+                               has been successfully written (otherwise a
+                               "BATCH_LOG" type would have been sent instead).
+                             - "UNLOGGED_BATCH": the write was an unlogged
+                               batch. No batch log write has been attempted.
+                             - "COUNTER": the write was a counter write
+                               (batched or not).
+                             - "BATCH_LOG": the timeout occurred during the
+                               write to the batch log when a (logged) batch
+                               write was requested.
+    0x1200    Read_timeout: Timeout exception during a read request. The rest
+              of the ERROR message body will be
+                <cl><received><blockfor><data_present>
+              where:
+                <cl> is the [consistency] level of the query having triggered
+                     the exception.
+                <received> is an [int] representing the number of nodes having
+                           answered the request.
+                <blockfor> is an [int] representing the number of replicas whose
+                           response is required to achieve <cl>. Please note that
+                           it is possible to have <received> >= <blockfor> if
+                           <data_present> is false. Also in the (unlikely)
+                           case where <cl> is achieved but the coordinator node
+                           times out while waiting for read-repair acknowledgement.
+                <data_present> is a single byte. If its value is 0, it means
+                               the replica that was asked for data has not
+                               responded. Otherwise, the value is != 0.
+    0x1300    Read_failure: A non-timeout exception during a read request. The rest
+              of the ERROR message body will be
+                <cl><received><blockfor><numfailures><data_present>
+              where:
+                <cl> is the [consistency] level of the query having triggered
+                     the exception.
+                <received> is an [int] representing the number of nodes having
+                           answered the request.
+                <blockfor> is an [int] representing the number of replicas whose
+                           acknowledgement is required to achieve <cl>.
+                <numfailures> is an [int] representing the number of nodes that
+                              experience a failure while executing the request.
+                <data_present> is a single byte. If its value is 0, it means
+                               the replica that was asked for data had not
+                               responded. Otherwise, the value is != 0.
+    0x1400    Function_failure: A (user defined) function failed during execution.
+              The rest of the ERROR message body will be
+                <keyspace><function><arg_types>
+              where:
+                <keyspace> is the keyspace [string] of the failed function
+                <function> is the name [string] of the failed function
+                <arg_types> [string list] one string for each argument type (as CQL type) of the failed function
+    0x1500    Write_failure: A non-timeout exception during a write request. The rest
+              of the ERROR message body will be
+                <cl><received><blockfor><numfailures><write_type>
+              where:
+                <cl> is the [consistency] level of the query having triggered
+                     the exception.
+                <received> is an [int] representing the number of nodes having
+                           answered the request.
+                <blockfor> is an [int] representing the number of replicas whose
+                           acknowledgement is required to achieve <cl>.
+                <numfailures> is an [int] representing the number of nodes that
+                              experience a failure while executing the request.
+                <writeType> is a [string] that describes the type of the write
+                            that failed. The value of that string can be one
+                            of:
+                             - "SIMPLE": the write was a non-batched
+                               non-counter write.
+                             - "BATCH": the write was a (logged) batch write.
+                               If this type is received, it means the batch log
+                               has been successfully written (otherwise a
+                               "BATCH_LOG" type would have been sent instead).
+                             - "UNLOGGED_BATCH": the write was an unlogged
+                               batch. No batch log write has been attempted.
+                             - "COUNTER": the write was a counter write
+                               (batched or not).
+                             - "BATCH_LOG": the failure occured during the
+                               write to the batch log when a (logged) batch
+                               write was requested.
+
+    0x2000    Syntax_error: The submitted query has a syntax error.
+    0x2100    Unauthorized: The logged user doesn't have the right to perform
+              the query.
+    0x2200    Invalid: The query is syntactically correct but invalid.
+    0x2300    Config_error: The query is invalid because of some configuration issue
+    0x2400    Already_exists: The query attempted to create a keyspace or a
+              table that was already existing. The rest of the ERROR message
+              body will be <ks><table> where:
+                <ks> is a [string] representing either the keyspace that
+                     already exists, or the keyspace in which the table that
+                     already exists is.
+                <table> is a [string] representing the name of the table that
+                        already exists. If the query was attempting to create a
+                        keyspace, <table> will be present but will be the empty
+                        string.
+    0x2500    Unprepared: Can be thrown while a prepared statement tries to be
+              executed if the provided prepared statement ID is not known by
+              this host. The rest of the ERROR message body will be [short
+              bytes] representing the unknown ID.
+
+10. Changes from v3
+
+  * Prepared responses (Section 4.2.5.4) now include partition-key bind indexes
+  * The format of "SCHEMA_CHANGE" events (Section 4.2.6) (and implicitly
+    "Schema_change" results (Section 4.2.5.5)) has been modified, and now includes
+    changes related to user defined functions and user defined aggregates.
+  * Read_failure error code was added.
+  * Function_failure error code was added.
+  * Add custom payload to frames for custom QueryHandler implementations (ignored by
+    Cassandra's standard QueryHandler)
+  * Add warnings to frames for responses for which the server generated a warning
+    during processing, which the client needs to address.
+  * Add the date and time data types
+  * Add the tinyint and smallint data types
+  * The <paging_state> returned in the v4 protocol is not compatible with the v3
+    protocol. In other words, a <paging_state> returned by a node using protocol v4
+    should not be used to query a node using protocol v3 (and vice-versa).
diff --git a/eclipse_compiler.properties b/eclipse_compiler.properties
new file mode 100644
index 0000000..e1f2802
--- /dev/null
+++ b/eclipse_compiler.properties
@@ -0,0 +1,88 @@
+# These options come from 
+# http://grepcode.com/file/repo1.maven.org/maven2/org.eclipse.jdt.core.compiler/ecj/4.2.1/org/eclipse/jdt/internal/compiler/impl/CompilerOptions.java#CompilerOptions
+
+#Look for important errors
+#
+# Autoclosables not in try-with-references
+org.eclipse.jdt.core.compiler.problem.explicitlyClosedAutoCloseable=error
+org.eclipse.jdt.core.compiler.problem.potentiallyUnclosedCloseable=error
+org.eclipse.jdt.core.compiler.problem.unclosedCloseable=ignore
+#Ignore and disable all other checks too keep the logs clean
+
+
+org.eclipse.jdt.core.compiler.problem.annotationSuperInterface=ignore
+org.eclipse.jdt.core.compiler.problem.autoboxing=ignore
+org.eclipse.jdt.core.compiler.problem.comparingIdentical=ignore
+org.eclipse.jdt.core.compiler.problem.deadCode=ignore
+org.eclipse.jdt.core.compiler.problem.deprecation=ignore
+org.eclipse.jdt.core.compiler.problem.deprecationInDeprecatedCode=disabled
+org.eclipse.jdt.core.compiler.problem.deprecationWhenOverridingDeprecatedMethod=disabled
+org.eclipse.jdt.core.compiler.problem.discouragedReference=ignore
+org.eclipse.jdt.core.compiler.problem.emptyStatement=ignore
+org.eclipse.jdt.core.compiler.problem.fallthroughCase=ignore
+org.eclipse.jdt.core.compiler.problem.fatalOptionalError=disabled
+org.eclipse.jdt.core.compiler.problem.fieldHiding=ignore
+org.eclipse.jdt.core.compiler.problem.finalParameterBound=ignore
+org.eclipse.jdt.core.compiler.problem.finallyBlockNotCompletingNormally=ignore
+org.eclipse.jdt.core.compiler.problem.forbiddenReference=ignore
+org.eclipse.jdt.core.compiler.problem.hiddenCatchBlock=ignore
+org.eclipse.jdt.core.compiler.problem.includeNullInfoFromAsserts=disabled
+org.eclipse.jdt.core.compiler.problem.incompatibleNonInheritedInterfaceMethod=ignore
+org.eclipse.jdt.core.compiler.problem.incompleteEnumSwitch=ignore
+org.eclipse.jdt.core.compiler.problem.indirectStaticAccess=ignore
+org.eclipse.jdt.core.compiler.problem.localVariableHiding=ignore
+org.eclipse.jdt.core.compiler.problem.methodWithConstructorName=ignore
+org.eclipse.jdt.core.compiler.problem.missingDefaultCase=ignore
+org.eclipse.jdt.core.compiler.problem.missingDeprecatedAnnotation=ignore
+org.eclipse.jdt.core.compiler.problem.missingEnumCaseDespiteDefault=disabled
+org.eclipse.jdt.core.compiler.problem.missingHashCodeMethod=ignore
+org.eclipse.jdt.core.compiler.problem.missingOverrideAnnotation=ignore
+org.eclipse.jdt.core.compiler.problem.missingOverrideAnnotationForInterfaceMethodImplementation=disabled
+org.eclipse.jdt.core.compiler.problem.missingSerialVersion=ignore
+org.eclipse.jdt.core.compiler.problem.missingSynchronizedOnInheritedMethod=ignore
+org.eclipse.jdt.core.compiler.problem.noEffectAssignment=ignore
+org.eclipse.jdt.core.compiler.problem.noImplicitStringConversion=ignore
+org.eclipse.jdt.core.compiler.problem.nonExternalizedStringLiteral=ignore
+org.eclipse.jdt.core.compiler.problem.nullAnnotationInferenceConflict=error
+org.eclipse.jdt.core.compiler.problem.nullReference=ignore
+org.eclipse.jdt.core.compiler.problem.nullSpecViolation=error
+org.eclipse.jdt.core.compiler.problem.nullUncheckedConversion=ignore
+org.eclipse.jdt.core.compiler.problem.overridingPackageDefaultMethod=ignore
+org.eclipse.jdt.core.compiler.problem.parameterAssignment=ignore
+org.eclipse.jdt.core.compiler.problem.possibleAccidentalBooleanAssignment=ignore
+org.eclipse.jdt.core.compiler.problem.potentialNullReference=ignore
+org.eclipse.jdt.core.compiler.problem.rawTypeReference=ignore
+org.eclipse.jdt.core.compiler.problem.redundantNullAnnotation=ignore
+org.eclipse.jdt.core.compiler.problem.redundantNullCheck=ignore
+org.eclipse.jdt.core.compiler.problem.redundantSpecificationOfTypeArguments=ignore
+org.eclipse.jdt.core.compiler.problem.redundantSuperinterface=ignore
+org.eclipse.jdt.core.compiler.problem.reportMethodCanBePotentiallyStatic=ignore
+org.eclipse.jdt.core.compiler.problem.reportMethodCanBeStatic=ignore
+org.eclipse.jdt.core.compiler.problem.specialParameterHidingField=disabled
+org.eclipse.jdt.core.compiler.problem.staticAccessReceiver=ignore
+org.eclipse.jdt.core.compiler.problem.suppressOptionalErrors=enabled
+org.eclipse.jdt.core.compiler.problem.suppressWarnings=enabled
+org.eclipse.jdt.core.compiler.problem.syntheticAccessEmulation=ignore
+org.eclipse.jdt.core.compiler.problem.typeParameterHiding=ignore
+org.eclipse.jdt.core.compiler.problem.unavoidableGenericTypeProblems=disabled
+org.eclipse.jdt.core.compiler.problem.uncheckedTypeOperation=ignore
+org.eclipse.jdt.core.compiler.problem.undocumentedEmptyBlock=ignore
+org.eclipse.jdt.core.compiler.problem.unhandledWarningToken=ignore
+org.eclipse.jdt.core.compiler.problem.unnecessaryElse=ignore
+org.eclipse.jdt.core.compiler.problem.unnecessaryTypeCheck=ignore
+org.eclipse.jdt.core.compiler.problem.unqualifiedFieldAccess=ignore
+org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownException=ignore
+org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionExemptExceptionAndThrowable=disabled
+org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionIncludeDocCommentReference=disabled
+org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionWhenOverriding=disabled
+org.eclipse.jdt.core.compiler.problem.unusedImport=ignore
+org.eclipse.jdt.core.compiler.problem.unusedLabel=ignore
+org.eclipse.jdt.core.compiler.problem.unusedLocal=ignore
+org.eclipse.jdt.core.compiler.problem.unusedObjectAllocation=ignore
+org.eclipse.jdt.core.compiler.problem.unusedParameter=ignore
+org.eclipse.jdt.core.compiler.problem.unusedParameterIncludeDocCommentReference=disabled
+org.eclipse.jdt.core.compiler.problem.unusedParameterWhenImplementingAbstract=disabled
+org.eclipse.jdt.core.compiler.problem.unusedParameterWhenOverridingConcrete=disabled
+org.eclipse.jdt.core.compiler.problem.unusedPrivateMember=ignore
+org.eclipse.jdt.core.compiler.problem.unusedWarningToken=ignore
+org.eclipse.jdt.core.compiler.problem.varargsArgumentNeedCast=ignore
diff --git a/examples/client_only/README.txt b/examples/client_only/README.txt
deleted file mode 100644
index 47e6a3d..0000000
--- a/examples/client_only/README.txt
+++ /dev/null
@@ -1,49 +0,0 @@
-The client_only example uses the fat client to insert data into and read
-data from Cassandra.
-
--- Warning --
-The method used in this example (the fat client) should generally
-not be used instead of the thrift interface because of possible
-instability of the internal Cassandra API.
-
--- Prerequisite --
-Build the Cassandra source in the current source tree. Also, if
-running the client_only code against a local Cassandra node, start
-the local node prior to running the client_only script. See the
-configuration below for more info.
-
--- Build --
-To build, run ant from the contrib/client_only directory. It will build
-the source, then jar up the compiled class, the conf/cassandra.yaml, and
-dependencies into build/client_only.jar.
-
--- Run --
-To run, from the contrib/client_only directory run:
-bin/client_only write
-or
-bin/client_only read
-
-'write' will create keyspace Keyspace1 and column family Standard1. If
-it is already there, it will error out. It will then write a bunch of
-data to the cluster it connects to.
-
-'read' will read the data that was written in the write step.
-
--- Configuration --
-The conf/cassandra.yaml is to start up the fat client. The fat client
-joins the gossip network but does not participate in storage. It
-needs to have the same configuration as the rest of the cluster except
-listen address and rpc address. If you are running your cluster just
-on your local machine, you'll need to use another address for this node.
-Therefore, your local full Cassandra node can be 127.0.0.1 and the fat 
-client can be 127.0.0.2. Such aliasing is enabled by default on linux.
-On Mac OS X, use the following command to use the second IP address:
-sudo ifconfig lo0 alias 127.0.0.2 up
-
-cassandra.yaml can be on the classpath as is done here, can be specified
-(by modifying the script) in a location within the classpath like this:
-java -Xmx1G -Dcassandra.config=/path/in/classpath/to/cassandra.yaml ...
-or can be retrieved from a location outside the classpath like this:
-... -Dcassandra.config=file:///path/to/cassandra.yaml ...
-or
-... -Dcassandra.config=http://awesomesauce.com/cassandra.yaml ...
\ No newline at end of file
diff --git a/examples/client_only/bin/client_only b/examples/client_only/bin/client_only
deleted file mode 100755
index 99487cc..0000000
--- a/examples/client_only/bin/client_only
+++ /dev/null
@@ -1,56 +0,0 @@
-#!/bin/sh
-
-# 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.
-
-cwd=`dirname $0`
-
-name=client_only
-
-# Cassandra class files
-if [ ! -d $cwd/../../../build/classes/main ]; then
-    echo "Unable to locate cassandra class files" >&2
-    exit 1
-fi
-
-# output jar
-if [ ! -e $cwd/../build/$name.jar ]; then
-    echo "Unable to locate $name jar" >&2
-    exit 1
-fi
-
-CLASSPATH=$CLASSPATH:$cwd/../build/$name.jar
-CLASSPATH=$CLASSPATH:$cwd/../../../build/classes/main
-for jar in $cwd/../../../lib/*.jar; do
-    CLASSPATH=$CLASSPATH:$jar
-done
-for jar in $cwd/../../../build/lib/jars/*.jar; do
-    CLASSPATH=$CLASSPATH:$jar
-done
-
-if [ -x $JAVA_HOME/bin/java ]; then
-    JAVA=$JAVA_HOME/bin/java
-else
-    JAVA=`which java`
-fi
-
-if [ "x$JAVA" = "x" ]; then
-    echo "Java executable not found (hint: set JAVA_HOME)" >&2
-    exit 1
-fi
-
-#echo $CLASSPATH
-"$JAVA" -Xmx1G -ea -cp "$CLASSPATH" ClientOnlyExample $@
diff --git a/examples/client_only/build.xml b/examples/client_only/build.xml
deleted file mode 100644
index 1052d43..0000000
--- a/examples/client_only/build.xml
+++ /dev/null
@@ -1,70 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- ~ 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.
- -->
-<project default="jar" name="client_only">
-    <property name="cassandra.dir" value="../.." />
-    <property name="cassandra.dir.lib" value="${cassandra.dir}/lib" />
-    <property name="cassandra.classes" value="${cassandra.dir}/build/classes/main" />
-    <property name="build.src" value="${basedir}/src" />
-    <property name="build.dir" value="${basedir}/build" />
-    <property name="build.classes" value="${build.dir}/classes" />
-    <property name="final.name" value="client_only" />
-
-    <path id="build.classpath">
-        <!-- cassandra dependencies -->
-        <fileset dir="${cassandra.dir.lib}">
-            <include name="**/*.jar" />
-        </fileset>
-        <fileset dir="${cassandra.dir}/build/lib/jars">
-            <include name="**/*.jar" />
-        </fileset>
-        <pathelement location="${cassandra.classes}" />
-    </path>
-
-    <target name="init">
-        <mkdir dir="${build.classes}" />
-    </target>
-
-    <target name="build" depends="init">
-        <javac destdir="${build.classes}" debug="true"
-               includeantruntime="false">
-            <src path="${build.src}" />
-            <classpath refid="build.classpath" />
-        </javac>
-    </target>
-
-    <target name="jar" depends="build">
-        <mkdir dir="${build.classes}/META-INF" />
-        <jar jarfile="${build.dir}/${final.name}.jar">
-           <fileset dir="${build.classes}" />
-           <fileset dir="${cassandra.classes}" />
-           <fileset dir="${cassandra.dir}">
-               <include name="lib/**/*.jar" />
-           </fileset>
-           <zipfileset dir="${cassandra.dir}/build/lib/jars/" prefix="lib">
-               <include name="**/*.jar" />
-           </zipfileset>
-           <fileset file="${basedir}/conf/cassandra.yaml" />
-        </jar>
-    </target>
-
-    <target name="clean">
-        <delete dir="${build.dir}" />
-    </target>
-</project>
diff --git a/examples/client_only/conf/cassandra.yaml b/examples/client_only/conf/cassandra.yaml
deleted file mode 100644
index a6b3b43..0000000
--- a/examples/client_only/conf/cassandra.yaml
+++ /dev/null
@@ -1,625 +0,0 @@
-# Cassandra storage config YAML 
-
-# NOTE:
-#   See http://wiki.apache.org/cassandra/StorageConfiguration for
-#   full explanations of configuration directives
-# /NOTE
-
-# The name of the cluster. This is mainly used to prevent machines in
-# one logical cluster from joining another.
-cluster_name: 'Test Cluster'
-
-# This defines the number of tokens randomly assigned to this node on the ring
-# The more tokens, relative to other nodes, the larger the proportion of data
-# that this node will store. You probably want all nodes to have the same number
-# of tokens assuming they have equal hardware capability.
-#
-# If you leave this unspecified, Cassandra will use the default of 1 token for legacy compatibility,
-# and will use the initial_token as described below.
-#
-# Specifying initial_token will override this setting.
-#
-# If you already have a cluster with 1 token per node, and wish to migrate to 
-# multiple tokens per node, see http://wiki.apache.org/cassandra/Operations
-# num_tokens: 256
-
-# If you haven't specified num_tokens, or have set it to the default of 1 then
-# you should always specify InitialToken when setting up a production
-# cluster for the first time, and often when adding capacity later.
-# The principle is that each node should be given an equal slice of
-# the token ring; see http://wiki.apache.org/cassandra/Operations
-# for more details.
-#
-# If blank, Cassandra will request a token bisecting the range of
-# the heaviest-loaded existing node.  If there is no load information
-# available, such as is the case with a new cluster, it will pick
-# a random token, which will lead to hot spots.
-initial_token:
-
-# See http://wiki.apache.org/cassandra/HintedHandoff
-hinted_handoff_enabled: true
-# this defines the maximum amount of time a dead host will have hints
-# generated.  After it has been dead this long, new hints for it will not be
-# created until it has been seen alive and gone down again.
-max_hint_window_in_ms: 10800000 # 3 hours
-# throttle in KB's per second, per delivery thread
-hinted_handoff_throttle_in_kb: 1024
-# Number of threads with which to deliver hints;
-# Consider increasing this number when you have multi-dc deployments, since
-# cross-dc handoff tends to be slower
-max_hints_delivery_threads: 2
-
-# authentication backend, implementing IAuthenticator; used to identify users
-authenticator: org.apache.cassandra.auth.AllowAllAuthenticator
-
-# authorization backend, implementing IAuthorizer; used to limit access/provide permissions
-authorizer: org.apache.cassandra.auth.AllowAllAuthorizer
-
-# The partitioner is responsible for distributing rows (by key) across
-# nodes in the cluster.  Any IPartitioner may be used, including your
-# own as long as it is on the classpath.  Out of the box, Cassandra
-# provides org.apache.cassandra.dht.{Murmur3Partitioner, RandomPartitioner
-# ByteOrderedPartitioner, OrderPreservingPartitioner (deprecated)}.
-# 
-# - RandomPartitioner distributes rows across the cluster evenly by md5.
-#   This is the default prior to 1.2 and is retained for compatibility.
-# - Murmur3Partitioner is similar to RandomPartioner but uses Murmur3_128
-#   Hash Function instead of md5.  When in doubt, this is the best option.
-# - ByteOrderedPartitioner orders rows lexically by key bytes.  BOP allows
-#   scanning rows in key order, but the ordering can generate hot spots
-#   for sequential insertion workloads.
-# - OrderPreservingPartitioner is an obsolete form of BOP, that stores
-# - keys in a less-efficient format and only works with keys that are
-#   UTF8-encoded Strings.
-# - CollatingOPP colates according to EN,US rules rather than lexical byte
-#   ordering.  Use this as an example if you need custom collation.
-#
-# See http://wiki.apache.org/cassandra/Operations for more on
-# partitioners and token selection.
-partitioner: org.apache.cassandra.dht.Murmur3Partitioner
-
-# directories where Cassandra should store data on disk.
-data_file_directories:
-    - /tmp/fat-client/data
-
-# commit log
-commitlog_directory: /tmp/fat-client/commitlog
-
-# policy for data disk failures:
-# stop: shut down gossip and Thrift, leaving the node effectively dead, but
-#       still inspectable via JMX.
-# best_effort: stop using the failed disk and respond to requests based on
-#              remaining available sstables.  This means you WILL see obsolete
-#              data at CL.ONE!
-# ignore: ignore fatal errors and let requests fail, as in pre-1.2 Cassandra
-disk_failure_policy: stop
-
-# Maximum size of the key cache in memory.
-#
-# Each key cache hit saves 1 seek and each row cache hit saves 2 seeks at the
-# minimum, sometimes more. The key cache is fairly tiny for the amount of
-# time it saves, so it's worthwhile to use it at large numbers.
-# The row cache saves even more time, but must store the whole values of
-# its rows, so it is extremely space-intensive. It's best to only use the
-# row cache if you have hot rows or static rows.
-#
-# NOTE: if you reduce the size, you may not get you hottest keys loaded on startup.
-#
-# Default value is empty to make it "auto" (min(5% of Heap (in MB), 100MB)). Set to 0 to disable key cache.
-key_cache_size_in_mb:
-
-# Duration in seconds after which Cassandra should
-# safe the keys cache. Caches are saved to saved_caches_directory as
-# specified in this configuration file.
-#
-# Saved caches greatly improve cold-start speeds, and is relatively cheap in
-# terms of I/O for the key cache. Row cache saving is much more expensive and
-# has limited use.
-#
-# Default is 14400 or 4 hours.
-key_cache_save_period: 14400
-
-# Number of keys from the key cache to save
-# Disabled by default, meaning all keys are going to be saved
-# key_cache_keys_to_save: 100
-
-# Maximum size of the row cache in memory.
-# NOTE: if you reduce the size, you may not get you hottest keys loaded on startup.
-#
-# Default value is 0, to disable row caching.
-row_cache_size_in_mb: 0
-
-# Duration in seconds after which Cassandra should
-# safe the row cache. Caches are saved to saved_caches_directory as specified
-# in this configuration file.
-#
-# Saved caches greatly improve cold-start speeds, and is relatively cheap in
-# terms of I/O for the key cache. Row cache saving is much more expensive and
-# has limited use.
-#
-# Default is 0 to disable saving the row cache.
-row_cache_save_period: 0
-
-# Number of keys from the row cache to save
-# Disabled by default, meaning all keys are going to be saved
-# row_cache_keys_to_save: 100
-
-# The provider for the row cache to use.
-#
-# Supported values are: ConcurrentLinkedHashCacheProvider, SerializingCacheProvider
-#
-# SerializingCacheProvider serialises the contents of the row and stores
-# it in native memory, i.e., off the JVM Heap. Serialized rows take
-# significantly less memory than "live" rows in the JVM, so you can cache
-# more rows in a given memory footprint.  And storing the cache off-heap
-# means you can use smaller heap sizes, reducing the impact of GC pauses.
-#
-# It is also valid to specify the fully-qualified class name to a class
-# that implements org.apache.cassandra.cache.IRowCacheProvider.
-#
-# Defaults to SerializingCacheProvider
-row_cache_provider: SerializingCacheProvider
-
-# The pluggable Memory allocation for Off heap row cache, Experiments show that JEMAlloc
-# saves some memory than the native GCC allocator.
-# 
-# Supported values are: NativeAllocator, JEMallocAllocator
-#
-# If you intend to use JEMallocAllocator you have to install JEMalloc as library and
-# modify cassandra-env.sh as directed in the file.
-#
-# Defaults to NativeAllocator
-# memory_allocator: NativeAllocator
-
-# saved caches
-saved_caches_directory: /tmp/fat-client/saved_caches
-
-# commitlog_sync may be either "periodic" or "batch." 
-# When in batch mode, Cassandra won't ack writes until the commit log
-# has been fsynced to disk.  It will wait up to
-# commitlog_sync_batch_window_in_ms milliseconds for other writes, before
-# performing the sync.
-#
-# commitlog_sync: batch
-# commitlog_sync_batch_window_in_ms: 50
-#
-# the other option is "periodic" where writes may be acked immediately
-# and the CommitLog is simply synced every commitlog_sync_period_in_ms
-# milliseconds.
-commitlog_sync: periodic
-commitlog_sync_period_in_ms: 10000
-
-# The size of the individual commitlog file segments.  A commitlog
-# segment may be archived, deleted, or recycled once all the data
-# in it (potentally from each columnfamily in the system) has been 
-# flushed to sstables.  
-#
-# The default size is 32, which is almost always fine, but if you are
-# archiving commitlog segments (see commitlog_archiving.properties),
-# then you probably want a finer granularity of archiving; 8 or 16 MB
-# is reasonable.
-commitlog_segment_size_in_mb: 32
-
-# any class that implements the SeedProvider interface and has a
-# constructor that takes a Map<String, String> of parameters will do.
-seed_provider:
-    # Addresses of hosts that are deemed contact points. 
-    # Cassandra nodes use this list of hosts to find each other and learn
-    # the topology of the ring.  You must change this if you are running
-    # multiple nodes!
-    - class_name: org.apache.cassandra.locator.SimpleSeedProvider
-      parameters:
-          # seeds is actually a comma-delimited list of addresses.
-          # Ex: "<ip1>,<ip2>,<ip3>"
-          - seeds: "127.0.0.1"
-
-# For workloads with more data than can fit in memory, Cassandra's
-# bottleneck will be reads that need to fetch data from
-# disk. "concurrent_reads" should be set to (16 * number_of_drives) in
-# order to allow the operations to enqueue low enough in the stack
-# that the OS and drives can reorder them.
-#
-# On the other hand, since writes are almost never IO bound, the ideal
-# number of "concurrent_writes" is dependent on the number of cores in
-# your system; (8 * number_of_cores) is a good rule of thumb.
-concurrent_reads: 32
-concurrent_writes: 32
-
-# Total memory to use for memtables.  Cassandra will flush the largest
-# memtable when this much memory is used.
-# If omitted, Cassandra will set it to 1/3 of the heap.
-# memtable_total_space_in_mb: 2048
-
-# Total space to use for commitlogs.  Since commitlog segments are
-# mmapped, and hence use up address space, the default size is 32
-# on 32-bit JVMs, and 1024 on 64-bit JVMs.
-#
-# If space gets above this value (it will round up to the next nearest
-# segment multiple), Cassandra will flush every dirty CF in the oldest
-# segment and remove it.  So a small total commitlog space will tend
-# to cause more flush activity on less-active columnfamilies.
-# commitlog_total_space_in_mb: 4096
-
-# This sets the amount of memtable flush writer threads.  These will
-# be blocked by disk io, and each one will hold a memtable in memory
-# while blocked. If you have a large heap and many data directories,
-# you can increase this value for better flush performance.
-# By default this will be set to the amount of data directories defined.
-#memtable_flush_writers: 1
-
-# the number of full memtables to allow pending flush, that is,
-# waiting for a writer thread.  At a minimum, this should be set to
-# the maximum number of secondary indexes created on a single CF.
-memtable_flush_queue_size: 4
-
-# Whether to, when doing sequential writing, fsync() at intervals in
-# order to force the operating system to flush the dirty
-# buffers. Enable this to avoid sudden dirty buffer flushing from
-# impacting read latencies. Almost always a good idea on SSD:s; not
-# necessarily on platters.
-trickle_fsync: false
-trickle_fsync_interval_in_kb: 10240
-
-# TCP port, for commands and data
-storage_port: 7000
-
-# SSL port, for encrypted communication.  Unused unless enabled in
-# encryption_options
-ssl_storage_port: 7001
-
-# Address to bind to and tell other Cassandra nodes to connect to. You
-# _must_ change this if you want multiple nodes to be able to
-# communicate!
-# 
-# Leaving it blank leaves it up to InetAddress.getLocalHost(). This
-# will always do the Right Thing *if* the node is properly configured
-# (hostname, name resolution, etc), and the Right Thing is to use the
-# address associated with the hostname (it might not be).
-#
-# Setting this to 0.0.0.0 is always wrong.
-listen_address: 127.0.0.2
-
-# Address to broadcast to other Cassandra nodes
-# Leaving this blank will set it to the same value as listen_address
-# broadcast_address: 1.2.3.4
-
-
-# Whether to start the native transport server.
-# Currently, only the thrift server is started by default because the native
-# transport is considered beta.
-# Please note that the address on which the native transport is bound is the
-# same as the rpc_address. The port however is different and specified below.
-start_native_transport: false
-# port for the CQL native transport to listen for clients on
-native_transport_port: 9042
-# The minimum and maximum threads for handling requests when the native
-# transport is used. The meaning is those is similar to the one of
-# rpc_min_threads and rpc_max_threads, though the default differ slightly and
-# are the ones below:
-# native_transport_min_threads: 16
-# native_transport_max_threads: 128
-
-
-# Whether to start the thrift rpc server.
-start_rpc: false
-# The address to bind the Thrift RPC service to -- clients connect
-# here. Unlike ListenAddress above, you *can* specify 0.0.0.0 here if
-# you want Thrift to listen on all interfaces.
-# 
-# Leaving this blank has the same effect it does for ListenAddress,
-# (i.e. it will be based on the configured hostname of the node).
-rpc_address: localhost
-# port for Thrift to listen for clients on
-rpc_port: 9160
-
-# enable or disable keepalive on rpc connections
-rpc_keepalive: true
-
-# Cassandra provides three out-of-the-box options for the RPC Server:
-#
-# sync  -> One thread per thrift connection. For a very large number of clients, memory
-#          will be your limiting factor. On a 64 bit JVM, 128KB is the minimum stack size
-#          per thread, and that will correspond to your use of virtual memory (but physical memory
-#          may be limited depending on use of stack space).
-#
-# hsha  -> Stands for "half synchronous, half asynchronous." All thrift clients are handled
-#          asynchronously using a small number of threads that does not vary with the amount
-#          of thrift clients (and thus scales well to many clients). The rpc requests are still
-#          synchronous (one thread per active request).
-#
-# The default is sync because on Windows hsha is about 30% slower.  On Linux,
-# sync/hsha performance is about the same, with hsha of course using less memory.
-#
-# Alternatively,  can provide your own RPC server by providing the fully-qualified class name
-# of an o.a.c.t.TServerFactory that can create an instance of it.
-rpc_server_type: sync
-
-# Uncomment rpc_min|max_thread to set request pool size limits.
-#
-# Regardless of your choice of RPC server (see above), the number of maximum requests in the
-# RPC thread pool dictates how many concurrent requests are possible (but if you are using the sync
-# RPC server, it also dictates the number of clients that can be connected at all).
-#
-# The default is unlimited and thus provide no protection against clients overwhelming the server. You are
-# encouraged to set a maximum that makes sense for you in production, but do keep in mind that
-# rpc_max_threads represents the maximum number of client requests this server may execute concurrently.
-#
-# rpc_min_threads: 16
-# rpc_max_threads: 2048
-
-# uncomment to set socket buffer sizes on rpc connections
-# rpc_send_buff_size_in_bytes:
-# rpc_recv_buff_size_in_bytes:
-
-# uncomment to set socket buffer size for internode communication
-# internode_send_buff_size_in_bytes:
-# internode_recv_buff_size_in_bytes:
-
-# Frame size for thrift (maximum field length).
-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
-# responsibility.
-incremental_backups: false
-
-# Whether or not to take a snapshot before each compaction.  Be
-# careful using this option, since Cassandra won't clean up the
-# snapshots for you.  Mostly useful if you're paranoid when there
-# is a data format change.
-snapshot_before_compaction: false
-
-# Whether or not a snapshot is taken of the data before keyspace truncation
-# or dropping of column families. The STRONGLY advised default of true 
-# should be used to provide data safety. If you set this flag to false, you will
-# lose data on truncation or drop.
-auto_snapshot: true
-
-# Add column indexes to a row after its contents reach this size.
-# Increase if your column values are large, or if you have a very large
-# number of columns.  The competing causes are, Cassandra has to
-# deserialize this much of the row to read a single column, so you want
-# it to be small - at least if you do many partial-row reads - but all
-# the index data is read for each access, so you don't want to generate
-# that wastefully either.
-column_index_size_in_kb: 64
-
-# Size limit for rows being compacted in memory.  Larger rows will spill
-# over to disk and use a slower two-pass compaction process.  A message
-# will be logged specifying the row key.
-in_memory_compaction_limit_in_mb: 64
-
-# Number of simultaneous compactions to allow, NOT including
-# validation "compactions" for anti-entropy repair.  Simultaneous
-# compactions can help preserve read performance in a mixed read/write
-# workload, by mitigating the tendency of small sstables to accumulate
-# during a single long running compactions. The default is usually
-# fine and if you experience problems with compaction running too
-# slowly or too fast, you should look at
-# compaction_throughput_mb_per_sec first.
-#
-# concurrent_compactors defaults to the number of cores.
-# Uncomment to make compaction mono-threaded, the pre-0.8 default.
-#concurrent_compactors: 1
-
-# Multi-threaded compaction. When enabled, each compaction will use
-# up to one thread per core, plus one thread per sstable being merged.
-# This is usually only useful for SSD-based hardware: otherwise, 
-# your concern is usually to get compaction to do LESS i/o (see:
-# compaction_throughput_mb_per_sec), not more.
-multithreaded_compaction: false
-
-# Throttles compaction to the given total throughput across the entire
-# system. The faster you insert data, the faster you need to compact in
-# order to keep the sstable count down, but in general, setting this to
-# 16 to 32 times the rate you are inserting data is more than sufficient.
-# Setting this to 0 disables throttling. Note that this account for all types
-# of compaction, including validation compaction.
-compaction_throughput_mb_per_sec: 16
-
-# Track cached row keys during compaction, and re-cache their new
-# positions in the compacted sstable.  Disable if you use really large
-# key caches.
-compaction_preheat_key_cache: true
-
-# Throttles all outbound streaming file transfers on this node to the
-# given total throughput in Mbps. This is necessary because Cassandra does
-# mostly sequential IO when streaming data during bootstrap or repair, which
-# can lead to saturating the network connection and degrading rpc performance.
-# When unset, the default is 400 Mbps or 50 MB/s.
-# stream_throughput_outbound_megabits_per_sec: 400
-
-# How long the coordinator should wait for read operations to complete
-read_request_timeout_in_ms: 10000
-# How long the coordinator should wait for seq or index scans to complete
-range_request_timeout_in_ms: 10000
-# How long the coordinator should wait for writes to complete
-write_request_timeout_in_ms: 10000
-# How long the coordinator should wait for truncates to complete
-# (This can be much longer, because unless auto_snapshot is disabled
-# we need to flush first so we can snapshot before removing the data.)
-truncate_request_timeout_in_ms: 60000
-# The default timeout for other, miscellaneous operations
-request_timeout_in_ms: 10000
-
-# Enable operation timeout information exchange between nodes to accurately
-# measure request timeouts, If disabled cassandra will assuming the request
-# was forwarded to the replica instantly by the coordinator
-#
-# Warning: before enabling this property make sure to ntp is installed
-# and the times are synchronized between the nodes.
-cross_node_timeout: false
-
-# Enable socket timeout for streaming operation.
-# When a timeout occurs during streaming, streaming is retried from the start
-# of the current file. This *can* involve re-streaming an important amount of
-# data, so you should avoid setting the value too low.
-# Default value is 0, which never timeout streams.
-# streaming_socket_timeout_in_ms: 0
-
-# phi value that must be reached for a host to be marked down.
-# most users should never need to adjust this.
-# phi_convict_threshold: 8
-
-# endpoint_snitch -- Set this to a class that implements
-# IEndpointSnitch.  The snitch has two functions:
-# - it teaches Cassandra enough about your network topology to route
-#   requests efficiently
-# - it allows Cassandra to spread replicas around your cluster to avoid
-#   correlated failures. It does this by grouping machines into
-#   "datacenters" and "racks."  Cassandra will do its best not to have
-#   more than one replica on the same "rack" (which may not actually
-#   be a physical location)
-#
-# IF YOU CHANGE THE SNITCH AFTER DATA IS INSERTED INTO THE CLUSTER,
-# YOU MUST RUN A FULL REPAIR, SINCE THE SNITCH AFFECTS WHERE REPLICAS
-# ARE PLACED.
-#
-# Out of the box, Cassandra provides
-#  - SimpleSnitch:
-#    Treats Strategy order as proximity. This improves cache locality
-#    when disabling read repair, which can further improve throughput.
-#    Only appropriate for single-datacenter deployments.
-#  - PropertyFileSnitch:
-#    Proximity is determined by rack and data center, which are
-#    explicitly configured in cassandra-topology.properties.
-#  - GossipingPropertyFileSnitch
-#    The rack and datacenter for the local node are defined in
-#    cassandra-rackdc.properties and propagated to other nodes via gossip.  If
-#    cassandra-topology.properties exists, it is used as a fallback, allowing
-#    migration from the PropertyFileSnitch.
-#  - RackInferringSnitch:
-#    Proximity is determined by rack and data center, which are
-#    assumed to correspond to the 3rd and 2nd octet of each node's
-#    IP address, respectively.  Unless this happens to match your
-#    deployment conventions (as it did Facebook's), this is best used
-#    as an example of writing a custom Snitch class.
-#  - Ec2Snitch:
-#    Appropriate for EC2 deployments in a single Region.  Loads Region
-#    and Availability Zone information from the EC2 API. The Region is
-#    treated as the Datacenter, and the Availability Zone as the rack.
-#    Only private IPs are used, so this will not work across multiple
-#    Regions.
-#  - Ec2MultiRegionSnitch:
-#    Uses public IPs as broadcast_address to allow cross-region
-#    connectivity.  (Thus, you should set seed addresses to the public
-#    IP as well.) You will need to open the storage_port or
-#    ssl_storage_port on the public IP firewall.  (For intra-Region
-#    traffic, Cassandra will switch to the private IP after
-#    establishing a connection.)
-#
-# You can use a custom Snitch by setting this to the full class name
-# of the snitch, which will be assumed to be on your classpath.
-endpoint_snitch: SimpleSnitch
-
-# controls how often to perform the more expensive part of host score
-# calculation
-dynamic_snitch_update_interval_in_ms: 100 
-# controls how often to reset all host scores, allowing a bad host to
-# possibly recover
-dynamic_snitch_reset_interval_in_ms: 600000
-# if set greater than zero and read_repair_chance is < 1.0, this will allow
-# 'pinning' of replicas to hosts in order to increase cache capacity.
-# The badness threshold will control how much worse the pinned host has to be
-# before the dynamic snitch will prefer other replicas over it.  This is
-# expressed as a double which represents a percentage.  Thus, a value of
-# 0.2 means Cassandra would continue to prefer the static snitch values
-# until the pinned host was 20% worse than the fastest.
-dynamic_snitch_badness_threshold: 0.1
-
-# request_scheduler -- Set this to a class that implements
-# RequestScheduler, which will schedule incoming client requests
-# according to the specific policy. This is useful for multi-tenancy
-# with a single Cassandra cluster.
-# NOTE: This is specifically for requests from the client and does
-# not affect inter node communication.
-# org.apache.cassandra.scheduler.NoScheduler - No scheduling takes place
-# org.apache.cassandra.scheduler.RoundRobinScheduler - Round robin of
-# client requests to a node with a separate queue for each
-# request_scheduler_id. The scheduler is further customized by
-# request_scheduler_options as described below.
-request_scheduler: org.apache.cassandra.scheduler.NoScheduler
-
-# Scheduler Options vary based on the type of scheduler
-# NoScheduler - Has no options
-# RoundRobin
-#  - throttle_limit -- The throttle_limit is the number of in-flight
-#                      requests per client.  Requests beyond 
-#                      that limit are queued up until
-#                      running requests can complete.
-#                      The value of 80 here is twice the number of
-#                      concurrent_reads + concurrent_writes.
-#  - default_weight -- default_weight is optional and allows for
-#                      overriding the default which is 1.
-#  - weights -- Weights are optional and will default to 1 or the
-#               overridden default_weight. The weight translates into how
-#               many requests are handled during each turn of the
-#               RoundRobin, based on the scheduler id.
-#
-# request_scheduler_options:
-#    throttle_limit: 80
-#    default_weight: 5
-#    weights:
-#      Keyspace1: 1
-#      Keyspace2: 5
-
-# request_scheduler_id -- An identifer based on which to perform
-# the request scheduling. Currently the only valid option is keyspace.
-# request_scheduler_id: keyspace
-
-# Enable or disable inter-node encryption
-# Default settings are TLS v1, RSA 1024-bit keys (it is imperative that
-# users generate their own keys) TLS_RSA_WITH_AES_128_CBC_SHA as the cipher
-# suite for authentication, key exchange and encryption of the actual data transfers.
-# NOTE: No custom encryption options are enabled at the moment
-# The available internode options are : all, none, dc, rack
-#
-# If set to dc cassandra will encrypt the traffic between the DCs
-# If set to rack cassandra will encrypt the traffic between the racks
-#
-# The passwords used in these options must match the passwords used when generating
-# the keystore and truststore.  For instructions on generating these files, see:
-# http://download.oracle.com/javase/6/docs/technotes/guides/security/jsse/JSSERefGuide.html#CreateKeystore
-#
-server_encryption_options:
-    internode_encryption: none
-    keystore: conf/.keystore
-    keystore_password: cassandra
-    truststore: conf/.truststore
-    truststore_password: cassandra
-    # More advanced defaults below:
-    # protocol: TLS
-    # algorithm: SunX509
-    # store_type: JKS
-    # cipher_suites: [TLS_RSA_WITH_AES_128_CBC_SHA,TLS_RSA_WITH_AES_256_CBC_SHA]
-    # require_client_auth: false
-
-# enable or disable client/server encryption.
-client_encryption_options:
-    enabled: false
-    keystore: conf/.keystore
-    keystore_password: cassandra
-    # More advanced defaults below:
-    # protocol: TLS
-    # algorithm: SunX509
-    # store_type: JKS
-    # cipher_suites: [TLS_RSA_WITH_AES_128_CBC_SHA,TLS_RSA_WITH_AES_256_CBC_SHA]
-    # require_client_auth: false
-
-# internode_compression controls whether traffic between nodes is
-# compressed.
-# can be:  all  - all traffic is compressed
-#          dc   - traffic between different datacenters is compressed
-#          none - nothing is compressed.
-internode_compression: all
-
-# Enable or disable tcp_nodelay for inter-dc communication.
-# Disabling it will result in larger (but fewer) network packets being sent,
-# reducing overhead from the TCP protocol itself, at the cost of increasing
-# latency if you block for cross-datacenter responses.
-inter_dc_tcp_nodelay: false
diff --git a/examples/client_only/src/ClientOnlyExample.java b/examples/client_only/src/ClientOnlyExample.java
deleted file mode 100644
index 7b42140..0000000
--- a/examples/client_only/src/ClientOnlyExample.java
+++ /dev/null
@@ -1,114 +0,0 @@
-/*
- * 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.
- */
-
-import java.util.concurrent.TimeUnit;
-
-import com.google.common.util.concurrent.Uninterruptibles;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.cql3.QueryProcessor;
-import org.apache.cassandra.cql3.UntypedResultSet;
-import org.apache.cassandra.db.ConsistencyLevel;
-import org.apache.cassandra.exceptions.RequestExecutionException;
-import org.apache.cassandra.exceptions.RequestValidationException;
-import org.apache.cassandra.service.*;
-
-public class ClientOnlyExample
-{
-    private static final Logger logger = LoggerFactory.getLogger(ClientOnlyExample.class);
-
-    private static final String KEYSPACE = "keyspace1";
-    private static final String COLUMN_FAMILY = "standard1";
-
-    private static void startClient() throws Exception
-    {
-        StorageService.instance.initClient();
-    }
-
-    private static void testWriting() throws Exception
-    {
-        // do some writing.
-        for (int i = 0; i < 100; i++)
-        {
-            QueryProcessor.process(String.format("INSERT INTO %s.%s (id, name, value) VALUES ( 'key%d', 'colb', 'value%d')",
-                                                 KEYSPACE,
-                                                 COLUMN_FAMILY,
-                                                 i,
-                                                 i),
-                                   ConsistencyLevel.QUORUM);
-
-            System.out.println("wrote key" + i);
-        }
-        System.out.println("Done writing.");
-    }
-
-    private static void testReading() throws Exception
-    {
-        // do some queries.
-        for (int i = 0; i < 100; i++)
-        {
-            String query = String.format("SELECT id, name, value FROM %s.%s WHERE id = 'key%d'",
-                                         KEYSPACE,
-                                         COLUMN_FAMILY,
-                                         i);
-            UntypedResultSet.Row row = QueryProcessor.process(query, ConsistencyLevel.QUORUM).one();
-            System.out.println(String.format("ID: %s, Name: %s, Value: %s", row.getString("id"), row.getString("name"), row.getString("value")));
-        }
-    }
-
-    /**
-     * First, bring one or more nodes up. Then run ClientOnlyExample with these VM arguments:
-     * <p/>
-     * -Xmx1G
-     * -Dcassandra.config=/Users/gary/cassandra/conf/cassandra.yaml (optional, will first look for cassandra.yaml on classpath)
-     * <p/>
-     * Pass "write" or "read" into the program to exercise the various methods.
-     * <p/>
-     * Caveats:
-     * <p/>
-     * 1.  Much of cassandra is not reentrant.  That is, you can't spin a client up, down, then back up in the same jvm.
-     * 2.  Because of the above, you still need to force-quit the process. StorageService.stopClient() doesn't (can't)
-     * spin everything down.
-     */
-    public static void main(String args[]) throws Exception
-    {
-        startClient();
-        setupKeyspace();
-        testWriting();
-        logger.info("Writing is done. Sleeping, then will try to read.");
-        Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS);
-
-        testReading();
-
-        // no need to do this:
-        // StorageService.instance().decommission();
-        // do this instead:
-        StorageService.instance.stopClient();
-        System.exit(0); // the only way to really stop the process.
-    }
-
-    private static void setupKeyspace() throws RequestExecutionException, RequestValidationException, InterruptedException
-    {
-        QueryProcessor.process("CREATE KEYSPACE IF NOT EXISTS " + KEYSPACE + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}",
-                               ConsistencyLevel.ANY);
-        QueryProcessor.process("CREATE TABLE IF NOT EXISTS " + KEYSPACE + "." + COLUMN_FAMILY + " (id ascii PRIMARY KEY, name ascii, value ascii )",
-                               ConsistencyLevel.ANY);
-        TimeUnit.MILLISECONDS.sleep(1000);
-    }
-}
diff --git a/examples/hadoop_cql3_word_count/README.txt b/examples/hadoop_cql3_word_count/README.txt
index e01632b..b6ee33f 100644
--- a/examples/hadoop_cql3_word_count/README.txt
+++ b/examples/hadoop_cql3_word_count/README.txt
@@ -5,31 +5,33 @@
 and counts them, with RandomPartitioner. The word_count_counters example sums
 the value of counter columns for a key.
 
-The scripts in bin/ assume you are running with cwd of contrib/word_count.
+The scripts in bin/ assume you are running with cwd of examples/word_count.
 
 
 Running
 =======
 
-First build and start a Cassandra server with the default configuration*, 
-then run
+First build and start a Cassandra server with the default configuration*. Ensure that the Thrift
+interface is enabled, either by setting start_rpc:true in cassandra.yaml or by running
+`nodetool enablethrift` after startup.
+Once Cassandra has started and the Thrift interface is available, run
 
 contrib/word_count$ ant
 contrib/word_count$ bin/word_count_setup
 contrib/word_count$ bin/word_count
 contrib/word_count$ bin/word_count_counters
 
-In order to view the results in Cassandra, one can use bin/cassandra-cli and
+In order to view the results in Cassandra, one can use bin/cqlsh and
 perform the following operations:
 $ bin/cqlsh localhost
-> use cql3_worldcount;
+> use cql3_wordcount;
 > select * from output_words;
 
 The output of the word count can now be configured. In the bin/word_count
 file, you can specify the OUTPUT_REDUCER. The two options are 'filesystem'
 and 'cassandra'. The filesystem option outputs to the /tmp/word_count*
 directories. The cassandra option outputs to the 'output_words' column family
-in the 'cql3_worldcount' keyspace.  'cassandra' is the default.
+in the 'cql3_wordcount' keyspace.  'cassandra' is the default.
 
 Read the code in src/ for more details.
 
@@ -45,5 +47,5 @@
 Troubleshooting
 ===============
 
-word_count uses conf/log4j.properties to log to wc.out.
+word_count uses conf/logback.xml to log to wc.out.
 
diff --git a/examples/hadoop_cql3_word_count/bin/word_count_counters b/examples/hadoop_cql3_word_count/bin/word_count_counters
index 0da5e1a..cc1243f 100755
--- a/examples/hadoop_cql3_word_count/bin/word_count_counters
+++ b/examples/hadoop_cql3_word_count/bin/word_count_counters
@@ -30,6 +30,7 @@
     exit 1
 fi
 
+CLASSPATH=$CLASSPATH:$cwd/../conf
 CLASSPATH=$CLASSPATH:$cwd/../build/word_count.jar
 CLASSPATH=$CLASSPATH:$cwd/../../../build/classes/main
 CLASSPATH=$CLASSPATH:$cwd/../../../build/classes/thrift
diff --git a/examples/hadoop_cql3_word_count/conf/log4j.properties b/examples/hadoop_cql3_word_count/conf/log4j.properties
deleted file mode 100644
index 508d60f..0000000
--- a/examples/hadoop_cql3_word_count/conf/log4j.properties
+++ /dev/null
@@ -1,32 +0,0 @@
-# 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.
-
-log4j.rootLogger=INFO,stdout,F
-
-#stdout
-log4j.appender.stdout=org.apache.log4j.ConsoleAppender
-log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
-log4j.appender.stdout.layout.ConversionPattern=%5p %d{HH:mm:ss,SSS} %m%n
-
-# log file
-log4j.appender.F=org.apache.log4j.FileAppender
-log4j.appender.F.Append=false
-log4j.appender.F.layout=org.apache.log4j.PatternLayout
-log4j.appender.F.layout.ConversionPattern=%5p [%t] %d{ISO8601} %F (line %L) %m%n
-# Edit the next line to point to your logs directory
-log4j.appender.F.File=wc.out
-
diff --git a/examples/hadoop_cql3_word_count/conf/logback.xml b/examples/hadoop_cql3_word_count/conf/logback.xml
new file mode 100644
index 0000000..443bd1c
--- /dev/null
+++ b/examples/hadoop_cql3_word_count/conf/logback.xml
@@ -0,0 +1,42 @@
+<!--
+ 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.
+-->
+
+<configuration scan="true">
+
+  <jmxConfigurator />
+
+  <appender name="FILE" class="ch.qos.logback.core.FileAppender">
+    <file>wc.out</file>
+    <encoder>
+      <pattern>%-5level [%thread] %date{ISO8601} %F:%L - %msg%n</pattern>
+    </encoder>
+  </appender>
+
+  <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
+    <encoder>
+      <pattern>%-5level %date{HH:mm:ss,SSS} %msg%n</pattern>
+    </encoder>
+  </appender>
+
+  <root level="INFO">
+    <appender-ref ref="FILE" />
+    <appender-ref ref="STDOUT" />
+  </root>
+
+</configuration>
diff --git a/examples/hadoop_cql3_word_count/src/WordCount.java b/examples/hadoop_cql3_word_count/src/WordCount.java
index 6a2f846..bc95736 100644
--- a/examples/hadoop_cql3_word_count/src/WordCount.java
+++ b/examples/hadoop_cql3_word_count/src/WordCount.java
@@ -37,16 +37,14 @@
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.hadoop.mapreduce.Mapper.Context;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 import com.datastax.driver.core.Row;
-import java.nio.charset.CharacterCodingException;
 
 /**
  * This counts the occurrences of words in ColumnFamily
- *   cql3_worldcount ( id uuid,
+ *   cql3_wordcount ( id uuid,
  *                   line  text,
  *                   PRIMARY KEY (id))
  *
@@ -62,7 +60,7 @@
 {
     private static final Logger logger = LoggerFactory.getLogger(WordCount.class);
     static final String INPUT_MAPPER_VAR = "input_mapper";
-    static final String KEYSPACE = "cql3_worldcount";
+    static final String KEYSPACE = "cql3_wordcount";
     static final String COLUMN_FAMILY = "inputs";
 
     static final String OUTPUT_REDUCER_VAR = "output_reducer";
@@ -122,7 +120,7 @@
         public void map(Long key, Row row, Context context) throws IOException, InterruptedException
         {
             String value = row.getString("line");
-            logger.debug("read {}:{}={} from {}", new Object[] {key, "line", value, context.getInputSplit()});
+            logger.debug("read {}:{}={} from {}", key, "line", value, context.getInputSplit());
             StringTokenizer itr = new StringTokenizer(value);
             while (itr.hasMoreTokens())
             {
diff --git a/examples/hadoop_word_count/README.txt b/examples/hadoop_word_count/README.txt
index 9a93908..e336b89 100644
--- a/examples/hadoop_word_count/README.txt
+++ b/examples/hadoop_word_count/README.txt
@@ -5,26 +5,27 @@
 and counts them, with RandomPartitioner. The word_count_counters example sums
 the value of counter columns for a key.
 
-The scripts in bin/ assume you are running with cwd of contrib/word_count.
+The scripts in bin/ assume you are running with cwd of examples/word_count.
 
 
 Running
 =======
 
-First build and start a Cassandra server with the default configuration*, 
-then run
+First build and start a Cassandra server with the default configuration*. Ensure that the Thrift
+interface is enabled, either by setting start_rpc:true in cassandra.yaml or by running
+`nodetool enablethrift` after startup.
+Once Cassandra has started and the Thrift interface is available, run
 
 contrib/word_count$ ant
 contrib/word_count$ bin/word_count_setup
 contrib/word_count$ bin/word_count
 contrib/word_count$ bin/word_count_counters
 
-In order to view the results in Cassandra, one can use bin/cassandra-cli and
+In order to view the results in Cassandra, one can use bin/cqlsh and
 perform the following operations:
-$ bin/cassandra-cli
-> connect localhost/9160;
+$ bin/cqlsh localhost
 > use wordcount;
-> list output_words;
+> select * from output_words;
 
 The output of the word count can now be configured. In the bin/word_count
 file, you can specify the OUTPUT_REDUCER. The two options are 'filesystem'
@@ -46,4 +47,4 @@
 Troubleshooting
 ===============
 
-word_count uses conf/log4j.properties to log to wc.out.
+word_count uses conf/logback.xml to log to wc.out.
diff --git a/examples/hadoop_word_count/bin/word_count_counters b/examples/hadoop_word_count/bin/word_count_counters
index aaf40b8..122565d 100755
--- a/examples/hadoop_word_count/bin/word_count_counters
+++ b/examples/hadoop_word_count/bin/word_count_counters
@@ -30,6 +30,7 @@
     exit 1
 fi
 
+CLASSPATH=$CLASSPATH:$cwd/../conf
 CLASSPATH=$CLASSPATH:$cwd/../build/word_count.jar
 CLASSPATH=$CLASSPATH:$cwd/../../../build/classes/main
 CLASSPATH=$CLASSPATH:$cwd/../../../build/classes/thrift
diff --git a/examples/hadoop_word_count/conf/log4j.properties b/examples/hadoop_word_count/conf/log4j.properties
deleted file mode 100644
index 508d60f..0000000
--- a/examples/hadoop_word_count/conf/log4j.properties
+++ /dev/null
@@ -1,32 +0,0 @@
-# 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.
-
-log4j.rootLogger=INFO,stdout,F
-
-#stdout
-log4j.appender.stdout=org.apache.log4j.ConsoleAppender
-log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
-log4j.appender.stdout.layout.ConversionPattern=%5p %d{HH:mm:ss,SSS} %m%n
-
-# log file
-log4j.appender.F=org.apache.log4j.FileAppender
-log4j.appender.F.Append=false
-log4j.appender.F.layout=org.apache.log4j.PatternLayout
-log4j.appender.F.layout.ConversionPattern=%5p [%t] %d{ISO8601} %F (line %L) %m%n
-# Edit the next line to point to your logs directory
-log4j.appender.F.File=wc.out
-
diff --git a/examples/hadoop_word_count/conf/logback.xml b/examples/hadoop_word_count/conf/logback.xml
new file mode 100644
index 0000000..443bd1c
--- /dev/null
+++ b/examples/hadoop_word_count/conf/logback.xml
@@ -0,0 +1,42 @@
+<!--
+ 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.
+-->
+
+<configuration scan="true">
+
+  <jmxConfigurator />
+
+  <appender name="FILE" class="ch.qos.logback.core.FileAppender">
+    <file>wc.out</file>
+    <encoder>
+      <pattern>%-5level [%thread] %date{ISO8601} %F:%L - %msg%n</pattern>
+    </encoder>
+  </appender>
+
+  <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
+    <encoder>
+      <pattern>%-5level %date{HH:mm:ss,SSS} %msg%n</pattern>
+    </encoder>
+  </appender>
+
+  <root level="INFO">
+    <appender-ref ref="FILE" />
+    <appender-ref ref="STDOUT" />
+  </root>
+
+</configuration>
diff --git a/examples/hadoop_word_count/src/WordCount.java b/examples/hadoop_word_count/src/WordCount.java
index f6bca77..d092f1f 100644
--- a/examples/hadoop_word_count/src/WordCount.java
+++ b/examples/hadoop_word_count/src/WordCount.java
@@ -20,15 +20,11 @@
 import java.nio.ByteBuffer;
 import java.util.*;
 
-import org.apache.cassandra.db.Cell;
-import org.apache.cassandra.thrift.*;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.hadoop.ColumnFamilyOutputFormat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.hadoop.ColumnFamilyInputFormat;
-import org.apache.cassandra.hadoop.ConfigHelper;
+import org.apache.cassandra.hadoop.*;
+import org.apache.cassandra.thrift.*;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
@@ -71,7 +67,7 @@
         System.exit(0);
     }
 
-    public static class TokenizerMapper extends Mapper<ByteBuffer, SortedMap<ByteBuffer, Cell>, Text, IntWritable>
+    public static class TokenizerMapper extends Mapper<ByteBuffer, SortedMap<ByteBuffer, ColumnFamilyRecordReader.Column>, Text, IntWritable>
     {
         private final static IntWritable one = new IntWritable(1);
         private Text word = new Text();
@@ -82,17 +78,17 @@
         {
         }
 
-        public void map(ByteBuffer key, SortedMap<ByteBuffer, Cell> columns, Context context) throws IOException, InterruptedException
+        public void map(ByteBuffer key, SortedMap<ByteBuffer, ColumnFamilyRecordReader.Column> columns, Context context) throws IOException, InterruptedException
         {
-            for (Cell cell : columns.values())
+            for (ColumnFamilyRecordReader.Column column : columns.values())
             {
-                String name  = ByteBufferUtil.string(cell.name().toByteBuffer());
+                String name  = ByteBufferUtil.string(column.name);
                 String value = null;
                 
                 if (name.contains("int"))
-                    value = String.valueOf(ByteBufferUtil.toInt(cell.value()));
+                    value = String.valueOf(ByteBufferUtil.toInt(column.value));
                 else
-                    value = ByteBufferUtil.string(cell.value());
+                    value = ByteBufferUtil.string(column.value);
                                
                 logger.debug("read {}:{}={} from {}",
                              new Object[] {ByteBufferUtil.string(key), name, value, context.getInputSplit()});
diff --git a/examples/hadoop_word_count/src/WordCountCounters.java b/examples/hadoop_word_count/src/WordCountCounters.java
index 39fb778..98c8579 100644
--- a/examples/hadoop_word_count/src/WordCountCounters.java
+++ b/examples/hadoop_word_count/src/WordCountCounters.java
@@ -20,26 +20,26 @@
 import java.nio.ByteBuffer;
 import java.util.SortedMap;
 
-import org.apache.cassandra.db.Cell;
-import org.apache.cassandra.thrift.*;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.hadoop.ColumnFamilyInputFormat;
+import org.apache.cassandra.hadoop.ColumnFamilyRecordReader;
+import org.apache.cassandra.hadoop.ConfigHelper;
+import org.apache.cassandra.thrift.SlicePredicate;
+import org.apache.cassandra.thrift.SliceRange;
+import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 
-import org.apache.cassandra.hadoop.ColumnFamilyInputFormat;
-import org.apache.cassandra.hadoop.ConfigHelper;
-import org.apache.cassandra.utils.ByteBufferUtil;
-
 /**
  * This sums the word count stored in the input_words_count ColumnFamily for the key "key-if-verse1".
  *
@@ -60,15 +60,15 @@
         System.exit(0);
     }
 
-    public static class SumMapper extends Mapper<ByteBuffer, SortedMap<ByteBuffer, Cell>, Text, LongWritable>
+    public static class SumMapper extends Mapper<ByteBuffer, SortedMap<ByteBuffer, ColumnFamilyRecordReader.Column>, Text, LongWritable>
     {
-        public void map(ByteBuffer key, SortedMap<ByteBuffer, Cell> columns, Context context) throws IOException, InterruptedException
+        public void map(ByteBuffer key, SortedMap<ByteBuffer, ColumnFamilyRecordReader.Column> columns, Context context) throws IOException, InterruptedException
         {
             long sum = 0;
-            for (Cell cell : columns.values())
+            for (ColumnFamilyRecordReader.Column column : columns.values())
             {
-                logger.debug("read " + key + ":" + cell.name() + " from " + context.getInputSplit());
-                sum += ByteBufferUtil.toLong(cell.value());
+                logger.debug("read " + key + ":" + ByteBufferUtil.string(column.name) + " from " + context.getInputSplit());
+                sum += ByteBufferUtil.toLong(column.value);
             }
             context.write(new Text(ByteBufferUtil.string(key)), new LongWritable(sum));
         }
diff --git a/examples/triggers/build.xml b/examples/triggers/build.xml
index 293b08d..450def6 100644
--- a/examples/triggers/build.xml
+++ b/examples/triggers/build.xml
@@ -24,6 +24,7 @@
 	<property name="cassandra.classes" value="${cassandra.dir}/build/classes/main" />
 	<property name="build.src" value="${basedir}/src" />
 	<property name="build.dir" value="${basedir}/build" />
+	<property name="conf.dir" value="${basedir}/conf" />
 	<property name="build.classes" value="${build.dir}/classes" />
 	<property name="final.name" value="trigger-example" />
 
@@ -50,6 +51,9 @@
 	<target name="jar" depends="build">
 		<jar jarfile="${build.dir}/${final.name}.jar">
 			<fileset dir="${build.classes}" />
+			<fileset dir="${conf.dir}">
+				<include name="**/*.properties" />
+			</fileset>
 		</jar>
 	</target>
 
diff --git a/examples/triggers/conf/InvertedIndex.properties b/examples/triggers/conf/InvertedIndex.properties
index 3ec2bb0..ea49a86 100644
--- a/examples/triggers/conf/InvertedIndex.properties
+++ b/examples/triggers/conf/InvertedIndex.properties
@@ -16,4 +16,4 @@
 # under the License.
 
 keyspace=Keyspace1
-columnfamily=InvertedIndex
+table=InvertedIndex
diff --git a/examples/triggers/src/org/apache/cassandra/triggers/InvertedIndex.java b/examples/triggers/src/org/apache/cassandra/triggers/InvertedIndex.java
index 11e98b5..2053387 100644
--- a/examples/triggers/src/org/apache/cassandra/triggers/InvertedIndex.java
+++ b/examples/triggers/src/org/apache/cassandra/triggers/InvertedIndex.java
@@ -41,13 +41,15 @@
     {
         List<Mutation> mutations = new ArrayList<>(update.getColumnCount());
 
+        String indexKeySpace = properties.getProperty("keyspace");
+        String indexColumnFamily = properties.getProperty("table");
         for (Cell cell : update)
         {
             // Skip the row marker and other empty values, since they lead to an empty key.
             if (cell.value().remaining() > 0)
             {
-                Mutation mutation = new Mutation(properties.getProperty("keyspace"), cell.value());
-                mutation.add(properties.getProperty("columnfamily"), cell.name(), key, System.currentTimeMillis());
+                Mutation mutation = new Mutation(indexKeySpace, cell.value());
+                mutation.add(indexColumnFamily, cell.name(), key, System.currentTimeMillis());
                 mutations.add(mutation);
             }
         }
diff --git a/ide/idea-iml-file.xml b/ide/idea-iml-file.xml
new file mode 100644
index 0000000..63d0e1d
--- /dev/null
+++ b/ide/idea-iml-file.xml
@@ -0,0 +1,69 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+
+<module type="JAVA_MODULE" version="4">
+    <component name="NewModuleRootManager" LANGUAGE_LEVEL="JDK_1_7" inherit-compiler-output="false">
+        <output url="file://$MODULE_DIR$/.idea/out/main" />
+        <output-test url="file://$MODULE_DIR$/.idea/out/test" />
+        <exclude-output />
+        <content url="file://$MODULE_DIR$">
+            <sourceFolder url="file://$MODULE_DIR$/src/java" isTestSource="false" />
+            <sourceFolder url="file://$MODULE_DIR$/src/gen-java" isTestSource="false" />
+            <sourceFolder url="file://$MODULE_DIR$/src/resources" type="java-resource" />
+            <sourceFolder url="file://$MODULE_DIR$/interface/thrift/gen-java" isTestSource="false" />
+            <sourceFolder url="file://$MODULE_DIR$/tools/stress/src" isTestSource="false" />
+            <sourceFolder url="file://$MODULE_DIR$/test/unit" isTestSource="true" />
+            <sourceFolder url="file://$MODULE_DIR$/test/long" isTestSource="true" />
+            <sourceFolder url="file://$MODULE_DIR$/test/microbench" isTestSource="true" />
+            <sourceFolder url="file://$MODULE_DIR$/test/burn" isTestSource="true" />
+            <sourceFolder url="file://$MODULE_DIR$/test/pig" isTestSource="true" />
+            <sourceFolder url="file://$MODULE_DIR$/test/resources" type="java-test-resource" />
+            <excludeFolder url="file://$MODULE_DIR$/.idea" />
+            <excludeFolder url="file://$MODULE_DIR$/.settings" />
+            <excludeFolder url="file://$MODULE_DIR$/build" />
+            <excludeFolder url="file://$MODULE_DIR$/data"/>
+            <excludeFolder url="file://$MODULE_DIR$/logs"/>
+        </content>
+        <orderEntry type="inheritedJdk" />
+        <orderEntry type="sourceFolder" forTests="false" />
+        <orderEntry type="module-library">
+            <library>
+                <CLASSES>
+                    <root url="file://$MODULE_DIR$/lib" />
+                </CLASSES>
+                <JAVADOC />
+                <SOURCES />
+                <jarDirectory url="file://$MODULE_DIR$/lib" recursive="false" />
+            </library>
+        </orderEntry>
+        <orderEntry type="module-library">
+            <library>
+                <CLASSES>
+                    <root url="file://$MODULE_DIR$/build/lib/jars" />
+                </CLASSES>
+                <JAVADOC />
+                <SOURCES>
+                    <root url="file://$MODULE_DIR$/build/lib/sources" />
+                </SOURCES>
+                <jarDirectory url="file://$MODULE_DIR$/build/lib/jars" recursive="false" />
+                <jarDirectory url="file://$MODULE_DIR$/build/lib/sources" recursive="false" type="SOURCES" />
+            </library>
+        </orderEntry>
+    </component>
+</module>
diff --git a/ide/idea/inspectionProfiles/Project_Default.xml b/ide/idea/inspectionProfiles/Project_Default.xml
index aa640e8..a609ae3 100644
--- a/ide/idea/inspectionProfiles/Project_Default.xml
+++ b/ide/idea/inspectionProfiles/Project_Default.xml
@@ -19,6 +19,7 @@
     <inspection_tool class="CollectionContainsUrl" enabled="true" level="WARNING" enabled_by_default="true" />
     <inspection_tool class="CollectionsFieldAccessReplaceableByMethodCall" enabled="true" level="WARNING" enabled_by_default="true" />
     <inspection_tool class="ConfusingOctalEscape" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="Convert2streamapi" enabled="false" level="WEAK WARNING" enabled_by_default="false" />
     <inspection_tool class="DanglingJavadoc" enabled="true" level="WARNING" enabled_by_default="true" />
     <inspection_tool class="DivideByZero" enabled="true" level="WARNING" enabled_by_default="true" />
     <inspection_tool class="DoubleCheckedLocking" enabled="true" level="WARNING" enabled_by_default="true">
@@ -29,7 +30,6 @@
     <inspection_tool class="EnumerationCanBeIteration" enabled="true" level="WARNING" enabled_by_default="true" />
     <inspection_tool class="EqualsCalledOnEnumConstant" enabled="true" level="WARNING" enabled_by_default="true" />
     <inspection_tool class="EqualsHashCodeCalledOnUrl" enabled="true" level="WARNING" enabled_by_default="true" />
-    <inspection_tool class="ExtendsUtilityClass" enabled="true" level="WARNING" enabled_by_default="true" />
     <inspection_tool class="FieldCanBeLocal" enabled="true" level="WARNING" enabled_by_default="true">
       <option name="EXCLUDE_ANNOS">
         <value>
@@ -43,7 +43,10 @@
     </inspection_tool>
     <inspection_tool class="FieldMayBeFinal" enabled="true" level="WARNING" enabled_by_default="true" />
     <inspection_tool class="FieldMayBeStatic" enabled="true" level="WARNING" enabled_by_default="true" />
-    <inspection_tool class="HtmlTagCanBeJavadocTag" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="ForCanBeForeach" enabled="false" level="WARNING" enabled_by_default="false">
+      <option name="REPORT_INDEXED_LOOP" value="true" />
+      <option name="ignoreUntypedCollections" value="false" />
+    </inspection_tool>
     <inspection_tool class="IOResource" enabled="true" level="WARNING" enabled_by_default="true">
       <option name="ignoredTypesString" value="java.io.ByteArrayOutputStream,java.io.ByteArrayInputStream,java.io.StringBufferInputStream,java.io.CharArrayWriter,java.io.CharArrayReader,java.io.StringWriter,java.io.StringReader" />
       <option name="insideTryAllowed" value="false" />
@@ -53,23 +56,12 @@
     <inspection_tool class="JavaLangImport" enabled="true" level="WARNING" enabled_by_default="true" />
     <inspection_tool class="KeySetIterationMayUseEntrySet" enabled="true" level="WARNING" enabled_by_default="true" />
     <inspection_tool class="LengthOneStringInIndexOf" enabled="true" level="WARNING" enabled_by_default="true" />
-    <inspection_tool class="LengthOneStringsInConcatenation" enabled="true" level="WARNING" enabled_by_default="true" />
-    <inspection_tool class="ListIndexOfReplaceableByContains" enabled="true" level="WARNING" enabled_by_default="true" />
-    <inspection_tool class="LiteralAsArgToStringEquals" enabled="true" level="WARNING" enabled_by_default="true" />
     <inspection_tool class="LoggerInitializedWithForeignClass" enabled="false" level="WARNING" enabled_by_default="false">
       <option name="loggerClassName" value="org.apache.log4j.Logger,org.slf4j.LoggerFactory,org.apache.commons.logging.LogFactory,java.util.logging.Logger" />
       <option name="loggerFactoryMethodName" value="getLogger,getLogger,getLog,getLogger" />
     </inspection_tool>
     <inspection_tool class="LoggingConditionDisagreesWithLogStatement" enabled="true" level="WARNING" enabled_by_default="true" />
     <inspection_tool class="MapReplaceableByEnumMap" enabled="true" level="WARNING" enabled_by_default="true" />
-    <inspection_tool class="MethodMayBeStatic" enabled="true" level="WARNING" enabled_by_default="true">
-      <option name="m_onlyPrivateOrFinal" value="false" />
-      <option name="m_ignoreEmptyMethods" value="true" />
-    </inspection_tool>
-    <inspection_tool class="NonFinalUtilityClass" enabled="true" level="WARNING" enabled_by_default="true" />
-    <inspection_tool class="NonProtectedConstructorInAbstractClass" enabled="true" level="WARNING" enabled_by_default="true">
-      <option name="m_ignoreNonPublicClasses" value="false" />
-    </inspection_tool>
     <inspection_tool class="NonStaticFinalLogger" enabled="true" level="WARNING" enabled_by_default="true">
       <option name="loggerClassName" value="" />
     </inspection_tool>
@@ -89,8 +81,6 @@
       <option name="ignoreRequiredObsoleteCollectionTypes" value="false" />
     </inspection_tool>
     <inspection_tool class="PointlessIndexOfComparison" enabled="true" level="WARNING" enabled_by_default="true" />
-    <inspection_tool class="ProtectedMemberInFinalClass" enabled="true" level="WARNING" enabled_by_default="true" />
-    <inspection_tool class="PublicConstructorInNonPublicClass" enabled="true" level="WARNING" enabled_by_default="true" />
     <inspection_tool class="PyArgumentListInspection" enabled="false" level="WARNING" enabled_by_default="false" />
     <inspection_tool class="PyPep8Inspection" enabled="false" level="WEAK WARNING" enabled_by_default="false">
       <option name="ignoredErrors">
@@ -108,8 +98,8 @@
     <inspection_tool class="PyShadowingBuiltinsInspection" enabled="false" level="WEAK WARNING" enabled_by_default="false" />
     <inspection_tool class="PyTrailingSemicolonInspection" enabled="false" level="WARNING" enabled_by_default="false" />
     <inspection_tool class="RedundantThrowsDeclaration" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="SafeLock" enabled="true" level="WARNING" enabled_by_default="true" />
     <inspection_tool class="SetReplaceableByEnumSet" enabled="true" level="WARNING" enabled_by_default="true" />
-    <inspection_tool class="SizeReplaceableByIsEmpty" enabled="true" level="WARNING" enabled_by_default="true" />
     <inspection_tool class="SocketResource" enabled="true" level="WARNING" enabled_by_default="true">
       <option name="insideTryAllowed" value="false" />
     </inspection_tool>
@@ -120,7 +110,7 @@
     </inspection_tool>
     <inspection_tool class="SqlDialectInspection" enabled="false" level="WARNING" enabled_by_default="false" />
     <inspection_tool class="SqlNoDataSourceInspection" enabled="false" level="WARNING" enabled_by_default="false" />
-    <inspection_tool class="StringBufferField" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="StaticPseudoFunctionalStyleMethod" enabled="false" level="WARNING" enabled_by_default="false" />
     <inspection_tool class="StringBufferToStringInConcatenation" enabled="true" level="WARNING" enabled_by_default="true" />
     <inspection_tool class="StringConcatenationInLoops" enabled="true" level="WARNING" enabled_by_default="true">
       <option name="m_ignoreUnlessAssigned" value="true" />
@@ -135,19 +125,8 @@
     </inspection_tool>
     <inspection_tool class="TrivialStringConcatenation" enabled="true" level="WARNING" enabled_by_default="true" />
     <inspection_tool class="UnnecessaryCallToStringValueOf" enabled="true" level="WARNING" enabled_by_default="true" />
-    <inspection_tool class="UnnecessaryConstantArrayCreationExpression" enabled="true" level="WARNING" enabled_by_default="true" />
-    <inspection_tool class="UnnecessaryInheritDoc" enabled="true" level="WARNING" enabled_by_default="true" />
-    <inspection_tool class="UnnecessaryLocalVariable" enabled="false" level="WARNING" enabled_by_default="false">
-      <option name="m_ignoreImmediatelyReturnedVariables" value="false" />
-      <option name="m_ignoreAnnotatedVariables" value="false" />
-    </inspection_tool>
-    <inspection_tool class="UtilityClassWithPublicConstructor" enabled="true" level="WARNING" enabled_by_default="true" />
-    <inspection_tool class="UtilityClassWithoutPrivateConstructor" enabled="true" level="WARNING" enabled_by_default="true">
-      <option name="ignorableAnnotations">
-        <value />
-      </option>
-      <option name="ignoreClassesWithOnlyMain" value="false" />
-    </inspection_tool>
+    <inspection_tool class="UnnecessaryInterfaceModifier" enabled="false" level="WARNING" enabled_by_default="false" />
+    <inspection_tool class="WhileCanBeForeach" enabled="false" level="WARNING" enabled_by_default="false" />
     <inspection_tool class="ZeroLengthArrayInitialization" enabled="true" level="WARNING" enabled_by_default="true" />
   </profile>
 </component>
\ No newline at end of file
diff --git a/ide/idea/misc.xml b/ide/idea/misc.xml
new file mode 100644
index 0000000..7aa7466
--- /dev/null
+++ b/ide/idea/misc.xml
@@ -0,0 +1,4 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<project version="4">
+<component name="ProjectRootManager" version="2" languageLevel="JDK_1_8" default="false" assert-keyword="true" jdk-15="true" project-jdk-name="1.8" project-jdk-type="JavaSDK" />
+</project>
\ No newline at end of file
diff --git a/ide/idea/vcs.xml b/ide/idea/vcs.xml
new file mode 100644
index 0000000..94a25f7
--- /dev/null
+++ b/ide/idea/vcs.xml
@@ -0,0 +1,6 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<project version="4">
+  <component name="VcsDirectoryMappings">
+    <mapping directory="$PROJECT_DIR$" vcs="Git" />
+  </component>
+</project>
\ No newline at end of file
diff --git a/ide/idea/workspace.xml b/ide/idea/workspace.xml
new file mode 100644
index 0000000..1645d56
--- /dev/null
+++ b/ide/idea/workspace.xml
@@ -0,0 +1,359 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<project version="4">
+  <component name="ChangeListManager">
+    <list default="true" id="52d1d29d-f11b-427d-b4eb-3d58d14d8c4d" name="Default" comment="">
+      <change type="MODIFICATION" beforePath="$PROJECT_DIR$/build.xml" afterPath="$PROJECT_DIR$/build.xml" />
+    </list>
+    <ignored path="Apache Cassandra trunk.iws" />
+    <ignored path=".idea/workspace.xml" />
+    <ignored path="$PROJECT_DIR$/.idea/" />
+    <ignored path="$PROJECT_DIR$/.settings/" />
+    <ignored path="$PROJECT_DIR$/build/" />
+    <ignored path="$PROJECT_DIR$/data/" />
+    <ignored path="$PROJECT_DIR$/logs/" />
+    <ignored path=".idea/dataSources.local.xml" />
+    <option name="EXCLUDED_CONVERTED_TO_IGNORED" value="true" />
+    <option name="TRACKING_ENABLED" value="true" />
+    <option name="SHOW_DIALOG" value="false" />
+    <option name="HIGHLIGHT_CONFLICTS" value="true" />
+    <option name="HIGHLIGHT_NON_ACTIVE_CHANGELIST" value="false" />
+    <option name="LAST_RESOLUTION" value="IGNORE" />
+  </component>
+  <component name="ChangesViewManager" flattened_view="true" show_ignored="false" />
+  <component name="CreatePatchCommitExecutor">
+    <option name="PATCH_PATH" value="" />
+  </component>
+  <component name="ExecutionTargetManager" SELECTED_TARGET="default_target" />
+  <component name="FavoritesManager">
+    <favorites_list name="Apache Cassandra trunk" />
+  </component>
+  <component name="FileEditorManager">
+    <leaf>
+      <file leaf-file-name="build.xml" pinned="false" current-in-tab="true">
+        <entry file="file://$PROJECT_DIR$/build.xml">
+          <provider selected="true" editor-type-id="text-editor">
+            <state vertical-scroll-proportion="0.11501211">
+              <caret line="1795" column="20" selection-start-line="1795" selection-start-column="20" selection-end-line="1795" selection-end-column="20" />
+              <folding />
+            </state>
+          </provider>
+        </entry>
+      </file>
+    </leaf>
+  </component>
+  <component name="Git.Settings">
+    <option name="RECENT_GIT_ROOT_PATH" value="$PROJECT_DIR$" />
+  </component>
+  <component name="GradleLocalSettings">
+    <option name="externalProjectsViewState">
+      <projects_view />
+    </option>
+  </component>
+  <component name="IdeDocumentHistory">
+    <option name="CHANGED_PATHS">
+      <list>
+        <option value="$PROJECT_DIR$/build.xml" />
+      </list>
+    </option>
+  </component>
+  <component name="JsBuildToolGruntFileManager" detection-done="true" />
+  <component name="JsGulpfileManager">
+    <detection-done>true</detection-done>
+  </component>
+  <component name="NamedScopeManager">
+    <order />
+  </component>
+  <component name="ProjectFrameBounds">
+    <option name="y" value="23" />
+    <option name="width" value="1845" />
+    <option name="height" value="946" />
+  </component>
+  <component name="ProjectLevelVcsManager" settingsEditedManually="false">
+    <OptionsSetting value="true" id="Add" />
+    <OptionsSetting value="true" id="Remove" />
+    <OptionsSetting value="true" id="Checkout" />
+    <OptionsSetting value="true" id="Update" />
+    <OptionsSetting value="true" id="Status" />
+    <OptionsSetting value="true" id="Edit" />
+    <ConfirmationsSetting value="0" id="Add" />
+    <ConfirmationsSetting value="0" id="Remove" />
+  </component>
+  <component name="ProjectView">
+    <navigator currentView="ProjectPane" proportions="" version="1">
+      <flattenPackages />
+      <showMembers />
+      <showModules />
+      <showLibraryContents />
+      <hideEmptyPackages />
+      <abbreviatePackageNames />
+      <autoscrollToSource />
+      <autoscrollFromSource />
+      <sortByType />
+    </navigator>
+    <panes>
+      <pane id="Scratches" />
+      <pane id="Scope" />
+      <pane id="ProjectPane">
+        <subPane>
+          <PATH>
+            <PATH_ELEMENT>
+              <option name="myItemId" value="Apache Cassandra trunk" />
+              <option name="myItemType" value="com.intellij.ide.projectView.impl.nodes.ProjectViewProjectNode" />
+            </PATH_ELEMENT>
+          </PATH>
+          <PATH>
+            <PATH_ELEMENT>
+              <option name="myItemId" value="Apache Cassandra trunk" />
+              <option name="myItemType" value="com.intellij.ide.projectView.impl.nodes.ProjectViewProjectNode" />
+            </PATH_ELEMENT>
+            <PATH_ELEMENT>
+              <option name="myItemId" value="trunk" />
+              <option name="myItemType" value="com.intellij.ide.projectView.impl.nodes.PsiDirectoryNode" />
+            </PATH_ELEMENT>
+          </PATH>
+          <PATH>
+            <PATH_ELEMENT>
+              <option name="myItemId" value="Apache Cassandra trunk" />
+              <option name="myItemType" value="com.intellij.ide.projectView.impl.nodes.ProjectViewProjectNode" />
+            </PATH_ELEMENT>
+            <PATH_ELEMENT>
+              <option name="myItemId" value="trunk" />
+              <option name="myItemType" value="com.intellij.ide.projectView.impl.nodes.PsiDirectoryNode" />
+            </PATH_ELEMENT>
+            <PATH_ELEMENT>
+              <option name="myItemId" value="ide" />
+              <option name="myItemType" value="com.intellij.ide.projectView.impl.nodes.PsiDirectoryNode" />
+            </PATH_ELEMENT>
+            <PATH_ELEMENT>
+              <option name="myItemId" value="idea" />
+              <option name="myItemType" value="com.intellij.ide.projectView.impl.nodes.PsiDirectoryNode" />
+            </PATH_ELEMENT>
+          </PATH>
+        </subPane>
+      </pane>
+      <pane id="PackagesPane" />
+    </panes>
+  </component>
+  <component name="PropertiesComponent">
+    <property name="recentsLimit" value="5" />
+    <property name="aspect.path.notification.shown" value="true" />
+    <property name="WebServerToolWindowFactoryState" value="false" />
+  </component>
+  <component name="RunManager" selected="Application.Cassandra">
+    <configuration default="true" type="Application" factoryName="Application">
+      <extension name="coverage" enabled="false" merge="false" sample_coverage="true" runner="idea" />
+      <option name="MAIN_CLASS_NAME" value="" />
+      <option name="VM_PARAMETERS" value="-Dcassandra.config=file://$PROJECT_DIR$/conf/cassandra.yaml -Dcassandra.storagedir=$PROJECT_DIR$/data -Dlogback.configurationFile=file://$PROJECT_DIR$/conf/logback.xml -Dcassandra.logdir=$PROJECT_DIR$/data/logs -ea" />
+      <option name="PROGRAM_PARAMETERS" value="" />
+      <option name="WORKING_DIRECTORY" value="" />
+      <option name="ALTERNATIVE_JRE_PATH_ENABLED" value="false" />
+      <option name="ALTERNATIVE_JRE_PATH" value="" />
+      <option name="ENABLE_SWING_INSPECTOR" value="false" />
+      <option name="ENV_VARIABLES" />
+      <option name="PASS_PARENT_ENVS" value="true" />
+      <module name="" />
+      <envs />
+      <method>
+        <option name="AntTarget" enabled="true" antfile="file://$PROJECT_DIR$/build.xml" target="gen-cql3-grammar" />
+        <option name="AntTarget" enabled="true" antfile="file://$PROJECT_DIR$/build.xml" target="gen-thrift-java" />
+        <option name="Make" enabled="true" />
+      </method>
+    </configuration>
+    <configuration default="true" type="JUnit" factoryName="JUnit">
+      <extension name="coverage" enabled="false" merge="false" sample_coverage="true" runner="idea" />
+      <module name="" />
+      <option name="ALTERNATIVE_JRE_PATH_ENABLED" value="false" />
+      <option name="ALTERNATIVE_JRE_PATH" value="" />
+      <option name="PACKAGE_NAME" />
+      <option name="MAIN_CLASS_NAME" value="" />
+      <option name="METHOD_NAME" value="" />
+      <option name="TEST_OBJECT" value="class" />
+      <option name="VM_PARAMETERS" value="-Dcassandra.config=file://$PROJECT_DIR$/test/conf/cassandra.yaml -Dlogback.configurationFile=file://$PROJECT_DIR$/test/conf/logback-test.xml -Dcassandra.logdir=$PROJECT_DIR$/build/test/logs -ea" />
+      <option name="PARAMETERS" value="" />
+      <option name="WORKING_DIRECTORY" value="" />
+      <option name="ENV_VARIABLES" />
+      <option name="PASS_PARENT_ENVS" value="true" />
+      <option name="TEST_SEARCH_SCOPE">
+        <value defaultName="singleModule" />
+      </option>
+      <envs />
+      <patterns />
+      <method>
+        <option name="AntTarget" enabled="true" antfile="file://$PROJECT_DIR$/build.xml" target="gen-cql3-grammar" />
+        <option name="AntTarget" enabled="true" antfile="file://$PROJECT_DIR$/build.xml" target="gen-thrift-java" />
+        <option name="Make" enabled="true" />
+      </method>
+    </configuration>
+    <configuration default="false" name="Cassandra" type="Application" factoryName="Application">
+      <extension name="coverage" enabled="false" merge="false" sample_coverage="true" runner="idea" />
+      <option name="MAIN_CLASS_NAME" value="org.apache.cassandra.service.CassandraDaemon" />
+      <option name="VM_PARAMETERS" value="-Dcassandra-foreground=yes -Dcassandra.config=file://$PROJECT_DIR$/conf/cassandra.yaml -Dcassandra.storagedir=$PROJECT_DIR$/data -Dlogback.configurationFile=file://$PROJECT_DIR$/conf/logback.xml -Dcassandra.logdir=$PROJECT_DIR$/data/logs -Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.port=7199 -Dcom.sun.management.jmxremote.local.only=false -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.ssl=false -ea -Xmx1G" />
+      <option name="PROGRAM_PARAMETERS" value="" />
+      <option name="WORKING_DIRECTORY" value="file://$PROJECT_DIR$" />
+      <option name="ALTERNATIVE_JRE_PATH_ENABLED" value="false" />
+      <option name="ALTERNATIVE_JRE_PATH" value="" />
+      <option name="ENABLE_SWING_INSPECTOR" value="false" />
+      <option name="ENV_VARIABLES" />
+      <option name="PASS_PARENT_ENVS" value="true" />
+      <module name="trunk" />
+      <envs />
+      <RunnerSettings RunnerId="Debug">
+        <option name="DEBUG_PORT" value="" />
+        <option name="TRANSPORT" value="0" />
+        <option name="LOCAL" value="true" />
+      </RunnerSettings>
+      <RunnerSettings RunnerId="Run" />
+      <ConfigurationWrapper RunnerId="Debug" />
+      <ConfigurationWrapper RunnerId="Run" />
+      <method>
+        <option name="AntTarget" enabled="true" antfile="file://$PROJECT_DIR$/build.xml" target="gen-cql3-grammar" />
+        <option name="Make" enabled="true" />
+      </method>
+    </configuration>
+    <list size="1">
+      <item index="0" class="java.lang.String" itemvalue="Application.Cassandra" />
+    </list>
+  </component>
+  <component name="SbtLocalSettings">
+    <option name="externalProjectsViewState">
+      <projects_view />
+    </option>
+  </component>
+  <component name="ShelveChangesManager" show_recycled="false" />
+  <component name="TaskManager">
+    <task active="true" id="Default" summary="Default task">
+      <changelist id="52d1d29d-f11b-427d-b4eb-3d58d14d8c4d" name="Default" comment="" />
+      <created>1437223363258</created>
+      <option name="number" value="Default" />
+      <updated>1437223363258</updated>
+      <workItem from="1437223365797" duration="161000" />
+    </task>
+    <servers />
+  </component>
+  <component name="TimeTrackingManager">
+    <option name="totallyTimeSpent" value="161000" />
+  </component>
+  <component name="ToolWindowManager">
+    <frame x="0" y="23" width="1845" height="946" extended-state="0" />
+    <editor active="true" />
+    <layout>
+      <window_info id="Palette" active="false" anchor="right" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.33" sideWeight="0.5" order="-1" side_tool="false" content_ui="tabs" />
+      <window_info id="TODO" active="false" anchor="bottom" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.33" sideWeight="0.5" order="6" side_tool="false" content_ui="tabs" />
+      <window_info id="SBT Console" active="false" anchor="bottom" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.33" sideWeight="0.5" order="-1" side_tool="false" content_ui="tabs" />
+      <window_info id="Palette&#9;" active="false" anchor="right" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.33" sideWeight="0.5" order="-1" side_tool="false" content_ui="tabs" />
+      <window_info id="Java Enterprise" active="false" anchor="bottom" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.33" sideWeight="0.5" order="-1" side_tool="false" content_ui="tabs" />
+      <window_info id="Event Log" active="false" anchor="bottom" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.33" sideWeight="0.5" order="-1" side_tool="true" content_ui="tabs" />
+      <window_info id="Application Servers" active="false" anchor="bottom" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.33" sideWeight="0.5" order="-1" side_tool="false" content_ui="tabs" />
+      <window_info id="Maven Projects" active="false" anchor="right" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.33" sideWeight="0.5" order="-1" side_tool="false" content_ui="tabs" />
+      <window_info id="Version Control" active="false" anchor="bottom" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.33" sideWeight="0.5" order="-1" side_tool="false" content_ui="tabs" />
+      <window_info id="IvyIDEA" active="false" anchor="bottom" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.33" sideWeight="0.5" order="-1" side_tool="false" content_ui="tabs" />
+      <window_info id="Terminal" active="false" anchor="bottom" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.33" sideWeight="0.5" order="-1" side_tool="false" content_ui="tabs" />
+      <window_info id="Designer" active="false" anchor="left" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.33" sideWeight="0.5" order="-1" side_tool="false" content_ui="tabs" />
+      <window_info id="Project" active="false" anchor="left" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="true" weight="0.24958402" sideWeight="0.5" order="0" side_tool="false" content_ui="tabs" />
+      <window_info id="Database" active="false" anchor="right" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.33" sideWeight="0.5" order="-1" side_tool="false" content_ui="tabs" />
+      <window_info id="Structure" active="false" anchor="left" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.25" sideWeight="0.5" order="1" side_tool="false" content_ui="tabs" />
+      <window_info id="Ant Build" active="false" anchor="right" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.25" sideWeight="0.5" order="1" side_tool="false" content_ui="tabs" />
+      <window_info id="UI Designer" active="false" anchor="left" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.33" sideWeight="0.5" order="-1" side_tool="false" content_ui="tabs" />
+      <window_info id="Favorites" active="false" anchor="left" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.33" sideWeight="0.5" order="-1" side_tool="true" content_ui="tabs" />
+      <window_info id="Cvs" active="false" anchor="bottom" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.25" sideWeight="0.5" order="4" side_tool="false" content_ui="tabs" />
+      <window_info id="Hierarchy" active="false" anchor="right" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.25" sideWeight="0.5" order="2" side_tool="false" content_ui="combo" />
+      <window_info id="Message" active="false" anchor="bottom" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.33" sideWeight="0.5" order="0" side_tool="false" content_ui="tabs" />
+      <window_info id="Commander" active="false" anchor="right" auto_hide="false" internal_type="SLIDING" type="SLIDING" visible="false" weight="0.4" sideWeight="0.5" order="0" side_tool="false" content_ui="tabs" />
+      <window_info id="Find" active="false" anchor="bottom" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.33" sideWeight="0.5" order="1" side_tool="false" content_ui="tabs" />
+      <window_info id="Inspection" active="false" anchor="bottom" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.4" sideWeight="0.5" order="5" side_tool="false" content_ui="tabs" />
+      <window_info id="Run" active="false" anchor="bottom" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.33" sideWeight="0.5" order="2" side_tool="false" content_ui="tabs" />
+      <window_info id="Debug" active="false" anchor="bottom" auto_hide="false" internal_type="DOCKED" type="DOCKED" visible="false" weight="0.4" sideWeight="0.5" order="3" side_tool="false" content_ui="tabs" />
+    </layout>
+  </component>
+  <component name="VcsContentAnnotationSettings">
+    <option name="myLimit" value="2678400000" />
+  </component>
+  <component name="XDebuggerManager">
+    <breakpoint-manager />
+    <watches-manager />
+  </component>
+  <component name="antWorkspaceConfiguration">
+    <option name="IS_AUTOSCROLL_TO_SOURCE" value="false" />
+    <option name="FILTER_TARGETS" value="false" />
+    <buildFile url="file://$PROJECT_DIR$/build.xml">
+      <targetFilters>
+        <filter targetName="init" isVisible="false" />
+        <filter targetName="clean" isVisible="true" />
+        <filter targetName="cleanall" isVisible="false" />
+        <filter targetName="realclean" isVisible="true" />
+        <filter targetName="check-gen-cql3-grammar" isVisible="false" />
+        <filter targetName="gen-cql3-grammar" isVisible="false" />
+        <filter targetName="generate-cql-html" isVisible="true" />
+        <filter targetName="maven-ant-tasks-localrepo" isVisible="true" />
+        <filter targetName="maven-ant-tasks-download" isVisible="true" />
+        <filter targetName="maven-ant-tasks-init" isVisible="true" />
+        <filter targetName="maven-declare-dependencies" isVisible="true" />
+        <filter targetName="maven-ant-tasks-retrieve-build" isVisible="false" />
+        <filter targetName="maven-ant-tasks-retrieve-test" isVisible="false" />
+        <filter targetName="maven-ant-tasks-retrieve-pig-test" isVisible="false" />
+        <filter targetName="check-gen-thrift-java" isVisible="false" />
+        <filter targetName="gen-thrift-java" isVisible="true" />
+        <filter targetName="_write-java-license-headers" isVisible="false" />
+        <filter targetName="write-java-license-headers" isVisible="true" />
+        <filter targetName="gen-thrift-py" isVisible="true" />
+        <filter targetName="createVersionPropFile" isVisible="false" />
+        <filter targetName="test-run" isVisible="true" />
+        <filter targetName="build" isVisible="true" />
+        <filter targetName="codecoverage" isVisible="true" />
+        <filter targetName="build-project" isVisible="false" />
+        <filter targetName="stress-build" isVisible="true" />
+        <filter targetName="_write-poms" isVisible="false" />
+        <filter targetName="write-poms" isVisible="false" />
+        <filter targetName="jar" isVisible="true" />
+        <filter targetName="javadoc-jar" isVisible="true" />
+        <filter targetName="sources-jar" isVisible="true" />
+        <filter targetName="artifacts" isVisible="true" />
+        <filter targetName="release" isVisible="true" />
+        <filter targetName="build-test" isVisible="true" />
+        <filter targetName="test-clientutil-jar" isVisible="true" />
+        <filter targetName="testold" isVisible="true" />
+        <filter targetName="jacoco-run" isVisible="true" />
+        <filter targetName="testsome" isVisible="true" />
+        <filter targetName="test-compression" isVisible="true" />
+        <filter targetName="msg-ser-gen-test" isVisible="true" />
+        <filter targetName="msg-ser-test" isVisible="true" />
+        <filter targetName="msg-ser-test-7" isVisible="true" />
+        <filter targetName="msg-ser-test-10" isVisible="true" />
+        <filter targetName="test-burn" isVisible="true" />
+        <filter targetName="long-test" isVisible="true" />
+        <filter targetName="cql-test" isVisible="true" />
+        <filter targetName="cql-test-some" isVisible="true" />
+        <filter targetName="pig-test" isVisible="true" />
+        <filter targetName="test-all" isVisible="true" />
+        <filter targetName="jacoco-init" isVisible="false" />
+        <filter targetName="jacoco-report" isVisible="false" />
+        <filter targetName="jacoco-cleanup" isVisible="true" />
+        <filter targetName="cobertura-instrument" isVisible="false" />
+        <filter targetName="cobertura-report" isVisible="false" />
+        <filter targetName="rat-init" isVisible="false" />
+        <filter targetName="rat-check" isVisible="false" />
+        <filter targetName="rat-write" isVisible="false" />
+        <filter targetName="javadoc" isVisible="true" />
+        <filter targetName="test" isVisible="true" />
+        <filter targetName="microbench" isVisible="false" />
+        <filter targetName="generate-idea-files" isVisible="true" />
+        <filter targetName="generate-eclipse-files" isVisible="true" />
+        <filter targetName="clean-eclipse-files" isVisible="false" />
+        <filter targetName="eclipse-warnings" isVisible="true" />
+        <filter targetName="mvn-install" isVisible="true" />
+        <filter targetName="publish" isVisible="true" />
+      </targetFilters>
+      <expanded value="true" />
+    </buildFile>
+  </component>
+  <component name="editorHistoryManager">
+    <entry file="file://$PROJECT_DIR$/build.xml">
+      <provider selected="true" editor-type-id="text-editor">
+        <state vertical-scroll-proportion="0.11501211">
+          <caret line="1795" column="20" selection-start-line="1795" selection-start-column="20" selection-end-line="1795" selection-end-column="20" />
+          <folding />
+        </state>
+      </provider>
+    </entry>
+  </component>
+</project>
\ No newline at end of file
diff --git a/interface/cassandra.thrift b/interface/cassandra.thrift
index 2484a5d..f5041c8 100644
--- a/interface/cassandra.thrift
+++ b/interface/cassandra.thrift
@@ -55,7 +55,7 @@
 # An effort should be made not to break forward-client-compatibility either
 # (e.g. one should avoid removing obsolete fields from the IDL), but no
 # guarantees in this respect are made by the Cassandra project.
-const string VERSION = "19.39.0"
+const string VERSION = "20.1.0"
 
 
 #
@@ -117,6 +117,7 @@
     4: optional CounterSuperColumn counter_super_column
 }
 
+
 #
 # Exceptions
 # (note that internal server errors will raise a TApplicationException, courtesy of Thrift)
@@ -570,7 +571,7 @@
     3: required i64 row_count
 }
 
-/** The ColumnSlice is used to select a set of columns from inside a row.
+/** The ColumnSlice is used to select a set of columns from inside a row. 
  * If start or finish are unspecified they will default to the start-of
  * end-of value.
  * @param start. The start of the ColumnSlice inclusive
@@ -782,7 +783,6 @@
   */
   list<ColumnOrSuperColumn> get_multi_slice(1:required MultiSliceRequest request)
        throws (1:InvalidRequestException ire, 2:UnavailableException ue, 3:TimedOutException te),
-
     
   // Meta-APIs -- APIs to get information about the node or cluster,
   // rather than user data.  The nodeprobe program provides usage examples.
@@ -884,7 +884,7 @@
 
 
   /**
-   * @deprecated Will become a no-op in 2.2. Please use the CQL3 version instead.
+   * @deprecated Throws InvalidRequestException since 2.2. Please use the CQL3 version instead.
    */
   CqlResult execute_cql_query(1:required binary query, 2:required Compression compression)
     throws (1:InvalidRequestException ire,
@@ -904,7 +904,7 @@
 
 
   /**
-   * @deprecated Will become a no-op in 2.2. Please use the CQL3 version instead.
+   * @deprecated Throws InvalidRequestException since 2.2. Please use the CQL3 version instead.
    */
   CqlPreparedResult prepare_cql_query(1:required binary query, 2:required Compression compression)
     throws (1:InvalidRequestException ire)
@@ -920,7 +920,7 @@
 
 
   /**
-   * @deprecated Will become a no-op in 2.2. Please use the CQL3 version instead.
+   * @deprecated Throws InvalidRequestException since 2.2. Please use the CQL3 version instead.
    */
   CqlResult execute_prepared_cql_query(1:required i32 itemId, 2:required list<binary> values)
     throws (1:InvalidRequestException ire,
diff --git a/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java b/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java
index 55f4734..cd4314b 100644
--- a/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java
+++ b/interface/thrift/gen-java/org/apache/cassandra/thrift/Cassandra.java
@@ -386,7 +386,7 @@
     public String system_update_column_family(CfDef cf_def) throws InvalidRequestException, SchemaDisagreementException, org.apache.thrift.TException;
 
     /**
-     * @deprecated Will become a no-op in 2.2. Please use the CQL3 version instead.
+     * @deprecated Throws InvalidRequestException since 2.2. Please use the CQL3 version instead.
      * 
      * @param query
      * @param compression
@@ -404,7 +404,7 @@
     public CqlResult execute_cql3_query(ByteBuffer query, Compression compression, ConsistencyLevel consistency) throws InvalidRequestException, UnavailableException, TimedOutException, SchemaDisagreementException, org.apache.thrift.TException;
 
     /**
-     * @deprecated Will become a no-op in 2.2. Please use the CQL3 version instead.
+     * @deprecated Throws InvalidRequestException since 2.2. Please use the CQL3 version instead.
      * 
      * @param query
      * @param compression
@@ -423,7 +423,7 @@
     public CqlPreparedResult prepare_cql3_query(ByteBuffer query, Compression compression) throws InvalidRequestException, org.apache.thrift.TException;
 
     /**
-     * @deprecated Will become a no-op in 2.2. Please use the CQL3 version instead.
+     * @deprecated Throws InvalidRequestException since 2.2. Please use the CQL3 version instead.
      * 
      * @param itemId
      * @param values
diff --git a/interface/thrift/gen-java/org/apache/cassandra/thrift/cassandraConstants.java b/interface/thrift/gen-java/org/apache/cassandra/thrift/cassandraConstants.java
index 81b0ffb..f84243e 100644
--- a/interface/thrift/gen-java/org/apache/cassandra/thrift/cassandraConstants.java
+++ b/interface/thrift/gen-java/org/apache/cassandra/thrift/cassandraConstants.java
@@ -56,6 +56,6 @@
 
 public class cassandraConstants {
 
-  public static final String VERSION = "19.39.0";
+  public static final String VERSION = "20.1.0";
 
 }
diff --git a/lib/cassandra-driver-core-2.2.0-rc2-SNAPSHOT-20150617-shaded.jar b/lib/cassandra-driver-core-2.2.0-rc2-SNAPSHOT-20150617-shaded.jar
new file mode 100644
index 0000000..7d971df
--- /dev/null
+++ b/lib/cassandra-driver-core-2.2.0-rc2-SNAPSHOT-20150617-shaded.jar
Binary files differ
diff --git a/lib/cassandra-driver-internal-only-2.7.2-2fc8a2b.zip b/lib/cassandra-driver-internal-only-2.7.2-2fc8a2b.zip
deleted file mode 100644
index cabe316..0000000
--- a/lib/cassandra-driver-internal-only-2.7.2-2fc8a2b.zip
+++ /dev/null
Binary files differ
diff --git a/lib/cassandra-driver-internal-only-3.5.0.post0-d8d0456.zip b/lib/cassandra-driver-internal-only-3.5.0.post0-d8d0456.zip
new file mode 100644
index 0000000..7d23b48
--- /dev/null
+++ b/lib/cassandra-driver-internal-only-3.5.0.post0-d8d0456.zip
Binary files differ
diff --git a/lib/crc32ex-0.1.1.jar b/lib/crc32ex-0.1.1.jar
new file mode 100644
index 0000000..4ba70ff
--- /dev/null
+++ b/lib/crc32ex-0.1.1.jar
Binary files differ
diff --git a/lib/ecj-4.4.2.jar b/lib/ecj-4.4.2.jar
new file mode 100644
index 0000000..d9411b3
--- /dev/null
+++ b/lib/ecj-4.4.2.jar
Binary files differ
diff --git a/lib/jcl-over-slf4j-1.7.7.jar b/lib/jcl-over-slf4j-1.7.7.jar
new file mode 100644
index 0000000..ed8d4dd
--- /dev/null
+++ b/lib/jcl-over-slf4j-1.7.7.jar
Binary files differ
diff --git a/lib/jline-1.0.jar b/lib/jline-1.0.jar
deleted file mode 100644
index 6c949b2..0000000
--- a/lib/jline-1.0.jar
+++ /dev/null
Binary files differ
diff --git a/lib/joda-time-2.4.jar b/lib/joda-time-2.4.jar
new file mode 100644
index 0000000..ace67d7
--- /dev/null
+++ b/lib/joda-time-2.4.jar
Binary files differ
diff --git a/lib/jsr223/clojure/README.txt b/lib/jsr223/clojure/README.txt
new file mode 100644
index 0000000..7ed7551
--- /dev/null
+++ b/lib/jsr223/clojure/README.txt
@@ -0,0 +1,8 @@
+Apache Cassandra User-Defined-Functions JSR 223 scripting
+=========================================================
+
+Unfortunately the JSR-223 support provided by the project https://github.com/ato/clojure-jsr223
+and the related ones do not provide compileable script support.
+
+The JSR-223 javax.script.Compilable implementation takes source file names or readers but not script sources
+as all other JSR-223 implementations do.
diff --git a/lib/jsr223/groovy/README.txt b/lib/jsr223/groovy/README.txt
new file mode 100644
index 0000000..09fef93
--- /dev/null
+++ b/lib/jsr223/groovy/README.txt
@@ -0,0 +1,35 @@
+Apache Cassandra User-Defined-Functions JSR 223 scripting
+=========================================================
+
+Using JSR-223 capable Groovy
+
+Tested with version 2.3.6
+
+Installation
+------------
+
+1. Download Groovy binary release
+2. Unpack the downloaded archive into a temporary directory
+3. Copy the jar groovy-all-2.3.6-indy.jar from the Groovy embeddable directory to $CASSANDRA_HOME/lib/jsr223/groovy
+   "indy" means "invokedynamic" and is a JVM instruction for scripting languages new to Java 7.
+4. Restart your Cassandra daemon if it's already running
+
+Cassandra log should contain a line like this:
+  INFO  10:49:45 Found scripting engine Groovy Scripting Engine 2.0 - Groovy 2.3.6 - language names: [groovy, Groovy]
+Such a line appears when you already have scripted UDFs in your system or add a scripted UDF for the first time (see below).
+
+Smoke Test
+----------
+
+To test Groovy functionality, open cqlsh and execute the following command:
+  CREATE OR REPLACE FUNCTION foobar ( input text ) RETURNS text LANGUAGE groovy AS 'return "foo";' ;
+
+If you get the error
+  code=2200 [Invalid query] message="Invalid language groovy for 'foobar'"
+Groovy for Apache Cassandra has not been installed correctly.
+
+Notes / Java7 invokedynamic
+---------------------------
+
+Groovy provides jars that support invokedynamic bytecode instruction. These jars are whose ending with
+"-indy.jar".
diff --git a/lib/jsr223/jaskell/README.txt b/lib/jsr223/jaskell/README.txt
new file mode 100644
index 0000000..53e942e
--- /dev/null
+++ b/lib/jsr223/jaskell/README.txt
@@ -0,0 +1,5 @@
+Apache Cassandra User-Defined-Functions JSR 223 scripting
+=========================================================
+
+Unfortunately Jaskell JSR-223 support is quite old and the Jaskell engine seems to be quite
+unsupported. If you find a solution, please open a ticket at Apache Cassandra JIRA.
diff --git a/lib/jsr223/jruby/README.txt b/lib/jsr223/jruby/README.txt
new file mode 100644
index 0000000..cbc12dc
--- /dev/null
+++ b/lib/jsr223/jruby/README.txt
@@ -0,0 +1,54 @@
+Apache Cassandra User-Defined-Functions JSR 223 scripting
+=========================================================
+
+Using JSR-223 capable JRuby
+
+Tested with version 1.7.15
+
+Installation
+------------
+
+1. Download JRuby binary release
+2. Unpack the downloaded archive into a temporary directory
+3. Copy everything from the JRuby lib directory to $CASSANDRA_HOME/lib/jsr223/jruby
+4. Restart your Cassandra daemon if it's already running
+
+Cassandra log should contain a line like this:
+  INFO  10:29:03 Found scripting engine JSR 223 JRuby Engine 1.7.15 - ruby jruby 1.7.15 - language names: [ruby, jruby]
+Such a line appears when you already have scripted UDFs in your system or add a scripted UDF for the first time (see below).
+
+
+Smoke Test
+----------
+
+To test JRuby functionality, open cqlsh and execute the following command:
+  CREATE OR REPLACE FUNCTION foobar ( input text ) RETURNS text LANGUAGE ruby AS 'return "foo";' ;
+
+If you get the error
+  code=2200 [Invalid query] message="Invalid language ruby for 'foobar'"
+JRuby for Apache Cassandra has not been installed correctly.
+
+
+Ruby require/include
+--------------------
+
+You can use Ruby require and include in your scripts as in the following example:
+
+
+CREATE OR REPLACE FUNCTION foobar ( input text ) RETURNS text LANGUAGE ruby AS '
+require "bigdecimal"
+require "bigdecimal/math"
+
+include BigMath
+
+a = BigDecimal((PI(100)/2).to_s)
+
+return "foo " + a.to_s;
+' ;
+
+
+Notes / Java7 invokedynamic
+---------------------------
+
+See JRuby wiki pages https://github.com/jruby/jruby/wiki/ConfiguringJRuby and
+https://github.com/jruby/jruby/wiki/PerformanceTuning for more information and optimization tips.
diff --git a/lib/jsr223/jython/README.txt b/lib/jsr223/jython/README.txt
new file mode 100644
index 0000000..bef3c83
--- /dev/null
+++ b/lib/jsr223/jython/README.txt
@@ -0,0 +1,33 @@
+Apache Cassandra User-Defined-Functions JSR 223 scripting
+=========================================================
+
+Using JSR-223 capable Jython
+
+Tested with version 2.3.5
+
+Installation
+------------
+
+1. Download Jython binary release
+2. Unpack the downloaded archive into a temporary directory
+3. Copy the jar jython.jar from the Jython directory to $CASSANDRA_HOME/lib/jsr223/jython
+4. Restart your Cassandra daemon if it's already running
+
+Cassandra log should contain a line like this:
+  INFO  10:58:18 Found scripting engine jython 2.5.3 - python 2.5 - language names: [python, jython]
+Such a line appears when you already have scripted UDFs in your system or add a scripted UDF for the first time (see below).
+
+Smoke Test
+----------
+
+To test Jython functionality, open cqlsh and execute the following command:
+  CREATE OR REPLACE FUNCTION foobar ( input text ) RETURNS text LANGUAGE python AS '''foo''' ;
+
+If you get the error
+  code=2200 [Invalid query] message="Invalid language python for 'foobar'"
+Jython for Apache Cassandra has not been installed correctly.
+
+Notes / Java7 invokedynamic
+---------------------------
+
+Jython currently targets Java6 only. They want to switch to Java7 + invokedynamic in Jython 3.
diff --git a/lib/jsr223/scala/README.txt b/lib/jsr223/scala/README.txt
new file mode 100644
index 0000000..7f5d6fe
--- /dev/null
+++ b/lib/jsr223/scala/README.txt
@@ -0,0 +1,37 @@
+Apache Cassandra User-Defined-Functions JSR 223 scripting
+=========================================================
+
+Using JSR-223 capable Scala
+
+Tested with version 2.11.2
+
+Installation
+------------
+
+1. Download Scala binary release
+2. Unpack the downloaded archive into a temporary directory
+3. Copy the following jars from the Scala lib directory to $CASSANDRA_HOME/lib/jsr223/scala
+   scala-compiler.jar
+   scala-library.jar
+   scala-reflect.jar
+4. Restart your Cassandra daemon if it's already running
+
+Cassandra log should contain a line like this:
+  INFO  11:42:35 Found scripting engine Scala Interpreter 1.0 - Scala version 2.11.2 - language names: [scala]
+Such a line appears when you already have scripted UDFs in your system or add a scripted UDF for the first time (see below).
+
+Smoke Test
+----------
+
+To test Scala functionality, open cqlsh and execute the following command:
+  CREATE OR REPLACE FUNCTION foobar ( input text ) RETURNS text LANGUAGE scala AS 'return "foo";' ;
+
+If you get the error
+  code=2200 [Invalid query] message="Invalid language scala for 'foobar'"
+Scala for Apache Cassandra has not been installed correctly.
+
+Notes / Java7 invokedynamic
+---------------------------
+
+Scala 2.10 has Java6 support only. 2.11 has experimental invokedynamic support (use at your own risk!).
+2.12 introduces an upgrade directly to Java8 - see https://stackoverflow.com/questions/14285894/advantages-of-scala-emitting-bytecode-for-the-jvm-1-7
\ No newline at end of file
diff --git a/lib/licenses/crc32ex-0.1.1.txt b/lib/licenses/crc32ex-0.1.1.txt
new file mode 100644
index 0000000..d645695
--- /dev/null
+++ b/lib/licenses/crc32ex-0.1.1.txt
@@ -0,0 +1,202 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed 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.
diff --git a/lib/licenses/ecj-4.4.2.txt b/lib/licenses/ecj-4.4.2.txt
new file mode 100644
index 0000000..6965e5e
--- /dev/null
+++ b/lib/licenses/ecj-4.4.2.txt
@@ -0,0 +1,210 @@
+Eclipse Public License - v 1.0
+
+THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE PUBLIC
+LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION OF THE PROGRAM
+CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT.
+
+1. DEFINITIONS
+
+"Contribution" means:
+
+a) in the case of the initial Contributor, the initial code and documentation
+distributed under this Agreement, and
+
+b) in the case of each subsequent Contributor:
+
+i) changes to the Program, and
+
+ii) additions to the Program;
+
+where such changes and/or additions to the Program originate from and are
+distributed by that particular Contributor. A Contribution 'originates' from a
+Contributor if it was added to the Program by such Contributor itself or anyone
+acting on such Contributor's behalf. Contributions do not include additions to
+the Program which: (i) are separate modules of software distributed in
+conjunction with the Program under their own license agreement, and (ii) are not
+derivative works of the Program.
+
+"Contributor" means any person or entity that distributes the Program.
+
+"Licensed Patents" mean patent claims licensable by a Contributor which are
+necessarily infringed by the use or sale of its Contribution alone or when
+combined with the Program.
+
+"Program" means the Contributions distributed in accordance with this Agreement.
+
+"Recipient" means anyone who receives the Program under this Agreement,
+including all Contributors.
+
+2. GRANT OF RIGHTS
+
+a) Subject to the terms of this Agreement, each Contributor hereby grants
+Recipient a non-exclusive, worldwide, royalty-free copyright license to
+reproduce, prepare derivative works of, publicly display, publicly perform,
+distribute and sublicense the Contribution of such Contributor, if any, and such
+derivative works, in source code and object code form.
+
+b) Subject to the terms of this Agreement, each Contributor hereby grants
+Recipient a non-exclusive, worldwide, royalty-free patent license under Licensed
+Patents to make, use, sell, offer to sell, import and otherwise transfer the
+Contribution of such Contributor, if any, in source code and object code form.
+This patent license shall apply to the combination of the Contribution and the
+Program if, at the time the Contribution is added by the Contributor, such
+addition of the Contribution causes such combination to be covered by the
+Licensed Patents. The patent license shall not apply to any other combinations
+which include the Contribution. No hardware per se is licensed hereunder.
+
+c) Recipient understands that although each Contributor grants the licenses to
+its Contributions set forth herein, no assurances are provided by any
+Contributor that the Program does not infringe the patent or other intellectual
+property rights of any other entity. Each Contributor disclaims any liability to
+Recipient for claims brought by any other entity based on infringement of
+intellectual property rights or otherwise. As a condition to exercising the
+rights and licenses granted hereunder, each Recipient hereby assumes sole
+responsibility to secure any other intellectual property rights needed, if any.
+For example, if a third party patent license is required to allow Recipient to
+distribute the Program, it is Recipient's responsibility to acquire that license
+before distributing the Program.
+
+d) Each Contributor represents that to its knowledge it has sufficient copyright
+rights in its Contribution, if any, to grant the copyright license set forth in
+this Agreement.
+
+3. REQUIREMENTS
+
+A Contributor may choose to distribute the Program in object code form under its
+own license agreement, provided that:
+
+a) it complies with the terms and conditions of this Agreement; and
+
+b) its license agreement:
+
+i) effectively disclaims on behalf of all Contributors all warranties and
+conditions, express and implied, including warranties or conditions of title and
+non-infringement, and implied warranties or conditions of merchantability and
+fitness for a particular purpose;
+
+ii) effectively excludes on behalf of all Contributors all liability for
+damages, including direct, indirect, special, incidental and consequential
+damages, such as lost profits;
+
+iii) states that any provisions which differ from this Agreement are offered by
+that Contributor alone and not by any other party; and
+
+iv) states that source code for the Program is available from such Contributor,
+and informs licensees how to obtain it in a reasonable manner on or through a
+medium customarily used for software exchange.
+
+When the Program is made available in source code form:
+
+a) it must be made available under this Agreement; and
+
+b) a copy of this Agreement must be included with each copy of the Program.
+
+Contributors may not remove or alter any copyright notices contained within the
+Program.
+
+Each Contributor must identify itself as the originator of its Contribution, if
+any, in a manner that reasonably allows subsequent Recipients to identify the
+originator of the Contribution.
+
+4. COMMERCIAL DISTRIBUTION
+
+Commercial distributors of software may accept certain responsibilities with
+respect to end users, business partners and the like. While this license is
+intended to facilitate the commercial use of the Program, the Contributor who
+includes the Program in a commercial product offering should do so in a manner
+which does not create potential liability for other Contributors. Therefore, if
+a Contributor includes the Program in a commercial product offering, such
+Contributor ("Commercial Contributor") hereby agrees to defend and indemnify
+every other Contributor ("Indemnified Contributor") against any losses, damages
+and costs (collectively "Losses") arising from claims, lawsuits and other legal
+actions brought by a third party against the Indemnified Contributor to the
+extent caused by the acts or omissions of such Commercial Contributor in
+connection with its distribution of the Program in a commercial product
+offering. The obligations in this section do not apply to any claims or Losses
+relating to any actual or alleged intellectual property infringement. In order
+to qualify, an Indemnified Contributor must: a) promptly notify the Commercial
+Contributor in writing of such claim, and b) allow the Commercial Contributor
+to control, and cooperate with the Commercial Contributor in, the defense and
+any related settlement negotiations. The Indemnified Contributor may
+participate in any such claim at its own expense.
+
+For example, a Contributor might include the Program in a commercial product
+offering, Product X. That Contributor is then a Commercial Contributor. If that
+Commercial Contributor then makes performance claims, or offers warranties
+related to Product X, those performance claims and warranties are such
+Commercial Contributor's responsibility alone. Under this section, the
+Commercial Contributor would have to defend claims against the other
+Contributors related to those performance claims and warranties, and if a court
+requires any other Contributor to pay any damages as a result, the Commercial
+Contributor must pay those damages.
+
+5. NO WARRANTY
+
+EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, THE PROGRAM IS PROVIDED ON AN
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, EITHER EXPRESS OR
+IMPLIED INCLUDING, WITHOUT LIMITATION, ANY WARRANTIES OR CONDITIONS OF TITLE,
+NON-INFRINGEMENT, MERCHANTABILITY OR FITNESS FOR A PARTICULAR PURPOSE. Each
+Recipient is solely responsible for determining the appropriateness of using and
+distributing the Program and assumes all risks associated with its exercise of
+rights under this Agreement , including but not limited to the risks and costs
+of program errors, compliance with applicable laws, damage to or loss of data,
+programs or equipment, and unavailability or interruption of operations.
+
+6. DISCLAIMER OF LIABILITY
+
+EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, NEITHER RECIPIENT NOR ANY
+CONTRIBUTORS SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING WITHOUT LIMITATION LOST
+PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT,
+STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY
+OUT OF THE USE OR DISTRIBUTION OF THE PROGRAM OR THE EXERCISE OF ANY RIGHTS
+GRANTED HEREUNDER, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGES.
+
+7. GENERAL
+
+If any provision of this Agreement is invalid or unenforceable under applicable
+law, it shall not affect the validity or enforceability of the remainder of the
+terms of this Agreement, and without further action by the parties hereto, such
+provision shall be reformed to the minimum extent necessary to make such
+provision valid and enforceable.
+
+If Recipient institutes patent litigation against any entity (including a
+cross-claim or counterclaim in a lawsuit) alleging that the Program itself
+(excluding combinations of the Program with other software or hardware)
+infringes such Recipient's patent(s), then such Recipient's rights granted under
+Section 2(b) shall terminate as of the date such litigation is filed.
+
+All Recipient's rights under this Agreement shall terminate if it fails to
+comply with any of the material terms or conditions of this Agreement and does
+not cure such failure in a reasonable period of time after becoming aware of
+such noncompliance. If all Recipient's rights under this Agreement terminate,
+Recipient agrees to cease use and distribution of the Program as soon as
+reasonably practicable. However, Recipient's obligations under this Agreement
+and any licenses granted by Recipient relating to the Program shall continue and
+survive.
+
+Everyone is permitted to copy and distribute copies of this Agreement, but in
+order to avoid inconsistency the Agreement is copyrighted and may only be
+modified in the following manner. The Agreement Steward reserves the right to
+publish new versions (including revisions) of this Agreement from time to time.
+No one other than the Agreement Steward has the right to modify this Agreement.
+The Eclipse Foundation is the initial Agreement Steward. The Eclipse Foundation
+may assign the responsibility to serve as the Agreement Steward to a suitable
+separate entity. Each new version of the Agreement will be given a
+distinguishing version number. The Program (including Contributions) may always
+be distributed subject to the version of the Agreement under which it was
+received. In addition, after a new version of the Agreement is published,
+Contributor may elect to distribute the Program (including its Contributions)
+under the new version. Except as expressly stated in Sections 2(a) and 2(b)
+above, Recipient receives no rights or licenses to the intellectual property of
+any Contributor under this Agreement, whether expressly, by implication,
+estoppel or otherwise. All rights in the Program not expressly granted under
+this Agreement are reserved.
+
+This Agreement is governed by the laws of the State of New York and the
+intellectual property laws of the United States of America. No party to this
+Agreement will bring a legal action under this Agreement more than one year
+after the cause of action arose. Each party waives its rights to a jury trial in
+any resulting litigation.
\ No newline at end of file
diff --git a/lib/licenses/slf4j-api-1.7.2.txt b/lib/licenses/jcl-over-slf4j-1.7.7.txt
similarity index 100%
copy from lib/licenses/slf4j-api-1.7.2.txt
copy to lib/licenses/jcl-over-slf4j-1.7.7.txt
diff --git a/lib/licenses/jline-1.0.txt b/lib/licenses/jline-1.0.txt
deleted file mode 100644
index 1cdc44c..0000000
--- a/lib/licenses/jline-1.0.txt
+++ /dev/null
@@ -1,33 +0,0 @@
-Copyright (c) 2002-2006, Marc Prud'hommeaux <mwp1@cornell.edu>
-All rights reserved.
-
-Redistribution and use in source and binary forms, with or
-without modification, are permitted provided that the following
-conditions are met:
-
-Redistributions of source code must retain the above copyright
-notice, this list of conditions and the following disclaimer.
-
-Redistributions in binary form must reproduce the above copyright
-notice, this list of conditions and the following disclaimer
-in the documentation and/or other materials provided with
-the distribution.
-
-Neither the name of JLine nor the names of its contributors
-may be used to endorse or promote products derived from this
-software without specific prior written permission.
-
-THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
-"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING,
-BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY
-AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO
-EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE
-FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY,
-OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
-PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
-DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED
-AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT
-LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING
-IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED
-OF THE POSSIBILITY OF SUCH DAMAGE.
-
diff --git a/lib/licenses/joda-time-2.4.txt b/lib/licenses/joda-time-2.4.txt
new file mode 100644
index 0000000..261eeb9
--- /dev/null
+++ b/lib/licenses/joda-time-2.4.txt
@@ -0,0 +1,201 @@
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed 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.
diff --git a/lib/licenses/slf4j-api-1.7.2.txt b/lib/licenses/log4j-over-slf4j-1.7.7.txt
similarity index 100%
copy from lib/licenses/slf4j-api-1.7.2.txt
copy to lib/licenses/log4j-over-slf4j-1.7.7.txt
diff --git a/lib/licenses/logback-classic-1.1.2.txt b/lib/licenses/logback-classic-1.1.3.txt
similarity index 100%
rename from lib/licenses/logback-classic-1.1.2.txt
rename to lib/licenses/logback-classic-1.1.3.txt
diff --git a/lib/licenses/logback-core-1.1.2.txt b/lib/licenses/logback-core-1.1.3.txt
similarity index 100%
rename from lib/licenses/logback-core-1.1.2.txt
rename to lib/licenses/logback-core-1.1.3.txt
diff --git a/lib/licenses/lz4-1.2.0.txt b/lib/licenses/lz4-1.3.0.txt
similarity index 100%
rename from lib/licenses/lz4-1.2.0.txt
rename to lib/licenses/lz4-1.3.0.txt
diff --git a/lib/licenses/metrics-core-2.2.0.txt b/lib/licenses/metrics-core-3.1.0.txt
similarity index 100%
rename from lib/licenses/metrics-core-2.2.0.txt
rename to lib/licenses/metrics-core-3.1.0.txt
diff --git a/lib/licenses/metrics-core-2.2.0.txt b/lib/licenses/metrics-jvm-3.1.0.txt
similarity index 100%
copy from lib/licenses/metrics-core-2.2.0.txt
copy to lib/licenses/metrics-jvm-3.1.0.txt
diff --git a/lib/licenses/metrics-core-2.2.0.txt b/lib/licenses/metrics-logback-3.1.0.txt
similarity index 100%
copy from lib/licenses/metrics-core-2.2.0.txt
copy to lib/licenses/metrics-logback-3.1.0.txt
diff --git a/lib/licenses/ohc-0.3.4.txt b/lib/licenses/ohc-0.3.4.txt
new file mode 100644
index 0000000..eb6b5d3
--- /dev/null
+++ b/lib/licenses/ohc-0.3.4.txt
@@ -0,0 +1,201 @@
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright 2014 Robert Stupp, Koeln, Germany, robert-stupp.de
+
+   Licensed 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.
diff --git a/lib/licenses/reporter-config-2.1.0.txt b/lib/licenses/reporter-config-base-3.0.0.txt
similarity index 100%
copy from lib/licenses/reporter-config-2.1.0.txt
copy to lib/licenses/reporter-config-base-3.0.0.txt
diff --git a/lib/licenses/reporter-config-2.1.0.txt b/lib/licenses/reporter-config3-3.0.0.txt
similarity index 100%
rename from lib/licenses/reporter-config-2.1.0.txt
rename to lib/licenses/reporter-config3-3.0.0.txt
diff --git a/lib/licenses/sigar-1.6.4.txt b/lib/licenses/sigar-1.6.4.txt
new file mode 100644
index 0000000..11069ed
--- /dev/null
+++ b/lib/licenses/sigar-1.6.4.txt
@@ -0,0 +1,201 @@
+                              Apache License
+                        Version 2.0, January 2004
+                     http://www.apache.org/licenses/
+
+TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+1. Definitions.
+
+   "License" shall mean the terms and conditions for use, reproduction,
+   and distribution as defined by Sections 1 through 9 of this document.
+
+   "Licensor" shall mean the copyright owner or entity authorized by
+   the copyright owner that is granting the License.
+
+   "Legal Entity" shall mean the union of the acting entity and all
+   other entities that control, are controlled by, or are under common
+   control with that entity. For the purposes of this definition,
+   "control" means (i) the power, direct or indirect, to cause the
+   direction or management of such entity, whether by contract or
+   otherwise, or (ii) ownership of fifty percent (50%) or more of the
+   outstanding shares, or (iii) beneficial ownership of such entity.
+
+   "You" (or "Your") shall mean an individual or Legal Entity
+   exercising permissions granted by this License.
+
+   "Source" form shall mean the preferred form for making modifications,
+   including but not limited to software source code, documentation
+   source, and configuration files.
+
+   "Object" form shall mean any form resulting from mechanical
+   transformation or translation of a Source form, including but
+   not limited to compiled object code, generated documentation,
+   and conversions to other media types.
+
+   "Work" shall mean the work of authorship, whether in Source or
+   Object form, made available under the License, as indicated by a
+   copyright notice that is included in or attached to the work
+   (an example is provided in the Appendix below).
+
+   "Derivative Works" shall mean any work, whether in Source or Object
+   form, that is based on (or derived from) the Work and for which the
+   editorial revisions, annotations, elaborations, or other modifications
+   represent, as a whole, an original work of authorship. For the purposes
+   of this License, Derivative Works shall not include works that remain
+   separable from, or merely link (or bind by name) to the interfaces of,
+   the Work and Derivative Works thereof.
+
+   "Contribution" shall mean any work of authorship, including
+   the original version of the Work and any modifications or additions
+   to that Work or Derivative Works thereof, that is intentionally
+   submitted to Licensor for inclusion in the Work by the copyright owner
+   or by an individual or Legal Entity authorized to submit on behalf of
+   the copyright owner. For the purposes of this definition, "submitted"
+   means any form of electronic, verbal, or written communication sent
+   to the Licensor or its representatives, including but not limited to
+   communication on electronic mailing lists, source code control systems,
+   and issue tracking systems that are managed by, or on behalf of, the
+   Licensor for the purpose of discussing and improving the Work, but
+   excluding communication that is conspicuously marked or otherwise
+   designated in writing by the copyright owner as "Not a Contribution."
+
+   "Contributor" shall mean Licensor and any individual or Legal Entity
+   on behalf of whom a Contribution has been received by Licensor and
+   subsequently incorporated within the Work.
+
+2. Grant of Copyright License. Subject to the terms and conditions of
+   this License, each Contributor hereby grants to You a perpetual,
+   worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+   copyright license to reproduce, prepare Derivative Works of,
+   publicly display, publicly perform, sublicense, and distribute the
+   Work and such Derivative Works in Source or Object form.
+
+3. Grant of Patent License. Subject to the terms and conditions of
+   this License, each Contributor hereby grants to You a perpetual,
+   worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+   (except as stated in this section) patent license to make, have made,
+   use, offer to sell, sell, import, and otherwise transfer the Work,
+   where such license applies only to those patent claims licensable
+   by such Contributor that are necessarily infringed by their
+   Contribution(s) alone or by combination of their Contribution(s)
+   with the Work to which such Contribution(s) was submitted. If You
+   institute patent litigation against any entity (including a
+   cross-claim or counterclaim in a lawsuit) alleging that the Work
+   or a Contribution incorporated within the Work constitutes direct
+   or contributory patent infringement, then any patent licenses
+   granted to You under this License for that Work shall terminate
+   as of the date such litigation is filed.
+
+4. Redistribution. You may reproduce and distribute copies of the
+   Work or Derivative Works thereof in any medium, with or without
+   modifications, and in Source or Object form, provided that You
+   meet the following conditions:
+
+   (a) You must give any other recipients of the Work or
+       Derivative Works a copy of this License; and
+
+   (b) You must cause any modified files to carry prominent notices
+       stating that You changed the files; and
+
+   (c) You must retain, in the Source form of any Derivative Works
+       that You distribute, all copyright, patent, trademark, and
+       attribution notices from the Source form of the Work,
+       excluding those notices that do not pertain to any part of
+       the Derivative Works; and
+
+   (d) If the Work includes a "NOTICE" text file as part of its
+       distribution, then any Derivative Works that You distribute must
+       include a readable copy of the attribution notices contained
+       within such NOTICE file, excluding those notices that do not
+       pertain to any part of the Derivative Works, in at least one
+       of the following places: within a NOTICE text file distributed
+       as part of the Derivative Works; within the Source form or
+       documentation, if provided along with the Derivative Works; or,
+       within a display generated by the Derivative Works, if and
+       wherever such third-party notices normally appear. The contents
+       of the NOTICE file are for informational purposes only and
+       do not modify the License. You may add Your own attribution
+       notices within Derivative Works that You distribute, alongside
+       or as an addendum to the NOTICE text from the Work, provided
+       that such additional attribution notices cannot be construed
+       as modifying the License.
+
+   You may add Your own copyright statement to Your modifications and
+   may provide additional or different license terms and conditions
+   for use, reproduction, or distribution of Your modifications, or
+   for any such Derivative Works as a whole, provided Your use,
+   reproduction, and distribution of the Work otherwise complies with
+   the conditions stated in this License.
+
+5. Submission of Contributions. Unless You explicitly state otherwise,
+   any Contribution intentionally submitted for inclusion in the Work
+   by You to the Licensor shall be under the terms and conditions of
+   this License, without any additional terms or conditions.
+   Notwithstanding the above, nothing herein shall supersede or modify
+   the terms of any separate license agreement you may have executed
+   with Licensor regarding such Contributions.
+
+6. Trademarks. This License does not grant permission to use the trade
+   names, trademarks, service marks, or product names of the Licensor,
+   except as required for reasonable and customary use in describing the
+   origin of the Work and reproducing the content of the NOTICE file.
+
+7. Disclaimer of Warranty. Unless required by applicable law or
+   agreed to in writing, Licensor provides the Work (and each
+   Contributor provides its Contributions) on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+   implied, including, without limitation, any warranties or conditions
+   of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+   PARTICULAR PURPOSE. You are solely responsible for determining the
+   appropriateness of using or redistributing the Work and assume any
+   risks associated with Your exercise of permissions under this License.
+
+8. Limitation of Liability. In no event and under no legal theory,
+   whether in tort (including negligence), contract, or otherwise,
+   unless required by applicable law (such as deliberate and grossly
+   negligent acts) or agreed to in writing, shall any Contributor be
+   liable to You for damages, including any direct, indirect, special,
+   incidental, or consequential damages of any character arising as a
+   result of this License or out of the use or inability to use the
+   Work (including but not limited to damages for loss of goodwill,
+   work stoppage, computer failure or malfunction, or any and all
+   other commercial damages or losses), even if such Contributor
+   has been advised of the possibility of such damages.
+
+9. Accepting Warranty or Additional Liability. While redistributing
+   the Work or Derivative Works thereof, You may choose to offer,
+   and charge a fee for, acceptance of support, warranty, indemnity,
+   or other liability obligations and/or rights consistent with this
+   License. However, in accepting such obligations, You may act only
+   on Your own behalf and on Your sole responsibility, not on behalf
+   of any other Contributor, and only if You agree to indemnify,
+   defend, and hold each Contributor harmless for any liability
+   incurred by, or claims asserted against, such Contributor by reason
+   of your accepting any such warranty or additional liability.
+
+END OF TERMS AND CONDITIONS
+
+APPENDIX: How to apply the Apache License to your work.
+
+   To apply the Apache License to your work, attach the following
+   boilerplate notice, with the fields enclosed by brackets "[]"
+   replaced with your own identifying information. (Don't include
+   the brackets!)  The text should be enclosed in the appropriate
+   comment syntax for the file format. We also recommend that a
+   file or class name and description of purpose be included on the
+   same "printed page" as the copyright notice for easier
+   identification within third-party archives.
+
+Copyright [yyyy] [name of copyright owner]
+
+Licensed 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.
diff --git a/lib/licenses/slf4j-api-1.7.2.txt b/lib/licenses/slf4j-api-1.7.7.txt
similarity index 100%
rename from lib/licenses/slf4j-api-1.7.2.txt
rename to lib/licenses/slf4j-api-1.7.7.txt
diff --git a/lib/licenses/snappy-java-1.0.5.2.txt b/lib/licenses/snappy-java-1.1.1.7.txt
similarity index 100%
rename from lib/licenses/snappy-java-1.0.5.2.txt
rename to lib/licenses/snappy-java-1.1.1.7.txt
diff --git a/lib/log4j-over-slf4j-1.7.7.jar b/lib/log4j-over-slf4j-1.7.7.jar
new file mode 100644
index 0000000..d2a102e
--- /dev/null
+++ b/lib/log4j-over-slf4j-1.7.7.jar
Binary files differ
diff --git a/lib/logback-classic-1.1.2.jar b/lib/logback-classic-1.1.2.jar
deleted file mode 100644
index 9230b2a..0000000
--- a/lib/logback-classic-1.1.2.jar
+++ /dev/null
Binary files differ
diff --git a/lib/logback-classic-1.1.3.jar b/lib/logback-classic-1.1.3.jar
new file mode 100644
index 0000000..2aa10a5
--- /dev/null
+++ b/lib/logback-classic-1.1.3.jar
Binary files differ
diff --git a/lib/logback-core-1.1.2.jar b/lib/logback-core-1.1.2.jar
deleted file mode 100644
index 391da64..0000000
--- a/lib/logback-core-1.1.2.jar
+++ /dev/null
Binary files differ
diff --git a/lib/logback-core-1.1.3.jar b/lib/logback-core-1.1.3.jar
new file mode 100644
index 0000000..996b722
--- /dev/null
+++ b/lib/logback-core-1.1.3.jar
Binary files differ
diff --git a/lib/lz4-1.2.0.jar b/lib/lz4-1.2.0.jar
deleted file mode 100644
index 9cdebd2..0000000
--- a/lib/lz4-1.2.0.jar
+++ /dev/null
Binary files differ
diff --git a/lib/lz4-1.3.0.jar b/lib/lz4-1.3.0.jar
new file mode 100644
index 0000000..0fb0109
--- /dev/null
+++ b/lib/lz4-1.3.0.jar
Binary files differ
diff --git a/lib/metrics-core-2.2.0.jar b/lib/metrics-core-2.2.0.jar
deleted file mode 100644
index 0f6d1cb..0000000
--- a/lib/metrics-core-2.2.0.jar
+++ /dev/null
Binary files differ
diff --git a/lib/metrics-core-3.1.0.jar b/lib/metrics-core-3.1.0.jar
new file mode 100644
index 0000000..564ffb7
--- /dev/null
+++ b/lib/metrics-core-3.1.0.jar
Binary files differ
diff --git a/lib/metrics-jvm-3.1.0.jar b/lib/metrics-jvm-3.1.0.jar
new file mode 100644
index 0000000..1e941f8
--- /dev/null
+++ b/lib/metrics-jvm-3.1.0.jar
Binary files differ
diff --git a/lib/metrics-logback-3.1.0.jar b/lib/metrics-logback-3.1.0.jar
new file mode 100644
index 0000000..ce0bfbe
--- /dev/null
+++ b/lib/metrics-logback-3.1.0.jar
Binary files differ
diff --git a/lib/ohc-core-0.3.4.jar b/lib/ohc-core-0.3.4.jar
new file mode 100644
index 0000000..0773e78
--- /dev/null
+++ b/lib/ohc-core-0.3.4.jar
Binary files differ
diff --git a/lib/ohc-core-j8-0.3.4.jar b/lib/ohc-core-j8-0.3.4.jar
new file mode 100644
index 0000000..faa102f
--- /dev/null
+++ b/lib/ohc-core-j8-0.3.4.jar
Binary files differ
diff --git a/lib/reporter-config-2.1.0.jar b/lib/reporter-config-2.1.0.jar
deleted file mode 100644
index 0e13098..0000000
--- a/lib/reporter-config-2.1.0.jar
+++ /dev/null
Binary files differ
diff --git a/lib/reporter-config-base-3.0.0.jar b/lib/reporter-config-base-3.0.0.jar
new file mode 100644
index 0000000..4250bb9
--- /dev/null
+++ b/lib/reporter-config-base-3.0.0.jar
Binary files differ
diff --git a/lib/reporter-config3-3.0.0.jar b/lib/reporter-config3-3.0.0.jar
new file mode 100644
index 0000000..0cd9852
--- /dev/null
+++ b/lib/reporter-config3-3.0.0.jar
Binary files differ
diff --git a/lib/sigar-1.6.4.jar b/lib/sigar-1.6.4.jar
new file mode 100644
index 0000000..58c733c
--- /dev/null
+++ b/lib/sigar-1.6.4.jar
Binary files differ
diff --git a/lib/sigar-bin/libsigar-amd64-freebsd-6.so b/lib/sigar-bin/libsigar-amd64-freebsd-6.so
new file mode 100644
index 0000000..3e94f0d
--- /dev/null
+++ b/lib/sigar-bin/libsigar-amd64-freebsd-6.so
Binary files differ
diff --git a/lib/sigar-bin/libsigar-amd64-linux.so b/lib/sigar-bin/libsigar-amd64-linux.so
new file mode 100644
index 0000000..5a2e4c2
--- /dev/null
+++ b/lib/sigar-bin/libsigar-amd64-linux.so
Binary files differ
diff --git a/lib/sigar-bin/libsigar-amd64-solaris.so b/lib/sigar-bin/libsigar-amd64-solaris.so
new file mode 100644
index 0000000..6396482
--- /dev/null
+++ b/lib/sigar-bin/libsigar-amd64-solaris.so
Binary files differ
diff --git a/lib/sigar-bin/libsigar-ia64-hpux-11.sl b/lib/sigar-bin/libsigar-ia64-hpux-11.sl
new file mode 100644
index 0000000..d92ea4a
--- /dev/null
+++ b/lib/sigar-bin/libsigar-ia64-hpux-11.sl
Binary files differ
diff --git a/lib/sigar-bin/libsigar-ia64-linux.so b/lib/sigar-bin/libsigar-ia64-linux.so
new file mode 100644
index 0000000..2bd2fc8
--- /dev/null
+++ b/lib/sigar-bin/libsigar-ia64-linux.so
Binary files differ
diff --git a/lib/sigar-bin/libsigar-pa-hpux-11.sl b/lib/sigar-bin/libsigar-pa-hpux-11.sl
new file mode 100644
index 0000000..0dfd8a1
--- /dev/null
+++ b/lib/sigar-bin/libsigar-pa-hpux-11.sl
Binary files differ
diff --git a/lib/sigar-bin/libsigar-ppc-aix-5.so b/lib/sigar-bin/libsigar-ppc-aix-5.so
new file mode 100644
index 0000000..7d4b519
--- /dev/null
+++ b/lib/sigar-bin/libsigar-ppc-aix-5.so
Binary files differ
diff --git a/lib/sigar-bin/libsigar-ppc-linux.so b/lib/sigar-bin/libsigar-ppc-linux.so
new file mode 100644
index 0000000..4394b1b
--- /dev/null
+++ b/lib/sigar-bin/libsigar-ppc-linux.so
Binary files differ
diff --git a/lib/sigar-bin/libsigar-ppc64-aix-5.so b/lib/sigar-bin/libsigar-ppc64-aix-5.so
new file mode 100644
index 0000000..35fd828
--- /dev/null
+++ b/lib/sigar-bin/libsigar-ppc64-aix-5.so
Binary files differ
diff --git a/lib/sigar-bin/libsigar-ppc64-linux.so b/lib/sigar-bin/libsigar-ppc64-linux.so
new file mode 100644
index 0000000..a1ba252
--- /dev/null
+++ b/lib/sigar-bin/libsigar-ppc64-linux.so
Binary files differ
diff --git a/lib/sigar-bin/libsigar-s390x-linux.so b/lib/sigar-bin/libsigar-s390x-linux.so
new file mode 100644
index 0000000..c275f4a
--- /dev/null
+++ b/lib/sigar-bin/libsigar-s390x-linux.so
Binary files differ
diff --git a/lib/sigar-bin/libsigar-sparc-solaris.so b/lib/sigar-bin/libsigar-sparc-solaris.so
new file mode 100644
index 0000000..aa847d2
--- /dev/null
+++ b/lib/sigar-bin/libsigar-sparc-solaris.so
Binary files differ
diff --git a/lib/sigar-bin/libsigar-sparc64-solaris.so b/lib/sigar-bin/libsigar-sparc64-solaris.so
new file mode 100644
index 0000000..6c4fe80
--- /dev/null
+++ b/lib/sigar-bin/libsigar-sparc64-solaris.so
Binary files differ
diff --git a/lib/sigar-bin/libsigar-universal-macosx.dylib b/lib/sigar-bin/libsigar-universal-macosx.dylib
new file mode 100644
index 0000000..27ab107
--- /dev/null
+++ b/lib/sigar-bin/libsigar-universal-macosx.dylib
Binary files differ
diff --git a/lib/sigar-bin/libsigar-universal64-macosx.dylib b/lib/sigar-bin/libsigar-universal64-macosx.dylib
new file mode 100644
index 0000000..0c721fe
--- /dev/null
+++ b/lib/sigar-bin/libsigar-universal64-macosx.dylib
Binary files differ
diff --git a/lib/sigar-bin/libsigar-x86-freebsd-5.so b/lib/sigar-bin/libsigar-x86-freebsd-5.so
new file mode 100644
index 0000000..8c50c61
--- /dev/null
+++ b/lib/sigar-bin/libsigar-x86-freebsd-5.so
Binary files differ
diff --git a/lib/sigar-bin/libsigar-x86-freebsd-6.so b/lib/sigar-bin/libsigar-x86-freebsd-6.so
new file mode 100644
index 0000000..f080027
--- /dev/null
+++ b/lib/sigar-bin/libsigar-x86-freebsd-6.so
Binary files differ
diff --git a/lib/sigar-bin/libsigar-x86-linux.so b/lib/sigar-bin/libsigar-x86-linux.so
new file mode 100644
index 0000000..a0b64ed
--- /dev/null
+++ b/lib/sigar-bin/libsigar-x86-linux.so
Binary files differ
diff --git a/lib/sigar-bin/libsigar-x86-solaris.so b/lib/sigar-bin/libsigar-x86-solaris.so
new file mode 100644
index 0000000..c6452e5
--- /dev/null
+++ b/lib/sigar-bin/libsigar-x86-solaris.so
Binary files differ
diff --git a/lib/sigar-bin/sigar-amd64-winnt.dll b/lib/sigar-bin/sigar-amd64-winnt.dll
new file mode 100644
index 0000000..1ec8a03
--- /dev/null
+++ b/lib/sigar-bin/sigar-amd64-winnt.dll
Binary files differ
diff --git a/lib/sigar-bin/sigar-x86-winnt.dll b/lib/sigar-bin/sigar-x86-winnt.dll
new file mode 100644
index 0000000..6afdc01
--- /dev/null
+++ b/lib/sigar-bin/sigar-x86-winnt.dll
Binary files differ
diff --git a/lib/sigar-bin/sigar-x86-winnt.lib b/lib/sigar-bin/sigar-x86-winnt.lib
new file mode 100644
index 0000000..04924a1
--- /dev/null
+++ b/lib/sigar-bin/sigar-x86-winnt.lib
Binary files differ
diff --git a/lib/slf4j-api-1.7.2.jar b/lib/slf4j-api-1.7.2.jar
deleted file mode 100644
index 73f38db..0000000
--- a/lib/slf4j-api-1.7.2.jar
+++ /dev/null
Binary files differ
diff --git a/lib/slf4j-api-1.7.7.jar b/lib/slf4j-api-1.7.7.jar
new file mode 100644
index 0000000..bebabd9
--- /dev/null
+++ b/lib/slf4j-api-1.7.7.jar
Binary files differ
diff --git a/lib/snappy-java-1.0.5.2.jar b/lib/snappy-java-1.0.5.2.jar
deleted file mode 100644
index e0efe83..0000000
--- a/lib/snappy-java-1.0.5.2.jar
+++ /dev/null
Binary files differ
diff --git a/lib/snappy-java-1.1.1.7.jar b/lib/snappy-java-1.1.1.7.jar
new file mode 100644
index 0000000..2bbd1fc
--- /dev/null
+++ b/lib/snappy-java-1.1.1.7.jar
Binary files differ
diff --git a/pylib/cqlshlib/copyutil.py b/pylib/cqlshlib/copyutil.py
index 3a45353..b72b517 100644
--- a/pylib/cqlshlib/copyutil.py
+++ b/pylib/cqlshlib/copyutil.py
@@ -52,7 +52,7 @@
 
 from cql3handling import CqlRuleSet
 from displaying import NO_COLOR_MAP
-from formatting import format_value_default, EMPTY, get_formatter
+from formatting import format_value_default, DateTimeFormat, EMPTY, get_formatter
 from sslhandling import ssl_settings
 
 PROFILE_ON = False
@@ -77,12 +77,18 @@
         printmsg(msg)
 
 
-def printmsg(msg, eol='\n'):
-    sys.stdout.write(msg + eol)
+def printmsg(msg, eol='\n', encoding='utf8'):
+    sys.stdout.write(msg.encode(encoding))
+    sys.stdout.write(eol)
     sys.stdout.flush()
 
 
-class OneWayChannel(object):
+# Keep arguments in sync with printmsg
+def swallowmsg(msg, eol='', encoding=''):
+    None
+
+
+class OneWayPipe(object):
     """
     A one way pipe protected by two process level locks, one for reading and one for writing.
     """
@@ -90,49 +96,11 @@
         self.reader, self.writer = mp.Pipe(duplex=False)
         self.rlock = mp.Lock()
         self.wlock = mp.Lock()
-        self.feeding_thread = None
-        self.pending_messages = None
-
-    def init_feeding_thread(self):
-        """
-        Initialize a thread that fetches messages from a queue and sends them to the channel.
-        We initialize the feeding thread lazily to avoid the fork(), since the channels are passed to child processes.
-        """
-        if self.feeding_thread is not None or self.pending_messages is not None:
-            raise RuntimeError("Feeding thread already initialized")
-
-        self.pending_messages = Queue()
-
-        def feed():
-            send = self._send
-            pending_messages = self.pending_messages
-
-            while True:
-                try:
-                    msg = pending_messages.get()
-                    send(msg)
-                except Exception, e:
-                    printmsg('%s: %s' % (e.__class__.__name__, e.message))
-
-        feeding_thread = threading.Thread(target=feed)
-        feeding_thread.setDaemon(True)
-        feeding_thread.start()
-
-        self.feeding_thread = feeding_thread
 
     def send(self, obj):
-        if self.feeding_thread is None:
-            self.init_feeding_thread()
-
-        self.pending_messages.put(obj)
-
-    def _send(self, obj):
         with self.wlock:
             self.writer.send(obj)
 
-    def num_pending(self):
-        return self.pending_messages.qsize() if self.pending_messages else 0
-
     def recv(self):
         with self.rlock:
             return self.reader.recv()
@@ -142,15 +110,77 @@
         self.writer.close()
 
 
-class OneWayChannels(object):
+class ReceivingChannel(object):
     """
-    A group of one way channels.
+    A one way channel that wraps a pipe to receive messages.
+    """
+    def __init__(self, pipe):
+        self.pipe = pipe
+
+    def recv(self):
+        return self.pipe.recv()
+
+    def close(self):
+        self.pipe.close()
+
+
+class SendingChannel(object):
+    """
+    A one way channel that wraps a pipe and provides a feeding thread to send messages asynchronously.
+    """
+    def __init__(self, pipe):
+        self.pipe = pipe
+        self.pending_messages = Queue()
+
+        def feed():
+            while True:
+                try:
+                    msg = self.pending_messages.get()
+                    self.pipe.send(msg)
+                except Exception, e:
+                    printmsg('%s: %s' % (e.__class__.__name__, e.message))
+
+        feeding_thread = threading.Thread(target=feed)
+        feeding_thread.setDaemon(True)
+        feeding_thread.start()
+
+    def send(self, obj):
+        self.pending_messages.put(obj)
+
+    def num_pending(self):
+        return self.pending_messages.qsize() if self.pending_messages else 0
+
+    def close(self):
+        self.pipe.close()
+
+
+class SendingChannels(object):
+    """
+    A group of one way channels for sending messages.
     """
     def __init__(self, num_channels):
-        self.channels = [OneWayChannel() for _ in xrange(num_channels)]
-        self._readers = [ch.reader for ch in self.channels]
-        self._rlocks = [ch.rlock for ch in self.channels]
-        self._rlocks_by_readers = dict([(ch.reader, ch.rlock) for ch in self.channels])
+        self.pipes = [OneWayPipe() for _ in xrange(num_channels)]
+        self.channels = [SendingChannel(p) for p in self.pipes]
+        self.num_channels = num_channels
+
+    def close(self):
+        for ch in self.channels:
+            try:
+                ch.close()
+            except:
+                pass
+
+
+class ReceivingChannels(object):
+    """
+    A group of one way channels for receiving messages.
+    """
+    def __init__(self, num_channels):
+        self.pipes = [OneWayPipe() for _ in xrange(num_channels)]
+        self.channels = [ReceivingChannel(p) for p in self.pipes]
+        self._readers = [p.reader for p in self.pipes]
+        self._rlocks = [p.rlock for p in self.pipes]
+        self._rlocks_by_readers = dict([(p.reader, p.rlock) for p in self.pipes])
         self.num_channels = num_channels
 
         self.recv = self.recv_select if IS_LINUX else self.recv_polling
@@ -203,7 +233,8 @@
         self.shell = shell
         self.ks = ks
         self.table = table
-        self.local_dc = shell.conn.metadata.get_host(shell.hostname).datacenter
+        self.table_meta = self.shell.get_table_meta(self.ks, self.table)
+        self.host = shell.conn.get_control_connection_host()
         self.fname = safe_normpath(fname)
         self.protocol_version = protocol_version
         self.config_file = config_file
@@ -215,18 +246,19 @@
 
         # do not display messages when exporting to STDOUT unless --debug is set
         self.printmsg = printmsg if self.fname is not None or direction == 'from' or DEBUG \
-            else lambda _, eol='\n': None
+            else swallowmsg
         self.options = self.parse_options(opts, direction)
 
         self.num_processes = self.options.copy['numprocesses']
+        self.encoding = self.options.copy['encoding']
         self.printmsg('Using %d child processes' % (self.num_processes,))
 
         if direction == 'from':
             self.num_processes += 1  # add the feeder process
 
         self.processes = []
-        self.inmsg = OneWayChannels(self.num_processes)
-        self.outmsg = OneWayChannels(self.num_processes)
+        self.inmsg = ReceivingChannels(self.num_processes)
+        self.outmsg = SendingChannels(self.num_processes)
 
         self.columns = CopyTask.get_columns(shell, ks, table, columns)
         self.time_start = time.time()
@@ -305,7 +337,8 @@
         # in the page size or 10 seconds if pagesize is smaller
         copy_options['pagetimeout'] = int(opts.pop('pagetimeout', max(10, 10 * (copy_options['pagesize'] / 1000))))
         copy_options['maxattempts'] = int(opts.pop('maxattempts', 5))
-        copy_options['dtformats'] = opts.pop('datetimeformat', shell.display_time_format)
+        copy_options['dtformats'] = DateTimeFormat(opts.pop('datetimeformat', shell.display_timestamp_format),
+                                                   shell.display_date_format, shell.display_nanotime_format)
         copy_options['float_precision'] = shell.display_float_precision
         copy_options['chunksize'] = int(opts.pop('chunksize', 5000))
         copy_options['ingestrate'] = int(opts.pop('ingestrate', 100000))
@@ -431,11 +464,11 @@
 
         return dict(ks=self.ks,
                     table=self.table,
-                    local_dc=self.local_dc,
+                    local_dc=self.host.datacenter,
                     columns=self.columns,
                     options=self.options,
                     connect_timeout=shell.conn.connect_timeout,
-                    hostname=shell.hostname,
+                    hostname=self.host.address,
                     port=shell.port,
                     ssl=shell.ssl,
                     auth_provider=shell.auth_provider,
@@ -446,15 +479,29 @@
                     debug=shell.debug
                     )
 
+    def validate_columns(self):
+        shell = self.shell
+
+        if not self.columns:
+            shell.printerr("No column specified")
+            return False
+
+        for c in self.columns:
+            if c not in self.table_meta.columns:
+                shell.printerr('Invalid column name %s' % (c,))
+                return False
+
+        return True
+
     def update_params(self, params, i):
         """
-        Add the communication channels to the parameters to be passed to the worker process:
-            inmsg is the message queue flowing from parent to child process, so outmsg from the parent point
-            of view and, vice-versa,  outmsg is the message queue flowing from child to parent, so inmsg
+        Add the communication pipes to the parameters to be passed to the worker process:
+            inpipe is the message pipe flowing from parent to child process, so outpipe from the parent point
+            of view and, vice-versa,  outpipe is the message pipe flowing from child to parent, so inpipe
             from the parent point of view, hence the two are swapped below.
         """
-        params['inmsg'] = self.outmsg.channels[i]
-        params['outmsg'] = self.inmsg.channels[i]
+        params['inpipe'] = self.outmsg.pipes[i]
+        params['outpipe'] = self.inmsg.pipes[i]
         return params
 
 
@@ -575,8 +622,7 @@
             shell.printerr('Unrecognized COPY TO options: %s' % ', '.join(self.options.unrecognized.keys()))
             return
 
-        if not self.columns:
-            shell.printerr("No column specified")
+        if not self.validate_columns():
             return 0
 
         ranges = self.get_ranges()
@@ -586,7 +632,8 @@
         if not self.writer.open():
             return 0
 
-        self.printmsg("\nStarting copy of %s.%s with columns %s." % (self.ks, self.table, self.columns))
+        columns = u"[" + u", ".join(self.columns) + u"]"
+        self.printmsg(u"\nStarting copy of %s.%s with columns %s." % (self.ks, self.table, columns), encoding=self.encoding)
 
         params = self.make_params()
         for i in xrange(self.num_processes):
@@ -616,8 +663,8 @@
         we use the cqlsh session host.
         """
         shell = self.shell
-        hostname = shell.hostname
-        local_dc = self.local_dc
+        hostname = self.host.address
+        local_dc = self.host.datacenter
         ranges = dict()
         min_token = self.get_min_token()
         begin_token = self.begin_token
@@ -647,11 +694,11 @@
             hosts = []
             if replicas:
                 for r in replicas:
-                    if r.is_up and r.datacenter == local_dc:
+                    if r.is_up is not False and r.datacenter == local_dc:
                         hosts.append(r.address)
             if not hosts:
                 hosts.append(hostname)  # fallback to default host if no replicas in current dc
-            return {'hosts': tuple(hosts), 'attempts': 0, 'rows': 0}
+            return {'hosts': tuple(hosts), 'attempts': 0, 'rows': 0, 'workerno': -1}
 
         if begin_token and begin_token < min_token:
             shell.printerr('Begin token %d must be bigger or equal to min token %d' % (begin_token, min_token))
@@ -718,8 +765,11 @@
             return None
 
     def send_work(self, ranges, tokens_to_send):
-        i = 0
+        prev_worker_no = ranges[tokens_to_send[0]]['workerno']
+        i = prev_worker_no + 1 if -1 <= prev_worker_no < (self.num_processes - 1) else 0
+
         for token_range in tokens_to_send:
+            ranges[token_range]['workerno'] = i
             self.outmsg.channels[i].send((token_range, ranges[token_range]))
             ranges[token_range]['attempts'] += 1
 
@@ -891,8 +941,8 @@
     """
     A class for reading rows received on a pipe, this is used for reading input from STDIN
     """
-    def __init__(self, inmsg, options):
-        self.inmsg = inmsg
+    def __init__(self, inpipe, options):
+        self.inpipe = inpipe
         self.chunk_size = options.copy['chunksize']
         self.header = options.copy['header']
         self.max_rows = options.copy['maxrows']
@@ -907,7 +957,7 @@
     def read_rows(self, max_rows):
         rows = []
         for i in xrange(min(max_rows, self.chunk_size)):
-            row = self.inmsg.recv()
+            row = self.inpipe.recv()
             if row is None:
                 self.exhausted = True
                 break
@@ -1018,11 +1068,11 @@
             shell.printerr("Failed to import %d rows: %s - %s,  given up without retries"
                            % (len(err.rows), err.name, err.msg))
         else:
-            self.insert_errors += len(err.rows)
             if not err.final:
                 shell.printerr("Failed to import %d rows: %s - %s,  will retry later, attempt %d of %d"
                                % (len(err.rows), err.name, err.msg, err.attempts, self.max_attempts))
             else:
+                self.insert_errors += len(err.rows)
                 self.add_failed_rows(err.rows)
                 shell.printerr("Failed to import %d rows: %s - %s,  given up after %d attempts"
                                % (len(err.rows), err.name, err.msg, err.attempts))
@@ -1039,7 +1089,6 @@
         options = self.options
         self.skip_columns = [c.strip() for c in self.options.copy['skipcols'].split(',')]
         self.valid_columns = [c for c in self.columns if c not in self.skip_columns]
-        self.table_meta = self.shell.get_table_meta(self.ks, self.table)
         self.receive_meter = RateMeter(log_fcn=self.printmsg,
                                        update_interval=options.copy['reportfrequency'],
                                        log_file=options.copy['ratefile'])
@@ -1053,6 +1102,22 @@
         ret['valid_columns'] = self.valid_columns
         return ret
 
+    def validate_columns(self):
+        if not CopyTask.validate_columns(self):
+            return False
+
+        shell = self.shell
+        if not self.valid_columns:
+            shell.printerr("No valid column specified")
+            return False
+
+        for c in self.table_meta.primary_key:
+            if c.name not in self.valid_columns:
+                shell.printerr("Primary key column '%s' missing or skipped" % (c.name,))
+                return False
+
+        return True
+
     def run(self):
         shell = self.shell
 
@@ -1060,16 +1125,11 @@
             shell.printerr('Unrecognized COPY FROM options: %s' % ', '.join(self.options.unrecognized.keys()))
             return
 
-        if not self.valid_columns:
-            shell.printerr("No column specified")
+        if not self.validate_columns():
             return 0
 
-        for c in self.table_meta.primary_key:
-            if c.name not in self.valid_columns:
-                shell.printerr("Primary key column '%s' missing or skipped" % (c.name,))
-                return 0
-
-        self.printmsg("\nStarting copy of %s.%s with columns %s." % (self.ks, self.table, self.valid_columns))
+        columns = u"[" + u", ".join(self.valid_columns) + u"]"
+        self.printmsg(u"\nStarting copy of %s.%s with columns %s." % (self.ks, self.table, columns), encoding=self.encoding)
 
         try:
             params = self.make_params()
@@ -1077,8 +1137,8 @@
             for i in range(self.num_processes - 1):
                 self.processes.append(ImportProcess(self.update_params(params, i)))
 
-            feeder = FeedingProcess(self.outmsg.channels[-1], self.inmsg.channels[-1],
-                                    self.outmsg.channels[:-1], self.fname, self.options,
+            feeder = FeedingProcess(self.outmsg.pipes[-1], self.inmsg.pipes[-1],
+                                    self.outmsg.pipes[:-1], self.fname, self.options,
                                     self.shell.conn if not IS_WINDOWS else None)
             self.processes.append(feeder)
 
@@ -1092,7 +1152,7 @@
                 profile_off(pr, file_name='parent_profile_%d.txt' % (os.getpid(),))
 
         except Exception, exc:
-            shell.printerr(str(exc))
+            shell.printerr(unicode(exc))
             if shell.debug:
                 traceback.print_exc()
             return 0
@@ -1108,7 +1168,7 @@
         """
         shell = self.shell
 
-        self.printmsg("[Use \. on a line by itself to end input]")
+        self.printmsg("[Use . on a line by itself to end input]")
         for row in shell.use_stdin_reader(prompt='[copy] ', until=r'.'):
             self.outmsg.channels[-1].send(row)
 
@@ -1186,12 +1246,15 @@
     """
     A process that reads from import sources and sends chunks to worker processes.
     """
-    def __init__(self, inmsg, outmsg, worker_channels, fname, options, parent_cluster):
+    def __init__(self, inpipe, outpipe, worker_pipes, fname, options, parent_cluster):
         mp.Process.__init__(self, target=self.run)
-        self.inmsg = inmsg
-        self.outmsg = outmsg
-        self.worker_channels = worker_channels
-        self.reader = FilesReader(fname, options) if fname else PipeReader(inmsg, options)
+        self.inpipe = inpipe
+        self.outpipe = outpipe
+        self.worker_pipes = worker_pipes
+        self.inmsg = None  # must be created after forking on Windows
+        self.outmsg = None  # must be created after forking on Windows
+        self.worker_channels = None  # must be created after forking on Windows
+        self.reader = FilesReader(fname, options) if fname else PipeReader(inpipe, options)
         self.send_meter = RateMeter(log_fcn=None, update_interval=1)
         self.ingest_rate = options.copy['ingestrate']
         self.num_worker_processes = options.copy['numprocesses']
@@ -1200,8 +1263,13 @@
 
     def on_fork(self):
         """
-        Release any parent connections after forking, see CASSANDRA-11749 for details.
+        Create the channels and release any parent connections after forking,
+        see CASSANDRA-11749 for details.
         """
+        self.inmsg = ReceivingChannel(self.inpipe)
+        self.outmsg = SendingChannel(self.outpipe)
+        self.worker_channels = [SendingChannel(p) for p in self.worker_pipes]
+
         if self.parent_cluster:
             printdebugmsg("Closing parent cluster sockets")
             self.parent_cluster.shutdown()
@@ -1241,7 +1309,7 @@
                     if rows:
                         sent += self.send_chunk(ch, rows)
                 except Exception, exc:
-                    self.outmsg.send(ImportTaskError(exc.__class__.__name__, str(exc)))
+                    self.outmsg.send(ImportTaskError(exc.__class__.__name__, exc.message))
 
                 if reader.exhausted:
                     break
@@ -1275,8 +1343,10 @@
 
     def __init__(self, params, target):
         mp.Process.__init__(self, target=target)
-        self.inmsg = params['inmsg']
-        self.outmsg = params['outmsg']
+        self.inpipe = params['inpipe']
+        self.outpipe = params['outpipe']
+        self.inmsg = None  # must be initialized after fork on Windows
+        self.outmsg = None  # must be initialized after fork on Windows
         self.ks = params['ks']
         self.table = params['table']
         self.local_dc = params['local_dc']
@@ -1293,12 +1363,13 @@
         self.config_file = params['config_file']
 
         options = params['options']
-        self.time_format = options.copy['dtformats']
+        self.date_time_format = options.copy['dtformats']
         self.consistency_level = options.copy['consistencylevel']
         self.decimal_sep = options.copy['decimalsep']
         self.thousands_sep = options.copy['thousandssep']
         self.boolean_styles = options.copy['boolstyle']
         self.max_attempts = options.copy['maxattempts']
+        self.encoding = options.copy['encoding']
         # Here we inject some failures for testing purposes, only if this environment variable is set
         if os.environ.get('CQLSH_COPY_TEST_FAILURES', ''):
             self.test_failures = json.loads(os.environ.get('CQLSH_COPY_TEST_FAILURES', ''))
@@ -1307,8 +1378,11 @@
 
     def on_fork(self):
         """
-        Release any parent connections after forking, see CASSANDRA-11749 for details.
+        Create the channels and release any parent connections after forking, see CASSANDRA-11749 for details.
         """
+        self.inmsg = ReceivingChannel(self.inpipe)
+        self.outmsg = SendingChannel(self.outpipe)
+
         if self.parent_cluster:
             printdebugmsg("Closing parent cluster sockets")
             self.parent_cluster.shutdown()
@@ -1414,7 +1488,6 @@
     def __init__(self, params):
         ChildProcess.__init__(self, params=params, target=self.run)
         options = params['options']
-        self.encoding = options.copy['encoding']
         self.float_precision = options.copy['float_precision']
         self.nullval = options.copy['nullval']
         self.max_requests = options.copy['maxrequests']
@@ -1459,7 +1532,7 @@
             if print_traceback and sys.exc_info()[1] == err:
                 traceback.print_exc()
         else:
-            msg = str(err)
+            msg = unicode(err)
         return msg
 
     def report_error(self, err, token_range):
@@ -1582,7 +1655,7 @@
             formatter = get_formatter(ctype)
             self.formatters[ctype] = formatter
 
-        return formatter(val, encoding=self.encoding, colormap=NO_COLOR_MAP, time_format=self.time_format,
+        return formatter(val, encoding=self.encoding, colormap=NO_COLOR_MAP, date_time_format=self.date_time_format,
                          float_precision=self.float_precision, nullval=self.nullval, quote=False,
                          decimal_sep=self.decimal_sep, thousands_sep=self.thousands_sep,
                          boolean_styles=self.boolean_styles)
@@ -1664,8 +1737,9 @@
         self.decimal_sep = parent.decimal_sep
         self.thousands_sep = parent.thousands_sep
         self.boolean_styles = parent.boolean_styles
-        self.time_format = parent.time_format
+        self.date_time_format = parent.date_time_format.timestamp_format
         self.debug = parent.debug
+        self.encoding = parent.encoding
 
         self.table_meta = table_meta
         self.primary_key_indexes = [self.columns.index(col.name) for col in self.table_meta.primary_key]
@@ -1677,6 +1751,7 @@
         else:
             self.use_prepared_statements = True
 
+        self.is_counter = parent.is_counter(table_meta)
         self.proto_version = statement.protocol_version
 
         # the cql types and converters for the prepared statement, either the full statement or only the primary keys
@@ -1685,10 +1760,16 @@
 
         # the cql types for the entire statement, these are the same as the types above but
         # only when using prepared statements
-        self.coltypes = [table_meta.columns[name].typestring for name in parent.valid_columns]
+        self.coltypes = [table_meta.columns[name].cql_type for name in parent.valid_columns]
         # these functions are used for non-prepared statements to protect values with quotes if required
-        self.protectors = [protect_value if t in ('ascii', 'text', 'timestamp', 'date', 'time', 'inet') else lambda v: v
-                           for t in self.coltypes]
+        self.protectors = [self._get_protector(t) for t in self.coltypes]
+
+    @staticmethod
+    def _get_protector(t):
+        if t in ('ascii', 'text', 'timestamp', 'date', 'time', 'inet'):
+            return lambda v: protect_value(v)
+        else:
+            return lambda v: v
 
     @staticmethod
     def _get_primary_key_statement(parent, table_meta):
@@ -1703,15 +1784,18 @@
                                                          where_clause)
         return parent.session.prepare(select_query)
 
+    @staticmethod
+    def unprotect(v):
+        if v is not None:
+            return CqlRuleSet.dequote_value(v)
+
     def _get_converter(self, cql_type):
         """
         Return a function that converts a string into a value the can be passed
         into BoundStatement.bind() for the given cql type. See cassandra.cqltypes
         for more details.
         """
-        def unprotect(v):
-            if v is not None:
-                return CqlRuleSet.dequote_value(v)
+        unprotect = self.unprotect
 
         def convert(t, v):
             v = unprotect(v)
@@ -1726,11 +1810,7 @@
             return converters.get(t.typename, convert_unknown)(v, ct=t)
 
         def convert_blob(v, **_):
-            try:
-                return bytearray.fromhex(v[2:])
-            except TypeError:
-                # Work-around for Python 2.6 bug
-                return bytearray.fromhex(unicode(v[2:]))
+            return bytearray.fromhex(v[2:])
 
         def convert_text(v, **_):
             return v
@@ -1798,14 +1878,14 @@
 
         def convert_datetime(val, **_):
             try:
-                tval = time.strptime(val, self.time_format)
+                tval = time.strptime(val, self.date_time_format)
                 return timegm(tval) * 1e3  # scale seconds to millis for the raw value
             except ValueError:
                 pass  # if it's not in the default format we try CQL formats
 
             m = p.match(val)
             if not m:
-                raise ValueError("can't interpret %r as a date, specified time format is %s" % (val, self.time_format))
+                raise ValueError("can't interpret %r as a date with this format: %s" % (val, self.date_time_format))
 
             # https://docs.python.org/2/library/time.html#time.struct_time
             tval = time.struct_time((int(m.group(1)), int(m.group(2)), int(m.group(3)),  # year, month, day
@@ -1909,7 +1989,14 @@
         return converters.get(cql_type.typename, convert_unknown)
 
     def get_null_val(self):
-        return None if self.use_prepared_statements else "NULL"
+        """
+        Return the null value that is inserted for fields that are missing from csv files.
+        For counters we should return zero so that the counter value won't be incremented.
+        For everything else we return nulls, this means None if we use prepared statements
+        or "NULL" otherwise. Note that for counters we never use prepared statements, so we
+        only check is_counter when use_prepared_statements is false.
+        """
+        return None if self.use_prepared_statements else ("0" if self.is_counter else "NULL")
 
     def convert_row(self, row):
         """
@@ -1932,7 +2019,7 @@
             except Exception, e:
                 if self.debug:
                     traceback.print_exc()
-                raise ParseError("Failed to parse %s : %s" % (val, str(e)))
+                raise ParseError("Failed to parse %s : %s" % (val, e.message))
 
         return [convert(conv, val) for conv, val in zip(converters, row)]
 
@@ -1955,7 +2042,7 @@
             return self.cqltypes[n].serialize(v, self.proto_version)
 
         def serialize_value_not_prepared(n, v):
-            return self.cqltypes[n].serialize(self.converters[n](v), self.proto_version)
+            return self.cqltypes[n].serialize(self.converters[n](self.unprotect(v)), self.proto_version)
 
         partition_key_indexes = self.partition_key_indexes
         serialize = serialize_value_prepared if self.use_prepared_statements else serialize_value_not_prepared
@@ -2024,7 +2111,7 @@
 
     def filter_replicas(self, hosts):
         shuffled = tuple(sorted(hosts, key=lambda k: random.random()))
-        return filter(lambda r: r.is_up and r.datacenter == self.local_dc, shuffled) if hosts else ()
+        return filter(lambda r: r.is_up is not False and r.datacenter == self.local_dc, shuffled) if hosts else ()
 
 
 class FastTokenAwarePolicy(DCAwareRoundRobinPolicy):
@@ -2055,7 +2142,7 @@
         ChildProcess.__init__(self, params=params, target=self.run)
 
         self.skip_columns = params['skip_columns']
-        self.valid_columns = params['valid_columns']
+        self.valid_columns = [c.encode(self.encoding) for c in params['valid_columns']]
         self.skip_column_indexes = [i for i, c in enumerate(self.columns) if c in self.skip_columns]
 
         options = params['options']
@@ -2102,8 +2189,7 @@
                 profile_off(pr, file_name='worker_profile_%d.txt' % (os.getpid(),))
 
         except Exception, exc:
-            if self.debug:
-                traceback.print_exc(exc)
+            self.report_error(exc)
 
         finally:
             self.close()
@@ -2113,13 +2199,15 @@
             self._session.cluster.shutdown()
         ChildProcess.close(self)
 
+    def is_counter(self, table_meta):
+        return "counter" in [table_meta.columns[name].cql_type for name in self.valid_columns]
+
     def make_params(self):
         metadata = self.session.cluster.metadata
         table_meta = metadata.keyspaces[self.ks].tables[self.table]
 
         prepared_statement = None
-        is_counter = ("counter" in [table_meta.columns[name].typestring for name in self.valid_columns])
-        if is_counter:
+        if self.is_counter(table_meta):
             query = 'UPDATE %s.%s SET %%s WHERE %%s' % (protect_name(self.ks), protect_name(self.table))
             make_statement = self.wrap_make_statement(self.make_counter_batch_statement)
         elif self.use_prepared_statements:
@@ -2192,20 +2280,25 @@
         return make_statement_with_failures if self.test_failures else make_statement
 
     def make_counter_batch_statement(self, query, conv, batch, replicas):
-        statement = BatchStatement(batch_type=BatchType.COUNTER, consistency_level=self.consistency_level)
-        statement.replicas = replicas
-        statement.keyspace = self.ks
-        for row in batch['rows']:
+        def make_full_query(r):
             where_clause = []
             set_clause = []
-            for i, value in enumerate(row):
+            for i, value in enumerate(r):
                 if i in conv.primary_key_indexes:
                     where_clause.append("%s=%s" % (self.valid_columns[i], value))
                 else:
                     set_clause.append("%s=%s+%s" % (self.valid_columns[i], self.valid_columns[i], value))
+            return query % (','.join(set_clause), ' AND '.join(where_clause))
 
-            full_query_text = query % (','.join(set_clause), ' AND '.join(where_clause))
-            statement.add(full_query_text)
+        if len(batch['rows']) == 1:
+            statement = SimpleStatement(make_full_query(batch['rows'][0]), consistency_level=self.consistency_level)
+        else:
+            statement = BatchStatement(batch_type=BatchType.COUNTER, consistency_level=self.consistency_level)
+            for row in batch['rows']:
+                statement.add(make_full_query(row))
+
+        statement.replicas = replicas
+        statement.keyspace = self.ks
         return statement
 
     def make_prepared_batch_statement(self, query, _, batch, replicas):
@@ -2219,17 +2312,25 @@
         We could optimize further by removing bound_statements altogether but we'd have to duplicate much
         more driver's code (BoundStatement.bind()).
         """
-        statement = BatchStatement(batch_type=BatchType.UNLOGGED, consistency_level=self.consistency_level)
+        if len(batch['rows']) == 1:
+            statement = query.bind(batch['rows'][0])
+        else:
+            statement = BatchStatement(batch_type=BatchType.UNLOGGED, consistency_level=self.consistency_level)
+            statement._statements_and_parameters = [(True, query.query_id, query.bind(r).values) for r in batch['rows']]
+
         statement.replicas = replicas
         statement.keyspace = self.ks
-        statement._statements_and_parameters = [(True, query.query_id, query.bind(r).values) for r in batch['rows']]
         return statement
 
     def make_non_prepared_batch_statement(self, query, _, batch, replicas):
-        statement = BatchStatement(batch_type=BatchType.UNLOGGED, consistency_level=self.consistency_level)
+        if len(batch['rows']) == 1:
+            statement = SimpleStatement(query % (','.join(batch['rows'][0]),), consistency_level=self.consistency_level)
+        else:
+            statement = BatchStatement(batch_type=BatchType.UNLOGGED, consistency_level=self.consistency_level)
+            statement._statements_and_parameters = [(False, query % (','.join(r),), ()) for r in batch['rows']]
+
         statement.replicas = replicas
         statement.keyspace = self.ks
-        statement._statements_and_parameters = [(False, query % (','.join(r),), ()) for r in batch['rows']]
         return statement
 
     def convert_rows(self, conv, chunk):
@@ -2250,7 +2351,7 @@
             try:
                 return conv.convert_row(r)
             except Exception, err:
-                errors[str(err)].append(r)
+                errors[err.message].append(r)
                 return None
 
         converted_rows = filter(None, [convert_row(r) for r in rows])
@@ -2315,7 +2416,7 @@
                 pk = get_row_partition_key_values(row)
                 rows_by_ring_pos[get_ring_pos(ring, pk_to_token_value(pk))].append(row)
             except Exception, e:
-                errors[str(e)].append(row)
+                errors[e.message].append(row)
 
         if errors:
             for msg, rows in errors.iteritems():
@@ -2350,11 +2451,11 @@
             future.add_callbacks(callback=self.result_callback, callback_args=(batch, chunk),
                                  errback=self.err_callback, errback_args=(batch, chunk, replicas))
 
-    def report_error(self, err, chunk, rows=None, attempts=1, final=True):
+    def report_error(self, err, chunk=None, rows=None, attempts=1, final=True):
         if self.debug and sys.exc_info()[1] == err:
             traceback.print_exc()
-        self.outmsg.send(ImportTaskError(err.__class__.__name__, str(err), rows, attempts, final))
-        if final:
+        self.outmsg.send(ImportTaskError(err.__class__.__name__, err.message, rows, attempts, final))
+        if final and chunk is not None:
             self.update_chunk(rows, chunk)
 
     def update_chunk(self, rows, chunk):
diff --git a/pylib/cqlshlib/cql3handling.py b/pylib/cqlshlib/cql3handling.py
index 029e0c7..897ee16 100644
--- a/pylib/cqlshlib/cql3handling.py
+++ b/pylib/cqlshlib/cql3handling.py
@@ -18,8 +18,8 @@
 from cassandra.metadata import maybe_escape_name
 
 
-simple_cql_types = set(('ascii', 'bigint', 'blob', 'boolean', 'counter', 'decimal', 'double', 'float', 'inet', 'int',
-                        'text', 'timestamp', 'timeuuid', 'uuid', 'varchar', 'varint'))
+simple_cql_types = set(('ascii', 'bigint', 'blob', 'boolean', 'counter', 'date', 'decimal', 'double', 'float', 'inet', 'int',
+                        'smallint', 'text', 'time', 'timestamp', 'timeuuid', 'tinyint', 'uuid', 'varchar', 'varint'))
 simple_cql_types.difference_update(('set', 'map', 'list'))
 
 from . import helptopics
@@ -34,7 +34,7 @@
     def __str__(self):
         return 'Unexpected table structure; may not translate correctly to CQL. ' + self.msg
 
-SYSTEM_KEYSPACES = ('system', 'system_traces', 'system_auth')
+SYSTEM_KEYSPACES = ('system', 'system_traces', 'system_auth', 'system_distributed')
 NONALTERBALE_KEYSPACES = ('system')
 
 
@@ -90,6 +90,12 @@
             return str(value)
         return "'%s'" % value.replace("'", "''")
 
+    @classmethod
+    def escape_name(cls, name):
+        if name is None:
+            return 'NULL'
+        return "'%s'" % name.replace("'", "''")
+
     @staticmethod
     def dequote_name(name):
         name = name.strip()
@@ -132,7 +138,10 @@
 
 JUNK ::= /([ \t\r\f\v]+|(--|[/][/])[^\n\r]*([\n\r]|$)|[/][*].*?[*][/])/ ;
 
-<stringLiteral> ::= /'([^']|'')*'/ ;
+<stringLiteral> ::= <quotedStringLiteral>
+                  | <pgStringLiteral> ;
+<quotedStringLiteral> ::= /'([^']|'')*'/ ;
+<pgStringLiteral> ::= /\$\$(?:(?!\$\$)|[^$])*\$\$/;
 <quotedName> ::=    /"([^"]|"")*"/ ;
 <float> ::=         /-?[0-9]+\.[0-9]+/ ;
 <uuid> ::=          /[0-9a-f]{8}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{12}/ ;
@@ -151,6 +160,7 @@
             | "false"
             ;
 
+<unclosedPgString>::= /\$\$(?:(?!\$\$)|[^$])*/ ;
 <unclosedString>  ::= /'([^']|'')*/ ;
 <unclosedName>    ::= /"([^"]|"")*/ ;
 <unclosedComment> ::= /[/][*].*$/ ;
@@ -162,10 +172,14 @@
          | <boolean>
          | <blobLiteral>
          | <collectionLiteral>
-         | <functionName> <functionArguments>
+         | <functionLiteral> <functionArguments>
          | "NULL"
          ;
 
+<functionLiteral> ::= (<identifier> ( "." <identifier> )?)
+                 | "TOKEN"
+                 ;
+
 <functionArguments> ::= "(" ( <term> ( "," <term> )* )? ")"
                  ;
 
@@ -189,7 +203,21 @@
 <mapLiteral> ::= "{" <term> ":" <term> ( "," <term> ":" <term> )* "}"
                ;
 
-<functionName> ::= <identifier> | "TOKEN"
+<anyFunctionName> ::= ( ksname=<cfOrKsName> dot="." )? udfname=<cfOrKsName> ;
+
+<userFunctionName> ::= ( ksname=<nonSystemKeyspaceName> dot="." )? udfname=<cfOrKsName> ;
+
+<refUserFunctionName> ::= udfname=<cfOrKsName> ;
+
+<userAggregateName> ::= ( ksname=<nonSystemKeyspaceName> dot="." )? udaname=<cfOrKsName> ;
+
+<functionAggregateName> ::= ( ksname=<nonSystemKeyspaceName> dot="." )? functionname=<cfOrKsName> ;
+
+<aggregateName> ::= <userAggregateName>
+                  ;
+
+<functionName> ::= <functionAggregateName>
+                 | "TOKEN"
                  ;
 
 <statementBody> ::= <useStatement>
@@ -211,11 +239,15 @@
                           | <createColumnFamilyStatement>
                           | <createIndexStatement>
                           | <createUserTypeStatement>
+                          | <createFunctionStatement>
+                          | <createAggregateStatement>
                           | <createTriggerStatement>
                           | <dropKeyspaceStatement>
                           | <dropColumnFamilyStatement>
                           | <dropIndexStatement>
                           | <dropUserTypeStatement>
+                          | <dropFunctionStatement>
+                          | <dropAggregateStatement>
                           | <dropTriggerStatement>
                           | <alterTableStatement>
                           | <alterKeyspaceStatement>
@@ -226,10 +258,16 @@
                             | <alterUserStatement>
                             | <dropUserStatement>
                             | <listUsersStatement>
+                            | <createRoleStatement>
+                            | <alterRoleStatement>
+                            | <dropRoleStatement>
+                            | <listRolesStatement>
                             ;
 
 <authorizationStatement> ::= <grantStatement>
+                           | <grantRoleStatement>
                            | <revokeStatement>
+                           | <revokeRoleStatement>
                            | <listPermissionsStatement>
                            ;
 
@@ -472,6 +510,7 @@
             opts.add('max_sstable_age_days')
             opts.add('min_threshold')
             opts.add('max_window_size_seconds')
+            opts.add('timestamp_resolution')
         return map(escape_value, opts)
     return ()
 
@@ -609,7 +648,7 @@
 syntax_rules += r'''
 <useStatement> ::= "USE" <keyspaceName>
                  ;
-<selectStatement> ::= "SELECT" <selectClause>
+<selectStatement> ::= "SELECT" ( "JSON" )? <selectClause>
                         "FROM" cf=<columnFamilyName>
                           ( "WHERE" <whereClause> )?
                           ( "ORDER" "BY" <orderByClause> ( "," <orderByClause> )* )?
@@ -618,7 +657,7 @@
                     ;
 <whereClause> ::= <relation> ( "AND" <relation> )*
                 ;
-<relation> ::= [rel_lhs]=<cident> ( "=" | "<" | ">" | "<=" | ">=" | "CONTAINS" ( "KEY" )? ) <term>
+<relation> ::= [rel_lhs]=<cident> ( "[" <term> "]" )? ( "=" | "<" | ">" | "<=" | ">=" | "CONTAINS" ( "KEY" )? ) <term>
              | token="TOKEN" "(" [rel_tokname]=<cident>
                                  ( "," [rel_tokname]=<cident> )*
                              ")" ("=" | "<" | ">" | "<=" | ">=") <tokenDefinition>
@@ -626,7 +665,6 @@
              ;
 <selectClause> ::= "DISTINCT"? <selector> ("AS" <cident>)? ("," <selector> ("AS" <cident>)?)*
                  | "*"
-                 | "COUNT" "(" star=( "*" | "1" ) ")" ("AS" <cident>)?
                  ;
 <udtSubfieldSelection> ::= <identifier> "." <identifier>
                          ;
@@ -634,6 +672,7 @@
              | <udtSubfieldSelection>
              | "WRITETIME" "(" [colname]=<cident> ")"
              | "TTL" "(" [colname]=<cident> ")"
+             | "COUNT" "(" star=( "*" | "1" ) ")"
              | <functionName> <selectionFunctionArguments>
              ;
 <selectionFunctionArguments> ::= "(" ( <selector> ( "," <selector> )* )? ")"
@@ -643,6 +682,68 @@
 '''
 
 
+def udf_name_completer(ctxt, cass):
+    ks = ctxt.get_binding('ksname', None)
+    if ks is not None:
+        ks = dequote_name(ks)
+    try:
+        udfnames = cass.get_userfunction_names(ks)
+    except Exception:
+        if ks is None:
+            return ()
+        raise
+    return map(maybe_escape_name, udfnames)
+
+
+def uda_name_completer(ctxt, cass):
+    ks = ctxt.get_binding('ksname', None)
+    if ks is not None:
+        ks = dequote_name(ks)
+    try:
+        udanames = cass.get_useraggregate_names(ks)
+    except Exception:
+        if ks is None:
+            return ()
+        raise
+    return map(maybe_escape_name, udanames)
+
+
+def udf_uda_name_completer(ctxt, cass):
+    ks = ctxt.get_binding('ksname', None)
+    if ks is not None:
+        ks = dequote_name(ks)
+    try:
+        functionnames = cass.get_userfunction_names(ks) + cass.get_useraggregate_names(ks)
+    except Exception:
+        if ks is None:
+            return ()
+        raise
+    return map(maybe_escape_name, functionnames)
+
+
+def ref_udf_name_completer(ctxt, cass):
+    try:
+        udanames = cass.get_userfunction_names(None)
+    except Exception:
+        return ()
+    return map(maybe_escape_name, udanames)
+
+
+completer_for('functionAggregateName', 'ksname')(cf_ks_name_completer)
+completer_for('functionAggregateName', 'dot')(cf_ks_dot_completer)
+completer_for('functionAggregateName', 'functionname')(udf_uda_name_completer)
+completer_for('anyFunctionName', 'ksname')(cf_ks_name_completer)
+completer_for('anyFunctionName', 'dot')(cf_ks_dot_completer)
+completer_for('anyFunctionName', 'udfname')(udf_name_completer)
+completer_for('userFunctionName', 'ksname')(cf_ks_name_completer)
+completer_for('userFunctionName', 'dot')(cf_ks_dot_completer)
+completer_for('userFunctionName', 'udfname')(udf_name_completer)
+completer_for('refUserFunctionName', 'udfname')(ref_udf_name_completer)
+completer_for('userAggregateName', 'ksname')(cf_ks_name_completer)
+completer_for('userAggregateName', 'dot')(cf_ks_dot_completer)
+completer_for('userAggregateName', 'udaname')(uda_name_completer)
+
+
 @completer_for('orderByClause', 'ordercol')
 def select_order_column_completer(ctxt, cass):
     prev_order_cols = ctxt.get_binding('ordercol', ())
@@ -672,36 +773,29 @@
 @completer_for('relation', 'rel_lhs')
 def select_relation_lhs_completer(ctxt, cass):
     layout = get_table_meta(ctxt, cass)
-    filterable = set((layout.partition_key[0].name, layout.clustering_key[0].name))
+    filterable = set()
     already_filtered_on = map(dequote_name, ctxt.get_binding('rel_lhs', ()))
-    for num in range(1, len(layout.partition_key)):
-        if layout.partition_key[num - 1].name in already_filtered_on:
+    for num in range(0, len(layout.partition_key)):
+        if num == 0 or layout.partition_key[num - 1].name in already_filtered_on:
             filterable.add(layout.partition_key[num].name)
         else:
             break
-    for num in range(1, len(layout.clustering_key)):
-        if layout.clustering_key[num - 1].name in already_filtered_on:
+    for num in range(0, len(layout.clustering_key)):
+        if num == 0 or layout.clustering_key[num - 1].name in already_filtered_on:
             filterable.add(layout.clustering_key[num].name)
         else:
             break
-    for cd in layout.columns.values():
-        if cd.index:
-            filterable.add(cd.name)
+    for idx in layout.indexes.itervalues():
+        filterable.add(idx.index_options["target"])
     return map(maybe_escape_name, filterable)
 
-
-@completer_for('selectClause', 'star')
-def select_count_star_completer(ctxt, cass):
-    return ['*']
-
 explain_completion('selector', 'colname')
 
 syntax_rules += r'''
 <insertStatement> ::= "INSERT" "INTO" cf=<columnFamilyName>
-                               "(" [colname]=<cident> "," [colname]=<cident>
-                                   ( "," [colname]=<cident> )* ")"
-                      "VALUES" "(" [newval]=<term> valcomma="," [newval]=<term>
-                                   ( valcomma="," [newval]=<term> )* valcomma=")"
+                      ( ( "(" [colname]=<cident> ( "," [colname]=<cident> )* ")"
+                          "VALUES" "(" [newval]=<term> ( valcomma="," [newval]=<term> )* valcomma=")")
+                        | ("JSON" <stringLiteral>))
                       ( "IF" "NOT" "EXISTS")?
                       ( "USING" [insertopt]=<usingOption>
                                 ( "AND" [insertopt]=<usingOption> )* )?
@@ -741,16 +835,16 @@
     if len(valuesdone) >= len(insertcols):
         return []
     curcol = insertcols[len(valuesdone)]
-    cqltype = layout.columns[curcol].data_type
-    coltype = cqltype.typename
+    coltype = layout.columns[curcol].cql_type
     if coltype in ('map', 'set'):
         return ['{']
     if coltype == 'list':
         return ['[']
     if coltype == 'boolean':
         return ['true', 'false']
+
     return [Hint('<value for %s (%s)>' % (maybe_escape_name(curcol),
-                                          cqltype.cql_parameterized_type()))]
+                                          coltype))]
 
 
 @completer_for('insertStatement', 'valcomma')
@@ -810,29 +904,28 @@
 def update_countername_completer(ctxt, cass):
     layout = get_table_meta(ctxt, cass)
     curcol = dequote_name(ctxt.get_binding('updatecol', ''))
-    cqltype = layout.columns[curcol].data_type
-    coltype = cqltype.typename
+    coltype = layout.columns[curcol].cql_type
     if coltype == 'counter':
         return [maybe_escape_name(curcol)]
     if coltype in ('map', 'set'):
         return ["{"]
     if coltype == 'list':
         return ["["]
-    return [Hint('<term (%s)>' % cqltype.cql_parameterized_type())]
+    return [Hint('<term (%s)>' % coltype)]
 
 
 @completer_for('assignment', 'counterop')
 def update_counterop_completer(ctxt, cass):
     layout = get_table_meta(ctxt, cass)
     curcol = dequote_name(ctxt.get_binding('updatecol', ''))
-    return ['+', '-'] if layout.columns[curcol].data_type.typename == 'counter' else []
+    return ['+', '-'] if layout.columns[curcol].cql_type == 'counter' else []
 
 
 @completer_for('assignment', 'inc')
 def update_counter_inc_completer(ctxt, cass):
     layout = get_table_meta(ctxt, cass)
     curcol = dequote_name(ctxt.get_binding('updatecol', ''))
-    if layout.columns[curcol].data_type.typename == 'counter':
+    if layout.columns[curcol].cql_type == 'counter':
         return [Hint('<wholenumber>')]
     return []
 
@@ -858,7 +951,7 @@
 def update_indexbracket_completer(ctxt, cass):
     layout = get_table_meta(ctxt, cass)
     curcol = dequote_name(ctxt.get_binding('updatecol', ''))
-    coltype = layout.columns[curcol].data_type.typename
+    coltype = layout.columns[curcol].cql_type
     if coltype in ('map', 'list'):
         return ['[']
     return []
@@ -1052,6 +1145,30 @@
                                 ( "," [newcolname]=<cident> <storageType> )*
                             ")"
                          ;
+
+<createFunctionStatement> ::= "CREATE" ("OR" "REPLACE")? "FUNCTION"
+                            ("IF" "NOT" "EXISTS")?
+                            <userFunctionName>
+                            ( "(" ( newcol=<cident> <storageType>
+                              ( "," [newcolname]=<cident> <storageType> )* )?
+                            ")" )?
+                            ("RETURNS" "NULL" | "CALLED") "ON" "NULL" "INPUT"
+                            "RETURNS" <storageType>
+                            "LANGUAGE" <cident> "AS" <stringLiteral>
+                         ;
+
+<createAggregateStatement> ::= "CREATE" ("OR" "REPLACE")? "AGGREGATE"
+                            ("IF" "NOT" "EXISTS")?
+                            <userAggregateName>
+                            ( "("
+                                 ( <storageType> ( "," <storageType> )* )?
+                              ")" )?
+                            "SFUNC" <refUserFunctionName>
+                            "STYPE" <storageType>
+                            ( "FINALFUNC" <refUserFunctionName> )?
+                            ( "INITCOND" <term> )?
+                         ;
+
 '''
 
 explain_completion('createIndexStatement', 'indexname', '<new_index_name>')
@@ -1061,8 +1178,10 @@
 
 @completer_for('createIndexStatement', 'col')
 def create_index_col_completer(ctxt, cass):
+    """ Return the columns for which an index doesn't exist yet. """
     layout = get_table_meta(ctxt, cass)
-    colnames = [cd.name for cd in layout.columns.values() if not cd.index]
+    idx_targets = [idx.index_options["target"] for idx in layout.indexes.itervalues()]
+    colnames = [cd.name for cd in layout.columns.values() if cd.name not in idx_targets]
     return map(maybe_escape_name, colnames)
 
 syntax_rules += r'''
@@ -1082,7 +1201,13 @@
                        ;
 
 <dropUserTypeStatement> ::= "DROP" "TYPE" ut=<userTypeName>
-                              ;
+                          ;
+
+<dropFunctionStatement> ::= "DROP" "FUNCTION" ( "IF" "EXISTS" )? <userFunctionName>
+                          ;
+
+<dropAggregateStatement> ::= "DROP" "AGGREGATE" ( "IF" "EXISTS" )? <userAggregateName>
+                          ;
 
 '''
 
@@ -1181,14 +1306,48 @@
 '''
 
 syntax_rules += r'''
-<grantStatement> ::= "GRANT" <permissionExpr> "ON" <resource> "TO" <username>
+<rolename> ::= <identifier>
+             | <quotedName>
+             | <unreservedKeyword>
+             ;
+
+<createRoleStatement> ::= "CREATE" "ROLE" <rolename>
+                              ( "WITH" <roleProperty> ("AND" <roleProperty>)*)?
+                        ;
+
+<alterRoleStatement> ::= "ALTER" "ROLE" <rolename>
+                              ( "WITH" <roleProperty> ("AND" <roleProperty>)*)?
+                       ;
+
+<roleProperty> ::= "PASSWORD" "=" <stringLiteral>
+                 | "OPTIONS" "=" <mapLiteral>
+                 | "SUPERUSER" "=" <boolean>
+                 | "LOGIN" "=" <boolean>
+                 ;
+
+<dropRoleStatement> ::= "DROP" "ROLE" <rolename>
+                      ;
+
+<grantRoleStatement> ::= "GRANT" <rolename> "TO" <rolename>
+                       ;
+
+<revokeRoleStatement> ::= "REVOKE" <rolename> "FROM" <rolename>
+                        ;
+
+<listRolesStatement> ::= "LIST" "ROLES"
+                              ( "OF" <rolename> )? "NORECURSIVE"?
+                       ;
+'''
+
+syntax_rules += r'''
+<grantStatement> ::= "GRANT" <permissionExpr> "ON" <resource> "TO" <rolename>
                    ;
 
-<revokeStatement> ::= "REVOKE" <permissionExpr> "ON" <resource> "FROM" <username>
+<revokeStatement> ::= "REVOKE" <permissionExpr> "ON" <resource> "FROM" <rolename>
                     ;
 
 <listPermissionsStatement> ::= "LIST" <permissionExpr>
-                                    ( "ON" <resource> )? ( "OF" <username> )? "NORECURSIVE"?
+                                    ( "ON" <resource> )? ( "OF" <rolename> )? "NORECURSIVE"?
                              ;
 
 <permission> ::= "AUTHORIZE"
@@ -1197,6 +1356,8 @@
                | "DROP"
                | "SELECT"
                | "MODIFY"
+               | "DESCRIBE"
+               | "EXECUTE"
                ;
 
 <permissionExpr> ::= ( <permission> "PERMISSION"? )
@@ -1204,12 +1365,26 @@
                    ;
 
 <resource> ::= <dataResource>
+             | <roleResource>
+             | <functionResource>
              ;
 
 <dataResource> ::= ( "ALL" "KEYSPACES" )
                  | ( "KEYSPACE" <keyspaceName> )
                  | ( "TABLE"? <columnFamilyName> )
                  ;
+
+<roleResource> ::= ("ALL" "ROLES")
+                 | ("ROLE" <rolename>)
+                 ;
+
+<functionResource> ::= ( "ALL" "FUNCTIONS" ("IN KEYSPACE" <keyspaceName>)? )
+                     | ( "FUNCTION" <functionAggregateName>
+                           ( "(" ( newcol=<cident> <storageType>
+                             ( "," [newcolname]=<cident> <storageType> )* )?
+                           ")" )
+                       )
+                     ;
 '''
 
 
@@ -1227,11 +1402,25 @@
     session = cass.session
     return [maybe_quote(row.values()[0].replace("'", "''")) for row in session.execute("LIST USERS")]
 
+
+@completer_for('rolename', 'role')
+def rolename_completer(ctxt, cass):
+    def maybe_quote(name):
+        if CqlRuleSet.is_valid_cql3_name(name):
+            return name
+        return "'%s'" % name
+
+    # disable completion for CREATE ROLE.
+    if ctxt.matched[0][1].upper() == 'CREATE':
+        return [Hint('<rolename>')]
+
+    session = cass.session
+    return [maybe_quote(row[0].replace("'", "''")) for row in session.execute("LIST ROLES")]
+
 syntax_rules += r'''
 <createTriggerStatement> ::= "CREATE" "TRIGGER" ( "IF" "NOT" "EXISTS" )? <cident>
                                "ON" cf=<columnFamilyName> "USING" class=<stringLiteral>
                            ;
-
 <dropTriggerStatement> ::= "DROP" "TRIGGER" ( "IF" "EXISTS" )? triggername=<cident>
                              "ON" cf=<columnFamilyName>
                          ;
diff --git a/pylib/cqlshlib/cqlhandling.py b/pylib/cqlshlib/cqlhandling.py
index 6ee3cf5..a8a0ba8 100644
--- a/pylib/cqlshlib/cqlhandling.py
+++ b/pylib/cqlshlib/cqlhandling.py
@@ -63,7 +63,7 @@
         We cannot let resreved cql keywords be simple 'identifier' since this caused
         problems with completion, see CASSANDRA-10415
         """
-        syntax = '<reserved_identifier> ::= /(' + '|'.join(r'\b%s\b' % (k,) for k in keywords) + ')/ ;'
+        syntax = '<reserved_identifier> ::= /(' + '|'.join(r'\b{}\b'.format(k) for k in keywords) + ')/ ;'
         self.append_rules(syntax)
 
     def completer_for(self, rulename, symname):
@@ -142,6 +142,7 @@
         stmts = util.split_list(tokens, lambda t: t[0] == 'endtoken')
         output = []
         in_batch = False
+        in_pg_string = len([st for st in tokens if len(st) > 0 and st[0] == 'unclosedPgString']) == 1
         for stmt in stmts:
             if in_batch:
                 output[-1].extend(stmt)
@@ -152,7 +153,7 @@
                     in_batch = False
                 elif stmt[0][1].upper() == 'BEGIN':
                     in_batch = True
-        return output, in_batch
+        return output, in_batch or in_pg_string
 
     def cql_complete_single(self, text, partial, init_bindings={}, ignore_case=True,
                             startsymbol='Start'):
@@ -306,7 +307,7 @@
                 first = first[:-1]
             if debug:
                 print "** Got a partial completion: %r." % (common_prefix,)
-            first += common_prefix
+            return first + common_prefix
         if debug:
             print "** New total completion: %r. Checking for further matches...\n" % (first,)
         return self.cql_complete_multiple(text, first, init_bindings, startsymbol=startsymbol)
@@ -321,7 +322,7 @@
         if tok[0] == 'unclosedName':
             # strip one quote
             return tok[1][1:].replace('""', '"')
-        if tok[0] == 'stringLiteral':
+        if tok[0] == 'quotedStringLiteral':
             # strip quotes
             return tok[1][1:-1].replace("''", "'")
         if tok[0] == 'unclosedString':
diff --git a/pylib/cqlshlib/displaying.py b/pylib/cqlshlib/displaying.py
index 7b260c2..f3b7a64 100644
--- a/pylib/cqlshlib/displaying.py
+++ b/pylib/cqlshlib/displaying.py
@@ -105,6 +105,8 @@
     error=RED,
     blob=DARK_MAGENTA,
     timestamp=GREEN,
+    date=GREEN,
+    time=GREEN,
     int=GREEN,
     float=GREEN,
     decimal=GREEN,
diff --git a/pylib/cqlshlib/formatting.py b/pylib/cqlshlib/formatting.py
index abb4896..dcd08da 100644
--- a/pylib/cqlshlib/formatting.py
+++ b/pylib/cqlshlib/formatting.py
@@ -15,17 +15,21 @@
 # limitations under the License.
 
 import binascii
-import sys
-import re
 import calendar
 import math
+import re
+import sys
+import platform
 import wcwidth
+
 from collections import defaultdict
 from displaying import colorme, get_str, FormattedValue, DEFAULT_VALUE_COLORS, NO_COLOR_MAP
 from cassandra.cqltypes import EMPTY
 from cassandra.util import datetime_from_timestamp
 from util import UTC
 
+is_win = platform.system() == 'Windows'
+
 unicode_controlchars_re = re.compile(r'[\x00-\x31\x7f-\xa0]')
 controlchars_re = re.compile(r'[\x00-\x31\x7f-\xff]')
 
@@ -50,14 +54,13 @@
     return _turn_bits_red
 
 default_null_placeholder = 'null'
-default_time_format = ''
 default_float_precision = 3
 default_colormap = DEFAULT_VALUE_COLORS
 empty_colormap = defaultdict(lambda: '')
 
 
 def format_by_type(cqltype, val, encoding, colormap=None, addcolor=False,
-                   nullval=None, time_format=None, float_precision=None,
+                   nullval=None, date_time_format=None, float_precision=None,
                    decimal_sep=None, thousands_sep=None, boolean_styles=None):
     if nullval is None:
         nullval = default_null_placeholder
@@ -67,12 +70,12 @@
         colormap = empty_colormap
     elif colormap is None:
         colormap = default_colormap
-    if time_format is None:
-        time_format = default_time_format
+    if date_time_format is None:
+        date_time_format = DateTimeFormat()
     if float_precision is None:
         float_precision = default_float_precision
     return format_value(cqltype, val, encoding=encoding, colormap=colormap,
-                        time_format=time_format, float_precision=float_precision,
+                        date_time_format=date_time_format, float_precision=float_precision,
                         nullval=nullval, decimal_sep=decimal_sep, thousands_sep=thousands_sep,
                         boolean_styles=boolean_styles)
 
@@ -94,6 +97,23 @@
         displaywidth -= bval.count(r'\\')
     return FormattedValue(bval, coloredval, displaywidth)
 
+DEFAULT_NANOTIME_FORMAT = '%H:%M:%S.%N'
+DEFAULT_DATE_FORMAT = '%Y-%m-%d'
+DEFAULT_TIMESTAMP_FORMAT = '%Y-%m-%d %H:%M:%S%z'
+
+if platform.system() == 'Windows':
+    DEFAULT_TIME_FORMAT = '%Y-%m-%d %H:%M:%S %Z'
+
+
+class DateTimeFormat():
+
+    def __init__(self, timestamp_format=DEFAULT_TIMESTAMP_FORMAT, date_format=DEFAULT_DATE_FORMAT,
+                 nanotime_format=DEFAULT_NANOTIME_FORMAT, timezone=None):
+        self.timestamp_format = timestamp_format
+        self.date_format = date_format
+        self.nanotime_format = nanotime_format
+        self.timezone = timezone
+
 
 def format_value_default(val, colormap, **_):
     val = str(val)
@@ -126,7 +146,7 @@
 
 @formatter_for('bytearray')
 def format_value_blob(val, colormap, **_):
-    bval = '0x' + binascii.hexlify(str(val))
+    bval = '0x' + binascii.hexlify(val)
     return colorme(bval, colormap, 'blob')
 formatter_for('buffer')(format_value_blob)
 
@@ -175,7 +195,7 @@
             if dpart_str:
                 bval += '%s%s' % ('.' if not decimal_sep else decimal_sep, dpart_str)
         else:
-            exponent = int(math.log10(abs(val))) if abs(val) > sys.float_info.epsilon else -sys.maxint - 1
+            exponent = int(math.log10(abs(val))) if abs(val) > sys.float_info.epsilon else -sys.maxsize - 1
             if -4 <= exponent < float_precision:
                 # when this is true %g will not use scientific notation,
                 # increasing precision should not change this decision
@@ -214,19 +234,29 @@
 formatter_for('int')(format_integer_type)
 
 
-@formatter_for('date')
-def format_value_timestamp(val, colormap, time_format, quote=False, **_):
-    bval = strftime(time_format, calendar.timegm(val.utctimetuple()))
+@formatter_for('datetime')
+def format_value_timestamp(val, colormap, date_time_format, quote=False, **_):
+    bval = strftime(date_time_format.timestamp_format, calendar.timegm(val.utctimetuple()), timezone=date_time_format.timezone)
     if quote:
         bval = "'%s'" % bval
     return colorme(bval, colormap, 'timestamp')
 
-formatter_for('datetime')(format_value_timestamp)
+
+def strftime(time_format, seconds, timezone=None):
+    ret_dt = datetime_from_timestamp(seconds).replace(tzinfo=UTC())
+    if timezone:
+        ret_dt = ret_dt.astimezone(timezone)
+    return ret_dt.strftime(time_format)
 
 
-def strftime(time_format, seconds):
-    tzless_dt = datetime_from_timestamp(seconds)
-    return tzless_dt.replace(tzinfo=UTC()).strftime(time_format)
+@formatter_for('Date')
+def format_value_date(val, colormap, **_):
+    return format_python_formatted_type(val, colormap, 'date')
+
+
+@formatter_for('Time')
+def format_value_time(val, colormap, **_):
+    return format_python_formatted_type(val, colormap, 'time')
 
 
 @formatter_for('str')
@@ -246,10 +276,10 @@
 
 
 def format_simple_collection(val, lbracket, rbracket, encoding,
-                             colormap, time_format, float_precision, nullval,
+                             colormap, date_time_format, float_precision, nullval,
                              decimal_sep, thousands_sep, boolean_styles):
     subs = [format_value(type(sval), sval, encoding=encoding, colormap=colormap,
-                         time_format=time_format, float_precision=float_precision,
+                         date_time_format=date_time_format, float_precision=float_precision,
                          nullval=nullval, quote=True, decimal_sep=decimal_sep,
                          thousands_sep=thousands_sep, boolean_styles=boolean_styles)
             for sval in val]
@@ -265,40 +295,38 @@
 
 
 @formatter_for('list')
-def format_value_list(val, encoding, colormap, time_format, float_precision, nullval,
+def format_value_list(val, encoding, colormap, date_time_format, float_precision, nullval,
                       decimal_sep, thousands_sep, boolean_styles, **_):
     return format_simple_collection(val, '[', ']', encoding, colormap,
-                                    time_format, float_precision, nullval,
+                                    date_time_format, float_precision, nullval,
                                     decimal_sep, thousands_sep, boolean_styles)
 
 
 @formatter_for('tuple')
-def format_value_tuple(val, encoding, colormap, time_format, float_precision, nullval,
+def format_value_tuple(val, encoding, colormap, date_time_format, float_precision, nullval,
                        decimal_sep, thousands_sep, boolean_styles, **_):
     return format_simple_collection(val, '(', ')', encoding, colormap,
-                                    time_format, float_precision, nullval,
+                                    date_time_format, float_precision, nullval,
                                     decimal_sep, thousands_sep, boolean_styles)
 
 
 @formatter_for('set')
-def format_value_set(val, encoding, colormap, time_format, float_precision, nullval,
+def format_value_set(val, encoding, colormap, date_time_format, float_precision, nullval,
                      decimal_sep, thousands_sep, boolean_styles, **_):
     return format_simple_collection(sorted(val), '{', '}', encoding, colormap,
-                                    time_format, float_precision, nullval,
+                                    date_time_format, float_precision, nullval,
                                     decimal_sep, thousands_sep, boolean_styles)
 formatter_for('frozenset')(format_value_set)
-# This code is used by cqlsh (bundled driver version 2.7.2 using sortedset),
-# and the dtests, which use whichever driver on the machine, i.e. 3.0.0 (SortedSet)
-formatter_for('SortedSet')(format_value_set)
 formatter_for('sortedset')(format_value_set)
+formatter_for('SortedSet')(format_value_set)
 
 
 @formatter_for('dict')
-def format_value_map(val, encoding, colormap, time_format, float_precision, nullval,
+def format_value_map(val, encoding, colormap, date_time_format, float_precision, nullval,
                      decimal_sep, thousands_sep, boolean_styles, **_):
     def subformat(v):
         return format_value(type(v), v, encoding=encoding, colormap=colormap,
-                            time_format=time_format, float_precision=float_precision,
+                            date_time_format=date_time_format, float_precision=float_precision,
                             nullval=nullval, quote=True, decimal_sep=decimal_sep,
                             thousands_sep=thousands_sep, boolean_styles=boolean_styles)
 
@@ -319,13 +347,13 @@
 formatter_for('OrderedMapSerializedKey')(format_value_map)
 
 
-def format_value_utype(val, encoding, colormap, time_format, float_precision, nullval,
+def format_value_utype(val, encoding, colormap, date_time_format, float_precision, nullval,
                        decimal_sep, thousands_sep, boolean_styles, **_):
     def format_field_value(v):
         if v is None:
             return colorme(nullval, colormap, 'error')
         return format_value(type(v), v, encoding=encoding, colormap=colormap,
-                            time_format=time_format, float_precision=float_precision,
+                            date_time_format=date_time_format, float_precision=float_precision,
                             nullval=nullval, quote=True, decimal_sep=decimal_sep,
                             thousands_sep=thousands_sep, boolean_styles=boolean_styles)
 
diff --git a/pylib/cqlshlib/helptopics.py b/pylib/cqlshlib/helptopics.py
index abc2567..279063b 100644
--- a/pylib/cqlshlib/helptopics.py
+++ b/pylib/cqlshlib/helptopics.py
@@ -14,813 +14,160 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-from .cql3handling import simple_cql_types
 
-
-class CQLHelpTopics(object):
+class CQL3HelpTopics(object):
 
     def get_help_topics(self):
         return [t[5:] for t in dir(self) if t.startswith('help_')]
 
-    def print_help_topic(self, topic):
-        getattr(self, 'help_' + topic.lower())()
+    def get_help_topic(self, topic):
+        return getattr(self, 'help_' + topic.lower())()
 
     def help_types(self):
-        print "\n        CQL types recognized by this version of cqlsh:\n"
-        for t in simple_cql_types:
-            print '          ' + t
-        print """
-        For information on the various recognizable input formats for these
-        types, or on controlling the formatting of cqlsh query output, see
-        one of the following topics:
+        return 'types'
 
-          HELP TIMESTAMP_INPUT
-          HELP BLOB_INPUT
-          HELP UUID_INPUT
-          HELP BOOLEAN_INPUT
+    def help_timestamp(self):
+        return 'usingtimestamps'
 
-          HELP TEXT_OUTPUT
-          HELP TIMESTAMP_OUTPUT
-        """
+    def help_date(self):
+        return 'usingdates'
 
-    def help_timestamp_input(self):
-        print """
-        Timestamp input
+    def help_time(self):
+        return 'usingtime'
 
-        CQL supports any of the following ISO 8601 formats for timestamp
-        specification:
+    def help_blob(self):
+        return 'constants'
 
-          yyyy-mm-dd HH:mm
-          yyyy-mm-dd HH:mm:ss
-          yyyy-mm-dd HH:mmZ
-          yyyy-mm-dd HH:mm:ssZ
-          yyyy-mm-dd'T'HH:mm
-          yyyy-mm-dd'T'HH:mmZ
-          yyyy-mm-dd'T'HH:mm:ss
-          yyyy-mm-dd'T'HH:mm:ssZ
-          yyyy-mm-dd
-          yyyy-mm-ddZ
+    def help_uuid(self):
+        return 'constants'
 
-        The Z in these formats refers to an RFC-822 4-digit time zone,
-        expressing the time zone's difference from UTC. For example, a
-        timestamp in Pacific Standard Time might be given thus:
+    def help_boolean(self):
+        return 'constants'
 
-          2012-01-20 16:14:12-0800
+    def help_int(self):
+        return 'constants'
 
-        If no time zone is supplied, the current time zone for the Cassandra
-        server node will be used.
-        """
+    def help_counter(self):
+        return 'counters'
 
-    def help_blob_input(self):
-        print """
-        Blob input
-
-        CQL blob data must be specified in a string literal as hexidecimal
-        data. Example: to store the ASCII values for the characters in the
-        string "CQL", use '43514c'.
-        """
-
-    def help_uuid_input(self):
-        print """
-        UUID input
-
-        UUIDs may be specified in CQL using 32 hexidecimal characters,
-        split up using dashes in the standard UUID format:
-
-          XXXXXXXX-XXXX-XXXX-XXXX-XXXXXXXXXXXX
-        """
-
-    def help_boolean_input(self):
-        print """
-        Boolean input
-
-        CQL accepts the strings 'true' and 'false' (case insensitive)
-        as input for boolean types.
-        """
-
-    def help_timestamp_output(self):
-        print """
-        Timestamp output
-
-        Cqlsh will display timestamps in the following format by default:
-
-          yyyy-mm-dd HH:mm:ssZ
-
-        which is a format acceptable as CQL timestamp input as well.
-        The output format can be changed by setting 'time_format' property
-        in the [ui] section of .cqlshrc file.
-        """
-
-    def help_text_output(self):
-        print """
-        Textual output
-
-        When control characters, or other characters which can't be encoded
-        in your current locale, are found in values of 'text' or 'ascii'
-        types, it will be shown as a backslash escape. If color is enabled,
-        any such backslash escapes will be shown in a different color from
-        the surrounding text.
-
-        Unicode code points in your data will be output intact, if the
-        encoding for your locale is capable of decoding them. If you prefer
-        that non-ascii characters be shown with Python-style "\\uABCD"
-        escape sequences, invoke cqlsh with an ASCII locale (for example,
-        by setting the $LANG environment variable to "C").
-        """
-    help_ascii_output = help_text_output
-
-    def help_create_index(self):
-        print """
-        CREATE INDEX [<indexname>] ON <cfname> ( <colname> );
-
-        A CREATE INDEX statement is used to create a new, automatic secondary
-        index on the given CQL table, for the named column. A name for the
-        index itself can be specified before the ON keyword, if desired. A
-        single column name must be specified inside the parentheses. It is not
-        necessary for the column to exist on any current rows (Cassandra is
-        schema-optional), but the column must already have a type (specified
-        during the CREATE TABLE, or added afterwards with ALTER TABLE).
-        """
-
-    def help_drop(self):
-        print """
-        There are different variants of DROP. For more information, see
-        one of the following:
-
-          HELP DROP_KEYSPACE;
-          HELP DROP_TABLE;
-          HELP DROP_INDEX;
-        """
-
-    def help_drop_keyspace(self):
-        print """
-        DROP KEYSPACE <keyspacename>;
-
-        A DROP KEYSPACE statement results in the immediate, irreversible
-        removal of a keyspace, including all column families in it, and all
-        data contained in those column families.
-        """
-
-    def help_drop_table(self):
-        print """
-        DROP TABLE <tablename>;
-
-        A DROP TABLE statement results in the immediate, irreversible
-        removal of a CQL table and the underlying column family, including all
-        data contained in it.
-        """
-    help_drop_columnfamily = help_drop_table
-
-    def help_drop_index(self):
-        print """
-        DROP INDEX <indexname>;
-
-        A DROP INDEX statement is used to drop an existing secondary index.
-        """
-
-    def help_truncate(self):
-        print """
-        TRUNCATE <tablename>;
-
-        TRUNCATE accepts a single argument for the table name, and permanently
-        removes all data from it.
-        """
-
-    def help_create(self):
-        print """
-        There are different variants of CREATE. For more information, see
-        one of the following:
-
-          HELP CREATE_KEYSPACE;
-          HELP CREATE_TABLE;
-          HELP CREATE_INDEX;
-        """
+    def help_text(self):
+        return 'constants'
+    help_ascii = help_text
 
     def help_use(self):
-        print """
-        USE <keyspacename>;
-
-        Tells cqlsh and the connected Cassandra instance that you will be
-        working in the given keyspace. All subsequent operations on tables
-        or indexes will be in the context of this keyspace, unless otherwise
-        specified, until another USE command is issued or the connection
-        terminates.
-
-        As always, when a keyspace name does not work as a normal identifier or
-        number, it can be quoted using single quotes (CQL 2) or double quotes
-        (CQL 3).
-        """
-
-    def help_create_table(self):
-        print """
-        CREATE TABLE <cfname> ( <colname> <type> PRIMARY KEY [,
-                                <colname> <type> [, ...]] )
-               [WITH <optionname> = <val> [AND <optionname> = <val> [...]]];
-
-        CREATE TABLE statements create a new CQL table under the current
-        keyspace. Valid table names are strings of alphanumeric characters and
-        underscores, which begin with a letter.
-
-        Each table requires a primary key, which will correspond to the
-        underlying columnfamily key and key validator. It's important to
-        note that the key type you use must be compatible with the partitioner
-        in use. For example, OrderPreservingPartitioner and
-        CollatingOrderPreservingPartitioner both require UTF-8 keys.
-
-        In cql3 mode, a table can have multiple columns composing the primary
-        key (see HELP COMPOUND_PRIMARY_KEYS).
-
-        For more information, see one of the following:
-
-          HELP CREATE_TABLE_TYPES;
-          HELP CREATE_TABLE_OPTIONS;
-        """
-    help_create_columnfamily = help_create_table
-
-    def help_compound_primary_keys(self):
-        print """
-        CREATE TABLE <cfname> ( <partition_key> <type>, <clustering_key1> type, <clustering_key2> type,
-                                [, ...]], PRIMARY KEY (<partition_key>, <clustering_key1>, <clustering_key2>);
-
-        CREATE TABLE allows a primary key composed of multiple columns. When this is the case, specify
-        the columns that take part in the compound key after all columns have been specified.
-
-        , PRIMARY KEY( <key1>, <key2>, ... )
-
-        The partitioning key itself can be a compound key, in which case the first element of the PRIMARY KEY
-        phrase should be parenthesized, as
-
-        PRIMARY KEY ((<partition_key_part1>, <partition_key_part2>), <clustering_key>)
-        """
-
-    def help_create_table_types(self):
-        print """
-        CREATE TABLE: Specifying column types
-
-          CREATE ... (KEY <type> PRIMARY KEY,
-                      othercol <type>) ...
-
-        It is possible to assign columns a type during table creation. Columns
-        configured with a type are validated accordingly when a write occurs,
-        and intelligent CQL drivers and interfaces will be able to decode the
-        column values correctly when receiving them. Column types are specified
-        as a parenthesized, comma-separated list of column term and type pairs.
-        See HELP TYPES; for the list of recognized types.
-        """
-    help_create_columnfamily_types = help_create_table_types
-
-    def help_create_table_options(self):
-        print """
-        CREATE TABLE: Specifying columnfamily options
-
-          CREATE TABLE blah (...)
-             WITH optionname = val AND otheroption = val2;
-
-        A number of optional keyword arguments can be supplied to control the
-        configuration of a new CQL table, such as the size of the associated
-        row and key caches for the underlying Cassandra columnfamily. Consult
-        your CQL reference for the complete list of options and possible
-        values.
-        """
-    help_create_columnfamily_options = help_create_table_options
-
-    def help_alter_alter(self):
-        print """
-        ALTER TABLE: altering existing typed columns
-
-          ALTER TABLE addamsFamily ALTER lastKnownLocation TYPE uuid;
-
-        ALTER TABLE ... ALTER changes the expected storage type for a column.
-        The column must already have a type in the column family metadata. The
-        column may or may not already exist in current rows-- but be aware that
-        no validation of existing data is done. The bytes stored in values for
-        that column will remain unchanged, and if existing data is not
-        deserializable according to the new type, this may cause your CQL
-        driver or interface to report errors.
-        """
-
-    def help_alter_add(self):
-        print """
-        ALTER TABLE: adding a typed column
-
-          ALTER TABLE addamsFamily ADD gravesite varchar;
-
-        The ALTER TABLE ... ADD variant adds a typed column to a column
-        family. The column must not already have a type in the column family
-        metadata. See the warnings on HELP ALTER_ALTER regarding the lack of
-        validation of existing data; they apply here as well.
-        """
-
-    def help_alter_drop(self):
-        print """
-        ALTER TABLE: dropping a typed column
-
-          ALTER TABLE addamsFamily DROP gender;
-
-        An ALTER TABLE ... DROP statement removes the type of a column
-        from the column family metadata. Note that this does _not_ remove the
-        column from current rows; it just removes the metadata saying that the
-        bytes stored under that column are expected to be deserializable
-        according to a certain type.
-        """
-
-    def help_alter_with(self):
-        print """
-        ALTER TABLE: changing column family properties
-
-          ALTER TABLE addamsFamily WITH comment = 'Glad to be here!'
-                                    AND read_repair_chance = 0.2;
-
-        An ALTER TABLE ... WITH statement makes adjustments to the
-        table properties, as defined when the table was created (see
-        HELP CREATE_TABLE_OPTIONS and your Cassandra documentation for
-        information about the supported parameter names and values).
-        """
-
-    def help_delete_columns(self):
-        print """
-        DELETE: specifying columns
-
-          DELETE col1, col2, col3 FROM ...
-
-        Following the DELETE keyword is an optional comma-delimited list of
-        column name terms. When no column names are given, the remove applies
-        to the entire row(s) matched by the WHERE clause.
-
-        When column names do not parse as valid CQL identifiers, they can be
-        quoted in single quotes (CQL 2) or double quotes (CQL 3).
-        """
-
-    def help_delete_where(self):
-        print """
-        DELETE: specifying rows
-
-          DELETE ... WHERE keycol = 'some_key_value';
-          DELETE ... WHERE keycol1 = 'val1' AND keycol2 = 'val2';
-          DELETE ... WHERE keycol IN (key1, key2);
-
-        The WHERE clause is used to determine to which row(s) a DELETE
-        applies. The first form allows the specification of a precise row
-        by specifying a particular primary key value (if the primary key has
-        multiple columns, values for each must be given). The second form
-        allows a list of key values to be specified using the IN operator
-        and a parenthesized list of comma-delimited key values.
-        """
-
-    def help_update_set(self):
-        print """
-        UPDATE: Specifying Columns and Row
-
-          UPDATE ... SET name1 = value1, name2 = value2
-                   WHERE <key> = keyname;
-          UPDATE ... SET name1 = value1, name2 = value2
-                   WHERE <key> IN ('<key1>', '<key2>', ...)
-
-        Rows are created or updated by supplying column names and values in
-        term assignment format.  Multiple columns can be set by separating the
-        name/value pairs using commas.
-        """
-
-    def help_update_counters(self):
-        print """
-        UPDATE: Updating Counter Columns
-
-          UPDATE ... SET name1 = name1 + <value> ...
-          UPDATE ... SET name1 = name1 - <value> ...
-
-        Counter columns can be incremented or decremented by an arbitrary
-        numeric value though the assignment of an expression that adds or
-        subtracts the value.
-        """
-
-    def help_update_where(self):
-        print """
-        UPDATE: Selecting rows to update
-
-          UPDATE ... WHERE <keyname> = <keyval>;
-          UPDATE ... WHERE <keyname> IN (<keyval1>, <keyval2>, ...);
-          UPDATE ... WHERE <keycol1> = <keyval1> AND <keycol2> = <keyval2>;
-
-        Each update statement requires a precise set of keys to be specified
-        using a WHERE clause.
-
-        If the table's primary key consists of multiple columns, an explicit
-        value must be given for each for the UPDATE statement to make sense.
-        """
-
-    def help_select_table(self):
-        print """
-        SELECT: Specifying Table
-
-          SELECT ... FROM [<keyspace>.]<tablename> ...
-
-        The FROM clause is used to specify the CQL table applicable to a SELECT
-        query. The keyspace in which the table exists can optionally be
-        specified along with the table name, separated by a dot (.). This will
-        not change the current keyspace of the session (see HELP USE).
-        """
-    help_select_columnfamily = help_select_table
-
-    def help_select_where(self):
-        print """
-        SELECT: Filtering rows
-
-          SELECT ... WHERE <key> = keyname AND name1 = value1
-          SELECT ... WHERE <key> >= startkey and <key> =< endkey AND name1 = value1
-          SELECT ... WHERE <key> IN ('<key>', '<key>', '<key>', ...)
-
-        The WHERE clause provides for filtering the rows that appear in
-        results.  The clause can filter on a key name, or range of keys, and in
-        the case of indexed columns, on column values.  Key filters are
-        specified using the KEY keyword or key alias name, a relational
-        operator (one of =, >, >=, <, and <=), and a term value.  When terms
-        appear on both sides of a relational operator it is assumed the filter
-        applies to an indexed column. With column index filters, the term on
-        the left of the operator is the name, the term on the right is the
-        value to filter _on_.
-
-        Note: The greater-than and less-than operators (> and <) result in key
-        ranges that are inclusive of the terms. There is no supported notion of
-        "strictly" greater-than or less-than; these operators are merely
-        supported as aliases to >= and <=.
-        """
-
-    def help_select_limit(self):
-        print """
-        SELECT: Limiting results
-
-          SELECT ... WHERE <clause> [LIMIT n] ...
-
-        Limiting the number of rows returned can be achieved by adding the
-        LIMIT option to a SELECT expression. LIMIT defaults to 10,000 when left
-        unset.
-        """
-
-
-class CQL3HelpTopics(CQLHelpTopics):
-
-    def help_create_keyspace(self):
-        print """
-        CREATE KEYSPACE <ksname>
-            WITH replication = {'class':'<strategy>' [,'<option>':<val>]};
-
-        The CREATE KEYSPACE statement creates a new top-level namespace (aka
-        "keyspace"). Valid names are any string constructed of alphanumeric
-        characters and underscores. Names which do not work as valid
-        identifiers or integers should be quoted as string literals. Properties
-        such as replication strategy and count are specified during creation
-        as key-value pairs in the 'replication' map:
-
-          class [required]: The name of the replication strategy class
-          which should be used for the new keyspace. Some often-used classes
-          are SimpleStrategy and NetworkTopologyStrategy.
-
-          other options [optional]: Most strategies require additional arguments
-          which can be supplied as key-value pairs in the 'replication' map.
-
-        Examples:
-
-          To create a keyspace with NetworkTopologyStrategy and strategy option of "DC1"
-          with a value of "1" and "DC2" with a value of "2" you would use
-          the following statement:
-            CREATE KEYSPACE <ksname>
-                WITH replication = {'class':'NetworkTopologyStrategy', 'DC1':1, 'DC2':2};
-
-         To create a keyspace with SimpleStrategy and "replication_factor" option
-         with a value of "3" you would use this statement:
-            CREATE KEYSPACE <ksname>
-                WITH replication = {'class':'SimpleStrategy', 'replication_factor':3};
-        """
-
-    def help_begin(self):
-        print """
-        BEGIN [UNLOGGED|COUNTER] BATCH [USING TIMESTAMP <timestamp>]
-          <insert or update or delete statement> ;
-          [ <another insert or update or delete statement ;
-            [...]]
-        APPLY BATCH;
-
-        BATCH supports setting a client-supplied optional global timestamp
-        which will be used for each of the operations included in the batch.
-
-        Only data modification statements (specifically, UPDATE, INSERT,
-        and DELETE) are allowed in a BATCH statement. BATCH is _not_ an
-        analogue for SQL transactions.
-
-        _NOTE: Counter mutations are allowed only within COUNTER batches._
-
-        _NOTE: While there are no isolation guarantees, UPDATE queries are
-        atomic within a given record._
-        """
-    help_apply = help_begin
-
-    def help_select(self):
-        print """
-        SELECT <selectExpr>
-          FROM [<keyspace>.]<table>
-            [WHERE <clause>]
-            [ORDER BY <colname> [DESC]]
-            [LIMIT m];
-
-        SELECT is used to read one or more records from a CQL table. It returns
-        a set of rows matching the selection criteria specified.
-
-        For more information, see one of the following:
-
-          HELP SELECT_EXPR
-          HELP SELECT_TABLE
-          HELP SELECT_WHERE
-          HELP SELECT_LIMIT
-        """
-
-    def help_delete(self):
-        print """
-        DELETE [<col1> [, <col2>, ...] FROM [<keyspace>.]<tablename>
-               [USING TIMESTAMP <timestamp>]
-            WHERE <keyname> = <keyvalue>;
-
-        A DELETE is used to perform the removal of one or more columns from one
-        or more rows. Each DELETE statement requires a precise set of row keys
-        to be specified using a WHERE clause and the KEY keyword or key alias.
-
-        For more information, see one of the following:
-
-          HELP DELETE_USING
-          HELP DELETE_COLUMNS
-          HELP DELETE_WHERE
-        """
-
-    def help_delete_using(self):
-        print """
-        DELETE: the USING clause
-
-          DELETE ... USING TIMESTAMP <timestamp>;
-
-        <timestamp> defines the optional timestamp for the new tombstone
-        record. It must be an integer. Cassandra timestamps are generally
-        specified using milliseconds since the Unix epoch (1970-01-01 00:00:00
-        UTC).
-        """
-
-    def help_update(self):
-        print """
-        UPDATE [<keyspace>.]<columnFamily>
-                              [USING [TIMESTAMP <timestamp>]
-                                [AND TTL <timeToLive>]]
-               SET name1 = value1, name2 = value2 WHERE <keycol> = keyval
-               [IF EXISTS];
-
-        An UPDATE is used to write one or more columns to a record in a table.
-        No results are returned. The record's primary key must be completely
-        and uniquely specified; that is, if the primary key includes multiple
-        columns, all must be explicitly given in the WHERE clause.
-
-        Statements begin with the UPDATE keyword followed by the name of the
-        table to be updated.
-
-        For more information, see one of the following:
-
-          HELP UPDATE_USING
-          HELP UPDATE_SET
-          HELP UPDATE_COUNTERS
-          HELP UPDATE_WHERE
-        """
-
-    def help_update_using(self):
-        print """
-        UPDATE: the USING clause
-
-          UPDATE ... USING TIMESTAMP <timestamp>;
-          UPDATE ... USING TTL <timeToLive>;
-
-        The USING clause allows setting of certain query and data parameters.
-        If multiple parameters need to be set, these may be joined using AND.
-        Example:
-
-          UPDATE ... USING TTL 43200 AND TIMESTAMP 1351620509603
-
-        <timestamp> defines the optional timestamp for the new column value(s).
-        It must be an integer. Cassandra timestamps are generally specified
-        using milliseconds since the Unix epoch (1970-01-01 00:00:00 UTC).
-
-        <timeToLive> defines the optional time to live (TTL) in seconds for the
-        new column value(s). It must be an integer.
-        """
+        return 'useStmt'
 
     def help_insert(self):
-        print """
-        INSERT INTO [<keyspace>.]<tablename>
-                    ( <colname1>, <colname2> [, <colname3> [, ...]] )
-               VALUES ( <colval1>, <colval2> [, <colval3> [, ...]] )
-               [USING TIMESTAMP <timestamp>]
-                 [AND TTL <timeToLive>];
+        return 'insertStmt'
 
-        An INSERT is used to write one or more columns to a record in a
-        CQL table. No results are returned.
+    def help_update(self):
+        return 'updateStmt'
 
-        Values for all component columns in the table's primary key must
-        be given. Also, there must be at least one non-primary-key column
-        specified (Cassandra rows are not considered to exist with only
-        a key and no associated columns).
+    def help_delete(self):
+        return 'deleteStmt'
 
-        Unlike in SQL, the semantics of INSERT and UPDATE are identical.
-        In either case a record is created if none existed before, and
-        udpated when it does. For more information, see one of the
-        following:
+    def help_select(self):
+        return 'selectStmt'
 
-          HELP UPDATE
-          HELP UPDATE_USING
-        """
+    def help_json(self):
+        return 'json'
 
-    def help_select_expr(self):
-        print """
-        SELECT: Specifying Columns
+    def help_select_json(self):
+        return 'selectJson'
 
-          SELECT name1, name2, name3 FROM ...
-          SELECT COUNT(*) FROM ...
+    def help_insert_json(self):
+        return 'insertJson'
 
-        The SELECT expression determines which columns will appear in the
-        results and takes the form of a comma separated list of names.
+    def help_batch(self):
+        return 'batchStmt'
+    help_begin = help_batch
+    help_apply = help_batch
 
-        It is worth noting that unlike the projection in a SQL SELECT, there is
-        no guarantee that the results will contain all of the columns
-        specified. This is because Cassandra is schema-less and there are no
-        guarantees that a given column exists.
+    def help_create_keyspace(self):
+        return 'createKeyspaceStmt'
 
-        When the COUNT aggregate function is specified as a column to fetch, a
-        single row will be returned, with a single column named "count" whose
-        value is the number of rows from the pre-aggregation resultset.
+    def help_alter_keyspace(self):
+        return 'alterKeyspaceStmt'
 
-        Currently, COUNT is the only function supported by CQL.
-        """
+    def help_drop_keyspace(self):
+        return 'dropKeyspaceStmt'
 
-    def help_alter_drop(self):
-        print """
-        ALTER TABLE: dropping a typed column
+    def help_create_table(self):
+        return 'createTableStmt'
+    help_create_columnfamily = help_create_table
 
-          ALTER TABLE addamsFamily DROP gender;
+    def help_alter_table(self):
+        return 'alterTableStmt'
 
-        An ALTER TABLE ... DROP statement removes the type of a column
-        from the column family metadata. Dropped columns will immediately
-        become unavailable in the queries and will not be included in
-        compacted sstables in the future. If a column is readded, queries
-        won't return values written before the column was last dropped.
-        It is assumed that timestamps represent actual time, so if this
-        is not your case, you should NOT readd previously dropped columns.
-        Columns can't be dropped from tables defined with COMPACT STORAGE.
-        """
+    def help_drop_table(self):
+        return 'dropTableStmt'
+    help_drop_columnfamily = help_drop_table
 
-    def help_create(self):
-        super(CQL3HelpTopics, self).help_create()
-        print "          HELP CREATE_USER;\n"
+    def help_create_index(self):
+        return 'createIndexStmt'
 
-    def help_alter(self):
-        print """
-        ALTER TABLE <tablename> ALTER <columnname> TYPE <type>;
-        ALTER TABLE <tablename> ADD <columnname> <type>;
-        ALTER TABLE <tablename> RENAME <columnname> TO <columnname>
-            [AND <columnname> TO <columnname>]
-        ALTER TABLE <tablename> WITH <optionname> = <val> [AND <optionname> = <val> [...]];
+    def help_drop_index(self):
+        return 'dropIndexStmt'
 
-        An ALTER statement is used to manipulate table metadata. It allows you
-        to add new typed columns, drop existing columns, change the data
-        storage type of existing columns, or change table properties.
-        No results are returned.
+    def help_truncate(self):
+        return 'truncateStmt'
 
-        See one of the following for more information:
+    def help_create_type(self):
+        return 'createTypeStmt'
 
-          HELP ALTER_ALTER;
-          HELP ALTER_ADD;
-          HELP ALTER_DROP;
-          HELP ALTER_RENAME;
-          HELP ALTER_WITH;
-        """
+    def help_alter_type(self):
+        return 'alterTypeStmt'
 
-    def help_alter_rename(self):
-        print """
-        ALTER TABLE: renaming a column
+    def help_drop_type(self):
+        return 'dropTypeStmt'
 
-          ALTER TABLE <tablename> RENAME <columnname> TO <columnname>
-              [AND <columnname> TO <columnname>]
+    def help_create_function(self):
+        return 'createFunctionStmt'
 
-        The ALTER TABLE ... RENAME variant renames a typed column in a column
-        family.
-        """
+    def help_drop_function(self):
+        return 'dropFunctionStmt'
 
-    def help_drop(self):
-        super(CQL3HelpTopics, self).help_drop()
-        print "          HELP DROP_USER;\n"
+    def help_functions(self):
+        return 'functions'
 
-    def help_list(self):
-        print """
-        There are different variants of LIST. For more information, see
-        one of the following:
+    def help_create_aggregate(self):
+        return 'createAggregateStmt'
 
-          HELP LIST_USERS;
-          HELP LIST_PERMISSIONS;
-        """
+    def help_drop_aggregate(self):
+        return 'dropAggregateStmt'
+
+    def help_aggregates(self):
+        return 'aggregates'
+
+    def help_create_trigger(self):
+        return 'createTriggerStmt'
+
+    def help_drop_trigger(self):
+        return 'dropTriggerStmt'
+
+    def help_keywords(self):
+        return 'appendixA'
 
     def help_create_user(self):
-        print """
-        CREATE USER <username> [WITH PASSWORD 'password'] [NOSUPERUSER | SUPERUSER];
-
-        CREATE USER creates a new Cassandra user account.
-        Only superusers can issue CREATE USER requests.
-        To create a superuser account use SUPERUSER option (NOSUPERUSER is the default).
-
-        WITH PASSWORD clause should only be used with password-based authenticators,
-        e.g. PasswordAuthenticator, SimpleAuthenticator.
-        """
+        return 'createUserStmt'
 
     def help_alter_user(self):
-        print """
-        ALTER USER <username> [WITH PASSWORD 'password'] [NOSUPERUSER | SUPERUSER];
-
-        Use ALTER USER to change a user's superuser status and/or password (only
-        with password-based authenticators).
-        Superusers can change a user's password or superuser status (except their own).
-        Users cannot change their own superuser status. Ordinary users can only change their
-        password (if the configured authenticator is password-based).
-        """
+        return 'alterUserStmt'
 
     def help_drop_user(self):
-        print """
-        DROP USER <username>;
-
-        DROP USER removes an existing user. You have to be logged in as a superuser
-        to issue a DROP USER statement. A user cannot drop themselves.
-        """
+        return 'dropUserStmt'
 
     def help_list_users(self):
-        print """
-        LIST USERS;
-
-        List existing users and their superuser status.
-        """
-
-    def help_grant(self):
-        print """
-        GRANT (<permission> [PERMISSION] | ALL [PERMISSIONS])
-                  ON ALL KEYSPACES
-                   | KEYSPACE <keyspace>
-                   | [TABLE] [<keyspace>.]<table>
-                  TO <username>
-
-        Grant the specified permission (or all permissions) on a resource
-        to a user.
-
-        To be able to grant a permission on some resource you have to
-        have that permission yourself and also AUTHORIZE permission on it,
-        or on one of its parent resources.
-
-        See HELP PERMISSIONS for more info on the available permissions.
-        """
-
-    def help_revoke(self):
-        print """
-        REVOKE (<permission> [PERMISSION] | ALL [PERMISSIONS])
-                  ON ALL KEYSPACES
-                   | KEYSPACE <keyspace>
-                   | [TABLE] [<keyspace>.]<table>
-                  FROM <username>
-
-        Revokes the specified permission (or all permissions) on a resource
-        from a user.
-
-        To be able to revoke a permission on some resource you have to
-        have that permission yourself and also AUTHORIZE permission on it,
-        or on one of its parent resources.
-
-        See HELP PERMISSIONS for more info on the available permissions.
-        """
-
-    def help_list_permissions(self):
-        print """
-        LIST (<permission> [PERMISSION] | ALL [PERMISSIONS])
-                  [ON ALL KEYSPACES
-                    | KEYSPACE <keyspace>
-                    | [TABLE] [<keyspace>.]<table>]
-                  [OF <username>]
-                  [NORECURSIVE]
-
-        Omitting ON <resource> part will list permissions on ALL KEYSPACES,
-        every keyspace and table.
-        Omitting OF <username> part will list permissions of all users.
-        Omitting NORECURSIVE specifier will list permissions of the resource
-        and all its parents (table, table's keyspace and ALL KEYSPACES).
-
-        See HELP PERMISSIONS for more info on the available permissions.
-        """
+        return 'listUsersStmt'
 
     def help_permissions(self):
-        print """
-        PERMISSIONS
+        return 'permissions'
 
-        Cassandra has 6 permissions:
-          ALTER: required for ALTER KEYSPCE, ALTER TABLE, CREATE INDEX, DROP INDEX
-          AUTHORIZE: required for GRANT, REVOKE
-          CREATE: required for CREATE KEYSPACE, CREATE TABLE
-          DROP: required for DROP KEYSPACE, DROP TABLE
-          MODIFY: required for INSERT, DELETE, UPDATE, TRUNCATE
-          SELECT: required for SELECT
-        """
+    def help_list_permissions(self):
+        return 'listPermissionsStmt'
+
+    def help_grant(self):
+        return 'grantRoleStmt'
+
+    def help_revoke(self):
+        return 'revokeRoleStmt'
diff --git a/pylib/cqlshlib/sslhandling.py b/pylib/cqlshlib/sslhandling.py
index abcae16..8765ffa 100644
--- a/pylib/cqlshlib/sslhandling.py
+++ b/pylib/cqlshlib/sslhandling.py
@@ -22,7 +22,7 @@
 
 def ssl_settings(host, config_file, env=os.environ):
     """
-    Function wcich generates SSL setting for cassandra.Cluster
+    Function which generates SSL setting for cassandra.Cluster
 
     Params:
     * host .........: hostname of Cassandra node.
diff --git a/pylib/cqlshlib/test/__init__.py b/pylib/cqlshlib/test/__init__.py
index 31f66f3..ba8f373 100644
--- a/pylib/cqlshlib/test/__init__.py
+++ b/pylib/cqlshlib/test/__init__.py
@@ -14,7 +14,7 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-from .cassconnect import create_test_db, remove_test_db
+from .cassconnect import create_db, remove_db
 
-setUp = create_test_db
-tearDown = remove_test_db
+setUp = create_db
+tearDown = remove_db
diff --git a/pylib/cqlshlib/test/basecase.py b/pylib/cqlshlib/test/basecase.py
index 5600f1a..d393769 100644
--- a/pylib/cqlshlib/test/basecase.py
+++ b/pylib/cqlshlib/test/basecase.py
@@ -18,7 +18,7 @@
 import sys
 import logging
 from itertools import izip
-from os.path import dirname, join, normpath, islink
+from os.path import dirname, join, normpath
 
 cqlshlog = logging.getLogger('test_cqlsh')
 
@@ -31,20 +31,15 @@
     import unittest
 
 rundir = dirname(__file__)
-path_to_cqlsh = normpath(join(rundir, '..', '..', '..', 'bin', 'cqlsh'))
+cqlshdir = normpath(join(rundir, '..', '..', '..', 'bin'))
+path_to_cqlsh = normpath(join(cqlshdir, 'cqlsh.py'))
 
-# symlink a ".py" file to cqlsh main script, so we can load it as a module
-modulepath = join(rundir, 'cqlsh.py')
-try:
-    if islink(modulepath):
-        os.unlink(modulepath)
-except OSError:
-    pass
-os.symlink(path_to_cqlsh, modulepath)
+sys.path.append(cqlshdir)
 
-sys.path.append(rundir)
 import cqlsh
 cql = cqlsh.cassandra.cluster.Cluster
+policy = cqlsh.cassandra.policies.RoundRobinPolicy()
+quote_name = cqlsh.cassandra.metadata.maybe_escape_name
 
 TEST_HOST = os.environ.get('CQL_TEST_HOST', '127.0.0.1')
 TEST_PORT = int(os.environ.get('CQL_TEST_PORT', 9042))
diff --git a/pylib/cqlshlib/test/cassconnect.py b/pylib/cqlshlib/test/cassconnect.py
index a67407b..71f7565 100644
--- a/pylib/cqlshlib/test/cassconnect.py
+++ b/pylib/cqlshlib/test/cassconnect.py
@@ -19,8 +19,7 @@
 import contextlib
 import tempfile
 import os.path
-from .basecase import cql, cqlsh, cqlshlog, TEST_HOST, TEST_PORT, rundir
-from cassandra.metadata import maybe_escape_name
+from .basecase import cql, cqlsh, cqlshlog, TEST_HOST, TEST_PORT, rundir, policy, quote_name
 from .run_cqlsh import run_cqlsh, call_cqlsh
 
 test_keyspace_init = os.path.join(rundir, 'test_keyspace_init.cql')
@@ -28,7 +27,7 @@
 def get_cassandra_connection(cql_version=cqlsh.DEFAULT_CQLVER):
     if cql_version is None:
         cql_version = cqlsh.DEFAULT_CQLVER
-    conn = cql((TEST_HOST,), TEST_PORT, cql_version=cql_version)
+    conn = cql((TEST_HOST,), TEST_PORT, cql_version=cql_version, load_balancing_policy=policy)
     # until the cql lib does this for us
     conn.cql_version = cql_version
     return conn
@@ -38,15 +37,15 @@
 
 TEST_KEYSPACES_CREATED = []
 
-def get_test_keyspace():
-    return TEST_KEYSPACES_CREATED[-1]
+def get_keyspace():
+    return None if len(TEST_KEYSPACES_CREATED) == 0 else TEST_KEYSPACES_CREATED[-1]
 
-def make_test_ks_name():
+def make_ks_name():
     # abuse mktemp to get a quick random-ish name
     return os.path.basename(tempfile.mktemp(prefix='CqlshTests_'))
 
-def create_test_keyspace(cursor):
-    ksname = make_test_ks_name()
+def create_keyspace(cursor):
+    ksname = make_ks_name()
     qksname = quote_name(ksname)
     cursor.execute('''
         CREATE KEYSPACE %s WITH replication =
@@ -58,8 +57,8 @@
 
 def split_cql_commands(source):
     ruleset = cql_rule_set()
-    statements, in_batch = ruleset.cql_split_statements(source)
-    if in_batch:
+    statements, endtoken_escaped = ruleset.cql_split_statements(source)
+    if endtoken_escaped:
         raise ValueError("CQL source ends unexpectedly")
 
     return [ruleset.cql_extract_orig(toks, source) for toks in statements if toks]
@@ -73,13 +72,13 @@
     with open(fname) as f:
         return execute_cql_commands(cursor, f.read())
 
-def create_test_db():
+def create_db():
     with cassandra_cursor(ks=None) as c:
-        k = create_test_keyspace(c)
+        k = create_keyspace(c)
         execute_cql_file(c, test_keyspace_init)
     return k
 
-def remove_test_db():
+def remove_db():
     with cassandra_cursor(ks=None) as c:
         c.execute('DROP KEYSPACE %s' % quote_name(TEST_KEYSPACES_CREATED.pop(-1)))
 
@@ -113,10 +112,12 @@
     """
 
     if ks == '':
-        ks = get_test_keyspace()
+        ks = get_keyspace()
     conn = get_cassandra_connection(cql_version=cql_version)
     try:
         c = conn.connect(ks)
+        # increase default timeout to fix flacky tests, see CASSANDRA-12481
+        c.default_timeout = 60.0
         # if ks is not None:
         #     c.execute('USE %s;' % quote_name(c, ks))
         yield c
@@ -126,19 +127,16 @@
 def cql_rule_set():
     return cqlsh.cql3handling.CqlRuleSet
 
-def quote_name(name):
-    return maybe_escape_name(name)
-
 class DEFAULTVAL: pass
 
 def testrun_cqlsh(keyspace=DEFAULTVAL, **kwargs):
     # use a positive default sentinel so that keyspace=None can be used
     # to override the default behavior
     if keyspace is DEFAULTVAL:
-        keyspace = get_test_keyspace()
+        keyspace = get_keyspace()
     return run_cqlsh(keyspace=keyspace, **kwargs)
 
 def testcall_cqlsh(keyspace=None, **kwargs):
     if keyspace is None:
-        keyspace = get_test_keyspace()
+        keyspace = get_keyspace()
     return call_cqlsh(keyspace=keyspace, **kwargs)
diff --git a/pylib/cqlshlib/test/run_cqlsh.py b/pylib/cqlshlib/test/run_cqlsh.py
index cc929e1..b011df4 100644
--- a/pylib/cqlshlib/test/run_cqlsh.py
+++ b/pylib/cqlshlib/test/run_cqlsh.py
@@ -17,17 +17,28 @@
 # NOTE: this testing tool is *nix specific
 
 import os
+import sys
 import re
-import pty
-import fcntl
 import contextlib
 import subprocess
 import signal
 import math
 from time import time
 from . import basecase
+from os.path import join, normpath
 
-DEFAULT_CQLSH_PROMPT = os.linesep + '(\S+@)?cqlsh(:\S+)?> '
+
+def is_win():
+    return sys.platform in ("cygwin", "win32")
+
+if is_win():
+    from winpty import WinPty
+    DEFAULT_PREFIX = ''
+else:
+    import pty
+    DEFAULT_PREFIX = os.linesep
+
+DEFAULT_CQLSH_PROMPT = DEFAULT_PREFIX + '(\S+@)?cqlsh(:\S+)?> '
 DEFAULT_CQLSH_TERM = 'xterm'
 
 cqlshlog = basecase.cqlshlog
@@ -41,10 +52,6 @@
         os.close(slave)
     os.close(os.open(os.ttyname(1), os.O_RDWR))
 
-def set_nonblocking(fd):
-    flags = fcntl.fcntl(fd, fcntl.F_GETFL)
-    fcntl.fcntl(fd, fcntl.F_SETFL, flags | os.O_NONBLOCK)
-
 @contextlib.contextmanager
 def raising_signal(signum, exc):
     """
@@ -93,12 +100,21 @@
         finally:
             signal.alarm(0)
 
-# setitimer is new in 2.6, but it's still worth supporting, for potentially
-# faster tests because of sub-second resolution on timeouts.
-if hasattr(signal, 'setitimer'):
-    timing_out = timing_out_itimer
+if is_win():
+    try:
+        import eventlet
+    except ImportError, e:
+        sys.exit("evenlet library required to run cqlshlib tests on Windows")
+
+    def timing_out(seconds):
+        return eventlet.Timeout(seconds, TimeoutError)
 else:
-    timing_out = timing_out_alarm
+    # setitimer is new in 2.6, but it's still worth supporting, for potentially
+    # faster tests because of sub-second resolution on timeouts.
+    if hasattr(signal, 'setitimer'):
+        timing_out = timing_out_itimer
+    else:
+        timing_out = timing_out_alarm
 
 def noop(*a):
     pass
@@ -108,6 +124,7 @@
         self.exe_path = path
         self.args = args
         self.tty = bool(tty)
+        self.realtty = self.tty and not is_win()
         if env is None:
             env = {}
         self.env = env
@@ -118,30 +135,35 @@
     def start_proc(self):
         preexec = noop
         stdin = stdout = stderr = None
-        if self.tty:
-            masterfd, slavefd = pty.openpty()
-            preexec = lambda: set_controlling_pty(masterfd, slavefd)
-        else:
-            stdin = stdout = subprocess.PIPE
-            stderr = subprocess.STDOUT
         cqlshlog.info("Spawning %r subprocess with args: %r and env: %r"
                       % (self.exe_path, self.args, self.env))
-        self.proc = subprocess.Popen((self.exe_path,) + tuple(self.args),
-                                     env=self.env, preexec_fn=preexec,
-                                     stdin=stdin, stdout=stdout, stderr=stderr,
-                                     close_fds=False)
-        if self.tty:
+        if self.realtty:
+            masterfd, slavefd = pty.openpty()
+            preexec = (lambda: set_controlling_pty(masterfd, slavefd))
+            self.proc = subprocess.Popen((self.exe_path,) + tuple(self.args),
+                                         env=self.env, preexec_fn=preexec,
+                                         stdin=stdin, stdout=stdout, stderr=stderr,
+                                         close_fds=False)
             os.close(slavefd)
             self.childpty = masterfd
             self.send = self.send_tty
             self.read = self.read_tty
         else:
+            stdin = stdout = subprocess.PIPE
+            stderr = subprocess.STDOUT
+            self.proc = subprocess.Popen((self.exe_path,) + tuple(self.args),
+                                         env=self.env, stdin=stdin, stdout=stdout,
+                                         stderr=stderr, bufsize=0, close_fds=False)
             self.send = self.send_pipe
-            self.read = self.read_pipe
+            if self.tty:
+                self.winpty = WinPty(self.proc.stdout)
+                self.read = self.read_winpty
+            else:
+                self.read = self.read_pipe
 
     def close(self):
         cqlshlog.info("Closing %r subprocess." % (self.exe_path,))
-        if self.tty:
+        if self.realtty:
             os.close(self.childpty)
         else:
             self.proc.stdin.close()
@@ -154,20 +176,24 @@
     def send_pipe(self, data):
         self.proc.stdin.write(data)
 
-    def read_tty(self, blksize):
+    def read_tty(self, blksize, timeout=None):
         return os.read(self.childpty, blksize)
 
-    def read_pipe(self, blksize):
+    def read_pipe(self, blksize, timeout=None):
         return self.proc.stdout.read(blksize)
 
-    def read_until(self, until, blksize=4096, timeout=None, flags=0):
+    def read_winpty(self, blksize, timeout=None):
+        return self.winpty.read(blksize, timeout)
+
+    def read_until(self, until, blksize=4096, timeout=None,
+                   flags=0, ptty_timeout=None):
         if not isinstance(until, re._pattern_type):
             until = re.compile(until, flags)
         got = self.readbuf
         self.readbuf = ''
         with timing_out(timeout):
             while True:
-                val = self.read(blksize)
+                val = self.read(blksize, ptty_timeout)
                 cqlshlog.debug("read %r from subproc" % (val,))
                 if val == '':
                     raise EOFError("'until' pattern %r not found" % (until.pattern,))
@@ -205,15 +231,22 @@
 
 class CqlshRunner(ProcRunner):
     def __init__(self, path=None, host=None, port=None, keyspace=None, cqlver=None,
-                 args=(), prompt=DEFAULT_CQLSH_PROMPT, env=None, **kwargs):
+                 args=(), prompt=DEFAULT_CQLSH_PROMPT, env=None,
+                 win_force_colors=True, tty=True, **kwargs):
         if path is None:
-            path = basecase.path_to_cqlsh
+            cqlsh_bin = 'cqlsh'
+            if is_win():
+                cqlsh_bin = 'cqlsh.bat'
+            path = normpath(join(basecase.cqlshdir, cqlsh_bin))
         if host is None:
             host = basecase.TEST_HOST
         if port is None:
             port = basecase.TEST_PORT
         if env is None:
             env = {}
+        if is_win():
+            env['PYTHONUNBUFFERED'] = '1'
+            env.update(os.environ.copy())
         env.setdefault('TERM', 'xterm')
         env.setdefault('CQLSH_NO_BUNDLED', os.environ.get('CQLSH_NO_BUNDLED', ''))
         env.setdefault('PYTHONPATH', os.environ.get('PYTHONPATH', ''))
@@ -222,8 +255,13 @@
             args += ('--cqlversion', str(cqlver))
         if keyspace is not None:
             args += ('--keyspace', keyspace)
+        if tty and is_win():
+            args += ('--tty',)
+            args += ('--encoding', 'utf-8')
+            if win_force_colors:
+                args += ('--color',)
         self.keyspace = keyspace
-        ProcRunner.__init__(self, path, args=args, env=env, **kwargs)
+        ProcRunner.__init__(self, path, tty=tty, args=args, env=env, **kwargs)
         self.prompt = prompt
         if self.prompt is None:
             self.output_header = ''
@@ -231,7 +269,7 @@
             self.output_header = self.read_to_next_prompt()
 
     def read_to_next_prompt(self):
-        return self.read_until(self.prompt, timeout=10.0)
+        return self.read_until(self.prompt, timeout=10.0, ptty_timeout=3)
 
     def read_up_to_timeout(self, timeout, blksize=4096):
         output = ProcRunner.read_up_to_timeout(self, timeout, blksize=blksize)
@@ -247,7 +285,7 @@
         output = output.replace(' \r', '')
         output = output.replace('\r', '')
         output = output.replace(' \b', '')
-        if self.tty:
+        if self.realtty:
             echo, output = output.split('\n', 1)
             assert echo == cmd, "unexpected echo %r instead of %r" % (echo, cmd)
         try:
@@ -255,7 +293,7 @@
         except ValueError:
             promptline = output
             output = ''
-        assert re.match(self.prompt, '\n' + promptline), \
+        assert re.match(self.prompt, DEFAULT_PREFIX + promptline), \
                 'last line of output %r does not match %r?' % (promptline, self.prompt)
         return output + '\n'
 
diff --git a/pylib/cqlshlib/test/test_cql_parsing.py b/pylib/cqlshlib/test/test_cql_parsing.py
index f88b839..ad60c9b 100644
--- a/pylib/cqlshlib/test/test_cql_parsing.py
+++ b/pylib/cqlshlib/test/test_cql_parsing.py
@@ -17,78 +17,733 @@
 # to configure behavior, define $CQL_TEST_HOST to the destination address
 # for Thrift connections, and $CQL_TEST_PORT to the associated port.
 
-from .basecase import BaseTestCase, cqlsh
-from .cassconnect import get_test_keyspace, testrun_cqlsh, testcall_cqlsh
+from unittest import TestCase
+from operator import itemgetter
 
-class TestCqlParsing(BaseTestCase):
-    def setUp(self):
-        self.cqlsh_runner = testrun_cqlsh(cqlver=cqlsh.DEFAULT_CQLVER, env={'COLUMNS': '100000'})
-        self.cqlsh = self.cqlsh_runner.__enter__()
+from ..cql3handling import CqlRuleSet
 
-    def tearDown(self):
-        pass
 
+class TestCqlParsing(TestCase):
     def test_parse_string_literals(self):
-        pass
+        for n in ["'eggs'", "'Sausage 1'", "'spam\nspam\n\tsausage'", "''"]:
+            self.assertSequenceEqual(tokens_with_types(CqlRuleSet.lex(n)),
+                                     [(n, 'quotedStringLiteral')])
+        self.assertSequenceEqual(tokens_with_types(CqlRuleSet.lex("'eggs'")),
+                                 [("'eggs'", 'quotedStringLiteral')])
+
+        tokens = CqlRuleSet.lex("'spam\nspam\n\tsausage'")
+        tokens = CqlRuleSet.cql_massage_tokens(tokens)
+        self.assertEqual(tokens[0][0], "quotedStringLiteral")
+
+        tokens = CqlRuleSet.lex("'spam\nspam\n")
+        tokens = CqlRuleSet.cql_massage_tokens(tokens)
+        self.assertEqual(tokens[0][0], "unclosedString")
+
+        tokens = CqlRuleSet.lex("'foo bar' 'spam\nspam\n")
+        tokens = CqlRuleSet.cql_massage_tokens(tokens)
+        self.assertEqual(tokens[1][0], "unclosedString")
+
+    def test_parse_pgstring_literals(self):
+        for n in ["$$eggs$$", "$$Sausage 1$$", "$$spam\nspam\n\tsausage$$", "$$$$"]:
+            self.assertSequenceEqual(tokens_with_types(CqlRuleSet.lex(n)),
+                                     [(n, 'pgStringLiteral')])
+        self.assertSequenceEqual(tokens_with_types(CqlRuleSet.lex("$$eggs$$")),
+                                 [("$$eggs$$", 'pgStringLiteral')])
+
+        tokens = CqlRuleSet.lex("$$spam\nspam\n\tsausage$$")
+        tokens = CqlRuleSet.cql_massage_tokens(tokens)
+        # [('pgStringLiteral', '$$spam\nspam\n\tsausage$$', (0, 22))]
+        self.assertEqual(tokens[0][0], "pgStringLiteral")
+
+        tokens = CqlRuleSet.lex("$$spam\nspam\n")
+        tokens = CqlRuleSet.cql_massage_tokens(tokens)
+        # [('unclosedPgString', '$$', (0, 2)), ('identifier', 'spam', (2, 6)), ('identifier', 'spam', (7, 11))]
+        self.assertEqual(tokens[0][0], "unclosedPgString")
+
+        tokens = CqlRuleSet.lex("$$foo bar$$ $$spam\nspam\n")
+        tokens = CqlRuleSet.cql_massage_tokens(tokens)
+        # [('pgStringLiteral', '$$foo bar$$', (0, 11)), ('unclosedPgString', '$$', (12, 14)), ('identifier', 'spam', (14, 18)), ('identifier', 'spam', (19, 23))]
+        self.assertEqual(tokens[0][0], "pgStringLiteral")
+        self.assertEqual(tokens[1][0], "unclosedPgString")
 
     def test_parse_numbers(self):
-        pass
+        for n in ['6', '398', '18018']:
+            self.assertSequenceEqual(tokens_with_types(CqlRuleSet.lex(n)),
+                                     [(n, 'wholenumber')])
 
     def test_parse_uuid(self):
-        pass
+        uuids = ['4feeae80-e9cc-11e4-b571-0800200c9a66',
+                 '7142303f-828f-4806-be9e-7a973da0c3f9',
+                 'dff8d435-9ca0-487c-b5d0-b0fe5c5768a8']
+        for u in uuids:
+            self.assertSequenceEqual(tokens_with_types(CqlRuleSet.lex(u)),
+                                     [(u, 'uuid')])
 
     def test_comments_in_string_literals(self):
-        pass
+        comment_strings = ["'sausage -- comment'",
+                           "'eggs and spam // comment string'",
+                           "'spam eggs sausage and spam /* still in string'"]
+        for s in comment_strings:
+            self.assertSequenceEqual(tokens_with_types(CqlRuleSet.lex(s)),
+                                     [(s, 'quotedStringLiteral')])
 
     def test_colons_in_string_literals(self):
-        pass
+        comment_strings = ["'Movie Title: The Movie'",
+                           "':a:b:c:'",
+                           "'(>>=) :: (Monad m) => m a -> (a -> m b) -> m b'"]
+        for s in comment_strings:
+            self.assertSequenceEqual(tokens_with_types(CqlRuleSet.lex(s)),
+                                     [(s, 'quotedStringLiteral')])
 
     def test_partial_parsing(self):
-        pass
+        [parsed] = CqlRuleSet.cql_parse('INSERT INTO ks.test')
+        self.assertSequenceEqual(parsed.matched, [])
+        self.assertSequenceEqual(tokens_with_types(parsed.remainder),
+                                 [('INSERT', 'reserved_identifier'),
+                                  ('INTO', 'reserved_identifier'),
+                                  ('ks', 'identifier'),
+                                  ('.', 'op'),
+                                  ('test', 'identifier')])
 
     def test_parse_select(self):
-        pass
+        parsed = parse_cqlsh_statements('SELECT FROM ks.tab;')
+        self.assertSequenceEqual(tokens_with_types(parsed),
+                                 [('SELECT', 'reserved_identifier'),
+                                  ('FROM', 'reserved_identifier'),
+                                  ('ks', 'identifier'),
+                                  ('.', 'op'),
+                                  ('tab', 'identifier'),
+                                  (';', 'endtoken')])
+
+        parsed = parse_cqlsh_statements('SELECT FROM "MyTable";')
+        self.assertSequenceEqual(tokens_with_types(parsed),
+                                 [('SELECT', 'reserved_identifier'),
+                                  ('FROM', 'reserved_identifier'),
+                                  ('"MyTable"', 'quotedName'),
+                                  (';', 'endtoken')])
+
+        parsed = parse_cqlsh_statements(
+            'SELECT FROM tab WHERE foo = 3;')
+        self.assertSequenceEqual(tokens_with_types(parsed),
+                                 [('SELECT', 'reserved_identifier'),
+                                  ('FROM', 'reserved_identifier'),
+                                  ('tab', 'identifier'),
+                                  ('WHERE', 'reserved_identifier'),
+                                  ('foo', 'identifier'),
+                                  ('=', 'op'),
+                                  ('3', 'wholenumber'),
+                                  (';', 'endtoken')])
+
+        parsed = parse_cqlsh_statements(
+            'SELECT FROM tab ORDER BY event_id DESC LIMIT 1000')
+        self.assertSequenceEqual(tokens_with_types(parsed),
+                                 [('SELECT', 'reserved_identifier'),
+                                  ('FROM', 'reserved_identifier'),
+                                  ('tab', 'identifier'),
+                                  ('ORDER', 'reserved_identifier'),
+                                  ('BY', 'reserved_identifier'),
+                                  ('event_id', 'identifier'),
+                                  ('DESC', 'reserved_identifier'),
+                                  ('LIMIT', 'reserved_identifier'),
+                                  ('1000', 'wholenumber')])
+
+        parsed = parse_cqlsh_statements(
+            'SELECT FROM tab WHERE clustering_column > 200 '
+            'AND clustering_column < 400 ALLOW FILTERING')
+        self.assertSequenceEqual(tokens_with_types(parsed),
+                                 [('SELECT', 'reserved_identifier'),
+                                  ('FROM', 'reserved_identifier'),
+                                  ('tab', 'identifier'),
+                                  ('WHERE', 'reserved_identifier'),
+                                  ('clustering_column', 'identifier'),
+                                  ('>', 'cmp'),
+                                  ('200', 'wholenumber'),
+                                  ('AND', 'reserved_identifier'),
+                                  ('clustering_column', 'identifier'),
+                                  ('<', 'cmp'),
+                                  ('400', 'wholenumber'),
+                                  # 'allow' and 'filtering' are not keywords
+                                  ('ALLOW', 'reserved_identifier'),
+                                  ('FILTERING', 'identifier')])
 
     def test_parse_insert(self):
-        pass
+        parsed = parse_cqlsh_statements('INSERT INTO mytable (x) VALUES (2);')
+        self.assertSequenceEqual(tokens_with_types(parsed),
+                                 [('INSERT', 'reserved_identifier'),
+                                  ('INTO', 'reserved_identifier'),
+                                  ('mytable', 'identifier'),
+                                  ('(', 'op'),
+                                  ('x', 'identifier'),
+                                  (')', 'op'),
+                                  ('VALUES', 'identifier'),
+                                  ('(', 'op'),
+                                  ('2', 'wholenumber'),
+                                  (')', 'op'),
+                                  (';', 'endtoken')])
+
+        parsed = parse_cqlsh_statements(
+            "INSERT INTO mytable (x, y) VALUES (2, 'eggs');")
+        self.assertSequenceEqual(tokens_with_types(parsed),
+                                 [('INSERT', 'reserved_identifier'),
+                                  ('INTO', 'reserved_identifier'),
+                                  ('mytable', 'identifier'),
+                                  ('(', 'op'),
+                                  ('x', 'identifier'),
+                                  (',', 'op'),
+                                  ('y', 'identifier'),
+                                  (')', 'op'),
+                                  ('VALUES', 'identifier'),
+                                  ('(', 'op'),
+                                  ('2', 'wholenumber'),
+                                  (',', 'op'),
+                                  ("'eggs'", 'quotedStringLiteral'),
+                                  (')', 'op'),
+                                  (';', 'endtoken')])
+
+        parsed = parse_cqlsh_statements(
+            "INSERT INTO mytable (x, y) VALUES (2, 'eggs');")
+        self.assertSequenceEqual(tokens_with_types(parsed),
+                                 [('INSERT', 'reserved_identifier'),
+                                  ('INTO', 'reserved_identifier'),
+                                  ('mytable', 'identifier'),
+                                  ('(', 'op'),
+                                  ('x', 'identifier'),
+                                  (',', 'op'),
+                                  ('y', 'identifier'),
+                                  (')', 'op'),
+                                  ('VALUES', 'identifier'),
+                                  ('(', 'op'),
+                                  ('2', 'wholenumber'),
+                                  (',', 'op'),
+                                  ("'eggs'", 'quotedStringLiteral'),
+                                  (')', 'op'),
+                                  (';', 'endtoken')])
+
+        parsed = parse_cqlsh_statements(
+            "INSERT INTO mytable (ids) VALUES "
+            "(7ee251da-af52-49a4-97f4-3f07e406c7a7) "
+            "USING TTL 86400;")
+        self.assertSequenceEqual(tokens_with_types(parsed),
+                                 [('INSERT', 'reserved_identifier'),
+                                  ('INTO', 'reserved_identifier'),
+                                  ('mytable', 'identifier'),
+                                  ('(', 'op'),
+                                  ('ids', 'identifier'),
+                                  (')', 'op'),
+                                  ('VALUES', 'identifier'),
+                                  ('(', 'op'),
+                                  ('7ee251da-af52-49a4-97f4-3f07e406c7a7', 'uuid'),
+                                  (')', 'op'),
+                                  ('USING', 'reserved_identifier'),
+                                  ('TTL', 'identifier'),
+                                  ('86400', 'wholenumber'),
+                                  (';', 'endtoken')])
+
+        parsed = parse_cqlsh_statements(
+            "INSERT INTO test_table (username) VALUES ('Albert') "
+            "USING TIMESTAMP 1240003134 AND TTL 600;")
+        self.assertSequenceEqual(tokens_with_types(parsed),
+                                 [('INSERT', 'reserved_identifier'),
+                                  ('INTO', 'reserved_identifier'),
+                                  ('test_table', 'identifier'),
+                                  ('(', 'op'),
+                                  ('username', 'identifier'),
+                                  (')', 'op'),
+                                  ('VALUES', 'identifier'),
+                                  ('(', 'op'),
+                                  ("'Albert'", 'quotedStringLiteral'),
+                                  (')', 'op'),
+                                  ('USING', 'reserved_identifier'),
+                                  ('TIMESTAMP', 'identifier'),
+                                  ('1240003134', 'wholenumber'),
+                                  ('AND', 'reserved_identifier'),
+                                  ('TTL', 'identifier'),
+                                  ('600', 'wholenumber'),
+                                  (';', 'endtoken')])
 
     def test_parse_update(self):
-        pass
+        parsed = parse_cqlsh_statements(
+            "UPDATE tab SET x = 15 WHERE y = 'eggs';")
+        self.assertSequenceEqual(tokens_with_types(parsed),
+                                 [('UPDATE', 'reserved_identifier'),
+                                  ('tab', 'identifier'),
+                                  ('SET', 'reserved_identifier'),
+                                  ('x', 'identifier'),
+                                  ('=', 'op'),
+                                  ('15', 'wholenumber'),
+                                  ('WHERE', 'reserved_identifier'),
+                                  ('y', 'identifier'),
+                                  ('=', 'op'),
+                                  ("'eggs'", 'quotedStringLiteral'),
+                                  (';', 'endtoken')])
+
+        parsed = parse_cqlsh_statements(
+            "UPDATE tab USING TTL 432000 SET x = 15 WHERE y = 'eggs';")
+        self.assertSequenceEqual(tokens_with_types(parsed),
+                                 [('UPDATE', 'reserved_identifier'),
+                                  ('tab', 'identifier'),
+                                  ('USING', 'reserved_identifier'),
+                                  ('TTL', 'identifier'),
+                                  ('432000', 'wholenumber'),
+                                  ('SET', 'reserved_identifier'),
+                                  ('x', 'identifier'),
+                                  ('=', 'op'),
+                                  ('15', 'wholenumber'),
+                                  ('WHERE', 'reserved_identifier'),
+                                  ('y', 'identifier'),
+                                  ('=', 'op'),
+                                  ("'eggs'", 'quotedStringLiteral'),
+                                  (';', 'endtoken')])
+
+        parsed = parse_cqlsh_statements(
+            "UPDATE tab SET x = 15, y = 'sausage' "
+            "WHERE y = 'eggs';")
+        self.assertSequenceEqual(tokens_with_types(parsed),
+                                 [('UPDATE', 'reserved_identifier'),
+                                  ('tab', 'identifier'),
+                                  ('SET', 'reserved_identifier'),
+                                  ('x', 'identifier'),
+                                  ('=', 'op'),
+                                  ('15', 'wholenumber'),
+                                  (',', 'op'),
+                                  ('y', 'identifier'),
+                                  ('=', 'op'),
+                                  ("'sausage'", 'quotedStringLiteral'),
+                                  ('WHERE', 'reserved_identifier'),
+                                  ('y', 'identifier'),
+                                  ('=', 'op'),
+                                  ("'eggs'", 'quotedStringLiteral'),
+                                  (';', 'endtoken')])
+
+        parsed = parse_cqlsh_statements(
+            "UPDATE tab SET x = 15 "
+            "WHERE y IN ('eggs', 'sausage', 'spam');")
+        self.assertSequenceEqual(tokens_with_types(parsed),
+                                 [('UPDATE', 'reserved_identifier'),
+                                  ('tab', 'identifier'),
+                                  ('SET', 'reserved_identifier'),
+                                  ('x', 'identifier'),
+                                  ('=', 'op'),
+                                  ('15', 'wholenumber'),
+                                  ('WHERE', 'reserved_identifier'),
+                                  ('y', 'identifier'),
+                                  ('IN', 'reserved_identifier'),
+                                  ('(', 'op'),
+                                  ("'eggs'", 'quotedStringLiteral'),
+                                  (',', 'op'),
+                                  ("'sausage'", 'quotedStringLiteral'),
+                                  (',', 'op'),
+                                  ("'spam'", 'quotedStringLiteral'),
+                                  (')', 'op'),
+                                  (';', 'endtoken')])
+
+        parsed = parse_cqlsh_statements(
+            "UPDATE tab SET x = 15 "
+            "WHERE y = 'spam' IF z = 'sausage';")
+        self.assertSequenceEqual(tokens_with_types(parsed),
+                                 [('UPDATE', 'reserved_identifier'),
+                                  ('tab', 'identifier'),
+                                  ('SET', 'reserved_identifier'),
+                                  ('x', 'identifier'),
+                                  ('=', 'op'),
+                                  ('15', 'wholenumber'),
+                                  ('WHERE', 'reserved_identifier'),
+                                  ('y', 'identifier'),
+                                  ('=', 'op'),
+                                  ("'spam'", 'quotedStringLiteral'),
+                                  ('IF', 'reserved_identifier'),
+                                  ('z', 'identifier'),
+                                  ('=', 'op'),
+                                  ("'sausage'", 'quotedStringLiteral'),
+                                  (';', 'endtoken')])
+
+        parsed = parse_cqlsh_statements(
+            "UPDATE tab SET x = 15 WHERE y = 'spam' "
+            "IF z = 'sausage' AND w = 'spam';")
+        self.assertSequenceEqual(tokens_with_types(parsed),
+                                 [('UPDATE', 'reserved_identifier'),
+                                  ('tab', 'identifier'),
+                                  ('SET', 'reserved_identifier'),
+                                  ('x', 'identifier'),
+                                  ('=', 'op'),
+                                  ('15', 'wholenumber'),
+                                  ('WHERE', 'reserved_identifier'),
+                                  ('y', 'identifier'),
+                                  ('=', 'op'),
+                                  ("'spam'", 'quotedStringLiteral'),
+                                  ('IF', 'reserved_identifier'),
+                                  ('z', 'identifier'),
+                                  ('=', 'op'),
+                                  ("'sausage'", 'quotedStringLiteral'),
+                                  ('AND', 'reserved_identifier'),
+                                  ('w', 'identifier'),
+                                  ('=', 'op'),
+                                  ("'spam'", 'quotedStringLiteral'),
+                                  (';', 'endtoken')])
+
+        parsed = parse_cqlsh_statements(
+            "UPDATE tab SET x = 15 WHERE y = 'spam' IF EXISTS")
+        self.assertSequenceEqual(tokens_with_types(parsed),
+                                 [('UPDATE', 'reserved_identifier'),
+                                  ('tab', 'identifier'),
+                                  ('SET', 'reserved_identifier'),
+                                  ('x', 'identifier'),
+                                  ('=', 'op'),
+                                  ('15', 'wholenumber'),
+                                  ('WHERE', 'reserved_identifier'),
+                                  ('y', 'identifier'),
+                                  ('=', 'op'),
+                                  ("'spam'", 'quotedStringLiteral'),
+                                  ('IF', 'reserved_identifier'),
+                                  ('EXISTS', 'identifier')])
 
     def test_parse_delete(self):
-        pass
+        parsed = parse_cqlsh_statements(
+            "DELETE FROM songs WHERE songid = 444;")
+        self.assertSequenceEqual(tokens_with_types(parsed),
+                                 [('DELETE', 'reserved_identifier'),
+                                  ('FROM', 'reserved_identifier'),
+                                  ('songs', 'identifier'),
+                                  ('WHERE', 'reserved_identifier'),
+                                  ('songid', 'identifier'),
+                                  ('=', 'op'),
+                                  ('444', 'wholenumber'),
+                                  (';', 'endtoken')])
+
+        parsed = parse_cqlsh_statements(
+            "DELETE FROM songs WHERE name IN "
+            "('Yellow Submarine', 'Eleanor Rigby');")
+        self.assertSequenceEqual(tokens_with_types(parsed),
+                                 [('DELETE', 'reserved_identifier'),
+                                  ('FROM', 'reserved_identifier'),
+                                  ('songs', 'identifier'),
+                                  ('WHERE', 'reserved_identifier'),
+                                  ('name', 'identifier'),
+                                  ('IN', 'reserved_identifier'),
+                                  ('(', 'op'),
+                                  ("'Yellow Submarine'", 'quotedStringLiteral'),
+                                  (',', 'op'),
+                                  ("'Eleanor Rigby'", 'quotedStringLiteral'),
+                                  (')', 'op'),
+                                  (';', 'endtoken')])
+
+        parsed = parse_cqlsh_statements(
+            "DELETE task_map ['2014-12-25'] FROM tasks WHERE user_id = 'Santa';")
+        self.assertSequenceEqual(tokens_with_types(parsed),
+                                 [('DELETE', 'reserved_identifier'),
+                                  ('task_map', 'identifier'),
+                                  ('[', 'brackets'),
+                                  ("'2014-12-25'", 'quotedStringLiteral'),
+                                  (']', 'brackets'),
+                                  ('FROM', 'reserved_identifier'),
+                                  ('tasks', 'identifier'),
+                                  ('WHERE', 'reserved_identifier'),
+                                  ('user_id', 'identifier'),
+                                  ('=', 'op'),
+                                  ("'Santa'", 'quotedStringLiteral'),
+                                  (';', 'endtoken')])
+
+        parsed = parse_cqlsh_statements(
+            "DELETE my_list[0] FROM lists WHERE user_id = 'Jim';")
+        self.assertSequenceEqual(tokens_with_types(parsed),
+                                 [('DELETE', 'reserved_identifier'),
+                                  ('my_list', 'identifier'),
+                                  ('[', 'brackets'),
+                                  ('0', 'wholenumber'),
+                                  (']', 'brackets'),
+                                  ('FROM', 'reserved_identifier'),
+                                  ('lists', 'identifier'),
+                                  ('WHERE', 'reserved_identifier'),
+                                  ('user_id', 'identifier'),
+                                  ('=', 'op'),
+                                  ("'Jim'", 'quotedStringLiteral'),
+                                  (';', 'endtoken')])
 
     def test_parse_batch(self):
         pass
 
     def test_parse_create_keyspace(self):
-        pass
+        parsed = parse_cqlsh_statements(
+            "CREATE KEYSPACE ks WITH REPLICATION = "
+            "{'class': 'SimpleStrategy', 'replication_factor': 1};")
+        self.assertSequenceEqual(tokens_with_types(parsed),
+                                 [('CREATE', 'reserved_identifier'),
+                                  ('KEYSPACE', 'reserved_identifier'),
+                                  ('ks', 'identifier'),
+                                  ('WITH', 'reserved_identifier'),
+                                  ('REPLICATION', 'identifier'),
+                                  ('=', 'op'),
+                                  ('{', 'brackets'),
+                                  ("'class'", 'quotedStringLiteral'),
+                                  (':', 'colon'),
+                                  ("'SimpleStrategy'", 'quotedStringLiteral'),
+                                  (',', 'op'),
+                                  ("'replication_factor'", 'quotedStringLiteral'),
+                                  (':', 'colon'),
+                                  ('1', 'wholenumber'),
+                                  ('}', 'brackets'),
+                                  (';', 'endtoken')])
+
+        parsed = parse_cqlsh_statements(
+            'CREATE KEYSPACE "Cql_test_KS" WITH REPLICATION = '
+            "{'class': 'NetworkTopologyStrategy', 'dc1' : 3, 'dc2': 2};")
+        self.assertSequenceEqual(tokens_with_types(parsed),
+                                 [('CREATE', 'reserved_identifier'),
+                                  ('KEYSPACE', 'reserved_identifier'),
+                                  ('"Cql_test_KS"', 'quotedName'),
+                                  ('WITH', 'reserved_identifier'),
+                                  ('REPLICATION', 'identifier'),
+                                  ('=', 'op'),
+                                  ('{', 'brackets'),
+                                  ("'class'", 'quotedStringLiteral'),
+                                  (':', 'colon'),
+                                  ("'NetworkTopologyStrategy'",
+                                   'quotedStringLiteral'),
+                                  (',', 'op'),
+                                  ("'dc1'", 'quotedStringLiteral'),
+                                  (':', 'colon'),
+                                  ('3', 'wholenumber'),
+                                  (',', 'op'),
+                                  ("'dc2'", 'quotedStringLiteral'),
+                                  (':', 'colon'),
+                                  ('2', 'wholenumber'),
+                                  ('}', 'brackets'),
+                                  (';', 'endtoken')])
+
+        parsed = parse_cqlsh_statements(
+            "CREATE KEYSPACE ks WITH REPLICATION = "
+            "{'class': 'NetworkTopologyStrategy', 'dc1': 3} AND "
+            "DURABLE_WRITES = false;")
+        self.assertSequenceEqual(tokens_with_types(parsed),
+                                 [('CREATE', 'reserved_identifier'),
+                                  ('KEYSPACE', 'reserved_identifier'),
+                                  ('ks', 'identifier'),
+                                  ('WITH', 'reserved_identifier'),
+                                  ('REPLICATION', 'identifier'),
+                                  ('=', 'op'),
+                                  ('{', 'brackets'),
+                                  ("'class'", 'quotedStringLiteral'),
+                                  (':', 'colon'),
+                                  ("'NetworkTopologyStrategy'",
+                                   'quotedStringLiteral'),
+                                  (',', 'op'),
+                                  ("'dc1'", 'quotedStringLiteral'),
+                                  (':', 'colon'),
+                                  ('3', 'wholenumber'),
+                                  ('}', 'brackets'),
+                                  ('AND', 'reserved_identifier'),
+                                  # 'DURABLE_WRITES' is not a keyword
+                                  ('DURABLE_WRITES', 'identifier'),
+                                  ('=', 'op'),
+                                  ('false', 'identifier'),
+                                  (';', 'endtoken')])
 
     def test_parse_drop_keyspace(self):
+        parsed = parse_cqlsh_statements(
+            'DROP KEYSPACE ks;')
+        self.assertSequenceEqual(tokens_with_types(parsed),
+                                 [('DROP', 'reserved_identifier'),
+                                  ('KEYSPACE', 'reserved_identifier'),
+                                  ('ks', 'identifier'),
+                                  (';', 'endtoken')])
+
+        parsed = parse_cqlsh_statements(
+            'DROP SCHEMA ks;')
+        self.assertSequenceEqual(tokens_with_types(parsed),
+                                 [('DROP', 'reserved_identifier'),
+                                  ('SCHEMA', 'reserved_identifier'),
+                                  ('ks', 'identifier'),
+                                  (';', 'endtoken')])
+
+        parsed = parse_cqlsh_statements(
+            'DROP KEYSPACE IF EXISTS "My_ks";')
+        self.assertSequenceEqual(tokens_with_types(parsed),
+                                 [('DROP', 'reserved_identifier'),
+                                  ('KEYSPACE', 'reserved_identifier'),
+                                  ('IF', 'reserved_identifier'),
+                                  ('EXISTS', 'identifier'),
+                                  ('"My_ks"', 'quotedName'),
+                                  (';', 'endtoken')])
+
+    def test_parse_create_table(self):
         pass
 
-    def test_parse_create_columnfamily(self):
-        pass
-
-    def test_parse_drop_columnfamily(self):
+    def test_parse_drop_table(self):
         pass
 
     def test_parse_truncate(self):
         pass
 
-    def test_parse_alter_columnfamily(self):
+    def test_parse_alter_table(self):
         pass
 
     def test_parse_use(self):
         pass
 
     def test_parse_create_index(self):
-        pass
+        parsed = parse_cqlsh_statements(
+            'CREATE INDEX idx ON ks.tab (i);')
+        self.assertSequenceEqual(tokens_with_types(parsed),
+                                 (('CREATE', 'reserved_identifier'),
+                                  ('INDEX', 'reserved_identifier'),
+                                  ('idx', 'identifier'),
+                                  ('ON', 'reserved_identifier'),
+                                  ('ks', 'identifier'),
+                                  ('.', 'op'),
+                                  ('tab', 'identifier'),
+                                  ('(', 'op'),
+                                  ('i', 'identifier'),
+                                  (')', 'op'),
+                                  (';', 'endtoken')))
+
+        parsed = parse_cqlsh_statements(
+            'CREATE INDEX idx ON ks.tab (i) IF NOT EXISTS;')
+        self.assertSequenceEqual(tokens_with_types(parsed),
+                                 (('CREATE', 'reserved_identifier'),
+                                  ('INDEX', 'reserved_identifier'),
+                                  ('idx', 'identifier'),
+                                  ('ON', 'reserved_identifier'),
+                                  ('ks', 'identifier'),
+                                  ('.', 'op'),
+                                  ('tab', 'identifier'),
+                                  ('(', 'op'),
+                                  ('i', 'identifier'),
+                                  (')', 'op'),
+                                  ('IF', 'reserved_identifier'),
+                                  ('NOT', 'reserved_identifier'),
+                                  ('EXISTS', 'identifier'),
+                                  (';', 'endtoken')))
+
+        parsed = parse_cqlsh_statements(
+            'CREATE INDEX idx ON tab (KEYS(i));')
+        self.assertSequenceEqual(tokens_with_types(parsed),
+                                 (('CREATE', 'reserved_identifier'),
+                                  ('INDEX', 'reserved_identifier'),
+                                  ('idx', 'identifier'),
+                                  ('ON', 'reserved_identifier'),
+                                  ('tab', 'identifier'),
+                                  ('(', 'op'),
+                                  ('KEYS', 'identifier'),
+                                  ('(', 'op'),
+                                  ('i', 'identifier'),
+                                  (')', 'op'),
+                                  (')', 'op'),
+                                  (';', 'endtoken')))
+
+        parsed = parse_cqlsh_statements(
+            'CREATE INDEX idx ON ks.tab FULL(i);')
+        self.assertSequenceEqual(tokens_with_types(parsed),
+                                 [('CREATE', 'reserved_identifier'),
+                                  ('INDEX', 'reserved_identifier'),
+                                  ('idx', 'identifier'),
+                                  ('ON', 'reserved_identifier'),
+                                  ('ks', 'identifier'),
+                                  ('.', 'op'),
+                                  ('tab', 'identifier'),
+                                  ('FULL', 'reserved_identifier'),
+                                  ('(', 'op'),
+                                  ('i', 'identifier'),
+                                  (')', 'op'),
+                                  (';', 'endtoken')])
+
+        parsed = parse_cqlsh_statements(
+            'CREATE CUSTOM INDEX idx ON ks.tab (i);')
+        self.assertSequenceEqual(tokens_with_types(parsed),
+                                 [('CREATE', 'reserved_identifier'),
+                                  ('CUSTOM', 'identifier'),
+                                  ('INDEX', 'reserved_identifier'),
+                                  ('idx', 'identifier'),
+                                  ('ON', 'reserved_identifier'),
+                                  ('ks', 'identifier'),
+                                  ('.', 'op'),
+                                  ('tab', 'identifier'),
+                                  ('(', 'op'),
+                                  ('i', 'identifier'),
+                                  (')', 'op'),
+                                  (';', 'endtoken')])
+
+        parsed = parse_cqlsh_statements(
+            "CREATE INDEX idx ON ks.tab (i) USING "
+            "'org.custom.index.MyIndexClass';")
+        self.assertSequenceEqual(tokens_with_types(parsed),
+                                 [('CREATE', 'reserved_identifier'),
+                                  ('INDEX', 'reserved_identifier'),
+                                  ('idx', 'identifier'),
+                                  ('ON', 'reserved_identifier'),
+                                  ('ks', 'identifier'),
+                                  ('.', 'op'),
+                                  ('tab', 'identifier'),
+                                  ('(', 'op'),
+                                  ('i', 'identifier'),
+                                  (')', 'op'),
+                                  ('USING', 'reserved_identifier'),
+                                  ("'org.custom.index.MyIndexClass'",
+                                   'quotedStringLiteral'),
+                                  (';', 'endtoken')])
+
+        parsed = parse_cqlsh_statements(
+            "CREATE INDEX idx ON ks.tab (i) WITH OPTIONS = "
+            "{'storage': '/mnt/ssd/indexes/'};")
+        self.assertSequenceEqual(tokens_with_types(parsed),
+                                 [('CREATE', 'reserved_identifier'),
+                                  ('INDEX', 'reserved_identifier'),
+                                  ('idx', 'identifier'),
+                                  ('ON', 'reserved_identifier'),
+                                  ('ks', 'identifier'),
+                                  ('.', 'op'),
+                                  ('tab', 'identifier'),
+                                  ('(', 'op'),
+                                  ('i', 'identifier'),
+                                  (')', 'op'),
+                                  ('WITH', 'reserved_identifier'),
+                                  ('OPTIONS', 'identifier'),
+                                  ('=', 'op'),
+                                  ('{', 'brackets'),
+                                  ("'storage'", 'quotedStringLiteral'),
+                                  (':', 'colon'),
+                                  ("'/mnt/ssd/indexes/'", 'quotedStringLiteral'),
+                                  ('}', 'brackets'),
+                                  (';', 'endtoken')])
 
     def test_parse_drop_index(self):
         pass
 
     def test_parse_select_token(self):
-        self.cqlsh.cmd_and_response('INSERT INTO has_all_types (num) VALUES (1);')
-        response = self.cqlsh.cmd_and_response("SELECT token(num) from has_all_types where num=1;")
-        self.assertIn('-4069959284402364209', response)
+        pass
+
+
+def parse_cqlsh_statements(text):
+    '''
+    Runs its argument through the sequence of parsing steps that cqlsh takes its
+    input through.
+
+    Currently does not handle batch statements.
+    '''
+    # based on onecmd
+    statements, _ = CqlRuleSet.cql_split_statements(text)
+    # stops here. For regular cql commands, onecmd just splits it and sends it
+    # off to the cql engine; parsing only happens for cqlsh-specific stmts.
+
+    return strip_final_empty_items(statements)[0]
+
+
+def tokens_with_types(lexed):
+    for x in lexed:
+        assert len(x) > 2, lexed
+    return tuple(itemgetter(1, 0)(token) for token in lexed)
+
+
+def strip_final_empty_items(xs):
+    '''
+    Returns its a copy of argument as a list, but with any terminating
+    subsequence of falsey values removed.
+
+    >>> strip_final_empty_items([[3, 4], [5, 6, 7], [], [], [1], []])
+    [[3, 4], [5, 6, 7], [], [], [1]]
+    '''
+    rv = list(xs)
+
+    while rv and not rv[-1]:
+        rv = rv[:-1]
+
+    return rv
diff --git a/pylib/cqlshlib/test/test_cqlsh_completion.py b/pylib/cqlshlib/test/test_cqlsh_completion.py
index 04cee5d..19bd092 100644
--- a/pylib/cqlshlib/test/test_cqlsh_completion.py
+++ b/pylib/cqlshlib/test/test_cqlsh_completion.py
@@ -22,6 +22,8 @@
 import re
 from .basecase import BaseTestCase, cqlsh
 from .cassconnect import testrun_cqlsh
+import unittest
+import sys
 
 BEL = '\x07'  # the terminal-bell character
 CTRL_C = '\x03'
@@ -36,7 +38,9 @@
 completion_separation_re = re.compile(r'\s+')
 
 
+@unittest.skipIf(sys.platform == "win32", 'Tab completion tests not supported on Windows')
 class CqlshCompletionCase(BaseTestCase):
+
     def setUp(self):
         self.cqlsh_runner = testrun_cqlsh(cqlver=cqlsh.DEFAULT_CQLVER, env={'COLUMNS': '100000'})
         self.cqlsh = self.cqlsh_runner.__enter__()
@@ -76,7 +80,7 @@
             # ensure the last line of the completion is the prompt
             prompt_regex = self.cqlsh.prompt.lstrip() + re.escape(inputstring)
             msg = ('Double-tab completion '
-                   'does not print prompt for input "%s"' % (inputstring, ))
+                   'does not print prompt for input "{}"'.format(inputstring))
             self.assertRegexpMatches(choice_lines[-1], prompt_regex, msg=msg)
 
         choice_lines = [line.strip() for line in choice_lines[:-1]]
@@ -84,7 +88,7 @@
 
         if split_completed_lines:
             completed_lines = map(set, (completion_separation_re.split(line.strip())
-                               for line in choice_lines))
+                                  for line in choice_lines))
 
             if not completed_lines:
                 return set()
@@ -109,7 +113,7 @@
         match the items in 'choices' (order is not important, but case is).
         """
         completed = self._get_completions(inputstring,
-                                         split_completed_lines=split_completed_lines)
+                                          split_completed_lines=split_completed_lines)
 
         if immediate:
             msg = 'cqlsh completed %r, but we expected %r' % (completed, immediate)
@@ -143,8 +147,8 @@
         self.trycompletions('', choices=('?', 'ALTER', 'BEGIN', 'CAPTURE', 'CONSISTENCY',
                                          'COPY', 'CREATE', 'DEBUG', 'DELETE', 'DESC', 'DESCRIBE',
                                          'DROP', 'GRANT', 'HELP', 'INSERT', 'LIST', 'LOGIN', 'PAGING', 'REVOKE',
-                                         'SELECT', 'SERIAL', 'SHOW', 'SOURCE', 'TRACING', 'EXPAND', 'TRUNCATE',
-                                         'UPDATE', 'USE', 'exit', 'quit'))
+                                         'SELECT', 'SHOW', 'SOURCE', 'TRACING', 'EXPAND', 'SERIAL', 'TRUNCATE',
+                                         'UPDATE', 'USE', 'exit', 'quit', 'CLEAR', 'CLS'))
 
     def test_complete_command_words(self):
         self.trycompletions('alt', '\b\b\bALTER ')
@@ -174,9 +178,9 @@
                                      'songs'),
                             other_choices_ok=True)
         self.trycompletions('INSERT INTO twenty_rows_composite_table',
-                            immediate=' (a, b ')
+                            immediate=' ')
         self.trycompletions('INSERT INTO twenty_rows_composite_table ',
-                            immediate='(a, b ')
+                            choices=['(', 'JSON'])
         self.trycompletions('INSERT INTO twenty_rows_composite_table (a, b ',
                             choices=(')', ','))
         self.trycompletions('INSERT INTO twenty_rows_composite_table (a, b, ',
@@ -229,8 +233,9 @@
             choices=['?', 'ALTER', 'BEGIN', 'CAPTURE', 'CONSISTENCY', 'COPY',
                      'CREATE', 'DEBUG', 'DELETE', 'DESC', 'DESCRIBE', 'DROP',
                      'EXPAND', 'GRANT', 'HELP', 'INSERT', 'LIST', 'LOGIN', 'PAGING',
-                     'REVOKE', 'SELECT', 'SERIAL', 'SHOW', 'SOURCE', 'TRACING',
-                     'TRUNCATE', 'UPDATE', 'USE', 'exit', 'quit'])
+                     'REVOKE', 'SELECT', 'SHOW', 'SOURCE', 'SERIAL', 'TRACING',
+                     'TRUNCATE', 'UPDATE', 'USE', 'exit', 'quit',
+                     'CLEAR', 'CLS'])
 
         self.trycompletions(
             ("INSERT INTO twenty_rows_composite_table (a, b, c) "
@@ -339,18 +344,18 @@
         self.trycompletions("UPDATE empty_table SET lonelycol = 'eggs'",
                             choices=[',', 'WHERE'])
         self.trycompletions("UPDATE empty_table SET lonelycol = 'eggs' WHERE ",
-                            choices=['TOKEN(', '<identifier>', '<quotedName>'])
+                            choices=['TOKEN(', 'lonelykey'])
         self.trycompletions("UPDATE empty_table SET lonelycol = 'eggs' WHERE ",
-                            choices=['TOKEN(', '<identifier>', '<quotedName>'])
+                            choices=['TOKEN(', 'lonelykey'])
 
         self.trycompletions("UPDATE empty_table SET lonelycol = 'eggs' WHERE lonel",
-                            choices=['<quotedName>', '<identifier>'])
+                            immediate='ykey ')
         self.trycompletions("UPDATE empty_table SET lonelycol = 'eggs' WHERE lonelykey ",
-                            choices=['=', '<=', '>=', '>', '<', 'CONTAINS', 'IN'])
+                            choices=['=', '<=', '>=', '>', '<', 'CONTAINS', 'IN', '['])
         self.trycompletions("UPDATE empty_table SET lonelycol = 'eggs' WHERE lonelykey = 0.0 ",
                             choices=['AND', 'IF', ';'])
         self.trycompletions("UPDATE empty_table SET lonelycol = 'eggs' WHERE lonelykey = 0.0 AND ",
-                            choices=['TOKEN(', '<identifier>', '<quotedName>'])
+                            choices=['TOKEN(', 'lonelykey'])
 
         self.trycompletions("UPDATE empty_table SET lonelycol = 'eggs' WHERE TOKEN(lonelykey ",
                             choices=[',', ')'])
@@ -365,7 +370,95 @@
                             choices=['>=', '!=', '<=', 'IN', '[', ';', '=', '<', '>'])
 
     def test_complete_in_delete(self):
-        pass
+        self.trycompletions('DELETE F', choices=['FROM', '<identifier>', '<quotedName>'])
+
+        self.trycompletions('DELETE a ', choices=['FROM', '[', ','])
+        self.trycompletions('DELETE a [',
+                            choices=['<wholenumber>', 'false', '-', '<uuid>',
+                                     '<pgStringLiteral>', '<float>', 'TOKEN',
+                                     '<identifier>', '<quotedStringLiteral>',
+                                     '{', '[', 'NULL', 'true', '<blobLiteral>'])
+
+        self.trycompletions('DELETE a, ',
+                            choices=['<identifier>', '<quotedName>'])
+
+        self.trycompletions('DELETE a FROM ',
+                            choices=['twenty_rows_table',
+                                     'ascii_with_special_chars', 'users',
+                                     'has_all_types', 'system.',
+                                     'empty_composite_table', 'empty_table',
+                                     'system_auth.', 'undefined_values_table',
+                                     'dynamic_columns',
+                                     'twenty_rows_composite_table',
+                                     'utf8_with_special_chars',
+                                     'system_traces.', 'songs',
+                                     '"' + self.cqlsh.keyspace + '".'],
+                            other_choices_ok=True)
+
+        self.trycompletions('DELETE FROM ',
+                            choices=['twenty_rows_table',
+                                     'ascii_with_special_chars', 'users',
+                                     'has_all_types', 'system.',
+                                     'empty_composite_table', 'empty_table',
+                                     'system_auth.', 'undefined_values_table',
+                                     'dynamic_columns',
+                                     'twenty_rows_composite_table',
+                                     'utf8_with_special_chars',
+                                     'system_traces.', 'songs',
+                                     'system_auth.', 'system_distributed.',
+                                     'system_traces.',
+                                     '"' + self.cqlsh.keyspace + '".'],
+                            other_choices_ok=True)
+        self.trycompletions('DELETE FROM twenty_rows_composite_table ',
+                            choices=['USING', 'WHERE'])
+
+        self.trycompletions('DELETE FROM twenty_rows_composite_table U',
+                            immediate='SING TIMESTAMP ')
+
+        self.trycompletions('DELETE FROM twenty_rows_composite_table USING TIMESTAMP ',
+                            choices=['<wholenumber>'])
+        self.trycompletions('DELETE FROM twenty_rows_composite_table USING TIMESTAMP 0',
+                            choices=['<wholenumber>'])
+        self.trycompletions('DELETE FROM twenty_rows_composite_table USING TIMESTAMP 0 ',
+                            immediate='WHERE ')
+        self.trycompletions('DELETE FROM twenty_rows_composite_table USING TIMESTAMP 0 WHERE ',
+                            choices=['a', 'b', 'TOKEN('])
+
+        self.trycompletions('DELETE FROM twenty_rows_composite_table USING TIMESTAMP 0 WHERE a ',
+                            choices=['<=', '>=', 'CONTAINS', 'IN', '[', '=', '<', '>'])
+
+        self.trycompletions('DELETE FROM twenty_rows_composite_table USING TIMESTAMP 0 WHERE TOKEN(',
+                            immediate='a ')
+        self.trycompletions('DELETE FROM twenty_rows_composite_table USING TIMESTAMP 0 WHERE TOKEN(a',
+                            immediate=' ')
+        self.trycompletions('DELETE FROM twenty_rows_composite_table USING TIMESTAMP 0 WHERE TOKEN(a ',
+                            choices=[')', ','])
+        self.trycompletions('DELETE FROM twenty_rows_composite_table USING TIMESTAMP 0 WHERE TOKEN(a) ',
+                            choices=['>=', '<=', '=', '<', '>'])
+        self.trycompletions('DELETE FROM twenty_rows_composite_table USING TIMESTAMP 0 WHERE TOKEN(a) >= ',
+                            choices=['false', 'true', '<pgStringLiteral>',
+                                     'token(', '-', '<float>', 'TOKEN',
+                                     '<identifier>', '<uuid>', '{', '[', 'NULL',
+                                     '<quotedStringLiteral>', '<blobLiteral>',
+                                     '<wholenumber>'])
+        self.trycompletions(('DELETE FROM twenty_rows_composite_table USING TIMESTAMP 0 WHERE '
+                             'TOKEN(a) >= TOKEN(0) '),
+                            choices=['AND', 'IF', ';'])
+        self.trycompletions(('DELETE FROM twenty_rows_composite_table USING TIMESTAMP 0 WHERE '
+                             'TOKEN(a) >= TOKEN(0) IF '),
+                            choices=['EXISTS', '<identifier>', '<quotedName>'])
+        self.trycompletions(('DELETE FROM twenty_rows_composite_table USING TIMESTAMP 0 WHERE '
+                             'TOKEN(a) >= TOKEN(0) IF b '),
+                            choices=['>=', '!=', '<=', 'IN', '[', '=', '<', '>'])
+        self.trycompletions(('DELETE FROM twenty_rows_composite_table USING TIMESTAMP 0 WHERE '
+                             'TOKEN(a) >= TOKEN(0) IF b < 0 '),
+                            choices=['AND', ';'])
+        self.trycompletions(('DELETE FROM twenty_rows_composite_table USING TIMESTAMP 0 WHERE '
+                             'TOKEN(a) >= TOKEN(0) IF b < 0 AND '),
+                            choices=['<identifier>', '<quotedName>'])
+        self.trycompletions(("DELETE FROM twenty_rows_composite_table USING TIMESTAMP 0 WHERE "
+                             "b = 'eggs'"),
+                            choices=['AND', 'IF', ';'])
 
     def test_complete_in_batch(self):
         pass
@@ -389,34 +482,34 @@
                             choices=self.strategies())
         # ttl is an "unreserved keyword". should work
         self.trycompletions("create keySPACE ttl with replication ="
-                               "{ 'class' : 'SimpleStrategy'", ", 'replication_factor': ")
+                            "{ 'class' : 'SimpleStrategy'", ", 'replication_factor': ")
         self.trycompletions("create   keyspace ttl with replication ="
-                               "{'class':'SimpleStrategy',", " 'replication_factor': ")
+                            "{'class':'SimpleStrategy',", " 'replication_factor': ")
         self.trycompletions("create keyspace \"ttl\" with replication ="
-                               "{'class': 'SimpleStrategy', ", "'replication_factor': ")
+                            "{'class': 'SimpleStrategy', ", "'replication_factor': ")
         self.trycompletions("create keyspace \"ttl\" with replication ="
-                               "{'class': 'SimpleStrategy', 'repl", "ication_factor'")
+                            "{'class': 'SimpleStrategy', 'repl", "ication_factor'")
         self.trycompletions("create keyspace foo with replication ="
-                               "{'class': 'SimpleStrategy', 'replication_factor': ", '',
+                            "{'class': 'SimpleStrategy', 'replication_factor': ", '',
                             choices=('<term>',))
         self.trycompletions("create keyspace foo with replication ="
-                               "{'class': 'SimpleStrategy', 'replication_factor': 1", '',
+                            "{'class': 'SimpleStrategy', 'replication_factor': 1", '',
                             choices=('<term>',))
         self.trycompletions("create keyspace foo with replication ="
-                               "{'class': 'SimpleStrategy', 'replication_factor': 1 ", '}')
+                            "{'class': 'SimpleStrategy', 'replication_factor': 1 ", '}')
         self.trycompletions("create keyspace foo with replication ="
-                               "{'class': 'SimpleStrategy', 'replication_factor': 1, ",
+                            "{'class': 'SimpleStrategy', 'replication_factor': 1, ",
                             '', choices=())
         self.trycompletions("create keyspace foo with replication ="
-                               "{'class': 'SimpleStrategy', 'replication_factor': 1} ",
+                            "{'class': 'SimpleStrategy', 'replication_factor': 1} ",
                             '', choices=('AND', ';'))
         self.trycompletions("create keyspace foo with replication ="
-                               "{'class': 'NetworkTopologyStrategy', ", '',
+                            "{'class': 'NetworkTopologyStrategy', ", '',
                             choices=('<dc_name>',))
         self.trycompletions("create keyspace \"PB and J\" with replication={"
-                               "'class': 'NetworkTopologyStrategy'", ', ')
+                            "'class': 'NetworkTopologyStrategy'", ', ')
         self.trycompletions("create keyspace PBJ with replication={"
-                               "'class': 'NetworkTopologyStrategy'} and ",
+                            "'class': 'NetworkTopologyStrategy'} and ",
                             "durable_writes = '")
 
     def test_complete_in_string_literals(self):
@@ -425,11 +518,264 @@
         self.trycompletions("create keyspace blah with replication = {'class': 'Sim",
                             "pleStrategy'")
 
+    def test_complete_in_drop(self):
+        self.trycompletions('DR', immediate='OP ')
+        self.trycompletions('DROP ',
+                            choices=['AGGREGATE', 'COLUMNFAMILY', 'FUNCTION',
+                                     'INDEX', 'KEYSPACE', 'ROLE', 'TABLE',
+                                     'TRIGGER', 'TYPE', 'USER'])
+
     def test_complete_in_drop_keyspace(self):
-        pass
+        self.trycompletions('DROP K', immediate='EYSPACE ')
+        quoted_keyspace = '"' + self.cqlsh.keyspace + '"'
+        self.trycompletions('DROP KEYSPACE ',
+                            choices=['IF', quoted_keyspace])
+
+        self.trycompletions('DROP KEYSPACE ' + quoted_keyspace,
+                            choices=[';'])
+
+        self.trycompletions('DROP KEYSPACE I',
+                            immediate='F EXISTS ' + quoted_keyspace + ';')
+
+    def create_columnfamily_table_template(self, name):
+        """Parameterized test for CREATE COLUMNFAMILY and CREATE TABLE. Since
+        they're synonyms, they should have the same completion behavior, so this
+        test avoids duplication between tests for the two statements."""
+        prefix = 'CREATE ' + name + ' '
+        quoted_keyspace = '"' + self.cqlsh.keyspace + '"'
+        self.trycompletions(prefix + '',
+                            choices=['IF', quoted_keyspace, '<new_table_name>'])
+        self.trycompletions(prefix + 'IF ',
+                            immediate='NOT EXISTS ')
+        self.trycompletions(prefix + 'IF NOT EXISTS ',
+                            choices=['<new_table_name>', quoted_keyspace])
+        self.trycompletions(prefix + 'IF NOT EXISTS new_table ',
+                            immediate='( ')
+
+        self.trycompletions(prefix + quoted_keyspace, choices=['.', '('])
+
+        self.trycompletions(prefix + quoted_keyspace + '( ',
+                            choices=['<new_column_name>', '<identifier>',
+                                     '<quotedName>'])
+
+        self.trycompletions(prefix + quoted_keyspace + '.',
+                            choices=['<new_table_name>'])
+        self.trycompletions(prefix + quoted_keyspace + '.new_table ',
+                            immediate='( ')
+        self.trycompletions(prefix + quoted_keyspace + '.new_table ( ',
+                            choices=['<new_column_name>', '<identifier>',
+                                     '<quotedName>'])
+
+        self.trycompletions(prefix + ' new_table ( ',
+                            choices=['<new_column_name>', '<identifier>',
+                                     '<quotedName>'])
+        self.trycompletions(prefix + ' new_table (col_a ine',
+                            immediate='t ')
+        self.trycompletions(prefix + ' new_table (col_a int ',
+                            choices=[',', 'PRIMARY'])
+        self.trycompletions(prefix + ' new_table (col_a int P',
+                            immediate='RIMARY KEY ')
+        self.trycompletions(prefix + ' new_table (col_a int PRIMARY KEY ',
+                            choices=[')', ','])
+
+        self.trycompletions(prefix + ' new_table (col_a int PRIMARY KEY,',
+                            choices=['<identifier>', '<quotedName>'])
+        self.trycompletions(prefix + ' new_table (col_a int PRIMARY KEY)',
+                            immediate=' ')
+        self.trycompletions(prefix + ' new_table (col_a int PRIMARY KEY) ',
+                            choices=[';', 'WITH'])
+        self.trycompletions(prefix + ' new_table (col_a int PRIMARY KEY) W',
+                            immediate='ITH ')
+        self.trycompletions(prefix + ' new_table (col_a int PRIMARY KEY) WITH ',
+                            choices=['bloom_filter_fp_chance', 'compaction',
+                                     'compression',
+                                     'dclocal_read_repair_chance',
+                                     'default_time_to_live', 'gc_grace_seconds',
+                                     'max_index_interval',
+                                     'memtable_flush_period_in_ms',
+                                     'read_repair_chance', 'CLUSTERING',
+                                     'COMPACT', 'caching', 'comment',
+                                     'min_index_interval', 'speculative_retry'])
+        self.trycompletions(prefix + ' new_table (col_a int PRIMARY KEY) WITH ',
+                            choices=['bloom_filter_fp_chance', 'compaction',
+                                     'compression',
+                                     'dclocal_read_repair_chance',
+                                     'default_time_to_live', 'gc_grace_seconds',
+                                     'max_index_interval',
+                                     'memtable_flush_period_in_ms',
+                                     'read_repair_chance', 'CLUSTERING',
+                                     'COMPACT', 'caching', 'comment',
+                                     'min_index_interval', 'speculative_retry'])
+        self.trycompletions(prefix + ' new_table (col_a int PRIMARY KEY) WITH bloom_filter_fp_chance ',
+                            immediate='= ')
+        self.trycompletions(prefix + ' new_table (col_a int PRIMARY KEY) WITH bloom_filter_fp_chance = ',
+                            choices=['<float_between_0_and_1>'])
+
+        self.trycompletions(prefix + ' new_table (col_a int PRIMARY KEY) WITH compaction ',
+                            immediate="= {'class': '")
+        self.trycompletions(prefix + " new_table (col_a int PRIMARY KEY) WITH compaction = "
+                            + "{'class': '",
+                            choices=['SizeTieredCompactionStrategy',
+                                     'LeveledCompactionStrategy',
+                                     'DateTieredCompactionStrategy'])
+        self.trycompletions(prefix + " new_table (col_a int PRIMARY KEY) WITH compaction = "
+                            + "{'class': 'S",
+                            immediate="izeTieredCompactionStrategy'")
+        self.trycompletions(prefix + " new_table (col_a int PRIMARY KEY) WITH compaction = "
+                            + "{'class': 'SizeTieredCompactionStrategy",
+                            immediate="'")
+        self.trycompletions(prefix + " new_table (col_a int PRIMARY KEY) WITH compaction = "
+                            + "{'class': 'SizeTieredCompactionStrategy'",
+                            choices=['}', ','])
+        self.trycompletions(prefix + " new_table (col_a int PRIMARY KEY) WITH compaction = "
+                            + "{'class': 'SizeTieredCompactionStrategy', ",
+                            immediate="'")
+        self.trycompletions(prefix + " new_table (col_a int PRIMARY KEY) WITH compaction = "
+                            + "{'class': 'SizeTieredCompactionStrategy', '",
+                            choices=['bucket_high', 'bucket_low', 'class',
+                                     'enabled', 'max_threshold',
+                                     'min_sstable_size', 'min_threshold',
+                                     'tombstone_compaction_interval',
+                                     'tombstone_threshold',
+                                     'unchecked_tombstone_compaction'])
+        self.trycompletions(prefix + " new_table (col_a int PRIMARY KEY) WITH compaction = "
+                            + "{'class': 'SizeTieredCompactionStrategy'}",
+                            choices=[';', 'AND'])
+        self.trycompletions(prefix + " new_table (col_a int PRIMARY KEY) WITH compaction = "
+                            + "{'class': 'SizeTieredCompactionStrategy'} AND ",
+                            choices=['bloom_filter_fp_chance', 'compaction',
+                                     'compression',
+                                     'dclocal_read_repair_chance',
+                                     'default_time_to_live', 'gc_grace_seconds',
+                                     'max_index_interval',
+                                     'memtable_flush_period_in_ms',
+                                     'read_repair_chance', 'CLUSTERING',
+                                     'COMPACT', 'caching', 'comment',
+                                     'min_index_interval', 'speculative_retry'])
+        self.trycompletions(prefix + " new_table (col_a int PRIMARY KEY) WITH compaction = "
+                            + "{'class': 'DateTieredCompactionStrategy', '",
+                            choices=['base_time_seconds', 'max_sstable_age_days',
+                                     'timestamp_resolution', 'min_threshold', 'class', 'max_threshold',
+                                     'tombstone_compaction_interval', 'tombstone_threshold',
+                                     'enabled', 'unchecked_tombstone_compaction',
+                                     'max_window_size_seconds'])
 
     def test_complete_in_create_columnfamily(self):
-        pass
+        self.trycompletions('CREATE C', choices=['COLUMNFAMILY', 'CUSTOM'])
+        self.trycompletions('CREATE CO', immediate='LUMNFAMILY ')
+        self.create_columnfamily_table_template('COLUMNFAMILY')
+
+    def test_complete_in_create_table(self):
+        self.trycompletions('CREATE T', choices=['TRIGGER', 'TABLE', 'TYPE'])
+        self.trycompletions('CREATE TA', immediate='BLE ')
+        self.create_columnfamily_table_template('TABLE')
+
+    def test_complete_in_describe(self):
+        """
+        Tests for Cassandra-10733
+        """
+        self.trycompletions('DES', immediate='C')
+        # quoted_keyspace = '"' + self.cqlsh.keyspace + '"'
+        self.trycompletions('DESCR', immediate='IBE ')
+        self.trycompletions('DESC TABLE ',
+                            choices=['twenty_rows_table',
+                                     'ascii_with_special_chars', 'users',
+                                     'has_all_types', 'system.',
+                                     'empty_composite_table', 'empty_table',
+                                     'system_auth.', 'undefined_values_table',
+                                     'dynamic_columns',
+                                     'twenty_rows_composite_table',
+                                     'utf8_with_special_chars',
+                                     'system_traces.', 'songs',
+                                     'system_distributed.',
+                                     '"' + self.cqlsh.keyspace + '".'],
+                            other_choices_ok=True)
+
+        self.trycompletions('DESC TYPE ',
+                            choices=['system.',
+                                     'system_auth.',
+                                     'system_traces.',
+                                     'system_distributed.',
+                                     'address',
+                                     'phone_number',
+                                     'band_info_type',
+                                     'tags'],
+                            other_choices_ok=True)
+
+        self.trycompletions('DESC FUNCTION ',
+                            choices=['system.',
+                                     'system_auth.',
+                                     'system_traces.',
+                                     'system_distributed.',
+                                     'fbestband',
+                                     'fbestsong',
+                                     'fmax',
+                                     'fmin',
+                                     '"' + self.cqlsh.keyspace + '".'],
+                            other_choices_ok=True)
+
+        self.trycompletions('DESC AGGREGATE ',
+                            choices=['system.',
+                                     'system_auth.',
+                                     'system_traces.',
+                                     'system_distributed.',
+                                     'aggmin',
+                                     'aggmax',
+                                     '"' + self.cqlsh.keyspace + '".'],
+                            other_choices_ok=True)
+
+        # Unfortunately these commented tests will not work. This is due to the keyspace name containing quotes;
+        # cqlsh auto-completes a DESC differently when the keyspace contains quotes. I'll leave the
+        # test here though in case we ever change this script to test using keyspace names without
+        # quotes
+
+        # self.trycompletions('DESC TABLE ' + '"' + self.cqlsh.keyspace + '"', immediate='.')
+
+        self.trycompletions('DESC TABLE ' + '"' + self.cqlsh.keyspace + '".',
+                            choices=['twenty_rows_table',
+                                     'ascii_with_special_chars',
+                                     'users',
+                                     'has_all_types',
+                                     'empty_composite_table',
+                                     'empty_table',
+                                     'undefined_values_table',
+                                     'dynamic_columns',
+                                     'twenty_rows_composite_table',
+                                     'utf8_with_special_chars',
+                                     'songs'],
+                            other_choices_ok=True)
+
+        # See comment above for DESC TABLE
+        # self.trycompletions('DESC TYPE ' + '"' + self.cqlsh.keyspace + '"', immediate='.')
+
+        self.trycompletions('DESC TYPE ' + '"' + self.cqlsh.keyspace + '".',
+                            choices=['address',
+                                     'phone_number',
+                                     'band_info_type',
+                                     'tags'],
+                            other_choices_ok=True)
+
+        # See comment above for DESC TABLE
+        # self.trycompletions('DESC FUNCTION ' + '"' + self.cqlsh.keyspace + '"', immediate='.f')
+
+        self.trycompletions('DESC FUNCTION ' + '"' + self.cqlsh.keyspace + '".', immediate='f')
+
+        self.trycompletions('DESC FUNCTION ' + '"' + self.cqlsh.keyspace + '".f',
+                            choices=['fbestband',
+                                     'fbestsong',
+                                     'fmax',
+                                     'fmin'],
+                            other_choices_ok=True)
+
+        # See comment above for DESC TABLE
+        # self.trycompletions('DESC AGGREGATE ' + '"' + self.cqlsh.keyspace + '"', immediate='.aggm')
+
+        self.trycompletions('DESC AGGREGATE ' + '"' + self.cqlsh.keyspace + '".', immediate='aggm')
+
+        self.trycompletions('DESC AGGREGATE ' + '"' + self.cqlsh.keyspace + '".aggm',
+                            choices=['aggmin',
+                                     'aggmax'],
+                            other_choices_ok=True)
 
     def test_complete_in_drop_columnfamily(self):
         pass
diff --git a/pylib/cqlshlib/test/test_cqlsh_output.py b/pylib/cqlshlib/test/test_cqlsh_output.py
index e3af8e8..e47b981 100644
--- a/pylib/cqlshlib/test/test_cqlsh_output.py
+++ b/pylib/cqlshlib/test/test_cqlsh_output.py
@@ -23,15 +23,16 @@
 from itertools import izip
 from .basecase import (BaseTestCase, cqlshlog, dedent, at_a_time, cqlsh,
                        TEST_HOST, TEST_PORT)
-from .cassconnect import (get_test_keyspace, testrun_cqlsh, testcall_cqlsh,
+from .cassconnect import (get_keyspace, testrun_cqlsh, testcall_cqlsh,
                           cassandra_cursor, split_cql_commands, quote_name)
 from .ansi_colors import (ColoredText, lookup_colorcode, lookup_colorname,
                           lookup_colorletter, ansi_seq)
+import unittest
+import sys
 
 CONTROL_C = '\x03'
 CONTROL_D = '\x04'
 
-
 class TestCqlshOutput(BaseTestCase):
 
     def setUp(self):
@@ -92,7 +93,8 @@
     def test_no_color_output(self):
         for termname in ('', 'dumb', 'vt100'):
             cqlshlog.debug('TERM=%r' % termname)
-            with testrun_cqlsh(tty=True, env={'TERM': termname}) as c:
+            with testrun_cqlsh(tty=True, env={'TERM': termname},
+                               win_force_colors=False) as c:
                 c.send('select * from has_all_types;\n')
                 self.assertNoHasColors(c.read_to_next_prompt())
                 c.send('select count(*) from has_all_types;\n')
@@ -110,13 +112,14 @@
             for line in output:
                 self.assertNoHasColors(line)
                 self.assertNotRegexpMatches(line, r'^cqlsh\S*>')
-            self.assertTrue(6 <= len(output) <= 8,
-                            msg='output: %r' % '\n'.join(output))
+            self.assertEqual(len(output), 6,
+                             msg='output: %r' % '\n'.join(output))
             self.assertEqual(output[0], '')
             self.assertNicelyFormattedTableHeader(output[1])
             self.assertNicelyFormattedTableRule(output[2])
             self.assertNicelyFormattedTableData(output[3])
             self.assertEqual(output[4].strip(), '')
+            self.assertEqual(output[5].strip(), '(1 rows)')
 
     def test_color_output(self):
         for termname in ('xterm', 'unknown-garbage'):
@@ -175,7 +178,7 @@
              MMMMM
             -------
 
-                10
+                20
                 GG
 
 
@@ -274,9 +277,9 @@
         # same query should show up as empty in cql 3
         self.assertQueriesGiveColoredOutput((
             (q, """
-             num | asciicol | bigintcol | blobcol | booleancol | decimalcol | doublecol | floatcol | intcol | textcol | timestampcol | uuidcol | varcharcol | varintcol
-             RRR   MMMMMMMM   MMMMMMMMM   MMMMMMM   MMMMMMMMMM   MMMMMMMMMM   MMMMMMMMM   MMMMMMMM   MMMMMM   MMMMMMM   MMMMMMMMMMMM   MMMMMMM   MMMMMMMMMM   MMMMMMMMM
-            -----+----------+-----------+---------+------------+------------+-----------+----------+--------+---------+--------------+---------+------------+-----------
+             num | asciicol | bigintcol | blobcol | booleancol | decimalcol | doublecol | floatcol | intcol | smallintcol | textcol | timestampcol | tinyintcol | uuidcol | varcharcol | varintcol
+             RRR   MMMMMMMM   MMMMMMMMM   MMMMMMM   MMMMMMMMMM   MMMMMMMMMM   MMMMMMMMM   MMMMMMMM   MMMMMM   MMMMMMMMMMM   MMMMMMM   MMMMMMMMMMMM   MMMMMMMMMM   MMMMMMM   MMMMMMMMMM   MMMMMMMMM
+            -----+----------+-----------+---------+------------+------------+-----------+----------+--------+-------------+---------+--------------+------------+---------+------------+-----------
 
 
             (0 rows)
@@ -367,21 +370,24 @@
             nnnnnnnn
             """),
         ), env={'TZ': 'Etc/UTC'})
+        try:
+            import pytz  # test only if pytz is available on PYTHONPATH
+            self.assertQueriesGiveColoredOutput((
+                ('''select timestampcol from has_all_types where num = 0;''', """
+                 timestampcol
+                 MMMMMMMMMMMM
+                --------------------------
 
-        self.assertQueriesGiveColoredOutput((
-            ('''select timestampcol from has_all_types where num = 0;''', """
-             timestampcol
-             MMMMMMMMMMMM
-            --------------------------
-
-             2012-05-14 07:53:20-0500
-             GGGGGGGGGGGGGGGGGGGGGGGG
+                 2012-05-14 09:53:20-0300
+                 GGGGGGGGGGGGGGGGGGGGGGGG
 
 
-            (1 rows)
-            nnnnnnnn
-            """),
-        ), env={'TZ': 'EST'})
+                (1 rows)
+                nnnnnnnn
+                """),
+            ), env={'TZ': 'America/Sao_Paulo'})
+        except ImportError:
+            pass
 
     def test_boolean_output(self):
         self.assertCqlverQueriesGiveColoredOutput((
@@ -528,10 +534,10 @@
             output = c.read_to_next_prompt().replace('\r\n', '\n')
             self.assertTrue(output.endswith('cqlsh> '))
 
-            cmd = "USE \"%s\";\n" % get_test_keyspace().replace('"', '""')
+            cmd = "USE \"%s\";\n" % get_keyspace().replace('"', '""')
             c.send(cmd)
             output = c.read_to_next_prompt().replace('\r\n', '\n')
-            self.assertTrue(output.endswith('cqlsh:%s> ' % (get_test_keyspace())))
+            self.assertTrue(output.endswith('cqlsh:%s> ' % (get_keyspace())))
 
             c.send('use system;\n')
             output = c.read_to_next_prompt().replace('\r\n', '\n')
@@ -540,18 +546,22 @@
             c.send('use NONEXISTENTKEYSPACE;\n')
             outputlines = c.read_to_next_prompt().splitlines()
 
-            self.assertEqual(outputlines[0], 'use NONEXISTENTKEYSPACE;')
-            self.assertTrue(outputlines[2].endswith('cqlsh:system> '))
-            midline = ColoredText(outputlines[1])
+            start_index = 0
+            if c.realtty:
+                self.assertEqual(outputlines[start_index], 'use NONEXISTENTKEYSPACE;')
+                start_index = 1
+
+            self.assertTrue(outputlines[start_index+1].endswith('cqlsh:system> '))
+            midline = ColoredText(outputlines[start_index])
             self.assertEqual(midline.plain(),
-                             'InvalidRequest: code=2200 [Invalid query] message="Keyspace \'nonexistentkeyspace\' does not exist"')
+                             'InvalidRequest: Error from server: code=2200 [Invalid query] message="Keyspace \'nonexistentkeyspace\' does not exist"')
             self.assertColorFromTags(midline,
                              "RRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRR")
 
     def test_describe_keyspace_output(self):
         fullcqlver = cqlsh.DEFAULT_CQLVER
         with testrun_cqlsh(tty=True, cqlver=fullcqlver) as c:
-            ks = get_test_keyspace()
+            ks = get_keyspace()
             qks = quote_name(ks)
             for cmd in ('describe keyspace', 'desc keyspace'):
                 for givename in ('system', '', qks):
@@ -601,8 +611,10 @@
                 doublecol double,
                 floatcol float,
                 intcol int,
+                smallintcol smallint,
                 textcol text,
                 timestampcol timestamp,
+                tinyintcol tinyint,
                 uuidcol uuid,
                 varcharcol text,
                 varintcol varint
@@ -620,7 +632,7 @@
                 AND read_repair_chance = 0.0
                 AND speculative_retry = '99.0PERCENTILE';
 
-        """ % quote_name(get_test_keyspace()))
+        """ % quote_name(get_keyspace()))
 
         with testrun_cqlsh(tty=True, cqlver=cqlsh.DEFAULT_CQLVER) as c:
             for cmdword in ('describe table', 'desc columnfamily'):
@@ -638,7 +650,7 @@
             \n
         '''
 
-        ks = get_test_keyspace()
+        ks = get_keyspace()
 
         with testrun_cqlsh(tty=True, keyspace=None, cqlver=cqlsh.DEFAULT_CQLVER) as c:
 
@@ -699,7 +711,7 @@
                 self.assertNoHasColors(output)
                 self.assertRegexpMatches(output, output_re + '$')
 
-            c.send('USE %s;\n' % quote_name(get_test_keyspace()))
+            c.send('USE %s;\n' % quote_name(get_keyspace()))
             c.read_to_next_prompt()
 
             for semicolon in ('', ';'):
@@ -728,6 +740,7 @@
             self.assertRegexpMatches(output, '^Connected to .* at %s:%d\.$'
                                              % (re.escape(TEST_HOST), TEST_PORT))
 
+    @unittest.skipIf(sys.platform == "win32", 'EOF signaling not supported on Windows')
     def test_eof_prints_newline(self):
         with testrun_cqlsh(tty=True) as c:
             c.send(CONTROL_D)
@@ -742,8 +755,9 @@
             with testrun_cqlsh(tty=True) as c:
                 cmd = 'exit%s\n' % semicolon
                 c.send(cmd)
-                out = c.read_lines(1)[0].replace('\r', '')
-                self.assertEqual(out, cmd)
+                if c.realtty:
+                    out = c.read_lines(1)[0].replace('\r', '')
+                    self.assertEqual(out, cmd)
                 with self.assertRaises(BaseException) as cm:
                     c.read_lines(1)
                 self.assertIn(type(cm.exception), (EOFError, OSError))
diff --git a/pylib/cqlshlib/test/test_cqlsh_parsing.py b/pylib/cqlshlib/test/test_cqlsh_parsing.py
new file mode 100644
index 0000000..7e7f08b
--- /dev/null
+++ b/pylib/cqlshlib/test/test_cqlsh_parsing.py
@@ -0,0 +1,26 @@
+# 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.
+
+# to configure behavior, define $CQL_TEST_HOST to the destination address
+# for Thrift connections, and $CQL_TEST_PORT to the associated port.
+
+from unittest import TestCase
+
+
+class TestCqlshParsing(TestCase):
+    def test_describe(self):
+        pass
+
diff --git a/pylib/cqlshlib/test/test_keyspace_init.cql b/pylib/cqlshlib/test/test_keyspace_init.cql
index 9a13918..c64163a 100644
--- a/pylib/cqlshlib/test/test_keyspace_init.cql
+++ b/pylib/cqlshlib/test/test_keyspace_init.cql
@@ -8,48 +8,51 @@
     decimalcol decimal,
     doublecol double,
     floatcol float,
+    smallintcol smallint,
     textcol text,
     timestampcol timestamp,
+    tinyintcol tinyint,
     uuidcol uuid,
     varcharcol varchar,
     varintcol varint
 ) WITH compression = {'sstable_compression':'LZ4Compressor'};
 
 INSERT INTO has_all_types (num, intcol, asciicol, bigintcol, blobcol, booleancol,
-                           decimalcol, doublecol, floatcol, textcol,
-                           timestampcol, uuidcol, varcharcol, varintcol)
+                           decimalcol, doublecol, floatcol, smallintcol, textcol,
+                           timestampcol, tinyintcol, uuidcol, varcharcol, varintcol)
 VALUES (0, -12, 'abcdefg', 1234567890123456789, 0x000102030405fffefd, true,
-        19952.11882, 1.0, -2.1, 'Voilá!', '2012-05-14 12:53:20+0000',
-        bd1924e1-6af8-44ae-b5e1-f24131dbd460, '"', 10000000000000000000000000);
+        19952.11882, 1.0, -2.1, 32767, 'Voilá!',
+        '2012-05-14 12:53:20+0000', 127, bd1924e1-6af8-44ae-b5e1-f24131dbd460, '"', 10000000000000000000000000);
 
 INSERT INTO has_all_types (num, intcol, asciicol, bigintcol, blobcol, booleancol,
-                           decimalcol, doublecol, floatcol, textcol,
-                           timestampcol, uuidcol, varcharcol, varintcol)
+                           decimalcol, doublecol, floatcol, smallintcol, textcol,
+                           timestampcol, tinyintcol, uuidcol, varcharcol, varintcol)
 VALUES (1, 2147483647, '__!''$#@!~"', 9223372036854775807, 0xffffffffffffffffff, true,
-        0.00000000000001, 9999999.999, 99999.999, '∭Ƕ⑮ฑ➳❏''', '1900-01-01+0000',
-        ffffffff-ffff-ffff-ffff-ffffffffffff, 'newline->
+        0.00000000000001, 9999999.999, 99999.999, 32767, '∭Ƕ⑮ฑ➳❏''',
+        '1950-01-01+0000', 127, ffffffff-ffff-ffff-ffff-ffffffffffff, 'newline->
 <-', 9);
 
 INSERT INTO has_all_types (num, intcol, asciicol, bigintcol, blobcol, booleancol,
-                           decimalcol, doublecol, floatcol, textcol,
-                           timestampcol, uuidcol, varcharcol, varintcol)
+                           decimalcol, doublecol, floatcol, smallintcol, textcol,
+                           timestampcol, tinyintcol, uuidcol, varcharcol, varintcol)
 VALUES (2, 0, '', 0, 0x, false,
-        0.0, 0.0, 0.0, '', 0,
-        00000000-0000-0000-0000-000000000000, '', 0);
+        0.0, 0.0, 0.0, 0, '',
+        0, 0, 00000000-0000-0000-0000-000000000000, '', 0);
 
 INSERT INTO has_all_types (num, intcol, asciicol, bigintcol, blobcol, booleancol,
-                           decimalcol, doublecol, floatcol, textcol,
-                           timestampcol, uuidcol, varcharcol, varintcol)
+                           decimalcol, doublecol, floatcol, smallintcol, textcol,
+                           timestampcol, tinyintcol, uuidcol, varcharcol, varintcol)
 VALUES (3, -2147483648, '''''''', -9223372036854775808, 0x80, false,
-        10.0000000000000, -1004.10, 100000000.9, '龍馭鬱', '2038-01-19T03:14-1200',
-        ffffffff-ffff-1fff-8fff-ffffffffffff, '''', -10000000000000000000000000);
+        10.0000000000000, -1004.10, 100000000.9, 32767, '龍馭鬱',
+        '2038-01-19T03:14-1200', 127, ffffffff-ffff-1fff-8fff-ffffffffffff,
+	    '''', -10000000000000000000000000);
 
 INSERT INTO has_all_types (num, intcol, asciicol, bigintcol, blobcol, booleancol,
-                           decimalcol, doublecol, floatcol, textcol,
-                           timestampcol, uuidcol, varcharcol, varintcol)
-VALUES (4, blobAsInt(0x), '', blobAsBigint(0x), 0x, blobAsBoolean(0x), blobAsDecimal(0x),
-        blobAsDouble(0x), blobAsFloat(0x), '', blobAsTimestamp(0x), blobAsUuid(0x), '',
-        blobAsVarint(0x));
+                           decimalcol, doublecol, floatcol, smallintcol, textcol,
+                           timestampcol, tinyintcol, uuidcol, varcharcol, varintcol)
+VALUES (4, blobAsInt(0x), '', blobAsBigint(0x), 0x, blobAsBoolean(0x),
+	blobAsDecimal(0x), blobAsDouble(0x), blobAsFloat(0x), blobAsSmallInt(0x0000), '',
+	blobAsTimestamp(0x), blobAsTinyInt(0x00), blobAsUuid(0x), '', blobAsVarint(0x));
 
 
 
@@ -255,3 +258,38 @@
             'origin':'england'
         }
     });
+
+CREATE FUNCTION fBestband ( input double )
+    RETURNS NULL ON NULL INPUT
+    RETURNS text 
+    LANGUAGE java
+    AS 'return "Iron Maiden";';
+
+CREATE FUNCTION fBestsong ( input double )
+    RETURNS NULL ON NULL INPUT
+    RETURNS text 
+    LANGUAGE java
+    AS 'return "Revelations";';
+
+CREATE FUNCTION fMax(current int, candidate int)
+    CALLED ON NULL INPUT
+    RETURNS int 
+    LANGUAGE java 
+    AS 'if (current == null) return candidate; else return Math.max(current, candidate);' ;
+
+CREATE FUNCTION fMin(current int, candidate int)
+    CALLED ON NULL INPUT
+    RETURNS int
+    LANGUAGE java 
+    AS 'if (current == null) return candidate; else return Math.min(current, candidate);' ;
+
+CREATE AGGREGATE aggMax(int)
+    SFUNC fMax
+    STYPE int
+    INITCOND null;
+
+CREATE AGGREGATE aggMin(int)
+    SFUNC fMin
+    STYPE int
+    INITCOND null;
+
diff --git a/pylib/cqlshlib/test/winpty.py b/pylib/cqlshlib/test/winpty.py
new file mode 100644
index 0000000..0db9ec3
--- /dev/null
+++ b/pylib/cqlshlib/test/winpty.py
@@ -0,0 +1,50 @@
+# 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.
+
+from threading import Thread
+from cStringIO import StringIO
+from Queue import Queue, Empty
+
+
+class WinPty:
+
+    def __init__(self, stdin):
+        self._s = stdin
+        self._q = Queue()
+
+        def _read_next_char(stdin, queue):
+            while True:
+                char = stdin.read(1)  # potentially blocking read
+                if char:
+                    queue.put(char)
+                else:
+                    break
+
+        self._t = Thread(target=_read_next_char, args=(self._s, self._q))
+        self._t.daemon = True
+        self._t.start()  # read characters asynchronously from stdin
+
+    def read(self, blksize=-1, timeout=1):
+        buf = StringIO()
+        count = 0
+        try:
+            while count < blksize or blksize == -1:
+                next = self._q.get(block=timeout is not None, timeout=timeout)
+                buf.write(next)
+                count = count + 1
+        except Empty:
+            pass
+        return buf.getvalue()
diff --git a/pylib/cqlshlib/tracing.py b/pylib/cqlshlib/tracing.py
index 40d22f0..cea3568 100644
--- a/pylib/cqlshlib/tracing.py
+++ b/pylib/cqlshlib/tracing.py
@@ -15,18 +15,19 @@
 # limitations under the License.
 
 from cqlshlib.displaying import MAGENTA
-from datetime import datetime
+from datetime import datetime, timedelta
 import time
 from cassandra.query import QueryTrace, TraceUnavailable
 
 
-def print_trace_session(shell, session, session_id):
+def print_trace_session(shell, session, session_id, partial_session=False):
     """
     Lookup a trace by session and trace session ID, then print it.
     """
     trace = QueryTrace(session_id, session)
     try:
-        trace.populate()
+        wait_for_complete = not partial_session
+        trace.populate(wait_for_complete=wait_for_complete)
     except TraceUnavailable:
         shell.printerr("Session %s wasn't found." % session_id)
     else:
@@ -65,18 +66,24 @@
         rows.append(["%s [%s]" % (event.description, event.thread_name),
                      str(datetime_from_utc_to_local(event.datetime)),
                      event.source,
-                     event.source_elapsed.microseconds if event.source_elapsed else "--"])
+                     total_micro_seconds(event.source_elapsed)])
     # append footer row (from sessions table).
     if trace.duration:
         finished_at = (datetime_from_utc_to_local(trace.started_at) + trace.duration)
+        rows.append(['Request complete', str(finished_at), trace.coordinator, total_micro_seconds(trace.duration)])
     else:
         finished_at = trace.duration = "--"
 
-    rows.append(['Request complete', str(finished_at), trace.coordinator, trace.duration.microseconds])
-
     return rows
 
 
+def total_micro_seconds(td):
+    """
+    Convert a timedelta into total microseconds
+    """
+    return int((td.microseconds + (td.seconds + td.days * 24 * 3600) * 10 ** 6)) if td else "--"
+
+
 def datetime_from_utc_to_local(utc_datetime):
     now_timestamp = time.time()
     offset = datetime.fromtimestamp(now_timestamp) - datetime.utcfromtimestamp(now_timestamp)
diff --git a/redhat/cassandra.spec b/redhat/cassandra.spec
index d87f760..ad4095d 100644
--- a/redhat/cassandra.spec
+++ b/redhat/cassandra.spec
@@ -75,6 +75,7 @@
 rm bin/*.orig
 rm bin/*.ps1
 rm bin/cassandra.in.sh
+rm lib/sigar-bin/*winnt*  # strip segfaults on dll..
 rm tools/bin/*.bat
 rm tools/bin/cassandra.in.sh
 
@@ -114,15 +115,16 @@
 %files
 %defattr(0644,root,root,0755)
 %doc CHANGES.txt LICENSE.txt README.asc NEWS.txt NOTICE.txt
-%attr(755,root,root) %{_bindir}/cassandra-cli
 %attr(755,root,root) %{_bindir}/cassandra-stress
 %attr(755,root,root) %{_bindir}/cqlsh
+%attr(755,root,root) %{_bindir}/cqlsh.py
 %attr(755,root,root) %{_bindir}/debug-cql
 %attr(755,root,root) %{_bindir}/nodetool
 %attr(755,root,root) %{_bindir}/sstablekeys
 %attr(755,root,root) %{_bindir}/sstableloader
 %attr(755,root,root) %{_bindir}/sstablescrub
 %attr(755,root,root) %{_bindir}/sstableupgrade
+%attr(755,root,root) %{_bindir}/sstableverify
 %attr(755,root,root) %{_bindir}/stop-server
 %attr(755,root,root) %{_sbindir}/cassandra
 %attr(755,root,root) /%{_sysconfdir}/rc.d/init.d/%{username}
diff --git a/src/java/org/apache/cassandra/auth/AllowAllAuthenticator.java b/src/java/org/apache/cassandra/auth/AllowAllAuthenticator.java
index def6045..bc00c3e 100644
--- a/src/java/org/apache/cassandra/auth/AllowAllAuthenticator.java
+++ b/src/java/org/apache/cassandra/auth/AllowAllAuthenticator.java
@@ -23,45 +23,16 @@
 
 import org.apache.cassandra.exceptions.AuthenticationException;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.exceptions.InvalidRequestException;
 
 public class AllowAllAuthenticator implements IAuthenticator
 {
+    private static final SaslNegotiator AUTHENTICATOR_INSTANCE = new Negotiator();
+
     public boolean requireAuthentication()
     {
         return false;
     }
 
-    public Set<Option> supportedOptions()
-    {
-        return Collections.emptySet();
-    }
-
-    public Set<Option> alterableOptions()
-    {
-        return Collections.emptySet();
-    }
-
-    public AuthenticatedUser authenticate(Map<String, String> credentials) throws AuthenticationException
-    {
-        return AuthenticatedUser.ANONYMOUS_USER;
-    }
-
-    public void create(String username, Map<Option, Object> options) throws InvalidRequestException
-    {
-        throw new InvalidRequestException("CREATE USER operation is not supported by AllowAllAuthenticator");
-    }
-
-    public void alter(String username, Map<Option, Object> options) throws InvalidRequestException
-    {
-        throw new InvalidRequestException("ALTER USER operation is not supported by AllowAllAuthenticator");
-    }
-
-    public void drop(String username) throws InvalidRequestException
-    {
-        throw new InvalidRequestException("DROP USER operation is not supported by AllowAllAuthenticator");
-    }
-
     public Set<IResource> protectedResources()
     {
         return Collections.emptySet();
@@ -74,4 +45,33 @@
     public void setup()
     {
     }
+
+    public SaslNegotiator newSaslNegotiator()
+    {
+        return AUTHENTICATOR_INSTANCE;
+    }
+
+    public AuthenticatedUser legacyAuthenticate(Map<String, String> credentialsData)
+    {
+        return AuthenticatedUser.ANONYMOUS_USER;
+    }
+
+    private static class Negotiator implements SaslNegotiator
+    {
+
+        public byte[] evaluateResponse(byte[] clientResponse) throws AuthenticationException
+        {
+            return null;
+        }
+
+        public boolean isComplete()
+        {
+            return true;
+        }
+
+        public AuthenticatedUser getAuthenticatedUser() throws AuthenticationException
+        {
+            return AuthenticatedUser.ANONYMOUS_USER;
+        }
+    }
 }
diff --git a/src/java/org/apache/cassandra/auth/AllowAllAuthorizer.java b/src/java/org/apache/cassandra/auth/AllowAllAuthorizer.java
index 04b4b52..bc6fee4 100644
--- a/src/java/org/apache/cassandra/auth/AllowAllAuthorizer.java
+++ b/src/java/org/apache/cassandra/auth/AllowAllAuthorizer.java
@@ -20,39 +20,34 @@
 import java.util.Collections;
 import java.util.Set;
 
-import org.apache.cassandra.exceptions.InvalidRequestException;
-
 public class AllowAllAuthorizer implements IAuthorizer
 {
     public Set<Permission> authorize(AuthenticatedUser user, IResource resource)
     {
-        return Permission.ALL;
+        return resource.applicablePermissions();
     }
 
-    public void grant(AuthenticatedUser performer, Set<Permission> permissions, IResource resource, String to)
-    throws InvalidRequestException
+    public void grant(AuthenticatedUser performer, Set<Permission> permissions, IResource resource, RoleResource to)
     {
-        throw new InvalidRequestException("GRANT operation is not supported by AllowAllAuthorizer");
+        throw new UnsupportedOperationException("GRANT operation is not supported by AllowAllAuthorizer");
     }
 
-    public void revoke(AuthenticatedUser performer, Set<Permission> permissions, IResource resource, String from)
-    throws InvalidRequestException
+    public void revoke(AuthenticatedUser performer, Set<Permission> permissions, IResource resource, RoleResource from)
     {
-        throw new InvalidRequestException("REVOKE operation is not supported by AllowAllAuthorizer");
+        throw new UnsupportedOperationException("REVOKE operation is not supported by AllowAllAuthorizer");
     }
 
-    public void revokeAll(String droppedUser)
+    public void revokeAllFrom(RoleResource droppedRole)
     {
     }
 
-    public void revokeAll(IResource droppedResource)
+    public void revokeAllOn(IResource droppedResource)
     {
     }
 
-    public Set<PermissionDetails> list(AuthenticatedUser performer, Set<Permission> permissions, IResource resource, String of)
-    throws InvalidRequestException
+    public Set<PermissionDetails> list(AuthenticatedUser performer, Set<Permission> permissions, IResource resource, RoleResource of)
     {
-        throw new InvalidRequestException("LIST PERMISSIONS operation is not supported by AllowAllAuthorizer");
+        throw new UnsupportedOperationException("LIST PERMISSIONS operation is not supported by AllowAllAuthorizer");
     }
 
     public Set<IResource> protectedResources()
diff --git a/src/java/org/apache/cassandra/auth/Auth.java b/src/java/org/apache/cassandra/auth/Auth.java
deleted file mode 100644
index dac2af8..0000000
--- a/src/java/org/apache/cassandra/auth/Auth.java
+++ /dev/null
@@ -1,295 +0,0 @@
-/*
- * 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.auth;
-
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Lists;
-import org.apache.commons.lang3.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.concurrent.ScheduledExecutors;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.KSMetaData;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.cql3.QueryOptions;
-import org.apache.cassandra.cql3.QueryProcessor;
-import org.apache.cassandra.cql3.UntypedResultSet;
-import org.apache.cassandra.cql3.statements.CFStatement;
-import org.apache.cassandra.cql3.statements.CreateTableStatement;
-import org.apache.cassandra.cql3.statements.SelectStatement;
-import org.apache.cassandra.db.ConsistencyLevel;
-import org.apache.cassandra.exceptions.RequestExecutionException;
-import org.apache.cassandra.exceptions.RequestValidationException;
-import org.apache.cassandra.locator.SimpleStrategy;
-import org.apache.cassandra.service.*;
-import org.apache.cassandra.transport.messages.ResultMessage;
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-public class Auth
-{
-    private static final Logger logger = LoggerFactory.getLogger(Auth.class);
-
-    public static final String DEFAULT_SUPERUSER_NAME = "cassandra";
-
-    public static final long SUPERUSER_SETUP_DELAY = Long.getLong("cassandra.superuser_setup_delay_ms", 10000);
-
-    public static final String AUTH_KS = "system_auth";
-    public static final String USERS_CF = "users";
-
-    // User-level permissions cache.
-    private static final PermissionsCache permissionsCache = new PermissionsCache(DatabaseDescriptor.getAuthorizer());
-
-    private static final String USERS_CF_SCHEMA = String.format("CREATE TABLE %s.%s ("
-                                                                + "name text,"
-                                                                + "super boolean,"
-                                                                + "PRIMARY KEY(name)"
-                                                                + ") WITH gc_grace_seconds=%d",
-                                                                AUTH_KS,
-                                                                USERS_CF,
-                                                                90 * 24 * 60 * 60); // 3 months.
-
-    private static SelectStatement selectUserStatement;
-
-    public static Set<Permission> getPermissions(AuthenticatedUser user, IResource resource)
-    {
-        return permissionsCache.getPermissions(user, resource);
-    }
-
-    /**
-     * Checks if the username is stored in AUTH_KS.USERS_CF.
-     *
-     * @param username Username to query.
-     * @return whether or not Cassandra knows about the user.
-     */
-    public static boolean isExistingUser(String username)
-    {
-        return !selectUser(username).isEmpty();
-    }
-
-    /**
-     * Checks if the user is a known superuser.
-     *
-     * @param username Username to query.
-     * @return true is the user is a superuser, false if they aren't or don't exist at all.
-     */
-    public static boolean isSuperuser(String username)
-    {
-        UntypedResultSet result = selectUser(username);
-        return !result.isEmpty() && result.one().getBoolean("super");
-    }
-
-    /**
-     * Inserts the user into AUTH_KS.USERS_CF (or overwrites their superuser status as a result of an ALTER USER query).
-     *
-     * @param username Username to insert.
-     * @param isSuper User's new status.
-     * @throws RequestExecutionException
-     */
-    public static void insertUser(String username, boolean isSuper) throws RequestExecutionException
-    {
-        QueryProcessor.process(String.format("INSERT INTO %s.%s (name, super) VALUES ('%s', %s)",
-                                             AUTH_KS,
-                                             USERS_CF,
-                                             escape(username),
-                                             isSuper),
-                               consistencyForUser(username));
-    }
-
-    /**
-     * Deletes the user from AUTH_KS.USERS_CF.
-     *
-     * @param username Username to delete.
-     * @throws RequestExecutionException
-     */
-    public static void deleteUser(String username) throws RequestExecutionException
-    {
-        QueryProcessor.process(String.format("DELETE FROM %s.%s WHERE name = '%s'",
-                                             AUTH_KS,
-                                             USERS_CF,
-                                             escape(username)),
-                               consistencyForUser(username));
-    }
-
-    /**
-     * Sets up Authenticator and Authorizer.
-     */
-    public static void setup()
-    {
-        if (DatabaseDescriptor.getAuthenticator() instanceof AllowAllAuthenticator)
-            return;
-
-        setupAuthKeyspace();
-        setupTable(USERS_CF, USERS_CF_SCHEMA);
-
-        DatabaseDescriptor.getAuthenticator().setup();
-        DatabaseDescriptor.getAuthorizer().setup();
-
-        // register a custom MigrationListener for permissions cleanup after dropped keyspaces/cfs.
-        MigrationManager.instance.register(new AuthMigrationListener());
-
-        // the delay is here to give the node some time to see its peers - to reduce
-        // "Skipped default superuser setup: some nodes were not ready" log spam.
-        // It's the only reason for the delay.
-        ScheduledExecutors.nonPeriodicTasks.schedule(new Runnable()
-        {
-            public void run()
-            {
-                setupDefaultSuperuser();
-            }
-        }, SUPERUSER_SETUP_DELAY, TimeUnit.MILLISECONDS);
-
-        try
-        {
-            String query = String.format("SELECT * FROM %s.%s WHERE name = ?", AUTH_KS, USERS_CF);
-            selectUserStatement = (SelectStatement) QueryProcessor.parseStatement(query).prepare().statement;
-        }
-        catch (RequestValidationException e)
-        {
-            throw new AssertionError(e); // not supposed to happen
-        }
-    }
-
-    // Only use QUORUM cl for the default superuser.
-    private static ConsistencyLevel consistencyForUser(String username)
-    {
-        if (username.equals(DEFAULT_SUPERUSER_NAME))
-            return ConsistencyLevel.QUORUM;
-        else
-            return ConsistencyLevel.LOCAL_ONE;
-    }
-
-    private static void setupAuthKeyspace()
-    {
-        if (Schema.instance.getKSMetaData(AUTH_KS) == null)
-        {
-            try
-            {
-                KSMetaData ksm = KSMetaData.newKeyspace(AUTH_KS, SimpleStrategy.class.getName(), ImmutableMap.of("replication_factor", "1"), true);
-                MigrationManager.announceNewKeyspace(ksm, 0, false);
-            }
-            catch (Exception e)
-            {
-                throw new AssertionError(e); // shouldn't ever happen.
-            }
-        }
-    }
-
-    /**
-     * Set up table from given CREATE TABLE statement under system_auth keyspace, if not already done so.
-     *
-     * @param name name of the table
-     * @param cql CREATE TABLE statement
-     */
-    public static void setupTable(String name, String cql)
-    {
-        if (Schema.instance.getCFMetaData(AUTH_KS, name) == null)
-        {
-            try
-            {
-                CFStatement parsed = (CFStatement)QueryProcessor.parseStatement(cql);
-                parsed.prepareKeyspace(AUTH_KS);
-                CreateTableStatement statement = (CreateTableStatement) parsed.prepare().statement;
-                CFMetaData cfm = statement.getCFMetaData().copy(CFMetaData.generateLegacyCfId(AUTH_KS, name));
-                assert cfm.cfName.equals(name);
-                MigrationManager.announceNewColumnFamily(cfm);
-            }
-            catch (Exception e)
-            {
-                throw new AssertionError(e);
-            }
-        }
-    }
-
-    private static void setupDefaultSuperuser()
-    {
-        try
-        {
-            // insert a default superuser if AUTH_KS.USERS_CF is empty.
-            if (!hasExistingUsers())
-            {
-                QueryProcessor.process(String.format("INSERT INTO %s.%s (name, super) VALUES ('%s', %s) USING TIMESTAMP 0",
-                                                     AUTH_KS,
-                                                     USERS_CF,
-                                                     DEFAULT_SUPERUSER_NAME,
-                                                     true),
-                                       ConsistencyLevel.ONE);
-                logger.info("Created default superuser '{}'", DEFAULT_SUPERUSER_NAME);
-            }
-        }
-        catch (RequestExecutionException e)
-        {
-            logger.warn("Skipped default superuser setup: some nodes were not ready");
-        }
-    }
-
-    private static boolean hasExistingUsers() throws RequestExecutionException
-    {
-        // Try looking up the 'cassandra' default super user first, to avoid the range query if possible.
-        String defaultSUQuery = String.format("SELECT * FROM %s.%s WHERE name = '%s'", AUTH_KS, USERS_CF, DEFAULT_SUPERUSER_NAME);
-        String allUsersQuery = String.format("SELECT * FROM %s.%s LIMIT 1", AUTH_KS, USERS_CF);
-        return !QueryProcessor.process(defaultSUQuery, ConsistencyLevel.ONE).isEmpty()
-            || !QueryProcessor.process(defaultSUQuery, ConsistencyLevel.QUORUM).isEmpty()
-            || !QueryProcessor.process(allUsersQuery, ConsistencyLevel.QUORUM).isEmpty();
-    }
-
-    // we only worry about one character ('). Make sure it's properly escaped.
-    private static String escape(String name)
-    {
-        return StringUtils.replace(name, "'", "''");
-    }
-
-    private static UntypedResultSet selectUser(String username)
-    {
-        try
-        {
-            ResultMessage.Rows rows = selectUserStatement.execute(QueryState.forInternalCalls(),
-                                                                  QueryOptions.forInternalCalls(consistencyForUser(username),
-                                                                                                Lists.newArrayList(ByteBufferUtil.bytes(username))));
-            return UntypedResultSet.create(rows.result);
-        }
-        catch (RequestValidationException e)
-        {
-            throw new AssertionError(e); // not supposed to happen
-        }
-        catch (RequestExecutionException e)
-        {
-            throw new RuntimeException(e);
-        }
-    }
-
-    /**
-     * MigrationListener implementation that cleans up permissions on dropped resources.
-     */
-    public static class AuthMigrationListener extends MigrationListener
-    {
-        public void onDropKeyspace(String ksName)
-        {
-            DatabaseDescriptor.getAuthorizer().revokeAll(DataResource.keyspace(ksName));
-        }
-
-        public void onDropColumnFamily(String ksName, String cfName)
-        {
-            DatabaseDescriptor.getAuthorizer().revokeAll(DataResource.columnFamily(ksName, cfName));
-        }
-    }
-}
diff --git a/src/java/org/apache/cassandra/auth/AuthKeyspace.java b/src/java/org/apache/cassandra/auth/AuthKeyspace.java
new file mode 100644
index 0000000..199b6e2
--- /dev/null
+++ b/src/java/org/apache/cassandra/auth/AuthKeyspace.java
@@ -0,0 +1,90 @@
+/*
+ * 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.auth;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.locator.SimpleStrategy;
+
+public class AuthKeyspace
+{
+    public static final String NAME = "system_auth";
+
+    public static final String ROLES = "roles";
+    public static final String ROLE_MEMBERS = "role_members";
+    public static final String ROLE_PERMISSIONS = "role_permissions";
+    public static final String RESOURCE_ROLE_INDEX = "resource_role_permissons_index";
+
+    public static final long SUPERUSER_SETUP_DELAY = Long.getLong("cassandra.superuser_setup_delay_ms", 10000);
+
+    private static final CFMetaData Roles =
+        compile(ROLES,
+                "role definitions",
+                "CREATE TABLE %s ("
+                + "role text,"
+                + "is_superuser boolean,"
+                + "can_login boolean,"
+                + "salted_hash text,"
+                + "member_of set<text>,"
+                + "PRIMARY KEY(role))");
+
+    private static final CFMetaData RoleMembers =
+        compile(ROLE_MEMBERS,
+                "role memberships lookup table",
+                "CREATE TABLE %s ("
+                + "role text,"
+                + "member text,"
+                + "PRIMARY KEY(role, member))");
+
+    private static final CFMetaData RolePermissions =
+        compile(ROLE_PERMISSIONS,
+                "permissions granted to db roles",
+                "CREATE TABLE %s ("
+                + "role text,"
+                + "resource text,"
+                + "permissions set<text>,"
+                + "PRIMARY KEY(role, resource))");
+
+    private static final CFMetaData ResourceRoleIndex =
+        compile(RESOURCE_ROLE_INDEX,
+                "index of db roles with permissions granted on a resource",
+                "CREATE TABLE %s ("
+                + "resource text,"
+                + "role text,"
+                + "PRIMARY KEY(resource, role))");
+
+
+    private static CFMetaData compile(String name, String description, String schema)
+    {
+        return CFMetaData.compile(String.format(schema, name), NAME)
+                         .comment(description)
+                         .gcGraceSeconds((int) TimeUnit.DAYS.toSeconds(90));
+    }
+
+    public static KSMetaData definition()
+    {
+        List<CFMetaData> tables = Arrays.asList(Roles, RoleMembers, RolePermissions, ResourceRoleIndex);
+        return new KSMetaData(NAME, SimpleStrategy.class, ImmutableMap.of("replication_factor", "1"), true, tables);
+    }
+}
diff --git a/src/java/org/apache/cassandra/auth/AuthMigrationListener.java b/src/java/org/apache/cassandra/auth/AuthMigrationListener.java
new file mode 100644
index 0000000..64fe7c6
--- /dev/null
+++ b/src/java/org/apache/cassandra/auth/AuthMigrationListener.java
@@ -0,0 +1,53 @@
+/*
+ * 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.auth;
+
+import java.util.List;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.service.MigrationListener;
+
+/**
+ * MigrationListener implementation that cleans up permissions on dropped resources.
+ */
+public class AuthMigrationListener extends MigrationListener
+{
+    public void onDropKeyspace(String ksName)
+    {
+        DatabaseDescriptor.getAuthorizer().revokeAllOn(DataResource.keyspace(ksName));
+        DatabaseDescriptor.getAuthorizer().revokeAllOn(FunctionResource.keyspace(ksName));
+    }
+
+    public void onDropColumnFamily(String ksName, String cfName)
+    {
+        DatabaseDescriptor.getAuthorizer().revokeAllOn(DataResource.table(ksName, cfName));
+    }
+
+    public void onDropFunction(String ksName, String functionName, List<AbstractType<?>> argTypes)
+    {
+        DatabaseDescriptor.getAuthorizer()
+                          .revokeAllOn(FunctionResource.function(ksName, functionName, argTypes));
+    }
+
+    public void onDropAggregate(String ksName, String aggregateName, List<AbstractType<?>> argTypes)
+    {
+        DatabaseDescriptor.getAuthorizer()
+                          .revokeAllOn(FunctionResource.function(ksName, aggregateName, argTypes));
+    }
+}
diff --git a/src/java/org/apache/cassandra/auth/AuthenticatedUser.java b/src/java/org/apache/cassandra/auth/AuthenticatedUser.java
index e142acf..5e57308 100644
--- a/src/java/org/apache/cassandra/auth/AuthenticatedUser.java
+++ b/src/java/org/apache/cassandra/auth/AuthenticatedUser.java
@@ -17,21 +17,37 @@
  */
 package org.apache.cassandra.auth;
 
+import java.util.Set;
+
 import com.google.common.base.Objects;
 
+import org.apache.cassandra.config.DatabaseDescriptor;
+
 /**
  * Returned from IAuthenticator#authenticate(), represents an authenticated user everywhere internally.
+ *
+ * Holds the name of the user and the roles that have been granted to the user. The roles will be cached
+ * for roles_validity_in_ms.
  */
 public class AuthenticatedUser
 {
+    public static final String SYSTEM_USERNAME = "system";
+    public static final AuthenticatedUser SYSTEM_USER = new AuthenticatedUser(SYSTEM_USERNAME);
+
     public static final String ANONYMOUS_USERNAME = "anonymous";
     public static final AuthenticatedUser ANONYMOUS_USER = new AuthenticatedUser(ANONYMOUS_USERNAME);
 
+    // User-level permissions cache.
+    private static final PermissionsCache permissionsCache = new PermissionsCache(DatabaseDescriptor.getAuthorizer());
+
     private final String name;
+    // primary Role of the logged in user
+    private final RoleResource role;
 
     public AuthenticatedUser(String name)
     {
         this.name = name;
+        this.role = RoleResource.role(name);
     }
 
     public String getName()
@@ -39,6 +55,11 @@
         return name;
     }
 
+    public RoleResource getPrimaryRole()
+    {
+        return role;
+    }
+
     /**
      * Checks the user's superuser status.
      * Only a superuser is allowed to perform CREATE USER and DROP USER queries.
@@ -47,7 +68,7 @@
      */
     public boolean isSuper()
     {
-        return !isAnonymous() && Auth.isSuperuser(name);
+        return !isAnonymous() && Roles.hasSuperuserStatus(role);
     }
 
     /**
@@ -58,6 +79,31 @@
         return this == ANONYMOUS_USER;
     }
 
+    /**
+     * Some internal operations are performed on behalf of Cassandra itself, in those cases
+     * the system user should be used where an identity is required
+     * see CreateRoleStatement#execute() and overrides of SchemaAlteringStatement#grantPermissionsToCreator()
+     */
+    public boolean isSystem()
+    {
+        return this == SYSTEM_USER;
+    }
+
+    /**
+     * Get the roles that have been granted to the user via the IRoleManager
+     *
+     * @return a list of roles that have been granted to the user
+     */
+    public Set<RoleResource> getRoles()
+    {
+        return Roles.getRoles(role);
+    }
+
+    public Set<Permission> getPermissions(IResource resource)
+    {
+        return permissionsCache.getPermissions(this, resource);
+    }
+
     @Override
     public String toString()
     {
diff --git a/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java b/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java
index 49eefb1..360d59a 100644
--- a/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java
+++ b/src/java/org/apache/cassandra/auth/CassandraAuthorizer.java
@@ -18,63 +18,69 @@
 package org.apache.cassandra.auth;
 
 import java.util.*;
+import java.util.concurrent.TimeUnit;
 
+import com.google.common.base.Predicate;
 import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.cql3.UntypedResultSet;
-import org.apache.cassandra.cql3.QueryProcessor;
-import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.concurrent.ScheduledExecutors;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.cql3.statements.BatchStatement;
+import org.apache.cassandra.cql3.statements.ModificationStatement;
 import org.apache.cassandra.cql3.statements.SelectStatement;
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.exceptions.*;
+import org.apache.cassandra.serializers.SetSerializer;
+import org.apache.cassandra.serializers.UTF8Serializer;
+import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.transport.messages.ResultMessage;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 /**
  * CassandraAuthorizer is an IAuthorizer implementation that keeps
- * permissions internally in C* - in system_auth.permissions CQL3 table.
+ * user permissions internally in C* using the system_auth.role_permissions
+ * table.
  */
 public class CassandraAuthorizer implements IAuthorizer
 {
     private static final Logger logger = LoggerFactory.getLogger(CassandraAuthorizer.class);
 
-    private static final String USERNAME = "username";
+    private static final String ROLE = "role";
     private static final String RESOURCE = "resource";
     private static final String PERMISSIONS = "permissions";
 
-    private static final String PERMISSIONS_CF = "permissions";
-    private static final String PERMISSIONS_CF_SCHEMA = String.format("CREATE TABLE %s.%s ("
-                                                                      + "username text,"
-                                                                      + "resource text,"
-                                                                      + "permissions set<text>,"
-                                                                      + "PRIMARY KEY(username, resource)"
-                                                                      + ") WITH gc_grace_seconds=%d",
-                                                                      Auth.AUTH_KS,
-                                                                      PERMISSIONS_CF,
-                                                                      90 * 24 * 60 * 60); // 3 months.
+    // used during upgrades to perform authz on mixed clusters
+    public static final String USERNAME = "username";
+    public static final String USER_PERMISSIONS = "permissions";
 
-    private SelectStatement authorizeStatement;
+    private SelectStatement authorizeRoleStatement;
+    private SelectStatement legacyAuthorizeRoleStatement;
 
-    // Returns every permission on the resource granted to the user.
+    public CassandraAuthorizer()
+    {
+    }
+
+    // Returns every permission on the resource granted to the user either directly
+    // or indirectly via roles granted to the user.
     public Set<Permission> authorize(AuthenticatedUser user, IResource resource)
     {
         if (user.isSuper())
-            return Permission.ALL;
+            return resource.applicablePermissions();
 
-        UntypedResultSet result;
+        Set<Permission> permissions = EnumSet.noneOf(Permission.class);
         try
         {
-            ResultMessage.Rows rows = authorizeStatement.execute(QueryState.forInternalCalls(),
-                                                                 QueryOptions.forInternalCalls(ConsistencyLevel.LOCAL_ONE,
-                                                                                               Lists.newArrayList(ByteBufferUtil.bytes(user.getName()),
-                                                                                                                  ByteBufferUtil.bytes(resource.getName()))));
-            result = UntypedResultSet.create(rows.result);
+            for (RoleResource role: user.getRoles())
+                addPermissionsForRole(permissions, resource, role);
         }
         catch (RequestValidationException e)
         {
@@ -86,53 +92,217 @@
             throw new RuntimeException(e);
         }
 
-        if (result.isEmpty() || !result.one().has(PERMISSIONS))
-            return Permission.NONE;
-
-        Set<Permission> permissions = EnumSet.noneOf(Permission.class);
-        for (String perm : result.one().getSet(PERMISSIONS, UTF8Type.instance))
-            permissions.add(Permission.valueOf(perm));
         return permissions;
     }
 
-    public void grant(AuthenticatedUser performer, Set<Permission> permissions, IResource resource, String to)
-    throws RequestExecutionException
+    public void grant(AuthenticatedUser performer, Set<Permission> permissions, IResource resource, RoleResource grantee)
+    throws RequestValidationException, RequestExecutionException
     {
-        modify(permissions, resource, to, "+");
+        modifyRolePermissions(permissions, resource, grantee, "+");
+        addLookupEntry(resource, grantee);
     }
 
-    public void revoke(AuthenticatedUser performer, Set<Permission> permissions, IResource resource, String from)
-    throws RequestExecutionException
+    public void revoke(AuthenticatedUser performer, Set<Permission> permissions, IResource resource, RoleResource revokee)
+    throws RequestValidationException, RequestExecutionException
     {
-        modify(permissions, resource, from, "-");
+        modifyRolePermissions(permissions, resource, revokee, "-");
+        removeLookupEntry(resource, revokee);
     }
 
-    // Adds or removes permissions from user's 'permissions' set (adds if op is "+", removes if op is "-")
-    private void modify(Set<Permission> permissions, IResource resource, String user, String op) throws RequestExecutionException
+    // Called when deleting a role with DROP ROLE query.
+    // Internal hook, so no permission checks are needed here.
+    // Executes a logged batch removing the granted premissions
+    // for the role as well as the entries from the reverse index
+    // table
+    public void revokeAllFrom(RoleResource revokee)
     {
-        process(String.format("UPDATE %s.%s SET permissions = permissions %s {%s} WHERE username = '%s' AND resource = '%s'",
-                              Auth.AUTH_KS,
-                              PERMISSIONS_CF,
+        try
+        {
+            UntypedResultSet rows = process(String.format("SELECT resource FROM %s.%s WHERE role = '%s'",
+                                                          AuthKeyspace.NAME,
+                                                          AuthKeyspace.ROLE_PERMISSIONS,
+                                                          escape(revokee.getRoleName())));
+
+            List<CQLStatement> statements = new ArrayList<>();
+            for (UntypedResultSet.Row row : rows)
+            {
+                statements.add(
+                    QueryProcessor.getStatement(String.format("DELETE FROM %s.%s WHERE resource = '%s' AND role = '%s'",
+                                                              AuthKeyspace.NAME,
+                                                              AuthKeyspace.RESOURCE_ROLE_INDEX,
+                                                              escape(row.getString("resource")),
+                                                              escape(revokee.getRoleName())),
+                                                ClientState.forInternalCalls()).statement);
+
+            }
+
+            statements.add(QueryProcessor.getStatement(String.format("DELETE FROM %s.%s WHERE role = '%s'",
+                                                                     AuthKeyspace.NAME,
+                                                                     AuthKeyspace.ROLE_PERMISSIONS,
+                                                                     escape(revokee.getRoleName())),
+                                                       ClientState.forInternalCalls()).statement);
+
+            executeLoggedBatch(statements);
+        }
+        catch (RequestExecutionException | RequestValidationException e)
+        {
+            logger.warn("CassandraAuthorizer failed to revoke all permissions of {}: {}",  revokee.getRoleName(), e);
+        }
+    }
+
+    // Called after a resource is removed (DROP KEYSPACE, DROP TABLE, etc.).
+    // Execute a logged batch removing all the permissions for the resource
+    // as well as the index table entry
+    public void revokeAllOn(IResource droppedResource)
+    {
+        try
+        {
+            UntypedResultSet rows = process(String.format("SELECT role FROM %s.%s WHERE resource = '%s'",
+                                                          AuthKeyspace.NAME,
+                                                          AuthKeyspace.RESOURCE_ROLE_INDEX,
+                                                          escape(droppedResource.getName())));
+
+            List<CQLStatement> statements = new ArrayList<>();
+            for (UntypedResultSet.Row row : rows)
+            {
+                statements.add(QueryProcessor.getStatement(String.format("DELETE FROM %s.%s WHERE role = '%s' AND resource = '%s'",
+                                                                         AuthKeyspace.NAME,
+                                                                         AuthKeyspace.ROLE_PERMISSIONS,
+                                                                         escape(row.getString("role")),
+                                                                         escape(droppedResource.getName())),
+                                                           ClientState.forInternalCalls()).statement);
+            }
+
+            statements.add(QueryProcessor.getStatement(String.format("DELETE FROM %s.%s WHERE resource = '%s'",
+                                                                                            AuthKeyspace.NAME,
+                                                                                            AuthKeyspace.RESOURCE_ROLE_INDEX,
+                                                                                            escape(droppedResource.getName())),
+                                                                               ClientState.forInternalCalls()).statement);
+
+            executeLoggedBatch(statements);
+        }
+        catch (RequestExecutionException | RequestValidationException e)
+        {
+            logger.warn("CassandraAuthorizer failed to revoke all permissions on {}: {}", droppedResource, e);
+            return;
+        }
+    }
+
+    private void executeLoggedBatch(List<CQLStatement> statements)
+    throws RequestExecutionException, RequestValidationException
+    {
+        BatchStatement batch = new BatchStatement(0,
+                                                  BatchStatement.Type.LOGGED,
+                                                  Lists.newArrayList(Iterables.filter(statements, ModificationStatement.class)),
+                                                  Attributes.none());
+        QueryProcessor.instance.processBatch(batch,
+                                             QueryState.forInternalCalls(),
+                                             BatchQueryOptions.withoutPerStatementVariables(QueryOptions.DEFAULT));
+
+    }
+
+    // Add every permission on the resource granted to the role
+    private void addPermissionsForRole(Set<Permission> permissions, IResource resource, RoleResource role)
+    throws RequestExecutionException, RequestValidationException
+    {
+        QueryOptions options = QueryOptions.forInternalCalls(ConsistencyLevel.LOCAL_ONE,
+                                                             Lists.newArrayList(ByteBufferUtil.bytes(role.getRoleName()),
+                                                                                ByteBufferUtil.bytes(resource.getName())));
+
+        SelectStatement statement;
+        // If it exists, read from the legacy user permissions table to handle the case where the cluster
+        // is being upgraded and so is running with mixed versions of the authz schema
+        if (Schema.instance.getCFMetaData(AuthKeyspace.NAME, USER_PERMISSIONS) == null)
+            statement = authorizeRoleStatement;
+        else
+        {
+            // If the permissions table was initialised only after the statement got prepared, re-prepare (CASSANDRA-12813)
+            if (legacyAuthorizeRoleStatement == null)
+                legacyAuthorizeRoleStatement = prepare(USERNAME, USER_PERMISSIONS);
+            statement = legacyAuthorizeRoleStatement;
+        }
+
+        ResultMessage.Rows rows = statement.execute(QueryState.forInternalCalls(), options) ;
+        UntypedResultSet result = UntypedResultSet.create(rows.result);
+
+        if (!result.isEmpty() && result.one().has(PERMISSIONS))
+        {
+            for (String perm : result.one().getSet(PERMISSIONS, UTF8Type.instance))
+            {
+                permissions.add(Permission.valueOf(perm));
+            }
+        }
+    }
+
+    // Adds or removes permissions from a role_permissions table (adds if op is "+", removes if op is "-")
+    private void modifyRolePermissions(Set<Permission> permissions, IResource resource, RoleResource role, String op)
+            throws RequestExecutionException
+    {
+        process(String.format("UPDATE %s.%s SET permissions = permissions %s {%s} WHERE role = '%s' AND resource = '%s'",
+                              AuthKeyspace.NAME,
+                              AuthKeyspace.ROLE_PERMISSIONS,
                               op,
                               "'" + StringUtils.join(permissions, "','") + "'",
-                              escape(user),
+                              escape(role.getRoleName()),
                               escape(resource.getName())));
     }
 
+    // Removes an entry from the inverted index table (from resource -> role with defined permissions)
+    private void removeLookupEntry(IResource resource, RoleResource role) throws RequestExecutionException
+    {
+        process(String.format("DELETE FROM %s.%s WHERE resource = '%s' and role = '%s'",
+                AuthKeyspace.NAME,
+                AuthKeyspace.RESOURCE_ROLE_INDEX,
+                escape(resource.getName()),
+                escape(role.getRoleName())));
+    }
+
+    // Adds an entry to the inverted index table (from resource -> role with defined permissions)
+    private void addLookupEntry(IResource resource, RoleResource role) throws RequestExecutionException
+    {
+        process(String.format("INSERT INTO %s.%s (resource, role) VALUES ('%s','%s')",
+                              AuthKeyspace.NAME,
+                              AuthKeyspace.RESOURCE_ROLE_INDEX,
+                              escape(resource.getName()),
+                              escape(role.getRoleName())));
+    }
+
     // 'of' can be null - in that case everyone's permissions have been requested. Otherwise only single user's.
-    // If the user requesting 'LIST PERMISSIONS' is not a superuser OR his username doesn't match 'of', we
+    // If the user requesting 'LIST PERMISSIONS' is not a superuser OR their username doesn't match 'of', we
     // throw UnauthorizedException. So only a superuser can view everybody's permissions. Regular users are only
     // allowed to see their own permissions.
-    public Set<PermissionDetails> list(AuthenticatedUser performer, Set<Permission> permissions, IResource resource, String of)
+    public Set<PermissionDetails> list(AuthenticatedUser performer,
+                                       Set<Permission> permissions,
+                                       IResource resource,
+                                       RoleResource grantee)
     throws RequestValidationException, RequestExecutionException
     {
-        if (!performer.isSuper() && !performer.getName().equals(of))
+        if (!performer.isSuper() && !performer.getRoles().contains(grantee))
             throw new UnauthorizedException(String.format("You are not authorized to view %s's permissions",
-                                                          of == null ? "everyone" : of));
+                                                          grantee == null ? "everyone" : grantee.getRoleName()));
 
-        Set<PermissionDetails> details = new HashSet<PermissionDetails>();
+        if (null == grantee)
+            return listPermissionsForRole(permissions, resource, grantee);
 
-        for (UntypedResultSet.Row row : process(buildListQuery(resource, of)))
+        Set<RoleResource> roles = DatabaseDescriptor.getRoleManager().getRoles(grantee, true);
+        Set<PermissionDetails> details = new HashSet<>();
+        for (RoleResource role : roles)
+            details.addAll(listPermissionsForRole(permissions, resource, role));
+
+        return details;
+    }
+
+    private Set<PermissionDetails> listPermissionsForRole(Set<Permission> permissions,
+                                                          IResource resource,
+                                                          RoleResource role)
+    throws RequestExecutionException
+    {
+        Set<PermissionDetails> details = new HashSet<>();
+        // If it exists, try the legacy user permissions table first. This is to handle the case
+        // where the cluster is being upgraded and so is running with mixed versions of the perms table
+        boolean useLegacyTable = Schema.instance.getCFMetaData(AuthKeyspace.NAME, USER_PERMISSIONS) != null;
+        String entityColumnName = useLegacyTable ? USERNAME : ROLE;
+        for (UntypedResultSet.Row row : process(buildListQuery(resource, role, useLegacyTable)))
         {
             if (row.has(PERMISSIONS))
             {
@@ -140,20 +310,21 @@
                 {
                     Permission permission = Permission.valueOf(p);
                     if (permissions.contains(permission))
-                        details.add(new PermissionDetails(row.getString(USERNAME),
-                                                          DataResource.fromName(row.getString(RESOURCE)),
+                        details.add(new PermissionDetails(row.getString(entityColumnName),
+                                                          Resources.fromName(row.getString(RESOURCE)),
                                                           permission));
                 }
             }
         }
-
         return details;
     }
 
-    private static String buildListQuery(IResource resource, String of)
+    private String buildListQuery(IResource resource, RoleResource grantee, boolean useLegacyTable)
     {
-        List<String> vars = Lists.newArrayList(Auth.AUTH_KS, PERMISSIONS_CF);
-        List<String> conditions = new ArrayList<String>();
+        String tableName = useLegacyTable ? USER_PERMISSIONS : AuthKeyspace.ROLE_PERMISSIONS;
+        String entityName = useLegacyTable ? USERNAME : ROLE;
+        List<String> vars = Lists.newArrayList(AuthKeyspace.NAME, tableName);
+        List<String> conditions = new ArrayList<>();
 
         if (resource != null)
         {
@@ -161,75 +332,27 @@
             vars.add(escape(resource.getName()));
         }
 
-        if (of != null)
+        if (grantee != null)
         {
-            conditions.add("username = '%s'");
-            vars.add(escape(of));
+            conditions.add(entityName + " = '%s'");
+            vars.add(escape(grantee.getRoleName()));
         }
 
-        String query = "SELECT username, resource, permissions FROM %s.%s";
+        String query = "SELECT " + entityName + ", resource, permissions FROM %s.%s";
 
         if (!conditions.isEmpty())
             query += " WHERE " + StringUtils.join(conditions, " AND ");
 
-        if (resource != null && of == null)
+        if (resource != null && grantee == null)
             query += " ALLOW FILTERING";
 
         return String.format(query, vars.toArray());
     }
 
-    // Called prior to deleting the user with DROP USER query. Internal hook, so no permission checks are needed here.
-    public void revokeAll(String droppedUser)
-    {
-        try
-        {
-            process(String.format("DELETE FROM %s.%s WHERE username = '%s'", Auth.AUTH_KS, PERMISSIONS_CF, escape(droppedUser)));
-        }
-        catch (RequestExecutionException e)
-        {
-            logger.warn("CassandraAuthorizer failed to revoke all permissions of {}: {}", droppedUser, e);
-        }
-    }
-
-    // Called after a resource is removed (DROP KEYSPACE, DROP TABLE, etc.).
-    public void revokeAll(IResource droppedResource)
-    {
-
-        UntypedResultSet rows;
-        try
-        {
-            // TODO: switch to secondary index on 'resource' once https://issues.apache.org/jira/browse/CASSANDRA-5125 is resolved.
-            rows = process(String.format("SELECT username FROM %s.%s WHERE resource = '%s' ALLOW FILTERING",
-                                         Auth.AUTH_KS,
-                                         PERMISSIONS_CF,
-                                         escape(droppedResource.getName())));
-        }
-        catch (RequestExecutionException e)
-        {
-            logger.warn("CassandraAuthorizer failed to revoke all permissions on {}: {}", droppedResource, e);
-            return;
-        }
-
-        for (UntypedResultSet.Row row : rows)
-        {
-            try
-            {
-                process(String.format("DELETE FROM %s.%s WHERE username = '%s' AND resource = '%s'",
-                                      Auth.AUTH_KS,
-                                      PERMISSIONS_CF,
-                                      escape(row.getString(USERNAME)),
-                                      escape(droppedResource.getName())));
-            }
-            catch (RequestExecutionException e)
-            {
-                logger.warn("CassandraAuthorizer failed to revoke all permissions on {}: {}", droppedResource, e);
-            }
-        }
-    }
 
     public Set<DataResource> protectedResources()
     {
-        return ImmutableSet.of(DataResource.columnFamily(Auth.AUTH_KS, PERMISSIONS_CF));
+        return ImmutableSet.of(DataResource.table(AuthKeyspace.NAME, AuthKeyspace.ROLE_PERMISSIONS));
     }
 
     public void validateConfiguration() throws ConfigurationException
@@ -238,27 +361,104 @@
 
     public void setup()
     {
-        Auth.setupTable(PERMISSIONS_CF, PERMISSIONS_CF_SCHEMA);
+        authorizeRoleStatement = prepare(ROLE, AuthKeyspace.ROLE_PERMISSIONS);
 
+        // If old user permissions table exists, migrate the legacy authz data to the new table
+        // The delay is to give the node a chance to see its peers before attempting the conversion
+        if (Schema.instance.getCFMetaData(AuthKeyspace.NAME, "permissions") != null)
+        {
+            legacyAuthorizeRoleStatement = prepare(USERNAME, USER_PERMISSIONS);
+
+            ScheduledExecutors.optionalTasks.schedule(new Runnable()
+            {
+                public void run()
+                {
+                    convertLegacyData();
+                }
+            }, AuthKeyspace.SUPERUSER_SETUP_DELAY, TimeUnit.MILLISECONDS);
+        }
+    }
+
+    private SelectStatement prepare(String entityname, String permissionsTable)
+    {
+        String query = String.format("SELECT permissions FROM %s.%s WHERE %s = ? AND resource = ?",
+                                     AuthKeyspace.NAME,
+                                     permissionsTable,
+                                     entityname);
+        return (SelectStatement) QueryProcessor.getStatement(query, ClientState.forInternalCalls()).statement;
+    }
+
+    /**
+     * Copy legacy authz data from the system_auth.permissions table to the new system_auth.role_permissions table and
+     * also insert entries into the reverse lookup table.
+     * In theory, we could simply rename the existing table as the schema is structurally the same, but this would
+     * break mixed clusters during a rolling upgrade.
+     * This setup is not performed if AllowAllAuthenticator is configured (see Auth#setup).
+     */
+    private void convertLegacyData()
+    {
         try
         {
-            String query = String.format("SELECT permissions FROM %s.%s WHERE username = ? AND resource = ?", Auth.AUTH_KS, PERMISSIONS_CF);
-            authorizeStatement = (SelectStatement) QueryProcessor.parseStatement(query).prepare().statement;
+            if (Schema.instance.getCFMetaData("system_auth", "permissions") != null)
+            {
+                logger.info("Converting legacy permissions data");
+                CQLStatement insertStatement =
+                    QueryProcessor.getStatement(String.format("INSERT INTO %s.%s (role, resource, permissions) " +
+                                                              "VALUES (?, ?, ?)",
+                                                              AuthKeyspace.NAME,
+                                                              AuthKeyspace.ROLE_PERMISSIONS),
+                                                ClientState.forInternalCalls()).statement;
+                CQLStatement indexStatement =
+                    QueryProcessor.getStatement(String.format("INSERT INTO %s.%s (resource, role) VALUES (?,?)",
+                                                              AuthKeyspace.NAME,
+                                                              AuthKeyspace.RESOURCE_ROLE_INDEX),
+                                                ClientState.forInternalCalls()).statement;
+
+                UntypedResultSet permissions = process("SELECT * FROM system_auth.permissions");
+                for (UntypedResultSet.Row row : permissions)
+                {
+                    final IResource resource = Resources.fromName(row.getString("resource"));
+                    Predicate<String> isApplicable = new Predicate<String>()
+                    {
+                        public boolean apply(String s)
+                        {
+                            return resource.applicablePermissions().contains(Permission.valueOf(s));
+                        }
+                    };
+                    SetSerializer<String> serializer = SetSerializer.getInstance(UTF8Serializer.instance);
+                    Set<String> originalPerms = serializer.deserialize(row.getBytes("permissions"));
+                    Set<String> filteredPerms = ImmutableSet.copyOf(Iterables.filter(originalPerms, isApplicable));
+                    insertStatement.execute(QueryState.forInternalCalls(),
+                                            QueryOptions.forInternalCalls(ConsistencyLevel.ONE,
+                                                                          Lists.newArrayList(row.getBytes("username"),
+                                                                                             row.getBytes("resource"),
+                                                                                             serializer.serialize(filteredPerms))));
+
+                    indexStatement.execute(QueryState.forInternalCalls(),
+                                           QueryOptions.forInternalCalls(ConsistencyLevel.ONE,
+                                                                         Lists.newArrayList(row.getBytes("resource"),
+                                                                                            row.getBytes("username"))));
+
+                }
+                logger.info("Completed conversion of legacy permissions");
+            }
         }
-        catch (RequestValidationException e)
+        catch (Exception e)
         {
-            throw new AssertionError(e); // not supposed to happen
+            logger.info("Unable to complete conversion of legacy permissions data (perhaps not enough nodes are upgraded yet). " +
+                        "Conversion should not be considered complete");
+            logger.trace("Conversion error", e);
         }
     }
 
     // We only worry about one character ('). Make sure it's properly escaped.
-    private static String escape(String name)
+    private String escape(String name)
     {
         return StringUtils.replace(name, "'", "''");
     }
 
-    private static UntypedResultSet process(String query) throws RequestExecutionException
+    private UntypedResultSet process(String query) throws RequestExecutionException
     {
-        return QueryProcessor.process(query, ConsistencyLevel.ONE);
+        return QueryProcessor.process(query, ConsistencyLevel.LOCAL_ONE);
     }
 }
diff --git a/src/java/org/apache/cassandra/auth/CassandraRoleManager.java b/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
new file mode 100644
index 0000000..bfd0483
--- /dev/null
+++ b/src/java/org/apache/cassandra/auth/CassandraRoleManager.java
@@ -0,0 +1,675 @@
+/*
+ * 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.auth;
+
+import java.util.*;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.base.*;
+import com.google.common.base.Objects;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.concurrent.ScheduledExecutors;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.cql3.statements.SelectStatement;
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.exceptions.*;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.mindrot.jbcrypt.BCrypt;
+
+/**
+ * Responsible for the creation, maintenance and deletion of roles
+ * for the purposes of authentication and authorization.
+ * Role data is stored internally, using the roles and role_members tables
+ * in the system_auth keyspace.
+ *
+ * Additionally, if org.apache.cassandra.auth.PasswordAuthenticator is used,
+ * encrypted passwords are also stored in the system_auth.roles table. This
+ * coupling between the IAuthenticator and IRoleManager implementations exists
+ * because setting a role's password via CQL is done with a CREATE ROLE or
+ * ALTER ROLE statement, the processing of which is handled by IRoleManager.
+ * As IAuthenticator is concerned only with credentials checking and has no
+ * means to modify passwords, PasswordAuthenticator depends on
+ * CassandraRoleManager for those functions.
+ *
+ * Alternative IAuthenticator implementations may be used in conjunction with
+ * CassandraRoleManager, but WITH PASSWORD = 'password' will not be supported
+ * in CREATE/ALTER ROLE statements.
+ *
+ * Such a configuration could be implemented using a custom IRoleManager that
+ * extends CassandraRoleManager and which includes Option.PASSWORD in the Set<Option>
+ * returned from supportedOptions/alterableOptions. Any additional processing
+ * of the password itself (such as storing it in an alternative location) would
+ * be added in overridden createRole and alterRole implementations.
+ */
+public class CassandraRoleManager implements IRoleManager
+{
+    private static final Logger logger = LoggerFactory.getLogger(CassandraRoleManager.class);
+
+    static final String DEFAULT_SUPERUSER_NAME = "cassandra";
+    static final String DEFAULT_SUPERUSER_PASSWORD = "cassandra";
+
+    // Transform a row in the AuthKeyspace.ROLES to a Role instance
+    private static final Function<UntypedResultSet.Row, Role> ROW_TO_ROLE = new Function<UntypedResultSet.Row, Role>()
+    {
+        public Role apply(UntypedResultSet.Row row)
+        {
+            try
+            {
+                return new Role(row.getString("role"),
+                         row.getBoolean("is_superuser"),
+                         row.getBoolean("can_login"),
+                         row.has("member_of") ? row.getSet("member_of", UTF8Type.instance)
+                                              : Collections.<String>emptySet());
+            }
+            // Failing to deserialize a boolean in is_superuser or can_login will throw an NPE
+            catch (NullPointerException e)
+            {
+                logger.warn("An invalid value has been detected in the {} table for role {}. If you are " +
+                            "unable to login, you may need to disable authentication and confirm " +
+                            "that values in that table are accurate", AuthKeyspace.ROLES, row.getString("role"));
+                throw new RuntimeException(String.format("Invalid metadata has been detected for role %s", row.getString("role")), e);
+            }
+
+        }
+    };
+
+    public static final String LEGACY_USERS_TABLE = "users";
+    // Transform a row in the legacy system_auth.users table to a Role instance,
+    // used to fallback to previous schema on a mixed cluster during an upgrade
+    private static final Function<UntypedResultSet.Row, Role> LEGACY_ROW_TO_ROLE = new Function<UntypedResultSet.Row, Role>()
+    {
+        public Role apply(UntypedResultSet.Row row)
+        {
+            return new Role(row.getString("name"),
+                            row.getBoolean("super"),
+                            true,
+                            Collections.<String>emptySet());
+        }
+    };
+
+    // 2 ** GENSALT_LOG2_ROUNDS rounds of hashing will be performed.
+    private static final String GENSALT_LOG2_ROUNDS_PROPERTY = Config.PROPERTY_PREFIX + "auth_bcrypt_gensalt_log2_rounds";
+    private static final int GENSALT_LOG2_ROUNDS = getGensaltLogRounds();
+
+    static int getGensaltLogRounds()
+    {
+         int rounds = Integer.getInteger(GENSALT_LOG2_ROUNDS_PROPERTY, 10);
+         if (rounds < 4 || rounds > 31)
+         throw new ConfigurationException(String.format("Bad value for system property -D%s." +
+                                                        "Please use a value between 4 and 31 inclusively",
+                                                        GENSALT_LOG2_ROUNDS_PROPERTY));
+         return rounds;
+    }
+
+    // NullObject returned when a supplied role name not found in AuthKeyspace.ROLES
+    private static final Role NULL_ROLE = new Role(null, false, false, Collections.<String>emptySet());
+
+    private SelectStatement loadRoleStatement;
+    private SelectStatement legacySelectUserStatement;
+
+    private final Set<Option> supportedOptions;
+    private final Set<Option> alterableOptions;
+
+    // Will be set to true when all nodes in the cluster are on a version which supports roles (i.e. 2.2+)
+    private volatile boolean isClusterReady = false;
+
+    public CassandraRoleManager()
+    {
+        supportedOptions = DatabaseDescriptor.getAuthenticator().getClass() == PasswordAuthenticator.class
+                         ? ImmutableSet.of(Option.LOGIN, Option.SUPERUSER, Option.PASSWORD)
+                         : ImmutableSet.of(Option.LOGIN, Option.SUPERUSER);
+        alterableOptions = DatabaseDescriptor.getAuthenticator().getClass().equals(PasswordAuthenticator.class)
+                         ? ImmutableSet.of(Option.PASSWORD)
+                         : ImmutableSet.<Option>of();
+    }
+
+    public void setup()
+    {
+        loadRoleStatement = (SelectStatement) prepare("SELECT * from %s.%s WHERE role = ?",
+                                                      AuthKeyspace.NAME,
+                                                      AuthKeyspace.ROLES);
+        // If the old users table exists, we may need to migrate the legacy authn
+        // data to the new table. We also need to prepare a statement to read from
+        // it, so we can continue to use the old tables while the cluster is upgraded.
+        // Otherwise, we may need to create a default superuser role to enable others
+        // to be added.
+        if (Schema.instance.getCFMetaData(AuthKeyspace.NAME, "users") != null)
+        {
+            legacySelectUserStatement = prepareLegacySelectUserStatement();
+
+            scheduleSetupTask(new Callable<Void>()
+            {
+                public Void call() throws Exception
+                {
+                    convertLegacyData();
+                    return null;
+                }
+            });
+        }
+        else
+        {
+            scheduleSetupTask(new Callable<Void>()
+            {
+                public Void call() throws Exception
+                {
+                    setupDefaultRole();
+                    return null;
+                }
+            });
+        }
+    }
+
+    public Set<Option> supportedOptions()
+    {
+        return supportedOptions;
+    }
+
+    public Set<Option> alterableOptions()
+    {
+        return alterableOptions;
+    }
+
+    public void createRole(AuthenticatedUser performer, RoleResource role, RoleOptions options)
+    throws RequestValidationException, RequestExecutionException
+    {
+        String insertCql = options.getPassword().isPresent()
+                         ? String.format("INSERT INTO %s.%s (role, is_superuser, can_login, salted_hash) VALUES ('%s', %s, %s, '%s')",
+                                         AuthKeyspace.NAME,
+                                         AuthKeyspace.ROLES,
+                                         escape(role.getRoleName()),
+                                         options.getSuperuser().or(false),
+                                         options.getLogin().or(false),
+                                         escape(hashpw(options.getPassword().get())))
+                         : String.format("INSERT INTO %s.%s (role, is_superuser, can_login) VALUES ('%s', %s, %s)",
+                                         AuthKeyspace.NAME,
+                                         AuthKeyspace.ROLES,
+                                         escape(role.getRoleName()),
+                                         options.getSuperuser().or(false),
+                                         options.getLogin().or(false));
+        process(insertCql, consistencyForRole(role.getRoleName()));
+    }
+
+    public void dropRole(AuthenticatedUser performer, RoleResource role) throws RequestValidationException, RequestExecutionException
+    {
+        process(String.format("DELETE FROM %s.%s WHERE role = '%s'",
+                              AuthKeyspace.NAME,
+                              AuthKeyspace.ROLES,
+                              escape(role.getRoleName())),
+                consistencyForRole(role.getRoleName()));
+        removeAllMembers(role.getRoleName());
+    }
+
+    public void alterRole(AuthenticatedUser performer, RoleResource role, RoleOptions options)
+    {
+        // Unlike most of the other data access methods here, this does not use a
+        // prepared statement in order to allow the set of assignments to be variable.
+        String assignments = Joiner.on(',').join(Iterables.filter(optionsToAssignments(options.getOptions()),
+                                                                  Predicates.notNull()));
+        if (!Strings.isNullOrEmpty(assignments))
+        {
+            process(String.format("UPDATE %s.%s SET %s WHERE role = '%s'",
+                                  AuthKeyspace.NAME,
+                                  AuthKeyspace.ROLES,
+                                  assignments,
+                                  escape(role.getRoleName())),
+                    consistencyForRole(role.getRoleName()));
+        }
+    }
+
+    public void grantRole(AuthenticatedUser performer, RoleResource role, RoleResource grantee)
+    throws RequestValidationException, RequestExecutionException
+    {
+        if (getRoles(grantee, true).contains(role))
+            throw new InvalidRequestException(String.format("%s is a member of %s",
+                                                            grantee.getRoleName(),
+                                                            role.getRoleName()));
+        if (getRoles(role, true).contains(grantee))
+            throw new InvalidRequestException(String.format("%s is a member of %s",
+                                                            role.getRoleName(),
+                                                            grantee.getRoleName()));
+
+        modifyRoleMembership(grantee.getRoleName(), role.getRoleName(), "+");
+        process(String.format("INSERT INTO %s.%s (role, member) values ('%s', '%s')",
+                              AuthKeyspace.NAME,
+                              AuthKeyspace.ROLE_MEMBERS,
+                              escape(role.getRoleName()),
+                              escape(grantee.getRoleName())),
+                consistencyForRole(role.getRoleName()));
+    }
+
+    public void revokeRole(AuthenticatedUser performer, RoleResource role, RoleResource revokee)
+    throws RequestValidationException, RequestExecutionException
+    {
+        if (!getRoles(revokee, false).contains(role))
+            throw new InvalidRequestException(String.format("%s is not a member of %s",
+                                                            revokee.getRoleName(),
+                                                            role.getRoleName()));
+
+        modifyRoleMembership(revokee.getRoleName(), role.getRoleName(), "-");
+        process(String.format("DELETE FROM %s.%s WHERE role = '%s' and member = '%s'",
+                              AuthKeyspace.NAME,
+                              AuthKeyspace.ROLE_MEMBERS,
+                              escape(role.getRoleName()),
+                              escape(revokee.getRoleName())),
+                consistencyForRole(role.getRoleName()));
+    }
+
+    public Set<RoleResource> getRoles(RoleResource grantee, boolean includeInherited) throws RequestValidationException, RequestExecutionException
+    {
+        Set<RoleResource> roles = new HashSet<>();
+        Role role = getRole(grantee.getRoleName());
+        if (!role.equals(NULL_ROLE))
+        {
+            roles.add(RoleResource.role(role.name));
+            collectRoles(role, roles, includeInherited);
+        }
+        return roles;
+    }
+
+    public Set<RoleResource> getAllRoles() throws RequestValidationException, RequestExecutionException
+    {
+        UntypedResultSet rows = process(String.format("SELECT role from %s.%s", AuthKeyspace.NAME, AuthKeyspace.ROLES), ConsistencyLevel.QUORUM);
+        Iterable<RoleResource> roles = Iterables.transform(rows, new Function<UntypedResultSet.Row, RoleResource>()
+        {
+            public RoleResource apply(UntypedResultSet.Row row)
+            {
+                return RoleResource.role(row.getString("role"));
+            }
+        });
+        return ImmutableSet.<RoleResource>builder().addAll(roles).build();
+    }
+
+    public boolean isSuper(RoleResource role)
+    {
+        return getRole(role.getRoleName()).isSuper;
+    }
+
+    public boolean canLogin(RoleResource role)
+    {
+        return getRole(role.getRoleName()).canLogin;
+    }
+
+    public Map<String, String> getCustomOptions(RoleResource role)
+    {
+        return Collections.emptyMap();
+    }
+
+    public boolean isExistingRole(RoleResource role)
+    {
+        return getRole(role.getRoleName()) != NULL_ROLE;
+    }
+
+    public Set<? extends IResource> protectedResources()
+    {
+        return ImmutableSet.of(DataResource.table(AuthKeyspace.NAME, AuthKeyspace.ROLES),
+                               DataResource.table(AuthKeyspace.NAME, AuthKeyspace.ROLE_MEMBERS));
+    }
+
+    public void validateConfiguration() throws ConfigurationException
+    {
+    }
+
+    /*
+     * Create the default superuser role to bootstrap role creation on a clean system. Preemptively
+     * gives the role the default password so PasswordAuthenticator can be used to log in (if
+     * configured)
+     */
+    private static void setupDefaultRole()
+    {
+        if (StorageService.instance.getTokenMetadata().sortedTokens().isEmpty())
+            throw new IllegalStateException("CassandraRoleManager skipped default role setup: no known tokens in ring");
+
+        try
+        {
+            if (!hasExistingRoles())
+            {
+                QueryProcessor.process(String.format("INSERT INTO %s.%s (role, is_superuser, can_login, salted_hash) " +
+                                                     "VALUES ('%s', true, true, '%s')",
+                                                     AuthKeyspace.NAME,
+                                                     AuthKeyspace.ROLES,
+                                                     DEFAULT_SUPERUSER_NAME,
+                                                     escape(hashpw(DEFAULT_SUPERUSER_PASSWORD))),
+                                       consistencyForRole(DEFAULT_SUPERUSER_NAME));
+                logger.info("Created default superuser role '{}'", DEFAULT_SUPERUSER_NAME);
+            }
+        }
+        catch (RequestExecutionException e)
+        {
+            logger.warn("CassandraRoleManager skipped default role setup: some nodes were not ready");
+            throw e;
+        }
+    }
+
+    private static boolean hasExistingRoles() throws RequestExecutionException
+    {
+        // Try looking up the 'cassandra' default role first, to avoid the range query if possible.
+        String defaultSUQuery = String.format("SELECT * FROM %s.%s WHERE role = '%s'", AuthKeyspace.NAME, AuthKeyspace.ROLES, DEFAULT_SUPERUSER_NAME);
+        String allUsersQuery = String.format("SELECT * FROM %s.%s LIMIT 1", AuthKeyspace.NAME, AuthKeyspace.ROLES);
+        return !QueryProcessor.process(defaultSUQuery, ConsistencyLevel.ONE).isEmpty()
+               || !QueryProcessor.process(defaultSUQuery, ConsistencyLevel.QUORUM).isEmpty()
+               || !QueryProcessor.process(allUsersQuery, ConsistencyLevel.QUORUM).isEmpty();
+    }
+
+    private void scheduleSetupTask(final Callable<Void> setupTask)
+    {
+        // The delay is to give the node a chance to see its peers before attempting the operation
+        ScheduledExecutors.optionalTasks.schedule(new Runnable()
+        {
+            public void run()
+            {
+                // If not all nodes are on 2.2, we don't want to initialize the role manager as this will confuse 2.1
+                // nodes (see CASSANDRA-9761 for details). So we re-schedule the setup for later, hoping that the upgrade
+                // will be finished by then.
+                if (!MessagingService.instance().areAllNodesAtLeast22())
+                {
+                    logger.trace("Not all nodes are upgraded to a version that supports Roles yet, rescheduling setup task");
+                    scheduleSetupTask(setupTask);
+                    return;
+                }
+
+                isClusterReady = true;
+                try
+                {
+                    setupTask.call();
+                }
+                catch (Exception e)
+                {
+                    logger.info("Setup task failed with error, rescheduling");
+                    scheduleSetupTask(setupTask);
+                }
+            }
+        }, AuthKeyspace.SUPERUSER_SETUP_DELAY, TimeUnit.MILLISECONDS);
+    }
+
+    /*
+     * Copy legacy auth data from the system_auth.users & system_auth.credentials tables to
+     * the new system_auth.roles table. This setup is not performed if AllowAllAuthenticator
+     * is configured (see Auth#setup).
+     */
+    private void convertLegacyData() throws Exception
+    {
+        try
+        {
+            // read old data at QUORUM as it may contain the data for the default superuser
+            if (Schema.instance.getCFMetaData("system_auth", "users") != null)
+            {
+                logger.info("Converting legacy users");
+                UntypedResultSet users = QueryProcessor.process("SELECT * FROM system_auth.users",
+                                                                ConsistencyLevel.QUORUM);
+                for (UntypedResultSet.Row row : users)
+                {
+                    RoleOptions options = new RoleOptions();
+                    options.setOption(Option.SUPERUSER, row.getBoolean("super"));
+                    options.setOption(Option.LOGIN, true);
+                    createRole(null, RoleResource.role(row.getString("name")), options);
+                }
+                logger.info("Completed conversion of legacy users");
+            }
+
+            if (Schema.instance.getCFMetaData("system_auth", "credentials") != null)
+            {
+                logger.info("Migrating legacy credentials data to new system table");
+                UntypedResultSet credentials = QueryProcessor.process("SELECT * FROM system_auth.credentials",
+                                                                      ConsistencyLevel.QUORUM);
+                for (UntypedResultSet.Row row : credentials)
+                {
+                    // Write the password directly into the table to avoid doubly encrypting it
+                    QueryProcessor.process(String.format("UPDATE %s.%s SET salted_hash = '%s' WHERE role = '%s'",
+                                                         AuthKeyspace.NAME,
+                                                         AuthKeyspace.ROLES,
+                                                         row.getString("salted_hash"),
+                                                         row.getString("username")),
+                                           consistencyForRole(row.getString("username")));
+                }
+                logger.info("Completed conversion of legacy credentials");
+            }
+        }
+        catch (Exception e)
+        {
+            logger.info("Unable to complete conversion of legacy auth data (perhaps not enough nodes are upgraded yet). " +
+                        "Conversion should not be considered complete");
+            logger.trace("Conversion error", e);
+            throw e;
+        }
+    }
+
+    private SelectStatement prepareLegacySelectUserStatement()
+    {
+        return (SelectStatement) prepare("SELECT * FROM %s.%s WHERE name = ?",
+                                         AuthKeyspace.NAME,
+                                         LEGACY_USERS_TABLE);
+    }
+
+    private CQLStatement prepare(String template, String keyspace, String table)
+    {
+        try
+        {
+            return QueryProcessor.parseStatement(String.format(template, keyspace, table)).prepare().statement;
+        }
+        catch (RequestValidationException e)
+        {
+            throw new AssertionError(e); // not supposed to happen
+        }
+    }
+
+    /*
+     * Retrieve all roles granted to the given role. includeInherited specifies
+     * whether to include only those roles granted directly or all inherited roles.
+     */
+    private void collectRoles(Role role, Set<RoleResource> collected, boolean includeInherited) throws RequestValidationException, RequestExecutionException
+    {
+        for (String memberOf : role.memberOf)
+        {
+            Role granted = getRole(memberOf);
+            if (granted.equals(NULL_ROLE))
+                continue;
+            collected.add(RoleResource.role(granted.name));
+            if (includeInherited)
+                collectRoles(granted, collected, true);
+        }
+    }
+
+    /*
+     * Get a single Role instance given the role name. This never returns null, instead it
+     * uses the null object NULL_ROLE when a role with the given name cannot be found. So
+     * it's always safe to call methods on the returned object without risk of NPE.
+     */
+    private Role getRole(String name)
+    {
+        try
+        {
+            // If it exists, try the legacy users table in case the cluster
+            // is in the process of being upgraded and so is running with mixed
+            // versions of the authn schema.
+            if (Schema.instance.getCFMetaData(AuthKeyspace.NAME, "users") == null)
+                return getRoleFromTable(name, loadRoleStatement, ROW_TO_ROLE);
+            else
+            {
+                if (legacySelectUserStatement == null)
+                    legacySelectUserStatement = prepareLegacySelectUserStatement();
+                return getRoleFromTable(name, legacySelectUserStatement, LEGACY_ROW_TO_ROLE);
+            }
+        }
+        catch (RequestExecutionException | RequestValidationException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    private Role getRoleFromTable(String name, SelectStatement statement, Function<UntypedResultSet.Row, Role> function)
+    throws RequestExecutionException, RequestValidationException
+    {
+        ResultMessage.Rows rows =
+            statement.execute(QueryState.forInternalCalls(),
+                              QueryOptions.forInternalCalls(consistencyForRole(name),
+                                                            Collections.singletonList(ByteBufferUtil.bytes(name))));
+        if (rows.result.isEmpty())
+            return NULL_ROLE;
+
+        return function.apply(UntypedResultSet.create(rows.result).one());
+    }
+
+    /*
+     * Adds or removes a role name from the membership list of an entry in the roles table table
+     * (adds if op is "+", removes if op is "-")
+     */
+    private void modifyRoleMembership(String grantee, String role, String op)
+    throws RequestExecutionException
+    {
+        process(String.format("UPDATE %s.%s SET member_of = member_of %s {'%s'} WHERE role = '%s'",
+                              AuthKeyspace.NAME,
+                              AuthKeyspace.ROLES,
+                              op,
+                              escape(role),
+                              escape(grantee)),
+                consistencyForRole(grantee));
+    }
+
+    /*
+     * Clear the membership list of the given role
+     */
+    private void removeAllMembers(String role) throws RequestValidationException, RequestExecutionException
+    {
+        // Get the membership list of the the given role
+        UntypedResultSet rows = process(String.format("SELECT member FROM %s.%s WHERE role = '%s'",
+                                                      AuthKeyspace.NAME,
+                                                      AuthKeyspace.ROLE_MEMBERS,
+                                                      escape(role)),
+                                        consistencyForRole(role));
+        if (rows.isEmpty())
+            return;
+
+        // Update each member in the list, removing this role from its own list of granted roles
+        for (UntypedResultSet.Row row : rows)
+            modifyRoleMembership(row.getString("member"), role, "-");
+
+        // Finally, remove the membership list for the dropped role
+        process(String.format("DELETE FROM %s.%s WHERE role = '%s'",
+                              AuthKeyspace.NAME,
+                              AuthKeyspace.ROLE_MEMBERS,
+                              escape(role)),
+                consistencyForRole(role));
+    }
+
+    /*
+     * Convert a map of Options from a CREATE/ALTER statement into
+     * assignment clauses used to construct a CQL UPDATE statement
+     */
+    private Iterable<String> optionsToAssignments(Map<Option, Object> options)
+    {
+        return Iterables.transform(
+                                  options.entrySet(),
+                                  new Function<Map.Entry<Option, Object>, String>()
+                                  {
+                                      public String apply(Map.Entry<Option, Object> entry)
+                                      {
+                                          switch (entry.getKey())
+                                          {
+                                              case LOGIN:
+                                                  return String.format("can_login = %s", entry.getValue());
+                                              case SUPERUSER:
+                                                  return String.format("is_superuser = %s", entry.getValue());
+                                              case PASSWORD:
+                                                  return String.format("salted_hash = '%s'", escape(hashpw((String) entry.getValue())));
+                                              default:
+                                                  return null;
+                                          }
+                                      }
+                                  });
+    }
+
+    protected static ConsistencyLevel consistencyForRole(String role)
+    {
+        if (role.equals(DEFAULT_SUPERUSER_NAME))
+            return ConsistencyLevel.QUORUM;
+        else
+            return ConsistencyLevel.LOCAL_ONE;
+    }
+
+    private static String hashpw(String password)
+    {
+        return BCrypt.hashpw(password, BCrypt.gensalt(GENSALT_LOG2_ROUNDS));
+    }
+
+    private static String escape(String name)
+    {
+        return StringUtils.replace(name, "'", "''");
+    }
+
+    /**
+     * Executes the provided query.
+     * This shouldn't be used during setup as this will directly return an error if the manager is not setup yet. Setup tasks
+     * should use QueryProcessor.process directly.
+     */
+    private UntypedResultSet process(String query, ConsistencyLevel consistencyLevel) throws RequestValidationException, RequestExecutionException
+    {
+        if (!isClusterReady)
+            throw new InvalidRequestException("Cannot process role related query as the role manager isn't yet setup. "
+                                            + "This is likely because some of nodes in the cluster are on version 2.1 or earlier. "
+                                            + "You need to upgrade all nodes to Cassandra 2.2 or more to use roles.");
+
+        return QueryProcessor.process(query, consistencyLevel);
+    }
+
+    private static final class Role
+    {
+        private String name;
+        private final boolean isSuper;
+        private final boolean canLogin;
+        private Set<String> memberOf;
+
+        private Role(String name, boolean isSuper, boolean canLogin, Set<String> memberOf)
+        {
+            this.name = name;
+            this.isSuper = isSuper;
+            this.canLogin = canLogin;
+            this.memberOf = memberOf;
+        }
+
+        public boolean equals(Object o)
+        {
+            if (this == o)
+                return true;
+
+            if (!(o instanceof Role))
+                return false;
+
+            Role r = (Role) o;
+            return Objects.equal(name, r.name);
+        }
+
+        public int hashCode()
+        {
+            return Objects.hashCode(name);
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/auth/DataResource.java b/src/java/org/apache/cassandra/auth/DataResource.java
index 6aa8add..f64ed93 100644
--- a/src/java/org/apache/cassandra/auth/DataResource.java
+++ b/src/java/org/apache/cassandra/auth/DataResource.java
@@ -17,7 +17,10 @@
  */
 package org.apache.cassandra.auth;
 
+import java.util.Set;
+
 import com.google.common.base.Objects;
+import com.google.common.collect.Sets;
 import org.apache.commons.lang3.StringUtils;
 
 import org.apache.cassandra.config.Schema;
@@ -25,44 +28,57 @@
 /**
  * The primary type of resource in Cassandra.
  *
- * Used to represent a column family or a keyspace or the root level "data" resource.
+ * Used to represent a table or a keyspace or the root level "data" resource.
  * "data"                                 - the root level data resource.
  * "data/keyspace_name"                   - keyspace-level data resource.
- * "data/keyspace_name/column_family_name" - cf-level data resource.
+ * "data/keyspace_name/table_name"        - table-level data resource.
  */
 public class DataResource implements IResource
 {
     enum Level
     {
-        ROOT, KEYSPACE, COLUMN_FAMILY
+        ROOT, KEYSPACE, TABLE
     }
 
+    // permissions which may be granted on tables
+    private static final Set<Permission> TABLE_LEVEL_PERMISSIONS = Sets.immutableEnumSet(Permission.ALTER,
+                                                                                         Permission.DROP,
+                                                                                         Permission.SELECT,
+                                                                                         Permission.MODIFY,
+                                                                                         Permission.AUTHORIZE);
+    // permissions which may be granted on one or all keyspaces
+    private static final Set<Permission> KEYSPACE_LEVEL_PERMISSIONS = Sets.immutableEnumSet(Permission.CREATE,
+                                                                                            Permission.ALTER,
+                                                                                            Permission.DROP,
+                                                                                            Permission.SELECT,
+                                                                                            Permission.MODIFY,
+                                                                                            Permission.AUTHORIZE);
     private static final String ROOT_NAME = "data";
     private static final DataResource ROOT_RESOURCE = new DataResource();
 
     private final Level level;
     private final String keyspace;
-    private final String columnFamily;
+    private final String table;
 
     private DataResource()
     {
         level = Level.ROOT;
         keyspace = null;
-        columnFamily = null;
+        table = null;
     }
 
     private DataResource(String keyspace)
     {
         level = Level.KEYSPACE;
         this.keyspace = keyspace;
-        columnFamily = null;
+        table = null;
     }
 
-    private DataResource(String keyspace, String columnFamily)
+    private DataResource(String keyspace, String table)
     {
-        level = Level.COLUMN_FAMILY;
+        level = Level.TABLE;
         this.keyspace = keyspace;
-        this.columnFamily = columnFamily;
+        this.table = table;
     }
 
     /**
@@ -85,15 +101,15 @@
     }
 
     /**
-     * Creates a DataResource instance representing a column family.
+     * Creates a DataResource instance representing a table.
      *
      * @param keyspace Name of the keyspace.
-     * @param columnFamily Name of the column family.
+     * @param table Name of the table.
      * @return DataResource instance representing the column family.
      */
-    public static DataResource columnFamily(String keyspace, String columnFamily)
+    public static DataResource table(String keyspace, String table)
     {
-        return new DataResource(keyspace, columnFamily);
+        return new DataResource(keyspace, table);
     }
 
     /**
@@ -115,7 +131,7 @@
         if (parts.length == 2)
             return keyspace(parts[1]);
 
-        return columnFamily(parts[1], parts[2]);
+        return table(parts[1], parts[2]);
     }
 
     /**
@@ -129,8 +145,8 @@
                 return ROOT_NAME;
             case KEYSPACE:
                 return String.format("%s/%s", ROOT_NAME, keyspace);
-            case COLUMN_FAMILY:
-                return String.format("%s/%s/%s", ROOT_NAME, keyspace, columnFamily);
+            case TABLE:
+                return String.format("%s/%s/%s", ROOT_NAME, keyspace, table);
         }
         throw new AssertionError();
     }
@@ -144,7 +160,7 @@
         {
             case KEYSPACE:
                 return root();
-            case COLUMN_FAMILY:
+            case TABLE:
                 return keyspace(keyspace);
         }
         throw new IllegalStateException("Root-level resource can't have a parent");
@@ -152,17 +168,17 @@
 
     public boolean isRootLevel()
     {
-        return level.equals(Level.ROOT);
+        return level == Level.ROOT;
     }
 
     public boolean isKeyspaceLevel()
     {
-        return level.equals(Level.KEYSPACE);
+        return level == Level.KEYSPACE;
     }
 
-    public boolean isColumnFamilyLevel()
+    public boolean isTableLevel()
     {
-        return level.equals(Level.COLUMN_FAMILY);
+        return level == Level.TABLE;
     }
     /**
      * @return keyspace of the resource. Throws IllegalStateException if it's the root-level resource.
@@ -175,13 +191,13 @@
     }
 
     /**
-     * @return column family of the resource. Throws IllegalStateException if it's not a cf-level resource.
+     * @return column family of the resource. Throws IllegalStateException if it's not a table-level resource.
      */
-    public String getColumnFamily()
+    public String getTable()
     {
-        if (!isColumnFamilyLevel())
-            throw new IllegalStateException(String.format("%s data resource has no column family", level));
-        return columnFamily;
+        if (!isTableLevel())
+            throw new IllegalStateException(String.format("%s data resource has no table", level));
+        return table;
     }
 
     /**
@@ -189,7 +205,7 @@
      */
     public boolean hasParent()
     {
-        return !level.equals(Level.ROOT);
+        return level != Level.ROOT;
     }
 
     /**
@@ -203,8 +219,21 @@
                 return true;
             case KEYSPACE:
                 return Schema.instance.getKeyspaces().contains(keyspace);
-            case COLUMN_FAMILY:
-                return Schema.instance.getCFMetaData(keyspace, columnFamily) != null;
+            case TABLE:
+                return Schema.instance.getCFMetaData(keyspace, table) != null;
+        }
+        throw new AssertionError();
+    }
+
+    public Set<Permission> applicablePermissions()
+    {
+        switch (level)
+        {
+            case ROOT:
+            case KEYSPACE:
+                return KEYSPACE_LEVEL_PERMISSIONS;
+            case TABLE:
+                return TABLE_LEVEL_PERMISSIONS;
         }
         throw new AssertionError();
     }
@@ -218,8 +247,8 @@
                 return "<all keyspaces>";
             case KEYSPACE:
                 return String.format("<keyspace %s>", keyspace);
-            case COLUMN_FAMILY:
-                return String.format("<table %s.%s>", keyspace, columnFamily);
+            case TABLE:
+                return String.format("<table %s.%s>", keyspace, table);
         }
         throw new AssertionError();
     }
@@ -237,12 +266,12 @@
 
         return Objects.equal(level, ds.level)
             && Objects.equal(keyspace, ds.keyspace)
-            && Objects.equal(columnFamily, ds.columnFamily);
+            && Objects.equal(table, ds.table);
     }
 
     @Override
     public int hashCode()
     {
-        return Objects.hashCode(level, keyspace, columnFamily);
+        return Objects.hashCode(level, keyspace, table);
     }
 }
diff --git a/src/java/org/apache/cassandra/auth/FunctionResource.java b/src/java/org/apache/cassandra/auth/FunctionResource.java
new file mode 100644
index 0000000..1421541
--- /dev/null
+++ b/src/java/org/apache/cassandra/auth/FunctionResource.java
@@ -0,0 +1,327 @@
+/*
+ * 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.auth;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Objects;
+import com.google.common.base.Splitter;
+import com.google.common.collect.Sets;
+import org.apache.commons.lang3.StringUtils;
+
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.functions.FunctionName;
+import org.apache.cassandra.cql3.functions.Functions;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.TypeParser;
+
+/**
+ * IResource implementation representing functions.
+ *
+ * The root level "functions" resource represents the collection of all Functions.
+ * "functions"                          - root level resource representing all functions defined across every keyspace
+ * "functions/keyspace"                 - keyspace level resource to apply permissions to all functions within a keyspace
+ * "functions/keyspace/function"        - a specific function, scoped to a given keyspace
+ */
+public class FunctionResource implements IResource
+{
+    enum Level
+    {
+        ROOT, KEYSPACE, FUNCTION
+    }
+
+    // permissions which may be granted on either a resource representing some collection of functions
+    // i.e. the root resource (all functions) or a keyspace level resource (all functions in a given keyspace)
+    private static final Set<Permission> COLLECTION_LEVEL_PERMISSIONS = Sets.immutableEnumSet(Permission.CREATE,
+                                                                                              Permission.ALTER,
+                                                                                              Permission.DROP,
+                                                                                              Permission.AUTHORIZE,
+                                                                                              Permission.EXECUTE);
+    // permissions which may be granted on resources representing a specific function
+    private static final Set<Permission> SCALAR_FUNCTION_PERMISSIONS = Sets.immutableEnumSet(Permission.ALTER,
+                                                                                             Permission.DROP,
+                                                                                             Permission.AUTHORIZE,
+                                                                                             Permission.EXECUTE);
+
+    private static final Set<Permission> AGGREGATE_FUNCTION_PERMISSIONS = Sets.immutableEnumSet(Permission.ALTER,
+                                                                                                Permission.DROP,
+                                                                                                Permission.AUTHORIZE,
+                                                                                                Permission.EXECUTE);
+
+    private static final String ROOT_NAME = "functions";
+    private static final FunctionResource ROOT_RESOURCE = new FunctionResource();
+
+    private final Level level;
+    private final String keyspace;
+    private final String name;
+    private final List<AbstractType<?>> argTypes;
+
+    private FunctionResource()
+    {
+        level = Level.ROOT;
+        keyspace = null;
+        name = null;
+        argTypes = null;
+    }
+
+    private FunctionResource(String keyspace)
+    {
+        level = Level.KEYSPACE;
+        this.keyspace = keyspace;
+        name = null;
+        argTypes = null;
+    }
+
+    private FunctionResource(String keyspace, String name, List<AbstractType<?>> argTypes)
+    {
+        level = Level.FUNCTION;
+        this.keyspace = keyspace;
+        this.name = name;
+        this.argTypes = argTypes;
+    }
+
+    /**
+     * @return the root-level resource.
+     */
+    public static FunctionResource root()
+    {
+        return ROOT_RESOURCE;
+    }
+
+    /**
+     * Creates a FunctionResource representing the collection of functions scoped
+     * to a specific keyspace.
+     *
+     * @param keyspace name of the keyspace
+     * @return FunctionResource instance representing all of the keyspace's functions
+     */
+    public static FunctionResource keyspace(String keyspace)
+    {
+        return new FunctionResource(keyspace);
+    }
+
+    /**
+     * Creates a FunctionResource representing a specific, keyspace-scoped function.
+     *
+     * @param keyspace the keyspace in which the function is scoped
+     * @param name     name of the function.
+     * @param argTypes the types of the arguments to the function
+     * @return FunctionResource instance reresenting the function.
+     */
+    public static FunctionResource function(String keyspace, String name, List<AbstractType<?>> argTypes)
+    {
+        return new FunctionResource(keyspace, name, argTypes);
+    }
+
+    /**
+     * Creates a FunctionResource representing a specific, keyspace-scoped function.
+     * This variant is used to create an instance during parsing of a CQL statement.
+     * It includes transposition of the arg types from CQL types to AbstractType
+     * implementations
+     *
+     * @param keyspace the keyspace in which the function is scoped
+     * @param name     name of the function.
+     * @param argTypes the types of the function arguments in raw CQL form
+     * @return FunctionResource instance reresenting the function.
+     */
+    public static FunctionResource functionFromCql(String keyspace, String name, List<CQL3Type.Raw> argTypes)
+    {
+        List<AbstractType<?>> abstractTypes = new ArrayList<>();
+        for (CQL3Type.Raw cqlType : argTypes)
+            abstractTypes.add(cqlType.prepare(keyspace).getType());
+
+        return new FunctionResource(keyspace, name, abstractTypes);
+    }
+
+    /**
+     * Parses a resource name into a FunctionResource instance.
+     *
+     * @param name Name of the function resource.
+     * @return FunctionResource instance matching the name.
+     */
+    public static FunctionResource fromName(String name)
+    {
+        String[] parts = StringUtils.split(name, '/');
+
+        if (!parts[0].equals(ROOT_NAME) || parts.length > 3)
+            throw new IllegalArgumentException(String.format("%s is not a valid function resource name", name));
+
+        if (parts.length == 1)
+            return root();
+
+        if (parts.length == 2)
+            return keyspace(parts[1]);
+
+        String[] nameAndArgs = StringUtils.split(parts[2], "[|]");
+        return function(parts[1], nameAndArgs[0], argsListFromString(nameAndArgs[1]));
+    }
+
+    /**
+     * @return Printable name of the resource.
+     */
+    public String getName()
+    {
+        switch (level)
+        {
+            case ROOT:
+                return ROOT_NAME;
+            case KEYSPACE:
+                return String.format("%s/%s", ROOT_NAME, keyspace);
+            case FUNCTION:
+                return String.format("%s/%s/%s[%s]", ROOT_NAME, keyspace, name, argListAsString());
+        }
+        throw new AssertionError();
+    }
+
+    /**
+     * Get the name of the keyspace this resource relates to. In the case of the
+     * global root resource, return null
+     *
+     * @return the keyspace name of this resource, or null for the root resource
+     */
+    public String getKeyspace()
+    {
+        return keyspace;
+    }
+
+    /**
+     * @return a qualified FunctionName instance for a function-level resource.
+     * Throws IllegalStateException if called on the resource which doens't represent a single function.
+     */
+    public FunctionName getFunctionName()
+    {
+        if (level != Level.FUNCTION)
+            throw new IllegalStateException(String.format("%s function resource has no function name", level));
+        return new FunctionName(keyspace, name);
+    }
+
+    /**
+     * @return Parent of the resource, if any. Throws IllegalStateException if it's the root-level resource.
+     */
+    public IResource getParent()
+    {
+        switch (level)
+        {
+            case KEYSPACE:
+                return root();
+            case FUNCTION:
+                return keyspace(keyspace);
+        }
+        throw new IllegalStateException("Root-level resource can't have a parent");
+    }
+
+    public boolean hasParent()
+    {
+        return level != Level.ROOT;
+    }
+
+    public boolean exists()
+    {
+        switch (level)
+        {
+            case ROOT:
+                return true;
+            case KEYSPACE:
+                return Schema.instance.getKeyspaces().contains(keyspace);
+            case FUNCTION:
+                return Functions.find(getFunctionName(), argTypes) != null;
+        }
+        throw new AssertionError();
+    }
+
+    public Set<Permission> applicablePermissions()
+    {
+        switch (level)
+        {
+            case ROOT:
+            case KEYSPACE:
+                return COLLECTION_LEVEL_PERMISSIONS;
+            case FUNCTION:
+            {
+                Function function = Functions.find(getFunctionName(), argTypes);
+                assert function != null : "Unable to find function object for resource " + toString();
+                return function.isAggregate() ? AGGREGATE_FUNCTION_PERMISSIONS : SCALAR_FUNCTION_PERMISSIONS;
+            }
+        }
+        throw new AssertionError();
+    }
+
+    public int compareTo(FunctionResource o)
+    {
+        return this.name.compareTo(o.name);
+    }
+
+    @Override
+    public String toString()
+    {
+        switch (level)
+        {
+            case ROOT:
+                return "<all functions>";
+            case KEYSPACE:
+                return String.format("<all functions in %s>", keyspace);
+            case FUNCTION:
+                return String.format("<function %s.%s(%s)>",
+                                     keyspace,
+                                     name,
+                                     Joiner.on(", ").join(AbstractType.asCQLTypeStringList(argTypes)));
+        }
+        throw new AssertionError();
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o)
+            return true;
+
+        if (!(o instanceof FunctionResource))
+            return false;
+
+        FunctionResource f = (FunctionResource) o;
+
+        return Objects.equal(level, f.level)
+               && Objects.equal(keyspace, f.keyspace)
+               && Objects.equal(name, f.name)
+               && Objects.equal(argTypes, f.argTypes);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hashCode(level, keyspace, name, argTypes);
+    }
+
+    private String argListAsString()
+    {
+        return Joiner.on("^").join(argTypes);
+    }
+
+    private static List<AbstractType<?>> argsListFromString(String s)
+    {
+        List<AbstractType<?>> argTypes = new ArrayList<>();
+        for(String type : Splitter.on("^").omitEmptyStrings().trimResults().split(s))
+            argTypes.add(TypeParser.parse(type));
+        return argTypes;
+    }
+}
diff --git a/src/java/org/apache/cassandra/auth/IAuthenticator.java b/src/java/org/apache/cassandra/auth/IAuthenticator.java
index 6086490..24792f6 100644
--- a/src/java/org/apache/cassandra/auth/IAuthenticator.java
+++ b/src/java/org/apache/cassandra/auth/IAuthenticator.java
@@ -22,85 +22,15 @@
 
 import org.apache.cassandra.exceptions.AuthenticationException;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.exceptions.RequestExecutionException;
-import org.apache.cassandra.exceptions.RequestValidationException;
 
 public interface IAuthenticator
 {
-    static final String USERNAME_KEY = "username";
-    static final String PASSWORD_KEY = "password";
-
-    /**
-     * Supported CREATE USER/ALTER USER options.
-     * Currently only PASSWORD is available.
-     */
-    enum Option
-    {
-        PASSWORD
-    }
-
     /**
      * Whether or not the authenticator requires explicit login.
      * If false will instantiate user with AuthenticatedUser.ANONYMOUS_USER.
      */
     boolean requireAuthentication();
 
-    /**
-     * Set of options supported by CREATE USER and ALTER USER queries.
-     * Should never return null - always return an empty set instead.
-     */
-    Set<Option> supportedOptions();
-
-    /**
-     * Subset of supportedOptions that users are allowed to alter when performing ALTER USER [themselves].
-     * Should never return null - always return an empty set instead.
-     */
-    Set<Option> alterableOptions();
-
-    /**
-     * Authenticates a user given a Map<String, String> of credentials.
-     * Should never return null - always throw AuthenticationException instead.
-     * Returning AuthenticatedUser.ANONYMOUS_USER is an option as well if authentication is not required.
-     *
-     * @throws AuthenticationException if credentials don't match any known user.
-     */
-    AuthenticatedUser authenticate(Map<String, String> credentials) throws AuthenticationException;
-
-    /**
-     * Called during execution of CREATE USER query (also may be called on startup, see seedSuperuserOptions method).
-     * If authenticator is static then the body of the method should be left blank, but don't throw an exception.
-     * options are guaranteed to be a subset of supportedOptions().
-     *
-     * @param username Username of the user to create.
-     * @param options Options the user will be created with.
-     * @throws RequestValidationException
-     * @throws RequestExecutionException
-     */
-    void create(String username, Map<Option, Object> options) throws RequestValidationException, RequestExecutionException;
-
-    /**
-     * Called during execution of ALTER USER query.
-     * options are always guaranteed to be a subset of supportedOptions(). Furthermore, if the user performing the query
-     * is not a superuser and is altering himself, then options are guaranteed to be a subset of alterableOptions().
-     * Keep the body of the method blank if your implementation doesn't support any options.
-     *
-     * @param username Username of the user that will be altered.
-     * @param options Options to alter.
-     * @throws RequestValidationException
-     * @throws RequestExecutionException
-     */
-    void alter(String username, Map<Option, Object> options) throws RequestValidationException, RequestExecutionException;
-
-
-    /**
-     * Called during execution of DROP USER query.
-     *
-     * @param username Username of the user that will be dropped.
-     * @throws RequestValidationException
-     * @throws RequestExecutionException
-     */
-    void drop(String username) throws RequestValidationException, RequestExecutionException;
-
      /**
      * Set of resources that should be made inaccessible to users and only accessible internally.
      *
@@ -121,4 +51,78 @@
      * For example, use this method to create any required keyspaces/column families.
      */
     void setup();
+
+    /**
+     * Provide a SASL handler to perform authentication for an single connection. SASL
+     * is a stateful protocol, so a new instance must be used for each authentication
+     * attempt.
+     * @return org.apache.cassandra.auth.IAuthenticator.SaslNegotiator implementation
+     * (see {@link org.apache.cassandra.auth.PasswordAuthenticator.PlainTextSaslAuthenticator})
+     */
+    SaslNegotiator newSaslNegotiator();
+
+    /**
+     * For implementations which support the Thrift login method that accepts arbitrary
+     * key/value pairs containing credentials data.
+     * Also used by CQL native protocol v1, in which username and password are sent from
+     * client to server in a {@link org.apache.cassandra.transport.messages.CredentialsMessage}
+     * Implementations where support for Thrift and CQL protocol v1 is not required should make
+     * this an unsupported operation.
+     *
+     * Should never return null - always throw AuthenticationException instead.
+     * Returning AuthenticatedUser.ANONYMOUS_USER is an option as well if authentication is not required.
+     *
+     * @param credentials implementation specific key/value pairs
+     * @return non-null representation of the authenticated subject
+     * @throws AuthenticationException
+     */
+    AuthenticatedUser legacyAuthenticate(Map<String, String> credentials) throws AuthenticationException;
+
+    /**
+     * Performs the actual SASL negotiation for a single authentication attempt.
+     * SASL is stateful, so a new instance should be used for each attempt.
+     * Non-trivial implementations may delegate to an instance of {@link javax.security.sasl.SaslServer}
+     */
+    public interface SaslNegotiator
+    {
+        /**
+         * Evaluates the client response data and generates a byte[] reply which may be a further challenge or purely
+         * informational in the case that the negotiation is completed on this round.
+         *
+         * This method is called each time a {@link org.apache.cassandra.transport.messages.AuthResponse} is received
+         * from a client. After it is called, {@link isComplete()} is checked to determine whether the negotiation has
+         * finished. If so, an AuthenticatedUser is obtained by calling {@link getAuthenticatedUser()} and that user
+         * associated with the active connection and the byte[] sent back to the client via an
+         * {@link org.apache.cassandra.transport.messages.AuthSuccess} message. If the negotiation is not yet complete,
+         * the byte[] is returned to the client as a further challenge in an
+         * {@link org.apache.cassandra.transport.messages.AuthChallenge} message. This continues until the negotiation
+         * does complete or an error is encountered.
+         *
+         * @param clientResponse The non-null (but possibly empty) response sent by the client
+         * @return The possibly null response to send to the client.
+         * @throws AuthenticationException
+         * see {@link javax.security.sasl.SaslServer#evaluateResponse(byte[])}
+         */
+        public byte[] evaluateResponse(byte[] clientResponse) throws AuthenticationException;
+
+        /**
+         * Called after each invocation of {@link evaluateResponse(byte[])} to determine whether the  authentication has
+         * completed successfully or should be continued.
+         *
+         * @return true if the authentication exchange has completed; false otherwise.
+         * see {@link javax.security.sasl.SaslServer#isComplete()}
+         */
+        public boolean isComplete();
+
+        /**
+         * Following a sucessful negotiation, get the AuthenticatedUser representing the logged in subject.
+         * This method should only be called if {@link isComplete()} returns true.
+         * Should never return null - always throw AuthenticationException instead.
+         * Returning AuthenticatedUser.ANONYMOUS_USER is an option if authentication is not required.
+         *
+         * @return non-null representation of the authenticated subject
+         * @throws AuthenticationException
+         */
+        public AuthenticatedUser getAuthenticatedUser() throws AuthenticationException;
+    }
 }
diff --git a/src/java/org/apache/cassandra/auth/IAuthorizer.java b/src/java/org/apache/cassandra/auth/IAuthorizer.java
index 8ad204f..01c05af 100644
--- a/src/java/org/apache/cassandra/auth/IAuthorizer.java
+++ b/src/java/org/apache/cassandra/auth/IAuthorizer.java
@@ -29,7 +29,12 @@
 public interface IAuthorizer
 {
     /**
-     * The primary IAuthorizer method. Returns a set of permissions of a user on a resource.
+     * Returns a set of permissions of a user on a resource.
+     * Since Roles were introduced in version 2.2, Cassandra does not distinguish in any
+     * meaningful way between users and roles. A role may or may not have login privileges
+     * and roles may be granted to other roles. In fact, Cassandra does not really have the
+     * concept of a user, except to link a client session to role. AuthenticatedUser can be
+     * thought of as a manifestation of a role, linked to a specific client connection.
      *
      * @param user Authenticated user requesting authorization.
      * @param resource Resource for which the authorization is being requested. @see DataResource.
@@ -38,66 +43,85 @@
     Set<Permission> authorize(AuthenticatedUser user, IResource resource);
 
     /**
-     * Grants a set of permissions on a resource to a user.
+     * Grants a set of permissions on a resource to a role.
      * The opposite of revoke().
+     * This method is optional and may be called internally, so implementations which do
+     * not support it should be sure to throw UnsupportedOperationException.
      *
      * @param performer User who grants the permissions.
      * @param permissions Set of permissions to grant.
-     * @param to Grantee of the permissions.
      * @param resource Resource on which to grant the permissions.
+     * @param grantee Role to which the permissions are to be granted.
      *
      * @throws RequestValidationException
      * @throws RequestExecutionException
+     * @throws java.lang.UnsupportedOperationException
      */
-    void grant(AuthenticatedUser performer, Set<Permission> permissions, IResource resource, String to)
+    void grant(AuthenticatedUser performer, Set<Permission> permissions, IResource resource, RoleResource grantee)
     throws RequestValidationException, RequestExecutionException;
 
     /**
      * Revokes a set of permissions on a resource from a user.
      * The opposite of grant().
+     * This method is optional and may be called internally, so implementations which do
+     * not support it should be sure to throw UnsupportedOperationException.
      *
      * @param performer User who revokes the permissions.
      * @param permissions Set of permissions to revoke.
-     * @param from Revokee of the permissions.
+     * @param revokee Role from which to the permissions are to be revoked.
      * @param resource Resource on which to revoke the permissions.
      *
      * @throws RequestValidationException
      * @throws RequestExecutionException
+     * @throws java.lang.UnsupportedOperationException
      */
-    void revoke(AuthenticatedUser performer, Set<Permission> permissions, IResource resource, String from)
+    void revoke(AuthenticatedUser performer, Set<Permission> permissions, IResource resource, RoleResource revokee)
     throws RequestValidationException, RequestExecutionException;
 
     /**
-     * Returns a list of permissions on a resource of a user.
+     * Returns a list of permissions on a resource granted to a role.
+     * This method is optional and may be called internally, so implementations which do
+     * not support it should be sure to throw UnsupportedOperationException.
      *
      * @param performer User who wants to see the permissions.
-     * @param permissions Set of Permission values the user is interested in. The result should only include the matching ones.
-     * @param resource The resource on which permissions are requested. Can be null, in which case permissions on all resources
-     *                 should be returned.
-     * @param of The user whose permissions are requested. Can be null, in which case permissions of every user should be returned.
+     * @param permissions Set of Permission values the user is interested in. The result should only include the
+     *                    matching ones.
+     * @param resource The resource on which permissions are requested. Can be null, in which case permissions on all
+     *                 resources should be returned.
+     * @param grantee The role whose permissions are requested. Can be null, in which case permissions of every
+     *           role should be returned.
      *
      * @return All of the matching permission that the requesting user is authorized to know about.
      *
      * @throws RequestValidationException
      * @throws RequestExecutionException
+     * @throws java.lang.UnsupportedOperationException
      */
-    Set<PermissionDetails> list(AuthenticatedUser performer, Set<Permission> permissions, IResource resource, String of)
+    Set<PermissionDetails> list(AuthenticatedUser performer, Set<Permission> permissions, IResource resource, RoleResource grantee)
     throws RequestValidationException, RequestExecutionException;
 
     /**
-     * This method is called before deleting a user with DROP USER query so that a new user with the same
-     * name wouldn't inherit permissions of the deleted user in the future.
+     * Called before deleting a role with DROP ROLE statement (or the alias provided for compatibility,
+     * DROP USER) so that a new role with the same name wouldn't inherit permissions of the deleted one in the future.
+     * This removes all permissions granted to the Role in question.
+     * This method is optional and may be called internally, so implementations which do
+     * not support it should be sure to throw UnsupportedOperationException.
      *
-     * @param droppedUser The user to revoke all permissions from.
+     * @param revokee The role to revoke all permissions from.
+     * @throws java.lang.UnsupportedOperationException
      */
-    void revokeAll(String droppedUser);
+    void revokeAllFrom(RoleResource revokee);
 
     /**
-     * This method is called after a resource is removed (i.e. keyspace or a table is dropped).
+     * This method is called after a resource is removed (i.e. keyspace, table or role is dropped) and revokes all
+     * permissions granted on the IResource in question.
+     * This method is optional and may be called internally, so implementations which do
+     * not support it should be sure to throw UnsupportedOperationException.
      *
      * @param droppedResource The resource to revoke all permissions on.
+     * @throws java.lang.UnsupportedOperationException
      */
-    void revokeAll(IResource droppedResource);
+    void revokeAllOn(IResource droppedResource);
 
     /**
      * Set of resources that should be made inaccessible to users and only accessible internally.
diff --git a/src/java/org/apache/cassandra/auth/IResource.java b/src/java/org/apache/cassandra/auth/IResource.java
index 75f8d2a..065c47b 100644
--- a/src/java/org/apache/cassandra/auth/IResource.java
+++ b/src/java/org/apache/cassandra/auth/IResource.java
@@ -17,6 +17,8 @@
  */
 package org.apache.cassandra.auth;
 
+import java.util.Set;
+
 /**
  * The interface at the core of Cassandra authorization.
  *
@@ -50,4 +52,17 @@
      * @return Whether or not this resource exists in Cassandra.
      */
     boolean exists();
+
+    /**
+     * Returns the set of Permissions that may be applied to this resource
+     *
+     * Certain permissions are not applicable to particular types of resources.
+     * For instance, it makes no sense to talk about CREATE permission on table, or SELECT on a Role.
+     * Here we filter a set of permissions depending on the specific resource they're being applied to.
+     * This is necessary because the CQL syntax supports ALL as wildcard, but the set of permissions that
+     * should resolve to varies by IResource.
+     *
+     * @return the permissions that may be granted on the specific resource
+     */
+    Set<Permission> applicablePermissions();
 }
diff --git a/src/java/org/apache/cassandra/auth/IRoleManager.java b/src/java/org/apache/cassandra/auth/IRoleManager.java
new file mode 100644
index 0000000..5afc7f3
--- /dev/null
+++ b/src/java/org/apache/cassandra/auth/IRoleManager.java
@@ -0,0 +1,211 @@
+/*
+ * 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.auth;
+
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.RequestExecutionException;
+import org.apache.cassandra.exceptions.RequestValidationException;
+
+/**
+ * Responsible for managing roles (which also includes what
+ * used to be known as users), including creation, deletion,
+ * alteration and the granting and revoking of roles to other
+ * roles.
+ */
+public interface IRoleManager
+{
+
+    /**
+     * Supported options for CREATE ROLE/ALTER ROLE (and
+     * CREATE USER/ALTER USER, which are aliases provided
+     * for backwards compatibility).
+     */
+    public enum Option
+    {
+        SUPERUSER, PASSWORD, LOGIN, OPTIONS
+    }
+
+    /**
+     * Set of options supported by CREATE ROLE and ALTER ROLE queries.
+     * Should never return null - always return an empty set instead.
+     */
+    Set<Option> supportedOptions();
+
+    /**
+     * Subset of supportedOptions that users are allowed to alter when performing ALTER ROLE [themselves].
+     * Should never return null - always return an empty set instead.
+     */
+    Set<Option> alterableOptions();
+
+    /**
+     * Called during execution of a CREATE ROLE statement.
+     * options are guaranteed to be a subset of supportedOptions().
+     *
+     * @param performer User issuing the create role statement.
+     * @param role Rolei being created
+     * @param options Options the role will be created with
+     * @throws RequestValidationException
+     * @throws RequestExecutionException
+     */
+    void createRole(AuthenticatedUser performer, RoleResource role, RoleOptions options)
+    throws RequestValidationException, RequestExecutionException;
+
+    /**
+     * Called during execution of DROP ROLE statement, as well we removing any main record of the role from the system
+     * this implies that we want to revoke this role from all other roles that it has been granted to.
+     *
+     * @param performer User issuing the drop role statement.
+     * @param role Role to be dropped.
+     * @throws RequestValidationException
+     * @throws RequestExecutionException
+     */
+    void dropRole(AuthenticatedUser performer, RoleResource role) throws RequestValidationException, RequestExecutionException;
+
+    /**
+     * Called during execution of ALTER ROLE statement.
+     * options are always guaranteed to be a subset of supportedOptions(). Furthermore, if the actor performing the query
+     * is not a superuser and is altering themself, then options are guaranteed to be a subset of alterableOptions().
+     * Keep the body of the method blank if your implementation doesn't support modification of any options.
+     *
+     * @param performer User issuing the alter role statement.
+     * @param role Role that will be altered.
+     * @param options Options to alter.
+     * @throws RequestValidationException
+     * @throws RequestExecutionException
+     */
+    void alterRole(AuthenticatedUser performer, RoleResource role, RoleOptions options)
+    throws RequestValidationException, RequestExecutionException;
+
+    /**
+     * Called during execution of GRANT ROLE query.
+     * Grant an role to another existing role. A grantee that has a role granted to it will inherit any
+     * permissions of the granted role.
+     *
+     * @param performer User issuing the grant statement.
+     * @param role Role to be granted to the grantee.
+     * @param grantee Role acting as the grantee.
+     * @throws RequestValidationException
+     * @throws RequestExecutionException
+     */
+    void grantRole(AuthenticatedUser performer, RoleResource role, RoleResource grantee)
+    throws RequestValidationException, RequestExecutionException;
+
+    /**
+     * Called during the execution of a REVOKE ROLE query.
+     * Revoke an granted role from an existing role. The revokee will lose any permissions inherited from the role being
+     * revoked.
+     *
+     * @param performer User issuing the revoke statement.
+     * @param role Role to be revoked.
+     * @param revokee Role from which the granted role is to be revoked.
+     * @throws RequestValidationException
+     * @throws RequestExecutionException
+     */
+    void revokeRole(AuthenticatedUser performer, RoleResource role, RoleResource revokee)
+    throws RequestValidationException, RequestExecutionException;
+
+    /**
+     * Called during execution of a LIST ROLES query.
+     * Returns a set of roles that have been granted to the grantee using GRANT ROLE.
+     *
+     * @param grantee Role whose granted roles will be listed.
+     * @param includeInherited if True will list inherited roles as well as those directly granted to the grantee.
+     * @return A list containing the granted roles for the user.
+     * @throws RequestValidationException
+     * @throws RequestExecutionException
+     */
+    Set<RoleResource> getRoles(RoleResource grantee, boolean includeInherited) throws RequestValidationException, RequestExecutionException;
+
+    /**
+     * Called during the execution of an unqualified LIST ROLES query.
+     * Returns the total set of distinct roles in the system.
+     *
+     * @return the set of all roles in the system.
+     * @throws RequestValidationException
+     * @throws RequestExecutionException
+     */
+    Set<RoleResource> getAllRoles() throws RequestValidationException, RequestExecutionException;
+
+    /**
+     * Return true if there exists a Role with the given name that also has
+     * superuser status. Superuser status may be inherited from another
+     * granted role, so this method should return true if either the named
+     * Role, or any other Role it is transitively granted has superuser
+     * status.
+     *
+     * @param role Role whose superuser status to verify
+     * @return true if the role exists and has superuser status, either
+     * directly or transitively, otherwise false.
+     */
+    boolean isSuper(RoleResource role);
+
+    /**
+     * Return true if there exists a Role with the given name which has login
+     * privileges. Such privileges is not inherited from other granted Roles
+     * and so must be directly granted to the named Role with the LOGIN option
+     * of CREATE ROLE or ALTER ROLE
+     *
+     * @param role Role whose login privileges to verify
+     * @return true if the role exists and is permitted to login, otherwise false
+     */
+    boolean canLogin(RoleResource role);
+
+    /**
+     * Where an implementation supports OPTIONS in CREATE and ALTER operations
+     * this method should return the Map<String, String> representing the custom
+     * options associated with the role, as supplied to CREATE or ALTER.
+     * It should never return null; if the implementation does not support
+     * OPTIONS or if none were supplied then it should return an empty map.
+     * @param role Role whose custom options are required
+     * @return Key/Value pairs representing the custom options for the Role
+     */
+    Map<String, String> getCustomOptions(RoleResource role);
+
+    /**
+     * Return true is a Role with the given name exists in the system.
+     *
+     * @param role Role whose existence to verify
+     * @return true if the name identifies an extant Role in the system,
+     * otherwise false
+     */
+    boolean isExistingRole(RoleResource role);
+
+    /**
+     * Set of resources that should be made inaccessible to users and only accessible internally.
+     *
+     * @return Keyspaces and column families that will be unmodifiable by users; other resources.
+     */
+    Set<? extends IResource> protectedResources();
+
+    /**
+     * Hook to perform validation of an implementation's configuration (if supported).
+     *
+     * @throws ConfigurationException
+     */
+    void validateConfiguration() throws ConfigurationException;
+
+    /**
+     * Hook to perform implementation specific initialization, called once upon system startup.
+     *
+     * For example, use this method to create any required keyspaces/column families.
+     */
+    void setup();
+}
diff --git a/src/java/org/apache/cassandra/auth/ISaslAwareAuthenticator.java b/src/java/org/apache/cassandra/auth/ISaslAwareAuthenticator.java
deleted file mode 100644
index 959506f..0000000
--- a/src/java/org/apache/cassandra/auth/ISaslAwareAuthenticator.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * 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.auth;
-
-import org.apache.cassandra.exceptions.AuthenticationException;
-
-public interface ISaslAwareAuthenticator extends IAuthenticator
-{
-    /**
-     * Provide a SaslAuthenticator to be used by the CQL binary protocol server. If
-     * the configured IAuthenticator requires authentication but does not implement this
-     * interface we refuse to start the binary protocol server as it will have no way
-     * of authenticating clients.
-     * @return SaslAuthenticator implementation
-     * (see {@link PasswordAuthenticator.PlainTextSaslAuthenticator})
-     */
-    SaslAuthenticator newAuthenticator();
-
-
-    public interface SaslAuthenticator
-    {
-        public byte[] evaluateResponse(byte[] clientResponse) throws AuthenticationException;
-        public boolean isComplete();
-        public AuthenticatedUser getAuthenticatedUser() throws AuthenticationException;
-    }
-}
diff --git a/src/java/org/apache/cassandra/auth/LegacyAuthenticator.java b/src/java/org/apache/cassandra/auth/LegacyAuthenticator.java
deleted file mode 100644
index c5fd8da..0000000
--- a/src/java/org/apache/cassandra/auth/LegacyAuthenticator.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/*
- * 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.auth;
-
-import java.util.Collections;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.cassandra.exceptions.AuthenticationException;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.exceptions.RequestExecutionException;
-import org.apache.cassandra.exceptions.RequestValidationException;
-
-/**
- * Provides a transitional IAuthenticator implementation for old-style (pre-1.2) authenticators.
- *
- * Comes with default implementation for the all of the new methods.
- * Subclass LegacyAuthenticator instead of implementing the old IAuthenticator and your old IAuthenticator
- * implementation should continue to work.
- */
-public abstract class LegacyAuthenticator implements IAuthenticator
-{
-    /**
-     * @return The user that a connection is initialized with, or 'null' if a user must call login().
-     */
-    public abstract AuthenticatedUser defaultUser();
-
-    /**
-     * @param credentials An implementation specific collection of identifying information.
-     * @return A successfully authenticated user: should throw AuthenticationException rather than ever returning null.
-     */
-    public abstract AuthenticatedUser authenticate(Map<String, String> credentials) throws AuthenticationException;
-
-    public abstract void validateConfiguration() throws ConfigurationException;
-
-    @Override
-    public boolean requireAuthentication()
-    {
-        return defaultUser() == null;
-    }
-
-    @Override
-    public Set<Option> supportedOptions()
-    {
-        return Collections.emptySet();
-    }
-
-    @Override
-    public Set<Option> alterableOptions()
-    {
-        return Collections.emptySet();
-    }
-
-    @Override
-    public void create(String username, Map<Option, Object> options) throws RequestValidationException, RequestExecutionException
-    {
-    }
-
-    @Override
-    public void alter(String username, Map<Option, Object> options) throws RequestValidationException, RequestExecutionException
-    {
-    }
-
-    @Override
-    public void drop(String username) throws RequestValidationException, RequestExecutionException
-    {
-    }
-
-    @Override
-    public Set<IResource> protectedResources()
-    {
-        return Collections.emptySet();
-    }
-
-    @Override
-    public void setup()
-    {
-    }
-}
diff --git a/src/java/org/apache/cassandra/auth/LegacyAuthorizer.java b/src/java/org/apache/cassandra/auth/LegacyAuthorizer.java
deleted file mode 100644
index f834793..0000000
--- a/src/java/org/apache/cassandra/auth/LegacyAuthorizer.java
+++ /dev/null
@@ -1,114 +0,0 @@
-/*
- * 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.auth;
-
-import java.util.*;
-
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.UnauthorizedException;
-
-/**
- * Provides a transitional IAuthorizer implementation for old-style (pre-1.2) authorizers.
- *
- * Translates old-style authorize() calls to the new-style, expands Permission.READ and Permission.WRITE
- * into the new Permission values, translates the new resource hierarchy into the old hierarchy.
- * Stubs the rest of the new methods.
- * Subclass LegacyAuthorizer instead of implementing the old IAuthority and your old IAuthority implementation should
- * continue to work.
- */
-public abstract class LegacyAuthorizer implements IAuthorizer
-{
-    /**
-     * @param user Authenticated user requesting authorization.
-     * @param resource List of Objects containing Strings and byte[]s: represents a resource in the old hierarchy.
-     * @return Set of permissions of the user on the resource. Should never return null. Use Permission.NONE instead.
-     */
-    public abstract EnumSet<Permission> authorize(AuthenticatedUser user, List<Object> resource);
-
-    public abstract void validateConfiguration() throws ConfigurationException;
-
-    /**
-     * Translates new-style authorize() method call to the old-style (including permissions and the hierarchy).
-     */
-    @Override
-    public Set<Permission> authorize(AuthenticatedUser user, IResource resource)
-    {
-        if (!(resource instanceof DataResource))
-            throw new IllegalArgumentException(String.format("%s resource is not supported by LegacyAuthorizer", resource.getName()));
-        DataResource dr = (DataResource) resource;
-
-        List<Object> legacyResource = new ArrayList<Object>();
-        legacyResource.add(Resources.ROOT);
-        legacyResource.add(Resources.KEYSPACES);
-        if (!dr.isRootLevel())
-            legacyResource.add(dr.getKeyspace());
-        if (dr.isColumnFamilyLevel())
-            legacyResource.add(dr.getColumnFamily());
-
-        Set<Permission> permissions = authorize(user, legacyResource);
-        if (permissions.contains(Permission.READ))
-            permissions.add(Permission.SELECT);
-        if (permissions.contains(Permission.WRITE))
-            permissions.addAll(EnumSet.of(Permission.CREATE, Permission.ALTER, Permission.DROP, Permission.MODIFY));
-
-        return permissions;
-    }
-
-    @Override
-    public void grant(AuthenticatedUser performer, Set<Permission> permissions, IResource resource, String to)
-    throws InvalidRequestException
-    {
-        throw new InvalidRequestException("GRANT operation is not supported by LegacyAuthorizer");
-    }
-
-    @Override
-    public void revoke(AuthenticatedUser performer, Set<Permission> permissions, IResource resource, String from)
-    throws InvalidRequestException
-    {
-        throw new InvalidRequestException("REVOKE operation is not supported by LegacyAuthorizer");
-    }
-
-    @Override
-    public void revokeAll(String droppedUser)
-    {
-    }
-
-    @Override
-    public void revokeAll(IResource droppedResource)
-    {
-    }
-
-    @Override
-    public Set<PermissionDetails> list(AuthenticatedUser performer, Set<Permission> permissions, IResource resource, String of)
-    throws InvalidRequestException, UnauthorizedException
-    {
-        throw new InvalidRequestException("LIST PERMISSIONS operation is not supported by LegacyAuthorizer");
-    }
-
-    @Override
-    public Set<IResource> protectedResources()
-    {
-        return Collections.emptySet();
-    }
-
-    @Override
-    public void setup()
-    {
-    }
-}
diff --git a/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java b/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java
index 7d30664..20f8790 100644
--- a/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java
+++ b/src/java/org/apache/cassandra/auth/PasswordAuthenticator.java
@@ -19,90 +19,123 @@
 
 import java.nio.charset.StandardCharsets;
 import java.util.Arrays;
-import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
-import java.util.concurrent.TimeUnit;
 
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
-import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.concurrent.ScheduledExecutors;
-import org.apache.cassandra.config.Config;
-import org.apache.cassandra.cql3.UntypedResultSet;
-import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.cql3.statements.SelectStatement;
-import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.exceptions.*;
+import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.transport.messages.ResultMessage;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.mindrot.jbcrypt.BCrypt;
 
+import static org.apache.cassandra.auth.CassandraRoleManager.consistencyForRole;
+
 /**
  * PasswordAuthenticator is an IAuthenticator implementation
- * that keeps credentials (usernames and bcrypt-hashed passwords)
- * internally in C* - in system_auth.credentials CQL3 table.
+ * that keeps credentials (rolenames and bcrypt-hashed passwords)
+ * internally in C* - in system_auth.roles CQL3 table.
+ * Since 2.2, the management of roles (creation, modification,
+ * querying etc is the responsibility of IRoleManager. Use of
+ * PasswordAuthenticator requires the use of CassandraRoleManager
+ * for storage and retrieval of encrypted passwords.
  */
-public class PasswordAuthenticator implements ISaslAwareAuthenticator
+public class PasswordAuthenticator implements IAuthenticator
 {
     private static final Logger logger = LoggerFactory.getLogger(PasswordAuthenticator.class);
 
-    // 2 ** GENSALT_LOG2_ROUNDS rounds of hashing will be performed.
-    private static final String GENSALT_LOG2_ROUNDS_PROPERTY = Config.PROPERTY_PREFIX + "auth_bcrypt_gensalt_log2_rounds";
-    private static final int GENSALT_LOG2_ROUNDS = getGensaltLogRounds();
-
-    static int getGensaltLogRounds()
-    {
-        int rounds = Integer.getInteger(GENSALT_LOG2_ROUNDS_PROPERTY, 10);
-        if (rounds < 4 || rounds > 31)
-            throw new RuntimeException(new ConfigurationException(String.format("Bad value for system property -D%s. " +
-                                                                                "Please use a value 4 and 31",
-                                                                                GENSALT_LOG2_ROUNDS_PROPERTY)));
-        return rounds;
-    }
-
     // name of the hash column.
     private static final String SALTED_HASH = "salted_hash";
 
-    private static final String DEFAULT_USER_NAME = Auth.DEFAULT_SUPERUSER_NAME;
-    private static final String DEFAULT_USER_PASSWORD = Auth.DEFAULT_SUPERUSER_NAME;
+    // really this is a rolename now, but as it only matters for Thrift, we leave it for backwards compatibility
+    public static final String USERNAME_KEY = "username";
+    public static final String PASSWORD_KEY = "password";
 
-    private static final String CREDENTIALS_CF = "credentials";
-    private static final String CREDENTIALS_CF_SCHEMA = String.format("CREATE TABLE %s.%s ("
-                                                                      + "username text,"
-                                                                      + "salted_hash text," // salt + hash + number of rounds
-                                                                      + "options map<text,text>," // for future extensions
-                                                                      + "PRIMARY KEY(username)"
-                                                                      + ") WITH gc_grace_seconds=%d",
-                                                                      Auth.AUTH_KS,
-                                                                      CREDENTIALS_CF,
-                                                                      90 * 24 * 60 * 60); // 3 months.
-
+    private static final byte NUL = 0;
     private SelectStatement authenticateStatement;
 
+    public static final String LEGACY_CREDENTIALS_TABLE = "credentials";
+    private SelectStatement legacyAuthenticateStatement;
+
     // No anonymous access.
     public boolean requireAuthentication()
     {
         return true;
     }
 
-    public Set<Option> supportedOptions()
+    private AuthenticatedUser authenticate(String username, String password) throws AuthenticationException
     {
-        return ImmutableSet.of(Option.PASSWORD);
+        try
+        {
+            SelectStatement authenticationStatement = authenticationStatement();
+            return doAuthenticate(username, password, authenticationStatement);
+        }
+        catch (RequestExecutionException e)
+        {
+            logger.trace("Error performing internal authentication", e);
+            throw new AuthenticationException(e.toString());
+        }
     }
 
-    // Let users alter their own password.
-    public Set<Option> alterableOptions()
+    /**
+     * If the legacy users table exists try to verify credentials there. This is to handle the case
+     * where the cluster is being upgraded and so is running with mixed versions of the authn tables
+     */
+    private SelectStatement authenticationStatement()
     {
-        return ImmutableSet.of(Option.PASSWORD);
+        if (Schema.instance.getCFMetaData(AuthKeyspace.NAME, LEGACY_CREDENTIALS_TABLE) == null)
+            return authenticateStatement;
+        else
+        {
+            // If the credentials was initialised only after statement got prepared, re-prepare (CASSANDRA-12813).
+            if (legacyAuthenticateStatement == null)
+                prepareLegacyAuthenticateStatement();
+            return legacyAuthenticateStatement;
+        }
     }
 
-    public AuthenticatedUser authenticate(Map<String, String> credentials) throws AuthenticationException
+    public Set<DataResource> protectedResources()
+    {
+        // Also protected by CassandraRoleManager, but the duplication doesn't hurt and is more explicit
+        return ImmutableSet.of(DataResource.table(AuthKeyspace.NAME, AuthKeyspace.ROLES));
+    }
+
+    public void validateConfiguration() throws ConfigurationException
+    {
+    }
+
+    public void setup()
+    {
+        String query = String.format("SELECT %s FROM %s.%s WHERE role = ?",
+                                     SALTED_HASH,
+                                     AuthKeyspace.NAME,
+                                     AuthKeyspace.ROLES);
+        authenticateStatement = prepare(query);
+
+        if (Schema.instance.getCFMetaData(AuthKeyspace.NAME, LEGACY_CREDENTIALS_TABLE) != null)
+            prepareLegacyAuthenticateStatement();
+    }
+
+    private void prepareLegacyAuthenticateStatement()
+    {
+        String query = String.format("SELECT %s from %s.%s WHERE username = ?",
+                                     SALTED_HASH,
+                                     AuthKeyspace.NAME,
+                                     LEGACY_CREDENTIALS_TABLE);
+        legacyAuthenticateStatement = prepare(query);
+    }
+
+    public AuthenticatedUser legacyAuthenticate(Map<String, String> credentials) throws AuthenticationException
     {
         String username = credentials.get(USERNAME_KEY);
         if (username == null)
@@ -112,183 +145,56 @@
         if (password == null)
             throw new AuthenticationException(String.format("Required key '%s' is missing", PASSWORD_KEY));
 
-        UntypedResultSet result;
-        try
-        {
-            ResultMessage.Rows rows = authenticateStatement.execute(QueryState.forInternalCalls(),
-                                                                    QueryOptions.forInternalCalls(consistencyForUser(username),
-                                                                                                  Lists.newArrayList(ByteBufferUtil.bytes(username))));
-            result = UntypedResultSet.create(rows.result);
-        }
-        catch (RequestValidationException e)
-        {
-            throw new AssertionError(e); // not supposed to happen
-        }
-        catch (RequestExecutionException e)
-        {
-            throw new AuthenticationException(e.toString());
-        }
+        return authenticate(username, password);
+    }
 
-        if (result.isEmpty() || !BCrypt.checkpw(password, result.one().getString(SALTED_HASH)))
+    public SaslNegotiator newSaslNegotiator()
+    {
+        return new PlainTextSaslAuthenticator();
+    }
+
+    private AuthenticatedUser doAuthenticate(String username, String password, SelectStatement authenticationStatement)
+    throws RequestExecutionException, AuthenticationException
+    {
+        ResultMessage.Rows rows = authenticationStatement.execute(QueryState.forInternalCalls(),
+                                                                  QueryOptions.forInternalCalls(consistencyForRole(username),
+                                                                                                Lists.newArrayList(ByteBufferUtil.bytes(username))));
+        UntypedResultSet result = UntypedResultSet.create(rows.result);
+
+        if ((result.isEmpty() || !result.one().has(SALTED_HASH)) || !BCrypt.checkpw(password, result.one().getString(SALTED_HASH)))
             throw new AuthenticationException("Username and/or password are incorrect");
 
         return new AuthenticatedUser(username);
     }
 
-    public void create(String username, Map<Option, Object> options) throws InvalidRequestException, RequestExecutionException
+    private SelectStatement prepare(String query)
     {
-        String password = (String) options.get(Option.PASSWORD);
-        if (password == null)
-            throw new InvalidRequestException("PasswordAuthenticator requires PASSWORD option");
-
-        process(String.format("INSERT INTO %s.%s (username, salted_hash) VALUES ('%s', '%s')",
-                              Auth.AUTH_KS,
-                              CREDENTIALS_CF,
-                              escape(username),
-                              escape(hashpw(password))),
-                consistencyForUser(username));
+        return (SelectStatement) QueryProcessor.getStatement(query, ClientState.forInternalCalls()).statement;
     }
 
-    public void alter(String username, Map<Option, Object> options) throws RequestExecutionException
+    private class PlainTextSaslAuthenticator implements SaslNegotiator
     {
-        process(String.format("UPDATE %s.%s SET salted_hash = '%s' WHERE username = '%s'",
-                              Auth.AUTH_KS,
-                              CREDENTIALS_CF,
-                              escape(hashpw((String) options.get(Option.PASSWORD))),
-                              escape(username)),
-                consistencyForUser(username));
-    }
-
-    public void drop(String username) throws RequestExecutionException
-    {
-        process(String.format("DELETE FROM %s.%s WHERE username = '%s'", Auth.AUTH_KS, CREDENTIALS_CF, escape(username)),
-                consistencyForUser(username));
-    }
-
-    public Set<DataResource> protectedResources()
-    {
-        return ImmutableSet.of(DataResource.columnFamily(Auth.AUTH_KS, CREDENTIALS_CF));
-    }
-
-    public void validateConfiguration() throws ConfigurationException
-    {
-    }
-
-    public void setup()
-    {
-        Auth.setupTable(CREDENTIALS_CF, CREDENTIALS_CF_SCHEMA);
-
-        // the delay is here to give the node some time to see its peers - to reduce
-        // "skipped default user setup: some nodes are were not ready" log spam.
-        // It's the only reason for the delay.
-        ScheduledExecutors.nonPeriodicTasks.schedule(new Runnable()
-        {
-            public void run()
-            {
-              setupDefaultUser();
-            }
-        }, Auth.SUPERUSER_SETUP_DELAY, TimeUnit.MILLISECONDS);
-
-        try
-        {
-            String query = String.format("SELECT %s FROM %s.%s WHERE username = ?",
-                                         SALTED_HASH,
-                                         Auth.AUTH_KS,
-                                         CREDENTIALS_CF);
-            authenticateStatement = (SelectStatement) QueryProcessor.parseStatement(query).prepare().statement;
-        }
-        catch (RequestValidationException e)
-        {
-            throw new AssertionError(e); // not supposed to happen
-        }
-    }
-
-    public SaslAuthenticator newAuthenticator()
-    {
-        return new PlainTextSaslAuthenticator();
-    }
-
-    // if there are no users yet - add default superuser.
-    private void setupDefaultUser()
-    {
-        try
-        {
-            // insert the default superuser if AUTH_KS.CREDENTIALS_CF is empty.
-            if (!hasExistingUsers())
-            {
-                process(String.format("INSERT INTO %s.%s (username, salted_hash) VALUES ('%s', '%s') USING TIMESTAMP 0",
-                                      Auth.AUTH_KS,
-                                      CREDENTIALS_CF,
-                                      DEFAULT_USER_NAME,
-                                      escape(hashpw(DEFAULT_USER_PASSWORD))),
-                        ConsistencyLevel.ONE);
-                logger.info("PasswordAuthenticator created default user '{}'", DEFAULT_USER_NAME);
-            }
-        }
-        catch (RequestExecutionException e)
-        {
-            logger.warn("PasswordAuthenticator skipped default user setup: some nodes were not ready");
-        }
-    }
-
-    private static boolean hasExistingUsers() throws RequestExecutionException
-    {
-        // Try looking up the 'cassandra' default user first, to avoid the range query if possible.
-        String defaultSUQuery = String.format("SELECT * FROM %s.%s WHERE username = '%s'", Auth.AUTH_KS, CREDENTIALS_CF, DEFAULT_USER_NAME);
-        String allUsersQuery = String.format("SELECT * FROM %s.%s LIMIT 1", Auth.AUTH_KS, CREDENTIALS_CF);
-        return !process(defaultSUQuery, ConsistencyLevel.ONE).isEmpty()
-            || !process(defaultSUQuery, ConsistencyLevel.QUORUM).isEmpty()
-            || !process(allUsersQuery, ConsistencyLevel.QUORUM).isEmpty();
-    }
-
-    private static String hashpw(String password)
-    {
-        return BCrypt.hashpw(password, BCrypt.gensalt(GENSALT_LOG2_ROUNDS));
-    }
-
-    private static String escape(String name)
-    {
-        return StringUtils.replace(name, "'", "''");
-    }
-
-    private static UntypedResultSet process(String query, ConsistencyLevel cl) throws RequestExecutionException
-    {
-        return QueryProcessor.process(query, cl);
-    }
-
-    private static ConsistencyLevel consistencyForUser(String username)
-    {
-        if (username.equals(DEFAULT_USER_NAME))
-            return ConsistencyLevel.QUORUM;
-        else
-            return ConsistencyLevel.LOCAL_ONE;
-    }
-
-    private class PlainTextSaslAuthenticator implements ISaslAwareAuthenticator.SaslAuthenticator
-    {
-        private static final byte NUL = 0;
-
         private boolean complete = false;
-        private Map<String, String> credentials;
+        private String username;
+        private String password;
 
-        @Override
         public byte[] evaluateResponse(byte[] clientResponse) throws AuthenticationException
         {
-            credentials = decodeCredentials(clientResponse);
+            decodeCredentials(clientResponse);
             complete = true;
             return null;
         }
 
-        @Override
         public boolean isComplete()
         {
             return complete;
         }
 
-        @Override
         public AuthenticatedUser getAuthenticatedUser() throws AuthenticationException
         {
-            return authenticate(credentials);
+            if (!complete)
+                throw new AuthenticationException("SASL negotiation not complete");
+            return authenticate(username, password);
         }
 
         /**
@@ -297,16 +203,16 @@
          * The form is : {code}authzId<NUL>authnId<NUL>password<NUL>{code}
          * authzId is optional, and in fact we don't care about it here as we'll
          * set the authzId to match the authnId (that is, there is no concept of
-         * a user being authorized to act on behalf of another).
+         * a user being authorized to act on behalf of another with this IAuthenticator).
          *
          * @param bytes encoded credentials string sent by the client
          * @return map containing the username/password pairs in the form an IAuthenticator
          * would expect
          * @throws javax.security.sasl.SaslException
          */
-        private Map<String, String> decodeCredentials(byte[] bytes) throws AuthenticationException
+        private void decodeCredentials(byte[] bytes) throws AuthenticationException
         {
-            logger.debug("Decoding credentials from client token");
+            logger.trace("Decoding credentials from client token");
             byte[] user = null;
             byte[] pass = null;
             int end = bytes.length;
@@ -327,10 +233,8 @@
             if (pass == null)
                 throw new AuthenticationException("Password must not be null");
 
-            Map<String, String> credentials = new HashMap<String, String>();
-            credentials.put(IAuthenticator.USERNAME_KEY, new String(user, StandardCharsets.UTF_8));
-            credentials.put(IAuthenticator.PASSWORD_KEY, new String(pass, StandardCharsets.UTF_8));
-            return credentials;
+            username = new String(user, StandardCharsets.UTF_8);
+            password = new String(pass, StandardCharsets.UTF_8);
         }
     }
 }
diff --git a/src/java/org/apache/cassandra/auth/Permission.java b/src/java/org/apache/cassandra/auth/Permission.java
index a982c85..d552280 100644
--- a/src/java/org/apache/cassandra/auth/Permission.java
+++ b/src/java/org/apache/cassandra/auth/Permission.java
@@ -21,6 +21,7 @@
 import java.util.Set;
 
 import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
 
 /**
  * An enum encapsulating the set of possible permissions that an authenticated user can have on a resource.
@@ -35,23 +36,34 @@
     @Deprecated
     WRITE,
 
-    // schema management
-    CREATE, // required for CREATE KEYSPACE and CREATE TABLE.
-    ALTER,  // required for ALTER KEYSPACE, ALTER TABLE, CREATE INDEX, DROP INDEX.
-    DROP,   // required for DROP KEYSPACE and DROP TABLE.
+    // schema and role management
+    // CREATE, ALTER and DROP permissions granted on an appropriate DataResource are required for
+    // CREATE KEYSPACE and CREATE TABLE.
+    // ALTER KEYSPACE, ALTER TABLE, CREATE INDEX and DROP INDEX require ALTER permission on the
+    // relevant DataResource.
+    // DROP KEYSPACE and DROP TABLE require DROP permission.
+    //
+    // In the context of Role management, these permissions may also be granted on a RoleResource.
+    // CREATE is only granted on the root-level role resource, and is required to create new roles.
+    // ALTER & DROP may be granted on either the root-level role resource, giving permissions on
+    // all roles, or on specific role-level resources.
+    CREATE,
+    ALTER,
+    DROP,
 
     // data access
-    SELECT, // required for SELECT.
-    MODIFY, // required for INSERT, UPDATE, DELETE, TRUNCATE.
+    SELECT, // required for SELECT on a table
+    MODIFY, // required for INSERT, UPDATE, DELETE, TRUNCATE on a DataResource.
 
     // permission management
-    AUTHORIZE; // required for GRANT and REVOKE.
+    AUTHORIZE, // required for GRANT and REVOKE of permissions or roles.
 
+    DESCRIBE, // required on the root-level RoleResource to list all Roles
 
-    public static final Set<Permission> ALL_DATA =
-            ImmutableSet.copyOf(EnumSet.range(Permission.CREATE, Permission.AUTHORIZE));
+    // UDF permissions
+    EXECUTE;  // required to invoke any user defined function or aggregate
 
     public static final Set<Permission> ALL =
-            ImmutableSet.copyOf(EnumSet.range(Permission.CREATE, Permission.AUTHORIZE));
+            Sets.immutableEnumSet(EnumSet.range(Permission.CREATE, Permission.EXECUTE));
     public static final Set<Permission> NONE = ImmutableSet.of();
 }
diff --git a/src/java/org/apache/cassandra/auth/PermissionDetails.java b/src/java/org/apache/cassandra/auth/PermissionDetails.java
index b2b4512..7e8c625 100644
--- a/src/java/org/apache/cassandra/auth/PermissionDetails.java
+++ b/src/java/org/apache/cassandra/auth/PermissionDetails.java
@@ -26,13 +26,13 @@
  */
 public class PermissionDetails implements Comparable<PermissionDetails>
 {
-    public final String username;
+    public final String grantee;
     public final IResource resource;
     public final Permission permission;
 
-    public PermissionDetails(String username, IResource resource, Permission permission)
+    public PermissionDetails(String grantee, IResource resource, Permission permission)
     {
-        this.username = username;
+        this.grantee = grantee;
         this.resource = resource;
         this.permission = permission;
     }
@@ -41,7 +41,7 @@
     public int compareTo(PermissionDetails other)
     {
         return ComparisonChain.start()
-                              .compare(username, other.username)
+                              .compare(grantee, other.grantee)
                               .compare(resource.getName(), other.resource.getName())
                               .compare(permission, other.permission)
                               .result();
@@ -50,8 +50,8 @@
     @Override
     public String toString()
     {
-        return String.format("<PermissionDetails username:%s resource:%s permission:%s>",
-                             username,
+        return String.format("<PermissionDetails grantee:%s resource:%s permission:%s>",
+                             grantee,
                              resource.getName(),
                              permission);
     }
@@ -66,7 +66,7 @@
             return false;
 
         PermissionDetails pd = (PermissionDetails) o;
-        return Objects.equal(username, pd.username)
+        return Objects.equal(grantee, pd.grantee)
             && Objects.equal(resource, pd.resource)
             && Objects.equal(permission, pd.permission);
     }
@@ -74,6 +74,6 @@
     @Override
     public int hashCode()
     {
-        return Objects.hashCode(username, resource, permission);
+        return Objects.hashCode(grantee, resource, permission);
     }
 }
diff --git a/src/java/org/apache/cassandra/auth/PermissionsCache.java b/src/java/org/apache/cassandra/auth/PermissionsCache.java
index bc96d82..8746b36 100644
--- a/src/java/org/apache/cassandra/auth/PermissionsCache.java
+++ b/src/java/org/apache/cassandra/auth/PermissionsCache.java
@@ -137,7 +137,7 @@
                                            }
                                            catch (Exception e)
                                            {
-                                               logger.debug("Error performing async refresh of user permissions", e);
+                                               logger.trace("Error performing async refresh of user permissions", e);
                                                throw e;
                                            }
                                        }
diff --git a/src/java/org/apache/cassandra/auth/Resources.java b/src/java/org/apache/cassandra/auth/Resources.java
index 86900f9..ebcfc16 100644
--- a/src/java/org/apache/cassandra/auth/Resources.java
+++ b/src/java/org/apache/cassandra/auth/Resources.java
@@ -43,6 +43,25 @@
         return chain;
     }
 
+    /**
+     * Creates an IResource instance from its external name.
+     * Resource implementation class is inferred by matching against the known IResource
+     * impls' root level resources.
+     * @param name
+     * @return an IResource instance created from the name
+     */
+    public static IResource fromName(String name)
+    {
+        if (name.startsWith(RoleResource.root().getName()))
+            return RoleResource.fromName(name);
+        else if (name.startsWith(DataResource.root().getName()))
+            return DataResource.fromName(name);
+        else if (name.startsWith(FunctionResource.root().getName()))
+            return FunctionResource.fromName(name);
+        else
+            throw new IllegalArgumentException(String.format("Name %s is not valid for any resource type", name));
+    }
+
     @Deprecated
     public final static String ROOT = "cassandra";
     @Deprecated
@@ -58,7 +77,7 @@
             if (component instanceof byte[])
                 buff.append(Hex.bytesToHex((byte[])component));
             else
-                buff.append(component.toString());
+                buff.append(component);
         }
         return buff.toString();
     }
diff --git a/src/java/org/apache/cassandra/auth/RoleOptions.java b/src/java/org/apache/cassandra/auth/RoleOptions.java
new file mode 100644
index 0000000..9609ff3
--- /dev/null
+++ b/src/java/org/apache/cassandra/auth/RoleOptions.java
@@ -0,0 +1,153 @@
+/*
+ * 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.auth;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import com.google.common.base.Optional;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.exceptions.SyntaxException;
+import org.apache.cassandra.utils.FBUtilities;
+
+public class RoleOptions
+{
+    private final Map<IRoleManager.Option, Object> options = new HashMap<>();
+
+    /**
+     * Set a value for a specific option.
+     * Throws SyntaxException if the same option is set multiple times
+     * @param option
+     * @param value
+     */
+    public void setOption(IRoleManager.Option option, Object value)
+    {
+        if (options.containsKey(option))
+            throw new SyntaxException(String.format("Multiple definition for property '%s'", option.name()));
+        options.put(option, value);
+    }
+
+    /**
+     * Return true if there are no options with values set, false otherwise
+     * @return whether any options have values set or not
+     */
+    public boolean isEmpty()
+    {
+        return options.isEmpty();
+    }
+
+    /**
+     * Return a map of all the options which have been set
+     * @return all options with values
+     */
+    public Map<IRoleManager.Option, Object> getOptions()
+    {
+        return options;
+    }
+
+    /**
+     * Return a boolean value of the superuser option
+     * @return superuser option value
+     */
+    public Optional<Boolean> getSuperuser()
+    {
+        return Optional.fromNullable((Boolean)options.get(IRoleManager.Option.SUPERUSER));
+    }
+
+    /**
+     * Return a boolean value of the login option
+     * @return login option value
+     */
+    public Optional<Boolean> getLogin()
+    {
+        return Optional.fromNullable((Boolean)options.get(IRoleManager.Option.LOGIN));
+    }
+
+    /**
+     * Return the string value of the password option
+     * @return password option value
+     */
+    public Optional<String> getPassword()
+    {
+        return Optional.fromNullable((String)options.get(IRoleManager.Option.PASSWORD));
+    }
+
+    /**
+     * Return a Map<String, String> representing custom options
+     * It is the responsiblity of IRoleManager implementations which support
+     * IRoleManager.Option.OPTION to handle type checking and conversion of these
+     * values, if present
+     * @return map of custom options
+     */
+    @SuppressWarnings("unchecked")
+    public Optional<Map<String, String>> getCustomOptions()
+    {
+        return Optional.fromNullable((Map<String, String>)options.get(IRoleManager.Option.OPTIONS));
+    }
+
+    /**
+     * Validate the contents of the options in two ways:
+     * - Ensure that only a subset of the options supported by the configured IRoleManager are set
+     * - Validate the type of any option values present.
+     * Should either condition fail, then InvalidRequestException is thrown. This method is called
+     * during validation of CQL statements, so the IRE results in a error response to the client.
+     *
+     * @throws InvalidRequestException if any options which are not supported by the configured IRoleManager
+     *     are set or if any option value is of an incorrect type.
+     */
+    public void validate()
+    {
+        for (Map.Entry<IRoleManager.Option, Object> option : options.entrySet())
+        {
+            if (!DatabaseDescriptor.getRoleManager().supportedOptions().contains(option.getKey()))
+                throw new InvalidRequestException(String.format("%s doesn't support %s",
+                                                                DatabaseDescriptor.getRoleManager().getClass().getName(),
+                                                                option.getKey()));
+            switch (option.getKey())
+            {
+                case LOGIN:
+                case SUPERUSER:
+                    if (!(option.getValue() instanceof Boolean))
+                        throw new InvalidRequestException(String.format("Invalid value for property '%s'. " +
+                                                                        "It must be a boolean",
+                                                                        option.getKey()));
+                    break;
+                case PASSWORD:
+                    if (!(option.getValue() instanceof String))
+                        throw new InvalidRequestException(String.format("Invalid value for property '%s'. " +
+                                                                        "It must be a string",
+                                                                        option.getKey()));
+                    break;
+                case OPTIONS:
+                    if (!(option.getValue() instanceof Map))
+                        throw new InvalidRequestException(String.format("Invalid value for property '%s'. " +
+                                                                        "It must be a map",
+                                                                        option.getKey()));
+                    break;
+
+            }
+        }
+    }
+
+    public String toString()
+    {
+        return FBUtilities.toString(options);
+    }
+}
diff --git a/src/java/org/apache/cassandra/auth/RoleResource.java b/src/java/org/apache/cassandra/auth/RoleResource.java
new file mode 100644
index 0000000..e994233
--- /dev/null
+++ b/src/java/org/apache/cassandra/auth/RoleResource.java
@@ -0,0 +1,185 @@
+/*
+ * 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.auth;
+
+import java.util.Set;
+
+import com.google.common.base.Objects;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
+import org.apache.commons.lang3.StringUtils;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+
+/**
+ * IResource implementation representing database roles.
+ *
+ * The root level "roles" resource represents the collection of all Roles.
+ * Individual roles are represented as members of the collection:
+ * "roles"                    - the root level collection resource
+ * "roles/role1"              - a specific database role
+ */
+public class RoleResource implements IResource, Comparable<RoleResource>
+{
+    enum Level
+    {
+        ROOT, ROLE
+    }
+
+    // permissions which may be granted on the root level resource
+    private static final Set<Permission> ROOT_LEVEL_PERMISSIONS = Sets.immutableEnumSet(Permission.CREATE,
+                                                                                        Permission.ALTER,
+                                                                                        Permission.DROP,
+                                                                                        Permission.AUTHORIZE,
+                                                                                        Permission.DESCRIBE);
+    // permissions which may be granted on role level resources
+    private static final Set<Permission> ROLE_LEVEL_PERMISSIONS = Sets.immutableEnumSet(Permission.ALTER,
+                                                                                        Permission.DROP,
+                                                                                        Permission.AUTHORIZE);
+
+    private static final String ROOT_NAME = "roles";
+    private static final RoleResource ROOT_RESOURCE = new RoleResource();
+
+    private final Level level;
+    private final String name;
+
+    private RoleResource()
+    {
+        level = Level.ROOT;
+        name = null;
+    }
+
+    private RoleResource(String name)
+    {
+        level = Level.ROLE;
+        this.name = name;
+    }
+
+    /**
+     * @return the root-level resource.
+     */
+    public static RoleResource root()
+    {
+        return ROOT_RESOURCE;
+    }
+
+    /**
+     * Creates a RoleResource representing an individual Role.
+     * @param name name of the Role.
+     * @return RoleResource instance reresenting the Role.
+     */
+    public static RoleResource role(String name)
+    {
+        return new RoleResource(name);
+    }
+
+    /**
+     * Parses a role resource name into a RoleResource instance.
+     *
+     * @param name Name of the data resource.
+     * @return RoleResource instance matching the name.
+     */
+    public static RoleResource fromName(String name)
+    {
+        String[] parts = StringUtils.split(name, '/');
+
+        if (!parts[0].equals(ROOT_NAME) || parts.length > 2)
+            throw new IllegalArgumentException(String.format("%s is not a valid role resource name", name));
+
+        if (parts.length == 1)
+            return root();
+
+        return role(parts[1]);
+    }
+
+    /**
+     * @return Printable name of the resource.
+     */
+    public String getName()
+    {
+        return level == Level.ROOT ? ROOT_NAME : String.format("%s/%s", ROOT_NAME, name);
+    }
+
+    /**
+     * @return short form name of a role level resource. i.e. not the full "root/name" version returned by getName().
+     * Throws IllegalStateException if called on the root-level resource.
+     */
+    public String getRoleName()
+    {
+        if (level == Level.ROOT)
+            throw new IllegalStateException(String.format("%s role resource has no role name", level));
+        return name;
+    }
+
+    /**
+     * @return Parent of the resource, if any. Throws IllegalStateException if it's the root-level resource.
+     */
+    public IResource getParent()
+    {
+        if (level == Level.ROLE)
+            return root();
+
+        throw new IllegalStateException("Root-level resource can't have a parent");
+    }
+
+    public boolean hasParent()
+    {
+        return level != Level.ROOT;
+    }
+
+    public boolean exists()
+    {
+        return level == Level.ROOT || DatabaseDescriptor.getRoleManager().isExistingRole(this);
+    }
+
+    public Set<Permission> applicablePermissions()
+    {
+        return level == Level.ROOT ? ROOT_LEVEL_PERMISSIONS : ROLE_LEVEL_PERMISSIONS;
+    }
+
+    public int compareTo(RoleResource o)
+    {
+        return this.name.compareTo(o.name);
+    }
+
+    @Override
+    public String toString()
+    {
+        return level == Level.ROOT ? "<all roles>" : String.format("<role %s>", name);
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o)
+            return true;
+
+        if (!(o instanceof RoleResource))
+            return false;
+
+        RoleResource rs = (RoleResource) o;
+
+        return Objects.equal(level, rs.level) && Objects.equal(name, rs.name);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hashCode(level, name);
+    }
+}
diff --git a/src/java/org/apache/cassandra/auth/Roles.java b/src/java/org/apache/cassandra/auth/Roles.java
new file mode 100644
index 0000000..da6804b
--- /dev/null
+++ b/src/java/org/apache/cassandra/auth/Roles.java
@@ -0,0 +1,56 @@
+/*
+ * 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.auth;
+
+import java.util.Set;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+
+public class Roles
+{
+    private static final RolesCache cache = new RolesCache(DatabaseDescriptor.getRoleManager());
+
+    /**
+     * Get all roles granted to the supplied Role, including both directly granted
+     * and inherited roles.
+     * The returned roles may be cached if roles_validity_in_ms > 0
+     *
+     * @param primaryRole the Role
+     * @return set of all granted Roles for the primary Role
+     */
+    public static Set<RoleResource> getRoles(RoleResource primaryRole)
+    {
+        return cache.getRoles(primaryRole);
+    }
+
+    /**
+     * Returns true if the supplied role or any other role granted to it
+     * (directly or indirectly) has superuser status.
+     *
+     * @param role the primary role
+     * @return true if the role has superuser status, false otherwise
+     */
+    public static boolean hasSuperuserStatus(RoleResource role)
+    {
+        IRoleManager roleManager = DatabaseDescriptor.getRoleManager();
+        for (RoleResource r : cache.getRoles(role))
+            if (roleManager.isSuper(r))
+                return true;
+        return false;
+    }
+}
diff --git a/src/java/org/apache/cassandra/auth/RolesCache.java b/src/java/org/apache/cassandra/auth/RolesCache.java
new file mode 100644
index 0000000..554df9e
--- /dev/null
+++ b/src/java/org/apache/cassandra/auth/RolesCache.java
@@ -0,0 +1,151 @@
+/*
+ * 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.auth;
+
+import java.lang.management.ManagementFactory;
+import java.util.Set;
+import java.util.concurrent.*;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListenableFutureTask;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor;
+import org.apache.cassandra.config.DatabaseDescriptor;
+
+import javax.management.MBeanServer;
+import javax.management.ObjectName;
+
+public class RolesCache implements RolesCacheMBean
+{
+    private static final Logger logger = LoggerFactory.getLogger(RolesCache.class);
+
+    private final String MBEAN_NAME = "org.apache.cassandra.auth:type=RolesCache";
+    private final ThreadPoolExecutor cacheRefreshExecutor = new DebuggableThreadPoolExecutor("RolesCacheRefresh",
+                                                                                             Thread.NORM_PRIORITY);
+    private final IRoleManager roleManager;
+    private volatile LoadingCache<RoleResource, Set<RoleResource>> cache;
+
+    public RolesCache(IRoleManager roleManager)
+    {
+        this.roleManager = roleManager;
+        this.cache = initCache(null);
+        try
+        {
+            MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
+            mbs.registerMBean(this, new ObjectName(MBEAN_NAME));
+        }
+        catch (Exception e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public Set<RoleResource> getRoles(RoleResource role)
+    {
+        if (cache == null)
+            return roleManager.getRoles(role, true);
+
+        try
+        {
+            return cache.get(role);
+        }
+        catch (ExecutionException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public void invalidate()
+    {
+        cache = initCache(null);
+    }
+
+    public void setValidity(int validityPeriod)
+    {
+        DatabaseDescriptor.setRolesValidity(validityPeriod);
+        cache = initCache(cache);
+    }
+
+    public int getValidity()
+    {
+        return DatabaseDescriptor.getRolesValidity();
+    }
+
+    public void setUpdateInterval(int updateInterval)
+    {
+        DatabaseDescriptor.setRolesUpdateInterval(updateInterval);
+        cache = initCache(cache);
+    }
+
+    public int getUpdateInterval()
+    {
+        return DatabaseDescriptor.getRolesUpdateInterval();
+    }
+
+
+    private LoadingCache<RoleResource, Set<RoleResource>> initCache(LoadingCache<RoleResource, Set<RoleResource>> existing)
+    {
+        if (DatabaseDescriptor.getAuthenticator() instanceof AllowAllAuthenticator)
+            return null;
+
+        if (DatabaseDescriptor.getRolesValidity() <= 0)
+            return null;
+
+        LoadingCache<RoleResource, Set<RoleResource>> newcache = CacheBuilder.newBuilder()
+                .refreshAfterWrite(DatabaseDescriptor.getRolesUpdateInterval(), TimeUnit.MILLISECONDS)
+                .expireAfterWrite(DatabaseDescriptor.getRolesValidity(), TimeUnit.MILLISECONDS)
+                .maximumSize(DatabaseDescriptor.getRolesCacheMaxEntries())
+                .build(new CacheLoader<RoleResource, Set<RoleResource>>()
+                {
+                    public Set<RoleResource> load(RoleResource primaryRole)
+                    {
+                        return roleManager.getRoles(primaryRole, true);
+                    }
+
+                    public ListenableFuture<Set<RoleResource>> reload(final RoleResource primaryRole,
+                                                                      final Set<RoleResource> oldValue)
+                    {
+                        ListenableFutureTask<Set<RoleResource>> task;
+                        task = ListenableFutureTask.create(new Callable<Set<RoleResource>>()
+                        {
+                            public Set<RoleResource> call() throws Exception
+                            {
+                                try
+                                {
+                                    return roleManager.getRoles(primaryRole, true);
+                                } catch (Exception e)
+                                {
+                                    logger.trace("Error performing async refresh of user roles", e);
+                                    throw e;
+                                }
+                            }
+                        });
+                        cacheRefreshExecutor.execute(task);
+                        return task;
+                    }
+                });
+        if (existing != null)
+            newcache.putAll(existing.asMap());
+        return newcache;
+    }
+}
diff --git a/src/java/org/apache/cassandra/auth/RolesCacheMBean.java b/src/java/org/apache/cassandra/auth/RolesCacheMBean.java
new file mode 100644
index 0000000..cf270e6
--- /dev/null
+++ b/src/java/org/apache/cassandra/auth/RolesCacheMBean.java
@@ -0,0 +1,31 @@
+/*
+ * 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.auth;
+
+public interface RolesCacheMBean
+{
+    public void invalidate();
+
+    public void setValidity(int validityPeriod);
+
+    public int getValidity();
+
+    public void setUpdateInterval(int updateInterval);
+
+    public int getUpdateInterval();
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/cache/AutoSavingCache.java b/src/java/org/apache/cassandra/cache/AutoSavingCache.java
index 1174c44..2c6820e 100644
--- a/src/java/org/apache/cassandra/cache/AutoSavingCache.java
+++ b/src/java/org/apache/cassandra/cache/AutoSavingCache.java
@@ -37,24 +37,26 @@
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.db.compaction.CompactionInfo;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.db.marshal.BytesType;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.util.*;
+import org.apache.cassandra.io.util.ChecksummedRandomAccessReader.CorruptFileException;
 import org.apache.cassandra.service.CacheService;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.UUIDGen;
 
 public class AutoSavingCache<K extends CacheKey, V> extends InstrumentingCache<K, V>
 {
     public interface IStreamFactory
     {
-        public InputStream getInputStream(File path) throws FileNotFoundException;
-        public OutputStream getOutputStream(File path) throws FileNotFoundException;
+        InputStream getInputStream(File dataPath, File crcPath) throws IOException;
+        OutputStream getOutputStream(File dataPath, File crcPath) throws FileNotFoundException;
     }
 
     private static final Logger logger = LoggerFactory.getLogger(AutoSavingCache.class);
@@ -65,7 +67,7 @@
     protected volatile ScheduledFuture<?> saveTask;
     protected final CacheService.CacheType cacheType;
 
-    private CacheSerializer<K, V> cacheLoader;
+    private final CacheSerializer<K, V> cacheLoader;
 
     /*
      * CASSANDRA-10155 required a format change to fix 2i indexes and caching.
@@ -74,18 +76,18 @@
      * Since cache versions match exactly and there is no partial fallback just add
      * a minor version letter.
      */
-    private static final String CURRENT_VERSION = "ba";
+    private static final String CURRENT_VERSION = "ca";
 
     private static volatile IStreamFactory streamFactory = new IStreamFactory()
     {
-        public InputStream getInputStream(File path) throws FileNotFoundException
+        public InputStream getInputStream(File dataPath, File crcPath) throws IOException
         {
-            return new FileInputStream(path);
+            return ChecksummedRandomAccessReader.open(dataPath, crcPath);
         }
 
-        public OutputStream getOutputStream(File path) throws FileNotFoundException
+        public OutputStream getOutputStream(File dataPath, File crcPath)
         {
-            return new FileOutputStream(path);
+            return SequentialWriter.open(dataPath, crcPath).finishOnClose();
         }
     };
 
@@ -102,9 +104,14 @@
         this.cacheLoader = cacheloader;
     }
 
-    public File getCachePath(String version)
+    public File getCacheDataPath(String version)
     {
-        return DatabaseDescriptor.getSerializedCachePath(cacheType, version);
+        return DatabaseDescriptor.getSerializedCachePath( cacheType, version, "db");
+    }
+
+    public File getCacheCrcPath(String version)
+    {
+        return DatabaseDescriptor.getSerializedCachePath( cacheType, version, "crc");
     }
 
     public Writer getWriter(int keysToSave)
@@ -170,14 +177,15 @@
         long start = System.nanoTime();
 
         // modern format, allows both key and value (so key cache load can be purely sequential)
-        File path = getCachePath(CURRENT_VERSION);
-        if (path.exists())
+        File dataPath = getCacheDataPath(CURRENT_VERSION);
+        File crcPath = getCacheCrcPath(CURRENT_VERSION);
+        if (dataPath.exists() && crcPath.exists())
         {
             DataInputStream in = null;
             try
             {
-                logger.info(String.format("reading saved cache %s", path));
-                in = new DataInputStream(new LengthAvailableInputStream(new BufferedInputStream(streamFactory.getInputStream(path)), path.length()));
+                logger.info(String.format("reading saved cache %s", dataPath));
+                in = new DataInputStream(new LengthAvailableInputStream(new BufferedInputStream(streamFactory.getInputStream(dataPath, crcPath)), dataPath.length()));
 
                 //Check the schema has not changed since CFs are looked up by name which is ambiguous
                 UUID schemaVersion = new UUID(in.readLong(), in.readLong());
@@ -188,7 +196,6 @@
                                               + Schema.instance.getVersion());
 
                 ArrayDeque<Future<Pair<K, V>>> futures = new ArrayDeque<Future<Pair<K, V>>>();
-
                 while (in.available() > 0)
                 {
                     //ksname and cfname are serialized by the serializers in CacheService
@@ -234,19 +241,24 @@
                         put(entry.left, entry.right);
                 }
             }
+            catch (CorruptFileException e)
+            {
+                JVMStabilityInspector.inspectThrowable(e);
+                logger.warn(String.format("Non-fatal checksum error reading saved cache %s", dataPath.getAbsolutePath()), e);
+            }
             catch (Throwable t)
             {
                 JVMStabilityInspector.inspectThrowable(t);
-                logger.info(String.format("Harmless error reading saved cache %s", path.getAbsolutePath()), t);
+                logger.info(String.format("Harmless error reading saved cache %s", dataPath.getAbsolutePath()), t);
             }
             finally
             {
                 FileUtils.closeQuietly(in);
             }
         }
-        if (logger.isDebugEnabled())
-            logger.debug("completed reading ({} ms; {} keys) saved cache {}",
-                    TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start), count, path);
+        if (logger.isTraceEnabled())
+            logger.trace("completed reading ({} ms; {} keys) saved cache {}",
+                    TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start), count, dataPath);
         return count;
     }
 
@@ -257,16 +269,24 @@
 
     public class Writer extends CompactionInfo.Holder
     {
-        private final Set<K> keys;
+        private final Iterator<K> keyIterator;
         private final CompactionInfo info;
         private long keysWritten;
+        private final long keysEstimate;
 
         protected Writer(int keysToSave)
         {
-            if (keysToSave >= getKeySet().size())
-                keys = getKeySet();
+            int size = size();
+            if (keysToSave >= size || keysToSave == 0)
+            {
+                keyIterator = keyIterator();
+                keysEstimate = size;
+            }
             else
-                keys = hotKeySet(keysToSave);
+            {
+                keyIterator = hotKeyIterator(keysToSave);
+                keysEstimate = keysToSave;
+            }
 
             OperationType type;
             if (cacheType == CacheService.CacheType.KEY_CACHE)
@@ -278,11 +298,12 @@
             else
                 type = OperationType.UNKNOWN;
 
-            info = new CompactionInfo(CFMetaData.denseCFMetaData(Keyspace.SYSTEM_KS, cacheType.toString(), BytesType.instance),
+            info = new CompactionInfo(CFMetaData.denseCFMetaData(SystemKeyspace.NAME, cacheType.toString(), BytesType.instance),
                                       type,
                                       0,
-                                      keys.size(),
-                                      "keys");
+                                      keysEstimate,
+                                      "keys",
+                                      UUIDGen.getTimeUUID());
         }
 
         public CacheService.CacheType cacheType()
@@ -293,29 +314,31 @@
         public CompactionInfo getCompactionInfo()
         {
             // keyset can change in size, thus total can too
-            return info.forProgress(keysWritten, Math.max(keysWritten, keys.size()));
+            // TODO need to check for this one... was: info.forProgress(keysWritten, Math.max(keysWritten, keys.size()));
+            return info.forProgress(keysWritten, Math.max(keysWritten, keysEstimate));
         }
 
+        @SuppressWarnings("resource")
         public void saveCache()
         {
-            logger.debug("Deleting old {} files.", cacheType);
+            logger.trace("Deleting old {} files.", cacheType);
             deleteOldCacheFiles();
 
-            if (keys.isEmpty())
+            if (!keyIterator.hasNext())
             {
-                logger.debug("Skipping {} save, cache is empty.", cacheType);
+                logger.trace("Skipping {} save, cache is empty.", cacheType);
                 return;
             }
 
             long start = System.nanoTime();
 
-            DataOutputStreamPlus writer = null;
-            File tempCacheFile = tempCacheFile();
+            WrappedDataOutputStreamPlus writer = null;
+            Pair<File, File> cacheFilePaths = tempCacheFiles();
             try
             {
                 try
                 {
-                    writer = new DataOutputStreamPlus(streamFactory.getOutputStream(tempCacheFile));
+                    writer = new WrappedDataOutputStreamPlus(streamFactory.getOutputStream(cacheFilePaths.left, cacheFilePaths.right));
                 }
                 catch (FileNotFoundException e)
                 {
@@ -334,8 +357,9 @@
                     writer.writeLong(schemaVersion.getMostSignificantBits());
                     writer.writeLong(schemaVersion.getLeastSignificantBits());
 
-                    for (K key : keys)
+                    while (keyIterator.hasNext())
                     {
+                        K key = keyIterator.next();
 
                         ColumnFamilyStore cfs = Schema.instance.getColumnFamilyStoreIncludingIndexes(key.ksAndCFName);
                         if (cfs == null)
@@ -344,12 +368,15 @@
                         cacheLoader.serialize(key, writer, cfs);
 
                         keysWritten++;
+                        if (keysWritten >= keysEstimate)
+                            break;
                     }
                 }
                 catch (IOException e)
                 {
-                    throw new FSWriteError(e, tempCacheFile);
+                    throw new FSWriteError(e, cacheFilePaths.left);
                 }
+
             }
             finally
             {
@@ -357,20 +384,27 @@
                     FileUtils.closeQuietly(writer);
             }
 
-            File cacheFile = getCachePath(CURRENT_VERSION);
+            File cacheFile = getCacheDataPath(CURRENT_VERSION);
+            File crcFile = getCacheCrcPath(CURRENT_VERSION);
 
             cacheFile.delete(); // ignore error if it didn't exist
+            crcFile.delete();
 
-            if (!tempCacheFile.renameTo(cacheFile))
-                logger.error("Unable to rename {} to {}", tempCacheFile, cacheFile);
+            if (!cacheFilePaths.left.renameTo(cacheFile))
+                logger.error("Unable to rename {} to {}", cacheFilePaths.left, cacheFile);
 
-            logger.info("Saved {} ({} items) in {} ms", cacheType, keys.size(), TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start));
+            if (!cacheFilePaths.right.renameTo(crcFile))
+                logger.error("Unable to rename {} to {}", cacheFilePaths.right, crcFile);
+
+            logger.info("Saved {} ({} items) in {} ms", cacheType, keysWritten, TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start));
         }
 
-        private File tempCacheFile()
+        private Pair<File, File> tempCacheFiles()
         {
-            File path = getCachePath(CURRENT_VERSION);
-            return FileUtils.createTempFile(path.getName(), null, path.getParentFile());
+            File dataPath = getCacheDataPath(CURRENT_VERSION);
+            File crcPath = getCacheCrcPath(CURRENT_VERSION);
+            return Pair.create(FileUtils.createTempFile(dataPath.getName(), null, dataPath.getParentFile()),
+                               FileUtils.createTempFile(crcPath.getName(), null, crcPath.getParentFile()));
         }
 
         private void deleteOldCacheFiles()
@@ -380,13 +414,14 @@
             File[] files = savedCachesDir.listFiles();
             if (files != null)
             {
+                String cacheNameFormat = String.format("%s-%s.db", cacheType.toString(), CURRENT_VERSION);
                 for (File file : files)
                 {
                     if (!file.isFile())
                         continue; // someone's been messing with our directory.  naughty!
 
-                    if (file.getName().endsWith(cacheType.toString())
-                            || file.getName().endsWith(String.format("%s-%s.db", cacheType.toString(), CURRENT_VERSION)))
+                    if (file.getName().endsWith(cacheNameFormat)
+                     || file.getName().endsWith(cacheType.toString()))
                     {
                         if (!file.delete())
                             logger.warn("Failed to delete {}", file.getAbsolutePath());
diff --git a/src/java/org/apache/cassandra/cache/CacheProvider.java b/src/java/org/apache/cassandra/cache/CacheProvider.java
new file mode 100644
index 0000000..6a97be3
--- /dev/null
+++ b/src/java/org/apache/cassandra/cache/CacheProvider.java
@@ -0,0 +1,23 @@
+/*
+ * 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.cache;
+
+public interface CacheProvider<K, V>
+{
+    ICache<K, V> create();
+}
diff --git a/src/java/org/apache/cassandra/cache/CachingOptions.java b/src/java/org/apache/cassandra/cache/CachingOptions.java
index 6eeaa37..1c82f55 100644
--- a/src/java/org/apache/cassandra/cache/CachingOptions.java
+++ b/src/java/org/apache/cassandra/cache/CachingOptions.java
@@ -76,18 +76,18 @@
             {
                 if (!(value.equals("ALL") || value.equals("NONE")))
                 {
-                    throw new ConfigurationException("'keys' can only have values 'ALL' or 'NONE'");
+                    throw new ConfigurationException("'keys' can only have values 'ALL' or 'NONE', but was '" + value + "'");
                 }
             }
             else if (entry.getKey().equals("rows_per_partition"))
             {
                 if (!(value.equals("ALL") || value.equals("NONE") || StringUtils.isNumeric(value)))
                 {
-                    throw new ConfigurationException("'rows_per_partition' can only have values 'ALL', 'NONE' or be numeric.");
+                    throw new ConfigurationException("'rows_per_partition' can only have values 'ALL', 'NONE' or be numeric, but was '" + value + "'.");
                 }
             }
             else
-                throw new ConfigurationException("Only supported CachingOptions parameters are 'keys' and 'rows_per_partition'");
+                throw new ConfigurationException("Only supported CachingOptions parameters are 'keys' and 'rows_per_partition', but was '" + entry.getKey() + "'");
         }
     }
 
@@ -130,11 +130,7 @@
         return result;
     }
 
-    public static boolean isLegacy(String CachingOptions)
-    {
-        return legacyOptions.contains(CachingOptions.toUpperCase());
-    }
-
+    // FIXME: move to ThriftConversion
     public static CachingOptions fromThrift(String caching, String cellsPerRow) throws ConfigurationException
     {
 
@@ -153,6 +149,7 @@
         return new CachingOptions(kc, rc);
     }
 
+    // FIXME: move to ThriftConversion
     public String toThriftCaching()
     {
         if (rowCache.isEnabled() && keyCache.isEnabled())
@@ -164,6 +161,7 @@
         return "NONE";
     }
 
+    // FIXME: move to ThriftConversion
     public String toThriftCellsPerRow()
     {
         if (rowCache.cacheFullPartitions())
@@ -171,7 +169,6 @@
         return String.valueOf(rowCache.rowsToCache);
     }
 
-
     public static class KeyCache
     {
         public final Type type;
@@ -191,7 +188,7 @@
 
         public boolean isEnabled()
         {
-            return type.equals(Type.ALL);
+            return type == Type.ALL;
         }
 
         @Override
@@ -226,7 +223,7 @@
 
         public RowCache(Type type)
         {
-            this(type, type.equals(Type.ALL) ? Integer.MAX_VALUE : 0);
+            this(type, (type == Type.ALL) ? Integer.MAX_VALUE : 0);
         }
         public RowCache(Type type, int rowsToCache)
         {
@@ -249,17 +246,17 @@
         }
         public boolean isEnabled()
         {
-            return type.equals(Type.ALL) || type.equals(Type.HEAD);
+            return (type == Type.ALL) || (type == Type.HEAD);
         }
         public boolean cacheFullPartitions()
         {
-            return type.equals(Type.ALL);
+            return type == Type.ALL;
         }
         @Override
         public String toString()
         {
-            if (type.equals(Type.ALL)) return "ALL";
-            if (type.equals(Type.NONE)) return "NONE";
+            if (type == Type.ALL) return "ALL";
+            if (type == Type.NONE) return "NONE";
             return String.valueOf(rowsToCache);
         }
 
diff --git a/src/java/org/apache/cassandra/cache/ConcurrentLinkedHashCache.java b/src/java/org/apache/cassandra/cache/ConcurrentLinkedHashCache.java
index 8182447..bb14055 100644
--- a/src/java/org/apache/cassandra/cache/ConcurrentLinkedHashCache.java
+++ b/src/java/org/apache/cassandra/cache/ConcurrentLinkedHashCache.java
@@ -17,7 +17,7 @@
  */
 package org.apache.cassandra.cache;
 
-import java.util.Set;
+import java.util.Iterator;
 
 import com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap;
 import com.googlecode.concurrentlinkedhashmap.EntryWeigher;
@@ -45,7 +45,7 @@
                                             .concurrencyLevel(DEFAULT_CONCURENCY_LEVEL)
                                             .build();
 
-        return new ConcurrentLinkedHashCache<K, V>(map);
+        return new ConcurrentLinkedHashCache<>(map);
     }
 
     public static <K extends IMeasurableMemory, V extends IMeasurableMemory> ConcurrentLinkedHashCache<K, V> create(long weightedCapacity)
@@ -116,14 +116,14 @@
         map.remove(key);
     }
 
-    public Set<K> keySet()
+    public Iterator<K> keyIterator()
     {
-        return map.keySet();
+        return map.keySet().iterator();
     }
 
-    public Set<K> hotKeySet(int n)
+    public Iterator<K> hotKeyIterator(int n)
     {
-        return map.descendingKeySetWithLimit(n);
+        return map.descendingKeySetWithLimit(n).iterator();
     }
 
     public boolean containsKey(K key)
diff --git a/src/java/org/apache/cassandra/cache/ICache.java b/src/java/org/apache/cassandra/cache/ICache.java
index 22dbb16..37b55cd 100644
--- a/src/java/org/apache/cassandra/cache/ICache.java
+++ b/src/java/org/apache/cassandra/cache/ICache.java
@@ -17,7 +17,7 @@
  */
 package org.apache.cassandra.cache;
 
-import java.util.Set;
+import java.util.Iterator;
 
 /**
  * This is similar to the Map interface, but requires maintaining a given capacity
@@ -46,9 +46,9 @@
 
     public void clear();
 
-    public Set<K> keySet();
+    public Iterator<K> keyIterator();
 
-    public Set<K> hotKeySet(int n);
+    public Iterator<K> hotKeyIterator(int n);
 
     public boolean containsKey(K key);
 }
diff --git a/src/java/org/apache/cassandra/cache/InstrumentingCache.java b/src/java/org/apache/cassandra/cache/InstrumentingCache.java
index 311b373..c8728fd 100644
--- a/src/java/org/apache/cassandra/cache/InstrumentingCache.java
+++ b/src/java/org/apache/cassandra/cache/InstrumentingCache.java
@@ -17,7 +17,7 @@
  */
 package org.apache.cassandra.cache;
 
-import java.util.Set;
+import java.util.Iterator;
 
 import org.apache.cassandra.metrics.CacheMetrics;
 
@@ -26,7 +26,6 @@
  */
 public class InstrumentingCache<K, V>
 {
-    private volatile boolean capacitySetManually;
     private final ICache<K, V> map;
     private final String type;
 
@@ -78,20 +77,9 @@
         return map.capacity();
     }
 
-    public boolean isCapacitySetManually()
-    {
-        return capacitySetManually;
-    }
-
-    public void updateCapacity(long capacity)
-    {
-        map.setCapacity(capacity);
-    }
-
     public void setCapacity(long capacity)
     {
-        updateCapacity(capacity);
-        capacitySetManually = true;
+        map.setCapacity(capacity);
     }
 
     public int size()
@@ -110,14 +98,14 @@
         metrics = new CacheMetrics(type, map);
     }
 
-    public Set<K> getKeySet()
+    public Iterator<K> keyIterator()
     {
-        return map.keySet();
+        return map.keyIterator();
     }
 
-    public Set<K> hotKeySet(int n)
+    public Iterator<K> hotKeyIterator(int n)
     {
-        return map.hotKeySet(n);
+        return map.hotKeyIterator(n);
     }
 
     public boolean containsKey(K key)
diff --git a/src/java/org/apache/cassandra/cache/NopCacheProvider.java b/src/java/org/apache/cassandra/cache/NopCacheProvider.java
new file mode 100644
index 0000000..20f837a
--- /dev/null
+++ b/src/java/org/apache/cassandra/cache/NopCacheProvider.java
@@ -0,0 +1,93 @@
+/*
+ * 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.cache;
+
+import java.util.Collections;
+import java.util.Iterator;
+
+public class NopCacheProvider implements CacheProvider<RowCacheKey, IRowCacheEntry>
+{
+    public ICache<RowCacheKey, IRowCacheEntry> create()
+    {
+        return new NopCache();
+    }
+
+    private static class NopCache implements ICache<RowCacheKey, IRowCacheEntry>
+    {
+        public long capacity()
+        {
+            return 0;
+        }
+
+        public void setCapacity(long capacity)
+        {
+        }
+
+        public void put(RowCacheKey key, IRowCacheEntry value)
+        {
+        }
+
+        public boolean putIfAbsent(RowCacheKey key, IRowCacheEntry value)
+        {
+            return false;
+        }
+
+        public boolean replace(RowCacheKey key, IRowCacheEntry old, IRowCacheEntry value)
+        {
+            return false;
+        }
+
+        public IRowCacheEntry get(RowCacheKey key)
+        {
+            return null;
+        }
+
+        public void remove(RowCacheKey key)
+        {
+        }
+
+        public int size()
+        {
+            return 0;
+        }
+
+        public long weightedSize()
+        {
+            return 0;
+        }
+
+        public void clear()
+        {
+        }
+
+        public Iterator<RowCacheKey> hotKeyIterator(int n)
+        {
+            return Collections.emptyIterator();
+        }
+
+        public Iterator<RowCacheKey> keyIterator()
+        {
+            return Collections.emptyIterator();
+        }
+
+        public boolean containsKey(RowCacheKey key)
+        {
+            return false;
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/cache/OHCProvider.java b/src/java/org/apache/cassandra/cache/OHCProvider.java
new file mode 100644
index 0000000..ab2745a
--- /dev/null
+++ b/src/java/org/apache/cassandra/cache/OHCProvider.java
@@ -0,0 +1,285 @@
+/*
+ * 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.cache;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.WritableByteChannel;
+import java.util.Iterator;
+
+import com.google.common.base.Function;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.ColumnFamily;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.Memory;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.utils.Pair;
+import org.caffinitas.ohc.OHCache;
+import org.caffinitas.ohc.OHCacheBuilder;
+
+public class OHCProvider implements CacheProvider<RowCacheKey, IRowCacheEntry>
+{
+    public ICache<RowCacheKey, IRowCacheEntry> create()
+    {
+        OHCacheBuilder<RowCacheKey, IRowCacheEntry> builder = OHCacheBuilder.newBuilder();
+        builder.capacity(DatabaseDescriptor.getRowCacheSizeInMB() * 1024 * 1024)
+               .keySerializer(new KeySerializer())
+               .valueSerializer(new ValueSerializer())
+               .throwOOME(true);
+
+        return new OHCacheAdapter(builder.build());
+    }
+
+    private static class OHCacheAdapter implements ICache<RowCacheKey, IRowCacheEntry>
+    {
+        private final OHCache<RowCacheKey, IRowCacheEntry> ohCache;
+
+        public OHCacheAdapter(OHCache<RowCacheKey, IRowCacheEntry> ohCache)
+        {
+            this.ohCache = ohCache;
+        }
+
+        public long capacity()
+        {
+            return ohCache.capacity();
+        }
+
+        public void setCapacity(long capacity)
+        {
+            ohCache.setCapacity(capacity);
+        }
+
+        public void put(RowCacheKey key, IRowCacheEntry value)
+        {
+            ohCache.put(key, value);
+        }
+
+        public boolean putIfAbsent(RowCacheKey key, IRowCacheEntry value)
+        {
+            return ohCache.putIfAbsent(key, value);
+        }
+
+        public boolean replace(RowCacheKey key, IRowCacheEntry old, IRowCacheEntry value)
+        {
+            return ohCache.addOrReplace(key, old, value);
+        }
+
+        public IRowCacheEntry get(RowCacheKey key)
+        {
+            return ohCache.get(key);
+        }
+
+        public void remove(RowCacheKey key)
+        {
+            ohCache.remove(key);
+        }
+
+        public int size()
+        {
+            return (int) ohCache.size();
+        }
+
+        public long weightedSize()
+        {
+            return ohCache.memUsed();
+        }
+
+        public void clear()
+        {
+            ohCache.clear();
+        }
+
+        public Iterator<RowCacheKey> hotKeyIterator(int n)
+        {
+            return ohCache.hotKeyIterator(n);
+        }
+
+        public Iterator<RowCacheKey> keyIterator()
+        {
+            return ohCache.keyIterator();
+        }
+
+        public boolean containsKey(RowCacheKey key)
+        {
+            return ohCache.containsKey(key);
+        }
+    }
+
+    private static class KeySerializer implements org.caffinitas.ohc.CacheSerializer<RowCacheKey>
+    {
+        public void serialize(RowCacheKey rowCacheKey, DataOutput dataOutput) throws IOException
+        {
+            dataOutput.writeUTF(rowCacheKey.ksAndCFName.left);
+            dataOutput.writeUTF(rowCacheKey.ksAndCFName.right);
+            dataOutput.writeInt(rowCacheKey.key.length);
+            dataOutput.write(rowCacheKey.key);
+        }
+
+        public RowCacheKey deserialize(DataInput dataInput) throws IOException
+        {
+            String ksName = dataInput.readUTF();
+            String cfName = dataInput.readUTF();
+            byte[] key = new byte[dataInput.readInt()];
+            dataInput.readFully(key);
+            return new RowCacheKey(Pair.create(ksName, cfName), key);
+        }
+
+        public int serializedSize(RowCacheKey rowCacheKey)
+        {
+            return TypeSizes.NATIVE.sizeof(rowCacheKey.ksAndCFName.left)
+                    + TypeSizes.NATIVE.sizeof(rowCacheKey.ksAndCFName.right)
+                    + 4
+                    + rowCacheKey.key.length;
+        }
+    }
+
+    private static class ValueSerializer implements org.caffinitas.ohc.CacheSerializer<IRowCacheEntry>
+    {
+        public void serialize(IRowCacheEntry entry, DataOutput out) throws IOException
+        {
+            assert entry != null; // unlike CFS we don't support nulls, since there is no need for that in the cache
+            boolean isSentinel = entry instanceof RowCacheSentinel;
+            out.writeBoolean(isSentinel);
+            if (isSentinel)
+                out.writeLong(((RowCacheSentinel) entry).sentinelId);
+            else
+                ColumnFamily.serializer.serialize((ColumnFamily) entry, new DataOutputPlusAdapter(out), MessagingService.current_version);
+        }
+
+        public IRowCacheEntry deserialize(DataInput in) throws IOException
+        {
+            boolean isSentinel = in.readBoolean();
+            if (isSentinel)
+                return new RowCacheSentinel(in.readLong());
+            return ColumnFamily.serializer.deserialize(in, MessagingService.current_version);
+        }
+
+        public int serializedSize(IRowCacheEntry entry)
+        {
+            TypeSizes typeSizes = TypeSizes.NATIVE;
+            int size = typeSizes.sizeof(true);
+            if (entry instanceof RowCacheSentinel)
+                size += typeSizes.sizeof(((RowCacheSentinel) entry).sentinelId);
+            else
+                size += ColumnFamily.serializer.serializedSize((ColumnFamily) entry, typeSizes, MessagingService.current_version);
+            return size;
+        }
+    }
+
+    static class DataOutputPlusAdapter implements DataOutputPlus
+    {
+        private final DataOutput out;
+
+        public void write(byte[] b) throws IOException
+        {
+            out.write(b);
+        }
+
+        public void write(byte[] b, int off, int len) throws IOException
+        {
+            out.write(b, off, len);
+        }
+
+        public void write(int b) throws IOException
+        {
+            out.write(b);
+        }
+
+        public void writeBoolean(boolean v) throws IOException
+        {
+            out.writeBoolean(v);
+        }
+
+        public void writeByte(int v) throws IOException
+        {
+            out.writeByte(v);
+        }
+
+        public void writeBytes(String s) throws IOException
+        {
+            out.writeBytes(s);
+        }
+
+        public void writeChar(int v) throws IOException
+        {
+            out.writeChar(v);
+        }
+
+        public void writeChars(String s) throws IOException
+        {
+            out.writeChars(s);
+        }
+
+        public void writeDouble(double v) throws IOException
+        {
+            out.writeDouble(v);
+        }
+
+        public void writeFloat(float v) throws IOException
+        {
+            out.writeFloat(v);
+        }
+
+        public void writeInt(int v) throws IOException
+        {
+            out.writeInt(v);
+        }
+
+        public void writeLong(long v) throws IOException
+        {
+            out.writeLong(v);
+        }
+
+        public void writeShort(int v) throws IOException
+        {
+            out.writeShort(v);
+        }
+
+        public void writeUTF(String s) throws IOException
+        {
+            out.writeUTF(s);
+        }
+
+        public DataOutputPlusAdapter(DataOutput out)
+        {
+            this.out = out;
+        }
+
+        public void write(ByteBuffer buffer) throws IOException
+        {
+            if (buffer.hasArray())
+                out.write(buffer.array(), buffer.arrayOffset() + buffer.position(), buffer.remaining());
+            else
+                throw new UnsupportedOperationException("IMPLEMENT ME");
+        }
+
+        public void write(Memory memory, long offset, long length) throws IOException
+        {
+            throw new UnsupportedOperationException("IMPLEMENT ME");
+        }
+
+        public <R> R applyToChannel(Function<WritableByteChannel, R> c) throws IOException
+        {
+            throw new UnsupportedOperationException("IMPLEMENT ME");
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/cache/RowCacheKey.java b/src/java/org/apache/cassandra/cache/RowCacheKey.java
index c959fd1..e02db42 100644
--- a/src/java/org/apache/cassandra/cache/RowCacheKey.java
+++ b/src/java/org/apache/cassandra/cache/RowCacheKey.java
@@ -31,6 +31,12 @@
 
     private static final long EMPTY_SIZE = ObjectSizes.measure(new RowCacheKey(null, ByteBufferUtil.EMPTY_BYTE_BUFFER));
 
+    public RowCacheKey(Pair<String, String> ksAndCFName, byte[] key)
+    {
+        super(ksAndCFName);
+        this.key = key;
+    }
+
     public RowCacheKey(Pair<String, String> ksAndCFName, DecoratedKey key)
     {
         this(ksAndCFName, key.getKey());
diff --git a/src/java/org/apache/cassandra/cache/SerializingCache.java b/src/java/org/apache/cassandra/cache/SerializingCache.java
index ca65fcc..01d70b4 100644
--- a/src/java/org/apache/cassandra/cache/SerializingCache.java
+++ b/src/java/org/apache/cassandra/cache/SerializingCache.java
@@ -18,7 +18,7 @@
 package org.apache.cassandra.cache;
 
 import java.io.IOException;
-import java.util.Set;
+import java.util.Iterator;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -92,7 +92,7 @@
         }
         catch (IOException e)
         {
-            logger.debug("Cannot fetch in memory data, we will fallback to read from disk ", e);
+            logger.trace("Cannot fetch in memory data, we will fallback to read from disk ", e);
             return null;
         }
     }
@@ -155,6 +155,7 @@
         map.clear();
     }
 
+    @SuppressWarnings("resource")
     public V get(K key)
     {
         RefCountedMemory mem = map.get(key);
@@ -172,6 +173,7 @@
         }
     }
 
+    @SuppressWarnings("resource")
     public void put(K key, V value)
     {
         RefCountedMemory mem = serialize(value);
@@ -193,6 +195,7 @@
             old.unreference();
     }
 
+    @SuppressWarnings("resource")
     public boolean putIfAbsent(K key, V value)
     {
         RefCountedMemory mem = serialize(value);
@@ -216,6 +219,7 @@
         return old == null;
     }
 
+    @SuppressWarnings("resource")
     public boolean replace(K key, V oldToReplace, V value)
     {
         // if there is no old value in our map, we fail
@@ -259,19 +263,20 @@
 
     public void remove(K key)
     {
+        @SuppressWarnings("resource")
         RefCountedMemory mem = map.remove(key);
         if (mem != null)
             mem.unreference();
     }
 
-    public Set<K> keySet()
+    public Iterator<K> keyIterator()
     {
-        return map.keySet();
+        return map.keySet().iterator();
     }
 
-    public Set<K> hotKeySet(int n)
+    public Iterator<K> hotKeyIterator(int n)
     {
-        return map.descendingKeySetWithLimit(n);
+        return map.descendingKeySetWithLimit(n).iterator();
     }
 
     public boolean containsKey(K key)
diff --git a/src/java/org/apache/cassandra/cache/SerializingCacheProvider.java b/src/java/org/apache/cassandra/cache/SerializingCacheProvider.java
index a058872..f540322 100644
--- a/src/java/org/apache/cassandra/cache/SerializingCacheProvider.java
+++ b/src/java/org/apache/cassandra/cache/SerializingCacheProvider.java
@@ -20,17 +20,18 @@
 import java.io.DataInput;
 import java.io.IOException;
 
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamily;
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.ISerializer;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.net.MessagingService;
 
-public class SerializingCacheProvider
+public class SerializingCacheProvider implements CacheProvider<RowCacheKey, IRowCacheEntry>
 {
-    public ICache<RowCacheKey, IRowCacheEntry> create(long capacity)
+    public ICache<RowCacheKey, IRowCacheEntry> create()
     {
-        return SerializingCache.create(capacity, new RowCacheSerializer());
+        return SerializingCache.create(DatabaseDescriptor.getRowCacheSizeInMB() * 1024 * 1024, new RowCacheSerializer());
     }
 
     // Package protected for tests
diff --git a/src/java/org/apache/cassandra/cli/Cli.g b/src/java/org/apache/cassandra/cli/Cli.g
deleted file mode 100644
index 07cdc1f..0000000
--- a/src/java/org/apache/cassandra/cli/Cli.g
+++ /dev/null
@@ -1,728 +0,0 @@
-/**
- * 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.
- */
-
-// ANTLR Grammar for the Cassandra Command Line Interface (CLI).
-
-grammar Cli;
-
-options {
-    output=AST;
-    ASTLabelType=CommonTree;
-    backtrack=true;
-}
-
-//
-// Nodes in the AST
-//
-tokens {
-    //
-    // Top-level nodes. These typically correspond to
-    // various top-level CLI statements.
-    //
-    NODE_CONNECT;
-    NODE_DESCRIBE;
-    NODE_DESCRIBE_CLUSTER;
-    NODE_USE_TABLE;
-    NODE_TRACE_NEXT_QUERY;
-    NODE_SET_TRACE_PROBABILITY;
-    NODE_EXIT;
-    NODE_HELP;
-    NODE_NO_OP;
-    NODE_SHOW_CLUSTER_NAME;
-    NODE_SHOW_VERSION;
-    NODE_SHOW_KEYSPACES;
-    NODE_SHOW_SCHEMA;
-    NODE_THRIFT_GET;
-    NODE_THRIFT_GET_WITH_CONDITIONS;
-    NODE_THRIFT_SET;
-    NODE_THRIFT_COUNT;
-    NODE_THRIFT_DEL;
-    NODE_THRIFT_INCR;
-    NODE_THRIFT_DECR;
-    NODE_ADD_COLUMN_FAMILY;
-    NODE_ADD_KEYSPACE;
-    NODE_DEL_KEYSPACE;
-    NODE_DEL_COLUMN_FAMILY;
-    NODE_UPDATE_KEYSPACE;
-    NODE_UPDATE_COLUMN_FAMILY;
-    NODE_LIST;
-    NODE_TRUNCATE;
-    NODE_ASSUME;
-    NODE_CONSISTENCY_LEVEL;
-    NODE_DROP_INDEX;
-
-    // Internal Nodes.
-    NODE_COLUMN_ACCESS;
-    NODE_ID_LIST;
-    NODE_NEW_CF_ACCESS;
-    NODE_NEW_KEYSPACE_ACCESS;
-    
-    CONVERT_TO_TYPE;
-    FUNCTION_CALL;
-    CONDITION;
-    CONDITIONS;
-    ARRAY;
-    HASH;
-    PAIR;
-
-    NODE_LIMIT;
-    NODE_COLUMNS;
-    NODE_REVERSED;
-    NODE_KEY_RANGE;
-}
-
-@parser::header {
-package org.apache.cassandra.cli;
-}
-
-@lexer::header {
-package org.apache.cassandra.cli;
-}
-
-@lexer::members
-{
-    public void reportError(RecognitionException e) 
-    {
-        StringBuilder errorMessage = new StringBuilder("Syntax error at position ").append(e.charPositionInLine).append(": ");
-
-        if (e instanceof NoViableAltException)
-        {
-            int index = e.charPositionInLine;
-            String error = this.input.substring(index, index);
-            String statement = this.input.substring(0, this.input.size() - 1);
-
-            errorMessage.append("unexpected \"").append(error).append("\" for `").append(statement).append("`.");
-        }
-        else
-        {
-            errorMessage.append(this.getErrorMessage(e, this.getTokenNames()));
-        }
-
-        throw new RuntimeException(errorMessage.toString());
-    }
-}
-
-@parser::members
-{
-    public void reportError(RecognitionException e) 
-    {
-        String errorMessage = "Syntax error at position " + e.charPositionInLine + ": " + this.getErrorMessage(e, this.getTokenNames());
-
-        throw new RuntimeException(errorMessage);
-    }
-}
-
-//
-// Parser Section
-//
-
-// the root node
-root: statement SEMICOLON? EOF -> statement;
-
-statement
-    : connectStatement
-    | exitStatement
-    | countStatement
-    | describeTable
-    | describeCluster
-    | addKeyspace
-    | addColumnFamily
-    | updateKeyspace
-    | updateColumnFamily
-    | delColumnFamily
-    | delKeyspace
-    | useKeyspace
-    | traceNextQuery
-    | setTraceProbability
-    | delStatement
-    | getStatement
-    | helpStatement
-    | setStatement
-    | incrStatement
-    | showStatement
-    | listStatement
-    | truncateStatement
-    | assumeStatement
-    | consistencyLevelStatement
-    | dropIndex
-    | -> ^(NODE_NO_OP)
-    ;
-
-connectStatement
-    : CONNECT host '/' port (username password)?
-        -> ^(NODE_CONNECT host port (username password)?)
-    | CONNECT ip_address '/' port (username password)?
-        -> ^(NODE_CONNECT ip_address port (username password)?)
-    ;
-
-helpStatement
-    : HELP HELP 
-        -> ^(NODE_HELP NODE_HELP)
-    | HELP CONNECT 
-        -> ^(NODE_HELP NODE_CONNECT)
-    | HELP USE 
-        -> ^(NODE_HELP NODE_USE_TABLE)
-    | HELP TRACE NEXT QUERY
-        -> ^(NODE_HELP NODE_TRACE_NEXT_QUERY)
-    | HELP SET TRACE PROBABILITY
-        -> ^(NODE_HELP NODE_SET_TRACE_PROBABILITY)
-    | HELP DESCRIBE
-        -> ^(NODE_HELP NODE_DESCRIBE)
-    | HELP DESCRIBE 'CLUSTER'
-        -> ^(NODE_HELP NODE_DESCRIBE_CLUSTER)
-    | HELP EXIT 
-        -> ^(NODE_HELP NODE_EXIT)
-    | HELP QUIT 
-        -> ^(NODE_HELP NODE_EXIT)
-    | HELP SHOW 'CLUSTER NAME'
-        -> ^(NODE_HELP NODE_SHOW_CLUSTER_NAME)
-    | HELP SHOW KEYSPACES 
-        -> ^(NODE_HELP NODE_SHOW_KEYSPACES)
-    | HELP SHOW SCHEMA
-            -> ^(NODE_HELP NODE_SHOW_SCHEMA)
-    | HELP SHOW API_VERSION
-        -> ^(NODE_HELP NODE_SHOW_VERSION)
-    | HELP CREATE KEYSPACE 
-        -> ^(NODE_HELP NODE_ADD_KEYSPACE)
-    | HELP UPDATE KEYSPACE
-        -> ^(NODE_HELP NODE_UPDATE_KEYSPACE)
-    | HELP CREATE COLUMN FAMILY 
-        -> ^(NODE_HELP NODE_ADD_COLUMN_FAMILY)
-    | HELP UPDATE COLUMN FAMILY
-        -> ^(NODE_HELP NODE_UPDATE_COLUMN_FAMILY)
-    | HELP DROP KEYSPACE 
-        -> ^(NODE_HELP NODE_DEL_KEYSPACE)
-    | HELP DROP COLUMN FAMILY 
-        -> ^(NODE_HELP NODE_DEL_COLUMN_FAMILY)
-    | HELP DROP INDEX
-        -> ^(NODE_HELP NODE_DROP_INDEX)
-    | HELP GET 
-        -> ^(NODE_HELP NODE_THRIFT_GET)
-    | HELP SET 
-        -> ^(NODE_HELP NODE_THRIFT_SET)
-    | HELP INCR
-        -> ^(NODE_HELP NODE_THRIFT_INCR)
-    | HELP DECR
-        -> ^(NODE_HELP NODE_THRIFT_DECR)
-    | HELP DEL 
-        -> ^(NODE_HELP NODE_THRIFT_DEL)
-    | HELP COUNT 
-        -> ^(NODE_HELP NODE_THRIFT_COUNT)
-    | HELP LIST 
-        -> ^(NODE_HELP NODE_LIST)
-    | HELP TRUNCATE
-        -> ^(NODE_HELP NODE_TRUNCATE)
-    | HELP ASSUME
-        -> ^(NODE_HELP NODE_ASSUME)
-    | HELP CONSISTENCYLEVEL
-        -> ^(NODE_HELP NODE_CONSISTENCY_LEVEL)
-    | HELP 
-        -> ^(NODE_HELP)
-    | '?'    
-        -> ^(NODE_HELP)
-    ;
-
-exitStatement
-    : QUIT -> ^(NODE_EXIT)
-    | EXIT -> ^(NODE_EXIT)
-    ;
-
-getStatement
-    : GET columnFamilyExpr ('AS' typeIdentifier)? ('LIMIT' limit=IntegerPositiveLiteral)?
-        -> ^(NODE_THRIFT_GET columnFamilyExpr ( ^(CONVERT_TO_TYPE typeIdentifier) )? ^(NODE_LIMIT $limit)?)
-    | GET columnFamily 'WHERE' getCondition ('AND' getCondition)* ('LIMIT' limit=IntegerPositiveLiteral)?
-        -> ^(NODE_THRIFT_GET_WITH_CONDITIONS columnFamily ^(CONDITIONS getCondition+) ^(NODE_LIMIT $limit)?) 
-    ;
-
-getCondition
-    : columnOrSuperColumn operator value
-        -> ^(CONDITION operator columnOrSuperColumn value)
-    ;
-
-operator
-    : '=' | '>' | '<' | '>=' | '<='
-    ;
-
-typeIdentifier
-    : Identifier | StringLiteral | IntegerPositiveLiteral 
-    ;
-
-setStatement
-    : SET columnFamilyExpr '=' objectValue=value (WITH TTL '=' ttlValue=IntegerPositiveLiteral)?
-        -> ^(NODE_THRIFT_SET columnFamilyExpr $objectValue ( $ttlValue )?)
-    ;
-
-incrStatement
-    : INCR columnFamilyExpr (BY byValue=incrementValue)?
-        -> ^(NODE_THRIFT_INCR columnFamilyExpr ( $byValue )?)
-    | DECR columnFamilyExpr (BY byValue=incrementValue)?
-        -> ^(NODE_THRIFT_DECR columnFamilyExpr ( $byValue )?)
-    ;
-
-countStatement
-    : COUNT columnFamilyExpr 
-        -> ^(NODE_THRIFT_COUNT columnFamilyExpr)
-    ;
-
-delStatement
-    : DEL columnFamilyExpr 
-        -> ^(NODE_THRIFT_DEL columnFamilyExpr)
-    ;
-
-showStatement
-    : showClusterName
-    | showVersion
-    | showKeyspaces
-    | showSchema
-    ;
-
-listStatement
-    : LIST columnFamily keyRangeExpr? rowLimitExpr? columnLimitExpr?
-        -> ^(NODE_LIST columnFamily keyRangeExpr? rowLimitExpr? columnLimitExpr?)
-    ;
-
-truncateStatement
-    : TRUNCATE columnFamily
-        -> ^(NODE_TRUNCATE columnFamily)
-    ;
-
-assumeStatement
-    : ASSUME columnFamily assumptionElement=Identifier 'AS' entityName
-        -> ^(NODE_ASSUME columnFamily $assumptionElement entityName)
-    ;
-
-consistencyLevelStatement
-    : CONSISTENCYLEVEL 'AS' defaultType=Identifier
-        -> ^(NODE_CONSISTENCY_LEVEL $defaultType)
-    ;
-
-showClusterName
-    : SHOW 'CLUSTER NAME'
-        -> ^(NODE_SHOW_CLUSTER_NAME)
-    ;
-
-addKeyspace
-    : CREATE KEYSPACE keyValuePairExpr 
-        -> ^(NODE_ADD_KEYSPACE keyValuePairExpr)
-    ;
-
-addColumnFamily
-    : CREATE COLUMN FAMILY keyValuePairExpr 
-        -> ^(NODE_ADD_COLUMN_FAMILY keyValuePairExpr)
-    ;
-
-updateKeyspace
-    : UPDATE KEYSPACE keyValuePairExpr
-        -> ^(NODE_UPDATE_KEYSPACE keyValuePairExpr)
-    ;
-
-updateColumnFamily
-    : UPDATE COLUMN FAMILY keyValuePairExpr
-        -> ^(NODE_UPDATE_COLUMN_FAMILY keyValuePairExpr)
-    ;
-
-delKeyspace
-    : DROP KEYSPACE keyspace 
-        -> ^(NODE_DEL_KEYSPACE keyspace)
-    ;
-
-delColumnFamily
-    : DROP COLUMN FAMILY columnFamily 
-        -> ^(NODE_DEL_COLUMN_FAMILY columnFamily)
-    ;
-
-dropIndex
-    : DROP INDEX ON columnFamily '.' columnName
-        -> ^(NODE_DROP_INDEX columnFamily columnName)
-    ;
-
-showVersion
-    : SHOW API_VERSION
-        -> ^(NODE_SHOW_VERSION)
-    ;
-
-showKeyspaces
-    : SHOW KEYSPACES 
-        -> ^(NODE_SHOW_KEYSPACES)
-    ;
-
-showSchema
-    : SHOW SCHEMA (keyspace)?
-        -> ^(NODE_SHOW_SCHEMA (keyspace)?)
-    ;
-
-describeTable
-    : DESCRIBE (keyspace)?
-        -> ^(NODE_DESCRIBE (keyspace)?)
-    ;
-    
-describeCluster
-    : DESCRIBE 'CLUSTER'
-        -> ^(NODE_DESCRIBE_CLUSTER)
-    ;
-
-useKeyspace
-    : USE keyspace ( username )? ( password )? 
-        -> ^(NODE_USE_TABLE keyspace ( username )? ( password )?)
-    ;
-    
-traceNextQuery
-    : TRACE NEXT QUERY
-        -> ^(NODE_TRACE_NEXT_QUERY)
-    ;
-
-setTraceProbability
-    : SET TRACE PROBABILITY tracingProbability
-        -> ^(NODE_SET_TRACE_PROBABILITY tracingProbability)
-    ;
-
-keyValuePairExpr
-    : entityName ( (AND | WITH) keyValuePair )*
-        -> ^(NODE_NEW_KEYSPACE_ACCESS entityName ( keyValuePair )* )
-    ;
-            
-keyValuePair 
-    : attr_name '=' attrValue 
-        -> attr_name attrValue
-    ;
-
-attrValue
-    : arrayConstruct
-    | hashConstruct
-    | attrValueString
-    | attrValueInt
-    | attrValueDouble
-    ;
-
-
-arrayConstruct 
-    : '[' (hashConstruct ','?)* ']'
-        -> ^(ARRAY (hashConstruct)*)
-    ; 
-
-hashConstruct 
-    : '{' hashElementPair (',' hashElementPair)* '}'
-        -> ^(HASH (hashElementPair)+)
-    ;
-
-hashElementPair
-    : rowKey ':' rowValue
-        -> ^(PAIR rowKey rowValue)
-    ;
-
-columnFamilyExpr
-    : columnFamily '[' rowKey ']' 
-        ( '[' column=columnOrSuperColumn ']' 
-            ('[' super_column=columnOrSuperColumn ']')? 
-        )?
-      -> ^(NODE_COLUMN_ACCESS columnFamily rowKey ($column ($super_column)? )?)
-    ;
-
-keyRangeExpr
-    :    '[' ( startKey=entityName? ':' endKey=entityName? )? ']'
-      -> ^(NODE_KEY_RANGE $startKey? $endKey?)
-    ;
-
-rowLimitExpr
-    : 'LIMIT' limit=IntegerPositiveLiteral
-        -> ^(NODE_LIMIT $limit)
-    ;
-
-columnLimitExpr
-    : 'COLUMNS' columns=IntegerPositiveLiteral reversedExpr?
-        -> ^(NODE_COLUMNS $columns reversedExpr?)
-    ;
-
-reversedExpr
-    : 'REVERSED'
-        -> ^(NODE_REVERSED)
-    ;
-
-columnName
-	: entityName
-	;
-
-attr_name
-    : Identifier
-    ;
-
-attrValueString
-    : (Identifier | StringLiteral)
-    ;
-      
-attrValueInt
-    : IntegerPositiveLiteral
-  | IntegerNegativeLiteral
-    ;
-
-attrValueDouble
-    : DoubleLiteral
-    ;
-  
-keyspace
-	: entityName
-	;
-
-replica_placement_strategy
-    : StringLiteral
-    ;
-
-keyspaceNewName
-	: entityName
-	;
-
-comparator
-    : StringLiteral
-    ;
-      
-command : Identifier
-    ;
-
-newColumnFamily
-	: entityName
-	;
-
-username: Identifier
-    ;
-
-password: StringLiteral
-    ;
-
-columnFamily
-  : entityName
-  ;
-
-entityName
-  : (Identifier | StringLiteral | IntegerPositiveLiteral | IntegerNegativeLiteral)
-  ;
-
-rowKey	
-    :  (Identifier | StringLiteral | IntegerPositiveLiteral | IntegerNegativeLiteral | functionCall)
-    ;
-
-rowValue  
-    :  (Identifier | StringLiteral | IntegerPositiveLiteral | IntegerNegativeLiteral | functionCall | hashConstruct)
-    ;
-
-value   
-    : (Identifier | IntegerPositiveLiteral | IntegerNegativeLiteral | StringLiteral | functionCall)
-    ;
-
-functionCall 
-    : functionName=Identifier '(' functionArgument? ')'
-        -> ^(FUNCTION_CALL $functionName functionArgument?)
-    ;
-
-functionArgument 
-    : Identifier | StringLiteral | IntegerPositiveLiteral | IntegerNegativeLiteral
-    ;
-
-columnOrSuperColumn
-    : (Identifier | IntegerPositiveLiteral | IntegerNegativeLiteral | StringLiteral | functionCall)
-    ;
-
-host    
-    : host_name
-        -> ^(NODE_ID_LIST host_name)
-    ;
-
-host_name
-    : Identifier ('.' Identifier)*
-    ;
-    
-ip_address
-    : IP_ADDRESS 
-        -> ^(NODE_ID_LIST IP_ADDRESS)
-    ;
-
-port    
-    : IntegerPositiveLiteral
-    ;
-
-incrementValue
-    : IntegerPositiveLiteral
-    | IntegerNegativeLiteral
-    ;
-
-traceSessionId
-    : Identifier
-    ;
-
-tracingProbability
-    : DoubleLiteral
-    ;
-
-//
-// Lexer Section
-//
-
-//
-// Keywords (in alphabetical order for convenience)
-//
-// CLI is case-insensitive with respect to these keywords.
-// However, they MUST be listed in upper case here.
-//
-CONFIG:      'CONFIG';
-CONNECT:     'CONNECT';
-COUNT:       'COUNT';
-DEL:         'DEL';
-DESCRIBE:    'DESCRIBE';
-USE:         'USE';
-TRACE:       'TRACE';
-PROBABILITY: 'PROBABILITY';
-NEXT:        'NEXT';
-QUERY:       'QUERY';
-GET:         'GET';
-HELP:        'HELP';
-EXIT:        'EXIT';
-FILE:        'FILE';
-QUIT:        'QUIT';
-SET:         'SET';
-INCR:        'INCR';
-DECR:        'DECR';
-SHOW:        'SHOW';
-KEYSPACE:    'KEYSPACE';
-KEYSPACES:   'KEYSPACES';
-API_VERSION: 'API VERSION';
-CREATE:      'CREATE';
-DROP:        'DROP';
-COLUMN:      'COLUMN';
-FAMILY:      'FAMILY';
-WITH:        'WITH';
-BY:          'BY';
-AND:         'AND';
-UPDATE:      'UPDATE';
-LIST:        'LIST';
-LIMIT:       'LIMIT';
-TRUNCATE:    'TRUNCATE';
-ASSUME:      'ASSUME';
-TTL:         'TTL';
-CONSISTENCYLEVEL:   'CONSISTENCYLEVEL';
-INDEX:       'INDEX';
-ON:          'ON';
-SCHEMA:      'SCHEMA';
-
-IP_ADDRESS 
-    : IntegerPositiveLiteral '.' IntegerPositiveLiteral '.' IntegerPositiveLiteral '.' IntegerPositiveLiteral
-    ;
-
-// private syntactic rules
-fragment
-Letter
-    : 'a'..'z' 
-    | 'A'..'Z'
-    ;
-
-fragment
-Digit
-    : '0'..'9'
-    ;
-
-fragment
-Alnum
-    : Letter
-    | Digit
-    ;
-
-// syntactic Elements
-IntegerPositiveLiteral
-   : Digit+
-   ;
-
-IntegerNegativeLiteral
-   : '-' Digit+
-   ;
-   
-DoubleLiteral
-   : Digit+ ('.' Digit+)?
-   | ' ' '.' Digit+
-   ;
-
-Identifier
-    : (Letter | Alnum) (Alnum | '_' | '-' )*
-    ;
-
-// literals
-StringLiteral
-    : '\'' SingleStringCharacter* '\''
-    ;
-
-fragment SingleStringCharacter
-    : ~('\'' | '\\')
-    | '\\' EscapeSequence
-    ;
-
-fragment EscapeSequence
-    : CharacterEscapeSequence
-    | '0'
-    | HexEscapeSequence
-    | UnicodeEscapeSequence
-    ;
-
-fragment CharacterEscapeSequence
-    : SingleEscapeCharacter
-    | NonEscapeCharacter
-    ;
-
-fragment NonEscapeCharacter
-    : ~(EscapeCharacter)
-    ;
-
-fragment SingleEscapeCharacter
-    : '\'' | '"' | '\\' | 'b' | 'f' | 'n' | 'r' | 't' | 'v'
-    ;
-
-fragment EscapeCharacter
-    : SingleEscapeCharacter
-    | DecimalDigit
-    | 'x'
-    | 'u'
-    ;
-
-fragment HexEscapeSequence
-    : 'x' HexDigit HexDigit
-    ;
-
-fragment UnicodeEscapeSequence
-    : 'u' HexDigit HexDigit HexDigit HexDigit
-    ;
-
-fragment HexDigit
-    : DecimalDigit | ('a'..'f') | ('A'..'F')
-    ;
-
-fragment DecimalDigit
-    : ('0'..'9')
-    ;
-
-//
-// syntactic elements
-//
-
-SEMICOLON
-    : ';'
-    ;
-
-WS
-    :  (' '|'\r'|'\t'|'\n') {$channel=HIDDEN;}  // whitepace
-    ;
-
-COMMENT 
-    : '--' (~('\n'|'\r'))*                     { $channel=HIDDEN; }
-    | '/*' (options {greedy=false;} : .)* '*/' { $channel=HIDDEN; }
-    ;
diff --git a/src/java/org/apache/cassandra/cli/CliClient.java b/src/java/org/apache/cassandra/cli/CliClient.java
deleted file mode 100644
index ccdb0b2..0000000
--- a/src/java/org/apache/cassandra/cli/CliClient.java
+++ /dev/null
@@ -1,3235 +0,0 @@
-/*
- * 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.cli;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.PrintStream;
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.nio.ByteBuffer;
-import java.nio.charset.CharacterCodingException;
-import java.nio.charset.StandardCharsets;
-import java.util.*;
-
-import com.google.common.base.Predicate;
-import com.google.common.collect.Collections2;
-import com.google.common.collect.Iterables;
-
-import org.apache.commons.lang3.StringUtils;
-
-import org.antlr.runtime.tree.Tree;
-import org.apache.cassandra.auth.IAuthenticator;
-import org.apache.cassandra.exceptions.RequestValidationException;
-import org.apache.cassandra.cql3.statements.CFPropDefs;
-import org.apache.cassandra.db.ColumnFamilyStoreMBean;
-import org.apache.cassandra.db.compaction.CompactionManagerMBean;
-import org.apache.cassandra.db.compaction.OperationType;
-import org.apache.cassandra.db.marshal.*;
-import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.locator.SimpleSnitch;
-import org.apache.cassandra.serializers.MarshalException;
-import org.apache.cassandra.thrift.*;
-import org.apache.cassandra.tools.NodeProbe;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.JVMStabilityInspector;
-import org.apache.cassandra.utils.UUIDGen;
-import org.apache.thrift.TBaseHelper;
-import org.apache.thrift.TException;
-import org.codehaus.jackson.*;
-import org.yaml.snakeyaml.Yaml;
-import org.yaml.snakeyaml.constructor.Constructor;
-
-// Cli Client Side Library
-public class CliClient
-{
-    /**
-     * Available value conversion functions
-     * Used by convertValueByFunction(Tree functionCall) method
-     */
-    public enum Function
-    {
-        BYTES         (BytesType.instance),
-        INTEGER       (IntegerType.instance),
-        LONG          (LongType.instance),
-        INT           (Int32Type.instance),
-        LEXICALUUID   (LexicalUUIDType.instance),
-        TIMEUUID      (TimeUUIDType.instance),
-        UTF8          (UTF8Type.instance),
-        ASCII         (AsciiType.instance),
-        DOUBLE        (DoubleType.instance),
-        COUNTERCOLUMN (CounterColumnType.instance);
-
-        private AbstractType<?> validator;
-
-        Function(AbstractType<?> validator)
-        {
-            this.validator = validator;
-        }
-
-        public AbstractType<?> getValidator()
-        {
-            return this.validator;
-        }
-
-        public static String getFunctionNames()
-        {
-            Function[] functions = Function.values();
-            StringBuilder functionNames = new StringBuilder();
-
-            for (int i = 0; i < functions.length; i++)
-            {
-                StringBuilder currentName = new StringBuilder(functions[i].name().toLowerCase());
-                functionNames.append(currentName.append(((i != functions.length-1) ? ", " : ".")));
-            }
-
-            return functionNames.toString();
-        }
-    }
-
-    /*
-     * the <i>add keyspace</i> command requires a list of arguments,
-     *  this enum defines which arguments are valid
-     */
-    private enum AddKeyspaceArgument {
-        PLACEMENT_STRATEGY,
-        STRATEGY_OPTIONS,
-        DURABLE_WRITES
-    }
-
-    /*
-        * the <i>add column family</i> command requires a list of arguments,
-        *  this enum defines which arguments are valid.
-        */
-    protected enum ColumnFamilyArgument
-    {
-        COLUMN_TYPE,
-        COMPARATOR,
-        SUBCOMPARATOR,
-        COMMENT,
-        READ_REPAIR_CHANCE,
-        DCLOCAL_READ_REPAIR_CHANCE,
-        GC_GRACE,
-        COLUMN_METADATA,
-        MEMTABLE_OPERATIONS,
-        MEMTABLE_THROUGHPUT,
-        DEFAULT_VALIDATION_CLASS,
-        MIN_COMPACTION_THRESHOLD,
-        MAX_COMPACTION_THRESHOLD,
-        REPLICATE_ON_WRITE,
-        KEY_VALIDATION_CLASS,
-        COMPACTION_STRATEGY,
-        COMPACTION_STRATEGY_OPTIONS,
-        COMPRESSION_OPTIONS,
-        BLOOM_FILTER_FP_CHANCE,
-        INDEX_INTERVAL,
-        MEMTABLE_FLUSH_PERIOD_IN_MS,
-        CACHING,
-        CELLS_PER_ROW_TO_CACHE,
-        DEFAULT_TIME_TO_LIVE,
-        SPECULATIVE_RETRY,
-        POPULATE_IO_CACHE_ON_FLUSH,
-    }
-
-    private static final String DEFAULT_PLACEMENT_STRATEGY = "org.apache.cassandra.locator.NetworkTopologyStrategy";
-    private static final String NEWLINE = System.getProperty("line.separator");
-    private static final String TAB = "  ";
-
-    private final Cassandra.Client thriftClient;
-    private final CliSessionState sessionState;
-    private String keySpace = null;
-    private String username = null;
-    private final Map<String, KsDef> keyspacesMap = new HashMap<String, KsDef>();
-    private final Map<String, Map<String, CfDef>> cql3KeyspacesMap = new HashMap<String, Map<String, CfDef>>();
-    private final Map<String, AbstractType<?>> cfKeysComparators;
-    private ConsistencyLevel consistencyLevel = ConsistencyLevel.ONE;
-    private final CfAssumptions assumptions = new CfAssumptions();
-    private CliUserHelp help;
-    public CliClient(CliSessionState cliSessionState, Cassandra.Client thriftClient)
-    {
-        this.sessionState = cliSessionState;
-        this.thriftClient = thriftClient;
-        this.cfKeysComparators = new HashMap<String, AbstractType<?>>();
-        assumptions.readAssumptions();
-    }
-
-    private CliUserHelp getHelp()
-    {
-        if (help == null)
-            help = loadHelp();
-        return help;
-    }
-
-    private CliUserHelp loadHelp()
-    {
-        final InputStream is = CliClient.class.getClassLoader().getResourceAsStream("org/apache/cassandra/cli/CliHelp.yaml");
-        assert is != null;
-
-        try
-        {
-            final Constructor constructor = new Constructor(CliUserHelp.class);
-            final Yaml yaml = new Yaml(constructor);
-            return (CliUserHelp) yaml.load(is);
-        }
-        finally
-        {
-            FileUtils.closeQuietly(is);
-        }
-    }
-
-    public void printBanner()
-    {
-        sessionState.out.println("Welcome to Cassandra CLI version " + FBUtilities.getReleaseVersionString() + "\n");
-
-        sessionState.out.println("The CLI is deprecated and will be removed in Cassandra 2.2.  Consider migrating to cqlsh.");
-        sessionState.out.println("CQL is fully backwards compatible with Thrift data; see http://www.datastax.com/dev/blog/thrift-to-cql3\n");
-
-        sessionState.out.println(getHelp().banner);
-    }
-
-    private void printCQL3TablesWarning(String cmd)
-    {
-        sessionState.err.println("\nWARNING: CQL3 tables are intentionally omitted from '" + cmd + "' output.");
-        sessionState.err.println("See https://issues.apache.org/jira/browse/CASSANDRA-4377 for details.\n");
-    }
-
-    // Execute a CLI Statement
-    public void executeCLIStatement(String statement) throws CharacterCodingException, TException, TimedOutException, NotFoundException, NoSuchFieldException, InvalidRequestException, UnavailableException, InstantiationException, IllegalAccessException
-    {
-        Tree tree = CliCompiler.compileQuery(statement);
-        try
-        {
-            switch (tree.getType())
-            {
-                case CliParser.NODE_EXIT:
-                    cleanupAndExit();
-                    break;
-                case CliParser.NODE_THRIFT_GET:
-                    executeGet(tree);
-                    break;
-                case CliParser.NODE_THRIFT_GET_WITH_CONDITIONS:
-                    executeGetWithConditions(tree);
-                    break;
-                case CliParser.NODE_HELP:
-                    executeHelp(tree);
-                    break;
-                case CliParser.NODE_THRIFT_SET:
-                    executeSet(tree);
-                    break;
-                case CliParser.NODE_THRIFT_DEL:
-                    executeDelete(tree);
-                    break;
-                case CliParser.NODE_THRIFT_COUNT:
-                    executeCount(tree);
-                    break;
-                case CliParser.NODE_ADD_KEYSPACE:
-                    executeAddKeySpace(tree.getChild(0));
-                    break;
-                case CliParser.NODE_ADD_COLUMN_FAMILY:
-                    executeAddColumnFamily(tree.getChild(0));
-                    break;
-                case CliParser.NODE_UPDATE_KEYSPACE:
-                    executeUpdateKeySpace(tree.getChild(0));
-                    break;
-                case CliParser.NODE_UPDATE_COLUMN_FAMILY:
-                    executeUpdateColumnFamily(tree.getChild(0));
-                    break;
-                case CliParser.NODE_DEL_COLUMN_FAMILY:
-                    executeDelColumnFamily(tree);
-                    break;
-                case CliParser.NODE_DEL_KEYSPACE:
-                    executeDelKeySpace(tree);
-                    break;
-                case CliParser.NODE_SHOW_CLUSTER_NAME:
-                    executeShowClusterName();
-                    break;
-                case CliParser.NODE_SHOW_VERSION:
-                    executeShowVersion();
-                    break;
-                case CliParser.NODE_SHOW_KEYSPACES:
-                    executeShowKeySpaces();
-                    break;
-                case CliParser.NODE_SHOW_SCHEMA:
-                    executeShowSchema(tree);
-                    break;
-                case CliParser.NODE_DESCRIBE:
-                    executeDescribe(tree);
-                    break;
-                case CliParser.NODE_DESCRIBE_CLUSTER:
-                    executeDescribeCluster();
-                    break;
-                case CliParser.NODE_USE_TABLE:
-                    executeUseKeySpace(tree);
-                    break;
-                case CliParser.NODE_TRACE_NEXT_QUERY:
-                    executeTraceNextQuery();
-                    break;
-                case CliParser.NODE_CONNECT:
-                    executeConnect(tree);
-                    break;
-                case CliParser.NODE_LIST:
-                    executeList(tree);
-                    break;
-                case CliParser.NODE_TRUNCATE:
-                    executeTruncate(tree.getChild(0).getText());
-                    break;
-                case CliParser.NODE_ASSUME:
-                    executeAssumeStatement(tree);
-                    break;
-                case CliParser.NODE_CONSISTENCY_LEVEL:
-                    executeConsistencyLevelStatement(tree);
-                    break;
-                case CliParser.NODE_THRIFT_INCR:
-                    executeIncr(tree, 1L);
-                    break;
-                case CliParser.NODE_THRIFT_DECR:
-                    executeIncr(tree, -1L);
-                    break;
-                case CliParser.NODE_DROP_INDEX:
-                    executeDropIndex(tree);
-                    break;
-
-                case CliParser.NODE_NO_OP:
-                    // comment lines come here; they are treated as no ops.
-                    break;
-                default:
-                    sessionState.err.println("Invalid Statement (Type: " + tree.getType() + ")");
-                    if (sessionState.batch)
-                        System.exit(2);
-                    break;
-            }
-        }
-        catch (SchemaDisagreementException e)
-        {
-            throw new RuntimeException("schema does not match across nodes, (try again later).", e);
-        }
-    }
-
-    private void cleanupAndExit()
-    {
-        CliMain.disconnect();
-        assumptions.writeAssumptions();
-        System.exit(0);
-    }
-
-    public KsDef getKSMetaData(String keyspace)
-            throws NotFoundException, InvalidRequestException, TException
-    {
-        // Lazily lookup keyspace meta-data.
-        if (!(keyspacesMap.containsKey(keyspace)))
-        {
-            KsDef ksDef = thriftClient.describe_keyspace(keyspace);
-            keyspacesMap.put(keyspace, ksDef);
-            cql3KeyspacesMap.put(keyspace, loadCql3Defs(thriftClient, ksDef));
-            assumptions.replayAssumptions(keyspace);
-        }
-
-        return keyspacesMap.get(keyspace);
-    }
-
-    public static Map<String, CfDef> loadCql3Defs(Cassandra.Client thriftClient, KsDef thriftKs)
-    {
-        try
-        {
-            return loadCql3DefsUnchecked(thriftClient, thriftKs);
-        }
-        catch (Exception e)
-        {
-            throw new RuntimeException(e);
-        }
-    }
-
-    public static Map<String, CfDef> loadCql3DefsUnchecked(Cassandra.Client thriftClient, KsDef thriftKs) throws Exception
-    {
-        Map<String, CfDef> cql3Defs = new HashMap<String, CfDef>();
-
-        String query = "SELECT columnfamily_name, comparator, default_validator, key_validator FROM system.schema_columnfamilies WHERE keyspace_name='%s'";
-        String formatted = String.format(query, thriftKs.name);
-        CqlResult result = thriftClient.execute_cql3_query(ByteBufferUtil.bytes(formatted),
-                                                           Compression.NONE,
-                                                           ConsistencyLevel.ONE);
-        outer:
-        for (CqlRow row : result.rows)
-        {
-            Column rawName = row.columns.get(0);
-            assert ByteBufferUtil.string(ByteBuffer.wrap(rawName.getName())).equals("columnfamily_name");
-            String name = ByteBufferUtil.string(ByteBuffer.wrap(rawName.getValue()));
-
-            Column rawComparator = row.columns.get(1);
-            assert ByteBufferUtil.string(ByteBuffer.wrap(rawComparator.getName())).equals("comparator");
-            String comparator = ByteBufferUtil.string(ByteBuffer.wrap(rawComparator.getValue()));
-
-            Column rawValidator = row.columns.get(2);
-            assert ByteBufferUtil.string(ByteBuffer.wrap(rawValidator.getName())).equals("default_validator");
-            String validator = ByteBufferUtil.string(ByteBuffer.wrap(rawValidator.getValue()));
-
-            Column rawKeyValidator = row.columns.get(3);
-            assert ByteBufferUtil.string(ByteBuffer.wrap(rawKeyValidator.getName())).equals("key_validator");
-            String keyValidator = ByteBufferUtil.string(ByteBuffer.wrap(rawKeyValidator.getValue()));
-
-            for (CfDef cf_def : thriftKs.cf_defs)
-            {
-                if (cf_def.name.equals(name))
-                    continue outer;
-            }
-
-            CfDef thriftDef = new CfDef(thriftKs.name, name)
-                              .setComparator_type(comparator)
-                              .setDefault_validation_class(validator)
-                              .setKey_validation_class(keyValidator)
-                              .setColumn_metadata(Collections.<ColumnDef>emptyList());
-            cql3Defs.put(name, thriftDef);
-        }
-
-        return cql3Defs;
-    }
-
-    private void executeHelp(Tree tree)
-    {
-        if (tree.getChildCount() > 0)
-        {
-            String token = tree.getChild(0).getText();
-            for (CliCommandHelp ch : getHelp().commands)
-            {
-                if (token.equals(ch.name))
-                {
-                    sessionState.out.println(ch.help);
-                    break;
-                }
-            }
-        }
-        else
-        {
-            sessionState.out.println(getHelp().help);
-        }
-    }
-
-    private void executeCount(Tree statement)
-            throws TException, InvalidRequestException, UnavailableException, TimedOutException
-    {
-        if (!CliMain.isConnected() || !hasKeySpace())
-            return;
-
-        Tree columnFamilySpec = statement.getChild(0);
-
-        String columnFamily = CliCompiler.getColumnFamily(columnFamilySpec, currentCfDefs());
-        int columnSpecCnt = CliCompiler.numColumnSpecifiers(columnFamilySpec);
-
-        ColumnParent colParent = new ColumnParent(columnFamily).setSuper_column((ByteBuffer) null);
-
-        if (columnSpecCnt != 0)
-        {
-            Tree columnTree = columnFamilySpec.getChild(2);
-
-            byte[] superColumn = (columnTree.getType() == CliParser.FUNCTION_CALL)
-                                  ? convertValueByFunction(columnTree, null, null).array()
-                                  : columnNameAsByteArray(CliCompiler.getColumn(columnFamilySpec, 0), columnFamily);
-
-            colParent = new ColumnParent(columnFamily).setSuper_column(superColumn);
-        }
-
-        SliceRange range = new SliceRange(ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, false, Integer.MAX_VALUE);
-        SlicePredicate predicate = new SlicePredicate().setColumn_names(null).setSlice_range(range);
-
-        int count = thriftClient.get_count(getKeyAsBytes(columnFamily, columnFamilySpec.getChild(1)), colParent, predicate, consistencyLevel);
-        sessionState.out.printf("%d cells%n", count);
-    }
-
-    private Iterable<CfDef> currentCfDefs()
-    {
-        return Iterables.concat(keyspacesMap.get(keySpace).cf_defs, cql3KeyspacesMap.get(keySpace).values());
-    }
-
-    private void executeDelete(Tree statement)
-            throws TException, InvalidRequestException, UnavailableException, TimedOutException
-    {
-        if (!CliMain.isConnected() || !hasKeySpace())
-            return;
-
-        Tree columnFamilySpec = statement.getChild(0);
-
-        String columnFamily = CliCompiler.getColumnFamily(columnFamilySpec, currentCfDefs());
-        CfDef cfDef = getCfDef(columnFamily);
-
-        ByteBuffer key = getKeyAsBytes(columnFamily, columnFamilySpec.getChild(1));
-        int columnSpecCnt = CliCompiler.numColumnSpecifiers(columnFamilySpec);
-
-        byte[] superColumnName = null;
-        byte[] columnName = null;
-
-        if ((columnSpecCnt < 0) || (columnSpecCnt > 2))
-        {
-            sessionState.out.println("Invalid row, super column, or column specification.");
-            return;
-        }
-
-        long startTime = System.nanoTime();
-
-        Tree columnTree = (columnSpecCnt >= 1)
-                           ? columnFamilySpec.getChild(2)
-                           : null;
-
-        Tree subColumnTree = (columnSpecCnt == 2)
-                              ? columnFamilySpec.getChild(3)
-                              : null;
-
-        if (columnSpecCnt == 1)
-        {
-            assert columnTree != null;
-
-            byte[] columnNameBytes = (columnTree.getType() == CliParser.FUNCTION_CALL)
-                                      ? convertValueByFunction(columnTree, null, null).array()
-                                      : columnNameAsByteArray(CliCompiler.getColumn(columnFamilySpec, 0), cfDef);
-
-            boolean isSuper = cfDef.column_type.equals("Super");
-            if (isSuper)
-                superColumnName = columnNameBytes;
-            else
-                columnName = columnNameBytes;
-        }
-        else if (columnSpecCnt == 2)
-        {
-            assert columnTree != null;
-            assert subColumnTree != null;
-
-            // keyspace.cf['key']['column']['column']
-            superColumnName = (columnTree.getType() == CliParser.FUNCTION_CALL)
-                                      ? convertValueByFunction(columnTree, null, null).array()
-                                      : columnNameAsByteArray(CliCompiler.getColumn(columnFamilySpec, 0), cfDef);
-
-            columnName = (subColumnTree.getType() == CliParser.FUNCTION_CALL)
-                                         ? convertValueByFunction(subColumnTree, null, null).array()
-                                         : subColumnNameAsByteArray(CliCompiler.getColumn(columnFamilySpec, 1), cfDef);
-        }
-
-        ColumnPath path = new ColumnPath(columnFamily);
-        if (superColumnName != null)
-            path.setSuper_column(superColumnName);
-
-        if (columnName != null)
-            path.setColumn(columnName);
-
-        if (isCounterCF(cfDef))
-        {
-            thriftClient.remove_counter(key, path, consistencyLevel);
-        }
-        else
-        {
-            thriftClient.remove(key, path, FBUtilities.timestampMicros(), consistencyLevel);
-        }
-        sessionState.out.println(String.format("%s removed.", (columnSpecCnt == 0) ? "row" : "cell"));
-        elapsedTime(startTime);
-    }
-
-    private void doSlice(String keyspace, ByteBuffer key, String columnFamily, byte[] superColumnName, int limit)
-            throws InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException
-    {
-
-        long startTime = System.nanoTime();
-        ColumnParent parent = new ColumnParent(columnFamily);
-        if(superColumnName != null)
-            parent.setSuper_column(superColumnName);
-
-        SliceRange range = new SliceRange(ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, false, limit);
-        SlicePredicate predicate = new SlicePredicate().setColumn_names(null).setSlice_range(range);
-
-        CfDef cfDef = getCfDef(columnFamily);
-        boolean isSuperCF = cfDef.column_type.equals("Super");
-
-        List<ColumnOrSuperColumn> columns = thriftClient.get_slice(key, parent, predicate, consistencyLevel);
-        AbstractType<?> validator;
-
-        // Print out super columns or columns.
-        for (ColumnOrSuperColumn cosc : columns)
-        {
-            if (cosc.isSetSuper_column())
-            {
-                SuperColumn superColumn = cosc.super_column;
-
-                sessionState.out.printf("=> (super_column=%s,", formatColumnName(keyspace, columnFamily, superColumn.name));
-                for (Column col : superColumn.getColumns())
-                {
-                    validator = getValidatorForValue(cfDef, col.getName());
-                    sessionState.out.printf("%n     (name=%s, value=%s, timestamp=%d%s)", formatSubcolumnName(keyspace, columnFamily, col.name),
-                                                    validator.getString(col.value), col.timestamp,
-                                                    col.isSetTtl() ? String.format(", ttl=%d", col.getTtl()) : "");
-                }
-
-                sessionState.out.println(")");
-            }
-            else if (cosc.isSetColumn())
-            {
-                Column column = cosc.column;
-                validator = getValidatorForValue(cfDef, column.getName());
-
-                String formattedName = isSuperCF
-                                       ? formatSubcolumnName(keyspace, columnFamily, column.name)
-                                       : formatColumnName(keyspace, columnFamily, column.name);
-
-                sessionState.out.printf("=> (name=%s, value=%s, timestamp=%d%s)%n",
-                                        formattedName,
-                                        validator.getString(column.value),
-                                        column.timestamp,
-                                        column.isSetTtl() ? String.format(", ttl=%d", column.getTtl()) : "");
-            }
-            else if (cosc.isSetCounter_super_column())
-            {
-                CounterSuperColumn superColumn = cosc.counter_super_column;
-
-                sessionState.out.printf("=> (super_column=%s,", formatColumnName(keyspace, columnFamily, superColumn.name));
-                for (CounterColumn col : superColumn.getColumns())
-                {
-                    sessionState.out.printf("%n     (counter=%s, value=%s)", formatSubcolumnName(keyspace, columnFamily, col.name), col.value);
-                }
-                sessionState.out.println(")");
-            }
-            else // cosc.isSetCounter_column()
-            {
-                CounterColumn column = cosc.counter_column;
-                String formattedName = isSuperCF
-                                       ? formatSubcolumnName(keyspace, columnFamily, column.name)
-                                       : formatColumnName(keyspace, columnFamily, column.name);
-
-                sessionState.out.printf("=> (counter=%s, value=%s)%n", formattedName, column.value);
-            }
-        }
-
-        sessionState.out.println("Returned " + columns.size() + " results.");
-        elapsedTime(startTime);
-    }
-
-    private AbstractType<?> getFormatType(String compareWith)
-    {
-        Function function;
-
-        try
-        {
-            function = Function.valueOf(compareWith.toUpperCase());
-        }
-        catch (IllegalArgumentException e)
-        {
-            try
-            {
-                return TypeParser.parse(compareWith);
-            }
-            catch (RequestValidationException ce)
-            {
-                String message = String.format("Unknown comparator '%s'. Available functions: %s", compareWith, Function.getFunctionNames());
-                throw new RuntimeException(message, e);
-            }
-        }
-
-        return function.getValidator();
-    }
-
-    // Execute GET statement
-    private void executeGet(Tree statement)
-            throws TException, NotFoundException, InvalidRequestException, UnavailableException, TimedOutException
-    {
-        if (!CliMain.isConnected() || !hasKeySpace())
-            return;
-        long startTime = System.nanoTime();
-        Tree columnFamilySpec = statement.getChild(0);
-        String columnFamily = CliCompiler.getColumnFamily(columnFamilySpec, currentCfDefs());
-        ByteBuffer key = getKeyAsBytes(columnFamily, columnFamilySpec.getChild(1));
-        int columnSpecCnt = CliCompiler.numColumnSpecifiers(columnFamilySpec);
-        CfDef cfDef = getCfDef(columnFamily);
-        boolean isSuper = cfDef.column_type.equals("Super");
-
-        byte[] superColumnName = null;
-        ByteBuffer columnName;
-
-        Tree typeTree = null;
-        Tree limitTree = null;
-
-        int limit = 1000000;
-
-        if (statement.getChildCount() >= 2)
-        {
-            if (statement.getChild(1).getType() == CliParser.CONVERT_TO_TYPE)
-            {
-                typeTree = statement.getChild(1).getChild(0);
-                if (statement.getChildCount() == 3)
-                    limitTree = statement.getChild(2).getChild(0);
-            }
-            else
-            {
-                limitTree = statement.getChild(1).getChild(0);
-            }
-        }
-
-        if (limitTree != null)
-        {
-            limit = Integer.parseInt(limitTree.getText());
-
-            if (limit == 0)
-            {
-                throw new IllegalArgumentException("LIMIT should be greater than zero.");
-            }
-        }
-
-        // keyspace.cf['key'] -- row slice
-        if (columnSpecCnt == 0)
-        {
-            doSlice(keySpace, key, columnFamily, superColumnName, limit);
-            return;
-        }
-        // keyspace.cf['key']['column'] -- slice of a super, or get of a standard
-        else if (columnSpecCnt == 1)
-        {
-            columnName = getColumnName(columnFamily, columnFamilySpec.getChild(2));
-
-            if (isSuper)
-            {
-                superColumnName = columnName.array();
-                doSlice(keySpace, key, columnFamily, superColumnName, limit);
-                return;
-            }
-        }
-        // keyspace.cf['key']['column']['column'] -- get of a sub-column
-        else if (columnSpecCnt == 2)
-        {
-            superColumnName = getColumnName(columnFamily, columnFamilySpec.getChild(2)).array();
-            columnName = getSubColumnName(columnFamily, columnFamilySpec.getChild(3));
-        }
-        // The parser groks an arbitrary number of these so it is possible to get here.
-        else
-        {
-            sessionState.out.println("Invalid row, super column, or column specification.");
-            return;
-        }
-
-        AbstractType<?> validator = getValidatorForValue(cfDef, TBaseHelper.byteBufferToByteArray(columnName));
-
-        // Perform a get()
-        ColumnPath path = new ColumnPath(columnFamily);
-        if(superColumnName != null) path.setSuper_column(superColumnName);
-        path.setColumn(columnName);
-
-        if (isCounterCF(cfDef))
-        {
-            doGetCounter(key, path);
-            elapsedTime(startTime);
-            return;
-        }
-
-        Column column;
-        try
-        {
-            column = thriftClient.get(key, path, consistencyLevel).column;
-        }
-        catch (NotFoundException e)
-        {
-            sessionState.out.println("Value was not found");
-            elapsedTime(startTime);
-            return;
-        }
-
-        byte[] columnValue = column.getValue();
-        String valueAsString;
-
-        // we have ^(CONVERT_TO_TYPE <type>) inside of GET statement
-        // which means that we should try to represent byte[] value according
-        // to specified type
-        if (typeTree != null)
-        {
-            // .getText() will give us <type>
-            String typeName = CliUtils.unescapeSQLString(typeTree.getText());
-            // building AbstractType from <type>
-            AbstractType<?> valueValidator = getFormatType(typeName);
-
-            // setting value for output
-            valueAsString = valueValidator.getString(ByteBuffer.wrap(columnValue));
-            // updating column value validator class
-            updateColumnMetaData(cfDef, columnName, valueValidator.toString());
-        }
-        else
-        {
-            valueAsString = (validator == null) ? new String(columnValue, StandardCharsets.UTF_8) : validator.getString(ByteBuffer.wrap(columnValue));
-        }
-
-        String formattedColumnName = isSuper
-                                     ? formatSubcolumnName(keySpace, columnFamily, column.name)
-                                     : formatColumnName(keySpace, columnFamily, column.name);
-
-        // print results
-        sessionState.out.printf("=> (name=%s, value=%s, timestamp=%d%s)%n",
-                                formattedColumnName,
-                                valueAsString,
-                                column.timestamp,
-                                column.isSetTtl() ? String.format(", ttl=%d", column.getTtl()) : "");
-        elapsedTime(startTime);
-    }
-
-    private void doGetCounter(ByteBuffer key, ColumnPath path)
-            throws TException, NotFoundException, InvalidRequestException, UnavailableException, TimedOutException
-    {
-        boolean isSuper = path.super_column != null;
-
-        CounterColumn column;
-        try
-        {
-            column = thriftClient.get(key, path, consistencyLevel).counter_column;
-        }
-        catch (NotFoundException e)
-        {
-            sessionState.out.println("Value was not found");
-            return;
-        }
-
-        String formattedColumnName = isSuper
-                                     ? formatSubcolumnName(keySpace, path.column_family, column.name)
-                                     : formatColumnName(keySpace, path.column_family, column.name);
-
-        // print results
-        sessionState.out.printf("=> (counter=%s, value=%d)%n",
-                                formattedColumnName,
-                                column.value);
-    }
-
-    /**
-     * Process get operation with conditions (using Thrift get_indexed_slices method)
-     * @param statement - tree representation of the current statement
-     * Format: ^(NODE_THRIFT_GET_WITH_CONDITIONS cf ^(CONDITIONS ^(CONDITION >= column1 value1) ...) ^(NODE_LIMIT int)*)
-     */
-    private void executeGetWithConditions(Tree statement)
-    {
-        if (!CliMain.isConnected() || !hasKeySpace())
-            return;
-
-        long startTime = System.nanoTime();
-
-        IndexClause clause = new IndexClause();
-        String columnFamily = CliCompiler.getColumnFamily(statement, currentCfDefs());
-        // ^(CONDITIONS ^(CONDITION $column $value) ...)
-        Tree conditions = statement.getChild(1);
-
-        // fetching column family definition
-        CfDef columnFamilyDef = getCfDef(columnFamily);
-
-        // fetching all columns
-        SlicePredicate predicate = new SlicePredicate();
-        SliceRange sliceRange = new SliceRange();
-        sliceRange.setStart(new byte[0]).setFinish(new byte[0]);
-        predicate.setSlice_range(sliceRange);
-
-        for (int i = 0; i < conditions.getChildCount(); i++)
-        {
-            // ^(CONDITION operator $column $value)
-            Tree condition = conditions.getChild(i);
-
-            // =, >, >=, <, <=
-            String operator = condition.getChild(0).getText();
-            String columnNameString  = CliUtils.unescapeSQLString(condition.getChild(1).getText());
-            // it could be a basic string or function call
-            Tree valueTree = condition.getChild(2);
-
-            try
-            {
-                ByteBuffer value;
-                ByteBuffer columnName = columnNameAsBytes(columnNameString, columnFamily);
-
-                if (valueTree.getType() == CliParser.FUNCTION_CALL)
-                {
-                    value = convertValueByFunction(valueTree, columnFamilyDef, columnName);
-                }
-                else
-                {
-                    String valueString = CliUtils.unescapeSQLString(valueTree.getText());
-                    value = columnValueAsBytes(columnName, columnFamily, valueString);
-                }
-
-                // index operator from string
-                IndexOperator idxOperator = CliUtils.getIndexOperator(operator);
-                // adding new index expression into index clause
-                clause.addToExpressions(new IndexExpression(columnName, idxOperator, value));
-            }
-            catch (Exception e)
-            {
-                throw new RuntimeException(e);
-            }
-        }
-
-        List<KeySlice> slices;
-        clause.setStart_key(new byte[] {});
-
-        // when we have ^(NODE_LIMIT Integer)
-        if (statement.getChildCount() == 3)
-        {
-            Tree limitNode = statement.getChild(2);
-            int limitValue = Integer.parseInt(limitNode.getChild(0).getText());
-
-            if (limitValue == 0)
-            {
-                throw new IllegalArgumentException("LIMIT should be greater than zero.");
-            }
-
-            clause.setCount(limitValue);
-        }
-
-        try
-        {
-            ColumnParent parent = new ColumnParent(columnFamily);
-            slices = thriftClient.get_indexed_slices(parent, clause, predicate, consistencyLevel);
-            printSliceList(columnFamilyDef, slices);
-        }
-        catch (InvalidRequestException e)
-        {
-            throw new RuntimeException(e);
-        }
-        catch (Exception e)
-        {
-            throw new RuntimeException(e);
-        }
-        elapsedTime(startTime);
-    }
-
-    // Execute SET statement
-    private void executeSet(Tree statement)
-        throws TException, InvalidRequestException, UnavailableException, TimedOutException
-    {
-        if (!CliMain.isConnected() || !hasKeySpace())
-            return;
-
-        long startTime = System.nanoTime();
-        // ^(NODE_COLUMN_ACCESS <cf> <key> <column>)
-        Tree columnFamilySpec = statement.getChild(0);
-        Tree keyTree = columnFamilySpec.getChild(1); // could be a function or regular text
-
-        String columnFamily = CliCompiler.getColumnFamily(columnFamilySpec, currentCfDefs());
-        CfDef cfDef = getCfDef(columnFamily);
-        int columnSpecCnt = CliCompiler.numColumnSpecifiers(columnFamilySpec);
-        String value = CliUtils.unescapeSQLString(statement.getChild(1).getText());
-        Tree valueTree = statement.getChild(1);
-
-        byte[] superColumnName = null;
-        ByteBuffer columnName;
-
-        // keyspace.cf['key']
-        if (columnSpecCnt == 0)
-        {
-            sessionState.err.println("No cell name specified, (type 'help;' or '?' for help on syntax).");
-            return;
-        }
-        // keyspace.cf['key']['column'] = 'value'
-        else if (columnSpecCnt == 1)
-        {
-            // get the column name
-            if (cfDef.column_type.equals("Super"))
-            {
-                sessionState.out.println("Column family " + columnFamily + " may only contain SuperColumns");
-                return;
-            }
-            columnName = getColumnName(columnFamily, columnFamilySpec.getChild(2));
-        }
-        // keyspace.cf['key']['super_column']['column'] = 'value'
-        else
-        {
-            assert (columnSpecCnt == 2) : "serious parsing error (this is a bug).";
-
-            superColumnName = getColumnName(columnFamily, columnFamilySpec.getChild(2)).array();
-            columnName = getSubColumnName(columnFamily, columnFamilySpec.getChild(3));
-        }
-
-        ByteBuffer columnValueInBytes;
-
-        switch (valueTree.getType())
-        {
-        case CliParser.FUNCTION_CALL:
-            columnValueInBytes = convertValueByFunction(valueTree, cfDef, columnName, true);
-            break;
-        default:
-            columnValueInBytes = columnValueAsBytes(columnName, columnFamily, value);
-        }
-
-        ColumnParent parent = new ColumnParent(columnFamily);
-        if(superColumnName != null)
-            parent.setSuper_column(superColumnName);
-
-        Column columnToInsert = new Column(columnName).setValue(columnValueInBytes).setTimestamp(FBUtilities.timestampMicros());
-
-        // children count = 3 mean that we have ttl in arguments
-        if (statement.getChildCount() == 3)
-        {
-            String ttl = statement.getChild(2).getText();
-
-            try
-            {
-                columnToInsert.setTtl(Integer.parseInt(ttl));
-            }
-            catch (NumberFormatException e)
-            {
-                sessionState.err.println(String.format("TTL '%s' is invalid, should be a positive integer.", ttl));
-                return;
-            }
-            catch (Exception e)
-            {
-                throw new RuntimeException(e);
-            }
-        }
-
-        // do the insert
-        thriftClient.insert(getKeyAsBytes(columnFamily, keyTree), parent, columnToInsert, consistencyLevel);
-        sessionState.out.println("Value inserted.");
-        elapsedTime(startTime);
-    }
-
-    // Execute INCR statement
-    private void executeIncr(Tree statement, long multiplier)
-            throws TException, NotFoundException, InvalidRequestException, UnavailableException, TimedOutException
-    {
-        if (!CliMain.isConnected() || !hasKeySpace())
-            return;
-
-        Tree columnFamilySpec = statement.getChild(0);
-
-        String columnFamily = CliCompiler.getColumnFamily(columnFamilySpec, currentCfDefs());
-        ByteBuffer key = getKeyAsBytes(columnFamily, columnFamilySpec.getChild(1));
-        int columnSpecCnt = CliCompiler.numColumnSpecifiers(columnFamilySpec);
-
-        byte[] superColumnName = null;
-        ByteBuffer columnName;
-
-        // keyspace.cf['key']['column'] -- incr standard
-        if (columnSpecCnt == 1)
-        {
-            columnName = getColumnName(columnFamily, columnFamilySpec.getChild(2));
-        }
-        // keyspace.cf['key']['column']['column'] -- incr super
-        else if (columnSpecCnt == 2)
-        {
-            superColumnName = getColumnName(columnFamily, columnFamilySpec.getChild(2)).array();
-            columnName = getSubColumnName(columnFamily, columnFamilySpec.getChild(3));
-        }
-        // The parser groks an arbitrary number of these so it is possible to get here.
-        else
-        {
-            sessionState.out.println("Invalid row, super column, or column specification.");
-            return;
-        }
-
-        ColumnParent parent = new ColumnParent(columnFamily);
-        if(superColumnName != null)
-            parent.setSuper_column(superColumnName);
-
-        long value = 1L;
-
-        // children count = 3 mean that we have by in arguments
-        if (statement.getChildCount() == 2)
-        {
-            String byValue = statement.getChild(1).getText();
-
-            try
-            {
-                value = Long.parseLong(byValue);
-            }
-            catch (NumberFormatException e)
-            {
-                sessionState.err.println(String.format("'%s' is an invalid value, should be an integer.", byValue));
-                return;
-            }
-            catch (Exception e)
-            {
-                throw new RuntimeException(e);
-            }
-        }
-
-        CounterColumn columnToInsert = new CounterColumn(columnName, multiplier * value);
-
-        // do the insert
-        thriftClient.add(key, parent, columnToInsert, consistencyLevel);
-        sessionState.out.printf("Value %s%n", multiplier < 0 ? "decremented." : "incremented.");
-    }
-
-    private void executeShowClusterName() throws TException
-    {
-        if (!CliMain.isConnected())
-            return;
-
-        sessionState.out.println(thriftClient.describe_cluster_name());
-    }
-
-    /**
-     * Add a keyspace
-     * @param statement - a token tree representing current statement
-     */
-    private void executeAddKeySpace(Tree statement)
-    {
-
-        if (!CliMain.isConnected())
-            return;
-
-        // first value is the keyspace name, after that it is all key=value
-        String keyspaceName = CliUtils.unescapeSQLString(statement.getChild(0).getText());
-        KsDef ksDef = new KsDef(keyspaceName, DEFAULT_PLACEMENT_STRATEGY, new LinkedList<CfDef>());
-
-        try
-        {
-            String mySchemaVersion = thriftClient.system_add_keyspace(updateKsDefAttributes(statement, ksDef));
-            sessionState.out.println(mySchemaVersion);
-
-            keyspacesMap.put(keyspaceName, thriftClient.describe_keyspace(keyspaceName));
-        }
-        catch (InvalidRequestException e)
-        {
-            throw new RuntimeException(e);
-        }
-        catch (Exception e)
-        {
-            throw new RuntimeException(e);
-        }
-    }
-
-
-    /**
-     * Add a column family
-     * @param statement - a token tree representing current statement
-     */
-    private void executeAddColumnFamily(Tree statement)
-    {
-        if (!CliMain.isConnected() || !hasKeySpace())
-            return;
-
-        // first value is the column family name, after that it is all key=value
-        CfDef cfDef = new CfDef(keySpace, CliUtils.unescapeSQLString(statement.getChild(0).getText()));
-
-        try
-        {
-            String mySchemaVersion = thriftClient.system_add_column_family(updateCfDefAttributes(statement, cfDef));
-            sessionState.out.println(mySchemaVersion);
-            keyspacesMap.put(keySpace, thriftClient.describe_keyspace(keySpace));
-        }
-        catch (InvalidRequestException e)
-        {
-            throw new RuntimeException(e);
-        }
-        catch (Exception e)
-        {
-            throw new RuntimeException(e);
-        }
-    }
-
-    /**
-     * Update existing keyspace identified by name
-     * @param statement - tree represeting statement
-     */
-    private void executeUpdateKeySpace(Tree statement)
-    {
-        if (!CliMain.isConnected())
-            return;
-
-        try
-        {
-            String keyspaceName = CliCompiler.getKeySpace(statement, thriftClient.describe_keyspaces());
-
-            KsDef currentKsDef = getKSMetaData(keyspaceName);
-            KsDef updatedKsDef = updateKsDefAttributes(statement, currentKsDef);
-
-            String mySchemaVersion = thriftClient.system_update_keyspace(updatedKsDef);
-            sessionState.out.println(mySchemaVersion);
-            keyspacesMap.remove(keyspaceName);
-            getKSMetaData(keyspaceName);
-        }
-        catch (InvalidRequestException e)
-        {
-            throw new RuntimeException(e);
-        }
-        catch (Exception e)
-        {
-            throw new RuntimeException(e);
-        }
-    }
-
-    /**
-     * Update existing column family identified by name
-     * @param statement - tree represeting statement
-     */
-    private void executeUpdateColumnFamily(Tree statement)
-    {
-        if (!CliMain.isConnected() || !hasKeySpace())
-            return;
-
-        String cfName = CliCompiler.getColumnFamily(statement, currentCfDefs());
-
-        try
-        {
-            // request correct cfDef from the server (we let that call include CQL3 cf even though
-            // they can't be modified by thrift because the error message that will be thrown by
-            // system_update_column_family will be more useful)
-            CfDef cfDef = getCfDef(thriftClient.describe_keyspace(this.keySpace), cfName, true);
-
-            if (cfDef == null)
-                throw new RuntimeException("Column Family " + cfName + " was not found in the current keyspace.");
-
-            String mySchemaVersion = thriftClient.system_update_column_family(updateCfDefAttributes(statement, cfDef));
-            sessionState.out.println(mySchemaVersion);
-            keyspacesMap.put(keySpace, thriftClient.describe_keyspace(keySpace));
-        }
-        catch (InvalidRequestException e)
-        {
-            throw new RuntimeException(e);
-        }
-        catch (Exception e)
-        {
-            throw new RuntimeException(e);
-        }
-    }
-
-    /**
-     * Used to update keyspace definition attributes
-     * @param statement - ANTRL tree representing current statement
-     * @param ksDefToUpdate - keyspace definition to update
-     * @return ksDef - updated keyspace definition
-     */
-    private KsDef updateKsDefAttributes(Tree statement, KsDef ksDefToUpdate)
-    {
-        KsDef ksDef = new KsDef(ksDefToUpdate);
-
-        // removing all column definitions - thrift system_update_keyspace method requires that
-        ksDef.setCf_defs(new LinkedList<CfDef>());
-
-        for(int i = 1; i < statement.getChildCount(); i += 2)
-        {
-            String currentStatement = statement.getChild(i).getText().toUpperCase();
-            AddKeyspaceArgument mArgument = AddKeyspaceArgument.valueOf(currentStatement);
-            String mValue = statement.getChild(i + 1).getText();
-
-            switch(mArgument)
-            {
-            case PLACEMENT_STRATEGY:
-                ksDef.setStrategy_class(CliUtils.unescapeSQLString(mValue));
-                break;
-            case STRATEGY_OPTIONS:
-                ksDef.setStrategy_options(getStrategyOptionsFromTree(statement.getChild(i + 1)));
-                break;
-            case DURABLE_WRITES:
-                ksDef.setDurable_writes(Boolean.parseBoolean(mValue));
-                break;
-            default:
-                //must match one of the above or we'd throw an exception at the valueOf statement above.
-                assert(false);
-            }
-        }
-
-        // using default snitch options if strategy is NetworkTopologyStrategy and no options were set.
-        if (ksDef.getStrategy_class().contains(".NetworkTopologyStrategy"))
-        {
-            Map<String, String> currentStrategyOptions = ksDef.getStrategy_options();
-
-            // adding default data center from SimpleSnitch
-            if (currentStrategyOptions == null || currentStrategyOptions.isEmpty())
-            {
-                SimpleSnitch snitch = new SimpleSnitch();
-                Map<String, String> options = new HashMap<String, String>();
-                try
-                {
-                    options.put(snitch.getDatacenter(InetAddress.getLocalHost()), "1");
-                }
-                catch (UnknownHostException e)
-                {
-                    throw new RuntimeException(e);
-                }
-                ksDef.setStrategy_options(options);
-            }
-        }
-
-        return ksDef;
-    }
-
-    /**
-     * Update column family definition attributes
-     * @param statement - ANTLR tree representing current statement
-     * @param cfDefToUpdate - column family definition to apply updates on
-     * @return cfDef - updated column family definition
-     */
-    private CfDef updateCfDefAttributes(Tree statement, CfDef cfDefToUpdate)
-    {
-        CfDef cfDef = new CfDef(cfDefToUpdate);
-
-        for (int i = 1; i < statement.getChildCount(); i += 2)
-        {
-            String currentArgument = statement.getChild(i).getText().toUpperCase();
-            ColumnFamilyArgument mArgument = ColumnFamilyArgument.valueOf(currentArgument);
-            String mValue = statement.getChild(i + 1).getText();
-
-            switch(mArgument)
-            {
-            case COLUMN_TYPE:
-                cfDef.setColumn_type(CliUtils.unescapeSQLString(mValue));
-                break;
-            case COMPARATOR:
-                cfDef.setComparator_type(CliUtils.unescapeSQLString(mValue));
-                break;
-            case SUBCOMPARATOR:
-                cfDef.setSubcomparator_type(CliUtils.unescapeSQLString(mValue));
-                break;
-            case COMMENT:
-                cfDef.setComment(CliUtils.unescapeSQLString(mValue));
-                break;
-            case READ_REPAIR_CHANCE:
-                double chance = Double.parseDouble(mValue);
-
-                if (chance < 0 || chance > 1)
-                    throw new RuntimeException("Error: read_repair_chance must be between 0 and 1.");
-
-                cfDef.setRead_repair_chance(chance);
-                break;
-            case DCLOCAL_READ_REPAIR_CHANCE:
-                double localChance = Double.parseDouble(mValue);
-
-                if (localChance < 0 || localChance > 1)
-                    throw new RuntimeException("Error: dclocal_read_repair_chance must be between 0 and 1.");
-
-                cfDef.setDclocal_read_repair_chance(localChance);
-                break;
-            case GC_GRACE:
-                cfDef.setGc_grace_seconds(Integer.parseInt(mValue));
-                break;
-            case COLUMN_METADATA:
-                Tree arrayOfMetaAttributes = statement.getChild(i + 1);
-                if (!arrayOfMetaAttributes.getText().equals("ARRAY"))
-                    throw new RuntimeException("'column_metadata' format - [{ k:v, k:v, ..}, { ... }, ...]");
-                cfDef.setColumn_metadata(getCFColumnMetaFromTree(cfDef, arrayOfMetaAttributes));
-                break;
-            case MEMTABLE_OPERATIONS:
-                break;
-            case MEMTABLE_THROUGHPUT:
-                break;
-            case DEFAULT_VALIDATION_CLASS:
-                cfDef.setDefault_validation_class(CliUtils.unescapeSQLString(mValue));
-                break;
-            case MIN_COMPACTION_THRESHOLD:
-                int threshold = Integer.parseInt(mValue);
-                if (threshold <= 0)
-                    throw new RuntimeException("Disabling compaction by setting min/max compaction thresholds to 0 has been deprecated, set compaction_strategy_options={'enabled':false} instead");
-                cfDef.setMin_compaction_threshold(threshold);
-                cfDef.putToCompaction_strategy_options(CFPropDefs.KW_MINCOMPACTIONTHRESHOLD, Integer.toString(threshold));
-                break;
-            case MAX_COMPACTION_THRESHOLD:
-                threshold = Integer.parseInt(mValue);
-                if (threshold <= 0)
-                    throw new RuntimeException("Disabling compaction by setting min/max compaction thresholds to 0 has been deprecated, set compaction_strategy_options={'enabled':false} instead");
-                cfDef.setMax_compaction_threshold(Integer.parseInt(mValue));
-                cfDef.putToCompaction_strategy_options(CFPropDefs.KW_MAXCOMPACTIONTHRESHOLD, Integer.toString(threshold));
-                break;
-            case REPLICATE_ON_WRITE:
-                cfDef.setReplicate_on_write(Boolean.parseBoolean(mValue));
-                break;
-            case KEY_VALIDATION_CLASS:
-                cfDef.setKey_validation_class(CliUtils.unescapeSQLString(mValue));
-                break;
-            case COMPACTION_STRATEGY:
-                cfDef.setCompaction_strategy(CliUtils.unescapeSQLString(mValue));
-                break;
-            case COMPACTION_STRATEGY_OPTIONS:
-                cfDef.setCompaction_strategy_options(getStrategyOptionsFromTree(statement.getChild(i+1)));
-                break;
-            case COMPRESSION_OPTIONS:
-                cfDef.setCompression_options(getStrategyOptionsFromTree(statement.getChild(i+1)));
-                break;
-            case BLOOM_FILTER_FP_CHANCE:
-                cfDef.setBloom_filter_fp_chance(Double.parseDouble(mValue));
-                break;
-            case MEMTABLE_FLUSH_PERIOD_IN_MS:
-                cfDef.setMemtable_flush_period_in_ms(Integer.parseInt(mValue));
-                break;
-            case CACHING:
-                cfDef.setCaching(CliUtils.unescapeSQLString(mValue));
-                break;
-            case CELLS_PER_ROW_TO_CACHE:
-                cfDef.setCells_per_row_to_cache(CliUtils.unescapeSQLString(mValue));
-                break;
-            case DEFAULT_TIME_TO_LIVE:
-                cfDef.setDefault_time_to_live(Integer.parseInt(mValue));
-                break;
-            case INDEX_INTERVAL:
-                cfDef.setIndex_interval(Integer.parseInt(mValue));
-                break;
-            case SPECULATIVE_RETRY:
-                cfDef.setSpeculative_retry(CliUtils.unescapeSQLString(mValue));
-                break;
-            case POPULATE_IO_CACHE_ON_FLUSH:
-                cfDef.setPopulate_io_cache_on_flush(Boolean.parseBoolean(mValue));
-                break;
-            default:
-                //must match one of the above or we'd throw an exception at the valueOf statement above.
-                assert(false);
-
-            }
-        }
-
-        return cfDef;
-    }
-
-    /**
-     * Delete a keyspace
-     * @param statement - a token tree representing current statement
-     * @throws TException - exception
-     * @throws InvalidRequestException - exception
-     * @throws NotFoundException - exception
-     * @throws SchemaDisagreementException
-     */
-    private void executeDelKeySpace(Tree statement)
-            throws TException, InvalidRequestException, NotFoundException, SchemaDisagreementException
-    {
-        if (!CliMain.isConnected())
-            return;
-
-        String keyspaceName = CliCompiler.getKeySpace(statement, thriftClient.describe_keyspaces());
-        String version = thriftClient.system_drop_keyspace(keyspaceName);
-        sessionState.out.println(version);
-
-        if (keyspaceName.equals(keySpace)) //we just deleted the keyspace we were authenticated too
-            keySpace = null;
-    }
-
-    /**
-     * Delete a column family
-     * @param statement - a token tree representing current statement
-     * @throws TException - exception
-     * @throws InvalidRequestException - exception
-     * @throws NotFoundException - exception
-     * @throws SchemaDisagreementException
-     */
-    private void executeDelColumnFamily(Tree statement)
-            throws TException, InvalidRequestException, NotFoundException, SchemaDisagreementException
-    {
-        if (!CliMain.isConnected() || !hasKeySpace())
-            return;
-
-        String cfName = CliCompiler.getColumnFamily(statement, currentCfDefs());
-        String mySchemaVersion = thriftClient.system_drop_column_family(cfName);
-        sessionState.out.println(mySchemaVersion);
-    }
-
-    private void executeList(Tree statement)
-            throws TException, InvalidRequestException, NotFoundException, IllegalAccessException, InstantiationException, NoSuchFieldException, UnavailableException, TimedOutException, CharacterCodingException
-    {
-        if (!CliMain.isConnected() || !hasKeySpace())
-            return;
-
-        long startTime = System.nanoTime();
-
-        // extract column family
-        String columnFamily = CliCompiler.getColumnFamily(statement, currentCfDefs());
-
-        String rawStartKey = "";
-        String rawEndKey = "";
-        int limitCount = Integer.MAX_VALUE; // will reset to default later if it's not specified
-        int columnCount = Integer.MAX_VALUE; // will reset to default later if it's not specified
-        boolean reversed = false;
-
-        // optional arguments: key range and limit
-        for (int i = 1; i < statement.getChildCount(); i++)
-        {
-            Tree child = statement.getChild(i);
-            if (child.getType() == CliParser.NODE_KEY_RANGE)
-            {
-                if (child.getChildCount() > 0)
-                {
-                    rawStartKey = CliUtils.unescapeSQLString(child.getChild(0).getText());
-                    if (child.getChildCount() > 1)
-                        rawEndKey = CliUtils.unescapeSQLString(child.getChild(1).getText());
-                }
-            }
-            else if (child.getType() == CliParser.NODE_LIMIT)
-            {
-                if (child.getChildCount() != 1)
-                {
-                    sessionState.out.println("Invalid limit clause");
-                    return;
-                }
-
-                limitCount = Integer.parseInt(child.getChild(0).getText());
-                if (limitCount <= 0)
-                {
-                    sessionState.out.println("Invalid limit " + limitCount);
-                    return;
-                }
-            }
-            else if (child.getType() == CliParser.NODE_COLUMNS)
-            {
-                if ((child.getChildCount() < 1) || (child.getChildCount() > 2))
-                {
-                    sessionState.err.println("Invalid cells clause.");
-                    return;
-                }
-
-                String columns = child.getChild(0).getText();
-
-                try
-                {
-                    columnCount = Integer.parseInt(columns);
-                    if (columnCount < 0)
-                    {
-                        sessionState.err.println("Invalid cell limit: " + columnCount);
-                        return;
-                    }
-
-                    if (child.getChildCount() == 2)
-                        reversed = child.getChild(1).getType() == CliParser.NODE_REVERSED;
-                }
-                catch (NumberFormatException nfe)
-                {
-                    sessionState.err.println("Invalid cell number format: " + columns);
-                    return;
-                }
-            }
-        }
-
-        if (limitCount == Integer.MAX_VALUE)
-        {
-            limitCount = 100;
-            sessionState.out.println("Using default limit of 100");
-        }
-        if (columnCount == Integer.MAX_VALUE)
-        {
-            columnCount = 100;
-            sessionState.out.println("Using default cell limit of 100");
-        }
-
-
-        CfDef columnFamilyDef = getCfDef(columnFamily);
-
-        // read all columns and superColumns
-        SlicePredicate predicate = new SlicePredicate();
-        SliceRange sliceRange = new SliceRange();
-        sliceRange.setStart(new byte[0]).setFinish(new byte[0]);
-        sliceRange.setCount(columnCount);
-        sliceRange.setReversed(reversed);
-        predicate.setSlice_range(sliceRange);
-
-        // set the key range
-        KeyRange range = new KeyRange(limitCount);
-        AbstractType<?> keyComparator = this.cfKeysComparators.get(columnFamily);
-        ByteBuffer startKey = rawStartKey.isEmpty() ? ByteBufferUtil.EMPTY_BYTE_BUFFER : getBytesAccordingToType(rawStartKey, keyComparator);
-        ByteBuffer endKey = rawEndKey.isEmpty() ? ByteBufferUtil.EMPTY_BYTE_BUFFER : getBytesAccordingToType(rawEndKey, keyComparator);
-        range.setStart_key(startKey).setEnd_key(endKey);
-
-        ColumnParent columnParent = new ColumnParent(columnFamily);
-        List<KeySlice> keySlices = thriftClient.get_range_slices(columnParent, predicate, range, consistencyLevel);
-        printSliceList(columnFamilyDef, keySlices);
-        elapsedTime(startTime);
-    }
-
-    // DROP INDEX ON <CF>.<COLUMN>
-    private void executeDropIndex(Tree statement) throws TException, SchemaDisagreementException, InvalidRequestException, NotFoundException
-    {
-        if (!CliMain.isConnected() || !hasKeySpace())
-            return;
-
-        // getColumnFamily will check if CF exists for us
-        String columnFamily = CliCompiler.getColumnFamily(statement, currentCfDefs());
-        String rawColumName = CliUtils.unescapeSQLString(statement.getChild(1).getText());
-
-        CfDef cfDef = getCfDef(columnFamily);
-
-        ByteBuffer columnName = columnNameAsBytes(rawColumName, cfDef);
-
-        boolean foundColumn = false;
-
-        for (ColumnDef column : cfDef.getColumn_metadata())
-        {
-            if (column.name.equals(columnName))
-            {
-                foundColumn = true;
-
-                if (column.getIndex_type() == null)
-                    throw new RuntimeException(String.format("Column '%s' does not have an index.", rawColumName));
-
-                column.setIndex_name(null);
-                column.setIndex_type(null);
-            }
-        }
-
-        if (!foundColumn)
-            throw new RuntimeException(String.format("Column '%s' definition was not found in ColumnFamily '%s'.",
-                                                     rawColumName,
-                                                     columnFamily));
-
-        String mySchemaVersion = thriftClient.system_update_column_family(cfDef);
-        sessionState.out.println(mySchemaVersion);
-        keyspacesMap.put(keySpace, thriftClient.describe_keyspace(keySpace));
-    }
-
-    // TRUNCATE <columnFamily>
-    private void executeTruncate(String columnFamily) throws TException, InvalidRequestException, UnavailableException, TimedOutException
-    {
-        if (!CliMain.isConnected() || !hasKeySpace())
-            return;
-
-        // getting CfDef, it will fail if there is no such column family in current keySpace.
-        CfDef cfDef = getCfDef(CliCompiler.getColumnFamily(columnFamily, currentCfDefs()));
-
-        thriftClient.truncate(cfDef.getName());
-        sessionState.out.println(columnFamily + " truncated.");
-    }
-
-    /**
-     * Command: CONSISTENCYLEVEL AS (ONE | QUORUM ...)
-     * Tree: ^(NODE_CONSISTENCY_LEVEL AS (ONE | QUORUM ...))
-     * @param statement - tree representing current statement
-     */
-    private void executeConsistencyLevelStatement(Tree statement)
-    {
-        if (!CliMain.isConnected())
-            return;
-
-        String userSuppliedLevel = statement.getChild(0).getText().toUpperCase();
-
-        try
-        {
-            consistencyLevel = ConsistencyLevel.valueOf(userSuppliedLevel);
-        }
-        catch (IllegalArgumentException e)
-        {
-            String elements = "ONE, TWO, THREE, QUORUM, ALL, LOCAL_QUORUM, EACH_QUORUM, ANY";
-            sessionState.out.println(String.format("'%s' is invalid. Available: %s", userSuppliedLevel, elements));
-            return;
-        }
-
-        sessionState.out.println(String.format("Consistency level is set to '%s'.", consistencyLevel));
-    }
-
-    /**
-     * Command: ASSUME <columnFamily> (VALIDATOR | COMPARATOR | KEYS | SUB_COMPARATOR) AS <type>
-     * Tree: ^(NODE_ASSUME <columnFamily> (VALIDATOR | COMPARATOR | KEYS | SUB_COMPARATOR) <type>))
-     * @param statement - tree representing current statement
-     */
-    private void executeAssumeStatement(Tree statement)
-    {
-        if (!CliMain.isConnected() || !hasKeySpace())
-            return;
-
-        String cfName = CliCompiler.getColumnFamily(statement, currentCfDefs());
-
-        // VALIDATOR | COMPARATOR | KEYS | SUB_COMPARATOR
-        String assumptionElement = statement.getChild(1).getText().toUpperCase();
-
-
-        // Could be UTF8Type, IntegerType, LexicalUUIDType etc.
-        String defaultType = CliUtils.unescapeSQLString(statement.getChild(2).getText());
-
-        if (applyAssumption(cfName, assumptionElement, defaultType))
-        {
-            assumptions.addAssumption(keySpace, cfName, assumptionElement, defaultType);
-            sessionState.out.println(String.format("Assumption for column family '%s' added successfully.", cfName));
-        }
-    }
-
-    private boolean applyAssumption(String cfName, String assumptionElement, String defaultType)
-    {
-        CfDef columnFamily;
-
-        try
-        {
-            columnFamily = getCfDef(cfName);
-        }
-        catch (RuntimeException e)
-        {
-            return false; // just fail if CF does not exist
-        }
-
-        // used to store in this.cfKeysComparator
-        AbstractType<?> comparator;
-
-        try
-        {
-            comparator = TypeParser.parse(defaultType);
-        }
-        catch (RequestValidationException e)
-        {
-            try
-            {
-                comparator = Function.valueOf(defaultType.toUpperCase()).getValidator();
-            }
-            catch (Exception ne)
-            {
-                JVMStabilityInspector.inspectThrowable(ne);
-                String functions = Function.getFunctionNames();
-                sessionState.out.println("Type '" + defaultType + "' was not found. Available: " + functions
-                                         + " Or any class which extends o.a.c.db.marshal.AbstractType.");
-                return false;
-            }
-        }
-
-        // making string representation look property e.g. o.a.c.db.marshal.UTF8Type
-        defaultType = comparator.getClass().getName();
-
-        if (assumptionElement.equals("COMPARATOR"))
-        {
-            columnFamily.setComparator_type(defaultType);
-        }
-        else if (assumptionElement.equals("SUB_COMPARATOR"))
-        {
-            columnFamily.setSubcomparator_type(defaultType);
-        }
-        else if (assumptionElement.equals("VALIDATOR"))
-        {
-            columnFamily.setDefault_validation_class(defaultType);
-        }
-        else if (assumptionElement.equals("KEYS"))
-        {
-            this.cfKeysComparators.put(columnFamily.getName(), comparator);
-        }
-        else
-        {
-            String elements = "VALIDATOR, COMPARATOR, KEYS, SUB_COMPARATOR.";
-            sessionState.out.println(String.format("'%s' is invalid. Available: %s", assumptionElement, elements));
-            return false;
-        }
-
-        return true;
-    }
-
-    // SHOW API VERSION
-    private void executeShowVersion() throws TException
-    {
-        if (!CliMain.isConnected())
-            return;
-
-        sessionState.out.println(thriftClient.describe_version());
-    }
-
-    // SHOW KEYSPACES
-    private void executeShowKeySpaces() throws TException, InvalidRequestException
-    {
-        if (!CliMain.isConnected())
-            return;
-
-        printCQL3TablesWarning("show keyspaces");
-
-        List<KsDef> keySpaces = thriftClient.describe_keyspaces();
-
-        Collections.sort(keySpaces, new KsDefNamesComparator());
-        for (KsDef keySpace : keySpaces)
-        {
-            describeKeySpace(keySpace.name, keySpace);
-        }
-    }
-
-    // SHOW SCHEMA
-    private void executeShowSchema(Tree statement) throws TException, InvalidRequestException
-    {
-        if (!CliMain.isConnected())
-            return;
-
-        printCQL3TablesWarning("show schema");
-
-        final List<KsDef> keyspaces = thriftClient.describe_keyspaces();
-        Collections.sort(keyspaces, new KsDefNamesComparator());
-        final String keyspaceName = (statement.getChildCount() == 0)
-                                ? keySpace
-                                : CliCompiler.getKeySpace(statement, keyspaces);
-
-        Iterator<KsDef> ksIter;
-        if (keyspaceName != null)
-            ksIter = Collections2.filter(keyspaces, new Predicate<KsDef>()
-            {
-                public boolean apply(KsDef ksDef)
-                {
-                    return keyspaceName.equals(ksDef.name);
-                }
-            }).iterator();
-        else
-            ksIter = keyspaces.iterator();
-
-        while (ksIter.hasNext())
-            showKeyspace(sessionState.out, ksIter.next());
-
-        sessionState.out.flush();
-    }
-
-    /**
-     * Creates a CLI script to create the Keyspace it's Column Families
-     *
-     * @param output StringBuilder to write to.
-     * @param ksDef KsDef to create the cli script for.
-     */
-    private void showKeyspace(PrintStream output, KsDef ksDef)
-    {
-        output.append("create keyspace ").append(CliUtils.maybeEscapeName(ksDef.name));
-
-        writeAttr(output, true, "placement_strategy", normaliseType(ksDef.strategy_class, "org.apache.cassandra.locator"));
-
-        if (ksDef.strategy_options != null && !ksDef.strategy_options.isEmpty())
-        {
-            final StringBuilder opts = new StringBuilder();
-            opts.append("{");
-            String prefix = "";
-            for (Map.Entry<String, String> opt : ksDef.strategy_options.entrySet())
-            {
-                opts.append(prefix).append(CliUtils.escapeSQLString(opt.getKey())).append(" : ").append(CliUtils.escapeSQLString(opt.getValue()));
-                prefix = ", ";
-            }
-            opts.append("}");
-            writeAttrRaw(output, false, "strategy_options", opts.toString());
-        }
-
-        writeAttr(output, false, "durable_writes", ksDef.durable_writes);
-
-        output.append(";").append(NEWLINE);
-        output.append(NEWLINE);
-
-        output.append("use ").append(CliUtils.maybeEscapeName(ksDef.name)).append(";");
-        output.append(NEWLINE);
-        output.append(NEWLINE);
-
-        Collections.sort(ksDef.cf_defs, new CfDefNamesComparator());
-        for (CfDef cfDef : ksDef.cf_defs)
-            showColumnFamily(output, cfDef);
-        output.append(NEWLINE);
-        output.append(NEWLINE);
-    }
-
-    /**
-     * Creates a CLI script for the CfDef including meta data to the supplied StringBuilder.
-     *
-     * @param output File to write to.
-     * @param cfDef  CfDef to export attributes from.
-     */
-    private void showColumnFamily(PrintStream output, CfDef cfDef)
-    {
-        output.append("create column family ").append(CliUtils.maybeEscapeName(cfDef.name));
-
-        writeAttr(output, true, "column_type", cfDef.column_type);
-        writeAttr(output, false, "comparator", normaliseType(cfDef.comparator_type, "org.apache.cassandra.db.marshal"));
-        if (cfDef.column_type.equals("Super"))
-            writeAttr(output, false, "subcomparator", normaliseType(cfDef.subcomparator_type, "org.apache.cassandra.db.marshal"));
-        if (!StringUtils.isEmpty(cfDef.default_validation_class))
-            writeAttr(output, false, "default_validation_class",
-                        normaliseType(cfDef.default_validation_class, "org.apache.cassandra.db.marshal"));
-        writeAttr(output, false, "key_validation_class",
-                    normaliseType(cfDef.key_validation_class, "org.apache.cassandra.db.marshal"));
-
-        writeAttr(output, false, "read_repair_chance", cfDef.read_repair_chance);
-        writeAttr(output, false, "dclocal_read_repair_chance", cfDef.dclocal_read_repair_chance);
-        writeAttr(output, false, "gc_grace", cfDef.gc_grace_seconds);
-        writeAttr(output, false, "min_compaction_threshold", cfDef.min_compaction_threshold);
-        writeAttr(output, false, "max_compaction_threshold", cfDef.max_compaction_threshold);
-        writeAttr(output, false, "compaction_strategy", cfDef.compaction_strategy);
-        writeAttr(output, false, "caching", cfDef.caching);
-        writeAttr(output, false, "cells_per_row_to_cache", cfDef.cells_per_row_to_cache);
-        writeAttr(output, false, "default_time_to_live", cfDef.default_time_to_live);
-        writeAttr(output, false, "speculative_retry", cfDef.speculative_retry);
-
-        if (cfDef.isSetBloom_filter_fp_chance())
-            writeAttr(output, false, "bloom_filter_fp_chance", cfDef.bloom_filter_fp_chance);
-        if (!cfDef.compaction_strategy_options.isEmpty())
-        {
-            StringBuilder cOptions = new StringBuilder();
-
-            cOptions.append("{");
-
-            Map<String, String> options = cfDef.compaction_strategy_options;
-
-            int i = 0, size = options.size();
-
-            for (Map.Entry<String, String> entry : options.entrySet())
-            {
-                cOptions.append(CliUtils.quote(entry.getKey())).append(" : ").append(CliUtils.quote(entry.getValue()));
-
-                if (i != size - 1)
-                    cOptions.append(", ");
-
-                i++;
-            }
-
-            cOptions.append("}");
-
-            writeAttrRaw(output, false, "compaction_strategy_options", cOptions.toString());
-        }
-        if (!StringUtils.isEmpty(cfDef.comment))
-            writeAttr(output, false, "comment", cfDef.comment);
-
-        if (!cfDef.column_metadata.isEmpty())
-        {
-            output.append(NEWLINE)
-                  .append(TAB)
-                  .append("and column_metadata = [");
-
-            boolean first = true;
-            for (ColumnDef colDef : cfDef.column_metadata)
-            {
-                if (!first)
-                    output.append(",");
-                first = false;
-                showColumnMeta(output, cfDef, colDef);
-            }
-
-            output.append("]");
-        }
-
-        if (cfDef.compression_options != null && !cfDef.compression_options.isEmpty())
-        {
-            StringBuilder compOptions = new StringBuilder();
-
-            compOptions.append("{");
-
-            int i = 0, size = cfDef.compression_options.size();
-
-            for (Map.Entry<String, String> entry : cfDef.compression_options.entrySet())
-            {
-                compOptions.append(CliUtils.quote(entry.getKey())).append(" : ").append(CliUtils.quote(entry.getValue()));
-
-                if (i != size - 1)
-                    compOptions.append(", ");
-
-                i++;
-            }
-
-            compOptions.append("}");
-
-            writeAttrRaw(output, false, "compression_options", compOptions.toString());
-        }
-        if (cfDef.isSetIndex_interval())
-            writeAttr(output, false, "index_interval", cfDef.index_interval);
-
-        output.append(";");
-        output.append(NEWLINE);
-        output.append(NEWLINE);
-    }
-
-    /**
-     * Writes the supplied ColumnDef to the StringBuilder as a cli script.
-     *
-     * @param output The File to write to.
-     * @param cfDef  The CfDef as a source for comparator/validator
-     * @param colDef The Column Definition to export
-     */
-    private void showColumnMeta(PrintStream output, CfDef cfDef, ColumnDef colDef)
-    {
-        output.append(NEWLINE + TAB + TAB + "{");
-
-        final AbstractType<?> comparator = getFormatType(cfDef.column_type.equals("Super")
-                                                      ? cfDef.subcomparator_type
-                                                      : cfDef.comparator_type);
-        output.append("column_name : '" + CliUtils.escapeSQLString(comparator.getString(colDef.name)) + "'," + NEWLINE);
-        String validationClass = normaliseType(colDef.validation_class, "org.apache.cassandra.db.marshal");
-        output.append(TAB + TAB + "validation_class : " + CliUtils.escapeSQLString(validationClass));
-        if (colDef.isSetIndex_name())
-        {
-            output.append(",").append(NEWLINE)
-                  .append(TAB + TAB + "index_name : '" + CliUtils.escapeSQLString(colDef.index_name) + "'," + NEWLINE)
-                  .append(TAB + TAB + "index_type : " + CliUtils.escapeSQLString(Integer.toString(colDef.index_type.getValue())));
-
-            if (colDef.index_options != null && !colDef.index_options.isEmpty())
-            {
-                output.append(",").append(NEWLINE);
-                output.append(TAB + TAB + "index_options : {" + NEWLINE);
-                int numOpts = colDef.index_options.size();
-                for (Map.Entry<String, String> entry : colDef.index_options.entrySet())
-                {
-                    String option = CliUtils.escapeSQLString(entry.getKey());
-                    String optionValue = CliUtils.escapeSQLString(entry.getValue());
-
-                    output.append(TAB + TAB + TAB)
-                          .append("'" + option + "' : '")
-                          .append(optionValue)
-                          .append("'");
-
-                    if (--numOpts > 0)
-                        output.append(",").append(NEWLINE);
-                }
-                output.append("}");
-            }
-        }
-        output.append("}");
-    }
-
-    private String normaliseType(String path, String expectedPackage)
-    {
-        if (path.startsWith(expectedPackage))
-            return path.substring(expectedPackage.length() + 1);
-
-        return path;
-    }
-
-    private void writeAttr(PrintStream output, boolean first, String name, Boolean value)
-    {
-        writeAttrRaw(output, first, name, value.toString());
-    }
-    private void writeAttr(PrintStream output, boolean first, String name, Number value)
-    {
-        writeAttrRaw(output, first, name, value.toString());
-    }
-
-    private void writeAttr(PrintStream output, boolean first, String name, String value)
-    {
-        writeAttrRaw(output, first, name, "'" + CliUtils.escapeSQLString(value) + "'");
-    }
-
-    private void writeAttrRaw(PrintStream output, boolean first, String name, String value)
-    {
-        output.append(NEWLINE).append(TAB);
-        output.append(first ? "with " : "and ");
-        output.append(name).append(" = ");
-        output.append(value);
-    }
-    /**
-     * Returns true if this.keySpace is set, false otherwise
-     * @return boolean
-     */
-    private boolean hasKeySpace(boolean printError)
-    {
-        boolean hasKeyspace = keySpace != null;
-
-        if (!hasKeyspace && printError)
-            sessionState.err.println("Not authorized to a working keyspace.");
-
-        return hasKeyspace;
-    }
-
-    private boolean hasKeySpace()
-    {
-        return hasKeySpace(true);
-    }
-
-    public String getKeySpace()
-    {
-        return keySpace == null ? "unknown" : keySpace;
-    }
-
-    public void setKeySpace(String keySpace) throws NotFoundException, InvalidRequestException, TException
-    {
-        this.keySpace = keySpace;
-        // We do nothing with the return value, but it hits a cache and the tab-completer.
-        getKSMetaData(keySpace);
-    }
-
-    public String getUsername()
-    {
-        return username == null ? "default" : username;
-    }
-
-    public void setUsername(String username)
-    {
-        this.username = username;
-    }
-
-    // USE <keyspace_name>
-    private void executeUseKeySpace(Tree statement)
-    {
-        if (!CliMain.isConnected())
-            return;
-
-        int childCount = statement.getChildCount();
-        String keySpaceName, username = null, password = null;
-
-        // Get keyspace name
-        keySpaceName = CliUtils.unescapeSQLString(statement.getChild(0).getText());
-
-        if (childCount == 3)
-        {
-            username  = statement.getChild(1).getText();
-            password  = statement.getChild(2).getText();
-        }
-
-        if (keySpaceName == null)
-        {
-            sessionState.out.println("Keyspace argument required");
-            return;
-        }
-
-        try
-        {
-            AuthenticationRequest authRequest;
-
-            keySpaceName = CliCompiler.getKeySpace(keySpaceName, thriftClient.describe_keyspaces());
-
-            thriftClient.set_keyspace(keySpaceName);
-
-            if (username != null && password != null)
-            {
-                Map<String, String> credentials = new HashMap<String, String>();
-                /* remove quotes */
-                password = password.replace("\'", "");
-                credentials.put(IAuthenticator.USERNAME_KEY, username);
-                credentials.put(IAuthenticator.PASSWORD_KEY, password);
-                authRequest = new AuthenticationRequest(credentials);
-                thriftClient.login(authRequest);
-            }
-
-            keySpace = keySpaceName;
-            this.username = username != null ? username : "default";
-
-            keyspacesMap.remove(keySpace);
-            CliMain.updateCompletor(CliUtils.getCfNamesByKeySpace(getKSMetaData(keySpace)));
-            sessionState.out.println("Authenticated to keyspace: " + keySpace);
-        }
-        catch (AuthenticationException e)
-        {
-            sessionState.err.println("Exception during authentication to the cassandra node: " +
-                                     "verify keyspace exists, and you are using correct credentials.");
-        }
-        catch (AuthorizationException e)
-        {
-            sessionState.err.println("You are not authorized to use keyspace: " + keySpaceName);
-        }
-        catch (InvalidRequestException e)
-        {
-            sessionState.err.println(keySpaceName + " does not exist.");
-        }
-        catch (NotFoundException e)
-        {
-            sessionState.err.println(keySpaceName + " does not exist.");
-        }
-        catch (TException e)
-        {
-            if (sessionState.debug)
-                e.printStackTrace(sessionState.err);
-
-            sessionState.err.println("Login failure. Did you specify 'keyspace', 'username' and 'password'?");
-        }
-    }
-
-    private void executeTraceNextQuery() throws TException
-    {
-        if (!CliMain.isConnected())
-            return;
-
-        UUID sessionId = TimeUUIDType.instance.compose(thriftClient.trace_next_query());
-
-        sessionState.out.println("Will trace next query. Session ID: " + sessionId.toString());
-    }
-
-    private void describeKeySpace(String keySpaceName, KsDef metadata) throws TException
-    {
-        NodeProbe probe = sessionState.getNodeProbe();
-
-        // getting compaction manager MBean to displaying index building information
-        CompactionManagerMBean compactionManagerMBean = (probe == null) ? null : probe.getCompactionManagerProxy();
-
-        // Describe and display
-        sessionState.out.println("Keyspace: " + keySpaceName + ":");
-        try
-        {
-            KsDef ks_def;
-            ks_def = metadata == null ? thriftClient.describe_keyspace(keySpaceName) : metadata;
-            sessionState.out.println("  Replication Strategy: " + ks_def.strategy_class);
-
-            sessionState.out.println("  Durable Writes: " + ks_def.durable_writes);
-
-            Map<String, String> options = ks_def.strategy_options;
-            sessionState.out.println("    Options: [" + ((options == null) ? "" : FBUtilities.toString(options)) + "]");
-
-            sessionState.out.println("  Column Families:");
-
-            Collections.sort(ks_def.cf_defs, new CfDefNamesComparator());
-
-            for (CfDef cf_def : ks_def.cf_defs)
-                describeColumnFamily(ks_def, cf_def, probe);
-
-            // compaction manager information
-            if (compactionManagerMBean != null)
-            {
-                for (Map<String, String> info : compactionManagerMBean.getCompactions())
-                {
-                    // if ongoing compaction type is index build
-                    if (info.get("taskType").equals(OperationType.INDEX_BUILD.toString()))
-                        continue;
-                    sessionState.out.printf("%nCurrently building index %s, completed %d of %d bytes.%n",
-                                            info.get("columnfamily"),
-                                            info.get("bytesComplete"),
-                                            info.get("totalBytes"));
-                }
-            }
-
-            // closing JMX connection
-            if (probe != null)
-                probe.close();
-        }
-        catch (InvalidRequestException e)
-        {
-            sessionState.out.println("Invalid request: " + e);
-        }
-        catch (NotFoundException e)
-        {
-            sessionState.out.println("Keyspace " + keySpaceName + " could not be found.");
-        }
-        catch (IOException e)
-        {
-            sessionState.out.println("Error while closing JMX connection: " + e.getMessage());
-        }
-    }
-
-    private void describeColumnFamily(KsDef ks_def, CfDef cf_def, NodeProbe probe)
-    {
-        // fetching bean for current column family store
-        ColumnFamilyStoreMBean cfMBean = (probe == null) ? null : probe.getCfsProxy(ks_def.getName(), cf_def.getName());
-
-        boolean isSuper = cf_def.column_type.equals("Super");
-        sessionState.out.printf("    ColumnFamily: %s%s%n", cf_def.name, isSuper ? " (Super)" : "");
-
-        if (cf_def.comment != null && !cf_def.comment.isEmpty())
-            sessionState.out.printf("    \"%s\"%n", cf_def.comment);
-
-        if (cf_def.key_validation_class != null)
-            sessionState.out.printf("      Key Validation Class: %s%n", cf_def.key_validation_class);
-
-        if (cf_def.default_validation_class != null)
-            sessionState.out.printf("      Default column value validator: %s%n", cf_def.default_validation_class);
-
-        sessionState.out.printf("      Cells sorted by: %s%s%n", cf_def.comparator_type, cf_def.column_type.equals("Super") ? "/" + cf_def.subcomparator_type : "");
-        sessionState.out.printf("      GC grace seconds: %s%n", cf_def.gc_grace_seconds);
-        sessionState.out.printf("      Compaction min/max thresholds: %s/%s%n", cf_def.min_compaction_threshold, cf_def.max_compaction_threshold);
-        sessionState.out.printf("      Read repair chance: %s%n", cf_def.read_repair_chance);
-        sessionState.out.printf("      DC Local Read repair chance: %s%n", cf_def.dclocal_read_repair_chance);
-        sessionState.out.printf("      Caching: %s%n", cf_def.caching);
-        sessionState.out.printf("      Default time to live: %s%n", cf_def.default_time_to_live);
-        sessionState.out.printf("      Bloom Filter FP chance: %s%n", cf_def.isSetBloom_filter_fp_chance() ? cf_def.bloom_filter_fp_chance : "default");
-        sessionState.out.printf("      Index interval: %s%n", cf_def.isSetIndex_interval() ? cf_def.index_interval : "default");
-        sessionState.out.printf("      Speculative Retry: %s%n", cf_def.speculative_retry);
-
-        // if we have connection to the cfMBean established
-        if (cfMBean != null)
-            sessionState.out.printf("      Built indexes: %s%n", cfMBean.getBuiltIndexes());
-
-        if (cf_def.getColumn_metadataSize() != 0)
-        {
-            String leftSpace = "      ";
-            String columnLeftSpace = leftSpace + "    ";
-
-            String compareWith = isSuper ? cf_def.subcomparator_type
-                    : cf_def.comparator_type;
-            AbstractType<?> columnNameValidator = getFormatType(compareWith);
-
-            sessionState.out.println(leftSpace + "Column Metadata:");
-            for (ColumnDef columnDef : cf_def.getColumn_metadata())
-            {
-                String columnName = columnNameValidator.getString(columnDef.name);
-                if (columnNameValidator instanceof BytesType)
-                {
-                    try
-                    {
-                        String columnString = UTF8Type.instance.getString(columnDef.name);
-                        columnName = columnString + " (" + columnName + ")";
-                    }
-                    catch (MarshalException e)
-                    {
-                        // guess it wasn't a utf8 column name after all
-                    }
-                }
-
-                sessionState.out.println(leftSpace + "  Column Name: " + columnName);
-                sessionState.out.println(columnLeftSpace + "Validation Class: " + columnDef.getValidation_class());
-
-                if (columnDef.isSetIndex_name())
-                    sessionState.out.println(columnLeftSpace + "Index Name: " + columnDef.getIndex_name());
-
-                if (columnDef.isSetIndex_type())
-                    sessionState.out.println(columnLeftSpace + "Index Type: " + columnDef.getIndex_type().name());
-
-                if (columnDef.isSetIndex_options())
-                    sessionState.out.println(columnLeftSpace + "Index Options: " + columnDef.getIndex_options());
-            }
-        }
-
-        sessionState.out.printf("      Compaction Strategy: %s%n", cf_def.compaction_strategy);
-
-        if (cf_def.compaction_strategy_options != null && !cf_def.compaction_strategy_options.isEmpty())
-        {
-            sessionState.out.println("      Compaction Strategy Options:");
-            for (Map.Entry<String, String> e : cf_def.compaction_strategy_options.entrySet())
-                sessionState.out.printf("        %s: %s%n", e.getKey(), e.getValue());
-        }
-
-        if (cf_def.compression_options != null && !cf_def.compression_options.isEmpty())
-        {
-            sessionState.out.println("      Compression Options:");
-            for (Map.Entry<String, String> e : cf_def.compression_options.entrySet())
-                sessionState.out.printf("        %s: %s%n", e.getKey(), e.getValue());
-        }
-    }
-
-    // DESCRIBE KEYSPACE (<keyspace> | <column_family>)?
-    private void executeDescribe(Tree statement) throws TException, InvalidRequestException, NotFoundException
-    {
-        if (!CliMain.isConnected())
-            return;
-
-        printCQL3TablesWarning("describe");
-
-        int argCount = statement.getChildCount();
-
-        if (keySpace == null && argCount == 0)
-        {
-            sessionState.out.println("Authenticate to a Keyspace, before using `describe` or `describe <column_family>`");
-            return;
-        }
-
-        KsDef currentKeySpace = null;
-        if (keySpace != null)
-        {
-            keyspacesMap.remove(keySpace);
-            currentKeySpace = getKSMetaData(keySpace);
-        }
-
-        if (argCount > 1) // in case somebody changes Cli grammar
-            throw new RuntimeException("`describe` command take maximum one argument. See `help describe;`");
-
-        if (argCount == 0)
-        {
-            if (currentKeySpace != null)
-            {
-                describeKeySpace(currentKeySpace.name, null);
-                return;
-            }
-
-            sessionState.out.println("Authenticate to a Keyspace, before using `describe` or `describe <column_family>`");
-        }
-        else if (argCount == 1)
-        {
-            // name of the keyspace or ColumnFamily
-            String entityName = statement.getChild(0).getText();
-
-            KsDef inputKsDef = CliUtils.getKeySpaceDef(entityName, thriftClient.describe_keyspaces());
-
-            if (inputKsDef == null && currentKeySpace == null)
-                throw new RuntimeException(String.format("Keyspace with name '%s' wasn't found, " +
-                                                         "to lookup ColumnFamily with that name, please, authorize to one " +
-                                                         "of the keyspaces first.", entityName));
-
-            CfDef inputCfDef = (inputKsDef == null)
-                    ? getCfDef(currentKeySpace, entityName, false)
-                    : null;  // no need to lookup CfDef if we know that it was keyspace
-
-            if (inputKsDef != null)
-            {
-                describeKeySpace(inputKsDef.name, inputKsDef);
-            }
-            else if (inputCfDef != null)
-            {
-                NodeProbe probe = sessionState.getNodeProbe();
-
-                try
-                {
-                    describeColumnFamily(currentKeySpace, inputCfDef, probe);
-
-                    if (probe != null)
-                        probe.close();
-                }
-                catch (IOException e)
-                {
-                    sessionState.out.println("Error while closing JMX connection: " + e.getMessage());
-                }
-            }
-            else
-            {
-                sessionState.out.println("Sorry, no Keyspace nor (non-CQL3) ColumnFamily was found with name: " + entityName + " (if this is a CQL3 table, you should use cqlsh instead)");
-            }
-        }
-    }
-
-    // ^(NODE_DESCRIBE_CLUSTER) or describe: schema_versions, partitioner, snitch
-    private void executeDescribeCluster()
-    {
-        if (!CliMain.isConnected())
-            return;
-
-        sessionState.out.println("Cluster Information:");
-        try
-        {
-            sessionState.out.println("   Name: " + thriftClient.describe_cluster_name());
-            sessionState.out.println("   Snitch: " + thriftClient.describe_snitch());
-            sessionState.out.println("   Partitioner: " + thriftClient.describe_partitioner());
-
-            sessionState.out.println("   Schema versions: ");
-            Map<String,List<String>> versions = thriftClient.describe_schema_versions();
-
-            for (String version : versions.keySet())
-            {
-                sessionState.out.println(String.format("\t%s: %s%n", version, versions.get(version)));
-            }
-        }
-        catch (Exception e)
-        {
-            String message = (e instanceof InvalidRequestException) ? ((InvalidRequestException) e).getWhy() : e.getMessage();
-            sessionState.err.println("Error retrieving data: " + message);
-        }
-    }
-
-    // process a statement of the form: connect hostname/port
-    private void executeConnect(Tree statement)
-    {
-        Tree idList = statement.getChild(0);
-        int portNumber = Integer.parseInt(statement.getChild(1).getText());
-
-        StringBuilder hostName = new StringBuilder();
-        int idCount = idList.getChildCount();
-        for (int idx = 0; idx < idCount; idx++)
-        {
-            hostName.append(idList.getChild(idx).getText());
-        }
-
-        // disconnect current connection, if any.
-        // This is a no-op, if you aren't currently connected.
-        CliMain.disconnect();
-
-        // now, connect to the newly specified host name and port
-        sessionState.hostName = hostName.toString();
-        sessionState.thriftPort = portNumber;
-
-        // if we have user name and password
-        if (statement.getChildCount() == 4)
-        {
-            sessionState.username = statement.getChild(2).getText();
-            sessionState.password = CliUtils.unescapeSQLString(statement.getChild(3).getText());
-        }
-
-        CliMain.connect(sessionState.hostName, sessionState.thriftPort);
-    }
-
-    /**
-     * To get Column Family Definition object from specified keyspace
-     * @param keySpaceName key space name to search for specific column family
-     * @param columnFamilyName column family name
-     * @return CfDef - Column family definition object
-     */
-    private CfDef getCfDef(String keySpaceName, String columnFamilyName)
-    {
-        KsDef ksDef = keyspacesMap.get(keySpaceName);
-        CfDef cfDef = getCfDef(ksDef, columnFamilyName, true);
-        if (cfDef == null)
-            throw new RuntimeException("No such column family: " + columnFamilyName);
-        return cfDef;
-    }
-
-    /**
-     * Uses getCfDef(keySpaceName, columnFamilyName) with current keyspace
-     * @param columnFamilyName column family name to find in specified keyspace
-     * @return CfDef - Column family definition object
-     */
-    private CfDef getCfDef(String columnFamilyName)
-    {
-        return getCfDef(this.keySpace, columnFamilyName);
-    }
-
-    private CfDef getCfDef(KsDef keyspace, String columnFamilyName, boolean includeCQL3)
-    {
-        for (CfDef cfDef : keyspace.cf_defs)
-        {
-            if (cfDef.name.equals(columnFamilyName))
-                return cfDef;
-        }
-
-        return includeCQL3 ? cql3KeyspacesMap.get(keyspace.name).get(columnFamilyName) : null;
-    }
-
-    /**
-     * Used to parse meta tree and compile meta attributes into List<ColumnDef>
-     * @param cfDef - column family definition
-     * @param meta (Tree representing Array of the hashes with metadata attributes)
-     * @return List<ColumnDef> List of the ColumnDef's
-     *
-     * meta is in following format - ^(ARRAY ^(HASH ^(PAIR .. ..) ^(PAIR .. ..)) ^(HASH ...))
-     */
-    private List<ColumnDef> getCFColumnMetaFromTree(CfDef cfDef, Tree meta)
-    {
-        // this list will be returned
-        List<ColumnDef> columnDefinitions = new ArrayList<ColumnDef>();
-
-        // each child node is a ^(HASH ...)
-        for (int i = 0; i < meta.getChildCount(); i++)
-        {
-            Tree metaHash = meta.getChild(i);
-
-            ColumnDef columnDefinition = new ColumnDef();
-
-            // each child node is ^(PAIR $key $value)
-            for (int j = 0; j < metaHash.getChildCount(); j++)
-            {
-                Tree metaPair = metaHash.getChild(j);
-
-                // current $key
-                String metaKey = CliUtils.unescapeSQLString(metaPair.getChild(0).getText());
-                // current $value
-                String metaVal = CliUtils.unescapeSQLString(metaPair.getChild(1).getText());
-
-                if (metaKey.equals("column_name"))
-                {
-                    if (cfDef.column_type.equals("Super"))
-                        columnDefinition.setName(subColumnNameAsByteArray(metaVal, cfDef));
-                    else
-                        columnDefinition.setName(columnNameAsByteArray(metaVal, cfDef));
-                }
-                else if (metaKey.equals("validation_class"))
-                {
-                    columnDefinition.setValidation_class(metaVal);
-                }
-                else if (metaKey.equals("index_type"))
-                {
-                    columnDefinition.setIndex_type(getIndexTypeFromString(metaVal));
-                }
-                else if (metaKey.equals("index_options"))
-                {
-                    columnDefinition.setIndex_options(getStrategyOptionsFromTree(metaPair.getChild(1)));
-                }
-                else if (metaKey.equals("index_name"))
-                {
-                    columnDefinition.setIndex_name(metaVal);
-                }
-                else
-                {
-                    throw new RuntimeException("Unsupported column_metadata pair given => " + metaKey + ": " + metaVal);
-                }
-            }
-
-            // validating columnDef structure, 'name' and 'validation_class' must be set
-            try
-            {
-                columnDefinition.validate();
-            }
-            catch (TException e)
-            {
-                throw new RuntimeException(e);
-            }
-
-            columnDefinitions.add(columnDefinition);
-        }
-
-        return columnDefinitions;
-    }
-
-    /**
-     * Getting IndexType object from indexType string
-     * @param indexTypeAsString - string return by parser corresponding to IndexType
-     * @return IndexType - an IndexType object
-     */
-    private IndexType getIndexTypeFromString(String indexTypeAsString)
-    {
-        IndexType indexType;
-
-        try
-        {
-            indexType = IndexType.findByValue(new Integer(indexTypeAsString));
-        }
-        catch (NumberFormatException e)
-        {
-            try
-            {
-                // if this is not an integer lets try to get IndexType by name
-                indexType = IndexType.valueOf(indexTypeAsString);
-            }
-            catch (IllegalArgumentException ie)
-            {
-                throw new RuntimeException("IndexType '" + indexTypeAsString + "' is unsupported.", ie);
-            }
-        }
-
-        if (indexType == null)
-        {
-            throw new RuntimeException("IndexType '" + indexTypeAsString + "' is unsupported.");
-        }
-
-        return indexType;
-    }
-
-    /**
-     * Converts object represented as string into byte[] according to comparator
-     * @param object - object to covert into byte array
-     * @param comparator - comparator used to convert object
-     * @return byte[] - object in the byte array representation
-     */
-    private ByteBuffer getBytesAccordingToType(String object, AbstractType<?> comparator)
-    {
-        if (comparator == null) // default comparator is BytesType
-            comparator = BytesType.instance;
-
-        try
-        {
-            return comparator.fromString(object);
-        }
-        catch (MarshalException e)
-        {
-            throw new RuntimeException(e);
-        }
-    }
-
-    /**
-     * Converts column name into byte[] according to comparator type
-     * @param column - column name from parser
-     * @param columnFamily - column family name from parser
-     * @return ByteBuffer - bytes into which column name was converted according to comparator type
-     */
-    private ByteBuffer columnNameAsBytes(String column, String columnFamily)
-    {
-        CfDef columnFamilyDef = getCfDef(columnFamily);
-        return columnNameAsBytes(column, columnFamilyDef);
-    }
-    /**
-     * Converts column name into byte[] according to comparator type
-     * @param column - column name from parser
-     * @param columnFamilyDef - column family from parser
-     * @return ByteBuffer bytes - into which column name was converted according to comparator type
-     */
-    private ByteBuffer columnNameAsBytes(String column, CfDef columnFamilyDef)
-    {
-        String comparatorClass = columnFamilyDef.comparator_type;
-        return getBytesAccordingToType(column, getFormatType(comparatorClass));
-    }
-
-    /**
-     * Converts column name into byte[] according to comparator type
-     * @param column - column name from parser
-     * @param columnFamily - column family name from parser
-     * @return bytes[] - into which column name was converted according to comparator type
-     */
-    private byte[] columnNameAsByteArray(String column, String columnFamily)
-    {
-        return TBaseHelper.byteBufferToByteArray(columnNameAsBytes(column, columnFamily));
-    }
-
-    /**
-     * Converts column name into byte[] according to comparator type
-     * @param column - column name from parser
-     * @param cfDef  - column family from parser
-     * @return bytes[] - into which column name was converted according to comparator type
-     */
-    private byte[] columnNameAsByteArray(String column, CfDef cfDef)
-    {
-        return TBaseHelper.byteBufferToByteArray(columnNameAsBytes(column, cfDef));
-    }
-
-    /**
-     * Converts sub-column name into ByteBuffer according to comparator type
-     * @param superColumn - sub-column name from parser
-     * @param columnFamily - column family name from parser
-     * @return ByteBuffer bytes - into which column name was converted according to comparator type
-     */
-    private ByteBuffer subColumnNameAsBytes(String superColumn, String columnFamily)
-    {
-        CfDef columnFamilyDef = getCfDef(columnFamily);
-        return subColumnNameAsBytes(superColumn, columnFamilyDef);
-    }
-
-    /**
-     * Converts column name into ByteBuffer according to comparator type
-     * @param superColumn - sub-column name from parser
-     * @param columnFamilyDef - column family from parser
-     * @return ByteBuffer bytes - into which column name was converted according to comparator type
-     */
-    private ByteBuffer subColumnNameAsBytes(String superColumn, CfDef columnFamilyDef)
-    {
-        String comparatorClass = columnFamilyDef.subcomparator_type;
-
-        if (comparatorClass == null)
-        {
-            sessionState.out.println(String.format("Notice: defaulting to BytesType subcomparator for '%s'", columnFamilyDef.getName()));
-            comparatorClass = "BytesType";
-        }
-
-        return getBytesAccordingToType(superColumn, getFormatType(comparatorClass));
-    }
-
-    /**
-     * Converts sub-column name into byte[] according to comparator type
-     * @param superColumn - sub-column name from parser
-     * @param cfDef - column family from parser
-     * @return bytes[] - into which column name was converted according to comparator type
-     */
-    private byte[] subColumnNameAsByteArray(String superColumn, CfDef cfDef)
-    {
-        return TBaseHelper.byteBufferToByteArray(subColumnNameAsBytes(superColumn, cfDef));
-    }
-
-    /**
-     * Converts column value into byte[] according to validation class
-     * @param columnName - column name to which value belongs
-     * @param columnFamilyName - column family name
-     * @param columnValue - actual column value
-     * @return value in byte array representation
-     */
-    private ByteBuffer columnValueAsBytes(ByteBuffer columnName, String columnFamilyName, String columnValue)
-    {
-        CfDef columnFamilyDef = getCfDef(columnFamilyName);
-        AbstractType<?> defaultValidator = getFormatType(columnFamilyDef.default_validation_class);
-
-        for (ColumnDef columnDefinition : columnFamilyDef.getColumn_metadata())
-        {
-            byte[] currentColumnName = columnDefinition.getName();
-
-            if (ByteBufferUtil.compare(currentColumnName, columnName) == 0)
-            {
-                try
-                {
-                    String validationClass = columnDefinition.getValidation_class();
-                    return getBytesAccordingToType(columnValue, getFormatType(validationClass));
-                }
-                catch (Exception e)
-                {
-                    throw new RuntimeException(e);
-                }
-            }
-        }
-
-        return defaultValidator.fromString(columnValue);
-    }
-
-    /**
-     * Get validator for specific column value
-     * @param cfDef - CfDef object representing column family with metadata
-     * @param columnNameInBytes - column name as byte array
-     * @return AbstractType - validator for column value
-     */
-    private AbstractType<?> getValidatorForValue(CfDef cfDef, byte[] columnNameInBytes)
-    {
-        String defaultValidator = cfDef.default_validation_class;
-
-        for (ColumnDef columnDefinition : cfDef.getColumn_metadata())
-        {
-            byte[] nameInBytes = columnDefinition.getName();
-
-            if (Arrays.equals(nameInBytes, columnNameInBytes))
-            {
-                return getFormatType(columnDefinition.getValidation_class());
-            }
-        }
-
-        if (defaultValidator != null && !defaultValidator.isEmpty())
-        {
-            return getFormatType(defaultValidator);
-        }
-
-        return null;
-    }
-
-    /**
-     * Used to get Map of the provided options by create/update keyspace commands
-     * @param options - tree representing options
-     * @return Map - strategy_options map
-     */
-    private Map<String, String> getStrategyOptionsFromTree(Tree options)
-    {
-        //Check for old [{}] syntax
-        if (options.getText().equalsIgnoreCase("ARRAY"))
-        {
-            System.err.println("WARNING: [{}] strategy_options syntax is deprecated, please use {}");
-
-            if (options.getChildCount() == 0)
-                return Collections.EMPTY_MAP;
-
-            return getStrategyOptionsFromTree(options.getChild(0));
-        }
-
-        // this map will be returned
-        Map<String, String> strategyOptions = new HashMap<String, String>();
-
-        // each child node is ^(PAIR $key $value)
-        for (int j = 0; j < options.getChildCount(); j++)
-        {
-            Tree optionPair = options.getChild(j);
-
-            // current $key
-            String key = CliUtils.unescapeSQLString(optionPair.getChild(0).getText());
-            // current $value
-            String val = CliUtils.unescapeSQLString(optionPair.getChild(1).getText());
-
-            strategyOptions.put(key, val);
-        }
-
-        return strategyOptions;
-    }
-
-    /**
-     * Used to convert value (function argument, string) into byte[]
-     * calls convertValueByFunction method with "withUpdate" set to false
-     * @param functionCall - tree representing function call ^(FUNCTION_CALL function_name value)
-     * @param columnFamily - column family definition (CfDef)
-     * @param columnName   - also updates column family metadata for given column
-     * @return byte[] - string value as byte[]
-     */
-    private ByteBuffer convertValueByFunction(Tree functionCall, CfDef columnFamily, ByteBuffer columnName)
-    {
-        return convertValueByFunction(functionCall, columnFamily, columnName, false);
-    }
-
-    /**
-     * Used to convert value (function argument, string) into byte[]
-     * @param functionCall - tree representing function call ^(FUNCTION_CALL function_name value)
-     * @param columnFamily - column family definition (CfDef)
-     * @param columnName   - column name as byte[] (used to update CfDef)
-     * @param withUpdate   - also updates column family metadata for given column
-     * @return byte[] - string value as byte[]
-     */
-    private ByteBuffer convertValueByFunction(Tree functionCall, CfDef columnFamily, ByteBuffer columnName, boolean withUpdate)
-    {
-        String functionName = functionCall.getChild(0).getText();
-        Tree argumentTree = functionCall.getChild(1);
-        String functionArg  = (argumentTree == null) ? "" : CliUtils.unescapeSQLString(argumentTree.getText());
-        AbstractType<?> validator = getTypeByFunction(functionName);
-
-        try
-        {
-
-            ByteBuffer value;
-
-            if (functionArg.isEmpty())
-            {
-                if (validator instanceof TimeUUIDType)
-                {
-                    value = ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes());
-                }
-                else if (validator instanceof LexicalUUIDType)
-                {
-                    value = ByteBuffer.wrap(UUIDGen.decompose(UUID.randomUUID()));
-                }
-                else if (validator instanceof BytesType)
-                {
-                    value = ByteBuffer.wrap(new byte[0]);
-                }
-                else
-                {
-                    throw new RuntimeException(String.format("Argument for '%s' could not be empty.", functionName));
-                }
-            }
-            else
-            {
-                value = getBytesAccordingToType(functionArg, validator);
-            }
-
-            // performing ColumnDef local validator update
-            if (withUpdate)
-            {
-                updateColumnMetaData(columnFamily, columnName, validator.toString());
-            }
-
-            return value;
-        }
-        catch (Exception e)
-        {
-            throw new RuntimeException(e);
-        }
-    }
-
-    /**
-     * Get AbstractType by function name
-     * @param functionName - name of the function e.g. utf8, integer, long etc.
-     * @return AbstractType type corresponding to the function name
-     */
-    public static AbstractType<?> getTypeByFunction(String functionName)
-    {
-        Function function;
-
-        try
-        {
-            function = Function.valueOf(functionName.toUpperCase());
-        }
-        catch (IllegalArgumentException e)
-        {
-            String message = String.format("Function '%s' not found. Available functions: %s", functionName, Function.getFunctionNames());
-            throw new RuntimeException(message, e);
-        }
-
-        return function.getValidator();
-    }
-
-    /**
-     * Used to locally update column family definition with new column metadata
-     * @param columnFamily    - CfDef record
-     * @param columnName      - column name represented as byte[]
-     * @param validationClass - value validation class
-     */
-    private void updateColumnMetaData(CfDef columnFamily, ByteBuffer columnName, String validationClass)
-    {
-        ColumnDef column = getColumnDefByName(columnFamily, columnName);
-
-        if (column != null)
-        {
-            // if validation class is the same - no need to modify it
-            if (column.getValidation_class().equals(validationClass))
-                return;
-
-            // updating column definition with new validation_class
-            column.setValidation_class(validationClass);
-        }
-        else
-        {
-            List<ColumnDef> columnMetaData = new ArrayList<ColumnDef>(columnFamily.getColumn_metadata());
-            columnMetaData.add(new ColumnDef(columnName, validationClass));
-            columnFamily.setColumn_metadata(columnMetaData);
-        }
-    }
-
-    /**
-     * Get specific ColumnDef in column family meta data by column name
-     * @param columnFamily - CfDef record
-     * @param columnName   - column name represented as byte[]
-     * @return ColumnDef   - found column definition
-     */
-    private ColumnDef getColumnDefByName(CfDef columnFamily, ByteBuffer columnName)
-    {
-        for (ColumnDef columnDef : columnFamily.getColumn_metadata())
-        {
-            byte[] currName = columnDef.getName();
-
-            if (ByteBufferUtil.compare(currName, columnName) == 0)
-            {
-                return columnDef;
-            }
-        }
-
-        return null;
-    }
-
-    /**
-     * Prints out KeySlice list
-     * @param columnFamilyDef - column family definition
-     * @param slices - list of the KeySlice's to print out
-     * @throws NotFoundException - column not found
-     * @throws TException - transfer is broken
-     * @throws IllegalAccessException - can't do operation
-     * @throws InstantiationException - can't instantiate a class
-     * @throws NoSuchFieldException - column not found
-     */
-    private void printSliceList(CfDef columnFamilyDef, List<KeySlice> slices)
-            throws NotFoundException, TException, IllegalAccessException, InstantiationException, NoSuchFieldException, CharacterCodingException
-    {
-        AbstractType<?> validator;
-        String columnFamilyName = columnFamilyDef.getName();
-        AbstractType<?> keyComparator = getKeyComparatorForCF(columnFamilyName);
-
-        for (KeySlice ks : slices)
-        {
-            String keyName = (keyComparator == null) ? ByteBufferUtil.string(ks.key) : keyComparator.getString(ks.key);
-
-            sessionState.out.printf("-------------------%n");
-            sessionState.out.printf("RowKey: %s%n", keyName);
-            Iterator<ColumnOrSuperColumn> iterator = ks.getColumnsIterator();
-
-            while (iterator.hasNext())
-            {
-                ColumnOrSuperColumn columnOrSuperColumn = iterator.next();
-
-                if (columnOrSuperColumn.column != null)
-                {
-                    Column col = columnOrSuperColumn.column;
-                    validator = getValidatorForValue(columnFamilyDef, col.getName());
-
-                    sessionState.out.printf("=> (name=%s, value=%s, timestamp=%d%s)%n",
-                                    formatColumnName(keySpace, columnFamilyName, col.name), validator.getString(col.value), col.timestamp,
-                                    col.isSetTtl() ? String.format(", ttl=%d", col.getTtl()) : "");
-                }
-                else if (columnOrSuperColumn.super_column != null)
-                {
-                    SuperColumn superCol = columnOrSuperColumn.super_column;
-                    sessionState.out.printf("=> (super_column=%s,", formatColumnName(keySpace, columnFamilyName, superCol.name));
-
-                    for (Column col : superCol.columns)
-                    {
-                        validator = getValidatorForValue(columnFamilyDef, col.getName());
-
-                        sessionState.out.printf("%n     (name=%s, value=%s, timestamp=%d%s)",
-                                        formatSubcolumnName(keySpace, columnFamilyName, col.name), validator.getString(col.value), col.timestamp,
-                                        col.isSetTtl() ? String.format(", ttl=%d", col.getTtl()) : "");
-                    }
-
-                    sessionState.out.println(")");
-                }
-                else if (columnOrSuperColumn.counter_column != null)
-                {
-                    CounterColumn col = columnOrSuperColumn.counter_column;
-
-                    sessionState.out.printf("=> (counter=%s, value=%s)%n", formatColumnName(keySpace, columnFamilyName, col.name), col.value);
-                }
-                else if (columnOrSuperColumn.counter_super_column != null)
-                {
-                    CounterSuperColumn superCol = columnOrSuperColumn.counter_super_column;
-                    sessionState.out.printf("=> (super_column=%s,", formatColumnName(keySpace, columnFamilyName, superCol.name));
-
-                    for (CounterColumn col : superCol.columns)
-                    {
-                        sessionState.out.printf("%n     (counter=%s, value=%s)", formatSubcolumnName(keySpace, columnFamilyName, col.name), col.value);
-                    }
-
-                    sessionState.out.println(")");
-                }
-            }
-        }
-
-        sessionState.out.printf("%n%d Row%s Returned.%n", slices.size(), (slices.size() > 1 ? "s" : ""));
-    }
-
-    // returns sub-column name in human-readable format
-    private String formatSubcolumnName(String keyspace, String columnFamily, ByteBuffer name)
-    {
-        return getFormatType(getCfDef(keyspace, columnFamily).subcomparator_type).getString(name);
-    }
-
-    // retuns column name in human-readable format
-    private String formatColumnName(String keyspace, String columnFamily, ByteBuffer name)
-    {
-        return getFormatType(getCfDef(keyspace, columnFamily).comparator_type).getString(name);
-    }
-
-    private ByteBuffer getColumnName(String columnFamily, Tree columnTree)
-    {
-        return (columnTree.getType() == CliParser.FUNCTION_CALL)
-                    ? convertValueByFunction(columnTree, null, null)
-                    : columnNameAsBytes(CliUtils.unescapeSQLString(columnTree.getText()), columnFamily);
-    }
-
-    private ByteBuffer getSubColumnName(String columnFamily, Tree columnTree)
-    {
-        return (columnTree.getType() == CliParser.FUNCTION_CALL)
-                    ? convertValueByFunction(columnTree, null, null)
-                    : subColumnNameAsBytes(CliUtils.unescapeSQLString(columnTree.getText()), columnFamily);
-    }
-
-    public ByteBuffer getKeyAsBytes(String columnFamily, Tree keyTree)
-    {
-        if (keyTree.getType() == CliParser.FUNCTION_CALL)
-            return convertValueByFunction(keyTree, null, null);
-
-        String key = CliUtils.unescapeSQLString(keyTree.getText());
-
-        return getBytesAccordingToType(key, getKeyComparatorForCF(columnFamily));
-    }
-
-    private AbstractType<?> getKeyComparatorForCF(String columnFamily)
-    {
-        AbstractType<?> keyComparator = cfKeysComparators.get(columnFamily);
-
-        if (keyComparator == null)
-        {
-            String defaultValidationClass = getCfDef(columnFamily).getKey_validation_class();
-            assert defaultValidationClass != null;
-            keyComparator = getFormatType(defaultValidationClass);
-        }
-
-        return keyComparator;
-    }
-
-    private static class KsDefNamesComparator implements Comparator<KsDef>
-    {
-        public int compare(KsDef a, KsDef b)
-        {
-            return a.name.compareTo(b.name);
-        }
-    }
-
-    private static class CfDefNamesComparator implements Comparator<CfDef>
-    {
-        public int compare(CfDef a, CfDef b)
-        {
-            return a.name.compareTo(b.name);
-        }
-    }
-
-    private boolean isCounterCF(CfDef cfdef)
-    {
-        String defaultValidator = cfdef.default_validation_class;
-        return defaultValidator != null
-               && !defaultValidator.isEmpty()
-               && getFormatType(defaultValidator) instanceof CounterColumnType;
-    }
-
-    /**
-     * Print elapsed time. Print 2 fraction digits if eta is under 10 ms.
-     * @param startTime starting time in nanoseconds
-     */
-    private void elapsedTime(long startTime)
-    {
-        /** time elapsed in nanoseconds */
-        long eta = System.nanoTime() - startTime;
-
-        sessionState.out.print("Elapsed time: ");
-        if (eta < 10000000)
-        {
-            sessionState.out.print(Math.round(eta/10000.0)/100.0);
-        }
-        else
-        {
-            sessionState.out.print(Math.round(eta/1000000.0));
-        }
-        sessionState.out.println(" msec(s).");
-    }
-
-    class CfAssumptions
-    {
-        private static final String ASSUMPTIONS_FILENAME = "assumptions.json";
-        //Map<KeySpace, Map<ColumnFamily, Map<Property, Value>>>
-        private Map<String, Map<String, Map<String, String>>> assumptions;
-        private boolean assumptionsChanged;
-        private File assumptionDirectory;
-
-        public CfAssumptions()
-        {
-            assumptions = new HashMap<String, Map<String, Map<String, String>>>();
-            assumptionsChanged = false;
-            assumptionDirectory = FBUtilities.getToolsOutputDirectory();
-
-            File oldAssumptionDir = new File(System.getProperty("user.home") + File.separator + ".cassandra-cli");
-            if (oldAssumptionDir.exists())
-            {
-                File oldAssumptionFile = new File(oldAssumptionDir, ASSUMPTIONS_FILENAME);
-                if (oldAssumptionFile.exists())
-                    FileUtils.renameWithConfirm(oldAssumptionFile, new File(assumptionDirectory, ASSUMPTIONS_FILENAME));
-                FileUtils.deleteRecursive(oldAssumptionDir);
-            }
-        }
-
-        public void addAssumption(String keyspace, String columnFamily, String property, String value)
-        {
-            Map<String, Map<String, String>> ksAssumes = assumptions.get(keyspace);
-            if (ksAssumes == null)
-            {
-                ksAssumes = new HashMap<String, Map<String, String>>();
-                assumptions.put(keyspace, ksAssumes);
-            }
-
-            Map<String, String> cfAssumes = ksAssumes.get(columnFamily);
-            if (cfAssumes == null)
-            {
-                cfAssumes = new HashMap<String, String>();
-                ksAssumes.put(columnFamily, cfAssumes);
-            }
-
-            cfAssumes.put(property, value);
-            assumptionsChanged = true;
-        }
-
-        public void replayAssumptions(String keyspace)
-        {
-            if (!CliMain.isConnected() || !hasKeySpace(false))
-                return;
-
-            Map<String, Map<String, String>> cfAssumes = assumptions.get(keyspace);
-            if (cfAssumes != null)
-            {
-                for (Map.Entry<String, Map<String, String>> cfEntry : cfAssumes.entrySet())
-                {
-                    String columnFamily = cfEntry.getKey();
-                    Map<String, String> props = cfEntry.getValue();
-
-                    for (Map.Entry<String, String> propEntry : props.entrySet())
-                    {
-                        applyAssumption(columnFamily, propEntry.getKey(), propEntry.getValue());
-                    }
-                }
-            }
-        }
-
-        private void readAssumptions()
-        {
-            File assumptionFile = new File(assumptionDirectory, ASSUMPTIONS_FILENAME);
-            if (assumptionFile.isFile())
-            {
-                try
-                {
-                    JsonFactory f = new JsonFactory();
-                    JsonParser p = f.createJsonParser(assumptionFile);
-                    JsonToken token = p.nextToken();
-                    while (token != JsonToken.END_OBJECT)
-                    {
-                        if (token == JsonToken.FIELD_NAME)
-                        {
-                            String keyspace = p.getText();
-                            Map<String, Map<String, String>> ksAssumes = assumptions.get(keyspace);
-                            if (ksAssumes == null)
-                            {
-                                ksAssumes = new HashMap<String, Map<String, String>>();
-                                assumptions.put(keyspace,  ksAssumes);
-                            }
-                            token = p.nextToken();
-                            while (token != JsonToken.END_ARRAY)
-                            {
-                                if (token == JsonToken.FIELD_NAME)
-                                {
-                                    String columnFamily = p.getText();
-                                    Map<String, String> cfAssumes = ksAssumes.get(columnFamily);
-                                    if (cfAssumes == null)
-                                    {
-                                        cfAssumes = new HashMap<String, String>();
-                                        ksAssumes.put(columnFamily, cfAssumes);
-                                    }
-
-                                    token = p.nextToken();
-                                    while (token != JsonToken.END_ARRAY)
-                                    {
-                                        if (token == JsonToken.FIELD_NAME)
-                                        {
-                                            String prop = p.getText();
-                                            p.nextToken();
-                                            String value = p.getText();
-                                            cfAssumes.put(prop, value);
-                                        }
-
-                                        token = p.nextToken();
-                                    }
-                                }
-                                token = p.nextToken();
-                            }
-                        }
-                        token = p.nextToken();
-                    }
-                    sessionState.out.println("Column Family assumptions read from " + assumptionFile);
-                }
-                catch (Exception e)
-                {
-                    sessionState.err.println("Failed reading " + assumptionFile + " file");
-                }
-            }
-        }
-
-        private void writeAssumptions()
-        {
-            if (assumptionsChanged)
-            {
-                File assumptionFile = new File(assumptionDirectory, ASSUMPTIONS_FILENAME);
-                try
-                {
-                    JsonFactory f = new JsonFactory();
-                    JsonGenerator g = f.createJsonGenerator(assumptionFile, JsonEncoding.UTF8);
-                    g.useDefaultPrettyPrinter();
-                    g.writeStartObject();
-                    for (Map.Entry<String, Map<String, Map<String, String>>> ksEntry : assumptions.entrySet())
-                    {
-                        g.writeFieldName(ksEntry.getKey());
-                        g.writeStartArray();
-                        for (Map.Entry<String, Map<String, String>> cfEntry : ksEntry.getValue().entrySet())
-                        {
-                            g.writeStartObject();
-                            g.writeFieldName(cfEntry.getKey());
-                            g.writeStartArray();
-                            for (Map.Entry<String, String> asEntry : cfEntry.getValue().entrySet())
-                            {
-                                g.writeStartObject();
-                                g.writeStringField(asEntry.getKey(), asEntry.getValue());
-                                g.writeEndObject();
-                            }
-                            g.writeEndArray();
-                            g.writeEndObject();
-                        }
-                        g.writeEndArray();
-                    }
-                    g.writeEndObject();
-                    g.close();
-                    sessionState.out.println("Column Family assumptions written to " + assumptionFile);
-                    assumptionsChanged = false;
-                }
-                catch (Exception e)
-                {
-                    sessionState.err.println("Failed writing " + assumptionFile + " file");
-                }
-            }
-        }
-    }
-}
diff --git a/src/java/org/apache/cassandra/cli/CliCommandHelp.java b/src/java/org/apache/cassandra/cli/CliCommandHelp.java
deleted file mode 100644
index dce8d60..0000000
--- a/src/java/org/apache/cassandra/cli/CliCommandHelp.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/*
- * 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.cli;
-
-public class CliCommandHelp
-{
-    public String name;
-    public String help;
-}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/cli/CliCompiler.java b/src/java/org/apache/cassandra/cli/CliCompiler.java
deleted file mode 100644
index 6b84be1..0000000
--- a/src/java/org/apache/cassandra/cli/CliCompiler.java
+++ /dev/null
@@ -1,172 +0,0 @@
-/*
- * 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.cli;
-
-import java.util.List;
-
-import org.antlr.runtime.ANTLRStringStream;
-import org.antlr.runtime.CharStream;
-import org.antlr.runtime.CommonTokenStream;
-import org.antlr.runtime.tree.Tree;
-import org.apache.cassandra.thrift.CfDef;
-import org.apache.cassandra.thrift.KsDef;
-
-
-public class CliCompiler
-{
-
-    // ANTLR does not provide case-insensitive tokenization support
-    // out of the box. So we override the LA (lookahead) function
-    // of the ANTLRStringStream class. Note: This doesn't change the
-    // token text-- but just relaxes the matching rules to match
-    // in upper case. [Logic borrowed from Hive code.]
-    //
-    // Also see discussion on this topic in:
-    // http://www.antlr.org/wiki/pages/viewpage.action?pageId=1782.
-    public static class ANTLRNoCaseStringStream  extends ANTLRStringStream
-    {
-        public ANTLRNoCaseStringStream(String input)
-        {
-            super(input);
-        }
-
-        public int LA(int i)
-        {
-            int returnChar = super.LA(i);
-            if (returnChar == CharStream.EOF)
-            {
-                return returnChar;
-            }
-            else if (returnChar == 0)
-            {
-                return returnChar;
-            }
-
-            return Character.toUpperCase((char)returnChar);
-        }
-    }
-
-    public static Tree compileQuery(String query)
-    {
-        Tree queryTree;
-
-        try
-        {
-            ANTLRStringStream input = new ANTLRNoCaseStringStream(query);
-
-            CliLexer lexer = new CliLexer(input);
-            CommonTokenStream tokens = new CommonTokenStream(lexer);
-
-            CliParser parser = new CliParser(tokens);
-
-            // start parsing...
-            queryTree = (Tree)(parser.root().getTree());
-
-            // semantic analysis if any...
-            //  [tbd]
-
-        }
-        catch(Exception e)
-        {
-            // if there was an exception we don't want to process request any further
-            throw new RuntimeException(e.getMessage(), e);
-        }
-
-        return queryTree;
-    }
-    /*
-     * NODE_COLUMN_ACCESS related functions.
-     */
-
-    public static String getColumnFamily(Tree astNode, Iterable<CfDef> cfDefs)
-    {
-        return getColumnFamily(CliUtils.unescapeSQLString(astNode.getChild(0).getText()), cfDefs);
-    }
-
-    public static String getColumnFamily(String cfName, Iterable<CfDef> cfDefs)
-    {
-        int matches = 0;
-        String lastMatchedName = "";
-
-        for (CfDef cfDef : cfDefs)
-        {
-            if (cfDef.name.equals(cfName))
-            {
-                return cfName;
-            }
-            else if (cfDef.name.toUpperCase().equals(cfName.toUpperCase()))
-            {
-                lastMatchedName = cfDef.name;
-                matches++;
-            }
-        }
-
-        if (matches > 1 || matches == 0)
-            throw new RuntimeException(cfName + " not found in current keyspace.");
-
-        return lastMatchedName;
-    }
-
-    public static String getKeySpace(Tree statement, List<KsDef> keyspaces)
-    {
-        return getKeySpace(CliUtils.unescapeSQLString(statement.getChild(0).getText()), keyspaces);
-    }
-
-    public static String getKeySpace(String ksName, List<KsDef> keyspaces)
-    {
-        int matches = 0;
-        String lastMatchedName = "";
-
-        for (KsDef ksDef : keyspaces)
-        {
-            if (ksDef.name.equals(ksName))
-            {
-                return ksName;
-            }
-            else if (ksDef.name.toUpperCase().equals(ksName.toUpperCase()))
-            {
-                lastMatchedName = ksDef.name;
-                matches++;
-            }
-        }
-
-        if (matches > 1 || matches == 0)
-            throw new RuntimeException("Keyspace '" + ksName + "' not found.");
-
-        return lastMatchedName;
-    }
-
-    public static String getKey(Tree astNode)
-    {
-        return CliUtils.unescapeSQLString(astNode.getChild(1).getText());
-    }
-
-    public static int numColumnSpecifiers(Tree astNode)
-    {
-        // Skip over keyspace, column family and rowKey
-        return astNode.getChildCount() - 2;
-    }
-
-    // Returns the pos'th (0-based index) column specifier in the astNode
-    public static String getColumn(Tree astNode, int pos)
-    {
-        // Skip over keyspace, column family and rowKey
-        return CliUtils.unescapeSQLString(astNode.getChild(pos + 2).getText());
-    }
-
-}
diff --git a/src/java/org/apache/cassandra/cli/CliCompleter.java b/src/java/org/apache/cassandra/cli/CliCompleter.java
deleted file mode 100644
index a95de54..0000000
--- a/src/java/org/apache/cassandra/cli/CliCompleter.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- * 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.cli;
-
-import jline.SimpleCompletor;
-
-public class CliCompleter extends SimpleCompletor
-{
-    private static final String[] commands = {
-            "connect",
-            "describe keyspace",
-            "exit",
-            "help",
-            "quit",
-            "show cluster name",
-            "show keyspaces",
-            "show schema",
-            "show api version",
-            "create keyspace",
-            "create column family",
-            "drop keyspace",
-            "drop column family",
-            "rename keyspace",
-            "rename column family",
-            "consistencylevel",
-
-            "help connect",
-            "help describe keyspace",
-            "help exit",
-            "help help",
-            "help quit",
-            "help show cluster name",
-            "help show keyspaces",
-            "help show schema",
-            "help show api version",
-            "help create keyspace",
-            "help create column family",
-            "help drop keyspace",
-            "help drop column family",
-            "help rename keyspace",
-            "help rename column family",
-            "help get",
-            "help set",
-            "help del",
-            "help count",
-            "help list",
-            "help truncate",
-            "help consistencylevel"
-    };
-    private static final String[] keyspaceCommands = {
-            "get",
-            "set",
-            "count",
-            "del",
-            "list",
-            "truncate",
-            "incr",
-            "decr"
-    };
-
-    public CliCompleter()
-    {
-        super(commands);
-    }
-
-    String[] getKeyspaceCommands()
-    {
-        return keyspaceCommands;
-    }
-}
diff --git a/src/java/org/apache/cassandra/cli/CliMain.java b/src/java/org/apache/cassandra/cli/CliMain.java
deleted file mode 100644
index 862b1af..0000000
--- a/src/java/org/apache/cassandra/cli/CliMain.java
+++ /dev/null
@@ -1,407 +0,0 @@
-/*
- * 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.cli;
-
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileReader;
-import java.io.IOException;
-import java.nio.charset.CharacterCodingException;
-import java.util.*;
-
-import org.apache.cassandra.auth.IAuthenticator;
-import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.thrift.*;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.JVMStabilityInspector;
-import org.apache.thrift.TException;
-import org.apache.thrift.protocol.TBinaryProtocol;
-import org.apache.thrift.transport.TTransport;
-import jline.ConsoleReader;
-import jline.History;
-
-/**
- * Cassandra Command Line Interface (CLI) Main
- */
-public class CliMain
-{
-    public final static String OLD_HISTORYFILE = ".cassandra.history";
-    public final static String HISTORYFILE = "cli.history";
-
-    private static TTransport transport = null;
-    private static Cassandra.Client thriftClient = null;
-    public  static final CliSessionState sessionState = new CliSessionState();
-    private static CliClient cliClient;
-    private static final CliCompleter completer = new CliCompleter();
-    private static int lineNumber = 1;
-
-    /**
-     * Establish a thrift connection to cassandra instance
-     *
-     * @param server - hostname or IP of the server
-     * @param port   - Thrift port number
-     */
-    public static void connect(String server, int port)
-    {
-        if (transport != null)
-            transport.close();
-
-        try
-        {
-            transport = sessionState.transportFactory.openTransport(server, port);
-        }
-        catch (Exception e)
-        {
-            e.printStackTrace(sessionState.err);
-
-            String error = (e.getCause() == null) ? e.getMessage() : e.getCause().getMessage();
-            throw new RuntimeException("Exception connecting to " + server + "/" + port + ". Reason: " + error + ".");
-        }
-
-        TBinaryProtocol binaryProtocol = new TBinaryProtocol(transport, true, true);
-        thriftClient = new Cassandra.Client(binaryProtocol);
-        cliClient = new CliClient(sessionState, thriftClient);
-
-        if ((sessionState.username != null) && (sessionState.password != null))
-        {
-            // Authenticate
-            Map<String, String> credentials = new HashMap<String, String>();
-            credentials.put(IAuthenticator.USERNAME_KEY, sessionState.username);
-            credentials.put(IAuthenticator.PASSWORD_KEY, sessionState.password);
-            AuthenticationRequest authRequest = new AuthenticationRequest(credentials);
-            try
-            {
-                thriftClient.login(authRequest);
-                cliClient.setUsername(sessionState.username);
-            }
-            catch (AuthenticationException e)
-            {
-                thriftClient = null;
-                sessionState.err.println("Exception during authentication to the cassandra node, " +
-                                         "Verify the keyspace exists, and that you are using the correct credentials.");
-                return;
-            }
-            catch (AuthorizationException e)
-            {
-                thriftClient = null;
-                sessionState.err.println("You are not authorized to use keyspace: " + sessionState.keyspace);
-                return;
-            }
-            catch (TException e)
-            {
-                thriftClient = null;
-                sessionState.err.println("Login failure. Did you specify 'keyspace', 'username' and 'password'?");
-                return;
-            }
-        }
-
-        if (sessionState.keyspace != null)
-        {
-            try
-            {
-                sessionState.keyspace = CliCompiler.getKeySpace(sessionState.keyspace, thriftClient.describe_keyspaces());;
-                thriftClient.set_keyspace(sessionState.keyspace);
-                cliClient.setKeySpace(sessionState.keyspace);
-                updateCompletor(CliUtils.getCfNamesByKeySpace(cliClient.getKSMetaData(sessionState.keyspace)));
-            }
-            catch (InvalidRequestException | NotFoundException e)
-            {
-                sessionState.err.println("Keyspace " + sessionState.keyspace + " not found");
-                return;
-            }
-            catch (TException e)
-            {
-                sessionState.err.println("Did you specify 'keyspace'?");
-                return;
-            }
-        }
-
-        // Lookup the cluster name, this is to make it clear which cluster the user is connected to
-        String clusterName;
-
-        try
-        {
-            clusterName = thriftClient.describe_cluster_name();
-        }
-        catch (Exception e)
-        {
-            JVMStabilityInspector.inspectThrowable(e);
-            sessionState.err.println("Exception retrieving information about the cassandra node, check you have connected to the thrift port.");
-
-            e.printStackTrace(sessionState.err);
-
-            return;
-        }
-
-        sessionState.out.printf("Connected to: \"%s\" on %s/%d%n", clusterName, server, port);
-    }
-
-    /**
-     * Disconnect thrift connection to cassandra instance
-     */
-    public static void disconnect()
-    {
-        if (transport != null)
-        {
-            transport.close();
-            transport = null;
-        }
-    }
-
-    /**
-     * Checks whether the thrift client is connected.
-     * @return boolean - true when connected, false otherwise
-     */
-    public static boolean isConnected()
-    {
-        if (thriftClient == null)
-        {
-            sessionState.out.println("Not connected to a cassandra instance.");
-            return false;
-        }
-        return true;
-    }
-
-    public static void updateCompletor(Set<String> candidates)
-    {
-        Set<String> actions = new HashSet<String>();
-        for (String cf : candidates)
-        {
-            for (String cmd : completer.getKeyspaceCommands())
-                actions.add(String.format("%s %s", cmd, cf));
-        }
-
-        String[] strs = Arrays.copyOf(actions.toArray(), actions.toArray().length, String[].class);
-
-        completer.setCandidateStrings(strs);
-    }
-
-    public static void processStatement(String query) throws CharacterCodingException, TException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException
-    {
-        cliClient.executeCLIStatement(query);
-    }
-
-    public static void processStatementInteractive(String query)
-    {
-        try
-        {
-            cliClient.executeCLIStatement(query);
-        }
-        catch (Exception e)
-        {
-            String errorTemplate = sessionState.inFileMode() ? "Line " + lineNumber + " => " : "";
-
-            Throwable exception = (e.getCause() == null) ? e : e.getCause();
-            String message = (exception instanceof InvalidRequestException) ? ((InvalidRequestException) exception).getWhy() : e.getMessage();
-
-            sessionState.err.println(errorTemplate + message);
-
-            if (sessionState.debug || !(e instanceof RuntimeException))
-                e.printStackTrace(sessionState.err);
-
-            if (sessionState.batch || sessionState.inFileMode())
-            {
-                System.exit(4);
-            }
-        }
-        finally
-        {
-            lineNumber++;
-        }
-    }
-
-    public static void main(String args[]) throws IOException
-    {
-        // process command line arguments
-        CliOptions cliOptions = new CliOptions();
-        cliOptions.processArgs(sessionState, args);
-
-        // connect to cassandra server if host argument specified.
-        if (sessionState.hostName != null)
-        {
-            try
-            {
-                connect(sessionState.hostName, sessionState.thriftPort);
-            }
-            catch (RuntimeException e)
-            {
-                sessionState.err.println(e.getMessage());
-            }
-        }
-
-        if ( cliClient == null )
-        {
-            // Connection parameter was either invalid or not present.
-            // User must connect explicitly using the "connect" CLI statement.
-            cliClient = new CliClient(sessionState, null);
-        }
-
-        // load statements from file and process them
-        if (sessionState.inFileMode())
-        {
-            BufferedReader reader = null;
-
-            try
-            {
-                reader = new BufferedReader(new FileReader(sessionState.filename));
-                evaluateFileStatements(reader);
-            }
-            catch (IOException e)
-            {
-                sessionState.err.println(e.getMessage());
-                System.exit(1);
-            }
-            finally
-            {
-                FileUtils.closeQuietly(reader);
-            }      
-
-            return;
-        }
-
-        ConsoleReader reader = new ConsoleReader();
-
-        if (!sessionState.batch)
-        {
-            reader.addCompletor(completer);
-            reader.setBellEnabled(false);
-            File historyFile = handleHistoryFiles();
-
-            try
-            {
-                History history = new History(historyFile);
-                reader.setHistory(history);
-            }
-            catch (IOException exp)
-            {
-                sessionState.err.printf("Unable to open %s for writing", historyFile.getAbsolutePath());
-            }
-        }
-        else if (!sessionState.verbose) // if in batch mode but no verbose flag
-        {
-            sessionState.out.close();
-        }
-
-        cliClient.printBanner();
-
-        String prompt;
-        String line = "";
-        String currentStatement = "";
-        boolean inCompoundStatement = false;
-
-        while (line != null)
-        {
-            prompt = (inCompoundStatement) ? "...\t" : getPrompt(cliClient);
-
-            try
-            {
-                line = reader.readLine(prompt);
-            }
-            catch (IOException e)
-            {
-                // retry on I/O Exception
-            }
-
-            if (line == null)
-                return;
-
-            line = line.trim();
-
-            // skipping empty and comment lines
-            if (line.isEmpty() || line.startsWith("--"))
-                continue;
-
-            currentStatement += line;
-
-            if (line.endsWith(";") || line.equals("?"))
-            {
-                processStatementInteractive(currentStatement);
-                currentStatement = "";
-                inCompoundStatement = false;
-            }
-            else
-            {
-                currentStatement += " "; // ready for new line
-                inCompoundStatement = true;
-            }
-        }
-    }
-
-    private static File handleHistoryFiles()
-    {
-        File outputDir = FBUtilities.getToolsOutputDirectory();
-        File historyFile = new File(outputDir, HISTORYFILE);
-        File oldHistoryFile = new File(System.getProperty("user.home"), OLD_HISTORYFILE);
-        if(oldHistoryFile.exists())
-            FileUtils.renameWithConfirm(oldHistoryFile, historyFile);
-
-        return historyFile;
-    }
-
-    private static void evaluateFileStatements(BufferedReader reader) throws IOException
-    {
-        String line;
-        String currentStatement = "";
-
-        boolean commentedBlock = false;
-
-        while ((line = reader.readLine()) != null)
-        {
-            line = line.trim();
-
-            // skipping empty and comment lines
-            if (line.isEmpty() || line.startsWith("--"))
-                continue;
-
-            if (line.startsWith("/*"))
-                commentedBlock = true;
-
-            if (line.startsWith("*/") || line.endsWith("*/"))
-            {
-                commentedBlock = false;
-                continue;
-            }
-
-            if (commentedBlock) // skip commented lines
-                continue;
-
-            currentStatement += line;
-
-            if (line.endsWith(";"))
-            {
-                processStatementInteractive(currentStatement);
-                currentStatement = "";
-            }
-            else
-            {
-                currentStatement += " "; // ready for new line
-            }
-        }
-    }
-
-    /**
-     * Returns prompt for current connection
-     * @param client - currently connected client
-     * @return String - prompt with username and keyspace (if any)
-     */
-    private static String getPrompt(CliClient client)
-    {
-        return "[" + client.getUsername() + "@" + client.getKeySpace() + "] ";
-    }
-
-}
diff --git a/src/java/org/apache/cassandra/cli/CliOptions.java b/src/java/org/apache/cassandra/cli/CliOptions.java
deleted file mode 100644
index 7894bf9..0000000
--- a/src/java/org/apache/cassandra/cli/CliOptions.java
+++ /dev/null
@@ -1,319 +0,0 @@
-/*
- * 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.cli;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import com.google.common.base.Joiner;
-import org.apache.commons.cli.*;
-
-import org.apache.cassandra.config.EncryptionOptions;
-import org.apache.cassandra.thrift.ITransportFactory;
-import org.apache.cassandra.thrift.SSLTransportFactory;
-
-/**
- *
- * Used to process, and act upon the arguments passed to the CLI.
- *
- */
-public class CliOptions
-{
-    private static final CLIOptions options; // Info about command line options
-
-    // Name of the command line tool (used for error messages)
-    private static final String TOOL_NAME = "cassandra-cli";
-
-    // Command line options
-    private static final String HOST_OPTION = "host";
-    private static final String PORT_OPTION = "port";
-    private static final String TRANSPORT_FACTORY = "transport-factory";
-    private static final String DEBUG_OPTION = "debug";
-    private static final String USERNAME_OPTION = "username";
-    private static final String PASSWORD_OPTION = "password";
-    private static final String KEYSPACE_OPTION = "keyspace";
-    private static final String BATCH_OPTION = "batch";
-    private static final String HELP_OPTION = "help";
-    private static final String FILE_OPTION = "file";
-    private static final String JMX_PORT_OPTION = "jmxport";
-    private static final String JMX_USERNAME_OPTION = "jmxusername";
-    private static final String JMX_PASSWORD_OPTION = "jmxpassword";
-    private static final String VERBOSE_OPTION  = "verbose";
-
-    private static final String SSL_TRUSTSTORE = "truststore";
-    private static final String SSL_TRUSTSTORE_PW = "truststore-password";
-    private static final String SSL_PROTOCOL = "ssl-protocol";
-    private static final String SSL_ALGORITHM = "ssl-alg";
-    private static final String SSL_STORE_TYPE = "store-type";
-    private static final String SSL_CIPHER_SUITES = "ssl-ciphers";
-
-    // Default values for optional command line arguments
-    private static final String DEFAULT_HOST        = "127.0.0.1";
-    private static final int    DEFAULT_THRIFT_PORT = 9160;
-
-    // Register the command line options and their properties (such as
-    // whether they take an extra argument, etc.
-    static
-    {
-        options = new CLIOptions();
-
-        options.addOption("h",  HOST_OPTION,     "HOSTNAME", "cassandra server's host name");
-        options.addOption("p",  PORT_OPTION,     "PORT",     "cassandra server's thrift port");
-        options.addOption("u",  USERNAME_OPTION, "USERNAME", "user name for cassandra authentication");
-        options.addOption("pw", PASSWORD_OPTION, "PASSWORD", "password for cassandra authentication");
-        options.addOption("k",  KEYSPACE_OPTION, "KEYSPACE", "cassandra keyspace user is authenticated against");
-        options.addOption("f",  FILE_OPTION,     "FILENAME", "load statements from the specific file");
-        options.addOption(null, JMX_PORT_OPTION, "JMX-PORT", "JMX service port");
-        options.addOption(null, JMX_USERNAME_OPTION, "JMX-USERNAME", "JMX service username");
-        options.addOption(null, JMX_PASSWORD_OPTION, "JMX-PASSWORD", "JMX service password");
-        options.addOption("tf", TRANSPORT_FACTORY, "TRANSPORT-FACTORY", "Fully-qualified ITransportFactory class name for creating a connection to cassandra");
-
-        // ssl connection-related options
-        options.addOption("ts", SSL_TRUSTSTORE, "TRUSTSTORE", "SSL: full path to truststore");
-        options.addOption("tspw", SSL_TRUSTSTORE_PW, "TRUSTSTORE-PASSWORD", "SSL: password of the truststore");
-        options.addOption("prtcl", SSL_PROTOCOL, "PROTOCOL", "SSL: connections protocol to use (default: TLS)");
-        options.addOption("alg", SSL_ALGORITHM, "ALGORITHM", "SSL: algorithm (default: SunX509)");
-        options.addOption("st", SSL_STORE_TYPE, "STORE-TYPE", "SSL: type of store");
-        options.addOption("ciphers", SSL_CIPHER_SUITES, "CIPHER-SUITES", "SSL: comma-separated list of encryption suites to use");
-
-        // options without argument
-        options.addOption("B",  BATCH_OPTION,   "enabled batch mode (suppress output; errors are fatal)");
-        options.addOption(null, DEBUG_OPTION,   "display stack-traces (NOTE: We print strack-traces in the places where it makes sense even without --debug)");
-        options.addOption("?",  HELP_OPTION,    "usage help");
-        options.addOption("v",  VERBOSE_OPTION, "verbose output when using batch mode");
-    }
-
-    private static void printUsage()
-    {
-        new HelpFormatter().printHelp(TOOL_NAME, options);
-    }
-
-    public void processArgs(CliSessionState css, String[] args)
-    {
-        CommandLineParser parser = new GnuParser();
-
-        try
-        {
-            CommandLine cmd = parser.parse(options, args, false);
-
-            if (cmd.hasOption(HOST_OPTION))
-            {
-                css.hostName = cmd.getOptionValue(HOST_OPTION);
-            }
-            else
-            {
-                css.hostName = DEFAULT_HOST;
-            }
-
-            if (cmd.hasOption(DEBUG_OPTION))
-            {
-                css.debug = true;
-            }
-
-            // Look for optional args.
-            if (cmd.hasOption(PORT_OPTION))
-            {
-                css.thriftPort = Integer.parseInt(cmd.getOptionValue(PORT_OPTION));
-            }
-            else
-            {
-                css.thriftPort = DEFAULT_THRIFT_PORT;
-            }
-
-            // Look for authentication credentials (username and password)
-            if (cmd.hasOption(USERNAME_OPTION))
-            {
-                css.username = cmd.getOptionValue(USERNAME_OPTION);
-            }
-
-            if (cmd.hasOption(PASSWORD_OPTION))
-            {
-                css.password = cmd.getOptionValue(PASSWORD_OPTION);
-            }
-
-            // Look for keyspace
-            if (cmd.hasOption(KEYSPACE_OPTION))
-            {
-                css.keyspace = cmd.getOptionValue(KEYSPACE_OPTION);
-            }
-
-            if (cmd.hasOption(BATCH_OPTION))
-            {
-                css.batch = true;
-            }
-
-            if (cmd.hasOption(FILE_OPTION))
-            {
-                css.filename = cmd.getOptionValue(FILE_OPTION);
-            }
-
-            if (cmd.hasOption(JMX_PORT_OPTION))
-            {
-                css.jmxPort = Integer.parseInt(cmd.getOptionValue(JMX_PORT_OPTION));
-            }
-
-            if (cmd.hasOption(JMX_USERNAME_OPTION))
-            {
-                css.jmxUsername = cmd.getOptionValue(JMX_USERNAME_OPTION);
-            }
-
-            if (cmd.hasOption(JMX_PASSWORD_OPTION))
-            {
-                css.jmxPassword = cmd.getOptionValue(JMX_PASSWORD_OPTION);
-            }
-
-            if (cmd.hasOption(HELP_OPTION))
-            {
-                printUsage();
-                System.exit(1);
-            }
-
-            if (cmd.hasOption(VERBOSE_OPTION))
-            {
-                css.verbose = true;
-            }
-
-            if(cmd.hasOption(SSL_TRUSTSTORE))
-            {
-                css.encOptions.truststore = cmd.getOptionValue(SSL_TRUSTSTORE);
-            }
-
-            if(cmd.hasOption(SSL_TRUSTSTORE_PW))
-            {
-                css.encOptions.truststore_password = cmd.getOptionValue(SSL_TRUSTSTORE_PW);
-            }
-
-            if(cmd.hasOption(SSL_PROTOCOL))
-            {
-                css.encOptions.protocol = cmd.getOptionValue(SSL_PROTOCOL);
-            }
-
-            if(cmd.hasOption(SSL_ALGORITHM))
-            {
-                css.encOptions.algorithm = cmd.getOptionValue(SSL_ALGORITHM);
-            }
-
-            if(cmd.hasOption(SSL_STORE_TYPE))
-            {
-                css.encOptions.store_type = cmd.getOptionValue(SSL_STORE_TYPE);
-            }
-
-            if(cmd.hasOption(SSL_CIPHER_SUITES))
-            {
-                css.encOptions.cipher_suites = cmd.getOptionValue(SSL_CIPHER_SUITES).split(",");
-            }
-
-            if (cmd.hasOption(TRANSPORT_FACTORY))
-            {
-                css.transportFactory = validateAndSetTransportFactory(cmd.getOptionValue(TRANSPORT_FACTORY));
-                configureTransportFactory(css.transportFactory, css.encOptions);
-            }
-
-            // Abort if there are any unrecognized arguments left
-            if (cmd.getArgs().length > 0)
-            {
-                System.err.printf("Unknown argument: %s%n", cmd.getArgs()[0]);
-                System.err.println();
-                printUsage();
-                System.exit(1);
-            }
-        }
-        catch (ParseException e)
-        {
-            System.err.println(e.getMessage());
-            System.err.println();
-            printUsage();
-            System.exit(1);
-        }
-    }
-
-    private static class CLIOptions extends Options
-    {
-        /**
-         * Add option with argument and argument name
-         * @param opt shortcut for option name
-         * @param longOpt complete option name
-         * @param argName argument name
-         * @param description description of the option
-         * @return updated Options object
-         */
-        public Options addOption(String opt, String longOpt, String argName, String description)
-        {
-            Option option = new Option(opt, longOpt, true, description);
-            option.setArgName(argName);
-
-            return addOption(option);
-        }
-
-        /**
-         * Add option without argument
-         * @param opt shortcut for option name
-         * @param longOpt complete option name
-         * @param description description of the option
-         * @return updated Options object
-         */
-        public Options addOption(String opt, String longOpt, String description)
-        {
-            return addOption(new Option(opt, longOpt, false, description));
-        }
-    }
-
-    private static ITransportFactory validateAndSetTransportFactory(String transportFactory)
-    {
-        try
-        {
-            Class<?> factory = Class.forName(transportFactory);
-            if (!ITransportFactory.class.isAssignableFrom(factory))
-                throw new IllegalArgumentException(String.format("transport factory '%s' " +
-                                                                 "not derived from ITransportFactory", transportFactory));
-            return (ITransportFactory) factory.newInstance();
-        }
-        catch (Exception e)
-        {
-            throw new IllegalArgumentException(String.format("Cannot create a transport factory '%s'.", transportFactory), e);
-        }
-    }
-
-    private static void configureTransportFactory(ITransportFactory transportFactory, EncryptionOptions encOptions)
-    {
-        Map<String, String> options = new HashMap<>();
-        // If the supplied factory supports the same set of options as our SSL impl, set those
-        if (transportFactory.supportedOptions().contains(SSLTransportFactory.TRUSTSTORE))
-            options.put(SSLTransportFactory.TRUSTSTORE, encOptions.truststore);
-        if (transportFactory.supportedOptions().contains(SSLTransportFactory.TRUSTSTORE_PASSWORD))
-            options.put(SSLTransportFactory.TRUSTSTORE_PASSWORD, encOptions.truststore_password);
-        if (transportFactory.supportedOptions().contains(SSLTransportFactory.PROTOCOL))
-            options.put(SSLTransportFactory.PROTOCOL, encOptions.protocol);
-        if (transportFactory.supportedOptions().contains(SSLTransportFactory.CIPHER_SUITES))
-            options.put(SSLTransportFactory.CIPHER_SUITES, Joiner.on(',').join(encOptions.cipher_suites));
-
-        if (transportFactory.supportedOptions().contains(SSLTransportFactory.KEYSTORE)
-                && encOptions.require_client_auth)
-            options.put(SSLTransportFactory.KEYSTORE, encOptions.keystore);
-        if (transportFactory.supportedOptions().contains(SSLTransportFactory.KEYSTORE_PASSWORD)
-                && encOptions.require_client_auth)
-            options.put(SSLTransportFactory.KEYSTORE_PASSWORD, encOptions.keystore_password);
-
-        // Now check if any of the factory's supported options are set as system properties
-        for (String optionKey : transportFactory.supportedOptions())
-            if (System.getProperty(optionKey) != null)
-                options.put(optionKey, System.getProperty(optionKey));
-
-        transportFactory.setOptions(options);
-    }
-}
diff --git a/src/java/org/apache/cassandra/cli/CliSessionState.java b/src/java/org/apache/cassandra/cli/CliSessionState.java
deleted file mode 100644
index 2bfe7fc..0000000
--- a/src/java/org/apache/cassandra/cli/CliSessionState.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/*
- * 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.cli;
-
-import java.io.InputStream;
-import java.io.PrintStream;
-
-import org.apache.cassandra.config.EncryptionOptions;
-import org.apache.cassandra.config.EncryptionOptions.ClientEncryptionOptions;
-import org.apache.cassandra.thrift.ITransportFactory;
-import org.apache.cassandra.thrift.TFramedTransportFactory;
-import org.apache.cassandra.tools.NodeProbe;
-import org.apache.cassandra.utils.JVMStabilityInspector;
-
-/**
- * Used to hold the state for the CLI.
- */
-public class CliSessionState
-{
-
-    public String  hostName;      // cassandra server name
-    public int     thriftPort;    // cassandra server's thrift port
-    public boolean debug = false; // print stack traces when errors occur in the CLI
-    public String  username;      // cassandra login name (if password-based authenticator is used)
-    public String  password;      // cassandra login password (if password-based authenticator is used)
-    public String  keyspace;      // cassandra keyspace user is authenticating
-    public boolean batch = false; // enable/disable batch processing mode
-    public String  filename = ""; // file to read commands from
-    public int     jmxPort = 7199;// JMX service port
-    public String  jmxUsername;   // JMX service username
-    public String  jmxPassword;   // JMX service password
-    public boolean verbose = false; // verbose output
-    public ITransportFactory transportFactory = new TFramedTransportFactory();
-    public EncryptionOptions encOptions = new ClientEncryptionOptions();
-
-    /*
-     * Streams to read/write from
-     */
-    public InputStream in;
-    public PrintStream out;
-    public PrintStream err;
-
-    public CliSessionState()
-    {
-        in = System.in;
-        out = System.out;
-        err = System.err;
-    }
-
-    public void setOut(PrintStream newOut)
-    {
-        this.out = newOut;
-    }
-
-    public void setErr(PrintStream newErr)
-    {
-        this.err = newErr;
-    }
-
-    public boolean inFileMode()
-    {
-        return !this.filename.isEmpty();
-    }
-
-    public NodeProbe getNodeProbe()
-    {
-        try
-        {
-            return jmxUsername != null && jmxPassword != null
-                   ? new NodeProbe(hostName, jmxPort, jmxUsername, jmxPassword)
-                   : new NodeProbe(hostName, jmxPort);
-        }
-        catch (Exception e)
-        {
-            JVMStabilityInspector.inspectThrowable(e);
-            err.printf("WARNING: Could not connect to the JMX on %s:%d - some information won't be shown.%n%n", hostName, jmxPort);
-        }
-
-        return null;
-    }
-}
diff --git a/src/java/org/apache/cassandra/cli/CliUserHelp.java b/src/java/org/apache/cassandra/cli/CliUserHelp.java
deleted file mode 100644
index aa4192c..0000000
--- a/src/java/org/apache/cassandra/cli/CliUserHelp.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * 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.cli;
-
-import java.util.List;
-
-public class CliUserHelp
-{
-    public String banner;
-
-    public String help;
-
-    public List<CliCommandHelp> commands;
-}
diff --git a/src/java/org/apache/cassandra/cli/CliUtils.java b/src/java/org/apache/cassandra/cli/CliUtils.java
deleted file mode 100644
index d0c8e37..0000000
--- a/src/java/org/apache/cassandra/cli/CliUtils.java
+++ /dev/null
@@ -1,128 +0,0 @@
-/*
- * 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.cli;
-
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.cassandra.thrift.CfDef;
-import org.apache.cassandra.thrift.IndexOperator;
-import org.apache.cassandra.thrift.KsDef;
-import org.apache.commons.lang3.StringEscapeUtils;
-
-public class CliUtils
-{
-    /**
-     * Strips leading and trailing "'" characters, and handles
-     * and escaped characters such as \n, \r, etc.
-     * @param b - string to unescape
-     * @return String - unexspaced string
-     */
-    public static String unescapeSQLString(String b)
-    {
-        if (b.charAt(0) == '\'' && b.charAt(b.length()-1) == '\'')
-            b = b.substring(1, b.length()-1);
-        return StringEscapeUtils.unescapeJava(b);
-    }
-
-    public static String escapeSQLString(String b)
-    {
-        // single quotes are not escaped in java, need to be for cli
-        return StringEscapeUtils.escapeJava(b).replace("\'", "\\'");
-    }
-
-    public static String maybeEscapeName(String name)
-    {
-        return Character.isLetter(name.charAt(0)) ? name : "\'" + name + "\'";
-    }
-
-    /**
-     * Returns IndexOperator from string representation
-     * @param operator - string representing IndexOperator (=, >=, >, <, <=)
-     * @return IndexOperator - enum value of IndexOperator or null if not found
-     */
-    public static IndexOperator getIndexOperator(String operator)
-    {
-        if (operator.equals("="))
-        {
-            return IndexOperator.EQ;
-        }
-        else if (operator.equals(">="))
-        {
-            return IndexOperator.GTE;
-        }
-        else if (operator.equals(">"))
-        {
-            return IndexOperator.GT;
-        }
-        else if (operator.equals("<"))
-        {
-            return IndexOperator.LT;
-        }
-        else if (operator.equals("<="))
-        {
-            return IndexOperator.LTE;
-        }
-
-        return null;
-    }
-
-    /**
-     * Returns set of column family names in specified keySpace.
-     * @param keySpace - keyspace definition to get column family names from.
-     * @return Set - column family names
-     */
-    public static Set<String> getCfNamesByKeySpace(KsDef keySpace)
-    {
-        Set<String> names = new LinkedHashSet<String>();
-
-        for (CfDef cfDef : keySpace.getCf_defs())
-        {
-            names.add(cfDef.getName());
-        }
-
-        return names;
-    }
-
-    /**
-     * Parse the statement from cli and return KsDef
-     *
-     * @param keyspaceName - name of the keyspace to lookup
-     * @param keyspaces - List of known keyspaces
-     *
-     * @return metadata about keyspace or null
-     */
-    public static KsDef getKeySpaceDef(String keyspaceName, List<KsDef> keyspaces)
-    {
-        keyspaceName = keyspaceName.toUpperCase();
-
-        for (KsDef ksDef : keyspaces)
-        {
-            if (ksDef.name.toUpperCase().equals(keyspaceName))
-                return ksDef;
-        }
-
-        return null;
-    }
-
-    public static String quote(String str)
-    {
-        return String.format("'%s'", str);
-    }
-}
diff --git a/src/java/org/apache/cassandra/client/RingCache.java b/src/java/org/apache/cassandra/client/RingCache.java
index d58d5d9..5196bce 100644
--- a/src/java/org/apache/cassandra/client/RingCache.java
+++ b/src/java/org/apache/cassandra/client/RingCache.java
@@ -28,7 +28,6 @@
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.hadoop.ConfigHelper;
 import org.apache.cassandra.thrift.Cassandra;
-import org.apache.cassandra.thrift.InvalidRequestException;
 import org.apache.cassandra.thrift.TokenRange;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.thrift.TException;
@@ -75,7 +74,7 @@
             {
                 Token left = partitioner.getTokenFactory().fromString(range.start_token);
                 Token right = partitioner.getTokenFactory().fromString(range.end_token);
-                Range<Token> r = new Range<Token>(left, right, partitioner);
+                Range<Token> r = new Range<Token>(left, right);
                 for (String host : range.endpoints)
                 {
                     try
@@ -88,13 +87,13 @@
                 }
             }
         }
-        catch (InvalidRequestException | IOException e)
+        catch (IOException e)
         {
             throw new RuntimeException(e);
         }
         catch (TException e)
         {
-            logger.debug("Error contacting seed list {} {}", ConfigHelper.getOutputInitialAddress(conf), e.getMessage());
+            logger.trace("Error contacting seed list {} {}", ConfigHelper.getOutputInitialAddress(conf), e.getMessage());
         }
     }
 
diff --git a/src/java/org/apache/cassandra/concurrent/AbstractLocalAwareExecutorService.java b/src/java/org/apache/cassandra/concurrent/AbstractLocalAwareExecutorService.java
new file mode 100644
index 0000000..088b43e
--- /dev/null
+++ b/src/java/org/apache/cassandra/concurrent/AbstractLocalAwareExecutorService.java
@@ -0,0 +1,229 @@
+/*
+ * 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.concurrent;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.tracing.TraceState;
+import org.apache.cassandra.tracing.Tracing;
+import org.apache.cassandra.utils.concurrent.SimpleCondition;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+
+import static org.apache.cassandra.tracing.Tracing.isTracing;
+
+public abstract class AbstractLocalAwareExecutorService implements LocalAwareExecutorService
+{
+    private static final Logger logger = LoggerFactory.getLogger(AbstractLocalAwareExecutorService.class);
+
+    protected abstract void addTask(FutureTask<?> futureTask);
+    protected abstract void onCompletion();
+
+    /** Task Submission / Creation / Objects **/
+
+    public <T> FutureTask<T> submit(Callable<T> task)
+    {
+        return submit(newTaskFor(task));
+    }
+
+    public FutureTask<?> submit(Runnable task)
+    {
+        return submit(newTaskFor(task, null));
+    }
+
+    public <T> FutureTask<T> submit(Runnable task, T result)
+    {
+        return submit(newTaskFor(task, result));
+    }
+
+    public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit) throws InterruptedException
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    public <T> T invokeAny(Collection<? extends Callable<T>> tasks) throws InterruptedException, ExecutionException
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    public <T> T invokeAny(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    protected <T> FutureTask<T> newTaskFor(Runnable runnable, T result)
+    {
+        return newTaskFor(runnable, result, ExecutorLocals.create());
+    }
+
+    protected <T> FutureTask<T> newTaskFor(Runnable runnable, T result, ExecutorLocals locals)
+    {
+        if (locals != null)
+        {
+            if (runnable instanceof LocalSessionFutureTask)
+                return (LocalSessionFutureTask<T>) runnable;
+            return new LocalSessionFutureTask<T>(runnable, result, locals);
+        }
+        if (runnable instanceof FutureTask)
+            return (FutureTask<T>) runnable;
+        return new FutureTask<>(runnable, result);
+    }
+
+    protected <T> FutureTask<T> newTaskFor(Callable<T> callable)
+    {
+        if (isTracing())
+        {
+            if (callable instanceof LocalSessionFutureTask)
+                return (LocalSessionFutureTask<T>) callable;
+            return new LocalSessionFutureTask<T>(callable, ExecutorLocals.create());
+        }
+        if (callable instanceof FutureTask)
+            return (FutureTask<T>) callable;
+        return new FutureTask<>(callable);
+    }
+
+    private class LocalSessionFutureTask<T> extends FutureTask<T>
+    {
+        private final ExecutorLocals locals;
+
+        public LocalSessionFutureTask(Callable<T> callable, ExecutorLocals locals)
+        {
+            super(callable);
+            this.locals = locals;
+        }
+
+        public LocalSessionFutureTask(Runnable runnable, T result, ExecutorLocals locals)
+        {
+            super(runnable, result);
+            this.locals = locals;
+        }
+
+        public void run()
+        {
+            ExecutorLocals old = ExecutorLocals.create();
+            ExecutorLocals.set(locals);
+            try
+            {
+                super.run();
+            }
+            finally
+            {
+                ExecutorLocals.set(old);
+            }
+        }
+    }
+
+    class FutureTask<T> extends SimpleCondition implements Future<T>, Runnable
+    {
+        private boolean failure;
+        private Object result = this;
+        private final Callable<T> callable;
+
+        public FutureTask(Callable<T> callable)
+        {
+            this.callable = callable;
+        }
+        public FutureTask(Runnable runnable, T result)
+        {
+            this(Executors.callable(runnable, result));
+        }
+
+        public void run()
+        {
+            try
+            {
+                result = callable.call();
+            }
+            catch (Throwable t)
+            {
+                JVMStabilityInspector.inspectThrowable(t);
+                logger.warn("Uncaught exception on thread {}: {}", Thread.currentThread(), t);
+                result = t;
+                failure = true;
+            }
+            finally
+            {
+                signalAll();
+                onCompletion();
+            }
+        }
+
+        public boolean cancel(boolean mayInterruptIfRunning)
+        {
+            return false;
+        }
+
+        public boolean isCancelled()
+        {
+            return false;
+        }
+
+        public boolean isDone()
+        {
+            return isSignaled();
+        }
+
+        public T get() throws InterruptedException, ExecutionException
+        {
+            await();
+            Object result = this.result;
+            if (failure)
+                throw new ExecutionException((Throwable) result);
+            return (T) result;
+        }
+
+        public T get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException
+        {
+            await(timeout, unit);
+            Object result = this.result;
+            if (failure)
+                throw new ExecutionException((Throwable) result);
+            return (T) result;
+        }
+    }
+
+    private <T> FutureTask<T> submit(FutureTask<T> task)
+    {
+        addTask(task);
+        return task;
+    }
+
+    public void execute(Runnable command)
+    {
+        addTask(newTaskFor(command, ExecutorLocals.create()));
+    }
+
+    public void execute(Runnable command, ExecutorLocals locals)
+    {
+        addTask(newTaskFor(command, null, locals));
+    }
+}
diff --git a/src/java/org/apache/cassandra/concurrent/AbstractTracingAwareExecutorService.java b/src/java/org/apache/cassandra/concurrent/AbstractTracingAwareExecutorService.java
deleted file mode 100644
index fb753b0..0000000
--- a/src/java/org/apache/cassandra/concurrent/AbstractTracingAwareExecutorService.java
+++ /dev/null
@@ -1,229 +0,0 @@
-/*
- * 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.concurrent;
-
-import java.util.Collection;
-import java.util.List;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.tracing.TraceState;
-import org.apache.cassandra.tracing.Tracing;
-import org.apache.cassandra.utils.concurrent.SimpleCondition;
-import org.apache.cassandra.utils.JVMStabilityInspector;
-
-import static org.apache.cassandra.tracing.Tracing.isTracing;
-
-public abstract class AbstractTracingAwareExecutorService implements TracingAwareExecutorService
-{
-    private static final Logger logger = LoggerFactory.getLogger(AbstractTracingAwareExecutorService.class);
-
-    protected abstract void addTask(FutureTask<?> futureTask);
-    protected abstract void onCompletion();
-
-    /** Task Submission / Creation / Objects **/
-
-    public <T> FutureTask<T> submit(Callable<T> task)
-    {
-        return submit(newTaskFor(task));
-    }
-
-    public FutureTask<?> submit(Runnable task)
-    {
-        return submit(newTaskFor(task, null));
-    }
-
-    public <T> FutureTask<T> submit(Runnable task, T result)
-    {
-        return submit(newTaskFor(task, result));
-    }
-
-    public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks)
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit) throws InterruptedException
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    public <T> T invokeAny(Collection<? extends Callable<T>> tasks) throws InterruptedException, ExecutionException
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    public <T> T invokeAny(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    protected <T> FutureTask<T> newTaskFor(Runnable runnable, T result)
-    {
-        return newTaskFor(runnable, result, Tracing.instance.get());
-    }
-
-    protected <T> FutureTask<T> newTaskFor(Runnable runnable, T result, TraceState traceState)
-    {
-        if (traceState != null)
-        {
-            if (runnable instanceof TraceSessionFutureTask)
-                return (TraceSessionFutureTask<T>) runnable;
-            return new TraceSessionFutureTask<T>(runnable, result, traceState);
-        }
-        if (runnable instanceof FutureTask)
-            return (FutureTask<T>) runnable;
-        return new FutureTask<>(runnable, result);
-    }
-
-    protected <T> FutureTask<T> newTaskFor(Callable<T> callable)
-    {
-        if (isTracing())
-        {
-            if (callable instanceof TraceSessionFutureTask)
-                return (TraceSessionFutureTask<T>) callable;
-            return new TraceSessionFutureTask<T>(callable, Tracing.instance.get());
-        }
-        if (callable instanceof FutureTask)
-            return (FutureTask<T>) callable;
-        return new FutureTask<>(callable);
-    }
-
-    private class TraceSessionFutureTask<T> extends FutureTask<T>
-    {
-        private final TraceState state;
-
-        public TraceSessionFutureTask(Callable<T> callable, TraceState state)
-        {
-            super(callable);
-            this.state = state;
-        }
-
-        public TraceSessionFutureTask(Runnable runnable, T result, TraceState state)
-        {
-            super(runnable, result);
-            this.state = state;
-        }
-
-        public void run()
-        {
-            TraceState oldState = Tracing.instance.get();
-            Tracing.instance.set(state);
-            try
-            {
-                super.run();
-            }
-            finally
-            {
-                Tracing.instance.set(oldState);
-            }
-        }
-    }
-
-    class FutureTask<T> extends SimpleCondition implements Future<T>, Runnable
-    {
-        private boolean failure;
-        private Object result = this;
-        private final Callable<T> callable;
-
-        public FutureTask(Callable<T> callable)
-        {
-            this.callable = callable;
-        }
-        public FutureTask(Runnable runnable, T result)
-        {
-            this(Executors.callable(runnable, result));
-        }
-
-        public void run()
-        {
-            try
-            {
-                result = callable.call();
-            }
-            catch (Throwable t)
-            {
-                JVMStabilityInspector.inspectThrowable(t);
-                logger.warn("Uncaught exception on thread {}: {}", Thread.currentThread(), t);
-                result = t;
-                failure = true;
-            }
-            finally
-            {
-                signalAll();
-                onCompletion();
-            }
-        }
-
-        public boolean cancel(boolean mayInterruptIfRunning)
-        {
-            return false;
-        }
-
-        public boolean isCancelled()
-        {
-            return false;
-        }
-
-        public boolean isDone()
-        {
-            return isSignaled();
-        }
-
-        public T get() throws InterruptedException, ExecutionException
-        {
-            await();
-            Object result = this.result;
-            if (failure)
-                throw new ExecutionException((Throwable) result);
-            return (T) result;
-        }
-
-        public T get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException
-        {
-            await(timeout, unit);
-            Object result = this.result;
-            if (failure)
-                throw new ExecutionException((Throwable) result);
-            return (T) result;
-        }
-    }
-
-    private <T> FutureTask<T> submit(FutureTask<T> task)
-    {
-        addTask(task);
-        return task;
-    }
-
-    public void execute(Runnable command)
-    {
-        addTask(newTaskFor(command, null));
-    }
-
-    public void execute(Runnable command, TraceState state)
-    {
-        addTask(newTaskFor(command, null, state));
-    }
-}
diff --git a/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java b/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java
index ea04af3..1fb0690 100644
--- a/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java
+++ b/src/java/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutor.java
@@ -44,7 +44,7 @@
  *   threads and the queue is full, we want the enqueuer to block.  But to allow the number of threads to drop if a
  *   stage is less busy, core thread timeout is enabled.
  */
-public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor implements TracingAwareExecutorService
+public class DebuggableThreadPoolExecutor extends ThreadPoolExecutor implements LocalAwareExecutorService
 {
     protected static final Logger logger = LoggerFactory.getLogger(DebuggableThreadPoolExecutor.class);
     public static final RejectedExecutionHandler blockingExecutionHandler = new RejectedExecutionHandler()
@@ -100,6 +100,21 @@
     }
 
     /**
+     * Creates a thread pool that creates new threads as needed, but
+     * will reuse previously constructed threads when they are
+     * available.
+     * @param threadPoolName the name of the threads created by this executor
+     * @return The new DebuggableThreadPoolExecutor
+     */
+    public static DebuggableThreadPoolExecutor createCachedThreadpoolWithMaxSize(String threadPoolName)
+    {
+        return new DebuggableThreadPoolExecutor(0, Integer.MAX_VALUE,
+                                                60L, TimeUnit.SECONDS,
+                                                new SynchronousQueue<Runnable>(),
+                                                new NamedThreadFactory(threadPoolName));
+    }
+
+    /**
      * Returns a ThreadPoolExecutor with a fixed number of threads.
      * When all threads are actively executing tasks, new tasks are queued.
      * If (most) threads are expected to be idle most of the time, prefer createWithMaxSize() instead.
@@ -131,11 +146,11 @@
     protected void onFinalAccept(Runnable task) {}
     protected void onFinalRejection(Runnable task) {}
 
-    public void execute(Runnable command, TraceState state)
+    public void execute(Runnable command, ExecutorLocals locals)
     {
-        super.execute(state == null || command instanceof TraceSessionWrapper
+        super.execute(locals == null || command instanceof LocalSessionWrapper
                       ? command
-                      : new TraceSessionWrapper<Object>(command, state));
+                      : new LocalSessionWrapper<Object>(command, locals));
     }
 
     public void maybeExecuteImmediately(Runnable command)
@@ -147,17 +162,17 @@
     @Override
     public void execute(Runnable command)
     {
-        super.execute(isTracing() && !(command instanceof TraceSessionWrapper)
-                      ? new TraceSessionWrapper<Object>(Executors.callable(command, null))
+        super.execute(isTracing() && !(command instanceof LocalSessionWrapper)
+                      ? new LocalSessionWrapper<Object>(Executors.callable(command, null))
                       : command);
     }
 
     @Override
     protected <T> RunnableFuture<T> newTaskFor(Runnable runnable, T result)
     {
-        if (isTracing() && !(runnable instanceof TraceSessionWrapper))
+        if (isTracing() && !(runnable instanceof LocalSessionWrapper))
         {
-            return new TraceSessionWrapper<T>(Executors.callable(runnable, result));
+            return new LocalSessionWrapper<T>(Executors.callable(runnable, result));
         }
         return super.newTaskFor(runnable, result);
     }
@@ -165,9 +180,9 @@
     @Override
     protected <T> RunnableFuture<T> newTaskFor(Callable<T> callable)
     {
-        if (isTracing() && !(callable instanceof TraceSessionWrapper))
+        if (isTracing() && !(callable instanceof LocalSessionWrapper))
         {
-            return new TraceSessionWrapper<T>(callable);
+            return new LocalSessionWrapper<T>(callable);
         }
         return super.newTaskFor(callable);
     }
@@ -183,9 +198,9 @@
 
     protected static void maybeResetTraceSessionWrapper(Runnable r)
     {
-        if (r instanceof TraceSessionWrapper)
+        if (r instanceof LocalSessionWrapper)
         {
-            TraceSessionWrapper tsw = (TraceSessionWrapper) r;
+            LocalSessionWrapper tsw = (LocalSessionWrapper) r;
             // we have to reset trace state as its presence is what denotes the current thread is tracing
             // and if left this thread might start tracing unrelated tasks
             tsw.reset();
@@ -195,8 +210,8 @@
     @Override
     protected void beforeExecute(Thread t, Runnable r)
     {
-        if (r instanceof TraceSessionWrapper)
-            ((TraceSessionWrapper) r).setupContext();
+        if (r instanceof LocalSessionWrapper)
+            ((LocalSessionWrapper) r).setupContext();
 
         super.beforeExecute(t, r);
     }
@@ -251,7 +266,7 @@
             }
             catch (CancellationException e)
             {
-                logger.debug("Task cancelled", e);
+                logger.trace("Task cancelled", e);
             }
             catch (ExecutionException e)
             {
@@ -263,35 +278,35 @@
     }
 
     /**
-     * Used to wrap a Runnable or Callable passed to submit or execute so we can clone the TraceSessionContext and move
-     * it into the worker thread.
+     * Used to wrap a Runnable or Callable passed to submit or execute so we can clone the ExecutorLocals and move
+     * them into the worker thread.
      *
      * @param <T>
      */
-    private static class TraceSessionWrapper<T> extends FutureTask<T>
+    private static class LocalSessionWrapper<T> extends FutureTask<T>
     {
-        private final TraceState state;
+        private final ExecutorLocals locals;
 
-        public TraceSessionWrapper(Callable<T> callable)
+        public LocalSessionWrapper(Callable<T> callable)
         {
             super(callable);
-            state = Tracing.instance.get();
+            locals = ExecutorLocals.create();
         }
 
-        public TraceSessionWrapper(Runnable command, TraceState state)
+        public LocalSessionWrapper(Runnable command, ExecutorLocals locals)
         {
             super(command, null);
-            this.state = state;
+            this.locals = locals;
         }
 
         private void setupContext()
         {
-            Tracing.instance.set(state);
+            ExecutorLocals.set(locals);
         }
 
         private void reset()
         {
-            Tracing.instance.set(null);
+            ExecutorLocals.set(null);
         }
     }
 }
diff --git a/src/java/org/apache/cassandra/concurrent/ExecutorLocal.java b/src/java/org/apache/cassandra/concurrent/ExecutorLocal.java
new file mode 100644
index 0000000..47826f3
--- /dev/null
+++ b/src/java/org/apache/cassandra/concurrent/ExecutorLocal.java
@@ -0,0 +1,44 @@
+/*
+ * 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.concurrent;
+
+import org.apache.cassandra.service.ClientWarn;
+import org.apache.cassandra.tracing.TraceState;
+import org.apache.cassandra.tracing.Tracing;
+
+public interface ExecutorLocal<T>
+{
+    ExecutorLocal[] all = { Tracing.instance, ClientWarn.instance };
+
+    /**
+     * This is called when scheduling the task, and also before calling {@link ExecutorLocal#set(T)} when running on a
+     * executor thread.
+     *
+     * @return The thread-local value that we want to copy across executor boundaries; may be null if not set.
+     */
+    T get();
+
+    /**
+     * Before a task has been run, this will be called with the value from the thread that scheduled the task, and after
+     * the task is finished, the value that was previously retrieved from this thread is restored.
+     *
+     * @param value Value to use for the executor local state; may be null.
+     */
+    void set(T value);
+}
diff --git a/src/java/org/apache/cassandra/concurrent/ExecutorLocals.java b/src/java/org/apache/cassandra/concurrent/ExecutorLocals.java
new file mode 100644
index 0000000..8e6d6ea
--- /dev/null
+++ b/src/java/org/apache/cassandra/concurrent/ExecutorLocals.java
@@ -0,0 +1,84 @@
+/*
+ * 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.concurrent;
+
+import java.util.Arrays;
+
+import org.apache.cassandra.service.ClientWarn;
+import org.apache.cassandra.tracing.TraceState;
+import org.apache.cassandra.tracing.Tracing;
+
+/*
+ * This class only knows about Tracing and ClientWarn, so if any different executor locals are added, it must be
+ * updated.
+ *
+ * We don't enumerate the ExecutorLocal.all array each time because it would be much slower.
+ */
+public class ExecutorLocals
+{
+    private static final ExecutorLocal<TraceState> tracing = Tracing.instance;
+    private static final ExecutorLocal<ClientWarn.State> clientWarn = ClientWarn.instance;
+
+    public final TraceState traceState;
+    public final ClientWarn.State clientWarnState;
+
+    private ExecutorLocals(TraceState traceState, ClientWarn.State clientWarnState)
+    {
+        this.traceState = traceState;
+        this.clientWarnState = clientWarnState;
+    }
+
+    static
+    {
+        assert Arrays.equals(ExecutorLocal.all, new ExecutorLocal[]{ tracing, clientWarn })
+        : "ExecutorLocals has not been updated to reflect new ExecutorLocal.all";
+    }
+
+    /**
+     * This creates a new ExecutorLocals object based on what is already set.
+     *
+     * @return an ExecutorLocals object which has the trace state and client warn state captured if either has been set,
+     *         or null if both are unset. The null result short-circuits logic in
+     *         {@link AbstractLocalAwareExecutorService#newTaskFor(Runnable, Object, ExecutorLocals)}, preventing
+     *         unnecessarily calling {@link ExecutorLocals#set(ExecutorLocals)}.
+     */
+    public static ExecutorLocals create()
+    {
+        TraceState traceState = tracing.get();
+        ClientWarn.State clientWarnState = clientWarn.get();
+        if (traceState == null && clientWarnState == null)
+            return null;
+        else
+            return new ExecutorLocals(traceState, clientWarnState);
+    }
+
+    public static ExecutorLocals create(TraceState traceState)
+    {
+        ClientWarn.State clientWarnState = clientWarn.get();
+        return new ExecutorLocals(traceState, clientWarnState);
+    }
+
+    public static void set(ExecutorLocals locals)
+    {
+        TraceState traceState = locals == null ? null : locals.traceState;
+        ClientWarn.State clientWarnState = locals == null ? null : locals.clientWarnState;
+        tracing.set(traceState);
+        clientWarn.set(clientWarnState);
+    }
+}
diff --git a/src/java/org/apache/cassandra/concurrent/IExecutorMBean.java b/src/java/org/apache/cassandra/concurrent/IExecutorMBean.java
deleted file mode 100644
index f085b9c..0000000
--- a/src/java/org/apache/cassandra/concurrent/IExecutorMBean.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * 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.concurrent;
-
-/**
- * @see org.apache.cassandra.metrics.ThreadPoolMetrics
- */
-@Deprecated
-public interface IExecutorMBean
-{
-    /**
-     * Get the current number of running tasks
-     */
-    public int getActiveCount();
-
-    /**
-     * Get the number of completed tasks
-     */
-    public long getCompletedTasks();
-
-    /**
-     * Get the number of tasks waiting to be executed
-     */
-    public long getPendingTasks();
-
-}
diff --git a/src/java/org/apache/cassandra/concurrent/JMXEnabledSharedExecutorPool.java b/src/java/org/apache/cassandra/concurrent/JMXEnabledSharedExecutorPool.java
deleted file mode 100644
index d70e524..0000000
--- a/src/java/org/apache/cassandra/concurrent/JMXEnabledSharedExecutorPool.java
+++ /dev/null
@@ -1,114 +0,0 @@
-/*
-* 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.concurrent;
-
-import java.lang.management.ManagementFactory;
-import java.util.List;
-import javax.management.MBeanServer;
-import javax.management.ObjectName;
-
-import org.apache.cassandra.metrics.SEPMetrics;
-
-public class JMXEnabledSharedExecutorPool extends SharedExecutorPool
-{
-
-    public static final JMXEnabledSharedExecutorPool SHARED = new JMXEnabledSharedExecutorPool("SharedPool");
-
-    public JMXEnabledSharedExecutorPool(String poolName)
-    {
-        super(poolName);
-    }
-
-    public interface JMXEnabledSEPExecutorMBean extends JMXEnabledThreadPoolExecutorMBean
-    {
-    }
-
-    public class JMXEnabledSEPExecutor extends SEPExecutor implements JMXEnabledSEPExecutorMBean
-    {
-
-        private final SEPMetrics metrics;
-        private final String mbeanName;
-
-        public JMXEnabledSEPExecutor(int poolSize, int maxQueuedLength, String name, String jmxPath)
-        {
-            super(JMXEnabledSharedExecutorPool.this, poolSize, maxQueuedLength);
-            metrics = new SEPMetrics(this, jmxPath, name);
-
-            MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
-            mbeanName = "org.apache.cassandra." + jmxPath + ":type=" + name;
-
-            try
-            {
-                mbs.registerMBean(this, new ObjectName(mbeanName));
-            }
-            catch (Exception e)
-            {
-                throw new RuntimeException(e);
-            }
-        }
-
-        private void unregisterMBean()
-        {
-            try
-            {
-                ManagementFactory.getPlatformMBeanServer().unregisterMBean(new ObjectName(mbeanName));
-            }
-            catch (Exception e)
-            {
-                throw new RuntimeException(e);
-            }
-
-            // release metrics
-            metrics.release();
-        }
-
-        @Override
-        public synchronized void shutdown()
-        {
-            // synchronized, because there is no way to access super.mainLock, which would be
-            // the preferred way to make this threadsafe
-            if (!isShutdown())
-            {
-                unregisterMBean();
-            }
-            super.shutdown();
-        }
-
-        public int getCoreThreads()
-        {
-            return 0;
-        }
-
-        public void setCoreThreads(int number)
-        {
-            throw new UnsupportedOperationException();
-        }
-
-        public void setMaximumThreads(int number)
-        {
-            throw new UnsupportedOperationException();
-        }
-    }
-
-    public TracingAwareExecutorService newExecutor(int maxConcurrency, int maxQueuedTasks, String name, String jmxPath)
-    {
-        JMXEnabledSEPExecutor executor = new JMXEnabledSEPExecutor(maxConcurrency, maxQueuedTasks, name, jmxPath);
-        executors.add(executor);
-        return executor;
-    }
-}
diff --git a/src/java/org/apache/cassandra/concurrent/JMXEnabledThreadPoolExecutor.java b/src/java/org/apache/cassandra/concurrent/JMXEnabledThreadPoolExecutor.java
index 3f60df1..2b86701 100644
--- a/src/java/org/apache/cassandra/concurrent/JMXEnabledThreadPoolExecutor.java
+++ b/src/java/org/apache/cassandra/concurrent/JMXEnabledThreadPoolExecutor.java
@@ -36,7 +36,7 @@
 public class JMXEnabledThreadPoolExecutor extends DebuggableThreadPoolExecutor implements JMXEnabledThreadPoolExecutorMBean
 {
     private final String mbeanName;
-    private final ThreadPoolMetrics metrics;
+    public final ThreadPoolMetrics metrics;
 
     public JMXEnabledThreadPoolExecutor(String threadPoolName)
     {
@@ -132,30 +132,17 @@
         return super.shutdownNow();
     }
 
-    /**
-     * Get the number of completed tasks
-     */
-    public long getCompletedTasks()
-    {
-        return getCompletedTaskCount();
-    }
 
-    /**
-     * Get the number of tasks waiting to be executed
-     */
-    public long getPendingTasks()
-    {
-        return getTaskCount() - getCompletedTaskCount();
-    }
+
 
     public int getTotalBlockedTasks()
     {
-        return (int) metrics.totalBlocked.count();
+        return (int) metrics.totalBlocked.getCount();
     }
 
     public int getCurrentlyBlockedTasks()
     {
-        return (int) metrics.currentBlocked.count();
+        return (int) metrics.currentBlocked.getCount();
     }
 
     public int getCoreThreads()
diff --git a/src/java/org/apache/cassandra/concurrent/JMXEnabledThreadPoolExecutorMBean.java b/src/java/org/apache/cassandra/concurrent/JMXEnabledThreadPoolExecutorMBean.java
index ed2a2ac..fb964ae 100644
--- a/src/java/org/apache/cassandra/concurrent/JMXEnabledThreadPoolExecutorMBean.java
+++ b/src/java/org/apache/cassandra/concurrent/JMXEnabledThreadPoolExecutorMBean.java
@@ -17,25 +17,10 @@
  */
 package org.apache.cassandra.concurrent;
 
-/**
- * @see org.apache.cassandra.metrics.ThreadPoolMetrics
- */
-@Deprecated
-public interface JMXEnabledThreadPoolExecutorMBean extends IExecutorMBean
+
+public interface JMXEnabledThreadPoolExecutorMBean
 {
     /**
-     * Get the number of tasks that had blocked before being accepted (or
-     * rejected).
-     */
-    public int getTotalBlockedTasks();
-
-    /**
-     * Get the number of tasks currently blocked, waiting to be accepted by
-     * the executor (because all threads are busy and the backing queue is full).
-     */
-    public int getCurrentlyBlockedTasks();
-
-    /**
      * Returns core pool size of thread pool.
      */
     public int getCoreThreads();
diff --git a/src/java/org/apache/cassandra/concurrent/LocalAwareExecutorService.java b/src/java/org/apache/cassandra/concurrent/LocalAwareExecutorService.java
new file mode 100644
index 0000000..5577d59
--- /dev/null
+++ b/src/java/org/apache/cassandra/concurrent/LocalAwareExecutorService.java
@@ -0,0 +1,34 @@
+/*
+ *
+ * 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.concurrent;
+
+import java.util.concurrent.ExecutorService;
+
+public interface LocalAwareExecutorService extends ExecutorService
+{
+    // we need a way to inject a TraceState directly into the Executor context without going through
+    // the global Tracing sessions; see CASSANDRA-5668
+    public void execute(Runnable command, ExecutorLocals locals);
+
+    // permits executing in the context of the submitting thread
+    public void maybeExecuteImmediately(Runnable command);
+}
diff --git a/src/java/org/apache/cassandra/concurrent/SEPExecutor.java b/src/java/org/apache/cassandra/concurrent/SEPExecutor.java
index f6d1940..8b12b82 100644
--- a/src/java/org/apache/cassandra/concurrent/SEPExecutor.java
+++ b/src/java/org/apache/cassandra/concurrent/SEPExecutor.java
@@ -21,20 +21,21 @@
 import java.util.List;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 
+import org.apache.cassandra.metrics.SEPMetrics;
 import org.apache.cassandra.utils.concurrent.SimpleCondition;
 import org.apache.cassandra.utils.concurrent.WaitQueue;
 
 import static org.apache.cassandra.concurrent.SEPWorker.Work;
 
-public class SEPExecutor extends AbstractTracingAwareExecutorService
+public class SEPExecutor extends AbstractLocalAwareExecutorService
 {
     private final SharedExecutorPool pool;
 
     public final int maxWorkers;
     private final int maxTasksQueued;
+    private final SEPMetrics metrics;
 
     // stores both a set of work permits and task permits:
     //  bottom 32 bits are number of queued tasks, in the range [0..maxTasksQueued]   (initially 0)
@@ -43,8 +44,6 @@
 
     // producers wait on this when there is no room on the queue
     private final WaitQueue hasRoom = new WaitQueue();
-    private final AtomicLong totalBlocked = new AtomicLong();
-    private final AtomicInteger currentlyBlocked = new AtomicInteger();
     private final AtomicLong completedTasks = new AtomicLong();
 
     volatile boolean shuttingDown = false;
@@ -53,12 +52,13 @@
     // TODO: see if other queue implementations might improve throughput
     protected final ConcurrentLinkedQueue<FutureTask<?>> tasks = new ConcurrentLinkedQueue<>();
 
-    SEPExecutor(SharedExecutorPool pool, int maxWorkers, int maxTasksQueued)
+    SEPExecutor(SharedExecutorPool pool, int maxWorkers, int maxTasksQueued, String jmxPath, String name)
     {
         this.pool = pool;
         this.maxWorkers = maxWorkers;
         this.maxTasksQueued = maxTasksQueued;
         this.permits.set(combine(0, maxWorkers));
+        this.metrics = new SEPMetrics(this, jmxPath, name);
     }
 
     protected void onCompletion()
@@ -116,10 +116,11 @@
                 // if we're blocking, we might as well directly schedule a worker if we aren't already at max
                 if (takeWorkPermit(true))
                     pool.schedule(new Work(this));
-                totalBlocked.incrementAndGet();
-                currentlyBlocked.incrementAndGet();
+
+                metrics.totalBlocked.inc();
+                metrics.currentBlocked.inc();
                 s.awaitUninterruptibly();
-                currentlyBlocked.decrementAndGet();
+                metrics.currentBlocked.dec();
             }
             else // don't propagate our signal when we cancel, just cancel
                 s.cancel();
@@ -207,6 +208,9 @@
         pool.executors.remove(this);
         if (getActiveCount() == 0)
             shutdown.signalAll();
+
+        // release metrics
+        metrics.release();
     }
 
     public synchronized List<Runnable> shutdownNow()
@@ -249,21 +253,6 @@
         return maxWorkers - workPermits(permits.get());
     }
 
-    public int getTotalBlockedTasks()
-    {
-        return (int) totalBlocked.get();
-    }
-
-    public int getMaximumThreads()
-    {
-        return maxWorkers;
-    }
-
-    public int getCurrentlyBlockedTasks()
-    {
-        return currentlyBlocked.get();
-    }
-
     private static int taskPermits(long both)
     {
         return (int) both;
diff --git a/src/java/org/apache/cassandra/concurrent/ScheduledExecutors.java b/src/java/org/apache/cassandra/concurrent/ScheduledExecutors.java
index 5935669..5962db9 100644
--- a/src/java/org/apache/cassandra/concurrent/ScheduledExecutors.java
+++ b/src/java/org/apache/cassandra/concurrent/ScheduledExecutors.java
@@ -31,10 +31,6 @@
      * This executor is used for tasks that can have longer execution times, and usually are non periodic.
      */
     public static final DebuggableScheduledThreadPoolExecutor nonPeriodicTasks = new DebuggableScheduledThreadPoolExecutor("NonPeriodicTasks");
-    static
-    {
-        nonPeriodicTasks.setExecuteExistingDelayedTasksAfterShutdownPolicy(false);
-    }
 
     /**
      * This executor is used for tasks that do not need to be waited for on shutdown/drain.
diff --git a/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java b/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java
index e03ec57..dfd7011 100644
--- a/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java
+++ b/src/java/org/apache/cassandra/concurrent/SharedExecutorPool.java
@@ -19,7 +19,6 @@
 
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -35,21 +34,28 @@
  * To keep producers from incurring unnecessary delays, once an executor is "spun up" (i.e. is processing tasks at a steady
  * rate), adding tasks to the executor often involves only placing the task on the work queue and updating the
  * task permits (which imposes our max queue length constraints). Only when it cannot be guaranteed the task will be serviced
- * promptly does the producer have to signal a thread itself to perform the work.
+ * promptly, and the maximum concurrency has not been reached, does the producer have to schedule a thread itself to perform 
+ * the work ('promptly' in this context means we already have a worker spinning for work, as described next).
  *
- * We do this by scheduling only if
- *
- * The worker threads schedule themselves as far as possible: when they are assigned a task, they will attempt to spawn
+ * Otherwise the worker threads schedule themselves: when they are assigned a task, they will attempt to spawn
  * a partner worker to service any other work outstanding on the queue (if any); once they have finished the task they
  * will either take another (if any remaining) and repeat this, or they will attempt to assign themselves to another executor
  * that does have tasks remaining. If both fail, it will enter a non-busy-spinning phase, where it will sleep for a short
  * random interval (based upon the number of threads in this mode, so that the total amount of non-sleeping time remains
- * approximately fixed regardless of the number of spinning threads), and upon waking up will again try to assign themselves
- * an executor with outstanding tasks to perform.
+ * approximately fixed regardless of the number of spinning threads), and upon waking will again try to assign itself to
+ * an executor with outstanding tasks to perform. As a result of always scheduling a partner before committing to performing
+ * any work, with a steady state of task arrival we should generally have either one spinning worker ready to promptly respond 
+ * to incoming work, or all possible workers actively committed to tasks.
+ * 
+ * In order to prevent this executor pool acting like a noisy neighbour to other processes on the system, workers also deschedule
+ * themselves when it is detected that there are too many for the current rate of operation arrival. This is decided as a function 
+ * of the total time spent spinning by all workers in an interval; as more workers spin, workers are descheduled more rapidly.
  */
 public class SharedExecutorPool
 {
 
+    public static final SharedExecutorPool SHARED = new SharedExecutorPool("SharedPool");
+
     // the name assigned to workers in the pool, and the id suffix
     final String poolName;
     final AtomicLong workerId = new AtomicLong();
@@ -96,4 +102,11 @@
         if (current == 0 && spinningCount.compareAndSet(0, 1))
             schedule(Work.SPINNING);
     }
-}
\ No newline at end of file
+
+    public LocalAwareExecutorService newExecutor(int maxConcurrency, int maxQueuedTasks, String jmxPath, String name)
+    {
+        SEPExecutor executor = new SEPExecutor(this, maxConcurrency, maxQueuedTasks, jmxPath, name);
+        executors.add(executor);
+        return executor;
+    }
+}
diff --git a/src/java/org/apache/cassandra/concurrent/Stage.java b/src/java/org/apache/cassandra/concurrent/Stage.java
index 6192cab..4e3700f 100644
--- a/src/java/org/apache/cassandra/concurrent/Stage.java
+++ b/src/java/org/apache/cassandra/concurrent/Stage.java
@@ -17,6 +17,11 @@
  */
 package org.apache.cassandra.concurrent;
 
+import java.util.Arrays;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterables;
+
 public enum Stage
 {
     READ,
@@ -31,6 +36,17 @@
     INTERNAL_RESPONSE,
     READ_REPAIR;
 
+    public static Iterable<Stage> jmxEnabledStages()
+    {
+        return Iterables.filter(Arrays.asList(values()), new Predicate<Stage>()
+        {
+            public boolean apply(Stage stage)
+            {
+                return stage != TRACING;
+            }
+        });
+    }
+
     public String getJmxType()
     {
         switch (this)
diff --git a/src/java/org/apache/cassandra/concurrent/StageManager.java b/src/java/org/apache/cassandra/concurrent/StageManager.java
index 303f658..343648c 100644
--- a/src/java/org/apache/cassandra/concurrent/StageManager.java
+++ b/src/java/org/apache/cassandra/concurrent/StageManager.java
@@ -24,7 +24,6 @@
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.tracing.TraceState;
 import org.apache.cassandra.utils.FBUtilities;
 
 import static org.apache.cassandra.config.DatabaseDescriptor.*;
@@ -39,7 +38,7 @@
 {
     private static final Logger logger = LoggerFactory.getLogger(StageManager.class);
 
-    private static final EnumMap<Stage, TracingAwareExecutorService> stages = new EnumMap<Stage, TracingAwareExecutorService>(Stage.class);
+    private static final EnumMap<Stage, LocalAwareExecutorService> stages = new EnumMap<Stage, LocalAwareExecutorService>(Stage.class);
 
     public static final long KEEPALIVE = 60; // seconds to keep "extra" threads alive for when idle
 
@@ -87,16 +86,16 @@
                                                 stage.getJmxType());
     }
 
-    private static TracingAwareExecutorService multiThreadedLowSignalStage(Stage stage, int numThreads)
+    private static LocalAwareExecutorService multiThreadedLowSignalStage(Stage stage, int numThreads)
     {
-        return JMXEnabledSharedExecutorPool.SHARED.newExecutor(numThreads, Integer.MAX_VALUE, stage.getJmxName(), stage.getJmxType());
+        return SharedExecutorPool.SHARED.newExecutor(numThreads, Integer.MAX_VALUE, stage.getJmxType(), stage.getJmxName());
     }
 
     /**
      * Retrieve a stage from the StageManager
      * @param stage name of the stage to be retrieved.
      */
-    public static TracingAwareExecutorService getStage(Stage stage)
+    public static LocalAwareExecutorService getStage(Stage stage)
     {
         return stages.get(stage);
     }
@@ -112,20 +111,30 @@
         }
     }
 
+    public final static Runnable NO_OP_TASK = new Runnable()
+    {
+        public void run()
+        {
+
+        }
+    };
+
     /**
      * A TPE that disallows submit so that we don't need to worry about unwrapping exceptions on the
-     * tracing stage.  See CASSANDRA-1123 for background.
+     * tracing stage.  See CASSANDRA-1123 for background. We allow submitting NO_OP tasks, to allow
+     * a final wait on pending trace events since typically the tracing executor is single-threaded, see
+     * CASSANDRA-11465.
      */
-    private static class ExecuteOnlyExecutor extends ThreadPoolExecutor implements TracingAwareExecutorService
+    private static class ExecuteOnlyExecutor extends ThreadPoolExecutor implements LocalAwareExecutorService
     {
         public ExecuteOnlyExecutor(int corePoolSize, int maximumPoolSize, long keepAliveTime, TimeUnit unit, BlockingQueue<Runnable> workQueue, ThreadFactory threadFactory, RejectedExecutionHandler handler)
         {
             super(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue, threadFactory, handler);
         }
 
-        public void execute(Runnable command, TraceState state)
+        public void execute(Runnable command, ExecutorLocals locals)
         {
-            assert state == null;
+            assert locals == null;
             super.execute(command);
         }
 
@@ -137,6 +146,11 @@
         @Override
         public Future<?> submit(Runnable task)
         {
+            if (task.equals(NO_OP_TASK))
+            {
+                assert getMaximumPoolSize() == 1 : "Cannot wait for pending tasks if running more than 1 thread";
+                return super.submit(task);
+            }
             throw new UnsupportedOperationException();
         }
 
diff --git a/src/java/org/apache/cassandra/concurrent/TracingAwareExecutorService.java b/src/java/org/apache/cassandra/concurrent/TracingAwareExecutorService.java
deleted file mode 100644
index f580fea..0000000
--- a/src/java/org/apache/cassandra/concurrent/TracingAwareExecutorService.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * 
- * 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.concurrent;
-
-import java.util.concurrent.ExecutorService;
-
-import org.apache.cassandra.tracing.TraceState;
-
-public interface TracingAwareExecutorService extends ExecutorService
-{
-    // we need a way to inject a TraceState directly into the Executor context without going through
-    // the global Tracing sessions; see CASSANDRA-5668
-    public void execute(Runnable command, TraceState state);
-
-    // permits executing in the context of the submitting thread
-    public void maybeExecuteImmediately(Runnable command);
-}
diff --git a/src/java/org/apache/cassandra/config/CFMetaData.java b/src/java/org/apache/cassandra/config/CFMetaData.java
index bce5c72..67a1c8c 100644
--- a/src/java/org/apache/cassandra/config/CFMetaData.java
+++ b/src/java/org/apache/cassandra/config/CFMetaData.java
@@ -37,7 +37,6 @@
 import org.apache.cassandra.cache.CachingOptions;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.cql3.QueryProcessor;
-import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.cql3.statements.CFStatement;
 import org.apache.cassandra.cql3.statements.CreateTableStatement;
 import org.apache.cassandra.db.*;
@@ -48,22 +47,15 @@
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.io.compress.CompressionParameters;
 import org.apache.cassandra.io.compress.LZ4Compressor;
-import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.serializers.MarshalException;
-import org.apache.cassandra.thrift.CfDef;
-import org.apache.cassandra.thrift.CqlResult;
-import org.apache.cassandra.thrift.CqlRow;
-import org.apache.cassandra.tracing.Tracing;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.schema.LegacySchemaTables;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.UUIDGen;
 import org.github.jamm.Unmetered;
 
-import static org.apache.cassandra.utils.FBUtilities.fromJsonList;
-import static org.apache.cassandra.utils.FBUtilities.fromJsonMap;
-import static org.apache.cassandra.utils.FBUtilities.json;
-
 /**
  * This class can be tricky to modify. Please read http://wiki.apache.org/cassandra/ConfigurationNotes for how to do so safely.
  */
@@ -96,216 +88,6 @@
         }
     };
 
-    public static final CFMetaData IndexCf = compile("CREATE TABLE \"" + SystemKeyspace.INDEX_CF + "\" ("
-                                                     + "table_name text,"
-                                                     + "index_name text,"
-                                                     + "PRIMARY KEY (table_name, index_name)"
-                                                     + ") WITH COMPACT STORAGE AND COMMENT='indexes that have been completed'");
-
-    public static final CFMetaData SchemaKeyspacesCf = compile("CREATE TABLE " + SystemKeyspace.SCHEMA_KEYSPACES_CF + " ("
-                                                               + "keyspace_name text PRIMARY KEY,"
-                                                               + "durable_writes boolean,"
-                                                               + "strategy_class text,"
-                                                               + "strategy_options text"
-                                                               + ") WITH COMPACT STORAGE AND COMMENT='keyspace definitions' AND gc_grace_seconds=604800");
-
-    public static final CFMetaData SchemaColumnFamiliesCf = compile("CREATE TABLE " + SystemKeyspace.SCHEMA_COLUMNFAMILIES_CF + " ("
-                                                                    + "keyspace_name text,"
-                                                                    + "columnfamily_name text,"
-                                                                    + "cf_id uuid," // post-2.1 UUID cfid
-                                                                    + "type text,"
-                                                                    + "is_dense boolean,"
-                                                                    + "comparator text,"
-                                                                    + "subcomparator text,"
-                                                                    + "comment text,"
-                                                                    + "read_repair_chance double,"
-                                                                    + "local_read_repair_chance double,"
-                                                                    + "gc_grace_seconds int,"
-                                                                    + "default_validator text,"
-                                                                    + "key_validator text,"
-                                                                    + "min_compaction_threshold int,"
-                                                                    + "max_compaction_threshold int,"
-                                                                    + "memtable_flush_period_in_ms int,"
-                                                                    + "key_aliases text,"
-                                                                    + "bloom_filter_fp_chance double,"
-                                                                    + "caching text,"
-                                                                    + "default_time_to_live int,"
-                                                                    + "compaction_strategy_class text,"
-                                                                    + "compression_parameters text,"
-                                                                    + "value_alias text,"
-                                                                    + "column_aliases text,"
-                                                                    + "compaction_strategy_options text,"
-                                                                    + "speculative_retry text,"
-                                                                    + "index_interval int,"
-                                                                    + "min_index_interval int,"
-                                                                    + "max_index_interval int,"
-                                                                    + "dropped_columns map<text, bigint>,"
-                                                                    + "PRIMARY KEY (keyspace_name, columnfamily_name)"
-                                                                    + ") WITH COMMENT='ColumnFamily definitions' AND gc_grace_seconds=604800");
-
-    public static final CFMetaData SchemaColumnsCf = compile("CREATE TABLE " + SystemKeyspace.SCHEMA_COLUMNS_CF + " ("
-                                                             + "keyspace_name text,"
-                                                             + "columnfamily_name text,"
-                                                             + "column_name text,"
-                                                             + "validator text,"
-                                                             + "index_type text,"
-                                                             + "index_options text,"
-                                                             + "index_name text,"
-                                                             + "component_index int,"
-                                                             + "type text,"
-                                                             + "PRIMARY KEY(keyspace_name, columnfamily_name, column_name)"
-                                                             + ") WITH COMMENT='ColumnFamily column attributes' AND gc_grace_seconds=604800");
-
-    public static final CFMetaData SchemaTriggersCf = compile("CREATE TABLE " + SystemKeyspace.SCHEMA_TRIGGERS_CF + " ("
-                                                              + "keyspace_name text,"
-                                                              + "columnfamily_name text,"
-                                                              + "trigger_name text,"
-                                                              + "trigger_options map<text, text>,"
-                                                              + "PRIMARY KEY (keyspace_name, columnfamily_name, trigger_name)"
-                                                              + ") WITH COMMENT='triggers metadata table' AND gc_grace_seconds=604800");
-
-    public static final CFMetaData SchemaUserTypesCf = compile("CREATE TABLE " + SystemKeyspace.SCHEMA_USER_TYPES_CF + " ("
-                                                               + "keyspace_name text,"
-                                                               + "type_name text,"
-                                                               + "field_names list<text>,"
-                                                               + "field_types list<text>,"
-                                                               + "PRIMARY KEY (keyspace_name, type_name)"
-                                                               + ") WITH COMMENT='Defined user types' AND gc_grace_seconds=604800");
-
-    public static final CFMetaData HintsCf = compile("CREATE TABLE " + SystemKeyspace.HINTS_CF + " ("
-                                                     + "target_id uuid,"
-                                                     + "hint_id timeuuid,"
-                                                     + "message_version int,"
-                                                     + "mutation blob,"
-                                                     + "PRIMARY KEY (target_id, hint_id, message_version)"
-                                                     + ") WITH COMPACT STORAGE "
-                                                     + "AND COMPACTION={'class' : 'SizeTieredCompactionStrategy', 'enabled' : false} "
-                                                     + "AND COMMENT='hints awaiting delivery'"
-                                                     + "AND gc_grace_seconds=0");
-
-    public static final CFMetaData PeersCf = compile("CREATE TABLE " + SystemKeyspace.PEERS_CF + " ("
-                                                     + "peer inet PRIMARY KEY,"
-                                                     + "host_id uuid,"
-                                                     + "tokens set<varchar>,"
-                                                     + "schema_version uuid,"
-                                                     + "release_version text,"
-                                                     + "rpc_address inet,"
-                                                     + "preferred_ip inet,"
-                                                     + "data_center text,"
-                                                     + "rack text"
-                                                     + ") WITH COMMENT='known peers in the cluster'");
-
-    public static final CFMetaData PeerEventsCf = compile("CREATE TABLE " + SystemKeyspace.PEER_EVENTS_CF + " ("
-                                                          + "peer inet PRIMARY KEY,"
-                                                          + "hints_dropped map<uuid, int>"
-                                                          + ") WITH COMMENT='cf contains events related to peers'");
-
-    public static final CFMetaData LocalCf = compile("CREATE TABLE " + SystemKeyspace.LOCAL_CF + " ("
-                                                     + "key text PRIMARY KEY,"
-                                                     + "tokens set<varchar>,"
-                                                     + "cluster_name text,"
-                                                     + "gossip_generation int,"
-                                                     + "bootstrapped text,"
-                                                     + "host_id uuid,"
-                                                     + "release_version text,"
-                                                     + "thrift_version text,"
-                                                     + "cql_version text,"
-                                                     + "native_protocol_version text,"
-                                                     + "data_center text,"
-                                                     + "rack text,"
-                                                     + "partitioner text,"
-                                                     + "schema_version uuid,"
-                                                     + "truncated_at map<uuid, blob>,"
-                                                     + "rpc_address inet,"
-                                                     + "broadcast_address inet,"
-                                                     + "listen_address inet"
-                                                     + ") WITH COMMENT='information about the local node'");
-
-    public static final CFMetaData TraceSessionsCf = compile("CREATE TABLE " + Tracing.SESSIONS_CF + " ("
-                                                             + "session_id uuid PRIMARY KEY,"
-                                                             + "coordinator inet,"
-                                                             + "request text,"
-                                                             + "started_at timestamp,"
-                                                             + "parameters map<text, text>,"
-                                                             + "duration int"
-                                                             + ") WITH COMMENT='traced sessions' AND default_time_to_live=86400",
-                                                             Tracing.TRACE_KS);
-
-    public static final CFMetaData TraceEventsCf = compile("CREATE TABLE " + Tracing.EVENTS_CF + " ("
-                                                           + "session_id uuid,"
-                                                           + "event_id timeuuid,"
-                                                           + "source inet,"
-                                                           + "thread text,"
-                                                           + "activity text,"
-                                                           + "source_elapsed int,"
-                                                           + "PRIMARY KEY (session_id, event_id)"
-                                                           + ") WITH default_time_to_live=86400",
-                                                           Tracing.TRACE_KS);
-
-    public static final CFMetaData BatchlogCf = compile("CREATE TABLE " + SystemKeyspace.BATCHLOG_CF + " ("
-                                                        + "id uuid PRIMARY KEY,"
-                                                        + "written_at timestamp,"
-                                                        + "data blob,"
-                                                        + "version int,"
-                                                        + ") WITH COMMENT='uncommited batches' AND gc_grace_seconds=0 "
-                                                        + "AND COMPACTION={'class' : 'SizeTieredCompactionStrategy', 'min_threshold' : 2}");
-
-    public static final CFMetaData RangeXfersCf = compile("CREATE TABLE " + SystemKeyspace.RANGE_XFERS_CF + " ("
-                                                          + "token_bytes blob PRIMARY KEY,"
-                                                          + "requested_at timestamp"
-                                                          + ") WITH COMMENT='ranges requested for transfer here'");
-
-    public static final CFMetaData CompactionLogCf = compile("CREATE TABLE " + SystemKeyspace.COMPACTION_LOG + " ("
-                                                             + "id uuid PRIMARY KEY,"
-                                                             + "keyspace_name text,"
-                                                             + "columnfamily_name text,"
-                                                             + "inputs set<int>"
-                                                             + ") WITH COMMENT='unfinished compactions'");
-
-    public static final CFMetaData PaxosCf = compile("CREATE TABLE " + SystemKeyspace.PAXOS_CF + " ("
-                                                     + "row_key blob,"
-                                                     + "cf_id UUID,"
-                                                     + "in_progress_ballot timeuuid,"
-                                                     + "proposal_ballot timeuuid,"
-                                                     + "proposal blob,"
-                                                     + "most_recent_commit_at timeuuid,"
-                                                     + "most_recent_commit blob,"
-                                                     + "PRIMARY KEY (row_key, cf_id)"
-                                                     + ") WITH COMMENT='in-progress paxos proposals' "
-                                                     + "AND COMPACTION={'class' : 'LeveledCompactionStrategy'}");
-
-    public static final CFMetaData SSTableActivityCF = compile("CREATE TABLE " + SystemKeyspace.SSTABLE_ACTIVITY_CF + " ("
-                                                               + "keyspace_name text,"
-                                                               + "columnfamily_name text,"
-                                                               + "generation int,"
-                                                               + "rate_15m double,"
-                                                               + "rate_120m double,"
-                                                               + "PRIMARY KEY ((keyspace_name, columnfamily_name, generation))"
-                                                               + ") WITH COMMENT='historic sstable read rates'");
-
-    public static final CFMetaData CompactionHistoryCf = compile("CREATE TABLE " + SystemKeyspace.COMPACTION_HISTORY_CF + " ("
-                                                                 + "id uuid,"
-                                                                 + "keyspace_name text,"
-                                                                 + "columnfamily_name text,"
-                                                                 + "compacted_at timestamp,"
-                                                                 + "bytes_in bigint,"
-                                                                 + "bytes_out bigint,"
-                                                                 + "rows_merged map<int, bigint>,"
-                                                                 + "PRIMARY KEY (id)"
-                                                                 + ") WITH COMMENT='show all compaction history' AND DEFAULT_TIME_TO_LIVE=604800");
-
-    public static final CFMetaData SizeEstimatesCf = compile("CREATE TABLE " + SystemKeyspace.SIZE_ESTIMATES_CF + " ("
-                                                             + "keyspace_name text,"
-                                                             + "table_name text,"
-                                                             + "range_start text,"
-                                                             + "range_end text,"
-                                                             + "mean_partition_size bigint,"
-                                                             + "partitions_count bigint,"
-                                                             + "PRIMARY KEY ((keyspace_name), table_name, range_start, range_end)"
-                                                             + ") WITH COMMENT='per-table primary range size estimates' "
-                                                             + "AND gc_grace_seconds=0");
-
-
     public static class SpeculativeRetry
     {
         public enum RetryType
@@ -331,7 +113,7 @@
                 {
                     double value = Double.parseDouble(name.substring(0, name.length() - 10));
                     if (value > 100 || value < 0)
-                        throw new ConfigurationException("PERCENTILE should be between 0 and 100");
+                        throw new ConfigurationException("PERCENTILE should be between 0 and 100, but was " + value);
                     return new SpeculativeRetry(RetryType.PERCENTILE, (value / 100));
                 }
                 else if (name.endsWith("MS"))
@@ -439,7 +221,7 @@
     public volatile CompressionParameters compressionParameters = new CompressionParameters(null);
 
     // attribute setters that return the modified CFMetaData instance
-    public CFMetaData comment(String prop) { comment = Strings.nullToEmpty(prop); return this;}
+    public CFMetaData comment(String prop) {comment = Strings.nullToEmpty(prop); return this;}
     public CFMetaData readRepairChance(double prop) {readRepairChance = prop; return this;}
     public CFMetaData dcLocalReadRepairChance(double prop) {dcLocalReadRepairChance = prop; return this;}
     public CFMetaData gcGraceSeconds(int prop) {gcGraceSeconds = prop; return this;}
@@ -450,7 +232,7 @@
     public CFMetaData compactionStrategyClass(Class<? extends AbstractCompactionStrategy> prop) {compactionStrategyClass = prop; return this;}
     public CFMetaData compactionStrategyOptions(Map<String, String> prop) {compactionStrategyOptions = prop; return this;}
     public CFMetaData compressionParameters(CompressionParameters prop) {compressionParameters = prop; return this;}
-    public CFMetaData bloomFilterFpChance(Double prop) {bloomFilterFpChance = prop; return this;}
+    public CFMetaData bloomFilterFpChance(double prop) {bloomFilterFpChance = prop; return this;}
     public CFMetaData caching(CachingOptions prop) {caching = prop; return this;}
     public CFMetaData minIndexInterval(int prop) {minIndexInterval = prop; return this;}
     public CFMetaData maxIndexInterval(int prop) {maxIndexInterval = prop; return this;}
@@ -460,6 +242,7 @@
     public CFMetaData droppedColumns(Map<ColumnIdentifier, Long> cols) {droppedColumns = cols; return this;}
     public CFMetaData triggers(Map<String, TriggerDefinition> prop) {triggers = prop; return this;}
     public CFMetaData isDense(Boolean prop) {isDense = prop; return this;}
+
     /**
      * Create new ColumnFamily metadata with generated random ID.
      * When loading from existing schema, use CFMetaData
@@ -505,7 +288,7 @@
         return denseCFMetaData(keyspace, name, comp, null);
     }
 
-    private static AbstractType<?> makeRawAbstractType(AbstractType<?> comparator, AbstractType<?> subComparator)
+    public static AbstractType<?> makeRawAbstractType(AbstractType<?> comparator, AbstractType<?> subComparator)
     {
         return subComparator == null ? comparator : CompositeType.getInstance(Arrays.asList(comparator, subComparator));
     }
@@ -515,27 +298,14 @@
         return triggers;
     }
 
-    private static CFMetaData compile(String cql)
-    {
-        return compile(cql, Keyspace.SYSTEM_KS);
-    }
-
-    @VisibleForTesting
     public static CFMetaData compile(String cql, String keyspace)
     {
-        try
-        {
-            CFStatement parsed = (CFStatement)QueryProcessor.parseStatement(cql);
-            parsed.prepareKeyspace(keyspace);
-            CreateTableStatement statement = (CreateTableStatement) parsed.prepare().statement;
-            CFMetaData cfm = newSystemMetadata(keyspace, statement.columnFamily(), "", statement.comparator);
-            statement.applyPropertiesTo(cfm);
-            return cfm.rebuild();
-        }
-        catch (RequestValidationException e)
-        {
-            throw new RuntimeException(e);
-        }
+        CFStatement parsed = (CFStatement)QueryProcessor.parseStatement(cql);
+        parsed.prepareKeyspace(keyspace);
+        CreateTableStatement statement = (CreateTableStatement) parsed.prepare().statement;
+        CFMetaData cfm = newSystemMetadata(keyspace, statement.columnFamily(), "", statement.comparator);
+        statement.applyPropertiesTo(cfm);
+        return cfm.rebuild();
     }
 
     /**
@@ -573,8 +343,8 @@
         // Depends on parent's cache setting, turn on its index CF's cache.
         // Row caching is never enabled; see CASSANDRA-5732
         CachingOptions indexCaching = parent.getCaching().keyCache.isEnabled()
-                             ? CachingOptions.KEYS_ONLY
-                             : CachingOptions.NONE;
+                                    ? CachingOptions.KEYS_ONLY
+                                    : CachingOptions.NONE;
 
         return new CFMetaData(parent.ksName, parent.indexColumnFamilyName(info), ColumnFamilyType.Standard, indexComparator, parent.cfId)
                              .keyValidator(info.type)
@@ -615,7 +385,8 @@
         return copyOpts(new CFMetaData(ksName, cfName, cfType, comparator, newCfId), this);
     }
 
-    static CFMetaData copyOpts(CFMetaData newCFMD, CFMetaData oldCFMD)
+    @VisibleForTesting
+    public static CFMetaData copyOpts(CFMetaData newCFMD, CFMetaData oldCFMD)
     {
         List<ColumnDefinition> clonedColumns = new ArrayList<>(oldCFMD.allColumns().size());
         for (ColumnDefinition cd : oldCFMD.allColumns())
@@ -633,7 +404,7 @@
                       .compactionStrategyClass(oldCFMD.compactionStrategyClass)
                       .compactionStrategyOptions(new HashMap<>(oldCFMD.compactionStrategyOptions))
                       .compressionParameters(oldCFMD.compressionParameters.copy())
-                      .bloomFilterFpChance(oldCFMD.bloomFilterFpChance)
+                      .bloomFilterFpChance(oldCFMD.getBloomFilterFpChance())
                       .caching(oldCFMD.caching)
                       .defaultTimeToLive(oldCFMD.defaultTimeToLive)
                       .minIndexInterval(oldCFMD.minIndexInterval)
@@ -678,6 +449,11 @@
         return cfName.contains(".");
     }
 
+    public Map<ByteBuffer, ColumnDefinition> getColumnMetadata()
+    {
+        return columnMetadata;
+    }
+
     /**
      *
      * @return The name of the parent cf if this is a seconday index
@@ -692,7 +468,7 @@
         return readRepairChance;
     }
 
-    public double getDcLocalReadRepair()
+    public double getDcLocalReadRepairChance()
     {
         return dcLocalReadRepairChance;
     }
@@ -703,7 +479,7 @@
         if (getReadRepairChance() > chance)
             return ReadRepairDecision.GLOBAL;
 
-        if (getDcLocalReadRepair() > chance)
+        if (getDcLocalReadRepairChance() > chance)
             return ReadRepairDecision.DC_LOCAL;
 
         return ReadRepairDecision.NONE;
@@ -734,17 +510,6 @@
         return maxCompactionThreshold;
     }
 
-    // Used by CQL2 only.
-    public String getCQL2KeyName()
-    {
-        if (partitionKeyColumns.size() > 1)
-            throw new IllegalStateException("Cannot acces column family with composite key from CQL < 3.0.0");
-
-        // For compatibility sake, we uppercase if it's the default alias as we used to return it that way in resultsets.
-        String str = partitionKeyColumns.get(0).name.toString();
-        return str.equalsIgnoreCase(DEFAULT_KEY_ALIAS) ? str.toUpperCase() : str;
-    }
-
     public CompressionParameters compressionParameters()
     {
         return compressionParameters;
@@ -909,7 +674,7 @@
             && Objects.equal(compactionStrategyClass, other.compactionStrategyClass)
             && Objects.equal(compactionStrategyOptions, other.compactionStrategyOptions)
             && Objects.equal(compressionParameters, other.compressionParameters)
-            && Objects.equal(bloomFilterFpChance, other.bloomFilterFpChance)
+            && Objects.equal(getBloomFilterFpChance(), other.getBloomFilterFpChance())
             && Objects.equal(memtableFlushPeriod, other.memtableFlushPeriod)
             && Objects.equal(caching, other.caching)
             && Objects.equal(defaultTimeToLive, other.defaultTimeToLive)
@@ -942,7 +707,7 @@
             .append(compactionStrategyClass)
             .append(compactionStrategyOptions)
             .append(compressionParameters)
-            .append(bloomFilterFpChance)
+            .append(getBloomFilterFpChance())
             .append(memtableFlushPeriod)
             .append(caching)
             .append(defaultTimeToLive)
@@ -961,209 +726,23 @@
         return def == null ? defaultValidator : def.type;
     }
 
-    /** applies implicit defaults to cf definition. useful in updates */
-    private static void applyImplicitDefaults(org.apache.cassandra.thrift.CfDef cf_def)
-    {
-        if (!cf_def.isSetComment())
-            cf_def.setComment("");
-        if (!cf_def.isSetMin_compaction_threshold())
-            cf_def.setMin_compaction_threshold(CFMetaData.DEFAULT_MIN_COMPACTION_THRESHOLD);
-        if (!cf_def.isSetMax_compaction_threshold())
-            cf_def.setMax_compaction_threshold(CFMetaData.DEFAULT_MAX_COMPACTION_THRESHOLD);
-        if (cf_def.compaction_strategy == null)
-            cf_def.compaction_strategy = DEFAULT_COMPACTION_STRATEGY_CLASS.getSimpleName();
-        if (cf_def.compaction_strategy_options == null)
-            cf_def.compaction_strategy_options = Collections.emptyMap();
-        if (!cf_def.isSetCompression_options())
-        {
-            cf_def.setCompression_options(new HashMap<String, String>()
-            {{
-                if (DEFAULT_COMPRESSOR != null)
-                    put(CompressionParameters.SSTABLE_COMPRESSION, DEFAULT_COMPRESSOR);
-            }});
-        }
-        if (!cf_def.isSetDefault_time_to_live())
-            cf_def.setDefault_time_to_live(CFMetaData.DEFAULT_DEFAULT_TIME_TO_LIVE);
-        if (!cf_def.isSetDclocal_read_repair_chance())
-            cf_def.setDclocal_read_repair_chance(CFMetaData.DEFAULT_DCLOCAL_READ_REPAIR_CHANCE);
-
-        // if index_interval was set, use that for the min_index_interval default
-        if (!cf_def.isSetMin_index_interval())
-        {
-            if (cf_def.isSetIndex_interval())
-                cf_def.setMin_index_interval(cf_def.getIndex_interval());
-            else
-                cf_def.setMin_index_interval(CFMetaData.DEFAULT_MIN_INDEX_INTERVAL);
-        }
-        if (!cf_def.isSetMax_index_interval())
-        {
-            // ensure the max is at least as large as the min
-            cf_def.setMax_index_interval(Math.max(cf_def.min_index_interval, CFMetaData.DEFAULT_MAX_INDEX_INTERVAL));
-        }
-    }
-
-    public static CFMetaData fromThrift(CfDef cf_def) throws InvalidRequestException, ConfigurationException
-    {
-        return internalFromThrift(cf_def, Collections.<ColumnDefinition>emptyList());
-    }
-
-    public static CFMetaData fromThriftForUpdate(CfDef cf_def, CFMetaData toUpdate) throws InvalidRequestException, ConfigurationException
-    {
-        return internalFromThrift(cf_def, toUpdate.allColumns());
-    }
-
-    // Convert a thrift CfDef, given a list of ColumnDefinitions to copy over to the created CFMetadata before the CQL metadata are rebuild
-    private static CFMetaData internalFromThrift(CfDef cf_def, Collection<ColumnDefinition> previousCQLMetadata) throws InvalidRequestException, ConfigurationException
-    {
-        ColumnFamilyType cfType = ColumnFamilyType.create(cf_def.column_type);
-        if (cfType == null)
-            throw new InvalidRequestException("Invalid column type " + cf_def.column_type);
-
-        applyImplicitDefaults(cf_def);
-
-        try
-        {
-            AbstractType<?> rawComparator = TypeParser.parse(cf_def.comparator_type);
-            AbstractType<?> subComparator = cfType == ColumnFamilyType.Standard
-                                          ? null
-                                          : cf_def.subcomparator_type == null ? BytesType.instance : TypeParser.parse(cf_def.subcomparator_type);
-
-            AbstractType<?> fullRawComparator = makeRawAbstractType(rawComparator, subComparator);
-
-            AbstractType<?> keyValidator = cf_def.isSetKey_validation_class() ? TypeParser.parse(cf_def.key_validation_class) : null;
-
-            // Convert the REGULAR definitions from the input CfDef
-            List<ColumnDefinition> defs = ColumnDefinition.fromThrift(cf_def.keyspace, cf_def.name, rawComparator, subComparator, cf_def.column_metadata);
-
-            // Add the keyAlias if there is one, since that's on CQL metadata that thrift can actually change (for
-            // historical reasons)
-            boolean hasKeyAlias = cf_def.isSetKey_alias() && keyValidator != null && !(keyValidator instanceof CompositeType);
-            if (hasKeyAlias)
-                defs.add(ColumnDefinition.partitionKeyDef(cf_def.keyspace, cf_def.name, cf_def.key_alias, keyValidator, null));
-
-            // Now add any CQL metadata that we want to copy, skipping the keyAlias if there was one
-            for (ColumnDefinition def : previousCQLMetadata)
-            {
-                // isPartOfCellName basically means 'is not just a CQL metadata'
-                if (def.isPartOfCellName())
-                    continue;
-
-                if (def.kind == ColumnDefinition.Kind.PARTITION_KEY && hasKeyAlias)
-                    continue;
-
-                defs.add(def);
-            }
-
-            CellNameType comparator = CellNames.fromAbstractType(fullRawComparator, calculateIsDense(fullRawComparator, defs));
-
-            UUID cfId = Schema.instance.getId(cf_def.keyspace, cf_def.name);
-            if (cfId == null)
-                cfId = UUIDGen.getTimeUUID();
-
-            CFMetaData newCFMD = new CFMetaData(cf_def.keyspace, cf_def.name, cfType, comparator, cfId);
-
-            newCFMD.addAllColumnDefinitions(defs);
-
-            if (keyValidator != null)
-                newCFMD.keyValidator(keyValidator);
-            if (cf_def.isSetGc_grace_seconds())
-                newCFMD.gcGraceSeconds(cf_def.gc_grace_seconds);
-            if (cf_def.isSetMin_compaction_threshold())
-                newCFMD.minCompactionThreshold(cf_def.min_compaction_threshold);
-            if (cf_def.isSetMax_compaction_threshold())
-                newCFMD.maxCompactionThreshold(cf_def.max_compaction_threshold);
-            if (cf_def.isSetCompaction_strategy())
-                newCFMD.compactionStrategyClass(createCompactionStrategy(cf_def.compaction_strategy));
-            if (cf_def.isSetCompaction_strategy_options())
-                newCFMD.compactionStrategyOptions(new HashMap<>(cf_def.compaction_strategy_options));
-            if (cf_def.isSetBloom_filter_fp_chance())
-                newCFMD.bloomFilterFpChance(cf_def.bloom_filter_fp_chance);
-            if (cf_def.isSetMemtable_flush_period_in_ms())
-                newCFMD.memtableFlushPeriod(cf_def.memtable_flush_period_in_ms);
-            if (cf_def.isSetCaching() || cf_def.isSetCells_per_row_to_cache())
-                newCFMD.caching(CachingOptions.fromThrift(cf_def.caching, cf_def.cells_per_row_to_cache));
-            if (cf_def.isSetRead_repair_chance())
-                newCFMD.readRepairChance(cf_def.read_repair_chance);
-            if (cf_def.isSetDefault_time_to_live())
-                newCFMD.defaultTimeToLive(cf_def.default_time_to_live);
-            if (cf_def.isSetDclocal_read_repair_chance())
-                newCFMD.dcLocalReadRepairChance(cf_def.dclocal_read_repair_chance);
-            if (cf_def.isSetMin_index_interval())
-                newCFMD.minIndexInterval(cf_def.min_index_interval);
-            if (cf_def.isSetMax_index_interval())
-                newCFMD.maxIndexInterval(cf_def.max_index_interval);
-            if (cf_def.isSetSpeculative_retry())
-                newCFMD.speculativeRetry(SpeculativeRetry.fromString(cf_def.speculative_retry));
-            if (cf_def.isSetTriggers())
-                newCFMD.triggers(TriggerDefinition.fromThrift(cf_def.triggers));
-
-            return newCFMD.comment(cf_def.comment)
-                          .defaultValidator(TypeParser.parse(cf_def.default_validation_class))
-                          .compressionParameters(CompressionParameters.create(cf_def.compression_options))
-                          .rebuild();
-        }
-        catch (SyntaxException | MarshalException e)
-        {
-            throw new ConfigurationException(e.getMessage());
-        }
-    }
-
-    /**
-     * Create CFMetaData from thrift {@link CqlRow} that contains columns from schema_columnfamilies.
-     *
-     * @param columnsRes CqlRow containing columns from schema_columnfamilies.
-     * @return CFMetaData derived from CqlRow
-     */
-    public static CFMetaData fromThriftCqlRow(CqlRow cf, CqlResult columnsRes)
-    {
-        UntypedResultSet.Row cfRow = new UntypedResultSet.Row(convertThriftCqlRow(cf));
-
-        List<Map<String, ByteBuffer>> cols = new ArrayList<>(columnsRes.rows.size());
-        for (CqlRow row : columnsRes.rows)
-            cols.add(convertThriftCqlRow(row));
-        UntypedResultSet colsRow = UntypedResultSet.create(cols);
-
-        return fromSchemaNoTriggers(cfRow, colsRow);
-    }
-
-    private static Map<String, ByteBuffer> convertThriftCqlRow(CqlRow row)
-    {
-        Map<String, ByteBuffer> m = new HashMap<>();
-        for (org.apache.cassandra.thrift.Column column : row.getColumns())
-            m.put(UTF8Type.instance.getString(column.bufferForName()), column.value);
-        return m;
-    }
-
     /**
      * Updates this object in place to match the definition in the system schema tables.
      * @return true if any columns were added, removed, or altered; otherwise, false is returned
      */
     public boolean reload()
     {
-        Row cfDefRow = SystemKeyspace.readSchemaRow(SystemKeyspace.SCHEMA_COLUMNFAMILIES_CF, ksName, cfName);
-
-        if (cfDefRow.cf == null || !cfDefRow.cf.hasColumns())
-            throw new RuntimeException(String.format("%s not found in the schema definitions keyspace.", ksName + ":" + cfName));
-
-        try
-        {
-            return apply(fromSchema(cfDefRow));
-        }
-        catch (ConfigurationException e)
-        {
-            throw new RuntimeException(e);
-        }
+        return apply(LegacySchemaTables.createTableFromName(ksName, cfName));
     }
 
     /**
-     * Updates CFMetaData in-place to match cf_def
-     *
-     * *Note*: This method left package-private only for DefsTest, don't use directly!
+     * Updates CFMetaData in-place to match cfm
      *
      * @return true if any columns were added, removed, or altered; otherwise, false is returned
      * @throws ConfigurationException if ks/cf names or cf ids didn't match
      */
-    boolean apply(CFMetaData cfm) throws ConfigurationException
+    @VisibleForTesting
+    public boolean apply(CFMetaData cfm) throws ConfigurationException
     {
         logger.debug("applying {} to {}", cfm, this);
 
@@ -1186,7 +765,7 @@
         minCompactionThreshold = cfm.minCompactionThreshold;
         maxCompactionThreshold = cfm.maxCompactionThreshold;
 
-        bloomFilterFpChance = cfm.bloomFilterFpChance;
+        bloomFilterFpChance = cfm.getBloomFilterFpChance();
         caching = cfm.caching;
         minIndexInterval = cfm.minIndexInterval;
         maxIndexInterval = cfm.maxIndexInterval;
@@ -1242,11 +821,11 @@
             throw new ConfigurationException(String.format("Column family ID mismatch (found %s; expected %s)",
                                                            cfm.cfId, cfId));
 
-        if (!cfm.cfType.equals(cfType))
-            throw new ConfigurationException("types do not match.");
+        if (cfm.cfType != cfType)
+            throw new ConfigurationException(String.format("Column family types do not match (found %s; expected %s).", cfm.cfType, cfType));
 
         if (!cfm.comparator.isCompatibleWith(comparator))
-            throw new ConfigurationException("comparators do not match or are not compatible.");
+            throw new ConfigurationException(String.format("Column family comparators do not match or are not compatible (found %s; expected %s).", cfm.comparator.getClass().getSimpleName(), comparator.getClass().getSimpleName()));
     }
 
     public static void validateCompactionOptions(Class<? extends AbstractCompactionStrategy> strategyClass, Map<String, String> options) throws ConfigurationException
@@ -1268,7 +847,7 @@
         {
             if (e.getTargetException() instanceof ConfigurationException)
                 throw (ConfigurationException) e.getTargetException();
-            throw new ConfigurationException("Failed to validate compaction options");
+            throw new ConfigurationException("Failed to validate compaction options: " + options);
         }
         catch (ConfigurationException e)
         {
@@ -1276,7 +855,7 @@
         }
         catch (Exception e)
         {
-            throw new ConfigurationException("Failed to validate compaction options");
+            throw new ConfigurationException("Failed to validate compaction options: " + options);
         }
     }
 
@@ -1314,51 +893,6 @@
         return createCompactionStrategyInstance(compactionStrategyClass, cfs, compactionStrategyOptions);
     }
 
-    // converts CFM to thrift CfDef
-    public org.apache.cassandra.thrift.CfDef toThrift()
-    {
-        org.apache.cassandra.thrift.CfDef def = new org.apache.cassandra.thrift.CfDef(ksName, cfName);
-        def.setColumn_type(cfType.name());
-
-        if (isSuper())
-        {
-            def.setComparator_type(comparator.subtype(0).toString());
-            def.setSubcomparator_type(comparator.subtype(1).toString());
-        }
-        else
-        {
-            def.setComparator_type(comparator.toString());
-        }
-
-        def.setComment(Strings.nullToEmpty(comment));
-        def.setRead_repair_chance(readRepairChance);
-        def.setDclocal_read_repair_chance(dcLocalReadRepairChance);
-        def.setGc_grace_seconds(gcGraceSeconds);
-        def.setDefault_validation_class(defaultValidator == null ? null : defaultValidator.toString());
-        def.setKey_validation_class(keyValidator.toString());
-        def.setMin_compaction_threshold(minCompactionThreshold);
-        def.setMax_compaction_threshold(maxCompactionThreshold);
-        // We only return the alias if only one is set since thrift don't know about multiple key aliases
-        if (partitionKeyColumns.size() == 1)
-            def.setKey_alias(partitionKeyColumns.get(0).name.bytes);
-        def.setColumn_metadata(ColumnDefinition.toThrift(columnMetadata));
-        def.setCompaction_strategy(compactionStrategyClass.getName());
-        def.setCompaction_strategy_options(new HashMap<>(compactionStrategyOptions));
-        def.setCompression_options(compressionParameters.asThriftOptions());
-        if (bloomFilterFpChance != null)
-            def.setBloom_filter_fp_chance(bloomFilterFpChance);
-        def.setMin_index_interval(minIndexInterval);
-        def.setMax_index_interval(maxIndexInterval);
-        def.setMemtable_flush_period_in_ms(memtableFlushPeriod);
-        def.setCaching(caching.toThriftCaching());
-        def.setCells_per_row_to_cache(caching.toThriftCellsPerRow());
-        def.setDefault_time_to_live(defaultTimeToLive);
-        def.setSpeculative_retry(speculativeRetry.toString());
-        def.setTriggers(TriggerDefinition.toThrift(triggers));
-
-        return def;
-    }
-
     /**
      * Returns the ColumnDefinition for {@code name}.
      */
@@ -1455,17 +989,17 @@
         return (cfName + "_" + columnName + "_idx").replaceAll("\\W", "");
     }
 
-    public Iterator<OnDiskAtom> getOnDiskIterator(DataInput in, Descriptor.Version version)
+    public Iterator<OnDiskAtom> getOnDiskIterator(FileDataInput in, Version version)
     {
         return getOnDiskIterator(in, ColumnSerializer.Flag.LOCAL, Integer.MIN_VALUE, version);
     }
 
-    public Iterator<OnDiskAtom> getOnDiskIterator(DataInput in, ColumnSerializer.Flag flag, int expireBefore, Descriptor.Version version)
+    public Iterator<OnDiskAtom> getOnDiskIterator(FileDataInput in, ColumnSerializer.Flag flag, int expireBefore, Version version)
     {
-        return AbstractCell.onDiskIterator(in, flag, expireBefore, version, comparator);
+        return version.getSSTableFormat().getOnDiskIterator(in, flag, expireBefore, this, version);
     }
 
-    public AtomDeserializer getOnDiskDeserializer(DataInput in, Descriptor.Version version)
+    public AtomDeserializer getOnDiskDeserializer(DataInput in, Version version)
     {
         return new AtomDeserializer(comparator, in, ColumnSerializer.Flag.LOCAL, Integer.MIN_VALUE, version);
     }
@@ -1591,89 +1125,6 @@
                                                            "interval (%d).", maxIndexInterval, minIndexInterval));
     }
 
-    /**
-     * Create schema mutations to update this metadata to provided new state.
-     *
-     * @param newState The new metadata (for the same CF)
-     * @param modificationTimestamp Timestamp to use for mutation
-     * @param fromThrift whether the newState comes from thrift
-     *
-     * @return Difference between attributes in form of schema mutation
-     */
-    public Mutation toSchemaUpdate(CFMetaData newState, long modificationTimestamp, boolean fromThrift)
-    {
-        Mutation mutation = new Mutation(Keyspace.SYSTEM_KS, SystemKeyspace.getSchemaKSKey(ksName));
-
-        newState.toSchemaNoColumnsNoTriggers(mutation, modificationTimestamp);
-
-        MapDifference<ByteBuffer, ColumnDefinition> columnDiff = Maps.difference(columnMetadata, newState.columnMetadata);
-
-        // columns that are no longer needed
-        for (ColumnDefinition cd : columnDiff.entriesOnlyOnLeft().values())
-        {
-            // Thrift only knows about the REGULAR ColumnDefinition type, so don't consider other type
-            // are being deleted just because they are not here.
-            if (fromThrift && cd.kind != ColumnDefinition.Kind.REGULAR)
-                continue;
-
-            cd.deleteFromSchema(mutation, modificationTimestamp);
-        }
-
-        // newly added columns
-        for (ColumnDefinition cd : columnDiff.entriesOnlyOnRight().values())
-            cd.toSchema(mutation, modificationTimestamp);
-
-        // old columns with updated attributes
-        for (ByteBuffer name : columnDiff.entriesDiffering().keySet())
-        {
-            ColumnDefinition cd = newState.columnMetadata.get(name);
-            cd.toSchema(mutation, modificationTimestamp);
-        }
-
-        MapDifference<String, TriggerDefinition> triggerDiff = Maps.difference(triggers, newState.triggers);
-
-        // dropped triggers
-        for (TriggerDefinition td : triggerDiff.entriesOnlyOnLeft().values())
-            td.deleteFromSchema(mutation, cfName, modificationTimestamp);
-
-        // newly created triggers
-        for (TriggerDefinition td : triggerDiff.entriesOnlyOnRight().values())
-            td.toSchema(mutation, cfName, modificationTimestamp);
-
-        return mutation;
-    }
-
-    /**
-     * Remove all CF attributes from schema
-     *
-     * @param timestamp Timestamp to use
-     *
-     * @return Mutation to use to completely remove cf from schema
-     */
-    public Mutation dropFromSchema(long timestamp)
-    {
-        Mutation mutation = new Mutation(Keyspace.SYSTEM_KS, SystemKeyspace.getSchemaKSKey(ksName));
-        ColumnFamily cf = mutation.addOrGet(SchemaColumnFamiliesCf);
-        int ldt = (int) (System.currentTimeMillis() / 1000);
-
-        Composite prefix = SchemaColumnFamiliesCf.comparator.make(cfName);
-        cf.addAtom(new RangeTombstone(prefix, prefix.end(), timestamp, ldt));
-
-        for (ColumnDefinition cd : allColumns())
-            cd.deleteFromSchema(mutation, timestamp);
-
-        for (TriggerDefinition td : triggers.values())
-            td.deleteFromSchema(mutation, cfName, timestamp);
-
-        for (String indexName : Keyspace.open(this.ksName).getColumnFamilyStore(this.cfName).getBuiltIndexes())
-        {
-            ColumnFamily indexCf = mutation.addOrGet(IndexCf);
-            indexCf.addTombstone(indexCf.getComparator().makeCellName(indexName), ldt, timestamp);
-        }
-
-        return mutation;
-    }
-
     public boolean isPurged()
     {
         return isPurged;
@@ -1684,254 +1135,6 @@
         isPurged = true;
     }
 
-    public void toSchema(Mutation mutation, long timestamp)
-    {
-        toSchemaNoColumnsNoTriggers(mutation, timestamp);
-
-        for (TriggerDefinition td : triggers.values())
-            td.toSchema(mutation, cfName, timestamp);
-
-        for (ColumnDefinition cd : allColumns())
-            cd.toSchema(mutation, timestamp);
-    }
-
-    private void toSchemaNoColumnsNoTriggers(Mutation mutation, long timestamp)
-    {
-        // For property that can be null (and can be changed), we insert tombstones, to make sure
-        // we don't keep a property the user has removed
-        ColumnFamily cf = mutation.addOrGet(SchemaColumnFamiliesCf);
-        Composite prefix = SchemaColumnFamiliesCf.comparator.make(cfName);
-        CFRowAdder adder = new CFRowAdder(cf, prefix, timestamp);
-
-        adder.add("cf_id", cfId);
-        adder.add("type", cfType.toString());
-
-        if (isSuper())
-        {
-            // We need to continue saving the comparator and subcomparator separatly, otherwise
-            // we won't know at deserialization if the subcomparator should be taken into account
-            // TODO: we should implement an on-start migration if we want to get rid of that.
-            adder.add("comparator", comparator.subtype(0).toString());
-            adder.add("subcomparator", comparator.subtype(1).toString());
-        }
-        else
-        {
-            adder.add("comparator", comparator.toString());
-        }
-
-        adder.add("comment", comment);
-        adder.add("read_repair_chance", readRepairChance);
-        adder.add("local_read_repair_chance", dcLocalReadRepairChance);
-        adder.add("gc_grace_seconds", gcGraceSeconds);
-        adder.add("default_validator", defaultValidator.toString());
-        adder.add("key_validator", keyValidator.toString());
-        adder.add("min_compaction_threshold", minCompactionThreshold);
-        adder.add("max_compaction_threshold", maxCompactionThreshold);
-        adder.add("bloom_filter_fp_chance", bloomFilterFpChance);
-
-        adder.add("memtable_flush_period_in_ms", memtableFlushPeriod);
-        adder.add("caching", caching.toString());
-        adder.add("default_time_to_live", defaultTimeToLive);
-        adder.add("compaction_strategy_class", compactionStrategyClass.getName());
-        adder.add("compression_parameters", json(compressionParameters.asThriftOptions()));
-        adder.add("compaction_strategy_options", json(compactionStrategyOptions));
-        adder.add("min_index_interval", minIndexInterval);
-        adder.add("max_index_interval", maxIndexInterval);
-        adder.add("index_interval", null);
-        adder.add("speculative_retry", speculativeRetry.toString());
-
-        for (Map.Entry<ColumnIdentifier, Long> entry : droppedColumns.entrySet())
-            adder.addMapEntry("dropped_columns", entry.getKey().toString(), entry.getValue());
-
-        adder.add("is_dense", isDense);
-
-        // Save the CQL3 metadata "the old way" for compatibility sake
-        adder.add("key_aliases", aliasesToJson(partitionKeyColumns));
-        adder.add("column_aliases", aliasesToJson(clusteringColumns));
-        adder.add("value_alias", compactValueColumn == null ? null : compactValueColumn.name.toString());
-    }
-
-    // Package protected for use by tests
-    static CFMetaData fromSchemaNoTriggers(UntypedResultSet.Row result, UntypedResultSet serializedColumnDefinitions)
-    {
-        try
-        {
-            String ksName = result.getString("keyspace_name");
-            String cfName = result.getString("columnfamily_name");
-
-            AbstractType<?> rawComparator = TypeParser.parse(result.getString("comparator"));
-            AbstractType<?> subComparator = result.has("subcomparator") ? TypeParser.parse(result.getString("subcomparator")) : null;
-            ColumnFamilyType cfType = ColumnFamilyType.valueOf(result.getString("type"));
-
-            AbstractType<?> fullRawComparator = makeRawAbstractType(rawComparator, subComparator);
-
-            List<ColumnDefinition> columnDefs = ColumnDefinition.fromSchema(serializedColumnDefinitions,
-                                                                            ksName,
-                                                                            cfName,
-                                                                            fullRawComparator,
-                                                                            cfType == ColumnFamilyType.Super);
-
-            boolean isDense = result.has("is_dense")
-                            ? result.getBoolean("is_dense")
-                            : calculateIsDense(fullRawComparator, columnDefs);
-
-            CellNameType comparator = CellNames.fromAbstractType(fullRawComparator, isDense);
-
-            // if we are upgrading, we use id generated from names initially
-            UUID cfId = result.has("cf_id")
-                      ? result.getUUID("cf_id")
-                      : generateLegacyCfId(ksName, cfName);
-
-            CFMetaData cfm = new CFMetaData(ksName, cfName, cfType, comparator, cfId);
-            cfm.isDense(isDense);
-
-            cfm.readRepairChance(result.getDouble("read_repair_chance"));
-            cfm.dcLocalReadRepairChance(result.getDouble("local_read_repair_chance"));
-            cfm.gcGraceSeconds(result.getInt("gc_grace_seconds"));
-            cfm.defaultValidator(TypeParser.parse(result.getString("default_validator")));
-            cfm.keyValidator(TypeParser.parse(result.getString("key_validator")));
-            cfm.minCompactionThreshold(result.getInt("min_compaction_threshold"));
-            cfm.maxCompactionThreshold(result.getInt("max_compaction_threshold"));
-            if (result.has("comment"))
-                cfm.comment(result.getString("comment"));
-            if (result.has("bloom_filter_fp_chance"))
-                cfm.bloomFilterFpChance(result.getDouble("bloom_filter_fp_chance"));
-            if (result.has("memtable_flush_period_in_ms"))
-                cfm.memtableFlushPeriod(result.getInt("memtable_flush_period_in_ms"));
-            cfm.caching(CachingOptions.fromString(result.getString("caching")));
-            if (result.has("default_time_to_live"))
-                cfm.defaultTimeToLive(result.getInt("default_time_to_live"));
-            if (result.has("speculative_retry"))
-                cfm.speculativeRetry(SpeculativeRetry.fromString(result.getString("speculative_retry")));
-            cfm.compactionStrategyClass(createCompactionStrategy(result.getString("compaction_strategy_class")));
-            cfm.compressionParameters(CompressionParameters.create(fromJsonMap(result.getString("compression_parameters"))));
-            cfm.compactionStrategyOptions(fromJsonMap(result.getString("compaction_strategy_options")));
-
-            // migrate old index_interval values to min_index_interval, if present
-            if (result.has("min_index_interval"))
-                cfm.minIndexInterval(result.getInt("min_index_interval"));
-            else if (result.has("index_interval"))
-                cfm.minIndexInterval(result.getInt("index_interval"));
-            if (result.has("max_index_interval"))
-                cfm.maxIndexInterval(result.getInt("max_index_interval"));
-
-            /*
-             * The info previously hold by key_aliases, column_aliases and value_alias is now stored in columnMetadata (because 1) this
-             * make more sense and 2) this allow to store indexing information).
-             * However, for upgrade sake we need to still be able to read those old values. Moreover, we cannot easily
-             * remove those old columns once "converted" to columnMetadata because that would screw up nodes that may
-             * not have upgraded. So for now we keep the both info and in sync, even though its redundant.
-             */
-            if (result.has("key_aliases"))
-                cfm.addColumnMetadataFromAliases(aliasesFromStrings(fromJsonList(result.getString("key_aliases"))), cfm.keyValidator, ColumnDefinition.Kind.PARTITION_KEY);
-            if (result.has("column_aliases"))
-                cfm.addColumnMetadataFromAliases(aliasesFromStrings(fromJsonList(result.getString("column_aliases"))), cfm.comparator.asAbstractType(), ColumnDefinition.Kind.CLUSTERING_COLUMN);
-            if (result.has("value_alias"))
-                cfm.addColumnMetadataFromAliases(Collections.singletonList(result.getBytes("value_alias")), cfm.defaultValidator, ColumnDefinition.Kind.COMPACT_VALUE);
-
-            if (result.has("dropped_columns"))
-                cfm.droppedColumns(convertDroppedColumns(result.getMap("dropped_columns", UTF8Type.instance, LongType.instance)));
-
-            for (ColumnDefinition cd : columnDefs)
-                cfm.addOrReplaceColumnDefinition(cd);
-
-            return cfm.rebuild();
-        }
-        catch (SyntaxException | ConfigurationException e)
-        {
-            throw new RuntimeException(e);
-        }
-    }
-
-    public void addColumnMetadataFromAliases(List<ByteBuffer> aliases, AbstractType<?> comparator, ColumnDefinition.Kind kind)
-    {
-        if (comparator instanceof CompositeType)
-        {
-            CompositeType ct = (CompositeType)comparator;
-            for (int i = 0; i < aliases.size(); ++i)
-            {
-                if (aliases.get(i) != null)
-                {
-                    addOrReplaceColumnDefinition(new ColumnDefinition(this, aliases.get(i), ct.types.get(i), i, kind));
-                }
-            }
-        }
-        else
-        {
-            assert aliases.size() <= 1;
-            if (!aliases.isEmpty() && aliases.get(0) != null)
-                addOrReplaceColumnDefinition(new ColumnDefinition(this, aliases.get(0), comparator, null, kind));
-        }
-    }
-
-    /**
-     * Deserialize CF metadata from low-level representation
-     *
-     * @return Thrift-based metadata deserialized from schema
-     */
-    public static CFMetaData fromSchema(UntypedResultSet.Row result)
-    {
-        String ksName = result.getString("keyspace_name");
-        String cfName = result.getString("columnfamily_name");
-
-        Row serializedColumns = SystemKeyspace.readSchemaRow(SystemKeyspace.SCHEMA_COLUMNS_CF, ksName, cfName);
-        CFMetaData cfm = fromSchemaNoTriggers(result, ColumnDefinition.resultify(serializedColumns));
-
-        Row serializedTriggers = SystemKeyspace.readSchemaRow(SystemKeyspace.SCHEMA_TRIGGERS_CF, ksName, cfName);
-        addTriggerDefinitionsFromSchema(cfm, serializedTriggers);
-
-        return cfm;
-    }
-
-    private static CFMetaData fromSchema(Row row)
-    {
-        UntypedResultSet.Row result = QueryProcessor.resultify("SELECT * FROM system.schema_columnfamilies", row).one();
-        return fromSchema(result);
-    }
-
-    private String aliasesToJson(List<ColumnDefinition> rawAliases)
-    {
-        if (rawAliases == null)
-            return null;
-
-        List<String> aliases = new ArrayList<>(rawAliases.size());
-        for (ColumnDefinition rawAlias : rawAliases)
-            aliases.add(rawAlias.name.toString());
-        return json(aliases);
-    }
-
-    private static List<ByteBuffer> aliasesFromStrings(List<String> aliases)
-    {
-        List<ByteBuffer> rawAliases = new ArrayList<>(aliases.size());
-        for (String alias : aliases)
-            rawAliases.add(UTF8Type.instance.decompose(alias));
-        return rawAliases;
-    }
-
-    private static Map<ColumnIdentifier, Long> convertDroppedColumns(Map<String, Long> raw)
-    {
-        Map<ColumnIdentifier, Long> converted = Maps.newHashMap();
-        for (Map.Entry<String, Long> entry : raw.entrySet())
-            converted.put(new ColumnIdentifier(entry.getKey(), true), entry.getValue());
-        return converted;
-    }
-
-    /**
-     * Convert current metadata into schema mutation
-     *
-     * @param timestamp Timestamp to use
-     *
-     * @return Low-level representation of the CF
-     *
-     * @throws ConfigurationException if any of the attributes didn't pass validation
-     */
-    public Mutation toSchema(long timestamp) throws ConfigurationException
-    {
-        Mutation mutation = new Mutation(Keyspace.SYSTEM_KS, SystemKeyspace.getSchemaKSKey(ksName));
-        toSchema(mutation, timestamp);
-        return mutation;
-    }
-
     // The comparator to validate the definition name.
 
     public AbstractType<?> getColumnDefinitionComparator(ColumnDefinition def)
@@ -1988,12 +1191,6 @@
         return columnMetadata.remove(def.name.bytes) != null;
     }
 
-    private static void addTriggerDefinitionsFromSchema(CFMetaData cfDef, Row serializedTriggerDefinitions)
-    {
-        for (TriggerDefinition td : TriggerDefinition.fromSchema(serializedTriggerDefinitions))
-            cfDef.triggers.put(td.name, td);
-    }
-
     public void addTriggerDefinition(TriggerDefinition def) throws InvalidRequestException
     {
         if (containsTriggerDefinition(def))
@@ -2166,7 +1363,7 @@
      * information for table just created through thrift, nor for table prior to CASSANDRA-7744, so this
      * method does its best to infer whether the table is dense or not based on other elements.
      */
-    private static boolean calculateIsDense(AbstractType<?> comparator, Collection<ColumnDefinition> defs)
+    public static boolean calculateIsDense(AbstractType<?> comparator, Collection<ColumnDefinition> defs)
     {
         /*
          * As said above, this method is only here because we need to deal with thrift upgrades.
@@ -2285,7 +1482,7 @@
             .append("compactionStrategyClass", compactionStrategyClass)
             .append("compactionStrategyOptions", compactionStrategyOptions)
             .append("compressionParameters", compressionParameters.asThriftOptions())
-            .append("bloomFilterFpChance", bloomFilterFpChance)
+            .append("bloomFilterFpChance", getBloomFilterFpChance())
             .append("memtableFlushPeriod", memtableFlushPeriod)
             .append("caching", caching)
             .append("defaultTimeToLive", defaultTimeToLive)
diff --git a/src/java/org/apache/cassandra/config/ColumnDefinition.java b/src/java/org/apache/cassandra/config/ColumnDefinition.java
index 51a378c..b33718f 100644
--- a/src/java/org/apache/cassandra/config/ColumnDefinition.java
+++ b/src/java/org/apache/cassandra/config/ColumnDefinition.java
@@ -21,32 +21,16 @@
 import java.util.*;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Function;
 import com.google.common.base.Objects;
-import com.google.common.collect.Maps;
+import com.google.common.collect.Lists;
 
 import org.apache.cassandra.cql3.*;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.composites.Composite;
 import org.apache.cassandra.db.marshal.*;
-import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.exceptions.*;
-import org.apache.cassandra.thrift.ColumnDef;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.FBUtilities;
-
-import static org.apache.cassandra.utils.FBUtilities.json;
 
 public class ColumnDefinition extends ColumnSpecification
 {
-    // system.schema_columns column names
-    private static final String COLUMN_NAME = "column_name";
-    private static final String TYPE = "validator";
-    private static final String INDEX_TYPE = "index_type";
-    private static final String INDEX_OPTIONS = "index_options";
-    private static final String INDEX_NAME = "index_name";
-    private static final String COMPONENT_INDEX = "component_index";
-    private static final String KIND = "type";
-
     /*
      * The type of CQL3 column this definition represents.
      * There is 3 main type of CQL3 columns: those parts of the partition key,
@@ -55,7 +39,7 @@
      * column, whose name is not stored in the data contrarily to the column of
      * type REGULAR. Hence the COMPACT_VALUE type to distinguish it below.
      *
-     * Note that thrift/CQL2 only know about definitions of type REGULAR (and
+     * Note that thrift only knows about definitions of type REGULAR (and
      * the ones whose componentIndex == null).
      */
     public enum Kind
@@ -64,20 +48,7 @@
         CLUSTERING_COLUMN,
         REGULAR,
         STATIC,
-        COMPACT_VALUE;
-
-        public String serialize()
-        {
-            // For backward compatibility we need to special case CLUSTERING_COLUMN
-            return this == CLUSTERING_COLUMN ? "clustering_key" : this.toString().toLowerCase();
-        }
-
-        public static Kind deserialize(String value)
-        {
-            if (value.equalsIgnoreCase("clustering_key"))
-                return CLUSTERING_COLUMN;
-            return Enum.valueOf(Kind.class, value.toUpperCase());
-        }
+        COMPACT_VALUE
     }
 
     public final Kind kind;
@@ -175,11 +146,31 @@
         return componentIndex == null;
     }
 
+    public boolean isPartitionKey()
+    {
+        return kind == Kind.PARTITION_KEY;
+    }
+
+    public boolean isClusteringColumn()
+    {
+        return kind == Kind.CLUSTERING_COLUMN;
+    }
+
     public boolean isStatic()
     {
         return kind == Kind.STATIC;
     }
 
+    public boolean isRegular()
+    {
+        return kind == Kind.REGULAR;
+    }
+
+    public boolean isCompactValue()
+    {
+        return kind == Kind.COMPACT_VALUE;
+    }
+
     // The componentIndex. This never return null however for convenience sake:
     // if componentIndex == null, this return 0. So caller should first check
     // isOnAllComponents() to distinguish if that's a possibility.
@@ -239,15 +230,6 @@
         return kind == Kind.PARTITION_KEY || kind == Kind.CLUSTERING_COLUMN;
     }
 
-    public static List<ColumnDef> toThrift(Map<ByteBuffer, ColumnDefinition> columns)
-    {
-        List<ColumnDef> thriftDefs = new ArrayList<>(columns.size());
-        for (ColumnDefinition def : columns.values())
-            if (def.kind == ColumnDefinition.Kind.REGULAR)
-                thriftDefs.add(def.toThrift());
-        return thriftDefs;
-    }
-
     /**
      * Whether the name of this definition is serialized in the cell nane, i.e. whether
      * it's not just a non-stored CQL metadata.
@@ -257,86 +239,6 @@
         return kind == Kind.REGULAR || kind == Kind.STATIC;
     }
 
-    public ColumnDef toThrift()
-    {
-        ColumnDef cd = new ColumnDef();
-
-        cd.setName(ByteBufferUtil.clone(name.bytes));
-        cd.setValidation_class(type.toString());
-        cd.setIndex_type(indexType == null ? null : org.apache.cassandra.thrift.IndexType.valueOf(indexType.name()));
-        cd.setIndex_name(indexName == null ? null : indexName);
-        cd.setIndex_options(indexOptions == null ? null : Maps.newHashMap(indexOptions));
-
-        return cd;
-    }
-
-    public static ColumnDefinition fromThrift(String ksName, String cfName, AbstractType<?> thriftComparator, AbstractType<?> thriftSubcomparator, ColumnDef thriftColumnDef) throws SyntaxException, ConfigurationException
-    {
-        // For super columns, the componentIndex is 1 because the ColumnDefinition applies to the column component.
-        Integer componentIndex = thriftSubcomparator != null ? 1 : null;
-        AbstractType<?> comparator = thriftSubcomparator == null ? thriftComparator : thriftSubcomparator;
-        try
-        {
-            comparator.validate(thriftColumnDef.name);
-        }
-        catch (MarshalException e)
-        {
-            throw new ConfigurationException(String.format("Column name %s is not valid for comparator %s", ByteBufferUtil.bytesToHex(thriftColumnDef.name), comparator));
-        }
-
-        return new ColumnDefinition(ksName,
-                                    cfName,
-                                    new ColumnIdentifier(ByteBufferUtil.clone(thriftColumnDef.name), comparator),
-                                    TypeParser.parse(thriftColumnDef.validation_class),
-                                    thriftColumnDef.index_type == null ? null : IndexType.valueOf(thriftColumnDef.index_type.name()),
-                                    thriftColumnDef.index_options,
-                                    thriftColumnDef.index_name,
-                                    componentIndex,
-                                    Kind.REGULAR);
-    }
-
-    public static List<ColumnDefinition> fromThrift(String ksName, String cfName, AbstractType<?> thriftComparator, AbstractType<?> thriftSubcomparator, List<ColumnDef> thriftDefs) throws SyntaxException, ConfigurationException
-    {
-        if (thriftDefs == null)
-            return new ArrayList<>();
-
-        List<ColumnDefinition> defs = new ArrayList<>(thriftDefs.size());
-        for (ColumnDef thriftColumnDef : thriftDefs)
-            defs.add(fromThrift(ksName, cfName, thriftComparator, thriftSubcomparator, thriftColumnDef));
-
-        return defs;
-    }
-
-    /**
-     * Drop specified column from the schema using given mutation.
-     *
-     * @param mutation  The schema mutation
-     * @param timestamp The timestamp to use for column modification
-     */
-    public void deleteFromSchema(Mutation mutation, long timestamp)
-    {
-        ColumnFamily cf = mutation.addOrGet(CFMetaData.SchemaColumnsCf);
-        int ldt = (int) (System.currentTimeMillis() / 1000);
-
-        // Note: we do want to use name.toString(), not name.bytes directly for backward compatibility (For CQL3, this won't make a difference).
-        Composite prefix = CFMetaData.SchemaColumnsCf.comparator.make(cfName, name.toString());
-        cf.addAtom(new RangeTombstone(prefix, prefix.end(), timestamp, ldt));
-    }
-
-    public void toSchema(Mutation mutation, long timestamp)
-    {
-        ColumnFamily cf = mutation.addOrGet(CFMetaData.SchemaColumnsCf);
-        Composite prefix = CFMetaData.SchemaColumnsCf.comparator.make(cfName, name.toString());
-        CFRowAdder adder = new CFRowAdder(cf, prefix, timestamp);
-
-        adder.add(TYPE, type.toString());
-        adder.add(INDEX_TYPE, indexType == null ? null : indexType.toString());
-        adder.add(INDEX_OPTIONS, json(indexOptions));
-        adder.add(INDEX_NAME, indexName);
-        adder.add(COMPONENT_INDEX, componentIndex);
-        adder.add(KIND, kind.serialize());
-    }
-
     public ColumnDefinition apply(ColumnDefinition def)  throws ConfigurationException
     {
         assert kind == def.kind && Objects.equal(componentIndex, def.componentIndex);
@@ -350,7 +252,7 @@
 
             assert getIndexName() != null;
             if (!getIndexName().equals(def.getIndexName()))
-                throw new ConfigurationException("Cannot modify index name");
+                throw new ConfigurationException("Cannot modify index name: " + def.getIndexName());
         }
 
         return new ColumnDefinition(ksName,
@@ -364,82 +266,6 @@
                                     kind);
     }
 
-    public static UntypedResultSet resultify(Row serializedColumns)
-    {
-        String query = String.format("SELECT * FROM %s.%s", Keyspace.SYSTEM_KS, SystemKeyspace.SCHEMA_COLUMNS_CF);
-        return QueryProcessor.resultify(query, serializedColumns);
-    }
-
-    /**
-     * Deserialize columns from storage-level representation
-     *
-     * @param serializedColumns storage-level partition containing the column definitions
-     * @return the list of processed ColumnDefinitions
-     */
-    public static List<ColumnDefinition> fromSchema(UntypedResultSet serializedColumns, String ksName, String cfName, AbstractType<?> rawComparator, boolean isSuper)
-    {
-        List<ColumnDefinition> cds = new ArrayList<>();
-        for (UntypedResultSet.Row row : serializedColumns)
-        {
-            Kind kind = row.has(KIND)
-                      ? Kind.deserialize(row.getString(KIND))
-                      : Kind.REGULAR;
-
-            Integer componentIndex = null;
-            if (kind == Kind.REGULAR && isSuper)
-                componentIndex = 1; // A ColumnDefinition for super columns applies to the column component
-                                    // (we don't trust the COMPONENT_INDEX if we have one due to #9582)
-            else if (row.has(COMPONENT_INDEX))
-                componentIndex = row.getInt(COMPONENT_INDEX);
-
-            // Note: we save the column name as string, but we should not assume that it is an UTF8 name, we
-            // we need to use the comparator fromString method
-            AbstractType<?> comparator = getComponentComparator(rawComparator, componentIndex, kind);
-            ColumnIdentifier name = new ColumnIdentifier(comparator.fromString(row.getString(COLUMN_NAME)), comparator);
-
-            AbstractType<?> validator;
-            try
-            {
-                validator = TypeParser.parse(row.getString(TYPE));
-            }
-            catch (RequestValidationException e)
-            {
-                throw new RuntimeException(e);
-            }
-
-            IndexType indexType = null;
-            if (row.has(INDEX_TYPE))
-                indexType = IndexType.valueOf(row.getString(INDEX_TYPE));
-
-            Map<String, String> indexOptions = null;
-            if (row.has(INDEX_OPTIONS))
-                indexOptions = FBUtilities.fromJsonMap(row.getString(INDEX_OPTIONS));
-
-            String indexName = null;
-            if (row.has(INDEX_NAME))
-                indexName = row.getString(INDEX_NAME);
-
-            cds.add(new ColumnDefinition(ksName, cfName, name, validator, indexType, indexOptions, indexName, componentIndex, kind));
-        }
-
-        return cds;
-    }
-
-    public static AbstractType<?> getComponentComparator(AbstractType<?> rawComparator, Integer componentIndex, ColumnDefinition.Kind kind)
-    {
-        switch (kind)
-        {
-            case REGULAR:
-                if (componentIndex == null || (componentIndex == 0 && !(rawComparator instanceof CompositeType)))
-                    return rawComparator;
-
-                return ((CompositeType)rawComparator).types.get(componentIndex);
-            default:
-                // CQL3 column names are UTF8
-                return UTF8Type.instance;
-        }
-    }
-
     public String getIndexName()
     {
         return indexName;
@@ -489,4 +315,22 @@
     {
         return indexOptions != null && indexOptions.containsKey(name);
     }
+
+    /**
+     * Converts the specified column definitions into column identifiers.
+     *
+     * @param definitions the column definitions to convert.
+     * @return the column identifiers corresponding to the specified definitions
+     */
+    public static List<ColumnIdentifier> toIdentifiers(List<ColumnDefinition> definitions)
+    {
+        return Lists.transform(definitions, new Function<ColumnDefinition, ColumnIdentifier>()
+        {
+            @Override
+            public ColumnIdentifier apply(ColumnDefinition columnDef)
+            {
+                return columnDef.name;
+            }
+        });
+    }
 }
diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java
index 2c5e36a..64b36a0 100644
--- a/src/java/org/apache/cassandra/config/Config.java
+++ b/src/java/org/apache/cassandra/config/Config.java
@@ -17,20 +17,32 @@
  */
 package org.apache.cassandra.config;
 
+import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.io.StringReader;
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.TimeUnit;
 
+import com.google.common.base.Joiner;
 import com.google.common.collect.Sets;
-import org.supercsv.io.CsvListReader;
-import org.supercsv.prefs.CsvPreference;
+
+import org.apache.commons.lang3.builder.ToStringBuilder;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.EncryptionOptions.ClientEncryptionOptions;
 import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.io.util.NativeAllocator;
-import org.apache.cassandra.utils.FBUtilities;
+import org.supercsv.io.CsvListReader;
+import org.supercsv.prefs.CsvPreference;
 
 /**
  * A class that contains configuration properties for the cassandra node it runs within.
@@ -39,6 +51,8 @@
  */
 public class Config
 {
+    private static final Logger logger = LoggerFactory.getLogger(Config.class);
+
     /*
      * Prefix for Java properties for internal Cassandra configuration options
      */
@@ -48,9 +62,13 @@
     public String cluster_name = "Test Cluster";
     public String authenticator;
     public String authorizer;
+    public String role_manager;
     public volatile int permissions_validity_in_ms = 2000;
     public int permissions_cache_max_entries = 1000;
     public volatile int permissions_update_interval_in_ms = -1;
+    public volatile int roles_validity_in_ms = 2000;
+    public int roles_cache_max_entries = 1000;
+    public volatile int roles_update_interval_in_ms = -1;
 
     /* Hashing strategy Random or OPHF */
     public String partitioner;
@@ -61,7 +79,7 @@
     public Set<String> hinted_handoff_enabled_by_dc = Sets.newConcurrentHashSet();
     public volatile Integer max_hint_window_in_ms = 3 * 3600 * 1000; // three hours
 
-    public SeedProviderDef seed_provider;
+    public ParameterizedClass seed_provider;
     public DiskAccessMode disk_access_mode = DiskAccessMode.auto;
 
     public DiskFailurePolicy disk_failure_policy = DiskFailurePolicy.ignore;
@@ -109,8 +127,11 @@
     public String listen_interface;
     public Boolean listen_interface_prefer_ipv6 = false;
     public String broadcast_address;
+    public Boolean listen_on_broadcast_address = false;
     public String internode_authenticator;
 
+    /* intentionally left set to true, despite being set to false in stock 2.2 cassandra.yaml
+       we don't want to surprise Thrift users who have the setting blank in the yaml during 2.1->2.2 upgrade */
     public Boolean start_rpc = true;
     public String rpc_address;
     public String rpc_interface;
@@ -143,12 +164,17 @@
 
     /* if the size of columns or super-columns are more than this, indexing will kick in */
     public Integer column_index_size_in_kb = 64;
-    public Integer batch_size_warn_threshold_in_kb = 5;
+    public volatile int batch_size_warn_threshold_in_kb = 5;
+    public volatile int batch_size_fail_threshold_in_kb = 50;
     public Integer unlogged_batch_across_partitions_warn_threshold = 10;
     public Integer concurrent_compactors;
     public volatile Integer compaction_throughput_mb_per_sec = 16;
     public volatile Integer compaction_large_partition_warning_threshold_mb = 100;
 
+    /**
+     * @deprecated retry support removed on CASSANDRA-10992
+     */
+    @Deprecated
     public Integer max_streaming_retries = 3;
 
     public volatile Integer stream_throughput_outbound_megabits_per_sec = 200;
@@ -165,8 +191,9 @@
     public Double commitlog_sync_batch_window_in_ms;
     public Integer commitlog_sync_period_in_ms;
     public int commitlog_segment_size_in_mb = 32;
-    public boolean commitlog_segment_recycling = false;
-
+    public ParameterizedClass commitlog_compression;
+    public int commitlog_max_compression_buffers_in_pool = 3;
+ 
     @Deprecated
     public int commitlog_periodic_queue_size = -1;
 
@@ -203,6 +230,7 @@
     public volatile int key_cache_save_period = 14400;
     public volatile int key_cache_keys_to_save = Integer.MAX_VALUE;
 
+    public String row_cache_class_name = "org.apache.cassandra.cache.OHCProvider";
     public long row_cache_size_in_mb = 0;
     public volatile int row_cache_save_period = 0;
     public volatile int row_cache_keys_to_save = Integer.MAX_VALUE;
@@ -211,7 +239,8 @@
     public volatile int counter_cache_save_period = 7200;
     public volatile int counter_cache_keys_to_save = Integer.MAX_VALUE;
 
-    public String memory_allocator = NativeAllocator.class.getSimpleName();
+    @Deprecated
+    public String memory_allocator;
 
     private static boolean isClientMode = false;
 
@@ -221,7 +250,11 @@
 
     public MemtableAllocationType memtable_allocation_type = MemtableAllocationType.heap_buffers;
 
-    private static boolean outboundBindAny = false;
+    /**
+     * @deprecated No longer needed for streaming protocol. See CASSANDRA-12673 for details.
+     */
+    @Deprecated
+    protected static boolean outboundBindAny = false;
 
     public volatile int tombstone_warn_threshold = 1000;
     public volatile int tombstone_failure_threshold = 100000;
@@ -229,26 +262,36 @@
     public volatile Long index_summary_capacity_in_mb;
     public volatile int index_summary_resize_interval_in_minutes = 60;
 
+    public int gc_log_threshold_in_ms = 200;
     public int gc_warn_threshold_in_ms = 0;
 
     private static final CsvPreference STANDARD_SURROUNDING_SPACES_NEED_QUOTES = new CsvPreference.Builder(CsvPreference.STANDARD_PREFERENCE)
                                                                                                   .surroundingSpacesNeedQuotes(true).build();
 
+    // TTL for different types of trace events.
+    public int tracetype_query_ttl = (int) TimeUnit.DAYS.toSeconds(1);
+    public int tracetype_repair_ttl = (int) TimeUnit.DAYS.toSeconds(7);
+
     /*
      * Strategy to use for coalescing messages in OutboundTcpConnection.
      * Can be fixed, movingaverage, timehorizon, disabled. Setting is case and leading/trailing
      * whitespace insensitive. You can also specify a subclass of CoalescingStrategies.CoalescingStrategy by name.
      */
-    public String otc_coalescing_strategy = "DISABLED";
+    public String otc_coalescing_strategy = "TIMEHORIZON";
 
     /*
      * How many microseconds to wait for coalescing. For fixed strategy this is the amount of time after the first
-     * messgae is received before it will be sent with any accompanying messages. For moving average this is the
+     * message is received before it will be sent with any accompanying messages. For moving average this is the
      * maximum amount of time that will be waited as well as the interval at which messages must arrive on average
      * for coalescing to be enabled.
      */
     public static final int otc_coalescing_window_us_default = 200;
     public int otc_coalescing_window_us = otc_coalescing_window_us_default;
+    public int otc_coalescing_enough_coalesced_messages = 8;
+
+    public int windows_timer_interval = 0;
+
+    public boolean enable_user_defined_functions = false;
 
     public static boolean getOutboundBindAny()
     {
@@ -262,7 +305,7 @@
 
     public static boolean isClientMode()
     {
-       return isClientMode;
+        return isClientMode;
     }
 
     public static void setClientMode(boolean clientMode)
@@ -298,8 +341,10 @@
 
     public static List<String> parseHintedHandoffEnabledDCs(final String dcNames) throws IOException
     {
-        final CsvListReader csvListReader = new CsvListReader(new StringReader(dcNames), STANDARD_SURROUNDING_SPACES_NEED_QUOTES);
-        return csvListReader.read();
+        try (final CsvListReader csvListReader = new CsvListReader(new StringReader(dcNames), STANDARD_SURROUNDING_SPACES_NEED_QUOTES))
+        {
+        	return csvListReader.read();
+        }
     }
 
     public static enum CommitLogSync
@@ -349,4 +394,41 @@
     {
         keyspace
     }
+
+    private static final List<String> SENSITIVE_KEYS = new ArrayList<String>() {{
+        add("client_encryption_options");
+        add("server_encryption_options");
+    }};
+
+    public static void log(Config config)
+    {
+        Map<String, String> configMap = new TreeMap<>();
+        for (Field field : Config.class.getFields())
+        {
+            // ignore the constants
+            if (Modifier.isFinal(field.getModifiers()))
+                continue;
+
+            String name = field.getName();
+            if (SENSITIVE_KEYS.contains(name))
+            {
+                configMap.put(name, "<REDACTED>");
+                continue;
+            }
+
+            String value;
+            try
+            {
+                // Field.get() can throw NPE if the value of the field is null
+                value = field.get(config).toString();
+            }
+            catch (NullPointerException | IllegalAccessException npe)
+            {
+                value = "null";
+            }
+            configMap.put(name, value);
+        }
+
+        logger.info("Node configuration:[" + Joiner.on("; ").join(configMap.entrySet()) + "]");
+    }
 }
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 559ba0b..981026d 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -18,64 +18,43 @@
 package org.apache.cassandra.config;
 
 import java.io.File;
-import java.io.FileFilter;
 import java.io.IOException;
-import java.net.Inet4Address;
-import java.net.Inet6Address;
-import java.net.InetAddress;
-import java.net.NetworkInterface;
-import java.net.SocketException;
-import java.net.UnknownHostException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Enumeration;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
+import java.net.*;
+import java.nio.file.FileStore;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.*;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableSet;
+import com.google.common.primitives.Ints;
 import com.google.common.primitives.Longs;
 
-import org.apache.cassandra.thrift.ThriftServer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.cassandra.auth.AllowAllAuthenticator;
-import org.apache.cassandra.auth.AllowAllAuthorizer;
-import org.apache.cassandra.auth.AllowAllInternodeAuthenticator;
-import org.apache.cassandra.auth.IAuthenticator;
-import org.apache.cassandra.auth.IAuthorizer;
-import org.apache.cassandra.auth.IInternodeAuthenticator;
+
+import org.apache.cassandra.auth.*;
+import org.apache.cassandra.config.Config.CommitLogSync;
 import org.apache.cassandra.config.Config.RequestSchedulerId;
 import org.apache.cassandra.config.EncryptionOptions.ClientEncryptionOptions;
 import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions;
 import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.DefsTables;
 import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
 import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.io.util.IAllocator;
-import org.apache.cassandra.locator.DynamicEndpointSnitch;
-import org.apache.cassandra.locator.EndpointSnitchInfo;
-import org.apache.cassandra.locator.IEndpointSnitch;
-import org.apache.cassandra.locator.SeedProvider;
+import org.apache.cassandra.locator.*;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.scheduler.IRequestScheduler;
 import org.apache.cassandra.scheduler.NoScheduler;
 import org.apache.cassandra.service.CacheService;
-import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.thrift.ThriftServer;
 import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.JVMStabilityInspector;
-import org.apache.cassandra.utils.memory.HeapPool;
-import org.apache.cassandra.utils.memory.NativePool;
-import org.apache.cassandra.utils.memory.MemtablePool;
-import org.apache.cassandra.utils.memory.SlabPool;
+import org.apache.cassandra.utils.memory.*;
 
 public class DatabaseDescriptor
 {
@@ -103,8 +82,13 @@
 
     private static Config conf;
 
+    private static SSTableFormat.Type sstable_format = SSTableFormat.Type.BIG;
+
     private static IAuthenticator authenticator = new AllowAllAuthenticator();
     private static IAuthorizer authorizer = new AllowAllAuthorizer();
+    // Don't initialize the role manager until applying config. The options supported by CassandraRoleManager
+    // depend on the configured IAuthenticator, so defer creating it until that's been set.
+    private static IRoleManager roleManager;
 
     private static IRequestScheduler requestScheduler;
     private static RequestSchedulerId requestSchedulerId;
@@ -112,12 +96,25 @@
 
     private static long keyCacheSizeInMB;
     private static long counterCacheSizeInMB;
-    private static IAllocator memoryAllocator;
     private static long indexSummaryCapacityInMB;
 
     private static String localDC;
     private static Comparator<InetAddress> localComparator;
+    private static boolean hasLoggedConfig;
 
+    private static boolean daemonInitialized;
+
+    public static boolean isDaemonInitialized()
+    {
+        return daemonInitialized;
+    }
+
+    public static void setDaemonInitialized()
+    {
+        daemonInitialized = true;
+    }
+
+    public static void forceStaticInitialization() {}
     static
     {
         // In client mode, we use a default configuration. Note that the fields of this class will be
@@ -128,26 +125,15 @@
             if (Config.isClientMode())
             {
                 conf = new Config();
-                // at least we have to set memoryAllocator to open SSTable in client mode
-                memoryAllocator = FBUtilities.newOffHeapAllocator(conf.memory_allocator);
             }
             else
             {
                 applyConfig(loadConfig());
             }
         }
-        catch (ConfigurationException e)
-        {
-            logger.error("Fatal configuration error", e);
-            System.err.println(e.getMessage() + "\nFatal configuration error; unable to start. See log for stacktrace.");
-            System.exit(1);
-        }
         catch (Exception e)
         {
-            logger.error("Fatal error during configuration loading", e);
-            System.err.println(e.getMessage() + "\nFatal error during configuration loading; unable to start. See log for stacktrace.");
-            JVMStabilityInspector.inspectThrowable(e);
-            System.exit(1);
+            throw new ExceptionInInitializerError(e);
         }
     }
 
@@ -158,7 +144,15 @@
         ConfigurationLoader loader = loaderClass == null
                                    ? new YamlConfigurationLoader()
                                    : FBUtilities.<ConfigurationLoader>construct(loaderClass, "configuration loading");
-        return loader.loadConfig();
+        Config config = loader.loadConfig();
+
+        if (!hasLoggedConfig)
+        {
+            hasLoggedConfig = true;
+            Config.log(config);
+        }
+
+        return config;
     }
 
     private static InetAddress getNetworkInterfaceAddress(String intf, String configName, boolean preferIPv6) throws ConfigurationException
@@ -167,10 +161,10 @@
         {
             NetworkInterface ni = NetworkInterface.getByName(intf);
             if (ni == null)
-                throw new ConfigurationException("Configured " + configName + " \"" + intf + "\" could not be found");
+                throw new ConfigurationException("Configured " + configName + " \"" + intf + "\" could not be found", false);
             Enumeration<InetAddress> addrs = ni.getInetAddresses();
             if (!addrs.hasMoreElements())
-                throw new ConfigurationException("Configured " + configName + " \"" + intf + "\" was found, but had no addresses");
+                throw new ConfigurationException("Configured " + configName + " \"" + intf + "\" was found, but had no addresses", false);
 
             /*
              * Try to return the first address of the preferred type, otherwise return the first address
@@ -179,8 +173,8 @@
             while (addrs.hasMoreElements())
             {
                 InetAddress temp = addrs.nextElement();
-                if (preferIPv6 && temp.getClass() == Inet6Address.class) return temp;
-                if (!preferIPv6 && temp.getClass() == Inet4Address.class) return temp;
+                if (preferIPv6 && temp instanceof Inet6Address) return temp;
+                if (!preferIPv6 && temp instanceof Inet4Address) return temp;
                 if (retval == null) retval = temp;
             }
             return retval;
@@ -192,7 +186,7 @@
     }
 
     @VisibleForTesting
-    static void applyAddressConfig(Config config) throws ConfigurationException
+    public static void applyAddressConfig(Config config) throws ConfigurationException
     {
         listenAddress = null;
         rpcAddress = null;
@@ -202,7 +196,7 @@
         /* Local IP, hostname or interface to bind services to */
         if (config.listen_address != null && config.listen_interface != null)
         {
-            throw new ConfigurationException("Set listen_address OR listen_interface, not both");
+            throw new ConfigurationException("Set listen_address OR listen_interface, not both", false);
         }
         else if (config.listen_address != null)
         {
@@ -212,11 +206,11 @@
             }
             catch (UnknownHostException e)
             {
-                throw new ConfigurationException("Unknown listen_address '" + config.listen_address + "'");
+                throw new ConfigurationException("Unknown listen_address '" + config.listen_address + "'", false);
             }
 
             if (listenAddress.isAnyLocalAddress())
-                throw new ConfigurationException("listen_address cannot be a wildcard address (" + config.listen_address + ")!");
+                throw new ConfigurationException("listen_address cannot be a wildcard address (" + config.listen_address + ")!", false);
         }
         else if (config.listen_interface != null)
         {
@@ -232,17 +226,17 @@
             }
             catch (UnknownHostException e)
             {
-                throw new ConfigurationException("Unknown broadcast_address '" + config.broadcast_address + "'");
+                throw new ConfigurationException("Unknown broadcast_address '" + config.broadcast_address + "'", false);
             }
 
             if (broadcastAddress.isAnyLocalAddress())
-                throw new ConfigurationException("broadcast_address cannot be a wildcard address (" + config.broadcast_address + ")!");
+                throw new ConfigurationException("broadcast_address cannot be a wildcard address (" + config.broadcast_address + ")!", false);
         }
 
         /* Local IP, hostname or interface to bind RPC server to */
         if (config.rpc_address != null && config.rpc_interface != null)
         {
-            throw new ConfigurationException("Set rpc_address OR rpc_interface, not both");
+            throw new ConfigurationException("Set rpc_address OR rpc_interface, not both", false);
         }
         else if (config.rpc_address != null)
         {
@@ -252,7 +246,7 @@
             }
             catch (UnknownHostException e)
             {
-                throw new ConfigurationException("Unknown host in rpc_address " + config.rpc_address);
+                throw new ConfigurationException("Unknown host in rpc_address " + config.rpc_address, false);
             }
         }
         else if (config.rpc_interface != null)
@@ -273,39 +267,38 @@
             }
             catch (UnknownHostException e)
             {
-                throw new ConfigurationException("Unknown broadcast_rpc_address '" + config.broadcast_rpc_address + "'");
+                throw new ConfigurationException("Unknown broadcast_rpc_address '" + config.broadcast_rpc_address + "'", false);
             }
 
             if (broadcastRpcAddress.isAnyLocalAddress())
-                throw new ConfigurationException("broadcast_rpc_address cannot be a wildcard address (" + config.broadcast_rpc_address + ")!");
+                throw new ConfigurationException("broadcast_rpc_address cannot be a wildcard address (" + config.broadcast_rpc_address + ")!", false);
         }
         else
         {
             if (rpcAddress.isAnyLocalAddress())
                 throw new ConfigurationException("If rpc_address is set to a wildcard address (" + config.rpc_address + "), then " +
-                                                 "you must set broadcast_rpc_address to a value other than " + config.rpc_address);
-            broadcastRpcAddress = rpcAddress;
+                                                 "you must set broadcast_rpc_address to a value other than " + config.rpc_address, false);
         }
     }
 
-    private static void applyConfig(Config config) throws ConfigurationException
+    public static void applyConfig(Config config) throws ConfigurationException
     {
         conf = config;
 
         if (conf.commitlog_sync == null)
         {
-            throw new ConfigurationException("Missing required directive CommitLogSync");
+            throw new ConfigurationException("Missing required directive CommitLogSync", false);
         }
 
         if (conf.commitlog_sync == Config.CommitLogSync.batch)
         {
             if (conf.commitlog_sync_batch_window_in_ms == null)
             {
-                throw new ConfigurationException("Missing value for commitlog_sync_batch_window_in_ms: Double expected.");
+                throw new ConfigurationException("Missing value for commitlog_sync_batch_window_in_ms: Double expected.", false);
             }
             else if (conf.commitlog_sync_period_in_ms != null)
             {
-                throw new ConfigurationException("Batch sync specified, but commitlog_sync_period_in_ms found. Only specify commitlog_sync_batch_window_in_ms when using batch sync");
+                throw new ConfigurationException("Batch sync specified, but commitlog_sync_period_in_ms found. Only specify commitlog_sync_batch_window_in_ms when using batch sync", false);
             }
             logger.debug("Syncing log with a batch window of {}", conf.commitlog_sync_batch_window_in_ms);
         }
@@ -313,49 +306,35 @@
         {
             if (conf.commitlog_sync_period_in_ms == null)
             {
-                throw new ConfigurationException("Missing value for commitlog_sync_period_in_ms: Integer expected");
+                throw new ConfigurationException("Missing value for commitlog_sync_period_in_ms: Integer expected", false);
             }
             else if (conf.commitlog_sync_batch_window_in_ms != null)
             {
-                throw new ConfigurationException("commitlog_sync_period_in_ms specified, but commitlog_sync_batch_window_in_ms found.  Only specify commitlog_sync_period_in_ms when using periodic sync.");
+                throw new ConfigurationException("commitlog_sync_period_in_ms specified, but commitlog_sync_batch_window_in_ms found.  Only specify commitlog_sync_period_in_ms when using periodic sync.", false);
             }
             logger.debug("Syncing log with a period of {}", conf.commitlog_sync_period_in_ms);
         }
 
-        if (conf.commitlog_total_space_in_mb == null)
-            conf.commitlog_total_space_in_mb = hasLargeAddressSpace() ? 8192 : 32;
-
-        // Always force standard mode access on Windows - CASSANDRA-6993. Windows won't allow deletion of hard-links to files that
-        // are memory-mapped which causes trouble with snapshots.
-        if (FBUtilities.isWindows())
+        /* evaluate the DiskAccessMode Config directive, which also affects indexAccessMode selection */
+        if (conf.disk_access_mode == Config.DiskAccessMode.auto)
+        {
+            conf.disk_access_mode = hasLargeAddressSpace() ? Config.DiskAccessMode.mmap : Config.DiskAccessMode.standard;
+            indexAccessMode = conf.disk_access_mode;
+            logger.info("DiskAccessMode 'auto' determined to be {}, indexAccessMode is {}", conf.disk_access_mode, indexAccessMode);
+        }
+        else if (conf.disk_access_mode == Config.DiskAccessMode.mmap_index_only)
         {
             conf.disk_access_mode = Config.DiskAccessMode.standard;
-            indexAccessMode = conf.disk_access_mode;
-            logger.info("Windows environment detected.  DiskAccessMode set to {}, indexAccessMode {}", conf.disk_access_mode, indexAccessMode);
+            indexAccessMode = Config.DiskAccessMode.mmap;
+            logger.info("DiskAccessMode is {}, indexAccessMode is {}", conf.disk_access_mode, indexAccessMode);
         }
         else
         {
-            /* evaluate the DiskAccessMode Config directive, which also affects indexAccessMode selection */
-            if (conf.disk_access_mode == Config.DiskAccessMode.auto)
-            {
-                conf.disk_access_mode = hasLargeAddressSpace() ? Config.DiskAccessMode.mmap : Config.DiskAccessMode.standard;
-                indexAccessMode = conf.disk_access_mode;
-                logger.info("DiskAccessMode 'auto' determined to be {}, indexAccessMode is {}", conf.disk_access_mode, indexAccessMode);
-            }
-            else if (conf.disk_access_mode == Config.DiskAccessMode.mmap_index_only)
-            {
-                conf.disk_access_mode = Config.DiskAccessMode.standard;
-                indexAccessMode = Config.DiskAccessMode.mmap;
-                logger.info("DiskAccessMode is {}, indexAccessMode is {}", conf.disk_access_mode, indexAccessMode);
-            }
-            else
-            {
-                indexAccessMode = conf.disk_access_mode;
-                logger.info("DiskAccessMode is {}, indexAccessMode is {}", conf.disk_access_mode, indexAccessMode);
-            }
+            indexAccessMode = conf.disk_access_mode;
+            logger.info("DiskAccessMode is {}, indexAccessMode is {}", conf.disk_access_mode, indexAccessMode);
         }
 
-        /* Authentication and authorization backend, implementing IAuthenticator and IAuthorizer */
+        /* Authentication, authorization and role management backend, implementing IAuthenticator, IAuthorizer & IRoleMapper*/
         if (conf.authenticator != null)
             authenticator = FBUtilities.newAuthenticator(conf.authenticator);
 
@@ -363,7 +342,15 @@
             authorizer = FBUtilities.newAuthorizer(conf.authorizer);
 
         if (authenticator instanceof AllowAllAuthenticator && !(authorizer instanceof AllowAllAuthorizer))
-            throw new ConfigurationException("AllowAllAuthenticator can't be used with " +  conf.authorizer);
+            throw new ConfigurationException("AllowAllAuthenticator can't be used with " +  conf.authorizer, false);
+
+        if (conf.role_manager != null)
+            roleManager = FBUtilities.newRoleManager(conf.role_manager);
+        else
+            roleManager = new CassandraRoleManager();
+
+        if (authenticator instanceof PasswordAuthenticator && !(roleManager instanceof CassandraRoleManager))
+            throw new ConfigurationException("CassandraRoleManager must be used with PasswordAuthenticator", false);
 
         if (conf.internode_authenticator != null)
             internodeAuthenticator = FBUtilities.construct(conf.internode_authenticator, "internode_authenticator");
@@ -372,12 +359,13 @@
 
         authenticator.validateConfiguration();
         authorizer.validateConfiguration();
+        roleManager.validateConfiguration();
         internodeAuthenticator.validateConfiguration();
 
         /* Hashing strategy */
         if (conf.partitioner == null)
         {
-            throw new ConfigurationException("Missing directive: partitioner");
+            throw new ConfigurationException("Missing directive: partitioner", false);
         }
         try
         {
@@ -385,10 +373,15 @@
         }
         catch (Exception e)
         {
-            throw new ConfigurationException("Invalid partitioner class " + conf.partitioner);
+            throw new ConfigurationException("Invalid partitioner class " + conf.partitioner, false);
         }
         paritionerName = partitioner.getClass().getCanonicalName();
 
+        if (config.gc_log_threshold_in_ms < 0)
+        {
+            throw new ConfigurationException("gc_log_threshold_in_ms must be a positive integer");
+        }
+
         if (conf.gc_warn_threshold_in_ms < 0)
         {
             throw new ConfigurationException("gc_warn_threshold_in_ms must be a positive integer");
@@ -396,28 +389,28 @@
 
         if (conf.max_hint_window_in_ms == null)
         {
-            throw new ConfigurationException("max_hint_window_in_ms cannot be set to null");
+            throw new ConfigurationException("max_hint_window_in_ms cannot be set to null", false);
         }
 
         /* phi convict threshold for FailureDetector */
         if (conf.phi_convict_threshold < 5 || conf.phi_convict_threshold > 16)
         {
-            throw new ConfigurationException("phi_convict_threshold must be between 5 and 16");
+            throw new ConfigurationException("phi_convict_threshold must be between 5 and 16", false);
         }
 
         /* Thread per pool */
         if (conf.concurrent_reads != null && conf.concurrent_reads < 2)
         {
-            throw new ConfigurationException("concurrent_reads must be at least 2");
+            throw new ConfigurationException("concurrent_reads must be at least 2", false);
         }
 
         if (conf.concurrent_writes != null && conf.concurrent_writes < 2)
         {
-            throw new ConfigurationException("concurrent_writes must be at least 2");
+            throw new ConfigurationException("concurrent_writes must be at least 2", false);
         }
 
         if (conf.concurrent_counter_writes != null && conf.concurrent_counter_writes < 2)
-            throw new ConfigurationException("concurrent_counter_writes must be at least 2");
+            throw new ConfigurationException("concurrent_counter_writes must be at least 2", false);
 
         if (conf.concurrent_replicates != null)
             logger.warn("concurrent_replicates has been deprecated and should be removed from cassandra.yaml");
@@ -425,15 +418,18 @@
         if (conf.file_cache_size_in_mb == null)
             conf.file_cache_size_in_mb = Math.min(512, (int) (Runtime.getRuntime().maxMemory() / (4 * 1048576)));
 
+        if (conf.memory_allocator != null)
+            logger.warn("memory_allocator has been deprecated and should be removed from cassandra.yaml");
+
         if (conf.memtable_offheap_space_in_mb == null)
             conf.memtable_offheap_space_in_mb = (int) (Runtime.getRuntime().maxMemory() / (4 * 1048576));
         if (conf.memtable_offheap_space_in_mb < 0)
-            throw new ConfigurationException("memtable_offheap_space_in_mb must be positive");
+            throw new ConfigurationException("memtable_offheap_space_in_mb must be positive", false);
         // for the moment, we default to twice as much on-heap space as off-heap, as heap overhead is very large
         if (conf.memtable_heap_space_in_mb == null)
             conf.memtable_heap_space_in_mb = (int) (Runtime.getRuntime().maxMemory() / (4 * 1048576));
         if (conf.memtable_heap_space_in_mb <= 0)
-            throw new ConfigurationException("memtable_heap_space_in_mb must be positive");
+            throw new ConfigurationException("memtable_heap_space_in_mb must be positive", false);
         logger.info("Global memtable on-heap threshold is enabled at {}MB", conf.memtable_heap_space_in_mb);
         if (conf.memtable_offheap_space_in_mb == 0)
             logger.info("Global memtable off-heap threshold is disabled, HeapAllocator will be used instead");
@@ -443,23 +439,24 @@
         applyAddressConfig(config);
 
         if (conf.thrift_framed_transport_size_in_mb <= 0)
-            throw new ConfigurationException("thrift_framed_transport_size_in_mb must be positive");
+            throw new ConfigurationException("thrift_framed_transport_size_in_mb must be positive", false);
 
         if (conf.native_transport_max_frame_size_in_mb <= 0)
-            throw new ConfigurationException("native_transport_max_frame_size_in_mb must be positive");
+            throw new ConfigurationException("native_transport_max_frame_size_in_mb must be positive", false);
 
         // fail early instead of OOMing (see CASSANDRA-8116)
         if (ThriftServer.HSHA.equals(conf.rpc_server_type) && conf.rpc_max_threads == Integer.MAX_VALUE)
             throw new ConfigurationException("The hsha rpc_server_type is not compatible with an rpc_max_threads " +
                                              "setting of 'unlimited'.  Please see the comments in cassandra.yaml " +
-                                             "for rpc_server_type and rpc_max_threads.");
+                                             "for rpc_server_type and rpc_max_threads.",
+                                             false);
         if (ThriftServer.HSHA.equals(conf.rpc_server_type) && conf.rpc_max_threads > (FBUtilities.getAvailableProcessors() * 2 + 1024))
             logger.warn("rpc_max_threads setting of {} may be too high for the hsha server and cause unnecessary thread contention, reducing performance", conf.rpc_max_threads);
 
         /* end point snitch */
         if (conf.endpoint_snitch == null)
         {
-            throw new ConfigurationException("Missing endpoint_snitch directive");
+            throw new ConfigurationException("Missing endpoint_snitch directive", false);
         }
         snitch = createEndpointSnitch(conf.endpoint_snitch);
         EndpointSnitchInfo.create();
@@ -494,7 +491,7 @@
             }
             catch (ClassNotFoundException e)
             {
-                throw new ConfigurationException("Invalid Request Scheduler class " + conf.request_scheduler);
+                throw new ConfigurationException("Invalid Request Scheduler class " + conf.request_scheduler, false);
             }
             catch (Exception e)
             {
@@ -522,49 +519,93 @@
         {
             conf.commitlog_directory = System.getProperty("cassandra.storagedir", null);
             if (conf.commitlog_directory == null)
-                throw new ConfigurationException("commitlog_directory is missing and -Dcassandra.storagedir is not set");
+                throw new ConfigurationException("commitlog_directory is missing and -Dcassandra.storagedir is not set", false);
             conf.commitlog_directory += File.separator + "commitlog";
         }
+
+        if (conf.commitlog_total_space_in_mb == null)
+        {
+            int preferredSize = 8192;
+            int minSize = 0;
+            try
+            {
+                // use 1/4 of available space.  See discussion on #10013 and #10199
+                minSize = Ints.checkedCast((guessFileStore(conf.commitlog_directory).getTotalSpace() / 1048576) / 4);
+            }
+            catch (IOException e)
+            {
+                logger.debug("Error checking disk space", e);
+                throw new ConfigurationException(String.format("Unable to check disk space available to %s. Perhaps the Cassandra user does not have the necessary permissions",
+                                                               conf.commitlog_directory), e);
+            }
+            if (minSize < preferredSize)
+            {
+                logger.warn("Small commitlog volume detected at {}; setting commitlog_total_space_in_mb to {}.  You can override this in cassandra.yaml",
+                            conf.commitlog_directory, minSize);
+                conf.commitlog_total_space_in_mb = minSize;
+            }
+            else
+            {
+                conf.commitlog_total_space_in_mb = preferredSize;
+            }
+        }
+
         if (conf.saved_caches_directory == null)
         {
             conf.saved_caches_directory = System.getProperty("cassandra.storagedir", null);
             if (conf.saved_caches_directory == null)
-                throw new ConfigurationException("saved_caches_directory is missing and -Dcassandra.storagedir is not set");
+                throw new ConfigurationException("saved_caches_directory is missing and -Dcassandra.storagedir is not set", false);
             conf.saved_caches_directory += File.separator + "saved_caches";
         }
         if (conf.data_file_directories == null)
         {
             String defaultDataDir = System.getProperty("cassandra.storagedir", null);
             if (defaultDataDir == null)
-                throw new ConfigurationException("data_file_directories is not missing and -Dcassandra.storagedir is not set");
+                throw new ConfigurationException("data_file_directories is not missing and -Dcassandra.storagedir is not set", false);
             conf.data_file_directories = new String[]{ defaultDataDir + File.separator + "data" };
         }
 
+        long dataFreeBytes = 0;
         /* data file and commit log directories. they get created later, when they're needed. */
         for (String datadir : conf.data_file_directories)
         {
             if (datadir.equals(conf.commitlog_directory))
-                throw new ConfigurationException("commitlog_directory must not be the same as any data_file_directories");
+                throw new ConfigurationException("commitlog_directory must not be the same as any data_file_directories", false);
             if (datadir.equals(conf.saved_caches_directory))
-                throw new ConfigurationException("saved_caches_directory must not be the same as any data_file_directories");
+                throw new ConfigurationException("saved_caches_directory must not be the same as any data_file_directories", false);
+
+            try
+            {
+                dataFreeBytes += guessFileStore(datadir).getUnallocatedSpace();
+            }
+            catch (IOException e)
+            {
+                logger.debug("Error checking disk space", e);
+                throw new ConfigurationException(String.format("Unable to check disk space available to %s. Perhaps the Cassandra user does not have the necessary permissions",
+                                                               datadir), e);
+            }
         }
+        if (dataFreeBytes < 64L * 1024 * 1048576) // 64 GB
+            logger.warn("Only {} MB free across all data volumes. Consider adding more capacity to your cluster or removing obsolete snapshots",
+                        dataFreeBytes / 1048576);
+
 
         if (conf.commitlog_directory.equals(conf.saved_caches_directory))
-            throw new ConfigurationException("saved_caches_directory must not be the same as the commitlog_directory");
+            throw new ConfigurationException("saved_caches_directory must not be the same as the commitlog_directory", false);
 
         if (conf.memtable_flush_writers == null)
             conf.memtable_flush_writers = Math.min(8, Math.max(2, Math.min(FBUtilities.getAvailableProcessors(), conf.data_file_directories.length)));
 
         if (conf.memtable_flush_writers < 1)
-            throw new ConfigurationException("memtable_flush_writers must be at least 1");
+            throw new ConfigurationException("memtable_flush_writers must be at least 1", false);
 
         if (conf.memtable_cleanup_threshold == null)
             conf.memtable_cleanup_threshold = (float) (1.0 / (1 + conf.memtable_flush_writers));
 
         if (conf.memtable_cleanup_threshold < 0.01f)
-            throw new ConfigurationException("memtable_cleanup_threshold must be >= 0.01");
+            throw new ConfigurationException("memtable_cleanup_threshold must be >= 0.01", false);
         if (conf.memtable_cleanup_threshold > 0.99f)
-            throw new ConfigurationException("memtable_cleanup_threshold must be <= 0.99");
+            throw new ConfigurationException("memtable_cleanup_threshold must be <= 0.99", false);
         if (conf.memtable_cleanup_threshold < 0.1f)
             logger.warn("memtable_cleanup_threshold is set very low, which may cause performance degradation");
 
@@ -572,7 +613,7 @@
             conf.concurrent_compactors = Math.min(8, Math.max(2, Math.min(FBUtilities.getAvailableProcessors(), conf.data_file_directories.length)));
 
         if (conf.concurrent_compactors <= 0)
-            throw new ConfigurationException("concurrent_compactors should be strictly greater than 0");
+            throw new ConfigurationException("concurrent_compactors should be strictly greater than 0", false);
 
         if (conf.initial_token != null)
             for (String token : tokensFromString(conf.initial_token))
@@ -581,7 +622,7 @@
         if (conf.num_tokens == null)
         	conf.num_tokens = 1;
         else if (conf.num_tokens > MAX_NUM_TOKENS)
-            throw new ConfigurationException(String.format("A maximum number of %d tokens per node is supported", MAX_NUM_TOKENS));
+            throw new ConfigurationException(String.format("A maximum number of %d tokens per node is supported", MAX_NUM_TOKENS), false);
 
         try
         {
@@ -596,7 +637,7 @@
         catch (NumberFormatException e)
         {
             throw new ConfigurationException("key_cache_size_in_mb option was set incorrectly to '"
-                    + conf.key_cache_size_in_mb + "', supported values are <integer> >= 0.");
+                    + conf.key_cache_size_in_mb + "', supported values are <integer> >= 0.", false);
         }
 
         try
@@ -612,7 +653,7 @@
         catch (NumberFormatException e)
         {
             throw new ConfigurationException("counter_cache_size_in_mb option was set incorrectly to '"
-                    + conf.counter_cache_size_in_mb + "', supported values are <integer> >= 0.");
+                    + conf.counter_cache_size_in_mb + "', supported values are <integer> >= 0.", false);
         }
 
         // if set to empty/"auto" then use 5% of Heap size
@@ -622,9 +663,7 @@
 
         if (indexSummaryCapacityInMB < 0)
             throw new ConfigurationException("index_summary_capacity_in_mb option was set incorrectly to '"
-                    + conf.index_summary_capacity_in_mb + "', it should be a non-negative integer.");
-
-        memoryAllocator = FBUtilities.newOffHeapAllocator(conf.memory_allocator);
+                    + conf.index_summary_capacity_in_mb + "', it should be a non-negative integer.", false);
 
         if(conf.encryption_options != null)
         {
@@ -633,16 +672,10 @@
             conf.server_encryption_options = conf.encryption_options;
         }
 
-        // Hardcoded system keyspaces
-        List<KSMetaData> systemKeyspaces = Arrays.asList(KSMetaData.systemKeyspace());
-        assert systemKeyspaces.size() == Schema.systemKeyspaceNames.size();
-        for (KSMetaData ksmd : systemKeyspaces)
-            Schema.instance.load(ksmd);
-
-        /* Load the seeds for node contact points */
+        // load the seeds for node contact points
         if (conf.seed_provider == null)
         {
-            throw new ConfigurationException("seeds configuration is missing; a minimum of one seed is required.");
+            throw new ConfigurationException("seeds configuration is missing; a minimum of one seed is required.", false);
         }
         try
         {
@@ -652,12 +685,35 @@
         // there are about 5 checked exceptions that could be thrown here.
         catch (Exception e)
         {
-            logger.error("Fatal configuration error", e);
-            System.err.println(e.getMessage() + "\nFatal configuration error; unable to start server.  See log for stacktrace.");
-            System.exit(1);
+            throw new ConfigurationException(e.getMessage() + "\nFatal configuration error; unable to start server.  See log for stacktrace.", true);
         }
         if (seedProvider.getSeeds().size() == 0)
-            throw new ConfigurationException("The seed provider lists no seeds.");
+            throw new ConfigurationException("The seed provider lists no seeds.", false);
+
+        if (conf.otc_coalescing_enough_coalesced_messages > 128)
+            throw new ConfigurationException("otc_coalescing_enough_coalesced_messages must be smaller than 128", false);
+
+        if (conf.otc_coalescing_enough_coalesced_messages <= 0)
+            throw new ConfigurationException("otc_coalescing_enough_coalesced_messages must be positive", false);
+    }
+
+    private static FileStore guessFileStore(String dir) throws IOException
+    {
+        Path path = Paths.get(dir);
+        while (true)
+        {
+            try
+            {
+                return Files.getFileStore(path);
+            }
+            catch (IOException e)
+            {
+                if (e instanceof NoSuchFileException)
+                    path = path.getParent();
+                else
+                    throw e;
+            }
+        }
     }
 
     private static IEndpointSnitch createEndpointSnitch(String snitchClassName) throws ConfigurationException
@@ -668,67 +724,6 @@
         return conf.dynamic_snitch ? new DynamicEndpointSnitch(snitch) : snitch;
     }
 
-    /**
-     * load keyspace (keyspace) definitions, but do not initialize the keyspace instances.
-     * Schema version may be updated as the result.
-     */
-    public static void loadSchemas()
-    {
-        loadSchemas(true);
-    }
-
-    /**
-     * Load schema definitions.
-     *
-     * @param updateVersion true if schema version needs to be updated
-     */
-    public static void loadSchemas(boolean updateVersion)
-    {
-        ColumnFamilyStore schemaCFS = SystemKeyspace.schemaCFS(SystemKeyspace.SCHEMA_KEYSPACES_CF);
-
-        // if keyspace with definitions is empty try loading the old way
-        if (schemaCFS.estimateKeys() == 0)
-        {
-            logger.info("Couldn't detect any schema definitions in local storage.");
-            // peek around the data directories to see if anything is there.
-            if (hasExistingNoSystemTables())
-                logger.info("Found keyspace data in data directories. Consider using cqlsh to define your schema.");
-            else
-                logger.info("To create keyspaces and column families, see 'help create' in cqlsh.");
-        }
-        else
-        {
-            Schema.instance.load(DefsTables.loadFromKeyspace());
-        }
-
-        if (updateVersion)
-            Schema.instance.updateVersion();
-    }
-
-    private static boolean hasExistingNoSystemTables()
-    {
-        for (String dataDir : getAllDataFileLocations())
-        {
-            File dataPath = new File(dataDir);
-            if (dataPath.exists() && dataPath.isDirectory())
-            {
-                // see if there are other directories present.
-                int dirCount = dataPath.listFiles(new FileFilter()
-                {
-                    public boolean accept(File pathname)
-                    {
-                        return (pathname.isDirectory() && !Schema.systemKeyspaceNames.contains(pathname.getName()));
-                    }
-                }).length;
-
-                if (dirCount > 0)
-                    return true;
-            }
-        }
-
-        return false;
-    }
-
     public static IAuthenticator getAuthenticator()
     {
         return authenticator;
@@ -739,6 +734,11 @@
         return authorizer;
     }
 
+    public static IRoleManager getRoleManager()
+    {
+        return roleManager;
+    }
+
     public static int getPermissionsValidity()
     {
         return conf.permissions_validity_in_ms;
@@ -761,6 +761,33 @@
              : conf.permissions_update_interval_in_ms;
     }
 
+    public static int getRolesValidity()
+    {
+        return conf.roles_validity_in_ms;
+    }
+
+    public static void setRolesValidity(int validity)
+    {
+        conf.roles_validity_in_ms = validity;
+    }
+
+    public static int getRolesCacheMaxEntries()
+    {
+        return conf.roles_cache_max_entries;
+    }
+
+    public static int getRolesUpdateInterval()
+    {
+        return conf.roles_update_interval_in_ms == -1
+             ? conf.roles_validity_in_ms
+             : conf.roles_update_interval_in_ms;
+    }
+
+    public static void setRolesUpdateInterval(int interval)
+    {
+        conf.roles_update_interval_in_ms = interval;
+    }
+
     public static void setPermissionsUpdateInterval(int updateInterval)
     {
         conf.permissions_update_interval_in_ms = updateInterval;
@@ -779,7 +806,7 @@
         try
         {
             if (conf.data_file_directories.length == 0)
-                throw new ConfigurationException("At least one DataFileDirectory must be specified");
+                throw new ConfigurationException("At least one DataFileDirectory must be specified", false);
 
             for (String dataFileDirectory : conf.data_file_directories)
             {
@@ -787,26 +814,22 @@
             }
 
             if (conf.commitlog_directory == null)
-                throw new ConfigurationException("commitlog_directory must be specified");
+                throw new ConfigurationException("commitlog_directory must be specified", false);
 
             FileUtils.createDirectory(conf.commitlog_directory);
 
             if (conf.saved_caches_directory == null)
-                throw new ConfigurationException("saved_caches_directory must be specified");
+                throw new ConfigurationException("saved_caches_directory must be specified", false);
 
             FileUtils.createDirectory(conf.saved_caches_directory);
         }
         catch (ConfigurationException e)
         {
-            logger.error("Fatal error: {}", e.getMessage());
-            System.err.println("Bad configuration; unable to start server");
-            System.exit(1);
+            throw new IllegalArgumentException("Bad configuration; unable to start server: "+e.getMessage());
         }
         catch (FSWriteError e)
         {
-            logger.error("Fatal error: {}", e.getMessage());
-            System.err.println(e.getCause().getMessage() + "; unable to start server");
-            System.exit(1);
+            throw new IllegalStateException(e.getCause().getMessage() + "; unable to start server");
         }
     }
 
@@ -860,11 +883,31 @@
         return conf.batch_size_warn_threshold_in_kb * 1024;
     }
 
+    public static long getBatchSizeFailThreshold()
+    {
+        return conf.batch_size_fail_threshold_in_kb * 1024L;
+    }
+
+    public static int getBatchSizeFailThresholdInKB()
+    {
+        return conf.batch_size_fail_threshold_in_kb;
+    }
+
     public static int getUnloggedBatchAcrossPartitionsWarnThreshold()
     {
         return conf.unlogged_batch_across_partitions_warn_threshold;
     }
 
+    public static void setBatchSizeWarnThresholdInKB(int threshold)
+    {
+        conf.batch_size_warn_threshold_in_kb = threshold;
+    }
+
+    public static void setBatchSizeFailThresholdInKB(int threshold)
+    {
+        conf.batch_size_fail_threshold_in_kb = threshold;
+    }
+
     public static Collection<String> getInitialTokens()
     {
         return tokensFromString(System.getProperty("cassandra.initial_token", conf.initial_token));
@@ -896,7 +939,7 @@
         }
         catch (UnknownHostException e)
         {
-            return null;
+            throw new RuntimeException("Replacement host name could not be resolved or scope_id was specified for a global IPv6 address", e);
         }
     }
 
@@ -931,11 +974,6 @@
         return conf.cluster_name;
     }
 
-    public static int getMaxStreamingRetries()
-    {
-        return conf.max_streaming_retries;
-    }
-
     public static int getStoragePort()
     {
         return Integer.parseInt(System.getProperty("cassandra.storage_port", conf.storage_port.toString()));
@@ -1150,6 +1188,21 @@
         return conf.commitlog_directory;
     }
 
+    public static ParameterizedClass getCommitLogCompression()
+    {
+        return conf.commitlog_compression;
+    }
+
+    public static void setCommitLogCompression(ParameterizedClass compressor)
+    {
+        conf.commitlog_compression = compressor;
+    }
+
+    public static int getCommitLogMaxCompressionBuffersInPool()
+    {
+        return conf.commitlog_max_compression_buffers_in_pool;
+    }
+
     public static int getTombstoneWarnThreshold()
     {
         return conf.tombstone_warn_threshold;
@@ -1170,11 +1223,6 @@
         conf.tombstone_failure_threshold = threshold;
     }
 
-    public static boolean getCommitLogSegmentRecyclingEnabled()
-    {
-        return conf.commitlog_segment_recycling;
-    }
-
     /**
      * size of commitlog segments to allocate
      */
@@ -1183,6 +1231,11 @@
         return conf.commitlog_segment_size_in_mb * 1024 * 1024;
     }
 
+    public static void setCommitLogSegmentSize(int sizeMegabytes)
+    {
+        conf.commitlog_segment_size_in_mb = sizeMegabytes;
+    }
+
     public static String getSavedCachesLocation()
     {
         return conf.saved_caches_directory;
@@ -1203,6 +1256,11 @@
         return broadcastAddress;
     }
 
+    public static Boolean shouldListenOnBroadcastAddress()
+    {
+        return conf.listen_on_broadcast_address;
+    }
+
     public static IInternodeAuthenticator getInternodeAuthenticator()
     {
         return internodeAuthenticator;
@@ -1228,6 +1286,9 @@
         broadcastRpcAddress = broadcastRPCAddr;
     }
 
+    /**
+     * May be null, please use {@link FBUtilities#getBroadcastRpcAddress()} instead.
+     */
     public static InetAddress getBroadcastRpcAddress()
     {
         return broadcastRpcAddress;
@@ -1317,14 +1378,19 @@
         return conf.commitlog_sync_batch_window_in_ms;
     }
 
+    public static void setCommitLogSyncBatchWindow(double windowMillis)
+    {
+        conf.commitlog_sync_batch_window_in_ms = windowMillis;
+    }
+
     public static int getCommitLogSyncPeriod()
     {
         return conf.commitlog_sync_period_in_ms;
     }
 
-    public static int getCommitLogPeriodicQueueSize()
+    public static void setCommitLogSyncPeriod(int periodMillis)
     {
-        return conf.commitlog_periodic_queue_size;
+        conf.commitlog_sync_period_in_ms = periodMillis;
     }
 
     public static Config.CommitLogSync getCommitLogSync()
@@ -1332,16 +1398,35 @@
         return conf.commitlog_sync;
     }
 
+    public static void setCommitLogSync(CommitLogSync sync)
+    {
+        conf.commitlog_sync = sync;
+    }
+
     public static Config.DiskAccessMode getDiskAccessMode()
     {
         return conf.disk_access_mode;
     }
 
+    // Do not use outside unit tests.
+    @VisibleForTesting
+    public static void setDiskAccessMode(Config.DiskAccessMode mode)
+    {
+        conf.disk_access_mode = mode;
+    }
+
     public static Config.DiskAccessMode getIndexAccessMode()
     {
         return indexAccessMode;
     }
 
+    // Do not use outside unit tests.
+    @VisibleForTesting
+    public static void setIndexAccessMode(Config.DiskAccessMode mode)
+    {
+        indexAccessMode = mode;
+    }
+
     public static void setDiskFailurePolicy(Config.DiskFailurePolicy policy)
     {
         conf.disk_failure_policy = policy;
@@ -1436,16 +1521,10 @@
         return conf.max_hint_window_in_ms;
     }
 
-    @Deprecated
-    public static Integer getIndexInterval()
-    {
-        return conf.index_interval;
-    }
-
-    public static File getSerializedCachePath(CacheService.CacheType cacheType, String version)
+    public static File getSerializedCachePath(CacheService.CacheType cacheType, String version, String extension)
     {
         String name = cacheType.toString()
-                + (version == null ? "" : "-" + version + ".db");
+                + (version == null ? "" : "-" + version + "." + extension);
         return new File(conf.saved_caches_directory, name);
     }
 
@@ -1529,7 +1608,7 @@
 
     public static int getSSTablePreempiveOpenIntervalInMB()
     {
-        return conf.sstable_preemptive_open_interval_in_mb;
+        return FBUtilities.isWindows() ? -1 : conf.sstable_preemptive_open_interval_in_mb;
     }
     public static void setSSTablePreempiveOpenIntervalInMB(int mb)
     {
@@ -1576,6 +1655,11 @@
         conf.key_cache_keys_to_save = keyCacheKeysToSave;
     }
 
+    public static String getRowCacheClassName()
+    {
+        return conf.row_cache_class_name;
+    }
+
     public static long getRowCacheSizeInMB()
     {
         return conf.row_cache_size_in_mb;
@@ -1607,6 +1691,11 @@
         return counterCacheSizeInMB;
     }
 
+    public static void setRowCacheKeysToSave(int rowCacheKeysToSave)
+    {
+        conf.row_cache_keys_to_save = rowCacheKeysToSave;
+    }
+
     public static int getCounterCacheSavePeriod()
     {
         return conf.counter_cache_save_period;
@@ -1627,16 +1716,6 @@
         conf.counter_cache_keys_to_save = counterCacheKeysToSave;
     }
 
-    public static IAllocator getoffHeapMemoryAllocator()
-    {
-        return memoryAllocator;
-    }
-
-    public static void setRowCacheKeysToSave(int rowCacheKeysToSave)
-    {
-        conf.row_cache_keys_to_save = rowCacheKeysToSave;
-    }
-
     public static int getStreamingSocketTimeout()
     {
         return conf.streaming_socket_timeout_in_ms;
@@ -1662,6 +1741,12 @@
         return conf.inter_dc_tcp_nodelay;
     }
 
+
+    public static SSTableFormat.Type getSSTableFormat()
+    {
+        return sstable_format;
+    }
+
     public static MemtablePool getMemtableAllocatorPool()
     {
         long heapLimit = ((long) conf.memtable_heap_space_in_mb) << 20;
@@ -1675,8 +1760,7 @@
             case offheap_buffers:
                 if (!FileUtils.isCleanerAvailable())
                 {
-                    logger.error("Could not free direct byte buffer: offheap_buffers is not a safe memtable_allocation_type without this ability, please adjust your config. This feature is only guaranteed to work on an Oracle JVM. Refusing to start.");
-                    System.exit(-1);
+                    throw new IllegalStateException("Could not free direct byte buffer: offheap_buffers is not a safe memtable_allocation_type without this ability, please adjust your config. This feature is only guaranteed to work on an Oracle JVM. Refusing to start.");
                 }
                 return new SlabPool(heapLimit, offHeapLimit, conf.memtable_cleanup_threshold, new ColumnFamilyStore.FlushLargestColumnFamily());
             case offheap_objects:
@@ -1686,6 +1770,11 @@
         }
     }
 
+    public static boolean getOutboundBindAny()
+    {
+        return Config.outboundBindAny || conf.listen_on_broadcast_address;
+    }
+
     public static int getIndexSummaryResizeIntervalInMinutes()
     {
         return conf.index_summary_resize_interval_in_minutes;
@@ -1707,6 +1796,16 @@
         return arch.contains("64") || arch.contains("sparcv9");
     }
 
+    public static int getTracetypeRepairTTL()
+    {
+        return conf.tracetype_repair_ttl;
+    }
+
+    public static int getTracetypeQueryTTL()
+    {
+        return conf.tracetype_query_ttl;
+    }
+
     public static String getOtcCoalescingStrategy()
     {
         return conf.otc_coalescing_strategy;
@@ -1717,6 +1816,31 @@
         return conf.otc_coalescing_window_us;
     }
 
+    public static int getOtcCoalescingEnoughCoalescedMessages()
+    {
+        return conf.otc_coalescing_enough_coalesced_messages;
+    }
+
+    public static void setOtcCoalescingEnoughCoalescedMessages(int otc_coalescing_enough_coalesced_messages)
+    {
+        conf.otc_coalescing_enough_coalesced_messages = otc_coalescing_enough_coalesced_messages;
+    }
+
+    public static boolean enableUserDefinedFunctions()
+    {
+        return conf.enable_user_defined_functions;
+    }
+
+    public static int getWindowsTimerInterval()
+    {
+        return conf.windows_timer_interval;
+    }
+
+    public static long getGCLogThreshold()
+    {
+        return conf.gc_log_threshold_in_ms;
+    }
+
     public static long getGCWarnThreshold()
     {
         return conf.gc_warn_threshold_in_ms;
diff --git a/src/java/org/apache/cassandra/config/KSMetaData.java b/src/java/org/apache/cassandra/config/KSMetaData.java
index 22c59ca..1537aae 100644
--- a/src/java/org/apache/cassandra/config/KSMetaData.java
+++ b/src/java/org/apache/cassandra/config/KSMetaData.java
@@ -20,19 +20,10 @@
 import java.util.*;
 
 import com.google.common.base.Objects;
-import com.google.common.collect.ImmutableMap;
 
-import org.apache.cassandra.cql3.QueryProcessor;
-import org.apache.cassandra.cql3.UntypedResultSet;
-import org.apache.cassandra.db.*;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.locator.*;
 import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.thrift.CfDef;
-import org.apache.cassandra.thrift.KsDef;
-import org.apache.cassandra.tracing.Tracing;
-
-import static org.apache.cassandra.utils.FBUtilities.*;
 
 public final class KSMetaData
 {
@@ -44,17 +35,29 @@
 
     public final UTMetaData userTypes;
 
-    KSMetaData(String name, Class<? extends AbstractReplicationStrategy> strategyClass, Map<String, String> strategyOptions, boolean durableWrites, Iterable<CFMetaData> cfDefs)
+    public KSMetaData(String name,
+                      Class<? extends AbstractReplicationStrategy> strategyClass,
+                      Map<String, String> strategyOptions,
+                      boolean durableWrites)
+    {
+        this(name, strategyClass, strategyOptions, durableWrites, Collections.<CFMetaData>emptyList(), new UTMetaData());
+    }
+
+    public KSMetaData(String name,
+                      Class<? extends AbstractReplicationStrategy> strategyClass,
+                      Map<String, String> strategyOptions,
+                      boolean durableWrites,
+                      Iterable<CFMetaData> cfDefs)
     {
         this(name, strategyClass, strategyOptions, durableWrites, cfDefs, new UTMetaData());
     }
 
-    KSMetaData(String name,
-               Class<? extends AbstractReplicationStrategy> strategyClass,
-               Map<String, String> strategyOptions,
-               boolean durableWrites,
-               Iterable<CFMetaData> cfDefs,
-               UTMetaData userTypes)
+    private KSMetaData(String name,
+                       Class<? extends AbstractReplicationStrategy> strategyClass,
+                       Map<String, String> strategyOptions,
+                       boolean durableWrites,
+                       Iterable<CFMetaData> cfDefs,
+                       UTMetaData userTypes)
     {
         this.name = name;
         this.strategyClass = strategyClass == null ? NetworkTopologyStrategy.class : strategyClass;
@@ -82,37 +85,27 @@
         return new KSMetaData(name, strategyClass, options, durablesWrites, cfDefs, new UTMetaData());
     }
 
-    public static KSMetaData cloneWith(KSMetaData ksm, Iterable<CFMetaData> cfDefs)
+    public KSMetaData cloneWithTableRemoved(CFMetaData table)
     {
-        return new KSMetaData(ksm.name, ksm.strategyClass, ksm.strategyOptions, ksm.durableWrites, cfDefs, ksm.userTypes);
+        // clone ksm but do not include the new table
+        List<CFMetaData> newTables = new ArrayList<>(cfMetaData().values());
+        newTables.remove(table);
+        assert newTables.size() == cfMetaData().size() - 1;
+        return cloneWith(newTables, userTypes);
     }
 
-    public static KSMetaData systemKeyspace()
+    public KSMetaData cloneWithTableAdded(CFMetaData table)
     {
-        List<CFMetaData> cfDefs = Arrays.asList(CFMetaData.BatchlogCf,
-                                                CFMetaData.RangeXfersCf,
-                                                CFMetaData.LocalCf,
-                                                CFMetaData.PeersCf,
-                                                CFMetaData.PeerEventsCf,
-                                                CFMetaData.HintsCf,
-                                                CFMetaData.IndexCf,
-                                                CFMetaData.SchemaKeyspacesCf,
-                                                CFMetaData.SchemaColumnFamiliesCf,
-                                                CFMetaData.SchemaColumnsCf,
-                                                CFMetaData.SchemaTriggersCf,
-                                                CFMetaData.SchemaUserTypesCf,
-                                                CFMetaData.CompactionLogCf,
-                                                CFMetaData.CompactionHistoryCf,
-                                                CFMetaData.PaxosCf,
-                                                CFMetaData.SSTableActivityCF,
-                                                CFMetaData.SizeEstimatesCf);
-        return new KSMetaData(Keyspace.SYSTEM_KS, LocalStrategy.class, Collections.<String, String>emptyMap(), true, cfDefs);
+        // clone ksm but include the new table
+        List<CFMetaData> newTables = new ArrayList<>(cfMetaData().values());
+        newTables.add(table);
+        assert newTables.size() == cfMetaData().size() + 1;
+        return cloneWith(newTables, userTypes);
     }
 
-    public static KSMetaData traceKeyspace()
+    public KSMetaData cloneWith(Iterable<CFMetaData> tables, UTMetaData types)
     {
-        List<CFMetaData> cfDefs = Arrays.asList(CFMetaData.TraceSessionsCf, CFMetaData.TraceEventsCf);
-        return new KSMetaData(Tracing.TRACE_KS, SimpleStrategy.class, ImmutableMap.of("replication_factor", "2"), true, cfDefs);
+        return new KSMetaData(name, strategyClass, strategyOptions, durableWrites, tables, types);
     }
 
     public static KSMetaData testMetadata(String name, Class<? extends AbstractReplicationStrategy> strategyClass, Map<String, String> strategyOptions, CFMetaData... cfDefs)
@@ -173,40 +166,6 @@
         return Collections.singletonMap("replication_factor", rf.toString());
     }
 
-    public static KSMetaData fromThrift(KsDef ksd, CFMetaData... cfDefs) throws ConfigurationException
-    {
-        Class<? extends AbstractReplicationStrategy> cls = AbstractReplicationStrategy.getClass(ksd.strategy_class);
-        if (cls.equals(LocalStrategy.class))
-            throw new ConfigurationException("Unable to use given strategy class: LocalStrategy is reserved for internal use.");
-
-        return new KSMetaData(ksd.name,
-                              cls,
-                              ksd.strategy_options == null ? Collections.<String, String>emptyMap() : ksd.strategy_options,
-                              ksd.durable_writes,
-                              Arrays.asList(cfDefs));
-    }
-
-    public KsDef toThrift()
-    {
-        List<CfDef> cfDefs = new ArrayList<>(cfMetaData.size());
-        for (CFMetaData cfm : cfMetaData().values())
-        {
-            // Don't expose CF that cannot be correctly handle by thrift; see CASSANDRA-4377 for further details
-            if (cfm.isThriftCompatible())
-                cfDefs.add(cfm.toThrift());
-        }
-        KsDef ksdef = new KsDef(name, strategyClass.getName(), cfDefs);
-        ksdef.setStrategy_options(strategyOptions);
-        ksdef.setDurable_writes(durableWrites);
-
-        return ksdef;
-    }
-
-    public Mutation toSchemaUpdate(KSMetaData newState, long modificationTimestamp)
-    {
-        return newState.toSchema(modificationTimestamp);
-    }
-
     public KSMetaData validate() throws ConfigurationException
     {
         if (!CFMetaData.isNameValid(name))
@@ -222,105 +181,4 @@
 
         return this;
     }
-
-    public KSMetaData reloadAttributes()
-    {
-        Row ksDefRow = SystemKeyspace.readSchemaRow(SystemKeyspace.SCHEMA_KEYSPACES_CF, name);
-
-        if (ksDefRow.cf == null)
-            throw new RuntimeException(String.format("%s not found in the schema definitions keyspaceName (%s).", name, SystemKeyspace.SCHEMA_KEYSPACES_CF));
-
-        return fromSchema(ksDefRow, Collections.<CFMetaData>emptyList(), userTypes);
-    }
-
-    public Mutation dropFromSchema(long timestamp)
-    {
-        Mutation mutation = new Mutation(Keyspace.SYSTEM_KS, SystemKeyspace.getSchemaKSKey(name));
-
-        mutation.delete(SystemKeyspace.SCHEMA_KEYSPACES_CF, timestamp);
-        mutation.delete(SystemKeyspace.SCHEMA_COLUMNFAMILIES_CF, timestamp);
-        mutation.delete(SystemKeyspace.SCHEMA_COLUMNS_CF, timestamp);
-        mutation.delete(SystemKeyspace.SCHEMA_TRIGGERS_CF, timestamp);
-        mutation.delete(SystemKeyspace.SCHEMA_USER_TYPES_CF, timestamp);
-        mutation.delete(SystemKeyspace.INDEX_CF, timestamp);
-
-        return mutation;
-    }
-
-    public Mutation toSchema(long timestamp)
-    {
-        Mutation mutation = new Mutation(Keyspace.SYSTEM_KS, SystemKeyspace.getSchemaKSKey(name));
-        ColumnFamily cf = mutation.addOrGet(CFMetaData.SchemaKeyspacesCf);
-        CFRowAdder adder = new CFRowAdder(cf, CFMetaData.SchemaKeyspacesCf.comparator.builder().build(), timestamp);
-
-        adder.add("durable_writes", durableWrites);
-        adder.add("strategy_class", strategyClass.getName());
-        adder.add("strategy_options", json(strategyOptions));
-
-        for (CFMetaData cfm : cfMetaData.values())
-            cfm.toSchema(mutation, timestamp);
-
-        userTypes.toSchema(mutation, timestamp);
-        return mutation;
-    }
-
-    /**
-     * Deserialize only Keyspace attributes without nested ColumnFamilies
-     *
-     * @param row Keyspace attributes in serialized form
-     *
-     * @return deserialized keyspace without cf_defs
-     */
-    public static KSMetaData fromSchema(Row row, Iterable<CFMetaData> cfms, UTMetaData userTypes)
-    {
-        UntypedResultSet.Row result = QueryProcessor.resultify("SELECT * FROM system.schema_keyspaces", row).one();
-        try
-        {
-            return new KSMetaData(result.getString("keyspace_name"),
-                                  AbstractReplicationStrategy.getClass(result.getString("strategy_class")),
-                                  fromJsonMap(result.getString("strategy_options")),
-                                  result.getBoolean("durable_writes"),
-                                  cfms,
-                                  userTypes);
-        }
-        catch (ConfigurationException e)
-        {
-            throw new RuntimeException(e);
-        }
-    }
-
-    /**
-     * Deserialize Keyspace with nested ColumnFamilies
-     *
-     * @param serializedKs Keyspace in serialized form
-     * @param serializedCFs Collection of the serialized ColumnFamilies
-     *
-     * @return deserialized keyspace with cf_defs
-     */
-    public static KSMetaData fromSchema(Row serializedKs, Row serializedCFs, Row serializedUserTypes)
-    {
-        Map<String, CFMetaData> cfs = deserializeColumnFamilies(serializedCFs);
-        UTMetaData userTypes = new UTMetaData(UTMetaData.fromSchema(serializedUserTypes));
-        return fromSchema(serializedKs, cfs.values(), userTypes);
-    }
-
-    /**
-     * Deserialize ColumnFamilies from low-level schema representation, all of them belong to the same keyspace
-     *
-     * @return map containing name of the ColumnFamily and it's metadata for faster lookup
-     */
-    public static Map<String, CFMetaData> deserializeColumnFamilies(Row row)
-    {
-        if (row.cf == null)
-            return Collections.emptyMap();
-
-        Map<String, CFMetaData> cfms = new HashMap<>();
-        UntypedResultSet results = QueryProcessor.resultify("SELECT * FROM system.schema_columnfamilies", row);
-        for (UntypedResultSet.Row result : results)
-        {
-            CFMetaData cfm = CFMetaData.fromSchema(result);
-            cfms.put(cfm.cfName, cfm);
-        }
-        return cfms;
-    }
 }
diff --git a/src/java/org/apache/cassandra/config/ParameterizedClass.java b/src/java/org/apache/cassandra/config/ParameterizedClass.java
new file mode 100644
index 0000000..6b7af63
--- /dev/null
+++ b/src/java/org/apache/cassandra/config/ParameterizedClass.java
@@ -0,0 +1,60 @@
+/*
+ * 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.config;
+
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+import com.google.common.base.Objects;
+
+public class ParameterizedClass
+{
+    public String class_name;
+    public Map<String, String> parameters;
+
+    public ParameterizedClass(String class_name, Map<String, String> parameters)
+    {
+        this.class_name = class_name;
+        this.parameters = parameters;
+    }
+
+    @SuppressWarnings("unchecked")
+    public ParameterizedClass(LinkedHashMap<String, ?> p)
+    {
+        this((String)p.get("class_name"),
+                p.containsKey("parameters") ? (Map<String, String>)((List<?>)p.get("parameters")).get(0) : null);
+    }
+
+    @Override
+    public boolean equals(Object that)
+    {
+        return that instanceof ParameterizedClass && equals((ParameterizedClass) that);
+    }
+
+    public boolean equals(ParameterizedClass that)
+    {
+        return Objects.equal(class_name, that.class_name) && Objects.equal(parameters, that.parameters);
+    }
+
+    @Override
+    public String toString()
+    {
+        return class_name + (parameters == null ? "" : parameters.toString());
+    }
+}
diff --git a/src/java/org/apache/cassandra/config/Schema.java b/src/java/org/apache/cassandra/config/Schema.java
index 739c8ca..2cd7611 100644
--- a/src/java/org/apache/cassandra/config/Schema.java
+++ b/src/java/org/apache/cassandra/config/Schema.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.config;
 
-import java.nio.charset.CharacterCodingException;
 import java.security.MessageDigest;
 import java.security.NoSuchAlgorithmException;
 import java.util.*;
@@ -28,16 +27,22 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.auth.Auth;
+import org.apache.cassandra.auth.AuthKeyspace;
+import org.apache.cassandra.cql3.functions.Functions;
+import org.apache.cassandra.cql3.functions.UDAggregate;
+import org.apache.cassandra.cql3.functions.UDFunction;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.commitlog.CommitLog;
+import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.db.marshal.UserType;
 import org.apache.cassandra.db.index.SecondaryIndex;
-import org.apache.cassandra.db.index.SecondaryIndexManager;
 import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.schema.LegacySchemaTables;
+import org.apache.cassandra.repair.SystemDistributedKeyspace;
 import org.apache.cassandra.service.MigrationManager;
-import org.apache.cassandra.tracing.Tracing;
+import org.apache.cassandra.tracing.TraceKeyspace;
 import org.apache.cassandra.utils.ConcurrentBiMap;
-import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.Pair;
 import org.cliffc.high_scale_lib.NonBlockingHashMap;
 
@@ -56,10 +61,10 @@
     public static final int NAME_LENGTH = 48;
 
     /* metadata map for faster keyspace lookup */
-    private final Map<String, KSMetaData> keyspaces = new NonBlockingHashMap<String, KSMetaData>();
+    private final Map<String, KSMetaData> keyspaces = new NonBlockingHashMap<>();
 
     /* Keyspace objects, one per keyspace. Only one instance should ever exist for any given keyspace. */
-    private final Map<String, Keyspace> keyspaceInstances = new NonBlockingHashMap<String, Keyspace>();
+    private final Map<String, Keyspace> keyspaceInstances = new NonBlockingHashMap<>();
 
     /* metadata map for faster ColumnFamily lookup */
     private final ConcurrentBiMap<Pair<String, String>, UUID> cfIdMap = new ConcurrentBiMap<>();
@@ -68,8 +73,10 @@
 
     // 59adb24e-f3cd-3e02-97f0-5b395827453f
     public static final UUID emptyVersion;
-    public static final ImmutableSet<String> systemKeyspaceNames = ImmutableSet.of(Keyspace.SYSTEM_KS);
-    public static final ImmutableSet<String> replicatedSystemKeyspaceNames = ImmutableSet.of(Tracing.TRACE_KS, Auth.AUTH_KS);
+
+    private static final ImmutableSet<String> replicatedSystemKeyspaceNames = ImmutableSet.of(TraceKeyspace.NAME,
+                                                                                              AuthKeyspace.NAME,
+                                                                                              SystemDistributedKeyspace.NAME);
 
     static
     {
@@ -84,10 +91,34 @@
     }
 
     /**
-     * Initialize empty schema object
+     * Initialize empty schema object and load the hardcoded system tables
      */
     public Schema()
-    {}
+    {
+        load(SystemKeyspace.definition());
+    }
+
+    /**
+     * load keyspace (keyspace) definitions, but do not initialize the keyspace instances.
+     * Schema version may be updated as the result.
+     */
+    public Schema loadFromDisk()
+    {
+        return loadFromDisk(true);
+    }
+
+    /**
+     * Load schema definitions from disk.
+     *
+     * @param updateVersion true if schema version needs to be updated
+     */
+    public Schema loadFromDisk(boolean updateVersion)
+    {
+        load(LegacySchemaTables.readSchemaFromSystemTables());
+        if (updateVersion)
+            updateVersion();
+        return this;
+    }
 
     /**
      * Load up non-system keyspaces
@@ -229,7 +260,7 @@
     }
 
     /**
-     * Given a keyspace name & column family name, get the column family
+     * Given a keyspace name and column family name, get the column family
      * meta data. If the keyspace name or column family name is not valid
      * this function returns null.
      *
@@ -264,21 +295,6 @@
     }
 
     /**
-     * Get type of the ColumnFamily but it's keyspace/name
-     *
-     * @param ksName The keyspace name
-     * @param cfName The ColumnFamily name
-     *
-     * @return The type of the ColumnFamily
-     */
-    public ColumnFamilyType getColumnFamilyType(String ksName, String cfName)
-    {
-        assert ksName != null && cfName != null;
-        CFMetaData cfMetaData = getCFMetaData(ksName, cfName);
-        return (cfMetaData == null) ? null : cfMetaData.cfType;
-    }
-
-    /**
      * Get metadata about keyspace by its name
      *
      * @param keyspaceName The name of the keyspace
@@ -293,7 +309,7 @@
 
     private Set<String> getNonSystemKeyspacesSet()
     {
-        return Sets.difference(keyspaces.keySet(), systemKeyspaceNames);
+        return Sets.difference(keyspaces.keySet(), Collections.singleton(SystemKeyspace.NAME));
     }
 
     /**
@@ -400,7 +416,7 @@
         Pair<String, String> key = Pair.create(cfm.ksName, cfm.cfName);
 
         if (cfIdMap.containsKey(key))
-            throw new RuntimeException(String.format("Attempting to load already loaded column family %s.%s", cfm.ksName, cfm.cfName));
+            throw new RuntimeException(String.format("Attempting to load already loaded table %s.%s", cfm.ksName, cfm.cfName));
 
         logger.debug("Adding {} to cfIdMap", cfm);
         cfIdMap.put(key, cfm.cfId);
@@ -433,28 +449,8 @@
      */
     public void updateVersion()
     {
-        try
-        {
-            MessageDigest versionDigest = MessageDigest.getInstance("MD5");
-
-            for (Row row : SystemKeyspace.serializedSchema())
-            {
-                if (invalidSchemaRow(row) || ignoredSchemaRow(row))
-                    continue;
-
-                // we want to digest only live columns
-                ColumnFamilyStore.removeDeletedColumnsOnly(row.cf, Integer.MAX_VALUE, SecondaryIndexManager.nullUpdater);
-                row.cf.purgeTombstones(Integer.MAX_VALUE);
-                row.cf.updateDigest(versionDigest);
-            }
-
-            version = UUID.nameUUIDFromBytes(versionDigest.digest());
-            SystemKeyspace.updateSchemaVersion(version);
-        }
-        catch (Exception e)
-        {
-            throw new RuntimeException(e);
-        }
+        version = LegacySchemaTables.calculateSchemaDigest();
+        SystemKeyspace.updateSchemaVersion(version);
     }
 
     /*
@@ -482,20 +478,204 @@
         updateVersionAndAnnounce();
     }
 
-    public static boolean invalidSchemaRow(Row row)
+    public void addKeyspace(KSMetaData ksm)
     {
-        return row.cf == null || (row.cf.isMarkedForDelete() && !row.cf.hasColumns());
+        assert getKSMetaData(ksm.name) == null;
+        load(ksm);
+
+        Keyspace.open(ksm.name);
+        MigrationManager.instance.notifyCreateKeyspace(ksm);
     }
 
-    public static boolean ignoredSchemaRow(Row row)
+    public void updateKeyspace(String ksName)
     {
-        try
+        KSMetaData oldKsm = getKSMetaData(ksName);
+        assert oldKsm != null;
+        KSMetaData newKsm = LegacySchemaTables.createKeyspaceFromName(ksName).cloneWith(oldKsm.cfMetaData().values(), oldKsm.userTypes);
+
+        setKeyspaceDefinition(newKsm);
+        Keyspace.open(ksName).setMetadata(newKsm);
+
+        MigrationManager.instance.notifyUpdateKeyspace(newKsm);
+    }
+
+    public void dropKeyspace(String ksName)
+    {
+        KSMetaData ksm = Schema.instance.getKSMetaData(ksName);
+        String snapshotName = Keyspace.getTimestampedSnapshotName(ksName);
+
+        CompactionManager.instance.interruptCompactionFor(ksm.cfMetaData().values(), true);
+
+        Keyspace keyspace = Keyspace.open(ksm.name);
+
+        // remove all cfs from the keyspace instance.
+        List<UUID> droppedCfs = new ArrayList<>();
+        for (CFMetaData cfm : ksm.cfMetaData().values())
         {
-            return systemKeyspaceNames.contains(ByteBufferUtil.string(row.key.getKey()));
+            ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfm.cfName);
+
+            purge(cfm);
+
+            if (DatabaseDescriptor.isAutoSnapshot())
+                cfs.snapshot(snapshotName);
+            Keyspace.open(ksm.name).dropCf(cfm.cfId);
+
+            droppedCfs.add(cfm.cfId);
         }
-        catch (CharacterCodingException e)
-        {
-            throw new RuntimeException(e);
-        }
+
+        // remove the keyspace from the static instances.
+        Keyspace.clear(ksm.name);
+        clearKeyspaceDefinition(ksm);
+
+        keyspace.writeOrder.awaitNewBarrier();
+
+        // force a new segment in the CL
+        CommitLog.instance.forceRecycleAllSegments(droppedCfs);
+
+        MigrationManager.instance.notifyDropKeyspace(ksm);
+    }
+
+    public void addTable(CFMetaData cfm)
+    {
+        assert getCFMetaData(cfm.ksName, cfm.cfName) == null;
+        KSMetaData ksm = getKSMetaData(cfm.ksName).cloneWithTableAdded(cfm);
+
+        logger.info("Loading {}", cfm);
+
+        load(cfm);
+
+        // make sure it's init-ed w/ the old definitions first,
+        // since we're going to call initCf on the new one manually
+        Keyspace.open(cfm.ksName);
+
+        // init the new CF before switching the KSM to the new one
+        // to avoid races as in CASSANDRA-10761
+        Keyspace.open(cfm.ksName).initCf(cfm, true);
+        setKeyspaceDefinition(ksm);
+        MigrationManager.instance.notifyCreateColumnFamily(cfm);
+    }
+
+    public void updateTable(String ksName, String tableName)
+    {
+        CFMetaData cfm = getCFMetaData(ksName, tableName);
+        assert cfm != null;
+        boolean columnsDidChange = cfm.reload();
+
+        Keyspace keyspace = Keyspace.open(cfm.ksName);
+        keyspace.getColumnFamilyStore(cfm.cfName).reload();
+        MigrationManager.instance.notifyUpdateColumnFamily(cfm, columnsDidChange);
+    }
+
+    public void dropTable(String ksName, String tableName)
+    {
+        KSMetaData ksm = getKSMetaData(ksName);
+        assert ksm != null;
+        ColumnFamilyStore cfs = Keyspace.open(ksName).getColumnFamilyStore(tableName);
+        assert cfs != null;
+
+        // reinitialize the keyspace.
+        CFMetaData cfm = ksm.cfMetaData().get(tableName);
+
+        purge(cfm);
+        setKeyspaceDefinition(ksm.cloneWithTableRemoved(cfm));
+
+        CompactionManager.instance.interruptCompactionFor(Arrays.asList(cfm), true);
+
+        if (DatabaseDescriptor.isAutoSnapshot())
+            cfs.snapshot(Keyspace.getTimestampedSnapshotName(cfs.name));
+        Keyspace.open(ksm.name).dropCf(cfm.cfId);
+        MigrationManager.instance.notifyDropColumnFamily(cfm);
+
+        CommitLog.instance.forceRecycleAllSegments(Collections.singleton(cfm.cfId));
+    }
+
+    public void addType(UserType ut)
+    {
+        KSMetaData ksm = getKSMetaData(ut.keyspace);
+        assert ksm != null;
+
+        logger.info("Loading {}", ut);
+
+        ksm.userTypes.addType(ut);
+
+        MigrationManager.instance.notifyCreateUserType(ut);
+    }
+
+    public void updateType(UserType ut)
+    {
+        KSMetaData ksm = getKSMetaData(ut.keyspace);
+        assert ksm != null;
+
+        logger.info("Updating {}", ut);
+
+        ksm.userTypes.addType(ut);
+
+        MigrationManager.instance.notifyUpdateUserType(ut);
+    }
+
+    public void dropType(UserType ut)
+    {
+        KSMetaData ksm = getKSMetaData(ut.keyspace);
+        assert ksm != null;
+
+        ksm.userTypes.removeType(ut);
+
+        MigrationManager.instance.notifyDropUserType(ut);
+    }
+
+    public void addFunction(UDFunction udf)
+    {
+        logger.info("Loading {}", udf);
+
+        Functions.addOrReplaceFunction(udf);
+
+        MigrationManager.instance.notifyCreateFunction(udf);
+    }
+
+    public void updateFunction(UDFunction udf)
+    {
+        logger.info("Updating {}", udf);
+
+        Functions.addOrReplaceFunction(udf);
+
+        MigrationManager.instance.notifyUpdateFunction(udf);
+    }
+
+    public void dropFunction(UDFunction udf)
+    {
+        logger.info("Drop {}", udf);
+
+        // TODO: this is kind of broken as this remove all overloads of the function name
+        Functions.removeFunction(udf.name(), udf.argTypes());
+
+        MigrationManager.instance.notifyDropFunction(udf);
+    }
+
+    public void addAggregate(UDAggregate udf)
+    {
+        logger.info("Loading {}", udf);
+
+        Functions.addOrReplaceFunction(udf);
+
+        MigrationManager.instance.notifyCreateAggregate(udf);
+    }
+
+    public void updateAggregate(UDAggregate udf)
+    {
+        logger.info("Updating {}", udf);
+
+        Functions.addOrReplaceFunction(udf);
+
+        MigrationManager.instance.notifyUpdateAggregate(udf);
+    }
+
+    public void dropAggregate(UDAggregate udf)
+    {
+        logger.info("Drop {}", udf);
+
+        // TODO: this is kind of broken as this remove all overloads of the function name
+        Functions.removeFunction(udf.name(), udf.argTypes());
+
+        MigrationManager.instance.notifyDropAggregate(udf);
     }
 }
diff --git a/src/java/org/apache/cassandra/config/SeedProviderDef.java b/src/java/org/apache/cassandra/config/SeedProviderDef.java
deleted file mode 100644
index cbe444a..0000000
--- a/src/java/org/apache/cassandra/config/SeedProviderDef.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * 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.config;
-
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-
-
-public class SeedProviderDef
-{
-    public String class_name;
-    public Map<String, String> parameters;
-
-    public SeedProviderDef(LinkedHashMap<String, ?> p)
-    {
-        class_name = (String)p.get("class_name");
-        parameters = (Map<String, String>)((List)p.get("parameters")).get(0);
-    }
-}
diff --git a/src/java/org/apache/cassandra/config/TriggerDefinition.java b/src/java/org/apache/cassandra/config/TriggerDefinition.java
index aaaf631..6a84379 100644
--- a/src/java/org/apache/cassandra/config/TriggerDefinition.java
+++ b/src/java/org/apache/cassandra/config/TriggerDefinition.java
@@ -18,22 +18,11 @@
  */
 package org.apache.cassandra.config;
 
-import java.util.*;
-
 import com.google.common.base.Objects;
 
-import org.apache.cassandra.cql3.QueryProcessor;
-import org.apache.cassandra.cql3.UntypedResultSet;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.composites.Composite;
-import org.apache.cassandra.db.marshal.UTF8Type;
-import org.apache.cassandra.thrift.TriggerDef;
-
 public class TriggerDefinition
 {
-    private static final String TRIGGER_NAME = "trigger_name";
-    private static final String TRIGGER_OPTIONS = "trigger_options";
-    private static final String CLASS = "class";
+    public static final String CLASS = "class";
 
     public final String name;
 
@@ -41,7 +30,7 @@
     // Proper trigger parametrization will be added later.
     public final String classOption;
 
-    TriggerDefinition(String name, String classOption)
+    public TriggerDefinition(String name, String classOption)
     {
         this.name = name;
         this.classOption = classOption;
@@ -52,88 +41,6 @@
         return new TriggerDefinition(name, classOption);
     }
 
-    /**
-     * Deserialize triggers from storage-level representation.
-     *
-     * @param serializedTriggers storage-level partition containing the trigger definitions
-     * @return the list of processed TriggerDefinitions
-     */
-    public static List<TriggerDefinition> fromSchema(Row serializedTriggers)
-    {
-        List<TriggerDefinition> triggers = new ArrayList<>();
-        String query = String.format("SELECT * FROM %s.%s", Keyspace.SYSTEM_KS, SystemKeyspace.SCHEMA_TRIGGERS_CF);
-        for (UntypedResultSet.Row row : QueryProcessor.resultify(query, serializedTriggers))
-        {
-            String name = row.getString(TRIGGER_NAME);
-            String classOption = row.getMap(TRIGGER_OPTIONS, UTF8Type.instance, UTF8Type.instance).get(CLASS);
-            triggers.add(new TriggerDefinition(name, classOption));
-        }
-        return triggers;
-    }
-
-    /**
-     * Add specified trigger to the schema using given mutation.
-     *
-     * @param mutation  The schema mutation
-     * @param cfName    The name of the parent ColumnFamily
-     * @param timestamp The timestamp to use for the columns
-     */
-    public void toSchema(Mutation mutation, String cfName, long timestamp)
-    {
-        ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SCHEMA_TRIGGERS_CF);
-
-        CFMetaData cfm = CFMetaData.SchemaTriggersCf;
-        Composite prefix = cfm.comparator.make(cfName, name);
-        CFRowAdder adder = new CFRowAdder(cf, prefix, timestamp);
-
-        adder.addMapEntry(TRIGGER_OPTIONS, CLASS, classOption);
-    }
-
-    /**
-     * Drop specified trigger from the schema using given mutation.
-     *
-     * @param mutation  The schema mutation
-     * @param cfName    The name of the parent ColumnFamily
-     * @param timestamp The timestamp to use for the tombstone
-     */
-    public void deleteFromSchema(Mutation mutation, String cfName, long timestamp)
-    {
-        ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SCHEMA_TRIGGERS_CF);
-        int ldt = (int) (System.currentTimeMillis() / 1000);
-
-        Composite prefix = CFMetaData.SchemaTriggersCf.comparator.make(cfName, name);
-        cf.addAtom(new RangeTombstone(prefix, prefix.end(), timestamp, ldt));
-    }
-
-    public static TriggerDefinition fromThrift(TriggerDef thriftDef)
-    {
-        return new TriggerDefinition(thriftDef.getName(), thriftDef.getOptions().get(CLASS));
-    }
-
-    public TriggerDef toThrift()
-    {
-        TriggerDef td = new TriggerDef();
-        td.setName(name);
-        td.setOptions(Collections.singletonMap(CLASS, classOption));
-        return td;
-    }
-
-    public static Map<String, TriggerDefinition> fromThrift(List<TriggerDef> thriftDefs)
-    {
-        Map<String, TriggerDefinition> triggerDefinitions = new HashMap<>();
-        for (TriggerDef thriftDef : thriftDefs)
-            triggerDefinitions.put(thriftDef.getName(), fromThrift(thriftDef));
-        return triggerDefinitions;
-    }
-
-    public static List<TriggerDef> toThrift(Map<String, TriggerDefinition> triggers)
-    {
-        List<TriggerDef> thriftDefs = new ArrayList<>(triggers.size());
-        for (TriggerDefinition def : triggers.values())
-            thriftDefs.add(def.toThrift());
-        return thriftDefs;
-    }
-
     @Override
     public boolean equals(Object o)
     {
diff --git a/src/java/org/apache/cassandra/config/UTMetaData.java b/src/java/org/apache/cassandra/config/UTMetaData.java
index ee653a8..08cedee 100644
--- a/src/java/org/apache/cassandra/config/UTMetaData.java
+++ b/src/java/org/apache/cassandra/config/UTMetaData.java
@@ -20,12 +20,7 @@
 import java.nio.ByteBuffer;
 import java.util.*;
 
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.composites.Composite;
 import org.apache.cassandra.db.marshal.*;
-import org.apache.cassandra.exceptions.RequestValidationException;
-import org.apache.cassandra.cql3.*;
-import org.apache.cassandra.utils.ByteBufferUtil;
 
 /**
  * Defined (and loaded) user types.
@@ -42,91 +37,11 @@
         this(new HashMap<ByteBuffer, UserType>());
     }
 
-    UTMetaData(Map<ByteBuffer, UserType> types)
+    public UTMetaData(Map<ByteBuffer, UserType> types)
     {
         this.userTypes = types;
     }
 
-    private static UserType fromSchema(UntypedResultSet.Row row)
-    {
-        try
-        {
-            String keyspace = row.getString("keyspace_name");
-            ByteBuffer name = ByteBufferUtil.bytes(row.getString("type_name"));
-            List<String> rawColumns = row.getList("field_names", UTF8Type.instance);
-            List<String> rawTypes = row.getList("field_types", UTF8Type.instance);
-
-            List<ByteBuffer> columns = new ArrayList<>(rawColumns.size());
-            for (String rawColumn : rawColumns)
-                columns.add(ByteBufferUtil.bytes(rawColumn));
-
-            List<AbstractType<?>> types = new ArrayList<>(rawTypes.size());
-            for (String rawType : rawTypes)
-                types.add(TypeParser.parse(rawType));
-
-            return new UserType(keyspace, name, columns, types);
-        }
-        catch (RequestValidationException e)
-        {
-            // If it has been written in the schema, it should be valid
-            throw new AssertionError();
-        }
-    }
-
-    public static Map<ByteBuffer, UserType> fromSchema(Row row)
-    {
-        UntypedResultSet results = QueryProcessor.resultify("SELECT * FROM system." + SystemKeyspace.SCHEMA_USER_TYPES_CF, row);
-        Map<ByteBuffer, UserType> types = new HashMap<>(results.size());
-        for (UntypedResultSet.Row result : results)
-        {
-            UserType type = fromSchema(result);
-            types.put(type.name, type);
-        }
-        return types;
-    }
-
-    public static Mutation toSchema(UserType newType, long timestamp)
-    {
-        return toSchema(new Mutation(Keyspace.SYSTEM_KS, SystemKeyspace.getSchemaKSKey(newType.keyspace)), newType, timestamp);
-    }
-
-    public static Mutation toSchema(Mutation mutation, UserType newType, long timestamp)
-    {
-        ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SCHEMA_USER_TYPES_CF);
-
-        Composite prefix = CFMetaData.SchemaUserTypesCf.comparator.make(newType.name);
-        CFRowAdder adder = new CFRowAdder(cf, prefix, timestamp);
-
-        adder.resetCollection("field_names");
-        adder.resetCollection("field_types");
-
-        for (int i = 0; i < newType.size(); i++)
-        {
-            adder.addListEntry("field_names", newType.fieldName(i));
-            adder.addListEntry("field_types", newType.fieldType(i).toString());
-        }
-        return mutation;
-    }
-
-    public Mutation toSchema(Mutation mutation, long timestamp)
-    {
-        for (UserType ut : userTypes.values())
-            toSchema(mutation, ut, timestamp);
-        return mutation;
-    }
-
-    public static Mutation dropFromSchema(UserType droppedType, long timestamp)
-    {
-        Mutation mutation = new Mutation(Keyspace.SYSTEM_KS, SystemKeyspace.getSchemaKSKey(droppedType.keyspace));
-        ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SCHEMA_USER_TYPES_CF);
-        int ldt = (int) (System.currentTimeMillis() / 1000);
-
-        Composite prefix = CFMetaData.SchemaUserTypesCf.comparator.make(droppedType.name);
-        cf.addAtom(new RangeTombstone(prefix, prefix.end(), timestamp, ldt));
-
-        return mutation;
-    }
-
     public UserType getType(ByteBuffer typeName)
     {
         return userTypes.get(typeName);
@@ -134,11 +49,11 @@
 
     public Map<ByteBuffer, UserType> getAllTypes()
     {
-        // Copy to avoid concurrent modification while iterating. Not intended to be called on a criticial path anyway
+        // Copy to avoid concurrent modification while iterating. Not intended to be called on a critical path anyway
         return new HashMap<>(userTypes);
     }
 
-    // This is *not* thread safe but is only called in DefsTables that is synchronized.
+    // This is *not* thread safe but is only called in Schema that is synchronized.
     public void addType(UserType type)
     {
         UserType old = userTypes.get(type.name);
diff --git a/src/java/org/apache/cassandra/config/YamlConfigurationLoader.java b/src/java/org/apache/cassandra/config/YamlConfigurationLoader.java
index 0061926..dc691c4 100644
--- a/src/java/org/apache/cassandra/config/YamlConfigurationLoader.java
+++ b/src/java/org/apache/cassandra/config/YamlConfigurationLoader.java
@@ -79,6 +79,7 @@
         return url;
     }
 
+    @Override
     public Config loadConfig() throws ConfigurationException
     {
         return loadConfig(getStorageConfigURL());
@@ -86,10 +87,9 @@
 
     public Config loadConfig(URL url) throws ConfigurationException
     {
-        InputStream input = null;
         try
         {
-            logger.info("Loading settings from {}", url);
+            logger.debug("Loading settings from {}", url);
             byte[] configBytes;
             try (InputStream is = url.openStream())
             {
@@ -101,10 +101,8 @@
                 throw new AssertionError(e);
             }
 
-            logConfig(configBytes);
-            
             org.yaml.snakeyaml.constructor.Constructor constructor = new org.yaml.snakeyaml.constructor.Constructor(Config.class);
-            TypeDescription seedDesc = new TypeDescription(SeedProviderDef.class);
+            TypeDescription seedDesc = new TypeDescription(ParameterizedClass.class);
             seedDesc.putMapPropertyType("parameters", String.class, String.class);
             constructor.addTypeDescription(seedDesc);
             MissingPropertiesChecker propertiesChecker = new MissingPropertiesChecker();
@@ -117,24 +115,10 @@
         }
         catch (YAMLException e)
         {
-            throw new ConfigurationException("Invalid yaml", e);
+            throw new ConfigurationException("Invalid yaml: " + url, e);
         }
     }
 
-    private void logConfig(byte[] configBytes)
-    {
-        Map<Object, Object> configMap = new TreeMap<>((Map<?, ?>) new Yaml().load(new ByteArrayInputStream(configBytes)));
-        // these keys contain passwords, don't log them
-        for (String sensitiveKey : new String[] { "client_encryption_options", "server_encryption_options" })
-        {
-            if (configMap.containsKey(sensitiveKey))
-            {
-                configMap.put(sensitiveKey, "<REDACTED>");
-            }
-        }
-        logger.info("Node configuration:[" + Joiner.on("; ").join(configMap.entrySet()) + "]");
-    }
-
     private static class MissingPropertiesChecker extends PropertyUtils
     {
         private final Set<String> missingProperties = new HashSet<>();
diff --git a/src/java/org/apache/cassandra/cql/AbstractModification.java b/src/java/org/apache/cassandra/cql/AbstractModification.java
deleted file mode 100644
index 8da2611..0000000
--- a/src/java/org/apache/cassandra/cql/AbstractModification.java
+++ /dev/null
@@ -1,126 +0,0 @@
-/*
- * 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.cql;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-
-import org.apache.cassandra.db.IMutation;
-import org.apache.cassandra.db.ConsistencyLevel;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.UnauthorizedException;
-import org.apache.cassandra.thrift.ThriftClientState;
-
-public abstract class AbstractModification
-{
-    public static final ConsistencyLevel defaultConsistency = ConsistencyLevel.ONE;
-
-    protected final String keyspace;
-    protected final String columnFamily;
-    protected final ConsistencyLevel cLevel;
-    protected final Long timestamp;
-    protected final int timeToLive;
-    protected final String keyName;
-
-    public AbstractModification(String keyspace, String columnFamily, String keyAlias, Attributes attrs)
-    {
-        this(keyspace, columnFamily, keyAlias, attrs.getConsistencyLevel(), attrs.getTimestamp(), attrs.getTimeToLive());
-    }
-
-    public AbstractModification(String keyspace, String columnFamily, String keyAlias, ConsistencyLevel cLevel, Long timestamp, int timeToLive)
-    {
-        this.keyspace = keyspace;
-        this.columnFamily = columnFamily;
-        this.cLevel = cLevel;
-        this.timestamp = timestamp;
-        this.timeToLive = timeToLive;
-        this.keyName = keyAlias.toUpperCase();
-    }
-
-    public String getKeyspace()
-    {
-        return keyspace;
-    }
-
-    public String getColumnFamily()
-    {
-        return columnFamily;
-    }
-
-    public ConsistencyLevel getConsistencyLevel()
-    {
-        return (cLevel != null) ? cLevel : defaultConsistency;
-    }
-
-    /**
-     * True if an explicit consistency level was parsed from the statement.
-     *
-     * @return true if a consistency was parsed, false otherwise.
-     */
-    public boolean isSetConsistencyLevel()
-    {
-        return cLevel != null;
-    }
-
-    public long getTimestamp(ThriftClientState clientState)
-    {
-        return timestamp == null ? clientState.getQueryState().getTimestamp() : timestamp;
-    }
-
-    public boolean isSetTimestamp()
-    {
-        return timestamp != null;
-    }
-
-    public int getTimeToLive()
-    {
-        return timeToLive;
-    }
-
-    public String getKeyName()
-    {
-        return keyName;
-    }
-
-    /**
-     * Convert statement into a list of mutations to apply on the server
-     *
-     * @param keyspace The working keyspace
-     * @param clientState current client status
-     *
-     * @return list of the mutations
-     *
-     * @throws InvalidRequestException on the wrong request
-     */
-    public abstract List<IMutation> prepareRowMutations(String keyspace, ThriftClientState clientState, List<ByteBuffer> variables)
-    throws InvalidRequestException, UnauthorizedException;
-
-    /**
-     * Convert statement into a list of mutations to apply on the server
-     *
-     * @param keyspace The working keyspace
-     * @param clientState current client status
-     * @param timestamp global timestamp to use for all mutations
-     *
-     * @return list of the mutations
-     *
-     * @throws InvalidRequestException on the wrong request
-     */
-    public abstract List<IMutation> prepareRowMutations(String keyspace, ThriftClientState clientState, Long timestamp, List<ByteBuffer> variables)
-    throws InvalidRequestException, UnauthorizedException;
-}
diff --git a/src/java/org/apache/cassandra/cql/AlterTableStatement.java b/src/java/org/apache/cassandra/cql/AlterTableStatement.java
deleted file mode 100644
index 5bc7011..0000000
--- a/src/java/org/apache/cassandra/cql/AlterTableStatement.java
+++ /dev/null
@@ -1,206 +0,0 @@
-/*
- * 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.cql;
-
-import java.nio.ByteBuffer;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.cassandra.cache.CachingOptions;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.db.marshal.TypeParser;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.RequestValidationException;
-import org.apache.cassandra.exceptions.SyntaxException;
-import org.apache.cassandra.io.compress.CompressionParameters;
-
-public class AlterTableStatement
-{
-    public static enum OperationType
-    {
-        ADD, ALTER, DROP, OPTS
-    }
-
-    public final OperationType oType;
-    public final String columnFamily, columnName, validator;
-    private final CFPropDefs cfProps = new CFPropDefs();
-
-    public AlterTableStatement(String columnFamily, OperationType type, String columnName)
-    {
-        this(columnFamily, type, columnName, null);
-    }
-
-    public AlterTableStatement(String columnFamily, OperationType type, String columnName, String validator)
-    {
-        this(columnFamily, type, columnName, validator, null);
-    }
-
-    public AlterTableStatement(String columnFamily, OperationType type, String columnName, String validator, Map<String, String> propertyMap)
-    {
-        this.columnFamily = columnFamily;
-        this.oType = type;
-        this.columnName = columnName;
-        this.validator = CFPropDefs.comparators.get(validator); // used only for ADD/ALTER commands
-
-        if (propertyMap != null)
-        {
-            for (Map.Entry<String, String> prop : propertyMap.entrySet())
-            {
-                cfProps.addProperty(prop.getKey(), prop.getValue());
-            }
-        }
-    }
-
-    public CFMetaData getCFMetaData(String keyspace) throws ConfigurationException, InvalidRequestException, SyntaxException
-    {
-        CFMetaData meta = Schema.instance.getCFMetaData(keyspace, columnFamily);
-        CFMetaData cfm = meta.copy();
-
-        ByteBuffer columnName = this.oType == OperationType.OPTS ? null
-                                                                 : meta.comparator.subtype(0).fromStringCQL2(this.columnName);
-
-        switch (oType)
-        {
-            case ADD:
-                cfm.addColumnDefinition(ColumnDefinition.regularDef(cfm, columnName, TypeParser.parse(validator), null));
-                break;
-
-            case ALTER:
-                // We only look for the first key alias which is ok for CQL2
-                ColumnDefinition partionKeyDef = cfm.partitionKeyColumns().get(0);
-                if (partionKeyDef.name.bytes.equals(columnName))
-                {
-                    cfm.keyValidator(TypeParser.parse(validator));
-                }
-                else
-                {
-                    ColumnDefinition toUpdate = null;
-
-                    for (ColumnDefinition columnDef : cfm.regularColumns())
-                    {
-                        if (columnDef.name.bytes.equals(columnName))
-                        {
-                            toUpdate = columnDef;
-                            break;
-                        }
-                    }
-
-                    if (toUpdate == null)
-                        throw new InvalidRequestException(String.format("Column '%s' was not found in CF '%s'",
-                                    this.columnName,
-                                    columnFamily));
-
-                    cfm.addOrReplaceColumnDefinition(toUpdate.withNewType(TypeParser.parse(validator)));
-                }
-                break;
-
-            case DROP:
-                ColumnDefinition toDelete = null;
-
-                for (ColumnDefinition columnDef : cfm.regularColumns())
-                {
-                    if (columnDef.name.bytes.equals(columnName))
-                    {
-                        toDelete = columnDef;
-                    }
-                }
-
-                if (toDelete == null)
-                    throw new InvalidRequestException(String.format("Column '%s' was not found in CF '%s'",
-                                                                    this.columnName,
-                                                                    columnFamily));
-
-                cfm.removeColumnDefinition(toDelete);
-                break;
-
-            case OPTS:
-                if (cfProps == null)
-                    throw new InvalidRequestException(String.format("ALTER COLUMNFAMILY WITH invoked, but no parameters found"));
-
-                cfProps.validate();
-                applyPropertiesToCFMetadata(cfm, cfProps);
-                break;
-        }
-
-        return cfm;
-    }
-
-    public String toString()
-    {
-        return String.format("AlterTableStatement(cf=%s, type=%s, column=%s, validator=%s)",
-                             columnFamily,
-                             oType,
-                             columnName,
-                             validator);
-    }
-
-    public static void applyPropertiesToCFMetadata(CFMetaData cfm, CFPropDefs cfProps) throws InvalidRequestException, ConfigurationException
-    {
-        if (cfProps.hasProperty(CFPropDefs.KW_COMPACTION_STRATEGY_CLASS))
-            cfm.compactionStrategyClass(cfProps.compactionStrategyClass);
-
-        if (cfProps.hasProperty(CFPropDefs.KW_COMPARATOR))
-            throw new InvalidRequestException("Can't change CF comparator after creation");
-
-        if (cfProps.hasProperty(CFPropDefs.KW_COMMENT))
-            cfm.comment(cfProps.getProperty(CFPropDefs.KW_COMMENT));
-
-        if (cfProps.hasProperty(CFPropDefs.KW_DEFAULTVALIDATION))
-        {
-            try
-            {
-                cfm.defaultValidator(cfProps.getValidator());
-            }
-            catch (RequestValidationException e)
-            {
-                throw new InvalidRequestException(String.format("Invalid validation type %s",
-                                                                cfProps.getProperty(CFPropDefs.KW_DEFAULTVALIDATION)));
-            }
-        }
-
-        cfm.readRepairChance(cfProps.getPropertyDouble(CFPropDefs.KW_READREPAIRCHANCE, cfm.getReadRepairChance()));
-        cfm.dcLocalReadRepairChance(cfProps.getPropertyDouble(CFPropDefs.KW_DCLOCALREADREPAIRCHANCE, cfm.getDcLocalReadRepair()));
-        cfm.gcGraceSeconds(cfProps.getPropertyInt(CFPropDefs.KW_GCGRACESECONDS, cfm.getGcGraceSeconds()));
-        int minCompactionThreshold = cfProps.getPropertyInt(CFPropDefs.KW_MINCOMPACTIONTHRESHOLD, cfm.getMinCompactionThreshold());
-        int maxCompactionThreshold = cfProps.getPropertyInt(CFPropDefs.KW_MAXCOMPACTIONTHRESHOLD, cfm.getMaxCompactionThreshold());
-        if (minCompactionThreshold <= 0 || maxCompactionThreshold <= 0)
-            throw new ConfigurationException("Disabling compaction by setting compaction thresholds to 0 has been deprecated, set the compaction option 'enabled' to false instead.");
-        cfm.minCompactionThreshold(minCompactionThreshold);
-        cfm.maxCompactionThreshold(maxCompactionThreshold);
-        cfm.caching(CachingOptions.fromString(cfProps.getPropertyString(CFPropDefs.KW_CACHING, cfm.getCaching().toString())));
-        cfm.defaultTimeToLive(cfProps.getPropertyInt(CFPropDefs.KW_DEFAULT_TIME_TO_LIVE, cfm.getDefaultTimeToLive()));
-        cfm.speculativeRetry(CFMetaData.SpeculativeRetry.fromString(cfProps.getPropertyString(CFPropDefs.KW_SPECULATIVE_RETRY, cfm.getSpeculativeRetry().toString())));
-        cfm.bloomFilterFpChance(cfProps.getPropertyDouble(CFPropDefs.KW_BF_FP_CHANCE, cfm.getBloomFilterFpChance()));
-        cfm.memtableFlushPeriod(cfProps.getPropertyInt(CFPropDefs.KW_MEMTABLE_FLUSH_PERIOD, cfm.getMemtableFlushPeriod()));
-
-        if (!cfProps.compactionStrategyOptions.isEmpty())
-        {
-            cfm.compactionStrategyOptions(new HashMap<String, String>());
-            for (Map.Entry<String, String> entry : cfProps.compactionStrategyOptions.entrySet())
-                cfm.compactionStrategyOptions.put(entry.getKey(), entry.getValue());
-        }
-
-        if (!cfProps.compressionParameters.isEmpty())
-        {
-            cfm.compressionParameters(CompressionParameters.create(cfProps.compressionParameters));
-        }
-    }
-}
diff --git a/src/java/org/apache/cassandra/cql/Attributes.java b/src/java/org/apache/cassandra/cql/Attributes.java
deleted file mode 100644
index faee3b8..0000000
--- a/src/java/org/apache/cassandra/cql/Attributes.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/*
- * 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.cql;
-
-import org.apache.cassandra.db.ConsistencyLevel;
-
-/**
- * Class to contain attributes for statements
- */
-public class Attributes
-{
-    private ConsistencyLevel cLevel;
-    private Long timestamp;
-    private int timeToLive;
-
-    public Attributes()
-    {}
-
-    public Attributes(ConsistencyLevel cLevel, Long timestamp, int timeToLive)
-    {
-        this.cLevel = cLevel;
-        this.timestamp = timestamp;
-        this.timeToLive = timeToLive;
-    }
-
-    public ConsistencyLevel getConsistencyLevel()
-    {
-        return cLevel;
-    }
-
-    public void setConsistencyLevel(ConsistencyLevel cLevel)
-    {
-        this.cLevel = cLevel;
-    }
-
-    public Long getTimestamp()
-    {
-        return timestamp;
-    }
-
-    public void setTimestamp(Long timestamp)
-    {
-        this.timestamp = timestamp;
-    }
-
-    public int getTimeToLive()
-    {
-        return timeToLive;
-    }
-
-    public void setTimeToLive(int timeToLive)
-    {
-        this.timeToLive = timeToLive;
-    }
-
-    public String toString()
-    {
-        return String.format("Attributes(consistency=%s, timestamp=%s, timeToLive=%s)", cLevel, timestamp, timeToLive);
-    }
-
-}
diff --git a/src/java/org/apache/cassandra/cql/BatchStatement.java b/src/java/org/apache/cassandra/cql/BatchStatement.java
deleted file mode 100644
index b141bcc..0000000
--- a/src/java/org/apache/cassandra/cql/BatchStatement.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- * 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.cql;
-
-import java.nio.ByteBuffer;
-import java.util.LinkedList;
-import java.util.List;
-
-import org.apache.cassandra.db.IMutation;
-import org.apache.cassandra.db.ConsistencyLevel;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.UnauthorizedException;
-import org.apache.cassandra.thrift.ThriftClientState;
-
-/**
- * A <code>BATCH</code> statement parsed from a CQL query.
- *
- */
-public class BatchStatement
-{
-    // statements to execute
-    protected final List<AbstractModification> statements;
-
-    // global consistency level
-    protected final ConsistencyLevel consistency;
-
-    // global timestamp to apply for each mutation
-    protected final Long timestamp;
-
-    // global time to live
-    protected final int timeToLive;
-
-    /**
-     * Creates a new BatchStatement from a list of statements and a
-     * Thrift consistency level.
-     *
-     * @param statements a list of UpdateStatements
-     * @param attrs additional attributes for statement (CL, timestamp, timeToLive)
-     */
-    public BatchStatement(List<AbstractModification> statements, Attributes attrs)
-    {
-        this.statements = statements;
-        this.consistency = attrs.getConsistencyLevel();
-        this.timestamp = attrs.getTimestamp();
-        this.timeToLive = attrs.getTimeToLive();
-    }
-
-    public List<AbstractModification> getStatements()
-    {
-        return statements;
-    }
-
-    public ConsistencyLevel getConsistencyLevel()
-    {
-        return consistency;
-    }
-
-    public int getTimeToLive()
-    {
-        return timeToLive;
-    }
-
-    public long getTimestamp()
-    {
-        return timestamp;
-    }
-
-    public List<IMutation> getMutations(String keyspace, ThriftClientState clientState, List<ByteBuffer> variables)
-    throws InvalidRequestException, UnauthorizedException
-    {
-        List<IMutation> batch = new LinkedList<IMutation>();
-
-        for (AbstractModification statement : statements) {
-            batch.addAll(statement.prepareRowMutations(keyspace, clientState, timestamp, variables));
-        }
-
-        return batch;
-    }
-
-    public boolean isSetTimestamp()
-    {
-        return timestamp != null;
-    }
-
-    public String toString()
-    {
-        return String.format("BatchStatement(statements=%s, consistency=%s)", statements, consistency);
-    }
-}
diff --git a/src/java/org/apache/cassandra/cql/CFPropDefs.java b/src/java/org/apache/cassandra/cql/CFPropDefs.java
deleted file mode 100644
index f65cb94..0000000
--- a/src/java/org/apache/cassandra/cql/CFPropDefs.java
+++ /dev/null
@@ -1,306 +0,0 @@
-/*
- * 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.cql;
-
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-import com.google.common.base.Strings;
-import com.google.common.collect.Sets;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.db.compaction.AbstractCompactionStrategy;
-import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.db.marshal.TypeParser;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.SyntaxException;
-import org.apache.commons.lang3.StringUtils;
-
-public class CFPropDefs {
-    private static final Logger logger = LoggerFactory.getLogger(CFPropDefs.class);
-
-    public static final String KW_COMPARATOR = "comparator";
-    public static final String KW_COMMENT = "comment";
-    public static final String KW_READREPAIRCHANCE = "read_repair_chance";
-    public static final String KW_DCLOCALREADREPAIRCHANCE = "dclocal_read_repair_chance";
-    public static final String KW_GCGRACESECONDS = "gc_grace_seconds";
-    public static final String KW_DEFAULTVALIDATION = "default_validation";
-    public static final String KW_MINCOMPACTIONTHRESHOLD = "min_compaction_threshold";
-    public static final String KW_MAXCOMPACTIONTHRESHOLD = "max_compaction_threshold";
-    public static final String KW_COMPACTION_STRATEGY_CLASS = "compaction_strategy_class";
-    public static final String KW_CACHING = "caching";
-    public static final String KW_ROWS_PER_PARTITION_TO_CACHE = "rows_per_partition_to_cache";
-    public static final String KW_DEFAULT_TIME_TO_LIVE = "default_time_to_live";
-    public static final String KW_SPECULATIVE_RETRY = "speculative_retry";
-    public static final String KW_BF_FP_CHANCE = "bloom_filter_fp_chance";
-    public static final String KW_MEMTABLE_FLUSH_PERIOD = "memtable_flush_period_in_ms";
-
-    // Maps CQL short names to the respective Cassandra comparator/validator class names
-    public static final Map<String, String> comparators = new HashMap<String, String>();
-    public static final Set<String> keywords = new HashSet<String>();
-    public static final Set<String> obsoleteKeywords = new HashSet<String>();
-    public static final Set<String> allowedKeywords = new HashSet<String>();
-
-    public static final String COMPACTION_OPTIONS_PREFIX = "compaction_strategy_options";
-    public static final String COMPRESSION_PARAMETERS_PREFIX = "compression_parameters";
-
-    static
-    {
-        comparators.put("ascii", "AsciiType");
-        comparators.put("bigint", "LongType");
-        comparators.put("blob", "BytesType");
-        comparators.put("boolean", "BooleanType");
-        comparators.put("counter", "CounterColumnType");
-        comparators.put("decimal", "DecimalType");
-        comparators.put("double", "DoubleType");
-        comparators.put("float", "FloatType");
-        comparators.put("int", "Int32Type");
-        comparators.put("text", "UTF8Type");
-        comparators.put("timestamp", "DateType");
-        comparators.put("uuid", "UUIDType");
-        comparators.put("varchar", "UTF8Type");
-        comparators.put("varint", "IntegerType");
-
-        keywords.add(KW_COMPARATOR);
-        keywords.add(KW_COMMENT);
-        keywords.add(KW_READREPAIRCHANCE);
-        keywords.add(KW_DCLOCALREADREPAIRCHANCE);
-        keywords.add(KW_GCGRACESECONDS);
-        keywords.add(KW_DEFAULTVALIDATION);
-        keywords.add(KW_MINCOMPACTIONTHRESHOLD);
-        keywords.add(KW_MAXCOMPACTIONTHRESHOLD);
-        keywords.add(KW_COMPACTION_STRATEGY_CLASS);
-        keywords.add(KW_CACHING);
-        keywords.add(KW_ROWS_PER_PARTITION_TO_CACHE);
-        keywords.add(KW_DEFAULT_TIME_TO_LIVE);
-        keywords.add(KW_SPECULATIVE_RETRY);
-        keywords.add(KW_BF_FP_CHANCE);
-        keywords.add(KW_MEMTABLE_FLUSH_PERIOD);
-
-        obsoleteKeywords.add("row_cache_size");
-        obsoleteKeywords.add("key_cache_size");
-        obsoleteKeywords.add("row_cache_save_period_in_seconds");
-        obsoleteKeywords.add("key_cache_save_period_in_seconds");
-        obsoleteKeywords.add("memtable_throughput_in_mb");
-        obsoleteKeywords.add("memtable_operations_in_millions");
-        obsoleteKeywords.add("memtable_flush_after_mins");
-        obsoleteKeywords.add("row_cache_provider");
-        obsoleteKeywords.add("replicate_on_write");
-        obsoleteKeywords.add("populate_io_cache_on_flush");
-
-        allowedKeywords.addAll(keywords);
-        allowedKeywords.addAll(obsoleteKeywords);
-    }
-
-    public final Map<String, String> properties = new HashMap<String, String>();
-    public Class<? extends AbstractCompactionStrategy> compactionStrategyClass;
-    public final Map<String, String> compactionStrategyOptions = new HashMap<String, String>();
-    public final Map<String, String> compressionParameters = new HashMap<String, String>();
-
-    public void validate() throws InvalidRequestException, ConfigurationException
-    {
-        compactionStrategyClass = CFMetaData.DEFAULT_COMPACTION_STRATEGY_CLASS;
-
-        // we need to remove parent:key = value pairs from the main properties
-        Set<String> propsToRemove = new HashSet<String>();
-
-        // check if we have compaction/compression options
-        for (String property : properties.keySet())
-        {
-            if (!property.contains(":"))
-                continue;
-
-            String key = property.split(":")[1];
-            String val = properties.get(property);
-
-            if (property.startsWith(COMPACTION_OPTIONS_PREFIX))
-            {
-                compactionStrategyOptions.put(key, val);
-                propsToRemove.add(property);
-            }
-
-            if (property.startsWith(COMPRESSION_PARAMETERS_PREFIX))
-            {
-                compressionParameters.put(key, val);
-                propsToRemove.add(property);
-            }
-        }
-
-        for (String property : propsToRemove)
-            properties.remove(property);
-        // Catch the case where someone passed a kwarg that is not recognized.
-        for (String bogus : Sets.difference(properties.keySet(), allowedKeywords))
-            throw new InvalidRequestException(bogus + " is not a valid keyword argument for CREATE COLUMNFAMILY");
-        for (String obsolete : Sets.intersection(properties.keySet(), obsoleteKeywords))
-            logger.warn("Ignoring obsolete property {}", obsolete);
-
-        // Validate min/max compaction thresholds
-        Integer minCompaction = getPropertyInt(KW_MINCOMPACTIONTHRESHOLD, null);
-        Integer maxCompaction = getPropertyInt(KW_MAXCOMPACTIONTHRESHOLD, null);
-
-        if ((minCompaction != null) && (maxCompaction != null))     // Both min and max are set
-        {
-            if ((minCompaction > maxCompaction) && (maxCompaction != 0))
-                throw new InvalidRequestException(String.format("%s cannot be larger than %s",
-                        KW_MINCOMPACTIONTHRESHOLD,
-                        KW_MAXCOMPACTIONTHRESHOLD));
-        }
-        else if (minCompaction != null)     // Only the min threshold is set
-        {
-            if (minCompaction > CFMetaData.DEFAULT_MAX_COMPACTION_THRESHOLD)
-                throw new InvalidRequestException(String.format("%s cannot be larger than %s, (default %s)",
-                        KW_MINCOMPACTIONTHRESHOLD,
-                        KW_MAXCOMPACTIONTHRESHOLD,
-                        CFMetaData.DEFAULT_MAX_COMPACTION_THRESHOLD));
-        }
-        else if (maxCompaction != null)     // Only the max threshold is set
-        {
-            if ((maxCompaction < CFMetaData.DEFAULT_MIN_COMPACTION_THRESHOLD) && (maxCompaction != 0))
-                throw new InvalidRequestException(String.format("%s cannot be smaller than %s, (default %s)",
-                        KW_MAXCOMPACTIONTHRESHOLD,
-                        KW_MINCOMPACTIONTHRESHOLD,
-                        CFMetaData.DEFAULT_MIN_COMPACTION_THRESHOLD));
-        }
-
-        Integer defaultTimeToLive = getPropertyInt(KW_DEFAULT_TIME_TO_LIVE, null);
-
-        if (defaultTimeToLive != null)
-        {
-            if (defaultTimeToLive < 0)
-                throw new InvalidRequestException(String.format("%s cannot be smaller than %s, (default %s)",
-                        KW_DEFAULT_TIME_TO_LIVE,
-                        0,
-                        CFMetaData.DEFAULT_DEFAULT_TIME_TO_LIVE));
-        }
-
-        CFMetaData.validateCompactionOptions(compactionStrategyClass, compactionStrategyOptions);
-    }
-
-    /** Map a keyword to the corresponding value */
-    public void addProperty(String name, String value)
-    {
-        properties.put(name, value);
-    }
-
-    public Boolean hasProperty(String name)
-    {
-        return properties.containsKey(name);
-    }
-
-    /* If not comparator/validator is not specified, default to text (BytesType is the wrong default for CQL
-     * since it uses hex terms).  If the value specified is not found in the comparators map, assume the user
-     * knows what they are doing (a custom comparator/validator for example), and pass it on as-is.
-     */
-
-    public AbstractType<?> getComparator() throws ConfigurationException, SyntaxException
-    {
-        return TypeParser.parse((comparators.get(getPropertyString(KW_COMPARATOR, "text")) != null)
-                                  ? comparators.get(getPropertyString(KW_COMPARATOR, "text"))
-                                  : getPropertyString(KW_COMPARATOR, "text"));
-    }
-
-    public AbstractType<?> getValidator() throws ConfigurationException, SyntaxException
-    {
-        return TypeParser.parse((comparators.get(getPropertyString(KW_DEFAULTVALIDATION, "text")) != null)
-                                  ? comparators.get(getPropertyString(KW_DEFAULTVALIDATION, "text"))
-                                  : getPropertyString(KW_DEFAULTVALIDATION, "text"));
-    }
-
-    public String getProperty(String name)
-    {
-        return properties.get(name);
-    }
-
-    public String getPropertyString(String key, String defaultValue)
-    {
-        String value = properties.get(key);
-        return value != null ? value : defaultValue;
-    }
-
-    // Return a property value, typed as a Boolean
-    public Boolean getPropertyBoolean(String key, Boolean defaultValue)
-    {
-        String value = properties.get(key);
-        return (value == null) ? defaultValue : value.toLowerCase().matches("(1|true|yes)");
-    }
-
-    // Return a property value, typed as a Double
-    public Double getPropertyDouble(String key, Double defaultValue) throws InvalidRequestException
-    {
-        Double result;
-        String value = properties.get(key);
-
-        if (value == null)
-            result = defaultValue;
-        else
-        {
-            try
-            {
-                result = Double.valueOf(value);
-            }
-            catch (NumberFormatException e)
-            {
-                throw new InvalidRequestException(String.format("%s not valid for \"%s\"", value, key));
-            }
-        }
-        return result;
-    }
-
-    // Return a property value, typed as an Integer
-    public Integer getPropertyInt(String key, Integer defaultValue) throws InvalidRequestException
-    {
-        Integer result;
-        String value = properties.get(key);
-
-        if (value == null)
-            result = defaultValue;
-        else
-        {
-            try
-            {
-                result = Integer.valueOf(value);
-            }
-            catch (NumberFormatException e)
-            {
-                throw new InvalidRequestException(String.format("%s not valid for \"%s\"", value, key));
-            }
-        }
-        return result;
-    }
-
-    public Set<String> getPropertySet(String key, Set<String> defaultValue)
-    {
-        String value = properties.get(key);
-        if (Strings.isNullOrEmpty(value))
-            return defaultValue;
-        return Sets.newHashSet(StringUtils.split(value, ','));
-    }
-
-    public String toString()
-    {
-        return String.format("CFPropDefs(%s, compaction: %s, compression: %s)",
-                             properties.toString(),
-                             compactionStrategyOptions.toString(),
-                             compressionParameters.toString());
-    }
-}
diff --git a/src/java/org/apache/cassandra/cql/CQLStatement.java b/src/java/org/apache/cassandra/cql/CQLStatement.java
deleted file mode 100644
index 157ce67..0000000
--- a/src/java/org/apache/cassandra/cql/CQLStatement.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * 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.cql;
-
-public class CQLStatement
-{
-    public final StatementType type;
-    public final Object statement;
-    public final int boundTerms;
-
-    public CQLStatement(StatementType type, Object statement, int lastMarker)
-    {
-        this.type = type;
-        this.statement = statement;
-        this.boundTerms = lastMarker + 1;
-    }
-}
diff --git a/src/java/org/apache/cassandra/cql/Cql.g b/src/java/org/apache/cassandra/cql/Cql.g
deleted file mode 100644
index f76be27..0000000
--- a/src/java/org/apache/cassandra/cql/Cql.g
+++ /dev/null
@@ -1,656 +0,0 @@
-
-/*
- * 
- * 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.
- * 
- */
- 
-grammar Cql;
-
-options {
-    language = Java;
-}
-
-@header {
-    package org.apache.cassandra.cql;
-    import java.util.Map;
-    import java.util.HashMap;
-    import java.util.Collections;
-    import java.util.List;
-    import java.util.ArrayList;
-    import org.apache.cassandra.exceptions.SyntaxException;
-    import org.apache.cassandra.utils.Pair;
-    import org.apache.cassandra.db.ConsistencyLevel;
-
-    import static org.apache.cassandra.cql.AlterTableStatement.OperationType;
-}
-
-@members {
-    private List<String> recognitionErrors = new ArrayList<String>();
-    private int currentBindMarkerIdx = -1;
-    
-    public void displayRecognitionError(String[] tokenNames, RecognitionException e)
-    {
-        String hdr = getErrorHeader(e);
-        String msg = getErrorMessage(e, tokenNames);
-        recognitionErrors.add(hdr + " " + msg);
-    }
-    
-    public List<String> getRecognitionErrors()
-    {
-        return recognitionErrors;
-    }
-    
-    public void throwLastRecognitionError() throws SyntaxException
-    {
-        if (recognitionErrors.size() > 0)
-            throw new SyntaxException(recognitionErrors.get((recognitionErrors.size()-1)));
-    }
-
-    // used by UPDATE of the counter columns to validate if '-' was supplied by user
-    public void validateMinusSupplied(Object op, final Term value, IntStream stream) throws MissingTokenException
-    {
-        if (op == null && Long.parseLong(value.getText()) > 0)
-            throw new MissingTokenException(102, stream, value);
-    }
-}
-
-@lexer::header {
-    package org.apache.cassandra.cql;
-    import org.apache.cassandra.exceptions.SyntaxException;
-}
-
-@lexer::members {
-    List<Token> tokens = new ArrayList<Token>();
-    
-    public void emit(Token token) {
-        state.token = token;
-        tokens.add(token);
-    }
-    
-    public Token nextToken() {
-        super.nextToken();
-        if (tokens.size() == 0)
-            return new CommonToken(Token.EOF);
-        return tokens.remove(0);
-    }
-    
-    private List<String> recognitionErrors = new ArrayList<String>();
-    
-    public void displayRecognitionError(String[] tokenNames, RecognitionException e)
-    {
-        String hdr = getErrorHeader(e);
-        String msg = getErrorMessage(e, tokenNames);
-        recognitionErrors.add(hdr + " " + msg);
-    }
-    
-    public List<String> getRecognitionErrors()
-    {
-        return recognitionErrors;
-    }
-    
-    public void throwLastRecognitionError() throws SyntaxException
-    {
-        if (recognitionErrors.size() > 0)
-            throw new SyntaxException(recognitionErrors.get((recognitionErrors.size()-1)));
-    }
-}
-
-query returns [CQLStatement stmnt]
-    : selectStatement   { $stmnt = new CQLStatement(StatementType.SELECT, $selectStatement.expr, currentBindMarkerIdx); }
-    | insertStatement endStmnt { $stmnt = new CQLStatement(StatementType.INSERT, $insertStatement.expr, currentBindMarkerIdx); }
-    | updateStatement endStmnt { $stmnt = new CQLStatement(StatementType.UPDATE, $updateStatement.expr, currentBindMarkerIdx); }
-    | batchStatement { $stmnt = new CQLStatement(StatementType.BATCH, $batchStatement.expr, currentBindMarkerIdx); }
-    | useStatement      { $stmnt = new CQLStatement(StatementType.USE, $useStatement.keyspace, currentBindMarkerIdx); }
-    | truncateStatement { $stmnt = new CQLStatement(StatementType.TRUNCATE, $truncateStatement.cf, currentBindMarkerIdx); }
-    | deleteStatement endStmnt { $stmnt = new CQLStatement(StatementType.DELETE, $deleteStatement.expr, currentBindMarkerIdx); }
-    | createKeyspaceStatement { $stmnt = new CQLStatement(StatementType.CREATE_KEYSPACE, $createKeyspaceStatement.expr, currentBindMarkerIdx); }
-    | createColumnFamilyStatement { $stmnt = new CQLStatement(StatementType.CREATE_COLUMNFAMILY, $createColumnFamilyStatement.expr, currentBindMarkerIdx); }
-    | createIndexStatement { $stmnt = new CQLStatement(StatementType.CREATE_INDEX, $createIndexStatement.expr, currentBindMarkerIdx); }
-    | dropIndexStatement   { $stmnt = new CQLStatement(StatementType.DROP_INDEX, $dropIndexStatement.expr, currentBindMarkerIdx); }
-    | dropKeyspaceStatement { $stmnt = new CQLStatement(StatementType.DROP_KEYSPACE, $dropKeyspaceStatement.ksp, currentBindMarkerIdx); }
-    | dropColumnFamilyStatement { $stmnt = new CQLStatement(StatementType.DROP_COLUMNFAMILY, $dropColumnFamilyStatement.cfam, currentBindMarkerIdx); }
-    | alterTableStatement { $stmnt = new CQLStatement(StatementType.ALTER_TABLE, $alterTableStatement.expr, currentBindMarkerIdx); }
-    ;
-
-// USE <KEYSPACE>;
-useStatement returns [String keyspace]
-    : K_USE name=( IDENT | INTEGER | STRING_LITERAL ) { $keyspace = $name.text; } endStmnt
-    ;
-
-/**
- * SELECT
- *  (REVERSED)? <expression>
- * FROM
- *     <CF>
- * USING
- *     CONSISTENCY <LEVEL>
- * WHERE
- *     KEY = "key1" AND KEY = "key2" AND
- *     COL > 1 AND COL < 100
- * LIMIT <NUMBER>;
- */
-selectStatement returns [SelectStatement expr]
-    : { 
-          int numRecords = 10000;
-          SelectExpression expression = null;
-          boolean isCountOp = false;
-          ConsistencyLevel cLevel = ConsistencyLevel.ONE;
-      }
-      K_SELECT
-          ( s1=selectExpression                 { expression = s1; }
-          | K_COUNT '(' s2=selectExpression ')' { expression = s2; isCountOp = true; }
-          )
-          K_FROM (keyspace=(IDENT | STRING_LITERAL | INTEGER) '.')? columnFamily=( IDENT | STRING_LITERAL | INTEGER )
-          ( K_USING K_CONSISTENCY K_LEVEL { cLevel = ConsistencyLevel.valueOf($K_LEVEL.text.toUpperCase()); } )?
-          ( K_WHERE whereClause )?
-          ( K_LIMIT rows=INTEGER { numRecords = Integer.parseInt($rows.text); } )?
-          endStmnt
-      {
-          return new SelectStatement(expression,
-                                     isCountOp,
-                                     $keyspace.text,
-                                     $columnFamily.text,
-                                     cLevel,
-                                     $whereClause.clause,
-                                     numRecords);
-      }
-    ;
-
-// [FIRST n] [REVERSED] name1[[[,name2],nameN],...]
-// [FIRST n] [REVERSED] name1..nameN
-selectExpression returns [SelectExpression expr]
-    : {
-          int count = 10000;
-          boolean reversed = false;
-          boolean hasFirstSet = false;
-      }
-      ( K_FIRST { hasFirstSet = true; } cols=INTEGER { count = Integer.parseInt($cols.text); } )?
-      ( K_REVERSED { reversed = true; } )?
-      ( first=term { $expr = new SelectExpression(first, count, reversed, hasFirstSet); }
-            (',' next=term { $expr.and(next); })*
-      | start=term RANGEOP finish=term { $expr = new SelectExpression(start, finish, count, reversed, false, hasFirstSet); }
-      | '\*' { $expr = new SelectExpression(new Term(), new Term(), count, reversed, true, hasFirstSet); }
-      )
-    ;
-
-// relation [[AND relation] ...]
-whereClause returns [WhereClause clause]
-    @init {
-        WhereClause inClause = new WhereClause();
-    }
-    : first=relation { $clause = new WhereClause(first); } 
-          (K_AND next=relation { $clause.and(next); })*
-      | key_alias=term { inClause.setKeyAlias(key_alias.getText()); }
-           K_IN '(' f1=term { inClause.andKeyEquals(f1); }
-                  (',' fN=term { inClause.andKeyEquals(fN); } )* ')'
-        { inClause.setMultiKey(true); $clause = inClause; }
-    ;
-
-/**
- * INSERT INTO
- *    <CF>
- *    (KEY, <column>, <column>, ...)
- * VALUES
- *    (<key>, <value>, <value>, ...)
- * (USING
- *    CONSISTENCY <level>
- *   (AND TIMESTAMP <long>)?
- * )?;
- *
- * Consistency level is set to ONE by default
- */
-insertStatement returns [UpdateStatement expr]
-    : {
-          Attributes attrs = new Attributes();
-
-          List<Term> columnNames  = new ArrayList<Term>();
-          List<Term> columnValues = new ArrayList<Term>();
-      }
-      K_INSERT K_INTO (keyspace=(IDENT | STRING_LITERAL | INTEGER) '.')? columnFamily=( IDENT | STRING_LITERAL | INTEGER )
-          '(' key_alias=term ( ',' column_name=term  { columnNames.add($column_name.item); } )+ ')'
-        K_VALUES
-          '(' key=term ( ',' column_value=term { columnValues.add($column_value.item); })+ ')'
-        ( usingClause[attrs] )?
-      {
-          return new UpdateStatement($keyspace.text, $columnFamily.text, key_alias.getText(), columnNames, columnValues, Collections.singletonList(key), attrs);
-      }
-    ;
-
-usingClause[Attributes attrs]
-    : K_USING usingClauseObjective[attrs] ( K_AND? usingClauseObjective[attrs] )*
-    ;
-
-usingClauseDelete[Attributes attrs]
-    : K_USING usingClauseDeleteObjective[attrs] ( K_AND? usingClauseDeleteObjective[attrs] )*
-    ;
-
-usingClauseDeleteObjective[Attributes attrs]
-    : K_CONSISTENCY K_LEVEL  { attrs.setConsistencyLevel(ConsistencyLevel.valueOf($K_LEVEL.text.toUpperCase())); }
-    | K_TIMESTAMP ts=INTEGER { attrs.setTimestamp(Long.valueOf($ts.text)); }
-    ;
-
-usingClauseObjective[Attributes attrs]
-    : usingClauseDeleteObjective[attrs]
-    | K_TTL t=INTEGER        { attrs.setTimeToLive(Integer.parseInt($t.text)); }
-    ;
-
-/**
- * BEGIN BATCH [USING CONSISTENCY <LVL>]
- *   UPDATE <CF> SET name1 = value1 WHERE KEY = keyname1;
- *   UPDATE <CF> SET name2 = value2 WHERE KEY = keyname2;
- *   UPDATE <CF> SET name3 = value3 WHERE KEY = keyname3;
- *   ...
- * APPLY BATCH
- *
- * OR
- *
- * BEGIN BATCH [USING CONSISTENCY <LVL>]
- *   INSERT INTO <CF> (KEY, <name>) VALUES ('<key>', '<value>');
- *   INSERT INTO <CF> (KEY, <name>) VALUES ('<key>', '<value>');
- *   ...
- * APPLY BATCH
- *
- * OR
- *
- * BEGIN BATCH [USING CONSISTENCY <LVL>]
- *   DELETE name1, name2 FROM <CF> WHERE key = <key>
- *   DELETE name3, name4 FROM <CF> WHERE key = <key>
- *   ...
- * APPLY BATCH
- */
-batchStatement returns [BatchStatement expr]
-    : {
-          Attributes attrs = new Attributes();
-          attrs.setConsistencyLevel(ConsistencyLevel.ONE);
-
-          List<AbstractModification> statements = new ArrayList<AbstractModification>();
-      }
-      K_BEGIN K_BATCH ( usingClause[attrs] )?
-          s1=batchStatementObjective ';'? { statements.add(s1); } ( sN=batchStatementObjective ';'? { statements.add(sN); } )*
-      K_APPLY K_BATCH endStmnt
-      {
-          return new BatchStatement(statements, attrs);
-      }
-    ;
-
-batchStatementObjective returns [AbstractModification statement]
-    : i=insertStatement  { $statement = i; }
-    | u=updateStatement  { $statement = u; }
-    | d=deleteStatement  { $statement = d; }
-    ;
-
-/**
- * UPDATE
- *     <CF>
- * (USING
- *     CONSISTENCY.ONE
- *    (AND TIMESTAMP <long>)?
- * )?
- * SET
- *     name1 = value1,
- *     name2 = value2
- * WHERE
- *     KEY = keyname;
- */
-updateStatement returns [UpdateStatement expr]
-    : {
-          Attributes attrs = new Attributes();
-          Map<Term, Operation> columns = new HashMap<Term, Operation>();
-          List<Term> keyList = null;
-      }
-      K_UPDATE (keyspace=(IDENT | STRING_LITERAL | INTEGER) '.')? columnFamily=( IDENT | STRING_LITERAL | INTEGER )
-          ( usingClause[attrs] )?
-          K_SET termPairWithOperation[columns] (',' termPairWithOperation[columns])*
-          K_WHERE ( key_alias=term ('=' key=term { keyList = Collections.singletonList(key); }
-                                    |
-                                    K_IN '(' keys=termList { keyList = $keys.items; } ')' ))
-      {
-          return new UpdateStatement($keyspace.text, $columnFamily.text, key_alias.getText(), columns, keyList, attrs);
-      }
-    ;
-
-/**
- * DELETE
- *     name1, name2
- * FROM
- *     <CF>
- * USING
- *     CONSISTENCY.<LVL>
- * WHERE
- *     KEY = keyname;
- */
-deleteStatement returns [DeleteStatement expr]
-    : {
-          Attributes attrs = new Attributes();
-          List<Term> keyList = null;
-          List<Term> columnsList = Collections.emptyList();
-      }
-      K_DELETE
-          ( cols=termList { columnsList = $cols.items; })?
-          K_FROM (keyspace=(IDENT | STRING_LITERAL | INTEGER) '.')? columnFamily=( IDENT | STRING_LITERAL | INTEGER )
-          ( usingClauseDelete[attrs] )?
-          ( K_WHERE key_alias=term ('=' key=term           { keyList = Collections.singletonList(key); }
-                                   | K_IN '(' keys=termList { keyList = $keys.items; } ')')
-                  )?
-      {
-          return new DeleteStatement(columnsList, $keyspace.text, $columnFamily.text, key_alias.getText(), keyList, attrs);
-      }
-    ;
-
-
-
-/** CREATE KEYSPACE <KEYSPACE> WITH attr1 = value1 AND attr2 = value2; */
-createKeyspaceStatement returns [CreateKeyspaceStatement expr]
-    : {
-          Map<String, String> attrs = new HashMap<String, String>();
-      }
-      K_CREATE K_KEYSPACE keyspace=( IDENT | STRING_LITERAL | INTEGER )
-          K_WITH  a1=( COMPIDENT | IDENT ) '=' v1=( STRING_LITERAL | INTEGER | IDENT ) { attrs.put($a1.text, $v1.text); }
-          ( K_AND aN=( COMPIDENT | IDENT ) '=' vN=( STRING_LITERAL | INTEGER | IDENT ) { attrs.put($aN.text, $vN.text); } )*
-          endStmnt
-      {
-          return new CreateKeyspaceStatement($keyspace.text, attrs);
-      }
-    ;
-    
-/**
- * CREATE COLUMNFAMILY <CF> (
- *     <name1> <type>,
- *     <name2> <type>,
- *     <name3> <type>
- * ) WITH comparator = <type> [AND ...];
- */
-createColumnFamilyStatement returns [CreateColumnFamilyStatement expr]
-    : K_CREATE K_COLUMNFAMILY name=( IDENT | STRING_LITERAL | INTEGER ) { $expr = new CreateColumnFamilyStatement($name.text); }
-      ( '(' createCfamColumns[expr] ( ',' createCfamColumns[expr] )* ')' )?
-      ( K_WITH prop1=(COMPIDENT | IDENT) '=' arg1=createCfamKeywordArgument { $expr.addProperty($prop1.text, $arg1.arg); }
-          ( K_AND propN=(COMPIDENT | IDENT) '=' argN=createCfamKeywordArgument { $expr.addProperty($propN.text, $argN.arg); } )*
-      )?
-      endStmnt
-    ;
-
-createCfamColumns[CreateColumnFamilyStatement expr]
-    : n=term v=createCfamColumnValidator { $expr.addColumn(n, $v.validator); }
-    | k=term v=createCfamColumnValidator K_PRIMARY K_KEY { $expr.setKeyAlias(k.getText()); $expr.setKeyType($v.validator); }
-    ;
-
-createCfamColumnValidator returns [String validator]
-    : comparatorType { $validator = $comparatorType.text; }
-    | STRING_LITERAL { $validator = $STRING_LITERAL.text; }
-    ;
-
-createCfamKeywordArgument returns [String arg]
-    : comparatorType { $arg = $comparatorType.text; }
-    | value=( STRING_LITERAL | IDENT | INTEGER | FLOAT ) { $arg = $value.text; }
-    ;
-
-/** CREATE INDEX [indexName] ON columnFamily (columnName); */
-createIndexStatement returns [CreateIndexStatement expr]
-    : K_CREATE K_INDEX (idxName=IDENT)? K_ON cf=( IDENT | STRING_LITERAL | INTEGER ) '(' columnName=term ')' endStmnt
-      { $expr = new CreateIndexStatement($idxName.text, $cf.text, columnName); }
-    ;
-/**
- * DROP INDEX ON <CF>.<COLUMN_OR_INDEX_NAME>
- * DROP INDEX <INDEX_NAME>
- */
-dropIndexStatement returns [DropIndexStatement expr]
-    :
-      K_DROP K_INDEX index=( IDENT | STRING_LITERAL | INTEGER ) endStmnt
-      { $expr = new DropIndexStatement($index.text); }
-    ;
-
-/** DROP KEYSPACE <KSP>; */
-dropKeyspaceStatement returns [String ksp]
-    : K_DROP K_KEYSPACE name=( IDENT | STRING_LITERAL | INTEGER ) endStmnt { $ksp = $name.text; }
-    ;
-
-
-alterTableStatement returns [AlterTableStatement expr]
-    :
-    {
-        OperationType type = null;
-        String columnFamily = null, columnName = null, validator = null;
-        Map<String, String> propertyMap = null;
-    }
-    K_ALTER K_COLUMNFAMILY name=( IDENT | STRING_LITERAL | INTEGER ) { columnFamily = $name.text; }
-          ( K_ALTER { type = OperationType.ALTER; }
-               (col=( IDENT | STRING_LITERAL | INTEGER ) { columnName = $col.text; })
-               K_TYPE alterValidator=comparatorType { validator = $alterValidator.text; }
-          | K_ADD { type = OperationType.ADD; }
-               (col=( IDENT | STRING_LITERAL | INTEGER ) { columnName = $col.text; })
-               addValidator=comparatorType { validator = $addValidator.text; }
-          | K_DROP { type = OperationType.DROP; }
-               (col=( IDENT | STRING_LITERAL | INTEGER ) { columnName = $col.text; })
-          | K_WITH { type = OperationType.OPTS; propertyMap = new HashMap<String, String>(); }
-               prop1=(COMPIDENT | IDENT) '=' arg1=createCfamKeywordArgument { propertyMap.put($prop1.text, $arg1.arg); }
-               ( K_AND propN=(COMPIDENT | IDENT) '=' argN=createCfamKeywordArgument { propertyMap.put($propN.text, $argN.arg); } )* )
-    endStmnt
-      {
-          $expr = new AlterTableStatement(columnFamily, type, columnName, validator, propertyMap);
-      }
-    ;
-
-/** DROP COLUMNFAMILY <CF>; */
-dropColumnFamilyStatement returns [String cfam]
-    : K_DROP K_COLUMNFAMILY name=( IDENT | STRING_LITERAL | INTEGER ) endStmnt { $cfam = $name.text; }
-    ;
-
-comparatorType
-    : 'blob' | 'ascii' | 'text' | 'varchar' | 'int' | 'varint' | 'bigint' | 'uuid' | 'counter' | 'boolean' | 'timestamp' | 'float' | 'double' | 'decimal'
-    ;
-
-term returns [Term item]
-    : (( t=K_KEY | t=STRING_LITERAL | t=INTEGER | t=UUID | t=IDENT | t=FLOAT ) { $item = new Term($t.text, $t.type); }
-       | t=QMARK { $item = new Term($t.text, $t.type, ++currentBindMarkerIdx); }
-      )
-    ;
-
-termList returns [List<Term> items]
-    : { $items = new ArrayList<Term>(); }
-      t1=term { $items.add(t1); } (',' tN=term { $items.add(tN); })*
-    ;
-
-// term = term
-termPair[Map<Term, Term> columns]
-    :   key=term '=' value=term { columns.put(key, value); }
-    ;
-
-intTerm returns [Term integer]
-    : t=INTEGER { $integer = new Term($t.text, $t.type); }
-    ;
-
-termPairWithOperation[Map<Term, Operation> columns]
-    : key=term '=' (value=term { columns.put(key, new Operation(value)); }
-		               | c=term ( '+'  v=term { columns.put(key, new Operation(c, org.apache.cassandra.cql.Operation.OperationType.PLUS, v)); }
-                            | op='-'? v=intTerm
-                                  { validateMinusSupplied(op, v, input);
-                                    if (op != null) v = new Term(-(Long.valueOf(v.getText())), v.getType());
-                                    columns.put(key, new Operation(c, org.apache.cassandra.cql.Operation.OperationType.MINUS, v)); } ))
-    ;
-
-// Note: ranges are inclusive so >= and >, and < and <= all have the same semantics.  
-relation returns [Relation rel]
-    : name=term type=('=' | '<' | '<=' | '>=' | '>') t=term
-      { return new Relation($name.item, $type.text, $t.item); }
-    ;
-
-// TRUNCATE <CF>;
-truncateStatement returns [Pair<String,String> cf]
-    : K_TRUNCATE (keyspace=( IDENT | STRING_LITERAL | INTEGER ) '.')? columnFamily=( IDENT | STRING_LITERAL | INTEGER ) { $cf = Pair.create($keyspace.text, $columnFamily.text); } endStmnt
-    ;
-
-endStmnt
-    : ';'?  EOF
-    ;
-
-
-// Case-insensitive keywords
-K_SELECT:      S E L E C T;
-K_FROM:        F R O M;
-K_WHERE:       W H E R E;
-K_AND:         A N D;
-K_KEY:         K E Y;
-K_INSERT:      I N S E R T;
-K_UPDATE:      U P D A T E;
-K_WITH:        W I T H;
-K_LIMIT:       L I M I T;
-K_USING:       U S I N G;
-K_CONSISTENCY: C O N S I S T E N C Y;
-K_LEVEL:       ( O N E 
-               | Q U O R U M 
-               | A L L
-               | A N Y
-               | L O C A L '_' Q U O R U M
-               | E A C H '_' Q U O R U M
-               | T W O
-               | T H R E E
-               )
-               ;
-K_USE:         U S E;
-K_FIRST:       F I R S T;
-K_REVERSED:    R E V E R S E D;
-K_COUNT:       C O U N T;
-K_SET:         S E T;
-K_BEGIN:       B E G I N;
-K_APPLY:       A P P L Y;
-K_BATCH:       B A T C H;
-K_TRUNCATE:    T R U N C A T E;
-K_DELETE:      D E L E T E;
-K_IN:          I N;
-K_CREATE:      C R E A T E;
-K_KEYSPACE:    ( K E Y S P A C E
-                 | S C H E M A );
-K_COLUMNFAMILY:( C O L U M N F A M I L Y
-                 | T A B L E );
-K_INDEX:       I N D E X;
-K_ON:          O N;
-K_DROP:        D R O P;
-K_PRIMARY:     P R I M A R Y;
-K_INTO:        I N T O;
-K_VALUES:      V A L U E S;
-K_TIMESTAMP:   T I M E S T A M P;
-K_TTL:         T T L;
-K_ALTER:       A L T E R;
-K_ADD:         A D D;
-K_TYPE:        T Y P E;
-
-// Case-insensitive alpha characters
-fragment A: ('a'|'A');
-fragment B: ('b'|'B');
-fragment C: ('c'|'C');
-fragment D: ('d'|'D');
-fragment E: ('e'|'E');
-fragment F: ('f'|'F');
-fragment G: ('g'|'G');
-fragment H: ('h'|'H');
-fragment I: ('i'|'I');
-fragment J: ('j'|'J');
-fragment K: ('k'|'K');
-fragment L: ('l'|'L');
-fragment M: ('m'|'M');
-fragment N: ('n'|'N');
-fragment O: ('o'|'O');
-fragment P: ('p'|'P');
-fragment Q: ('q'|'Q');
-fragment R: ('r'|'R');
-fragment S: ('s'|'S');
-fragment T: ('t'|'T');
-fragment U: ('u'|'U');
-fragment V: ('v'|'V');
-fragment W: ('w'|'W');
-fragment X: ('x'|'X');
-fragment Y: ('y'|'Y');
-fragment Z: ('z'|'Z');
-    
-STRING_LITERAL
-    : '\''
-      { StringBuilder b = new StringBuilder(); }
-      ( c=~('\'') { b.appendCodePoint(c);}
-      | '\'' '\''            { b.appendCodePoint('\'');}
-      )*
-      '\''
-      { setText(b.toString()); }
-    ;
-
-fragment DIGIT
-    : '0'..'9'
-    ;
-
-fragment LETTER
-    : ('A'..'Z' | 'a'..'z')
-    ;
-    
-fragment HEX
-    : ('A'..'F' | 'a'..'f' | '0'..'9')
-    ;
-
-RANGEOP
-    : '..'
-    ;
-
-INTEGER
-    : '-'? DIGIT+
-    ;
-    
-QMARK
-    : '?'
-    ;
-
-
-/* Normally a lexer only emits one token at a time, but ours is tricked out
- * to support multiple (see @lexer::members near the top of the grammar).
- */
-FLOAT
-    : d=INTEGER r=RANGEOP
-      {
-          $d.setType(INTEGER);
-          emit($d);
-          $r.setType(RANGEOP);
-          emit($r);
-      }
-      | INTEGER '.' INTEGER
-    ;
-
-IDENT
-    : LETTER (LETTER | DIGIT | '_')*
-    ;
-    
-COMPIDENT
-    : IDENT ( ':' (IDENT | INTEGER))*
-    ;
-   
-UUID
-    : HEX HEX HEX HEX HEX HEX HEX HEX '-'
-      HEX HEX HEX HEX '-'
-      HEX HEX HEX HEX '-'
-      HEX HEX HEX HEX '-'
-      HEX HEX HEX HEX HEX HEX HEX HEX HEX HEX HEX HEX
-    ;
-
-WS
-    : (' ' | '\t' | '\n' | '\r')+ { $channel = HIDDEN; }
-    ;
-
-COMMENT
-    : ('--' | '//') .* ('\n'|'\r') { $channel = HIDDEN; }
-    ;
-    
-MULTILINE_COMMENT
-    : '/*' .* '*/' { $channel = HIDDEN; }
-    ;
diff --git a/src/java/org/apache/cassandra/cql/CreateColumnFamilyStatement.java b/src/java/org/apache/cassandra/cql/CreateColumnFamilyStatement.java
deleted file mode 100644
index d3bafe6..0000000
--- a/src/java/org/apache/cassandra/cql/CreateColumnFamilyStatement.java
+++ /dev/null
@@ -1,248 +0,0 @@
-/*
- * 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.cql;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.cassandra.cache.CachingOptions;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.db.composites.CellNames;
-import org.apache.cassandra.db.ColumnFamilyType;
-import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.db.marshal.TypeParser;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.SyntaxException;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.io.compress.CompressionParameters;
-
-/** A <code>CREATE COLUMNFAMILY</code> parsed from a CQL query statement. */
-public class CreateColumnFamilyStatement
-{
-    private final String name;
-    private final Map<Term, String> columns = new HashMap<Term, String>();
-    private final List<String> keyValidator = new ArrayList<String>();
-    private ByteBuffer keyAlias = null;
-    private final CFPropDefs cfProps = new CFPropDefs();
-
-    public CreateColumnFamilyStatement(String name)
-    {
-        this.name = name;
-    }
-
-    /** Perform validation of parsed params */
-    private void validate(List<ByteBuffer> variables) throws InvalidRequestException
-    {
-        // Ensure that exactly one key has been specified.
-        if (keyValidator.size() < 1)
-            throw new InvalidRequestException("You must specify a PRIMARY KEY");
-        else if (keyValidator.size() > 1)
-            throw new InvalidRequestException("You may only specify one PRIMARY KEY");
-
-        AbstractType<?> comparator;
-
-        try
-        {
-            cfProps.validate();
-            comparator = cfProps.getComparator();
-        }
-        catch (ConfigurationException e)
-        {
-            throw new InvalidRequestException(e.toString());
-        }
-        catch (SyntaxException e)
-        {
-            throw new InvalidRequestException(e.toString());
-        }
-
-        for (Map.Entry<Term, String> column : columns.entrySet())
-        {
-            ByteBuffer name = column.getKey().getByteBuffer(comparator, variables);
-
-            if (keyAlias != null && keyAlias.equals(name))
-                throw new InvalidRequestException("Invalid column name: "
-                                                  + column.getKey().getText()
-                                                  + ", because it equals to the key_alias.");
-
-        }
-    }
-
-    /** Map a column name to a validator for its value */
-    public void addColumn(Term term, String comparator)
-    {
-        columns.put(term, comparator);
-    }
-
-    public void setKeyType(String validator)
-    {
-        keyValidator.add(validator);
-    }
-
-    public String getKeyType()
-    {
-        return keyValidator.get(0);
-    }
-
-    public void setKeyAlias(String alias)
-    {
-        // if we got KEY in input we don't need to set an alias
-        if (!alias.toUpperCase().equals("KEY"))
-            keyAlias = ByteBufferUtil.bytes(alias);
-    }
-
-    /** Map a keyword to the corresponding value */
-    public void addProperty(String name, String value)
-    {
-        cfProps.addProperty(name, value);
-    }
-
-    /** Name of the column family to create */
-    public String getName()
-    {
-        return name;
-    }
-
-    // Column definitions
-    private List<ColumnDefinition> getColumns(CFMetaData cfm) throws InvalidRequestException
-    {
-        List<ColumnDefinition> columnDefs = new ArrayList<>(columns.size());
-
-        for (Map.Entry<Term, String> col : columns.entrySet())
-        {
-            try
-            {
-                ByteBuffer columnName = cfm.comparator.asAbstractType().fromStringCQL2(col.getKey().getText());
-                String validatorClassName = CFPropDefs.comparators.containsKey(col.getValue())
-                                          ? CFPropDefs.comparators.get(col.getValue())
-                                          : col.getValue();
-                AbstractType<?> validator = TypeParser.parse(validatorClassName);
-                columnDefs.add(ColumnDefinition.regularDef(cfm, columnName, validator, null));
-            }
-            catch (ConfigurationException e)
-            {
-                InvalidRequestException ex = new InvalidRequestException(e.toString());
-                ex.initCause(e);
-                throw ex;
-            }
-            catch (SyntaxException e)
-            {
-                InvalidRequestException ex = new InvalidRequestException(e.toString());
-                ex.initCause(e);
-                throw ex;
-            }
-        }
-
-        return columnDefs;
-    }
-
-    /**
-     * Returns a CFMetaData instance based on the parameters parsed from this
-     * <code>CREATE</code> statement, or defaults where applicable.
-     *
-     * @param keyspace keyspace to apply this column family to
-     * @return a CFMetaData instance corresponding to the values parsed from this statement
-     * @throws InvalidRequestException on failure to validate parsed parameters
-     */
-    public CFMetaData getCFMetaData(String keyspace, List<ByteBuffer> variables) throws InvalidRequestException
-    {
-        validate(variables);
-
-        try
-        {
-            boolean isDense = columns.isEmpty();
-            CFMetaData newCFMD = new CFMetaData(keyspace,
-                                                name,
-                                                ColumnFamilyType.Standard,
-                                                CellNames.fromAbstractType(cfProps.getComparator(), isDense));
-
-            if (CFMetaData.DEFAULT_COMPRESSOR != null && cfProps.compressionParameters.isEmpty())
-                cfProps.compressionParameters.put(CompressionParameters.SSTABLE_COMPRESSION, CFMetaData.DEFAULT_COMPRESSOR);
-            int maxCompactionThreshold = getPropertyInt(CFPropDefs.KW_MAXCOMPACTIONTHRESHOLD, CFMetaData.DEFAULT_MAX_COMPACTION_THRESHOLD);
-            int minCompactionThreshold = getPropertyInt(CFPropDefs.KW_MINCOMPACTIONTHRESHOLD, CFMetaData.DEFAULT_MIN_COMPACTION_THRESHOLD);
-            if (minCompactionThreshold <= 0 || maxCompactionThreshold <= 0)
-                throw new ConfigurationException("Disabling compaction by setting compaction thresholds to 0 has been deprecated, set the compaction option 'enabled' to false instead.");
-
-            newCFMD.isDense(isDense)
-                   .addAllColumnDefinitions(getColumns(newCFMD))
-                   .comment(cfProps.getProperty(CFPropDefs.KW_COMMENT))
-                   .readRepairChance(getPropertyDouble(CFPropDefs.KW_READREPAIRCHANCE, CFMetaData.DEFAULT_READ_REPAIR_CHANCE))
-                   .dcLocalReadRepairChance(getPropertyDouble(CFPropDefs.KW_DCLOCALREADREPAIRCHANCE, CFMetaData.DEFAULT_DCLOCAL_READ_REPAIR_CHANCE))
-                   .gcGraceSeconds(getPropertyInt(CFPropDefs.KW_GCGRACESECONDS, CFMetaData.DEFAULT_GC_GRACE_SECONDS))
-                   .defaultValidator(cfProps.getValidator())
-                   .minCompactionThreshold(minCompactionThreshold)
-                   .maxCompactionThreshold(maxCompactionThreshold)
-                   .keyValidator(TypeParser.parse(CFPropDefs.comparators.get(getKeyType())))
-                   .compactionStrategyClass(cfProps.compactionStrategyClass)
-                   .compactionStrategyOptions(cfProps.compactionStrategyOptions)
-                   .compressionParameters(CompressionParameters.create(cfProps.compressionParameters))
-                   .caching(CachingOptions.fromString(getPropertyString(CFPropDefs.KW_CACHING, CFMetaData.DEFAULT_CACHING_STRATEGY.toString())))
-                   .speculativeRetry(CFMetaData.SpeculativeRetry.fromString(getPropertyString(CFPropDefs.KW_SPECULATIVE_RETRY, CFMetaData.DEFAULT_SPECULATIVE_RETRY.toString())))
-                   .bloomFilterFpChance(getPropertyDouble(CFPropDefs.KW_BF_FP_CHANCE, null))
-                   .memtableFlushPeriod(getPropertyInt(CFPropDefs.KW_MEMTABLE_FLUSH_PERIOD, 0))
-                   .defaultTimeToLive(getPropertyInt(CFPropDefs.KW_DEFAULT_TIME_TO_LIVE, CFMetaData.DEFAULT_DEFAULT_TIME_TO_LIVE));
-
-            // CQL2 can have null keyAliases
-            if (keyAlias != null)
-                newCFMD.addColumnDefinition(ColumnDefinition.partitionKeyDef(newCFMD, keyAlias, newCFMD.getKeyValidator(), null));
-
-            return newCFMD.rebuild();
-        }
-        catch (ConfigurationException | SyntaxException e)
-        {
-            throw new InvalidRequestException(e.toString());
-        }
-    }
-
-    private String getPropertyString(String key, String defaultValue)
-    {
-        return cfProps.getPropertyString(key, defaultValue);
-    }
-
-    private Boolean getPropertyBoolean(String key, Boolean defaultValue)
-    {
-        return cfProps.getPropertyBoolean(key, defaultValue);
-    }
-
-    private Double getPropertyDouble(String key, Double defaultValue) throws InvalidRequestException
-    {
-        return cfProps.getPropertyDouble(key, defaultValue);
-    }
-
-    private Integer getPropertyInt(String key, Integer defaultValue) throws InvalidRequestException
-    {
-        return cfProps.getPropertyInt(key, defaultValue);
-    }
-
-    private Set<String> getPropertySet(String key, Set<String> defaultValue)
-    {
-        return cfProps.getPropertySet(key, defaultValue);
-    }
-
-    public Map<Term, String> getColumns()
-    {
-        return columns;
-    }
-
-}
-
diff --git a/src/java/org/apache/cassandra/cql/CreateIndexStatement.java b/src/java/org/apache/cassandra/cql/CreateIndexStatement.java
deleted file mode 100644
index 54b5eef..0000000
--- a/src/java/org/apache/cassandra/cql/CreateIndexStatement.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.cql;
-
-/** A <code>CREATE INDEX</code> statement parsed from a CQL query. */
-public class CreateIndexStatement
-{
-    private final String columnFamily;
-    private final String indexName;
-    private final Term columnName;
-
-    public CreateIndexStatement(String indexName, String columnFamily, Term columnName)
-    {
-        this.indexName = indexName;
-        this.columnFamily = columnFamily;
-        this.columnName = columnName;
-    }
-
-    /** Column family namespace. */
-    public String getColumnFamily()
-    {
-        return columnFamily;
-    }
-
-    /** Column name to index. */
-    public Term getColumnName()
-    {
-        return columnName;
-    }
-
-    /** Index name (or null). */
-    public String getIndexName()
-    {
-        return indexName;
-    }
-}
diff --git a/src/java/org/apache/cassandra/cql/CreateKeyspaceStatement.java b/src/java/org/apache/cassandra/cql/CreateKeyspaceStatement.java
deleted file mode 100644
index 8c2aa1b..0000000
--- a/src/java/org/apache/cassandra/cql/CreateKeyspaceStatement.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * 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.cql;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.cassandra.exceptions.InvalidRequestException;
-
-/** A <code>CREATE KEYSPACE</code> statement parsed from a CQL query. */
-public class CreateKeyspaceStatement
-{
-    private final String name;
-    private final Map<String, String> attrs;
-    private String strategyClass;
-    private final Map<String, String> strategyOptions = new HashMap<String, String>();
-
-    /**
-     * Creates a new <code>CreateKeyspaceStatement</code> instance for a given
-     * keyspace name and keyword arguments.
-     *
-     * @param name the name of the keyspace to create
-     * @param attrs map of the raw keyword arguments that followed the <code>WITH</code> keyword.
-     */
-    public CreateKeyspaceStatement(String name, Map<String, String> attrs)
-    {
-        this.name = name;
-        this.attrs = attrs;
-    }
-
-    /**
-     * The <code>CqlParser</code> only goes as far as extracting the keyword arguments
-     * from these statements, so this method is responsible for processing and
-     * validating, and must be called prior to access.
-     *
-     * @throws InvalidRequestException if arguments are missing or unacceptable
-     */
-    public void validate() throws InvalidRequestException
-    {
-        // required
-        if (!attrs.containsKey("strategy_class"))
-            throw new InvalidRequestException("missing required argument \"strategy_class\"");
-        strategyClass = attrs.get("strategy_class");
-
-        // optional
-        for (String key : attrs.keySet())
-            if ((key.contains(":")) && (key.startsWith("strategy_options")))
-                strategyOptions.put(key.split(":")[1], attrs.get(key));
-    }
-
-    public String getName()
-    {
-        return name;
-    }
-
-    public String getStrategyClass()
-    {
-        return strategyClass;
-    }
-
-    public Map<String, String> getStrategyOptions()
-    {
-        return strategyOptions;
-    }
-}
diff --git a/src/java/org/apache/cassandra/cql/DeleteStatement.java b/src/java/org/apache/cassandra/cql/DeleteStatement.java
deleted file mode 100644
index 71942e4..0000000
--- a/src/java/org/apache/cassandra/cql/DeleteStatement.java
+++ /dev/null
@@ -1,123 +0,0 @@
-/*
- * 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.cql;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.db.Mutation;
-import org.apache.cassandra.db.composites.CellName;
-import org.apache.cassandra.db.IMutation;
-import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.UnauthorizedException;
-import org.apache.cassandra.thrift.ThriftClientState;
-
-import static org.apache.cassandra.thrift.ThriftValidation.validateColumnFamily;
-import static org.apache.cassandra.cql.QueryProcessor.validateColumnName;
-
-/**
- * A <code>DELETE</code> parsed from a CQL query statement.
- *
- */
-public class DeleteStatement extends AbstractModification
-{
-    private List<Term> columns;
-    private List<Term> keys;
-
-    public DeleteStatement(List<Term> columns, String keyspace, String columnFamily, String keyName, List<Term> keys, Attributes attrs)
-    {
-        super(keyspace, columnFamily, keyName, attrs);
-
-        this.columns = columns;
-        this.keys = keys;
-    }
-
-    public List<Term> getColumns()
-    {
-        return columns;
-    }
-
-    public List<Term> getKeys()
-    {
-        return keys;
-    }
-
-    public List<IMutation> prepareRowMutations(String keyspace, ThriftClientState clientState, List<ByteBuffer> variables)
-    throws InvalidRequestException, UnauthorizedException
-    {
-        return prepareRowMutations(keyspace, clientState, null, variables);
-    }
-
-    public List<IMutation> prepareRowMutations(String keyspace, ThriftClientState clientState, Long timestamp, List<ByteBuffer> variables)
-    throws InvalidRequestException, UnauthorizedException
-    {
-        CFMetaData metadata = validateColumnFamily(keyspace, columnFamily);
-
-        clientState.hasColumnFamilyAccess(keyspace, columnFamily, Permission.MODIFY);
-        AbstractType<?> keyType = Schema.instance.getCFMetaData(keyspace, columnFamily).getKeyValidator();
-
-        List<IMutation> mutations = new ArrayList<IMutation>(keys.size());
-
-        for (Term key : keys)
-            mutations.add(mutationForKey(key.getByteBuffer(keyType, variables), keyspace, timestamp, clientState, variables, metadata));
-
-        return mutations;
-    }
-
-    public Mutation mutationForKey(ByteBuffer key, String keyspace, Long timestamp, ThriftClientState clientState, List<ByteBuffer> variables, CFMetaData metadata)
-    throws InvalidRequestException
-    {
-        Mutation mutation = new Mutation(keyspace, key);
-
-        QueryProcessor.validateKeyAlias(metadata, keyName);
-
-        if (columns.size() < 1)
-        {
-            // No columns, delete the partition
-            mutation.delete(columnFamily, (timestamp == null) ? getTimestamp(clientState) : timestamp);
-        }
-        else
-        {
-            // Delete specific columns
-            AbstractType<?> at = metadata.comparator.asAbstractType();
-            for (Term column : columns)
-            {
-                CellName columnName = metadata.comparator.cellFromByteBuffer(column.getByteBuffer(at, variables));
-                validateColumnName(columnName);
-                mutation.delete(columnFamily, columnName, (timestamp == null) ? getTimestamp(clientState) : timestamp);
-            }
-        }
-
-        return mutation;
-    }
-
-    public String toString()
-    {
-        return String.format("DeleteStatement(columns=%s, keyspace=%s, columnFamily=%s, consistency=%s keys=%s)",
-                             columns,
-                             keyspace,
-                             columnFamily,
-                             cLevel,
-                             keys);
-    }
-}
diff --git a/src/java/org/apache/cassandra/cql/DropIndexStatement.java b/src/java/org/apache/cassandra/cql/DropIndexStatement.java
deleted file mode 100644
index 1a24b7e..0000000
--- a/src/java/org/apache/cassandra/cql/DropIndexStatement.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * 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.cql;
-
-import org.apache.cassandra.config.*;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-
-public class DropIndexStatement
-{
-    public final String indexName;
-    private String keyspace;
-
-    public DropIndexStatement(String indexName)
-    {
-        this.indexName = indexName;
-    }
-
-    public void setKeyspace(String keyspace)
-    {
-        this.keyspace = keyspace;
-    }
-
-    public String getColumnFamily() throws InvalidRequestException
-    {
-        return findIndexedCF().cfName;
-    }
-
-    public CFMetaData generateCFMetadataUpdate() throws InvalidRequestException
-    {
-        return updateCFMetadata(findIndexedCF());
-    }
-
-    private CFMetaData updateCFMetadata(CFMetaData cfm)
-    {
-        ColumnDefinition column = findIndexedColumn(cfm);
-        assert column != null;
-        CFMetaData cloned = cfm.copy();
-        ColumnDefinition toChange = cloned.getColumnDefinition(column.name);
-        assert toChange.getIndexName() != null && toChange.getIndexName().equals(indexName);
-        toChange.setIndexName(null);
-        toChange.setIndexType(null, null);
-        return cloned;
-    }
-
-    private CFMetaData findIndexedCF() throws InvalidRequestException
-    {
-        KSMetaData ksm = Schema.instance.getKSMetaData(keyspace);
-        for (CFMetaData cfm : ksm.cfMetaData().values())
-        {
-            if (findIndexedColumn(cfm) != null)
-                return cfm;
-        }
-        throw new InvalidRequestException("Index '" + indexName + "' could not be found in any of the column families of keyspace '" + keyspace + "'");
-    }
-
-    private ColumnDefinition findIndexedColumn(CFMetaData cfm)
-    {
-        for (ColumnDefinition column : cfm.regularColumns())
-        {
-            if (column.getIndexType() != null && column.getIndexName() != null && column.getIndexName().equals(indexName))
-                return column;
-        }
-        return null;
-    }
-}
diff --git a/src/java/org/apache/cassandra/cql/Operation.java b/src/java/org/apache/cassandra/cql/Operation.java
deleted file mode 100644
index 0f06433..0000000
--- a/src/java/org/apache/cassandra/cql/Operation.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * 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.cql;
-
-public class Operation
-{
-    public static enum OperationType
-    { PLUS, MINUS }
-
-    public final OperationType type;
-    public final Term a, b;
-
-    // unary operation
-    public Operation(Term a)
-    {
-        this.a = a;
-        type = null;
-        b = null;
-    }
-
-    // binary operation
-    public Operation(Term a, OperationType type, Term b)
-    {
-        this.a = a;
-        this.type = type;
-        this.b = b;
-    }
-
-    public boolean isUnary()
-    {
-        return type == null && b == null;
-    }
-
-    public String toString()
-    {
-        return (isUnary())
-                ? String.format("UnaryOperation(%s)", a)
-                : String.format("BinaryOperation(%s, %s, %s)", a, type, b);
-    }
-}
diff --git a/src/java/org/apache/cassandra/cql/QueryProcessor.java b/src/java/org/apache/cassandra/cql/QueryProcessor.java
deleted file mode 100644
index d947cdb..0000000
--- a/src/java/org/apache/cassandra/cql/QueryProcessor.java
+++ /dev/null
@@ -1,913 +0,0 @@
-/*
- * 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.cql;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.*;
-import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.TimeoutException;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.config.*;
-import org.apache.cassandra.cli.CliUtils;
-import org.apache.cassandra.cql.hooks.ExecutionContext;
-import org.apache.cassandra.cql.hooks.PostPreparationHook;
-import org.apache.cassandra.cql.hooks.PreExecutionHook;
-import org.apache.cassandra.cql.hooks.PreparationContext;
-import org.apache.cassandra.cql3.Operator;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.composites.CellName;
-import org.apache.cassandra.db.composites.CellNameType;
-import org.apache.cassandra.db.composites.Composite;
-import org.apache.cassandra.db.context.CounterContext;
-import org.apache.cassandra.db.filter.*;
-import org.apache.cassandra.db.index.SecondaryIndexManager;
-import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.db.marshal.AsciiType;
-import org.apache.cassandra.db.marshal.TypeParser;
-import org.apache.cassandra.dht.*;
-import org.apache.cassandra.exceptions.*;
-import org.apache.cassandra.serializers.MarshalException;
-import org.apache.cassandra.service.StorageProxy;
-import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.service.MigrationManager;
-import org.apache.cassandra.thrift.Column;
-import org.apache.cassandra.thrift.CqlMetadata;
-import org.apache.cassandra.thrift.CqlResult;
-import org.apache.cassandra.thrift.CqlResultType;
-import org.apache.cassandra.thrift.CqlRow;
-import org.apache.cassandra.thrift.CqlPreparedResult;
-import org.apache.cassandra.thrift.ThriftValidation;
-import org.apache.cassandra.thrift.ThriftClientState;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.Pair;
-import org.apache.cassandra.utils.SemanticVersion;
-import org.antlr.runtime.*;
-
-
-import static org.apache.cassandra.thrift.ThriftValidation.validateColumnFamily;
-
-public class QueryProcessor
-{
-    public static final SemanticVersion CQL_VERSION = new SemanticVersion("2.0.0");
-
-    private static final Logger logger = LoggerFactory.getLogger(QueryProcessor.class);
-
-    public static final String DEFAULT_KEY_NAME = CFMetaData.DEFAULT_KEY_ALIAS.toUpperCase();
-
-    private static final List<PreExecutionHook> preExecutionHooks = new CopyOnWriteArrayList<>();
-    private static final List<PostPreparationHook> postPreparationHooks = new CopyOnWriteArrayList<>();
-
-    public static void addPreExecutionHook(PreExecutionHook hook)
-    {
-        preExecutionHooks.add(hook);
-    }
-
-    public static void removePreExecutionHook(PreExecutionHook hook)
-    {
-        preExecutionHooks.remove(hook);
-    }
-
-    public static void addPostPreparationHook(PostPreparationHook hook)
-    {
-        postPreparationHooks.add(hook);
-    }
-
-    public static void removePostPreparationHook(PostPreparationHook hook)
-    {
-        postPreparationHooks.remove(hook);
-    }
-
-    private static List<org.apache.cassandra.db.Row> getSlice(CFMetaData metadata, SelectStatement select, List<ByteBuffer> variables, long now)
-    throws InvalidRequestException, ReadTimeoutException, UnavailableException, IsBootstrappingException
-    {
-        List<ReadCommand> commands = new ArrayList<ReadCommand>();
-
-        // ...of a list of column names
-        if (!select.isColumnRange())
-        {
-            SortedSet<CellName> columnNames = getColumnNames(select, metadata, variables);
-            validateColumnNames(columnNames);
-
-            for (Term rawKey: select.getKeys())
-            {
-                ByteBuffer key = rawKey.getByteBuffer(metadata.getKeyValidator(),variables);
-
-                validateKey(key);
-                commands.add(new SliceByNamesReadCommand(metadata.ksName, key, select.getColumnFamily(), now, new NamesQueryFilter(columnNames)));
-            }
-        }
-        // ...a range (slice) of column names
-        else
-        {
-            AbstractType<?> at = metadata.comparator.asAbstractType();
-            Composite start = metadata.comparator.fromByteBuffer(select.getColumnStart().getByteBuffer(at,variables));
-            Composite finish = metadata.comparator.fromByteBuffer(select.getColumnFinish().getByteBuffer(at,variables));
-
-            for (Term rawKey : select.getKeys())
-            {
-                ByteBuffer key = rawKey.getByteBuffer(metadata.getKeyValidator(),variables);
-
-                validateKey(key);
-                validateSliceFilter(metadata, start, finish, select.isColumnsReversed());
-                commands.add(new SliceFromReadCommand(metadata.ksName,
-                                                      key,
-                                                      select.getColumnFamily(),
-                                                      now,
-                                                      new SliceQueryFilter(start, finish, select.isColumnsReversed(), select.getColumnsLimit())));
-            }
-        }
-
-        return StorageProxy.read(commands, select.getConsistencyLevel());
-    }
-
-    private static SortedSet<CellName> getColumnNames(SelectStatement select, CFMetaData metadata, List<ByteBuffer> variables)
-    throws InvalidRequestException
-    {
-        String keyString = metadata.getCQL2KeyName();
-        List<Term> selectColumnNames = select.getColumnNames();
-        SortedSet<CellName> columnNames = new TreeSet<>(metadata.comparator);
-        for (Term column : selectColumnNames)
-        {
-            // skip the key for the slice op; we'll add it to the resultset in extractThriftColumns
-            if (!column.getText().equalsIgnoreCase(keyString))
-                columnNames.add(metadata.comparator.cellFromByteBuffer(column.getByteBuffer(metadata.comparator.asAbstractType(),variables)));
-        }
-        return columnNames;
-    }
-
-    private static List<org.apache.cassandra.db.Row> multiRangeSlice(CFMetaData metadata, SelectStatement select, List<ByteBuffer> variables, long now)
-    throws ReadTimeoutException, UnavailableException, InvalidRequestException
-    {
-        IPartitioner p = StorageService.getPartitioner();
-
-        AbstractType<?> keyType = Schema.instance.getCFMetaData(metadata.ksName, select.getColumnFamily()).getKeyValidator();
-
-        ByteBuffer startKeyBytes = (select.getKeyStart() != null)
-                                   ? select.getKeyStart().getByteBuffer(keyType,variables)
-                                   : null;
-
-        ByteBuffer finishKeyBytes = (select.getKeyFinish() != null)
-                                    ? select.getKeyFinish().getByteBuffer(keyType,variables)
-                                    : null;
-
-        RowPosition startKey = RowPosition.ForKey.get(startKeyBytes, p), finishKey = RowPosition.ForKey.get(finishKeyBytes, p);
-        if (startKey.compareTo(finishKey) > 0 && !finishKey.isMinimum(p))
-        {
-            if (p instanceof RandomPartitioner)
-                throw new InvalidRequestException("Start key sorts after end key. This is not allowed; you probably should not specify end key at all, under RandomPartitioner");
-            else
-                throw new InvalidRequestException("Start key must sort before (or equal to) finish key in your partitioner!");
-        }
-        AbstractBounds<RowPosition> bounds = new Bounds<RowPosition>(startKey, finishKey);
-
-        IDiskAtomFilter columnFilter = filterFromSelect(select, metadata, variables);
-        validateFilter(metadata, columnFilter);
-
-        List<Relation> columnRelations = select.getColumnRelations();
-        List<IndexExpression> expressions = new ArrayList<IndexExpression>(columnRelations.size());
-        for (Relation columnRelation : columnRelations)
-        {
-            // Left and right side of relational expression encoded according to comparator/validator.
-            ByteBuffer entity = columnRelation.getEntity().getByteBuffer(metadata.comparator.asAbstractType(), variables);
-            ByteBuffer value = columnRelation.getValue().getByteBuffer(metadata.getValueValidator(metadata.comparator.cellFromByteBuffer(entity)), variables);
-
-            expressions.add(new IndexExpression(entity,
-                                                Operator.valueOf(columnRelation.operator().name()),
-                                                value));
-        }
-
-        int limit = select.isKeyRange() && select.getKeyStart() != null
-                  ? select.getNumRecords() + 1
-                  : select.getNumRecords();
-
-        List<org.apache.cassandra.db.Row> rows = StorageProxy.getRangeSlice(new RangeSliceCommand(metadata.ksName,
-                                                                                                  select.getColumnFamily(),
-                                                                                                  now,
-                                                                                                  columnFilter,
-                                                                                                  bounds,
-                                                                                                  expressions,
-                                                                                                  limit),
-                                                                            select.getConsistencyLevel());
-
-        // if start key was set and relation was "greater than"
-        if (select.getKeyStart() != null && !select.includeStartKey() && !rows.isEmpty())
-        {
-            if (rows.get(0).key.getKey().equals(startKeyBytes))
-                rows.remove(0);
-        }
-
-        // if finish key was set and relation was "less than"
-        if (select.getKeyFinish() != null && !select.includeFinishKey() && !rows.isEmpty())
-        {
-            int lastIndex = rows.size() - 1;
-            if (rows.get(lastIndex).key.getKey().equals(finishKeyBytes))
-                rows.remove(lastIndex);
-        }
-
-        return rows.subList(0, select.getNumRecords() < rows.size() ? select.getNumRecords() : rows.size());
-    }
-
-    private static IDiskAtomFilter filterFromSelect(SelectStatement select, CFMetaData metadata, List<ByteBuffer> variables)
-    throws InvalidRequestException
-    {
-        if (select.isColumnRange() || select.getColumnNames().size() == 0)
-        {
-            AbstractType<?> comparator = metadata.comparator.asAbstractType();
-            return new SliceQueryFilter(metadata.comparator.fromByteBuffer(select.getColumnStart().getByteBuffer(comparator, variables)),
-                                        metadata.comparator.fromByteBuffer(select.getColumnFinish().getByteBuffer(comparator, variables)),
-                                        select.isColumnsReversed(),
-                                        select.getColumnsLimit());
-        }
-        else
-        {
-            return new NamesQueryFilter(getColumnNames(select, metadata, variables));
-        }
-    }
-
-    /* Test for SELECT-specific taboos */
-    private static void validateSelect(String keyspace, SelectStatement select, List<ByteBuffer> variables) throws InvalidRequestException
-    {
-        select.getConsistencyLevel().validateForRead(keyspace);
-
-        // Finish key w/o start key (KEY < foo)
-        if (!select.isKeyRange() && (select.getKeyFinish() != null))
-            throw new InvalidRequestException("Key range clauses must include a start key (i.e. KEY > term)");
-
-        // Key range and by-key(s) combined (KEY > foo AND KEY = bar)
-        if (select.isKeyRange() && select.getKeys().size() > 0)
-            throw new InvalidRequestException("You cannot combine key range and by-key clauses in a SELECT");
-
-        // Start and finish keys, *and* column relations (KEY > foo AND KEY < bar and name1 = value1).
-        if (select.isKeyRange() && (select.getKeyFinish() != null) && (select.getColumnRelations().size() > 0))
-            throw new InvalidRequestException("You cannot combine key range and by-column clauses in a SELECT");
-
-        // Can't use more than one KEY =
-        if (!select.isMultiKey() && select.getKeys().size() > 1)
-            throw new InvalidRequestException("You cannot use more than one KEY = in a SELECT");
-
-        if (select.getColumnRelations().size() > 0)
-        {
-            ColumnFamilyStore cfstore = Keyspace.open(keyspace).getColumnFamilyStore(select.getColumnFamily());
-            CellNameType comparator = cfstore.metadata.comparator;
-            AbstractType<?> at = comparator.asAbstractType();
-            SecondaryIndexManager idxManager = cfstore.indexManager;
-            for (Relation relation : select.getColumnRelations())
-            {
-                ByteBuffer name = relation.getEntity().getByteBuffer(at, variables);
-                if ((relation.operator() == RelationType.EQ) && idxManager.indexes(comparator.cellFromByteBuffer(name)))
-                    return;
-            }
-            throw new InvalidRequestException("No indexed columns present in by-columns clause with \"equals\" operator");
-        }
-    }
-
-    public static void validateKey(ByteBuffer key) throws InvalidRequestException
-    {
-        if (key == null || key.remaining() == 0)
-        {
-            throw new InvalidRequestException("Key may not be empty");
-        }
-
-        // check that key can be handled by FBUtilities.writeShortByteArray
-        if (key.remaining() > FBUtilities.MAX_UNSIGNED_SHORT)
-        {
-            throw new InvalidRequestException("Key length of " + key.remaining() +
-                                              " is longer than maximum of " + FBUtilities.MAX_UNSIGNED_SHORT);
-        }
-    }
-
-    public static void validateKeyAlias(CFMetaData cfm, String key) throws InvalidRequestException
-    {
-        assert key.toUpperCase().equals(key); // should always be uppercased by caller
-        String realKeyAlias = cfm.getCQL2KeyName().toUpperCase();
-        if (!realKeyAlias.equals(key))
-            throw new InvalidRequestException(String.format("Expected key '%s' to be present in WHERE clause for '%s'", realKeyAlias, cfm.cfName));
-    }
-
-    private static void validateColumnNames(Iterable<CellName> columns)
-    throws InvalidRequestException
-    {
-        for (CellName name : columns)
-        {
-            if (name.dataSize() > org.apache.cassandra.db.Cell.MAX_NAME_LENGTH)
-                throw new InvalidRequestException(String.format("column name is too long (%s > %s)",
-                                                                name.dataSize(),
-                                                                org.apache.cassandra.db.Cell.MAX_NAME_LENGTH));
-            if (name.isEmpty())
-                throw new InvalidRequestException("zero-length column name");
-        }
-    }
-
-    public static void validateColumnName(CellName column)
-    throws InvalidRequestException
-    {
-        validateColumnNames(Arrays.asList(column));
-    }
-
-    public static void validateColumn(CFMetaData metadata, CellName name, ByteBuffer value)
-    throws InvalidRequestException
-    {
-        validateColumnName(name);
-        AbstractType<?> validator = metadata.getValueValidator(name);
-
-        try
-        {
-            if (validator != null)
-                validator.validate(value);
-        }
-        catch (MarshalException me)
-        {
-            throw new InvalidRequestException(String.format("Invalid column value for column (name=%s); %s",
-                                                            ByteBufferUtil.bytesToHex(name.toByteBuffer()),
-                                                            me.getMessage()));
-        }
-    }
-
-    private static void validateFilter(CFMetaData metadata, IDiskAtomFilter filter)
-    throws InvalidRequestException
-    {
-        if (filter instanceof SliceQueryFilter)
-            validateSliceFilter(metadata, (SliceQueryFilter)filter);
-        else
-            validateColumnNames(((NamesQueryFilter)filter).columns);
-    }
-
-    private static void validateSliceFilter(CFMetaData metadata, SliceQueryFilter range)
-    throws InvalidRequestException
-    {
-        validateSliceFilter(metadata, range.start(), range.finish(), range.reversed);
-    }
-
-    private static void validateSliceFilter(CFMetaData metadata, Composite start, Composite finish, boolean reversed)
-    throws InvalidRequestException
-    {
-        CellNameType comparator = metadata.comparator;
-        Comparator<Composite> orderedComparator = reversed ? comparator.reverseComparator(): comparator;
-        if (!start.isEmpty() && !finish.isEmpty() && orderedComparator.compare(start, finish) > 0)
-            throw new InvalidRequestException("range finish must come after start in traversal order");
-    }
-
-    public static CqlResult processStatement(CQLStatement statement, ExecutionContext context)
-    throws RequestExecutionException, RequestValidationException
-    {
-        String keyspace = null;
-        ThriftClientState clientState = context.clientState;
-        List<ByteBuffer> variables = context.variables;
-
-        // Some statements won't have (or don't need) a keyspace (think USE, or CREATE).
-        if (statement.type != StatementType.SELECT && StatementType.REQUIRES_KEYSPACE.contains(statement.type))
-            keyspace = clientState.getKeyspace();
-
-        CqlResult result = new CqlResult();
-
-        if (!preExecutionHooks.isEmpty())
-            for (PreExecutionHook hook : preExecutionHooks)
-                statement = hook.processStatement(statement, context);
-
-        if (logger.isDebugEnabled()) logger.debug("CQL statement type: {}", statement.type.toString());
-        CFMetaData metadata;
-        switch (statement.type)
-        {
-            case SELECT:
-                SelectStatement select = (SelectStatement)statement.statement;
-
-                final String oldKeyspace = clientState.getRawKeyspace();
-
-                if (select.isSetKeyspace())
-                {
-                    keyspace = CliUtils.unescapeSQLString(select.getKeyspace());
-                    ThriftValidation.validateKeyspace(keyspace);
-                }
-                else if (oldKeyspace == null)
-                    throw new InvalidRequestException("no keyspace has been specified");
-                else
-                    keyspace = oldKeyspace;
-
-                clientState.hasColumnFamilyAccess(keyspace, select.getColumnFamily(), Permission.SELECT);
-                metadata = validateColumnFamily(keyspace, select.getColumnFamily());
-
-                // need to do this in here because we need a CFMD.getKeyName()
-                select.extractKeyAliasFromColumns(metadata);
-
-                if (select.getKeys().size() > 0)
-                    validateKeyAlias(metadata, select.getKeyAlias());
-
-                validateSelect(keyspace, select, variables);
-
-                List<org.apache.cassandra.db.Row> rows;
-
-                long now = System.currentTimeMillis();
-                // By-key
-                if (!select.isKeyRange() && (select.getKeys().size() > 0))
-                {
-                    rows = getSlice(metadata, select, variables, now);
-                }
-                else
-                {
-                    rows = multiRangeSlice(metadata, select, variables, now);
-                }
-
-                // count resultset is a single column named "count"
-                result.type = CqlResultType.ROWS;
-                if (select.isCountOperation())
-                {
-                    validateCountOperation(select);
-
-                    ByteBuffer countBytes = ByteBufferUtil.bytes("count");
-                    result.schema = new CqlMetadata(Collections.<ByteBuffer, String>emptyMap(),
-                                                    Collections.<ByteBuffer, String>emptyMap(),
-                                                    "AsciiType",
-                                                    "LongType");
-                    List<Column> columns = Collections.singletonList(new Column(countBytes).setValue(ByteBufferUtil.bytes((long) rows.size())));
-                    result.rows = Collections.singletonList(new CqlRow(countBytes, columns));
-                    return result;
-                }
-
-                // otherwise create resultset from query results
-                result.schema = new CqlMetadata(new HashMap<ByteBuffer, String>(),
-                                                new HashMap<ByteBuffer, String>(),
-                                                TypeParser.getShortName(metadata.comparator.asAbstractType()),
-                                                TypeParser.getShortName(metadata.getDefaultValidator()));
-                List<CqlRow> cqlRows = new ArrayList<CqlRow>(rows.size());
-                for (org.apache.cassandra.db.Row row : rows)
-                {
-                    List<Column> thriftColumns = new ArrayList<Column>();
-                    if (select.isColumnRange())
-                    {
-                        if (select.isFullWildcard())
-                        {
-                            // prepend key
-                            ByteBuffer keyName = ByteBufferUtil.bytes(metadata.getCQL2KeyName());
-                            thriftColumns.add(new Column(keyName).setValue(row.key.getKey()).setTimestamp(-1));
-                            result.schema.name_types.put(keyName, TypeParser.getShortName(AsciiType.instance));
-                            result.schema.value_types.put(keyName, TypeParser.getShortName(metadata.getKeyValidator()));
-                        }
-
-                        // preserve comparator order
-                        if (row.cf != null)
-                        {
-                            for (org.apache.cassandra.db.Cell c : row.cf.getSortedColumns())
-                            {
-                                if (!c.isLive(now))
-                                    continue;
-
-                                ColumnDefinition cd = metadata.getColumnDefinition(c.name());
-                                if (cd != null)
-                                    result.schema.value_types.put(c.name().toByteBuffer(), TypeParser.getShortName(cd.type));
-
-                                thriftColumns.add(thriftify(c));
-                            }
-                        }
-                    }
-                    else
-                    {
-                        String keyString = metadata.getCQL2KeyName();
-
-                        // order columns in the order they were asked for
-                        for (Term term : select.getColumnNames())
-                        {
-                            if (term.getText().equalsIgnoreCase(keyString))
-                            {
-                                // preserve case of key as it was requested
-                                ByteBuffer requestedKey = ByteBufferUtil.bytes(term.getText());
-                                thriftColumns.add(new Column(requestedKey).setValue(row.key.getKey()).setTimestamp(-1));
-                                result.schema.name_types.put(requestedKey, TypeParser.getShortName(AsciiType.instance));
-                                result.schema.value_types.put(requestedKey, TypeParser.getShortName(metadata.getKeyValidator()));
-                                continue;
-                            }
-
-                            if (row.cf == null)
-                                continue;
-
-                            ByteBuffer nameBytes;
-                            try
-                            {
-                                nameBytes = term.getByteBuffer(metadata.comparator.asAbstractType(), variables);
-                            }
-                            catch (InvalidRequestException e)
-                            {
-                                throw new AssertionError(e);
-                            }
-
-                            CellName name = metadata.comparator.cellFromByteBuffer(nameBytes);
-                            ColumnDefinition cd = metadata.getColumnDefinition(name);
-                            if (cd != null)
-                                result.schema.value_types.put(nameBytes, TypeParser.getShortName(cd.type));
-                            org.apache.cassandra.db.Cell c = row.cf.getColumn(name);
-                            if (c == null || !c.isLive())
-                                thriftColumns.add(new Column().setName(nameBytes));
-                            else
-                                thriftColumns.add(thriftify(c));
-                        }
-                    }
-
-                    // Create a new row, add the columns to it, and then add it to the list of rows
-                    CqlRow cqlRow = new CqlRow();
-                    cqlRow.key = row.key.getKey();
-                    cqlRow.columns = thriftColumns;
-                    if (select.isColumnsReversed())
-                        Collections.reverse(cqlRow.columns);
-                    cqlRows.add(cqlRow);
-                }
-
-                result.rows = cqlRows;
-                return result;
-
-            case INSERT: // insert uses UpdateStatement
-            case UPDATE:
-                UpdateStatement update = (UpdateStatement)statement.statement;
-                update.getConsistencyLevel().validateForWrite(keyspace);
-
-                keyspace = update.keyspace == null ? clientState.getKeyspace() : update.keyspace;
-                // permission is checked in prepareRowMutations()
-                List<IMutation> rowMutations = update.prepareRowMutations(keyspace, clientState, variables);
-
-                for (IMutation mutation : rowMutations)
-                {
-                    validateKey(mutation.key());
-                }
-
-                StorageProxy.mutateWithTriggers(rowMutations, update.getConsistencyLevel(), false);
-
-                result.type = CqlResultType.VOID;
-                return result;
-
-            case BATCH:
-                BatchStatement batch = (BatchStatement) statement.statement;
-                batch.getConsistencyLevel().validateForWrite(keyspace);
-
-                if (batch.getTimeToLive() != 0)
-                    throw new InvalidRequestException("Global TTL on the BATCH statement is not supported.");
-
-                for (AbstractModification up : batch.getStatements())
-                {
-                    if (up.isSetConsistencyLevel())
-                        throw new InvalidRequestException(
-                                "Consistency level must be set on the BATCH, not individual statements");
-
-                    if (batch.isSetTimestamp() && up.isSetTimestamp())
-                        throw new InvalidRequestException(
-                                "Timestamp must be set either on BATCH or individual statements");
-                }
-
-                List<IMutation> mutations = batch.getMutations(keyspace, clientState, variables);
-                for (IMutation mutation : mutations)
-                {
-                    validateKey(mutation.key());
-                }
-
-                StorageProxy.mutateWithTriggers(mutations, batch.getConsistencyLevel(), false);
-
-                result.type = CqlResultType.VOID;
-                return result;
-
-            case USE:
-                clientState.validateLogin();
-                clientState.setKeyspace(CliUtils.unescapeSQLString((String) statement.statement));
-
-                result.type = CqlResultType.VOID;
-                return result;
-
-            case TRUNCATE:
-                Pair<String, String> columnFamily = (Pair<String, String>)statement.statement;
-                keyspace = columnFamily.left == null ? clientState.getKeyspace() : columnFamily.left;
-
-                validateColumnFamily(keyspace, columnFamily.right);
-                clientState.hasColumnFamilyAccess(keyspace, columnFamily.right, Permission.MODIFY);
-
-                try
-                {
-                    StorageProxy.truncateBlocking(keyspace, columnFamily.right);
-                }
-                catch (TimeoutException e)
-                {
-                    throw new TruncateException(e);
-                }
-                catch (IOException e)
-                {
-                    throw new RuntimeException(e);
-                }
-
-                result.type = CqlResultType.VOID;
-                return result;
-
-            case DELETE:
-                DeleteStatement delete = (DeleteStatement)statement.statement;
-
-                keyspace = delete.keyspace == null ? clientState.getKeyspace() : delete.keyspace;
-                // permission is checked in prepareRowMutations()
-                List<IMutation> deletions = delete.prepareRowMutations(keyspace, clientState, variables);
-                for (IMutation deletion : deletions)
-                {
-                    validateKey(deletion.key());
-                }
-
-                StorageProxy.mutateWithTriggers(deletions, delete.getConsistencyLevel(), false);
-
-                result.type = CqlResultType.VOID;
-                return result;
-
-            case CREATE_KEYSPACE:
-                CreateKeyspaceStatement create = (CreateKeyspaceStatement)statement.statement;
-                create.validate();
-                ThriftValidation.validateKeyspaceNotSystem(create.getName());
-                clientState.hasAllKeyspacesAccess(Permission.CREATE);
-
-                try
-                {
-                    KSMetaData ksm = KSMetaData.newKeyspace(create.getName(),
-                                                            create.getStrategyClass(),
-                                                            create.getStrategyOptions(),
-                                                            true);
-                    ThriftValidation.validateKeyspaceNotYetExisting(ksm.name);
-                    MigrationManager.announceNewKeyspace(ksm);
-                }
-                catch (ConfigurationException e)
-                {
-                    InvalidRequestException ex = new InvalidRequestException(e.getMessage());
-                    ex.initCause(e);
-                    throw ex;
-                }
-
-                result.type = CqlResultType.VOID;
-                return result;
-
-            case CREATE_COLUMNFAMILY:
-                CreateColumnFamilyStatement createCf = (CreateColumnFamilyStatement)statement.statement;
-                clientState.hasKeyspaceAccess(keyspace, Permission.CREATE);
-
-                try
-                {
-                    MigrationManager.announceNewColumnFamily(createCf.getCFMetaData(keyspace, variables));
-                }
-                catch (ConfigurationException e)
-                {
-                    InvalidRequestException ex = new InvalidRequestException(e.toString());
-                    ex.initCause(e);
-                    throw ex;
-                }
-
-                result.type = CqlResultType.VOID;
-                return result;
-
-            case CREATE_INDEX:
-                CreateIndexStatement createIdx = (CreateIndexStatement)statement.statement;
-                clientState.hasColumnFamilyAccess(keyspace, createIdx.getColumnFamily(), Permission.ALTER);
-                CFMetaData oldCfm = Schema.instance.getCFMetaData(keyspace, createIdx.getColumnFamily());
-                if (oldCfm == null)
-                    throw new InvalidRequestException("No such column family: " + createIdx.getColumnFamily());
-
-                boolean columnExists = false;
-                ByteBuffer columnName = createIdx.getColumnName().getByteBuffer();
-                // mutating oldCfm directly would be bad, but mutating a copy is fine.
-                CFMetaData cfm = oldCfm.copy();
-                for (ColumnDefinition cd : cfm.regularColumns())
-                {
-                    if (cd.name.bytes.equals(columnName))
-                    {
-                        if (cd.getIndexType() != null)
-                            throw new InvalidRequestException("Index already exists");
-                        if (logger.isDebugEnabled())
-                            logger.debug("Updating column {} definition for index {}", cfm.comparator.getString(cfm.comparator.fromByteBuffer(columnName)), createIdx.getIndexName());
-                        cd.setIndexType(IndexType.KEYS, Collections.<String, String>emptyMap());
-                        cd.setIndexName(createIdx.getIndexName());
-                        columnExists = true;
-                        break;
-                    }
-                }
-                if (!columnExists)
-                    throw new InvalidRequestException("No column definition found for column " + oldCfm.comparator.getString(cfm.comparator.fromByteBuffer(columnName)));
-
-                try
-                {
-                    cfm.addDefaultIndexNames();
-                    MigrationManager.announceColumnFamilyUpdate(cfm, true); // As far as metadata are concerned, CQL2 == thrift
-                }
-                catch (ConfigurationException e)
-                {
-                    InvalidRequestException ex = new InvalidRequestException(e.toString());
-                    ex.initCause(e);
-                    throw ex;
-                }
-
-                result.type = CqlResultType.VOID;
-                return result;
-
-            case DROP_INDEX:
-                DropIndexStatement dropIdx = (DropIndexStatement)statement.statement;
-                keyspace = clientState.getKeyspace();
-                dropIdx.setKeyspace(keyspace);
-                clientState.hasColumnFamilyAccess(keyspace, dropIdx.getColumnFamily(), Permission.ALTER);
-
-                try
-                {
-                    CFMetaData updatedCF = dropIdx.generateCFMetadataUpdate();
-                    MigrationManager.announceColumnFamilyUpdate(updatedCF, true); // As far as metadata are concerned, CQL2 == thrift
-                }
-                catch (ConfigurationException e)
-                {
-                    InvalidRequestException ex = new InvalidRequestException(e.toString());
-                    ex.initCause(e);
-                    throw ex;
-                }
-
-                result.type = CqlResultType.VOID;
-                return result;
-
-            case DROP_KEYSPACE:
-                String deleteKeyspace = (String)statement.statement;
-                ThriftValidation.validateKeyspaceNotSystem(deleteKeyspace);
-                clientState.hasKeyspaceAccess(deleteKeyspace, Permission.DROP);
-
-                try
-                {
-                    MigrationManager.announceKeyspaceDrop(deleteKeyspace);
-                }
-                catch (ConfigurationException e)
-                {
-                    InvalidRequestException ex = new InvalidRequestException(e.getMessage());
-                    ex.initCause(e);
-                    throw ex;
-                }
-
-                result.type = CqlResultType.VOID;
-                return result;
-
-            case DROP_COLUMNFAMILY:
-                String deleteColumnFamily = (String)statement.statement;
-                clientState.hasColumnFamilyAccess(keyspace, deleteColumnFamily, Permission.DROP);
-
-                try
-                {
-                    MigrationManager.announceColumnFamilyDrop(keyspace, deleteColumnFamily);
-                }
-                catch (ConfigurationException e)
-                {
-                    InvalidRequestException ex = new InvalidRequestException(e.getMessage());
-                    ex.initCause(e);
-                    throw ex;
-                }
-
-                result.type = CqlResultType.VOID;
-                return result;
-
-            case ALTER_TABLE:
-                AlterTableStatement alterTable = (AlterTableStatement) statement.statement;
-
-                validateColumnFamily(keyspace, alterTable.columnFamily);
-                clientState.hasColumnFamilyAccess(keyspace, alterTable.columnFamily, Permission.ALTER);
-
-                try
-                {
-                    MigrationManager.announceColumnFamilyUpdate(alterTable.getCFMetaData(keyspace), true); // As far as metadata are concerned, CQL2 == thrift
-                }
-                catch (ConfigurationException e)
-                {
-                    InvalidRequestException ex = new InvalidRequestException(e.getMessage());
-                    ex.initCause(e);
-                    throw ex;
-                }
-
-                result.type = CqlResultType.VOID;
-                return result;
-        }
-        return null;    // We should never get here.
-    }
-
-    public static CqlResult process(String queryString, ThriftClientState clientState)
-    throws RequestValidationException, RequestExecutionException
-    {
-        logger.trace("CQL QUERY: {}", queryString);
-        return processStatement(getStatement(queryString),
-                                new ExecutionContext(clientState, queryString, Collections.<ByteBuffer>emptyList()));
-    }
-
-    public static CqlPreparedResult prepare(String queryString, ThriftClientState clientState)
-    throws RequestValidationException
-    {
-        logger.trace("CQL QUERY: {}", queryString);
-
-        CQLStatement statement = getStatement(queryString);
-        int statementId = makeStatementId(queryString);
-        logger.trace("Discovered "+ statement.boundTerms + " bound variables.");
-
-        clientState.getPrepared().put(statementId, statement);
-        logger.trace(String.format("Stored prepared statement #%d with %d bind markers",
-                                   statementId,
-                                   statement.boundTerms));
-
-        if (!postPreparationHooks.isEmpty())
-        {
-            PreparationContext context = new PreparationContext(clientState, queryString, statement);
-            for (PostPreparationHook hook : postPreparationHooks)
-                hook.processStatement(statement, context);
-        }
-
-        return new CqlPreparedResult(statementId, statement.boundTerms);
-    }
-
-    public static CqlResult processPrepared(CQLStatement statement, ThriftClientState clientState, List<ByteBuffer> variables)
-    throws RequestValidationException, RequestExecutionException
-    {
-        // Check to see if there are any bound variables to verify
-        if (!(variables.isEmpty() && (statement.boundTerms == 0)))
-        {
-            if (variables.size() != statement.boundTerms)
-                throw new InvalidRequestException(String.format("there were %d markers(?) in CQL but %d bound variables",
-                                                                statement.boundTerms,
-                                                                variables.size()));
-
-            // at this point there is a match in count between markers and variables that is non-zero
-
-            if (logger.isTraceEnabled())
-                for (int i = 0; i < variables.size(); i++)
-                    logger.trace("[{}] '{}'", i+1, variables.get(i));
-        }
-
-        return processStatement(statement, new ExecutionContext(clientState, null, variables));
-    }
-
-    private static final int makeStatementId(String cql)
-    {
-        // use the hash of the string till something better is provided
-        return cql.hashCode();
-    }
-
-    private static Column thriftify(org.apache.cassandra.db.Cell c)
-    {
-        ByteBuffer value = (c instanceof CounterCell)
-                           ? ByteBufferUtil.bytes(CounterContext.instance().total(c.value()))
-                           : c.value();
-        return new Column(c.name().toByteBuffer()).setValue(value).setTimestamp(c.timestamp());
-    }
-
-    private static CQLStatement getStatement(String queryStr) throws SyntaxException
-    {
-        try
-        {
-            // Lexer and parser
-            CharStream stream = new ANTLRStringStream(queryStr);
-            CqlLexer lexer = new CqlLexer(stream);
-            TokenStream tokenStream = new CommonTokenStream(lexer);
-            CqlParser parser = new CqlParser(tokenStream);
-
-            // Parse the query string to a statement instance
-            CQLStatement statement = parser.query();
-
-            // The lexer and parser queue up any errors they may have encountered
-            // along the way, if necessary, we turn them into exceptions here.
-            lexer.throwLastRecognitionError();
-            parser.throwLastRecognitionError();
-
-            return statement;
-        }
-        catch (RuntimeException re)
-        {
-            SyntaxException ire = new SyntaxException("Failed parsing statement: [" + queryStr + "] reason: " + re.getClass().getSimpleName() + " " + re.getMessage());
-            throw ire;
-        }
-        catch (RecognitionException e)
-        {
-            SyntaxException ire = new SyntaxException("Invalid or malformed CQL query string: " + e.getMessage());
-            throw ire;
-        }
-    }
-
-    private static void validateCountOperation(SelectStatement select) throws InvalidRequestException
-    {
-        if (select.isWildcard())
-            return; // valid count(*)
-
-        if (!select.isColumnRange())
-        {
-            List<Term> columnNames = select.getColumnNames();
-            String firstColumn = columnNames.get(0).getText();
-
-            if (columnNames.size() == 1 && (firstColumn.equals("*") || firstColumn.equals("1")))
-                return; // valid count(*) || count(1)
-        }
-
-        throw new InvalidRequestException("Only COUNT(*) and COUNT(1) operations are currently supported.");
-    }
-}
diff --git a/src/java/org/apache/cassandra/cql/Relation.java b/src/java/org/apache/cassandra/cql/Relation.java
deleted file mode 100644
index 669c5f2..0000000
--- a/src/java/org/apache/cassandra/cql/Relation.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * 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.cql;
-
-/**
- * Relations encapsulate the relationship between an entity of some kind, and
- * a value (term). For example, <key> > "start" or "colname1" = "somevalue".
- *
- */
-public class Relation
-{
-    private final Term entity;
-    private final RelationType relationType;
-    private final Term value;
-
-    /**
-     * Creates a new relation.
-     *
-     * @param entity the kind of relation this is; what the term is being compared to.
-     * @param type the type that describes how this entity relates to the value.
-     * @param value the value being compared.
-     */
-    public Relation(Term entity, String type, Term value)
-    {
-        this.entity = entity;
-        this.relationType = RelationType.forString(type);
-        this.value = value;
-    }
-
-    public RelationType operator()
-    {
-        return relationType;
-    }
-
-    public Term getEntity()
-    {
-        return entity;
-    }
-
-    public Term getValue()
-    {
-        return value;
-    }
-
-    @Override
-    public String toString()
-    {
-        return String.format("Relation(%s, %s, %s)", entity, relationType, value);
-    }
-}
-
-enum RelationType
-{
-    EQ, LT, LTE, GTE, GT;
-
-    public static RelationType forString(String s)
-    {
-        if (s.equals("="))
-            return EQ;
-        else if (s.equals("<"))
-            return LT;
-        else if (s.equals("<="))
-            return LTE;
-        else if (s.equals(">="))
-            return GTE;
-        else if (s.equals(">"))
-            return GT;
-
-        return null;
-    }
-}
diff --git a/src/java/org/apache/cassandra/cql/SelectExpression.java b/src/java/org/apache/cassandra/cql/SelectExpression.java
deleted file mode 100644
index 4d3cb2f..0000000
--- a/src/java/org/apache/cassandra/cql/SelectExpression.java
+++ /dev/null
@@ -1,164 +0,0 @@
-/*
- * 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.cql;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Select expressions are analogous to the projection in a SQL query. They
- * determine which columns will appear in the result set.  SelectExpression
- * instances encapsulate a parsed expression from a <code>SELECT</code>
- * statement.
- *
- * See: doc/cql/CQL.html#SpecifyingColumns
- */
-public class SelectExpression
-{
-    public static final int MAX_COLUMNS_DEFAULT = 10000;
-
-    private int numColumns = MAX_COLUMNS_DEFAULT;
-    private boolean reverseColumns = false;
-    private final boolean hasFirstSet;
-    private final boolean wildcard;
-    private final Term start, finish;
-    private final List<Term> columns;
-
-    /**
-     * Create a new SelectExpression for a range (slice) of columns.
-     *
-     * @param start the starting column name
-     * @param finish the finishing column name
-     * @param count the number of columns to limit the results to
-     * @param reverse true to reverse column order
-     * @param wildcard determines weather this statement is wildcard
-     * @param firstSet determines weather "FIRST" keyword was set
-     */
-    public SelectExpression(Term start, Term finish, int count, boolean reverse, boolean wildcard, boolean firstSet)
-    {
-        this.start = start;
-        this.finish = finish;
-        numColumns = count;
-        reverseColumns = reverse;
-        this.wildcard = wildcard;
-        hasFirstSet = firstSet;
-        this.columns = null;
-    }
-
-    /**
-     * Create a new SelectExpression for a list of columns.
-     *
-     * @param first the first (possibly only) column name to select on.
-     * @param count the number of columns to limit the results on
-     * @param reverse true to reverse column order
-     * @param firstSet determines weather "FIRST" keyword was set
-     */
-    public SelectExpression(Term first, int count, boolean reverse, boolean firstSet)
-    {
-        wildcard = false;
-        columns = new ArrayList<Term>();
-        columns.add(first);
-        numColumns = count;
-        reverseColumns = reverse;
-        hasFirstSet = firstSet;
-        start = null;
-        finish = null;
-    }
-
-    /**
-     * Add an additional column name to a SelectExpression.
-     *
-     * @param addTerm
-     */
-    public void and(Term addTerm)
-    {
-        assert !isColumnRange();    // Not possible when invoked by parser
-        columns.add(addTerm);
-    }
-
-    public boolean isColumnRange()
-    {
-        return (start != null);
-    }
-
-    public boolean isColumnList()
-    {
-        return !isColumnRange();
-    }
-    public int getColumnsLimit()
-    {
-        return numColumns;
-    }
-
-    public boolean isColumnsReversed()
-    {
-        return reverseColumns;
-    }
-
-    public void setColumnsReversed(boolean reversed)
-    {
-        reverseColumns = reversed;
-    }
-
-    public void setColumnsLimit(int limit)
-    {
-        numColumns = limit;
-    }
-
-    /**
-     * @return weather expression includes "FIRST" keyword
-     */
-    public boolean hasFirstSet()
-    {
-        return hasFirstSet;
-    }
-
-    public Term getStart()
-    {
-        return start;
-    }
-
-    public Term getFinish()
-    {
-        return finish;
-    }
-
-    public List<Term> getColumns()
-    {
-        return columns;
-    }
-
-    public boolean isWildcard()
-    {
-        return wildcard;
-    }
-
-    public String toString()
-    {
-        return String.format("SelectExpression [numColumns=%s, reverseColumns=%s, hasFirstSet=%s, wildcard=%s, start=%s, finish=%s, columns=%s]",
-                             numColumns,
-                             reverseColumns,
-                             hasFirstSet,
-                             wildcard,
-                             start,
-                             finish,
-                             columns);
-    }
-
-
-}
diff --git a/src/java/org/apache/cassandra/cql/SelectStatement.java b/src/java/org/apache/cassandra/cql/SelectStatement.java
deleted file mode 100644
index b4ccd56..0000000
--- a/src/java/org/apache/cassandra/cql/SelectStatement.java
+++ /dev/null
@@ -1,188 +0,0 @@
-/*
- * 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.cql;
-
-import java.util.List;
-import java.util.Set;
-
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.db.ConsistencyLevel;
-
-/**
- * Encapsulates a completely parsed SELECT query, including the target
- * column family, expression, result count, and ordering clause.
- *
- */
-public class SelectStatement
-{
-    private final SelectExpression expression;
-    private final boolean isCountOper;
-    private final String columnFamily;
-    private final String keyspace;
-    private final ConsistencyLevel cLevel;
-    private final WhereClause clause;
-    private final int numRecords;
-
-    public SelectStatement(SelectExpression expression, boolean isCountOper, String keyspace, String columnFamily,
-            ConsistencyLevel cLevel, WhereClause clause, int numRecords)
-    {
-        this.expression = expression;
-        this.isCountOper = isCountOper;
-        this.keyspace = keyspace;
-        this.columnFamily = columnFamily;
-        this.cLevel = cLevel;
-        this.clause = (clause != null) ? clause : new WhereClause();
-        this.numRecords = numRecords;
-    }
-
-    public boolean isKeyRange()
-    {
-        return clause.isKeyRange();
-    }
-
-    public Set<Term> getKeys()
-    {
-        return clause.getKeys();
-    }
-
-    public Term getKeyStart()
-    {
-        return clause.getStartKey();
-    }
-
-    public Term getKeyFinish()
-    {
-        return clause.getFinishKey();
-    }
-
-    public List<Relation> getColumnRelations()
-    {
-        return clause.getColumnRelations();
-    }
-
-    public boolean isColumnRange()
-    {
-        return expression.isColumnRange();
-    }
-
-    public boolean isWildcard()
-    {
-        return expression.isWildcard();
-    }
-    public boolean isFullWildcard()
-    {
-        return expression.isWildcard() && !expression.hasFirstSet();
-    }
-
-    public List<Term> getColumnNames()
-    {
-        return expression.getColumns();
-    }
-
-    public Term getColumnStart()
-    {
-        return expression.getStart();
-    }
-
-    public Term getColumnFinish()
-    {
-        return expression.getFinish();
-    }
-
-    public boolean isSetKeyspace()
-    {
-        return keyspace != null;
-    }
-
-    public String getKeyspace()
-    {
-        return keyspace;
-    }
-
-    public String getColumnFamily()
-    {
-        return columnFamily;
-    }
-
-    public boolean isColumnsReversed()
-    {
-        return expression.isColumnsReversed();
-    }
-
-    public ConsistencyLevel getConsistencyLevel()
-    {
-        return cLevel;
-    }
-
-    public int getNumRecords()
-    {
-        return numRecords;
-    }
-
-    public int getColumnsLimit()
-    {
-        return expression.getColumnsLimit();
-    }
-
-    public boolean isCountOperation()
-    {
-        return isCountOper;
-    }
-
-    public boolean includeStartKey()
-    {
-        return clause.includeStartKey();
-    }
-
-    public boolean includeFinishKey()
-    {
-        return clause.includeFinishKey();
-    }
-
-    public String getKeyAlias()
-    {
-        return clause.getKeyAlias();
-    }
-
-    public boolean isMultiKey()
-    {
-        return clause.isMultiKey();
-    }
-
-    public void extractKeyAliasFromColumns(CFMetaData cfm)
-    {
-        clause.extractKeysFromColumns(cfm);
-    }
-
-    public List<Relation> getClauseRelations()
-    {
-        return clause.getClauseRelations();
-    }
-
-    public String toString()
-    {
-        return String.format("SelectStatement [expression=%s, isCountOper=%s, columnFamily=%s, keyspace=%s, cLevel=%s, clause=%s, numRecords=%s]",
-                             expression,
-                             isCountOper,
-                             columnFamily,
-                             keyspace,
-                             cLevel,
-                             clause,
-                             numRecords);
-    }
-}
diff --git a/src/java/org/apache/cassandra/cql/StatementType.java b/src/java/org/apache/cassandra/cql/StatementType.java
deleted file mode 100644
index 94db6a3..0000000
--- a/src/java/org/apache/cassandra/cql/StatementType.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * 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.cql;
-
-import java.util.EnumSet;
-
-public enum StatementType
-{
-    SELECT, INSERT, UPDATE, BATCH, USE, TRUNCATE, DELETE, CREATE_KEYSPACE, CREATE_COLUMNFAMILY, CREATE_INDEX, DROP_INDEX,
-        DROP_KEYSPACE, DROP_COLUMNFAMILY, ALTER_TABLE;
-
-    /** Statement types that don't require a keyspace to be set */
-    private static final EnumSet<StatementType> TOP_LEVEL = EnumSet.of(USE, CREATE_KEYSPACE, DROP_KEYSPACE);
-
-    /** Statement types that require a keyspace to be set */
-    public static final EnumSet<StatementType> REQUIRES_KEYSPACE = EnumSet.complementOf(TOP_LEVEL);
-}
diff --git a/src/java/org/apache/cassandra/cql/Term.java b/src/java/org/apache/cassandra/cql/Term.java
deleted file mode 100644
index 6e47bfa..0000000
--- a/src/java/org/apache/cassandra/cql/Term.java
+++ /dev/null
@@ -1,208 +0,0 @@
-/*
- * 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.cql;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-
-import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.db.marshal.AsciiType;
-import org.apache.cassandra.db.marshal.FloatType;
-import org.apache.cassandra.db.marshal.IntegerType;
-import org.apache.cassandra.db.marshal.LexicalUUIDType;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.serializers.MarshalException;
-
-/** A term parsed from a CQL statement. */
-public class Term
-{
-    private final String text;
-    private final TermType type;
-    private Integer bindIndex = -1;
-
-    public Term(String text, TermType type)
-    {
-        this.text = text == null ? "" : text;
-        this.type = type;
-    }
-
-    /**
-     * Create new Term instance from a string, and an integer that corresponds
-     * with the token ID from CQLParser.
-     *
-     * @param text the text representation of the term.
-     * @param type the term's type as an integer token ID.
-     */
-    public Term(String text, int type)
-    {
-        this(text == null ? "" : text, TermType.forInt(type));
-    }
-
-    public Term(long value, TermType type)
-    {
-        this(String.valueOf(value), type);
-    }
-
-    protected Term()
-    {
-        this("", TermType.STRING);
-    }
-
-    public Term(String text, int type, int index)
-    {
-        this(text, type);
-        this.bindIndex = index;
-    }
-
-    /**
-     * Returns the text parsed to create this term.
-     *
-     * @return the string term acquired from a CQL statement.
-     */
-    public String getText()
-    {
-        return text;
-    }
-
-    /**
-     * Returns the typed value, serialized to a ByteBuffer according to a
-     * comparator/validator.
-     *
-     * @return a ByteBuffer of the value.
-     * @throws InvalidRequestException if unable to coerce the string to its type.
-     */
-    public ByteBuffer getByteBuffer(AbstractType<?> validator, List<ByteBuffer> variables) throws InvalidRequestException
-    {
-        try
-        {
-            if (!isBindMarker()) return validator.fromStringCQL2(text);
-
-            // must be a marker term so check for a CqlBindValue stored in the term
-            if (bindIndex == null)
-                throw new AssertionError("a marker Term was encountered with no index value");
-
-            return variables.get(bindIndex);
-        }
-        catch (MarshalException e)
-        {
-            throw new InvalidRequestException(e.getMessage());
-        }
-    }
-
-    /**
-     * Returns the typed value, serialized to a ByteBuffer.
-     *
-     * @return a ByteBuffer of the value.
-     * @throws InvalidRequestException if unable to coerce the string to its type.
-     */
-    public ByteBuffer getByteBuffer() throws InvalidRequestException
-    {
-        switch (type)
-        {
-            case STRING:
-                return AsciiType.instance.fromString(text);
-            case INTEGER:
-                return IntegerType.instance.fromString(text);
-            case UUID:
-                // we specifically want the Lexical class here, not "UUIDType," because we're supposed to have
-                // a uuid-shaped string here, and UUIDType also accepts integer or date strings (and turns them into version 1 uuids).
-                return LexicalUUIDType.instance.fromString(text);
-            case FLOAT:
-              return FloatType.instance.fromString(text);
-        }
-
-        // FIXME: handle scenario that should never happen
-        return null;
-    }
-
-    /**
-     * Obtain the term's type.
-     *
-     * @return the type
-     */
-    public TermType getType()
-    {
-        return type;
-    }
-
-    public String toString()
-    {
-        return String.format("Term(%s, type=%s)", getText(), type);
-    }
-
-    public boolean isBindMarker()
-    {
-        return type==TermType.QMARK;
-    }
-
-    @Override
-    public int hashCode()
-    {
-        final int prime = 31;
-        int result = 1;
-        result = prime * result + ((text == null) ? 0 : text.hashCode());
-        result = prime * result + ((type == null) ? 0 : type.hashCode());
-        return result;
-    }
-
-    @Override
-    public boolean equals(Object obj)
-    {
-        if (this == obj)
-            return true;
-        if (obj == null)
-            return false;
-        if (getClass() != obj.getClass())
-            return false;
-        Term other = (Term) obj;
-        if (type==TermType.QMARK) return false; // markers are never equal
-        if (text == null)
-        {
-            if (other.text != null)
-                return false;
-        } else if (!text.equals(other.text))
-            return false;
-        if (type != other.type)
-            return false;
-        return true;
-    }
-
-
-}
-
-enum TermType
-{
-    STRING, INTEGER, UUID, FLOAT, QMARK;
-
-    static TermType forInt(int type)
-    {
-        if ((type == CqlParser.STRING_LITERAL) || (type == CqlParser.IDENT))
-            return STRING;
-        else if (type == CqlParser.INTEGER)
-            return INTEGER;
-        else if (type == CqlParser.UUID)
-          return UUID;
-        else if (type == CqlParser.FLOAT)
-            return FLOAT;
-        else if (type == CqlParser.QMARK)
-            return QMARK;
-
-        // FIXME: handled scenario that should never occur.
-        return null;
-    }
-}
diff --git a/src/java/org/apache/cassandra/cql/UpdateStatement.java b/src/java/org/apache/cassandra/cql/UpdateStatement.java
deleted file mode 100644
index 16a0d76..0000000
--- a/src/java/org/apache/cassandra/cql/UpdateStatement.java
+++ /dev/null
@@ -1,288 +0,0 @@
-/*
- * 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.cql;
-
-import java.nio.ByteBuffer;
-import java.util.*;
-
-import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.composites.CellName;
-import org.apache.cassandra.db.composites.CellNameType;
-import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.UnauthorizedException;
-import org.apache.cassandra.thrift.ThriftClientState;
-
-import static org.apache.cassandra.cql.QueryProcessor.validateColumn;
-import static org.apache.cassandra.cql.QueryProcessor.validateKey;
-import static org.apache.cassandra.thrift.ThriftValidation.validateColumnFamily;
-
-/**
- * An <code>UPDATE</code> statement parsed from a CQL query statement.
- *
- */
-public class UpdateStatement extends AbstractModification
-{
-    private Map<Term, Operation> columns;
-    private List<Term> columnNames, columnValues;
-    private final List<Term> keys;
-
-    /**
-     * Creates a new UpdateStatement from a column family name, columns map, consistency
-     * level, and key term.
-     *
-     * @param keyspace Keyspace (optional)
-     * @param columnFamily column family name
-     * @param keyName alias key name
-     * @param columns a map of column name/values pairs
-     * @param keys the keys to update
-     * @param attrs additional attributes for statement (CL, timestamp, timeToLive)
-     */
-    public UpdateStatement(String keyspace,
-                           String columnFamily,
-                           String keyName,
-                           Map<Term, Operation> columns,
-                           List<Term> keys,
-                           Attributes attrs)
-    {
-        super(keyspace, columnFamily, keyName, attrs);
-
-        this.columns = columns;
-        this.keys = keys;
-    }
-
-    /**
-     * Creates a new UpdateStatement from a column family name, a consistency level,
-     * key, and lists of column names and values.  It is intended for use with the
-     * alternate update format, <code>INSERT</code>.
-     *
-     * @param keyspace Keyspace (optional)
-     * @param columnFamily column family name
-     * @param keyName alias key name
-     * @param columnNames list of column names
-     * @param columnValues list of column values (corresponds to names)
-     * @param keys the keys to update
-     * @param attrs additional attributes for statement (CL, timestamp, timeToLive)
-     */
-    public UpdateStatement(String keyspace,
-                           String columnFamily,
-                           String keyName,
-                           List<Term> columnNames,
-                           List<Term> columnValues,
-                           List<Term> keys,
-                           Attributes attrs)
-    {
-        super(keyspace, columnFamily, keyName, attrs);
-
-        this.columnNames = columnNames;
-        this.columnValues = columnValues;
-        this.keys = keys;
-    }
-
-    /**
-     * Returns the consistency level of this <code>UPDATE</code> statement, either
-     * one parsed from the CQL statement, or the default level otherwise.
-     *
-     * @return the consistency level as a Thrift enum.
-     */
-    public ConsistencyLevel getConsistencyLevel()
-    {
-        return (cLevel != null) ? cLevel : defaultConsistency;
-    }
-
-    /**
-     * True if an explicit consistency level was parsed from the statement.
-     *
-     * @return true if a consistency was parsed, false otherwise.
-     */
-    public boolean isSetConsistencyLevel()
-    {
-        return (cLevel != null);
-    }
-
-    /** {@inheritDoc} */
-    public List<IMutation> prepareRowMutations(String keyspace, ThriftClientState clientState, List<ByteBuffer> variables)
-    throws InvalidRequestException, UnauthorizedException
-    {
-        return prepareRowMutations(keyspace, clientState, null, variables);
-    }
-
-    /** {@inheritDoc} */
-    public List<IMutation> prepareRowMutations(String keyspace, ThriftClientState clientState, Long timestamp, List<ByteBuffer> variables)
-    throws InvalidRequestException, UnauthorizedException
-    {
-        boolean hasCommutativeOperation = false;
-
-        for (Map.Entry<Term, Operation> column : getColumns().entrySet())
-        {
-            if (!column.getValue().isUnary())
-                hasCommutativeOperation = true;
-
-            if (hasCommutativeOperation && column.getValue().isUnary())
-                throw new InvalidRequestException("Mix of commutative and non-commutative operations is not allowed.");
-        }
-
-        CFMetaData metadata = validateColumnFamily(keyspace, columnFamily, hasCommutativeOperation);
-        if (hasCommutativeOperation)
-            getConsistencyLevel().validateCounterForWrite(metadata);
-
-        QueryProcessor.validateKeyAlias(metadata, keyName);
-
-        clientState.hasColumnFamilyAccess(keyspace, columnFamily, Permission.MODIFY);
-
-        List<IMutation> mutations = new LinkedList<>();
-
-        for (Term key: keys)
-            mutations.add(mutationForKey(keyspace, key.getByteBuffer(getKeyType(keyspace),variables), metadata, timestamp, clientState, variables));
-
-        return mutations;
-    }
-
-    /**
-     * Compute a mutation for a single key
-     *
-     *
-     * @param keyspace working keyspace
-     * @param key key to change
-     * @param metadata information about CF
-     * @param timestamp global timestamp to use for every key mutation
-     *
-     * @param clientState
-     * @return mutation
-     *
-     * @throws InvalidRequestException on the wrong request
-     */
-    private IMutation mutationForKey(String keyspace, ByteBuffer key, CFMetaData metadata, Long timestamp, ThriftClientState clientState, List<ByteBuffer> variables)
-    throws InvalidRequestException
-    {
-        validateKey(key);
-        CellNameType comparator = metadata.comparator;
-        AbstractType<?> at = comparator.asAbstractType();
-
-        // if true we need to wrap Mutation into CounterMutation
-        boolean hasCounterColumn = false;
-        Mutation mutation = new Mutation(keyspace, key);
-
-        for (Map.Entry<Term, Operation> column : getColumns().entrySet())
-        {
-            CellName colName = comparator.cellFromByteBuffer(column.getKey().getByteBuffer(at, variables));
-            Operation op = column.getValue();
-
-            if (op.isUnary())
-            {
-                if (hasCounterColumn)
-                    throw new InvalidRequestException("Mix of commutative and non-commutative operations is not allowed.");
-
-                ByteBuffer colValue = op.a.getByteBuffer(metadata.getValueValidator(colName),variables);
-
-                validateColumn(metadata, colName, colValue);
-                mutation.add(columnFamily,
-                             colName,
-                             colValue,
-                             (timestamp == null) ? getTimestamp(clientState) : timestamp,
-                             getTimeToLive());
-            }
-            else
-            {
-                hasCounterColumn = true;
-
-                if (!column.getKey().getText().equals(op.a.getText()))
-                    throw new InvalidRequestException("Only expressions like X = X + <long> are supported.");
-
-                long value;
-
-                try
-                {
-                    value = Long.parseLong(op.b.getText());
-                }
-                catch (NumberFormatException e)
-                {
-                    throw new InvalidRequestException(String.format("'%s' is an invalid value, should be a long.",
-                                                      op.b.getText()));
-                }
-
-                mutation.addCounter(columnFamily, colName, value);
-            }
-        }
-
-        return (hasCounterColumn) ? new CounterMutation(mutation, getConsistencyLevel()) : mutation;
-    }
-
-    public String getColumnFamily()
-    {
-        return columnFamily;
-    }
-
-    public List<Term> getKeys()
-    {
-        return keys;
-    }
-
-    public Map<Term, Operation> getColumns() throws InvalidRequestException
-    {
-        // Created from an UPDATE
-        if (columns != null)
-            return columns;
-
-        // Created from an INSERT
-
-        // Don't hate, validate.
-        if (columnNames.size() != columnValues.size())
-            throw new InvalidRequestException("unmatched column names/values");
-        if (columnNames.size() < 1)
-            throw new InvalidRequestException("no columns specified for INSERT");
-
-        columns = new HashMap<Term, Operation>();
-
-        for (int i = 0; i < columnNames.size(); i++)
-            columns.put(columnNames.get(i), new Operation(columnValues.get(i)));
-
-        return columns;
-    }
-
-    public String toString()
-    {
-        return String.format("UpdateStatement(keyspace=%s, columnFamily=%s, keys=%s, columns=%s, consistency=%s, timestamp=%s, timeToLive=%s)",
-                             keyspace,
-                             columnFamily,
-                             keys,
-                             columns,
-                             getConsistencyLevel(),
-                             timestamp,
-                             timeToLive);
-    }
-
-    public AbstractType<?> getKeyType(String keyspace)
-    {
-        return Schema.instance.getCFMetaData(keyspace, columnFamily).getKeyValidator();
-    }
-
-    public List<Term> getColumnNames()
-    {
-        return columnNames;
-    }
-
-    public List<Term> getColumnValues()
-    {
-        return columnValues;
-    }
-
-}
diff --git a/src/java/org/apache/cassandra/cql/WhereClause.java b/src/java/org/apache/cassandra/cql/WhereClause.java
deleted file mode 100644
index 3df5102..0000000
--- a/src/java/org/apache/cassandra/cql/WhereClause.java
+++ /dev/null
@@ -1,192 +0,0 @@
-/*
- * 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.cql;
-
-import java.util.ArrayList;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.cassandra.config.CFMetaData;
-
-/**
- * WhereClauses encapsulate all of the predicates of a SELECT query.
- *
- */
-public class WhereClause
-{
-    // all relations (except for `<key> IN (.., .., ..)` which can be directly interpreted) from parser
-    // are stored into this array and are filtered to the keys/columns by extractKeysFromColumns(...)
-    private final List<Relation> clauseRelations = new ArrayList<Relation>();
-    private final List<Relation> columns = new ArrayList<Relation>();
-
-    // added to either by the parser from an IN clause or by extractKeysFromColumns
-    private final Set<Term> keys = new LinkedHashSet<Term>();
-    private Term startKey, finishKey;
-    private boolean includeStartKey = false, includeFinishKey = false, multiKey = false;
-    // set by extractKeysFromColumns
-    private String keyAlias = null;
-
-    /**
-     * Create a new WhereClause with the first parsed relation.
-     *
-     * @param firstRelation key or column relation
-     */
-    public WhereClause(Relation firstRelation)
-    {
-        and(firstRelation);
-    }
-
-    public WhereClause()
-    {}
-
-    /**
-     * Add an additional relation to this WHERE clause.
-     *
-     * @param relation the relation to add.
-     */
-    public void and(Relation relation)
-    {
-        clauseRelations.add(relation);
-    }
-
-    /**
-     * The same as KEY = <key> to avoid using Relation object
-     * @param key key to include into clause
-     */
-    public void andKeyEquals(Term key)
-    {
-        keys.add(key);
-    }
-
-    public List<Relation> getColumnRelations()
-    {
-        return columns;
-    }
-
-    public boolean isKeyRange()
-    {
-        return startKey != null;
-    }
-
-    public Term getStartKey()
-    {
-        return startKey;
-    }
-
-    public Term getFinishKey()
-    {
-        return finishKey;
-    }
-
-    public Set<Term> getKeys()
-    {
-        return keys;
-    }
-
-    public boolean includeStartKey()
-    {
-        return includeStartKey;
-    }
-
-    public boolean includeFinishKey()
-    {
-        return includeFinishKey;
-    }
-
-    public void setKeyAlias(String alias)
-    {
-        keyAlias = alias.toUpperCase();
-    }
-
-    public boolean isMultiKey() {
-        return multiKey;
-    }
-
-    public void setMultiKey(boolean multiKey)
-    {
-        this.multiKey = multiKey;
-    }
-
-    public String getKeyAlias()
-    {
-        // TODO fix special casing here, key alias should always be set post-extract
-        // key alias as not related to keys in here, it can be unset when we have a query like
-        // SELECT * FROM <CF> WHERE key = 1 and col > 2 and col < 3;
-        // it will be always set when statement looks like this
-        // SELECT * FROM <CF> WHERE <key> IN (.., .., ..);
-        // key is NULL when KEY keyword is used or when key alias given by user was not recognized
-        // validateKeyAlias will throw an exception for us in that case
-        return keyAlias == null ? QueryProcessor.DEFAULT_KEY_NAME : keyAlias;
-    }
-
-    public void extractKeysFromColumns(CFMetaData cfm)
-    {
-        String realKeyAlias = cfm.getCQL2KeyName();
-
-        if (!keys.isEmpty())
-            return; // we already have key(s) set (<key> IN (.., ...) construction used)
-
-        for (Relation relation : clauseRelations)
-        {
-            String name = relation.getEntity().getText().toUpperCase();
-            if (name.equals(realKeyAlias))
-            {
-                if (keyAlias == null) // setting found key as an alias
-                    keyAlias = name;
-
-                if (relation.operator() == RelationType.EQ)
-                {
-                    keys.add(relation.getValue());
-                }
-                else if ((relation.operator() == RelationType.GT) || (relation.operator() == RelationType.GTE))
-                {
-                    startKey = relation.getValue();
-                    includeStartKey = relation.operator() == RelationType.GTE;
-                }
-                else if ((relation.operator() == RelationType.LT) || (relation.operator() == RelationType.LTE))
-                {
-                    finishKey = relation.getValue();
-                    includeFinishKey = relation.operator() == RelationType.LTE;
-                }
-            }
-            else
-            {
-                columns.add(relation);
-            }
-        }
-    }
-
-    public List<Relation> getClauseRelations()
-    {
-        return clauseRelations;
-    }
-
-    public String toString()
-    {
-        return String.format("WhereClause [keys=%s, startKey=%s, finishKey=%s, columns=%s, includeStartKey=%s, includeFinishKey=%s, multiKey=%s, keyAlias=%s]",
-                             keys,
-                             startKey,
-                             finishKey,
-                             columns,
-                             includeStartKey,
-                             includeFinishKey,
-                             multiKey,
-                             keyAlias);
-    }
-}
diff --git a/src/java/org/apache/cassandra/cql/hooks/ExecutionContext.java b/src/java/org/apache/cassandra/cql/hooks/ExecutionContext.java
deleted file mode 100644
index deb785c..0000000
--- a/src/java/org/apache/cassandra/cql/hooks/ExecutionContext.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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.cql.hooks;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-
-import com.google.common.base.Optional;
-
-import org.apache.cassandra.thrift.ThriftClientState;
-
-/**
- * Contextual information about the execution of a CQLStatement.
- * Used by {@link org.apache.cassandra.cql.hooks.PreExecutionHook}
- *
- * The CQL string representing the statement being executed is optional
- * and is not present for prepared statements. Contexts created for the
- * execution of regular (i.e. non-prepared) statements will always
- * contain a CQL string.
- */
-public class ExecutionContext
-{
-    public final ThriftClientState clientState;
-    public final Optional<String> queryString;
-    public final List<ByteBuffer> variables;
-
-    public ExecutionContext(ThriftClientState clientState, String queryString, List<ByteBuffer> variables)
-    {
-        this.clientState = clientState;
-        this.queryString = Optional.fromNullable(queryString);
-        this.variables = variables;
-    }
-}
diff --git a/src/java/org/apache/cassandra/cql/hooks/PostPreparationHook.java b/src/java/org/apache/cassandra/cql/hooks/PostPreparationHook.java
deleted file mode 100644
index 1de9c70..0000000
--- a/src/java/org/apache/cassandra/cql/hooks/PostPreparationHook.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.cql.hooks;
-
-import org.apache.cassandra.cql.CQLStatement;
-import org.apache.cassandra.exceptions.RequestValidationException;
-
-/**
- * Run directly after a CQL Statement is prepared in
- * {@link org.apache.cassandra.cql.QueryProcessor}.
- */
-public interface PostPreparationHook
-{
-    /**
-     * Called in QueryProcessor, once a CQL statement has been prepared.
-     *
-     * @param statement the statement to perform additional processing on
-     * @param context preparation context containing additional info
-     *                about the operation and statement
-     * @throws RequestValidationException
-     */
-    void processStatement(CQLStatement statement, PreparationContext context) throws RequestValidationException;
-}
diff --git a/src/java/org/apache/cassandra/cql/hooks/PreExecutionHook.java b/src/java/org/apache/cassandra/cql/hooks/PreExecutionHook.java
deleted file mode 100644
index 29ed38e..0000000
--- a/src/java/org/apache/cassandra/cql/hooks/PreExecutionHook.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * 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.cql.hooks;
-
-import org.apache.cassandra.cql.CQLStatement;
-import org.apache.cassandra.exceptions.RequestExecutionException;
-import org.apache.cassandra.exceptions.RequestValidationException;
-
-/**
- * Run before the CQL Statement is executed in
- * {@link org.apache.cassandra.cql.QueryProcessor}. The CQLStatement
- * returned from the processStatement method is what is actually executed
- * by the QueryProcessor.
- */
-public interface PreExecutionHook
-{
-    /**
-     * Perform pre-processing on a CQL statement prior to it being
-     * executed by the QueryProcessor. If required, implementations
-     * may modify the statement as the returned instance is what
-     * is actually executed.
-     *
-     * @param statement the statement to perform pre-processing on
-     * @param context execution context containing additional info
-     *                about the operation and statement
-     * @return the actual statement that will be executed, possibly
-     *         a modification of the initial statement
-     * @throws RequestExecutionException, RequestValidationException
-     */
-    CQLStatement processStatement(CQLStatement statement, ExecutionContext context) throws RequestExecutionException, RequestValidationException;
-}
diff --git a/src/java/org/apache/cassandra/cql/hooks/PreparationContext.java b/src/java/org/apache/cassandra/cql/hooks/PreparationContext.java
deleted file mode 100644
index 00cce78..0000000
--- a/src/java/org/apache/cassandra/cql/hooks/PreparationContext.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * 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.cql.hooks;
-
-import org.apache.cassandra.cql.CQLStatement;
-import org.apache.cassandra.thrift.ThriftClientState;
-
-/**
- * Contextual information about the preparation of a CQLStatement.
- * Used by {@link org.apache.cassandra.cql.hooks.PostPreparationHook}
- */
-public class PreparationContext
-{
-    public final ThriftClientState clientState;
-    public final String queryString;
-    public final CQLStatement statement;
-
-    public PreparationContext(ThriftClientState clientState, String queryString, CQLStatement statement)
-    {
-        this.clientState = clientState;
-        this.queryString = queryString;
-        this.statement = statement;
-    }
-}
diff --git a/src/java/org/apache/cassandra/cql3/AbstractMarker.java b/src/java/org/apache/cassandra/cql3/AbstractMarker.java
index d18790c..d11b8e2 100644
--- a/src/java/org/apache/cassandra/cql3/AbstractMarker.java
+++ b/src/java/org/apache/cassandra/cql3/AbstractMarker.java
@@ -17,6 +17,9 @@
  */
 package org.apache.cassandra.cql3;
 
+import java.util.Collections;
+
+import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.db.marshal.CollectionType;
 import org.apache.cassandra.db.marshal.ListType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
@@ -45,6 +48,11 @@
         return true;
     }
 
+    public Iterable<Function> getFunctions()
+    {
+        return Collections.emptySet();
+    }
+
     /**
      * A parsed, but non prepared, bind marker.
      */
@@ -57,7 +65,7 @@
             this.bindIndex = bindIndex;
         }
 
-        public AbstractMarker prepare(String keyspace, ColumnSpecification receiver) throws InvalidRequestException
+        public NonTerminal prepare(String keyspace, ColumnSpecification receiver) throws InvalidRequestException
         {
             if (!(receiver.type instanceof CollectionType))
                 return new Constants.Marker(bindIndex, receiver);
@@ -71,9 +79,9 @@
             throw new AssertionError();
         }
 
-        public boolean isAssignableTo(String keyspace, ColumnSpecification receiver)
+        public AssignmentTestable.TestResult testAssignment(String keyspace, ColumnSpecification receiver)
         {
-            return true;
+            return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
         }
 
         @Override
diff --git a/src/java/org/apache/cassandra/cql3/AssignementTestable.java b/src/java/org/apache/cassandra/cql3/AssignementTestable.java
deleted file mode 100644
index 2253cf7..0000000
--- a/src/java/org/apache/cassandra/cql3/AssignementTestable.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * 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.cql3;
-
-import org.apache.cassandra.exceptions.InvalidRequestException;
-
-public interface AssignementTestable
-{
-    /**
-     * @return whether this object can be assigned to the provided receiver
-     */
-    public boolean isAssignableTo(String keyspace, ColumnSpecification receiver) throws InvalidRequestException;
-}
diff --git a/src/java/org/apache/cassandra/cql3/AssignmentTestable.java b/src/java/org/apache/cassandra/cql3/AssignmentTestable.java
new file mode 100644
index 0000000..41b80eb
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/AssignmentTestable.java
@@ -0,0 +1,72 @@
+/*
+ * 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.cql3;
+
+import java.util.Collection;
+
+public interface AssignmentTestable
+{
+    /**
+     * @return whether this object can be assigned to the provided receiver. We distinguish
+     * between 3 values: 
+     *   - EXACT_MATCH if this object is exactly of the type expected by the receiver
+     *   - WEAKLY_ASSIGNABLE if this object is not exactly the expected type but is assignable nonetheless
+     *   - NOT_ASSIGNABLE if it's not assignable
+     * Most caller should just call the isAssignable() method on the result, though functions have a use for
+     * testing "strong" equality to decide the most precise overload to pick when multiple could match.
+     */
+    public TestResult testAssignment(String keyspace, ColumnSpecification receiver);
+
+    public enum TestResult
+    {
+        EXACT_MATCH, WEAKLY_ASSIGNABLE, NOT_ASSIGNABLE;
+
+        public boolean isAssignable()
+        {
+            return this != NOT_ASSIGNABLE;
+        }
+
+        public boolean isExactMatch()
+        {
+            return this == EXACT_MATCH;
+        }
+
+        // Test all elements of toTest for assignment. If all are exact match, return exact match. If any is not assignable,
+        // return not assignable. Otherwise, return weakly assignable.
+        public static TestResult testAll(String keyspace, ColumnSpecification receiver, Collection<? extends AssignmentTestable> toTest)
+        {
+            TestResult res = EXACT_MATCH;
+            for (AssignmentTestable rt : toTest)
+            {
+                if (rt == null)
+                {
+                    res = WEAKLY_ASSIGNABLE;
+                    continue;
+                }
+
+                TestResult t = rt.testAssignment(keyspace, receiver);
+                if (t == NOT_ASSIGNABLE)
+                    return NOT_ASSIGNABLE;
+
+                if (t == WEAKLY_ASSIGNABLE)
+                    res = WEAKLY_ASSIGNABLE;
+            }
+            return res;
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/Attributes.java b/src/java/org/apache/cassandra/cql3/Attributes.java
index 435757b..7b38e9f 100644
--- a/src/java/org/apache/cassandra/cql3/Attributes.java
+++ b/src/java/org/apache/cassandra/cql3/Attributes.java
@@ -18,13 +18,17 @@
 package org.apache.cassandra.cql3;
 
 import java.nio.ByteBuffer;
-import java.util.List;
+import java.util.Collections;
 
+import com.google.common.collect.Iterables;
+
+import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.db.ExpiringCell;
 import org.apache.cassandra.db.marshal.Int32Type;
 import org.apache.cassandra.db.marshal.LongType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.serializers.MarshalException;
+import org.apache.cassandra.utils.ByteBufferUtil;
 
 /**
  * Utility class for the Parser to gather attributes for modification
@@ -46,6 +50,18 @@
         this.timeToLive = timeToLive;
     }
 
+    public Iterable<Function> getFunctions()
+    {
+        if (timestamp != null && timeToLive != null)
+            return Iterables.concat(timestamp.getFunctions(), timeToLive.getFunctions());
+        else if (timestamp != null)
+            return timestamp.getFunctions();
+        else if (timeToLive != null)
+            return timeToLive.getFunctions();
+        else
+            return Collections.emptySet();
+    }
+
     public boolean isTimestampSet()
     {
         return timestamp != null;
@@ -65,13 +81,16 @@
         if (tval == null)
             throw new InvalidRequestException("Invalid null value of timestamp");
 
+        if (tval == ByteBufferUtil.UNSET_BYTE_BUFFER)
+            return now;
+
         try
         {
             LongType.instance.validate(tval);
         }
         catch (MarshalException e)
         {
-            throw new InvalidRequestException("Invalid timestamp value");
+            throw new InvalidRequestException("Invalid timestamp value: " + tval);
         }
 
         return LongType.instance.compose(tval);
@@ -86,18 +105,21 @@
         if (tval == null)
             throw new InvalidRequestException("Invalid null value of TTL");
 
+        if (tval == ByteBufferUtil.UNSET_BYTE_BUFFER) // treat as unlimited
+            return 0;
+
         try
         {
             Int32Type.instance.validate(tval);
         }
         catch (MarshalException e)
         {
-            throw new InvalidRequestException("Invalid timestamp value");
+            throw new InvalidRequestException("Invalid timestamp value: " + tval);
         }
 
         int ttl = Int32Type.instance.compose(tval);
         if (ttl < 0)
-            throw new InvalidRequestException("A TTL must be greater or equal to 0");
+            throw new InvalidRequestException("A TTL must be greater or equal to 0, but was " + ttl);
 
         if (ttl > ExpiringCell.MAX_TTL)
             throw new InvalidRequestException(String.format("ttl is too large. requested (%d) maximum (%d)", ttl, ExpiringCell.MAX_TTL));
diff --git a/src/java/org/apache/cassandra/cql3/BatchQueryOptions.java b/src/java/org/apache/cassandra/cql3/BatchQueryOptions.java
index 2bb8071..db7fa39 100644
--- a/src/java/org/apache/cassandra/cql3/BatchQueryOptions.java
+++ b/src/java/org/apache/cassandra/cql3/BatchQueryOptions.java
@@ -22,9 +22,10 @@
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.cassandra.utils.MD5Digest;
+
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.service.QueryState;
-import org.apache.cassandra.service.pager.PagingState;
 
 public abstract class BatchQueryOptions
 {
@@ -51,6 +52,11 @@
 
     public abstract QueryOptions forStatement(int i);
 
+    public void prepareStatement(int i, List<ColumnSpecification> boundNames)
+    {
+        forStatement(i).prepare(boundNames);
+    }
+
     public ConsistencyLevel getConsistency()
     {
         return wrapped.getConsistency();
@@ -108,5 +114,26 @@
         {
             return perStatementOptions.get(i);
         }
+
+        @Override
+        public void prepareStatement(int i, List<ColumnSpecification> boundNames)
+        {
+            if (isPreparedStatement(i))
+            {
+                QueryOptions options = perStatementOptions.get(i);
+                options.prepare(boundNames);
+                options = QueryOptions.addColumnSpecifications(options, boundNames);
+                perStatementOptions.set(i, options);
+            }
+            else
+            {
+                super.prepareStatement(i, boundNames);
+            }
+        }
+
+        private boolean isPreparedStatement(int i)
+        {
+            return getQueryOrIdList().get(i) instanceof MD5Digest;
+        }
     }
 }
diff --git a/src/java/org/apache/cassandra/cql3/CQL3Type.java b/src/java/org/apache/cassandra/cql3/CQL3Type.java
index db7fa5c..1823108 100644
--- a/src/java/org/apache/cassandra/cql3/CQL3Type.java
+++ b/src/java/org/apache/cassandra/cql3/CQL3Type.java
@@ -22,6 +22,7 @@
 
 import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.ConfigurationException;
@@ -38,22 +39,26 @@
 
     public enum Native implements CQL3Type
     {
-        ASCII    (AsciiType.instance),
-        BIGINT   (LongType.instance),
-        BLOB     (BytesType.instance),
-        BOOLEAN  (BooleanType.instance),
-        COUNTER  (CounterColumnType.instance),
-        DECIMAL  (DecimalType.instance),
-        DOUBLE   (DoubleType.instance),
-        FLOAT    (FloatType.instance),
-        INET     (InetAddressType.instance),
-        INT      (Int32Type.instance),
-        TEXT     (UTF8Type.instance),
-        TIMESTAMP(TimestampType.instance),
-        UUID     (UUIDType.instance),
-        VARCHAR  (UTF8Type.instance),
-        VARINT   (IntegerType.instance),
-        TIMEUUID (TimeUUIDType.instance);
+        ASCII       (AsciiType.instance),
+        BIGINT      (LongType.instance),
+        BLOB        (BytesType.instance),
+        BOOLEAN     (BooleanType.instance),
+        COUNTER     (CounterColumnType.instance),
+        DECIMAL     (DecimalType.instance),
+        DOUBLE      (DoubleType.instance),
+        FLOAT       (FloatType.instance),
+        INET        (InetAddressType.instance),
+        INT         (Int32Type.instance),
+        SMALLINT    (ShortType.instance),
+        TEXT        (UTF8Type.instance),
+        TIMESTAMP   (TimestampType.instance),
+        TINYINT     (ByteType.instance),
+        UUID        (UUIDType.instance),
+        VARCHAR     (UTF8Type.instance),
+        VARINT      (IntegerType.instance),
+        TIMEUUID    (TimeUUIDType.instance),
+        DATE        (SimpleDateType.instance),
+        TIME        (TimeType.instance);
 
         private final AbstractType<?> type;
 
@@ -310,11 +315,26 @@
             return false;
         }
 
+        public boolean isFrozen()
+        {
+            return this.frozen;
+        }
+
+        public boolean canBeNonFrozen()
+        {
+            return true;
+        }
+
         public boolean isCounter()
         {
             return false;
         }
 
+        public String keyspace()
+        {
+            return null;
+        }
+
         public void freeze() throws InvalidRequestException
         {
             String message = String.format("frozen<> is only allowed on collections, tuples, and user-defined types (got %s)", this);
@@ -476,18 +496,28 @@
                 this.name = name;
             }
 
+            public String keyspace()
+            {
+                return name.getKeyspace();
+            }
+
             public void freeze()
             {
                 frozen = true;
             }
 
+            public boolean canBeNonFrozen()
+            {
+                return false;
+            }
+
             public CQL3Type prepare(String keyspace) throws InvalidRequestException
             {
                 if (name.hasKeyspace())
                 {
                     // The provided keyspace is the one of the current statement this is part of. If it's different from the keyspace of
                     // the UTName, we reject since we want to limit user types to their own keyspace (see #6643)
-                    if (!keyspace.equals(name.getKeyspace()))
+                    if (keyspace != null && !SystemKeyspace.NAME.equals(name.getKeyspace()) && !keyspace.equals(name.getKeyspace()))
                         throw new InvalidRequestException(String.format("Statement on keyspace %s cannot refer to a user type in keyspace %s; "
                                                                         + "user types can only be used in the keyspace they are defined in",
                                                                         keyspace, name.getKeyspace()));
diff --git a/src/java/org/apache/cassandra/cql3/CQLStatement.java b/src/java/org/apache/cassandra/cql3/CQLStatement.java
index a1642ef..02292ad 100644
--- a/src/java/org/apache/cassandra/cql3/CQLStatement.java
+++ b/src/java/org/apache/cassandra/cql3/CQLStatement.java
@@ -17,10 +17,11 @@
  */
 package org.apache.cassandra.cql3;
 
-import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
-import org.apache.cassandra.exceptions.*;
+import org.apache.cassandra.transport.messages.ResultMessage;
 
 public interface CQLStatement
 {
@@ -53,9 +54,16 @@
     public ResultMessage execute(QueryState state, QueryOptions options) throws RequestValidationException, RequestExecutionException;
 
     /**
-     * Variante of execute used for internal query against the system tables, and thus only query the local node.
+     * Variant of execute used for internal query against the system tables, and thus only query the local node.
      *
      * @param state the current query state
      */
     public ResultMessage executeInternal(QueryState state, QueryOptions options) throws RequestValidationException, RequestExecutionException;
+
+    /**
+     * Return an Iterable over all of the functions (both native and user-defined) used by any component
+     * of the statement
+     * @return functions all functions found (may contain duplicates)
+     */
+    public Iterable<Function> getFunctions();
 }
diff --git a/src/java/org/apache/cassandra/cql3/ColumnCondition.java b/src/java/org/apache/cassandra/cql3/ColumnCondition.java
index 1a8e5a3..3412e71 100644
--- a/src/java/org/apache/cassandra/cql3/ColumnCondition.java
+++ b/src/java/org/apache/cassandra/cql3/ColumnCondition.java
@@ -21,11 +21,14 @@
 import java.util.*;
 
 import com.google.common.base.Predicate;
+import com.google.common.collect.Iterables;
 import com.google.common.collect.Iterators;
-import static com.google.common.collect.Lists.newArrayList;
 
 import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.db.*;
+import org.apache.cassandra.cql3.Term.Terminal;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.db.Cell;
+import org.apache.cassandra.db.ColumnFamily;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.composites.CellNameType;
 import org.apache.cassandra.db.composites.Composite;
@@ -34,15 +37,14 @@
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.transport.Server;
 import org.apache.cassandra.utils.ByteBufferUtil;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+
+import static com.google.common.collect.Lists.newArrayList;
 
 /**
  * A CQL3 condition on the value of a column or collection element.  For example, "UPDATE .. IF a = 0".
  */
 public class ColumnCondition
 {
-    private static final Logger logger = LoggerFactory.getLogger(ColumnCondition.class);
 
     public final ColumnDefinition column;
 
@@ -62,7 +64,7 @@
         this.inValues = inValues;
         this.operator = op;
 
-        if (!operator.equals(Operator.IN))
+        if (operator != Operator.IN)
             assert this.inValues == null;
     }
 
@@ -96,6 +98,20 @@
         return new ColumnCondition(column, collectionElement, inMarker, null, Operator.IN);
     }
 
+    public Iterable<Function> getFunctions()
+    {
+        Iterable<Function> iter = Collections.emptyList();
+        if (collectionElement != null)
+           iter = Iterables.concat(iter, collectionElement.getFunctions());
+        if (value != null)
+            iter = Iterables.concat(iter, value.getFunctions());
+        if (inValues != null)
+            for (Term value : inValues)
+                if (value != null)
+                    iter = Iterables.concat(iter, value.getFunctions());
+        return iter;
+    }
+
     /**
      * Collects the column specification for the bind variables of this operation.
      *
@@ -107,7 +123,7 @@
         if (collectionElement != null)
             collectionElement.collectMarkerSpecification(boundNames);
 
-        if (operator.equals(Operator.IN) && inValues != null)
+        if ((operator == Operator.IN) && inValues != null)
         {
             for (Term value : inValues)
                 value.collectMarkerSpecification(boundNames);
@@ -120,7 +136,7 @@
 
     public ColumnCondition.Bound bind(QueryOptions options) throws InvalidRequestException
     {
-        boolean isInCondition = operator.equals(Operator.IN);
+        boolean isInCondition = operator == Operator.IN;
         if (column.type instanceof CollectionType)
         {
             if (collectionElement == null)
@@ -161,6 +177,8 @@
         /** Returns true if the operator is satisfied (i.e. "value operator otherValue == true"), false otherwise. */
         protected boolean compareWithOperator(Operator operator, AbstractType<?> type, ByteBuffer value, ByteBuffer otherValue) throws InvalidRequestException
         {
+            if (value == ByteBufferUtil.UNSET_BYTE_BUFFER)
+                throw new InvalidRequestException("Invalid 'unset' value in condition");
             if (value == null)
             {
                 switch (operator)
@@ -176,7 +194,7 @@
             else if (otherValue == null)
             {
                 // the condition value is not null, so only NEQ can return true
-                return operator.equals(Operator.NEQ);
+                return operator == Operator.NEQ;
             }
             int comparison = type.compare(otherValue, value);
             switch (operator)
@@ -226,7 +244,7 @@
         {
             super(condition.column, condition.operator);
             assert !(column.type instanceof CollectionType) && condition.collectionElement == null;
-            assert !condition.operator.equals(Operator.IN);
+            assert condition.operator != Operator.IN;
             this.value = condition.value.bindAndGet(options);
         }
 
@@ -248,14 +266,28 @@
         {
             super(condition.column, condition.operator);
             assert !(column.type instanceof CollectionType) && condition.collectionElement == null;
-            assert condition.operator.equals(Operator.IN);
+            assert condition.operator == Operator.IN;
             if (condition.inValues == null)
-                this.inValues = ((Lists.Marker) condition.value).bind(options).getElements();
+            {
+                Terminal terminal = condition.value.bind(options);
+
+                if (terminal == null)
+                    throw new InvalidRequestException("Invalid null list in IN condition");
+
+                if (terminal == Constants.UNSET_VALUE)
+                    throw new InvalidRequestException("Invalid 'unset' value in condition");
+
+                this.inValues = ((Lists.Value) terminal).getElements();
+            }
             else
             {
                 this.inValues = new ArrayList<>(condition.inValues.size());
                 for (Term value : condition.inValues)
-                    this.inValues.add(value.bindAndGet(options));
+                {
+                    ByteBuffer buffer = value.bindAndGet(options);
+                    if (buffer != ByteBufferUtil.UNSET_BYTE_BUFFER)
+                        this.inValues.add(value.bindAndGet(options));
+                }
             }
         }
 
@@ -281,7 +313,7 @@
         {
             super(condition.column, condition.operator);
             assert column.type instanceof CollectionType && condition.collectionElement != null;
-            assert !condition.operator.equals(Operator.IN);
+            assert condition.operator != Operator.IN;
             this.collectionElement = condition.collectionElement.bindAndGet(options);
             this.value = condition.value.bindAndGet(options);
         }
@@ -334,7 +366,7 @@
             return idx;
         }
 
-        static ByteBuffer getListItem(Iterator<Cell> iter, int index) throws InvalidRequestException
+        static ByteBuffer getListItem(Iterator<Cell> iter, int index)
         {
             int adv = Iterators.advance(iter, index);
             if (adv == index && iter.hasNext())
@@ -361,12 +393,22 @@
             this.collectionElement = condition.collectionElement.bindAndGet(options);
 
             if (condition.inValues == null)
-                this.inValues = ((Lists.Marker) condition.value).bind(options).getElements();
+            {
+                Terminal terminal = condition.value.bind(options);
+                if (terminal == Constants.UNSET_VALUE)
+                    throw new InvalidRequestException("Invalid 'unset' value in condition");
+                this.inValues = ((Lists.Value) terminal).getElements();
+            }
             else
             {
                 this.inValues = new ArrayList<>(condition.inValues.size());
                 for (Term value : condition.inValues)
-                    this.inValues.add(value.bindAndGet(options));
+                {
+                    ByteBuffer buffer = value.bindAndGet(options);
+                    // We want to ignore unset values
+                    if (buffer != ByteBufferUtil.UNSET_BYTE_BUFFER)
+                        this.inValues.add(buffer);
+                }
             }
         }
 
@@ -458,7 +500,7 @@
         {
             super(condition.column, condition.operator);
             assert column.type.isCollection() && condition.collectionElement == null;
-            assert !condition.operator.equals(Operator.IN);
+            assert condition.operator != Operator.IN;
             this.value = condition.value.bind(options);
         }
 
@@ -471,9 +513,9 @@
                 Iterator<Cell> iter = collectionColumns(current.metadata().comparator.create(rowPrefix, column), current, now);
                 if (value == null)
                 {
-                    if (operator.equals(Operator.EQ))
+                    if (operator == Operator.EQ)
                         return !iter.hasNext();
-                    else if (operator.equals(Operator.NEQ))
+                    else if (operator == Operator.NEQ)
                         return iter.hasNext();
                     else
                         throw new InvalidRequestException(String.format("Invalid comparison with null for operator \"%s\"", operator));
@@ -497,11 +539,11 @@
             // make sure we use v3 serialization format for comparison
             ByteBuffer conditionValue;
             if (type.kind == CollectionType.Kind.LIST)
-                conditionValue = ((Lists.Value) value).getWithProtocolVersion(Server.VERSION_3);
+                conditionValue = ((Lists.Value) value).get(Server.VERSION_3);
             else if (type.kind == CollectionType.Kind.SET)
-                conditionValue = ((Sets.Value) value).getWithProtocolVersion(Server.VERSION_3);
+                conditionValue = ((Sets.Value) value).get(Server.VERSION_3);
             else
-                conditionValue = ((Maps.Value) value).getWithProtocolVersion(Server.VERSION_3);
+                conditionValue = ((Maps.Value) value).get(Server.VERSION_3);
 
             return compareWithOperator(operator, type, conditionValue, cell.value());
         }
@@ -513,9 +555,15 @@
 
             switch (type.kind)
             {
-                case LIST: return listAppliesTo((ListType)type, iter, ((Lists.Value)value).elements, operator);
-                case SET: return setAppliesTo((SetType)type, iter, ((Sets.Value)value).elements, operator);
-                case MAP: return mapAppliesTo((MapType)type, iter, ((Maps.Value)value).map, operator);
+                case LIST:
+                    List<ByteBuffer> valueList = ((Lists.Value) value).elements;
+                    return listAppliesTo((ListType)type, iter, valueList, operator);
+                case SET:
+                    Set<ByteBuffer> valueSet = ((Sets.Value) value).elements;
+                    return setAppliesTo((SetType)type, iter, valueSet, operator);
+                case MAP:
+                    Map<ByteBuffer, ByteBuffer> valueMap = ((Maps.Value) value).map;
+                    return mapAppliesTo((MapType)type, iter, valueMap, operator);
             }
             throw new AssertionError();
         }
@@ -525,7 +573,7 @@
             while(iter.hasNext())
             {
                 if (!conditionIter.hasNext())
-                    return operator.equals(Operator.GT) || operator.equals(Operator.GTE) || operator.equals(Operator.NEQ);
+                    return (operator == Operator.GT) || (operator == Operator.GTE) || (operator == Operator.NEQ);
 
                 // for lists we use the cell value; for sets we use the cell name
                 ByteBuffer cellValue = isSet? iter.next().name().collectionElement() : iter.next().value();
@@ -535,7 +583,7 @@
             }
 
             if (conditionIter.hasNext())
-                return operator.equals(Operator.LT) || operator.equals(Operator.LTE) || operator.equals(Operator.NEQ);
+                return (operator == Operator.LT) || (operator == Operator.LTE) || (operator == Operator.NEQ);
 
             // they're equal
             return operator == Operator.EQ || operator == Operator.LTE || operator == Operator.GTE;
@@ -580,7 +628,7 @@
             while(iter.hasNext())
             {
                 if (!conditionIter.hasNext())
-                    return operator.equals(Operator.GT) || operator.equals(Operator.GTE) || operator.equals(Operator.NEQ);
+                    return (operator == Operator.GT) || (operator == Operator.GTE) || (operator == Operator.NEQ);
 
                 Map.Entry<ByteBuffer, ByteBuffer> conditionEntry = conditionIter.next();
                 Cell c = iter.next();
@@ -597,7 +645,7 @@
             }
 
             if (conditionIter.hasNext())
-                return operator.equals(Operator.LT) || operator.equals(Operator.LTE) || operator.equals(Operator.NEQ);
+                return (operator == Operator.LT) || (operator == Operator.LTE) || (operator == Operator.NEQ);
 
             // they're equal
             return operator == Operator.EQ || operator == Operator.LTE || operator == Operator.GTE;
@@ -612,18 +660,29 @@
         {
             super(condition.column, condition.operator);
             assert column.type instanceof CollectionType && condition.collectionElement == null;
-            assert condition.operator.equals(Operator.IN);
+            assert condition.operator == Operator.IN;
             inValues = new ArrayList<>();
             if (condition.inValues == null)
             {
                 // We have a list of serialized collections that need to be deserialized for later comparisons
                 CollectionType collectionType = (CollectionType) column.type;
                 Lists.Marker inValuesMarker = (Lists.Marker) condition.value;
+                Terminal terminal = inValuesMarker.bind(options);
+
+                if (terminal == null)
+                    throw new InvalidRequestException("Invalid null list in IN condition");
+
+                if (terminal == Constants.UNSET_VALUE)
+                    throw new InvalidRequestException("Invalid 'unset' value in condition");
+
                 if (column.type instanceof ListType)
                 {
                     ListType deserializer = ListType.getInstance(collectionType.valueComparator(), false);
-                    for (ByteBuffer buffer : inValuesMarker.bind(options).elements)
+                    for (ByteBuffer buffer : ((Lists.Value) terminal).elements)
                     {
+                        if (buffer == ByteBufferUtil.UNSET_BYTE_BUFFER)
+                            continue;
+
                         if (buffer == null)
                             this.inValues.add(null);
                         else
@@ -633,8 +692,11 @@
                 else if (column.type instanceof MapType)
                 {
                     MapType deserializer = MapType.getInstance(collectionType.nameComparator(), collectionType.valueComparator(), false);
-                    for (ByteBuffer buffer : inValuesMarker.bind(options).elements)
+                    for (ByteBuffer buffer : ((Lists.Value) terminal).elements)
                     {
+                        if (buffer == ByteBufferUtil.UNSET_BYTE_BUFFER)
+                            continue;
+
                         if (buffer == null)
                             this.inValues.add(null);
                         else
@@ -644,8 +706,11 @@
                 else if (column.type instanceof SetType)
                 {
                     SetType deserializer = SetType.getInstance(collectionType.valueComparator(), false);
-                    for (ByteBuffer buffer : inValuesMarker.bind(options).elements)
+                    for (ByteBuffer buffer : ((Lists.Value) terminal).elements)
                     {
+                        if (buffer == ByteBufferUtil.UNSET_BYTE_BUFFER)
+                            continue;
+
                         if (buffer == null)
                             this.inValues.add(null);
                         else
@@ -656,7 +721,11 @@
             else
             {
                 for (Term value : condition.inValues)
-                    this.inValues.add(value.bind(options));
+                {
+                    Terminal terminal = value.bind(options);
+                    if (terminal != Constants.UNSET_VALUE)
+                        this.inValues.add(terminal);
+                }
             }
         }
 
@@ -685,7 +754,7 @@
                         if (cell == null || !cell.isLive(now))
                             return true;
                     }
-                    else if (type.compare(((Term.CollectionTerminal)value).getWithProtocolVersion(Server.VERSION_3), cell.value()) == 0)
+                    else if (type.compare(value.get(Server.VERSION_3), cell.value()) == 0)
                     {
                         return true;
                     }
@@ -758,7 +827,7 @@
 
             if (collectionElement == null)
             {
-                if (operator.equals(Operator.IN))
+                if (operator == Operator.IN)
                 {
                     if (inValues == null)
                         return ColumnCondition.inCondition(receiver, inMarker.prepare(keyspace, receiver));
@@ -792,7 +861,7 @@
                 default:
                     throw new AssertionError();
             }
-            if (operator.equals(Operator.IN))
+            if (operator == Operator.IN)
             {
                 if (inValues == null)
                     return ColumnCondition.inCondition(receiver, collectionElement.prepare(keyspace, elementSpec), inMarker.prepare(keyspace, valueSpec));
diff --git a/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java b/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
index 1501479..823af94 100644
--- a/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
+++ b/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
@@ -17,16 +17,20 @@
  */
 package org.apache.cassandra.cql3;
 
-import java.util.Locale;
 import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Locale;
 
 import org.apache.cassandra.cache.IMeasurableMemory;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.cql3.statements.Selectable;
+import org.apache.cassandra.cql3.selection.Selectable;
+import org.apache.cassandra.cql3.selection.Selector;
+import org.apache.cassandra.cql3.selection.SimpleSelector;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.CompositeType;
 import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.ObjectSizes;
 import org.apache.cassandra.utils.memory.AbstractAllocator;
@@ -35,7 +39,7 @@
  * Represents an identifer for a CQL column definition.
  * TODO : should support light-weight mode without text representation for when not interned
  */
-public class ColumnIdentifier implements Selectable, IMeasurableMemory
+public class ColumnIdentifier extends org.apache.cassandra.cql3.selection.Selectable implements IMeasurableMemory
 {
     public final ByteBuffer bytes;
     private final String text;
@@ -105,6 +109,14 @@
         return new ColumnIdentifier(allocator.clone(bytes), text);
     }
 
+    public Selector.Factory newSelectorFactory(CFMetaData cfm, List<ColumnDefinition> defs) throws InvalidRequestException
+    {
+        ColumnDefinition def = cfm.getColumnDefinition(this);
+        if (def == null)
+            throw new InvalidRequestException(String.format("Undefined name %s in selection clause", this));
+
+        return SimpleSelector.newFactory(def, addAndGetIndex(def, defs));
+    }
 
     /**
      * Because Thrift-created tables may have a non-text comparator, we cannot determine the proper 'key' until
diff --git a/src/java/org/apache/cassandra/cql3/ColumnSpecification.java b/src/java/org/apache/cassandra/cql3/ColumnSpecification.java
index 836c6b9..e64f5f9 100644
--- a/src/java/org/apache/cassandra/cql3/ColumnSpecification.java
+++ b/src/java/org/apache/cassandra/cql3/ColumnSpecification.java
@@ -20,6 +20,10 @@
 import com.google.common.base.Objects;
 
 import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.ReversedType;
+
+import java.util.Collection;
+import java.util.Iterator;
 
 public class ColumnSpecification
 {
@@ -36,19 +40,52 @@
         this.type = type;
     }
 
-    public boolean equals(Object obj)
+    /**
+     * Returns a new <code>ColumnSpecification</code> for the same column but with the specified alias.
+     *
+     * @param alias the column alias
+     * @return a new <code>ColumnSpecification</code> for the same column but with the specified alias.
+     */
+    public ColumnSpecification withAlias(ColumnIdentifier alias)
     {
-        if (null == obj)
+        return new ColumnSpecification(ksName, cfName, alias, type);
+    }
+
+    public boolean isReversedType()
+    {
+        return type instanceof ReversedType;
+    }
+
+    /**
+     * Returns true if all ColumnSpecifications are in the same table, false otherwise.
+     */
+    public static boolean allInSameTable(Collection<ColumnSpecification> names)
+    {
+        if (names == null || names.isEmpty())
             return false;
 
-        if(!(obj instanceof ColumnSpecification))
+        Iterator<ColumnSpecification> iter = names.iterator();
+        ColumnSpecification first = iter.next();
+        while (iter.hasNext())
+        {
+            ColumnSpecification name = iter.next();
+            if (!name.ksName.equals(first.ksName) || !name.cfName.equals(first.cfName))
+                return false;
+        }
+        return true;
+    }
+
+    @Override
+    public boolean equals(Object other)
+    {
+        if (!(other instanceof ColumnSpecification))
             return false;
 
-        ColumnSpecification other = (ColumnSpecification)obj;
-        return Objects.equal(ksName, other.ksName)
-            && Objects.equal(cfName, other.cfName)
-            && Objects.equal(name, other.name)
-            && Objects.equal(type, other.type);
+        ColumnSpecification that = (ColumnSpecification) other;
+        return this.ksName.equals(that.ksName) &&
+               this.cfName.equals(that.cfName) &&
+               this.name.equals(that.name) &&
+               this.type.equals(that.type);
     }
 
     public int hashCode()
@@ -64,5 +101,4 @@
                       .add("type", type)
                       .toString();
     }
-
 }
diff --git a/src/java/org/apache/cassandra/cql3/Constants.java b/src/java/org/apache/cassandra/cql3/Constants.java
index 01fbdf0..07b848c 100644
--- a/src/java/org/apache/cassandra/cql3/Constants.java
+++ b/src/java/org/apache/cassandra/cql3/Constants.java
@@ -21,7 +21,6 @@
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.composites.CellName;
@@ -44,38 +43,42 @@
 
     public enum Type
     {
-        STRING, INTEGER, UUID, FLOAT, BOOLEAN, HEX;
+        STRING, INTEGER, UUID, FLOAT, DATE, TIME, BOOLEAN, HEX;
     }
 
+    public static final Value UNSET_VALUE = new Value(ByteBufferUtil.UNSET_BYTE_BUFFER);
+
     public static final Term.Raw NULL_LITERAL = new Term.Raw()
     {
-        private final Term.Terminal NULL_VALUE = new Value(null)
-        {
-            @Override
-            public Terminal bind(QueryOptions options)
-            {
-                // We return null because that makes life easier for collections
-                return null;
-            }
-
-            @Override
-            public String toString()
-            {
-                return "null";
-            }
-        };
-
         public Term prepare(String keyspace, ColumnSpecification receiver) throws InvalidRequestException
         {
-            if (!isAssignableTo(keyspace, receiver))
+            if (!testAssignment(keyspace, receiver).isAssignable())
                 throw new InvalidRequestException("Invalid null value for counter increment/decrement");
 
             return NULL_VALUE;
         }
 
-        public boolean isAssignableTo(String keyspace, ColumnSpecification receiver)
+        public AssignmentTestable.TestResult testAssignment(String keyspace, ColumnSpecification receiver)
         {
-            return !(receiver.type instanceof CounterColumnType);
+            return receiver.type instanceof CounterColumnType
+                 ? AssignmentTestable.TestResult.NOT_ASSIGNABLE
+                 : AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
+        }
+
+        @Override
+        public String toString()
+        {
+            return "null";
+        }
+    };
+
+    public static final Term.Terminal NULL_VALUE = new Value(null)
+    {
+        @Override
+        public Terminal bind(QueryOptions options)
+        {
+            // We return null because that makes life easier for collections
+            return null;
         }
 
         @Override
@@ -129,7 +132,7 @@
 
         public Value prepare(String keyspace, ColumnSpecification receiver) throws InvalidRequestException
         {
-            if (!isAssignableTo(keyspace, receiver))
+            if (!testAssignment(keyspace, receiver).isAssignable())
                 throw new InvalidRequestException(String.format("Invalid %s constant (%s) for \"%s\" of type %s", type, text, receiver.name, receiver.type.asCQL3Type()));
 
             return new Value(parsedValue(receiver.type));
@@ -159,15 +162,15 @@
             return text;
         }
 
-        public boolean isAssignableTo(String keyspace, ColumnSpecification receiver)
+        public AssignmentTestable.TestResult testAssignment(String keyspace, ColumnSpecification receiver)
         {
             CQL3Type receiverType = receiver.type.asCQL3Type();
             if (receiverType.isCollection())
-                return false;
+                return AssignmentTestable.TestResult.NOT_ASSIGNABLE;
 
             if (!(receiverType instanceof CQL3Type.Native))
                 // Skip type validation for custom types. May or may not be a good idea
-                return true;
+                return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
 
             CQL3Type.Native nt = (CQL3Type.Native)receiverType;
             switch (type)
@@ -179,57 +182,62 @@
                         case TEXT:
                         case INET:
                         case VARCHAR:
+                        case DATE:
+                        case TIME:
                         case TIMESTAMP:
-                            return true;
+                            return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
                     }
-                    return false;
+                    break;
                 case INTEGER:
                     switch (nt)
                     {
                         case BIGINT:
                         case COUNTER:
+                        case DATE:
                         case DECIMAL:
                         case DOUBLE:
                         case FLOAT:
                         case INT:
+                        case SMALLINT:
                         case TIMESTAMP:
+                        case TINYINT:
                         case VARINT:
-                            return true;
+                            return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
                     }
-                    return false;
+                    break;
                 case UUID:
                     switch (nt)
                     {
                         case UUID:
                         case TIMEUUID:
-                            return true;
+                            return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
                     }
-                    return false;
+                    break;
                 case FLOAT:
                     switch (nt)
                     {
                         case DECIMAL:
                         case DOUBLE:
                         case FLOAT:
-                            return true;
+                            return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
                     }
-                    return false;
+                    break;
                 case BOOLEAN:
                     switch (nt)
                     {
                         case BOOLEAN:
-                            return true;
+                            return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
                     }
-                    return false;
+                    break;
                 case HEX:
                     switch (nt)
                     {
                         case BLOB:
-                            return true;
+                            return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
                     }
-                    return false;
+                    break;
             }
-            return false;
+            return AssignmentTestable.TestResult.NOT_ASSIGNABLE;
         }
 
         @Override
@@ -251,7 +259,7 @@
             this.bytes = bytes;
         }
 
-        public ByteBuffer get(QueryOptions options)
+        public ByteBuffer get(int protocolVersion)
         {
             return bytes;
         }
@@ -283,7 +291,7 @@
             try
             {
                 ByteBuffer value = options.getValues().get(bindIndex);
-                if (value != null)
+                if (value != null && value != ByteBufferUtil.UNSET_BYTE_BUFFER)
                     receiver.type.validate(value);
                 return value;
             }
@@ -296,7 +304,11 @@
         public Value bind(QueryOptions options) throws InvalidRequestException
         {
             ByteBuffer bytes = bindAndGet(options);
-            return bytes == null ? null : new Constants.Value(bytes);
+            if (bytes == null)
+                return null;
+            if (bytes == ByteBufferUtil.UNSET_BYTE_BUFFER)
+                return Constants.UNSET_VALUE;
+            return new Constants.Value(bytes);
         }
     }
 
@@ -309,9 +321,12 @@
 
         public void execute(ByteBuffer rowKey, ColumnFamily cf, Composite prefix, UpdateParameters params) throws InvalidRequestException
         {
-            CellName cname = cf.getComparator().create(prefix, column);
             ByteBuffer value = t.bindAndGet(params.options);
-            cf.addColumn(value == null ? params.makeTombstone(cname) : params.makeColumn(cname, value));
+            if (value != ByteBufferUtil.UNSET_BYTE_BUFFER) // use reference equality and not object equality
+            {
+                CellName cname = cf.getComparator().create(prefix, column);
+                cf.addColumn(value == null ? params.makeTombstone(cname) : params.makeColumn(cname, value));
+            }
         }
     }
 
@@ -327,6 +342,9 @@
             ByteBuffer bytes = t.bindAndGet(params.options);
             if (bytes == null)
                 throw new InvalidRequestException("Invalid null value for counter increment");
+            if (bytes == ByteBufferUtil.UNSET_BYTE_BUFFER)
+                return;
+
             long increment = ByteBufferUtil.toLong(bytes);
             CellName cname = cf.getComparator().create(prefix, column);
             cf.addColumn(params.makeCounter(cname, increment));
@@ -345,6 +363,8 @@
             ByteBuffer bytes = t.bindAndGet(params.options);
             if (bytes == null)
                 throw new InvalidRequestException("Invalid null value for counter increment");
+            if (bytes == ByteBufferUtil.UNSET_BYTE_BUFFER)
+                return;
 
             long increment = ByteBufferUtil.toLong(bytes);
             if (increment == Long.MIN_VALUE)
diff --git a/src/java/org/apache/cassandra/cql3/Cql.g b/src/java/org/apache/cassandra/cql3/Cql.g
index fc8f614..035e704 100644
--- a/src/java/org/apache/cassandra/cql3/Cql.g
+++ b/src/java/org/apache/cassandra/cql3/Cql.g
@@ -37,12 +37,11 @@
     import java.util.Map;
     import java.util.Set;
 
-    import org.apache.cassandra.auth.Permission;
-    import org.apache.cassandra.auth.DataResource;
-    import org.apache.cassandra.auth.IResource;
+    import org.apache.cassandra.auth.*;
     import org.apache.cassandra.cql3.*;
     import org.apache.cassandra.cql3.statements.*;
-    import org.apache.cassandra.cql3.functions.FunctionCall;
+    import org.apache.cassandra.cql3.selection.*;
+    import org.apache.cassandra.cql3.functions.*;
     import org.apache.cassandra.db.marshal.CollectionType;
     import org.apache.cassandra.exceptions.ConfigurationException;
     import org.apache.cassandra.exceptions.InvalidRequestException;
@@ -57,7 +56,6 @@
     public static final Set<String> reservedTypeNames = new HashSet<String>()
     {{
         add("byte");
-        add("smallint");
         add("complex");
         add("enum");
         add("date");
@@ -94,6 +92,13 @@
         return marker;
     }
 
+    public Json.Marker newJsonBindVariables(ColumnIdentifier name)
+    {
+        Json.Marker marker = new Json.Marker(bindVariables.size());
+        bindVariables.add(name);
+        return marker;
+    }
+
     public void addErrorListener(ErrorListener listener)
     {
         this.listeners.add(listener);
@@ -163,6 +168,19 @@
         }
         operations.add(Pair.create(key, update));
     }
+
+    public Set<Permission> filterPermissions(Set<Permission> permissions, IResource resource)
+    {
+        if (resource == null)
+            return Collections.emptySet();
+        Set<Permission> filtered = new HashSet<>(permissions);
+        filtered.retainAll(resource.applicablePermissions());
+        if (filtered.isEmpty())
+            addRecognitionError("Resource type " + resource.getClass().getSimpleName() +
+                                    " does not support any of the requested permissions");
+
+        return filtered;
+    }
 }
 
 @lexer::header {
@@ -230,8 +248,8 @@
     | st13=dropIndexStatement          { $stmt = st13; }
     | st14=alterTableStatement         { $stmt = st14; }
     | st15=alterKeyspaceStatement      { $stmt = st15; }
-    | st16=grantStatement              { $stmt = st16; }
-    | st17=revokeStatement             { $stmt = st17; }
+    | st16=grantPermissionsStatement   { $stmt = st16; }
+    | st17=revokePermissionsStatement  { $stmt = st17; }
     | st18=listPermissionsStatement    { $stmt = st18; }
     | st19=createUserStatement         { $stmt = st19; }
     | st20=alterUserStatement          { $stmt = st20; }
@@ -242,6 +260,16 @@
     | st25=createTypeStatement         { $stmt = st25; }
     | st26=alterTypeStatement          { $stmt = st26; }
     | st27=dropTypeStatement           { $stmt = st27; }
+    | st28=createFunctionStatement     { $stmt = st28; }
+    | st29=dropFunctionStatement       { $stmt = st29; }
+    | st30=createAggregateStatement    { $stmt = st30; }
+    | st31=dropAggregateStatement      { $stmt = st31; }
+    | st32=createRoleStatement         { $stmt = st32; }
+    | st33=alterRoleStatement          { $stmt = st33; }
+    | st34=dropRoleStatement           { $stmt = st34; }
+    | st35=listRolesStatement          { $stmt = st35; }
+    | st36=grantRoleStatement          { $stmt = st36; }
+    | st37=revokeRoleStatement         { $stmt = st37; }
     ;
 
 /*
@@ -260,14 +288,14 @@
 selectStatement returns [SelectStatement.RawStatement expr]
     @init {
         boolean isDistinct = false;
-        boolean isCount = false;
-        ColumnIdentifier countAlias = null;
         Term.Raw limit = null;
         Map<ColumnIdentifier.Raw, Boolean> orderings = new LinkedHashMap<ColumnIdentifier.Raw, Boolean>();
         boolean allowFiltering = false;
+        boolean isJson = false;
     }
-    : K_SELECT ( ( K_DISTINCT { isDistinct = true; } )? sclause=selectClause
-               | (K_COUNT '(' sclause=selectCountClause ')' { isCount = true; } (K_AS c=ident { countAlias = c; })?) )
+    : K_SELECT 
+      ( K_JSON { isJson = true; } )?
+      ( ( K_DISTINCT { isDistinct = true; } )? sclause=selectClause )
       K_FROM cf=columnFamilyName
       ( K_WHERE wclause=whereClause )?
       ( K_ORDER K_BY orderByClause[orderings] ( ',' orderByClause[orderings] )* )?
@@ -276,9 +304,8 @@
       {
           SelectStatement.Parameters params = new SelectStatement.Parameters(orderings,
                                                                              isDistinct,
-                                                                             isCount,
-                                                                             countAlias,
-                                                                             allowFiltering);
+                                                                             allowFiltering,
+                                                                             isJson);
           $expr = new SelectStatement.RawStatement(cf, params, sclause, wclause, limit);
       }
     ;
@@ -296,6 +323,7 @@
 unaliasedSelector returns [Selectable.Raw s]
     @init { Selectable.Raw tmp = null; }
     :  ( c=cident                                  { tmp = c; }
+       | K_COUNT '(' countArgument ')'             { tmp = new Selectable.WithFunction.Raw(FunctionName.nativeFunction("countRows"), Collections.<Selectable.Raw>emptyList());}
        | K_WRITETIME '(' c=cident ')'              { tmp = new Selectable.WritetimeOrTTL.Raw(c, true); }
        | K_TTL       '(' c=cident ')'              { tmp = new Selectable.WritetimeOrTTL.Raw(c, false); }
        | f=functionName args=selectionFunctionArgs { tmp = new Selectable.WithFunction.Raw(f, args); }
@@ -309,9 +337,9 @@
       ')' { $a = args; }
     ;
 
-selectCountClause returns [List<RawSelector> expr]
-    : '\*'           { $expr = Collections.<RawSelector>emptyList();}
-    | i=INTEGER      { if (!i.getText().equals("1")) addRecognitionError("Only COUNT(1) is supported, got COUNT(" + i.getText() + ")"); $expr = Collections.<RawSelector>emptyList();}
+countArgument
+    : '\*'
+    | i=INTEGER { if (!i.getText().equals("1")) addRecognitionError("Only COUNT(1) is supported, got COUNT(" + i.getText() + ")");}
     ;
 
 whereClause returns [List<Relation> clause]
@@ -321,10 +349,9 @@
 
 orderByClause[Map<ColumnIdentifier.Raw, Boolean> orderings]
     @init{
-        ColumnIdentifier.Raw orderBy = null;
         boolean reversed = false;
     }
-    : c=cident { orderBy = c; } (K_ASC | K_DESC { reversed = true; })? { orderings.put(c, reversed); }
+    : c=cident (K_ASC | K_DESC { reversed = true; })? { orderings.put(c, reversed); }
     ;
 
 /**
@@ -333,29 +360,49 @@
  * USING TIMESTAMP <long>;
  *
  */
-insertStatement returns [UpdateStatement.ParsedInsert expr]
+insertStatement returns [ModificationStatement.Parsed expr]
+    : K_INSERT K_INTO cf=columnFamilyName
+        ( st1=normalInsertStatement[cf] { $expr = st1; }
+        | K_JSON st2=jsonInsertStatement[cf] { $expr = st2; })
+    ;
+
+normalInsertStatement [CFName cf] returns [UpdateStatement.ParsedInsert expr]
     @init {
         Attributes.Raw attrs = new Attributes.Raw();
         List<ColumnIdentifier.Raw> columnNames  = new ArrayList<ColumnIdentifier.Raw>();
         List<Term.Raw> values = new ArrayList<Term.Raw>();
         boolean ifNotExists = false;
     }
-    : K_INSERT K_INTO cf=columnFamilyName
-          '(' c1=cident { columnNames.add(c1); }  ( ',' cn=cident { columnNames.add(cn); } )* ')'
-        K_VALUES
-          '(' v1=term { values.add(v1); } ( ',' vn=term { values.add(vn); } )* ')'
-
-        ( K_IF K_NOT K_EXISTS { ifNotExists = true; } )?
-        ( usingClause[attrs] )?
+    : '(' c1=cident { columnNames.add(c1); }  ( ',' cn=cident { columnNames.add(cn); } )* ')'
+      K_VALUES
+      '(' v1=term { values.add(v1); } ( ',' vn=term { values.add(vn); } )* ')'
+      ( K_IF K_NOT K_EXISTS { ifNotExists = true; } )?
+      ( usingClause[attrs] )?
       {
-          $expr = new UpdateStatement.ParsedInsert(cf,
-                                                   attrs,
-                                                   columnNames,
-                                                   values,
-                                                   ifNotExists);
+          $expr = new UpdateStatement.ParsedInsert(cf, attrs, columnNames, values, ifNotExists);
       }
     ;
 
+jsonInsertStatement [CFName cf] returns [UpdateStatement.ParsedInsertJson expr]
+    @init {
+        Attributes.Raw attrs = new Attributes.Raw();
+        boolean ifNotExists = false;
+    }
+    : val=jsonValue
+      ( K_IF K_NOT K_EXISTS { ifNotExists = true; } )?
+      ( usingClause[attrs] )?
+      {
+          $expr = new UpdateStatement.ParsedInsertJson(cf, attrs, val, ifNotExists);
+      }
+    ;
+
+jsonValue returns [Json.Raw value]
+    :
+    | s=STRING_LITERAL { $value = new Json.Literal($s.text); }
+    | ':' id=ident     { $value = newJsonBindVariables(id); }
+    | QMARK            { $value = newJsonBindVariables(null); }
+    ;
+
 usingClause[Attributes.Raw attrs]
     : K_USING usingClauseObjective[attrs] ( K_AND usingClauseObjective[attrs] )*
     ;
@@ -488,6 +535,103 @@
     | d=deleteStatement  { $statement = d; }
     ;
 
+createAggregateStatement returns [CreateAggregateStatement expr]
+    @init {
+        boolean orReplace = false;
+        boolean ifNotExists = false;
+
+        List<CQL3Type.Raw> argsTypes = new ArrayList<>();
+    }
+    : K_CREATE (K_OR K_REPLACE { orReplace = true; })?
+      K_AGGREGATE
+      (K_IF K_NOT K_EXISTS { ifNotExists = true; })?
+      fn=functionName
+      '('
+        (
+          v=comparatorType { argsTypes.add(v); }
+          ( ',' v=comparatorType { argsTypes.add(v); } )*
+        )?
+      ')'
+      K_SFUNC sfunc = allowedFunctionName
+      K_STYPE stype = comparatorType
+      (
+        K_FINALFUNC ffunc = allowedFunctionName
+      )?
+      (
+        K_INITCOND ival = term
+      )?
+      { $expr = new CreateAggregateStatement(fn, argsTypes, sfunc, stype, ffunc, ival, orReplace, ifNotExists); }
+    ;
+
+dropAggregateStatement returns [DropAggregateStatement expr]
+    @init {
+        boolean ifExists = false;
+        List<CQL3Type.Raw> argsTypes = new ArrayList<>();
+        boolean argsPresent = false;
+    }
+    : K_DROP K_AGGREGATE
+      (K_IF K_EXISTS { ifExists = true; } )?
+      fn=functionName
+      (
+        '('
+          (
+            v=comparatorType { argsTypes.add(v); }
+            ( ',' v=comparatorType { argsTypes.add(v); } )*
+          )?
+        ')'
+        { argsPresent = true; }
+      )?
+      { $expr = new DropAggregateStatement(fn, argsTypes, argsPresent, ifExists); }
+    ;
+
+createFunctionStatement returns [CreateFunctionStatement expr]
+    @init {
+        boolean orReplace = false;
+        boolean ifNotExists = false;
+
+        List<ColumnIdentifier> argsNames = new ArrayList<>();
+        List<CQL3Type.Raw> argsTypes = new ArrayList<>();
+        boolean calledOnNullInput = false;
+    }
+    : K_CREATE (K_OR K_REPLACE { orReplace = true; })?
+      K_FUNCTION
+      (K_IF K_NOT K_EXISTS { ifNotExists = true; })?
+      fn=functionName
+      '('
+        (
+          k=ident v=comparatorType { argsNames.add(k); argsTypes.add(v); }
+          ( ',' k=ident v=comparatorType { argsNames.add(k); argsTypes.add(v); } )*
+        )?
+      ')'
+      ( (K_RETURNS K_NULL) | (K_CALLED { calledOnNullInput=true; })) K_ON K_NULL K_INPUT
+      K_RETURNS rt = comparatorType
+      K_LANGUAGE language = IDENT
+      K_AS body = STRING_LITERAL
+      { $expr = new CreateFunctionStatement(fn, $language.text.toLowerCase(), $body.text,
+                                            argsNames, argsTypes, rt, calledOnNullInput, orReplace, ifNotExists); }
+    ;
+
+dropFunctionStatement returns [DropFunctionStatement expr]
+    @init {
+        boolean ifExists = false;
+        List<CQL3Type.Raw> argsTypes = new ArrayList<>();
+        boolean argsPresent = false;
+    }
+    : K_DROP K_FUNCTION
+      (K_IF K_EXISTS { ifExists = true; } )?
+      fn=functionName
+      (
+        '('
+          (
+            v=comparatorType { argsTypes.add(v); }
+            ( ',' v=comparatorType { argsTypes.add(v); } )*
+          )?
+        ')'
+        { argsPresent = true; }
+      )?
+      { $expr = new DropFunctionStatement(fn, argsTypes, argsPresent, ifExists); }
+    ;
+
 /**
  * CREATE KEYSPACE [IF NOT EXISTS] <KEYSPACE> WITH attr1 = value1 AND attr2 = value2;
  */
@@ -579,9 +723,10 @@
     ;
 
 indexIdent returns [IndexTarget.Raw id]
-    : c=cident                { $id = IndexTarget.Raw.valuesOf(c); }
-    | K_KEYS '(' c=cident ')' { $id = IndexTarget.Raw.keysOf(c); }
-    | K_FULL '(' c=cident ')' { $id = IndexTarget.Raw.fullCollection(c); }
+    : c=cident                   { $id = IndexTarget.Raw.valuesOf(c); }
+    | K_KEYS '(' c=cident ')'    { $id = IndexTarget.Raw.keysOf(c); }
+    | K_ENTRIES '(' c=cident ')' { $id = IndexTarget.Raw.keysAndValuesOf(c); }
+    | K_FULL '(' c=cident ')'    { $id = IndexTarget.Raw.fullCollection(c); }
     ;
 
 
@@ -703,116 +848,252 @@
     ;
 
 /**
- * GRANT <permission> ON <resource> TO <username>
+ * GRANT <permission> ON <resource> TO <rolename>
  */
-grantStatement returns [GrantStatement stmt]
+grantPermissionsStatement returns [GrantPermissionsStatement stmt]
     : K_GRANT
           permissionOrAll
       K_ON
           resource
       K_TO
-          username
-      { $stmt = new GrantStatement($permissionOrAll.perms, $resource.res, $username.text); }
+          grantee=userOrRoleName
+      { $stmt = new GrantPermissionsStatement(filterPermissions($permissionOrAll.perms, $resource.res), $resource.res, grantee); }
     ;
 
 /**
- * REVOKE <permission> ON <resource> FROM <username>
+ * REVOKE <permission> ON <resource> FROM <rolename>
  */
-revokeStatement returns [RevokeStatement stmt]
+revokePermissionsStatement returns [RevokePermissionsStatement stmt]
     : K_REVOKE
           permissionOrAll
       K_ON
           resource
       K_FROM
-          username
-      { $stmt = new RevokeStatement($permissionOrAll.perms, $resource.res, $username.text); }
+          revokee=userOrRoleName
+      { $stmt = new RevokePermissionsStatement(filterPermissions($permissionOrAll.perms, $resource.res), $resource.res, revokee); }
+    ;
+
+/**
+ * GRANT ROLE <rolename> TO <grantee>
+ */
+grantRoleStatement returns [GrantRoleStatement stmt]
+    : K_GRANT
+          role=userOrRoleName
+      K_TO
+          grantee=userOrRoleName
+      { $stmt = new GrantRoleStatement(role, grantee); }
+    ;
+
+/**
+ * REVOKE ROLE <rolename> FROM <revokee>
+ */
+revokeRoleStatement returns [RevokeRoleStatement stmt]
+    : K_REVOKE
+          role=userOrRoleName
+      K_FROM
+          revokee=userOrRoleName
+      { $stmt = new RevokeRoleStatement(role, revokee); }
     ;
 
 listPermissionsStatement returns [ListPermissionsStatement stmt]
     @init {
         IResource resource = null;
-        String username = null;
         boolean recursive = true;
+        RoleName grantee = new RoleName();
     }
     : K_LIST
           permissionOrAll
       ( K_ON resource { resource = $resource.res; } )?
-      ( K_OF username { username = $username.text; } )?
+      ( K_OF roleName[grantee] )?
       ( K_NORECURSIVE { recursive = false; } )?
-      { $stmt = new ListPermissionsStatement($permissionOrAll.perms, resource, username, recursive); }
+      { $stmt = new ListPermissionsStatement($permissionOrAll.perms, resource, grantee, recursive); }
     ;
 
 permission returns [Permission perm]
-    : p=(K_CREATE | K_ALTER | K_DROP | K_SELECT | K_MODIFY | K_AUTHORIZE)
+    : p=(K_CREATE | K_ALTER | K_DROP | K_SELECT | K_MODIFY | K_AUTHORIZE | K_DESCRIBE | K_EXECUTE)
     { $perm = Permission.valueOf($p.text.toUpperCase()); }
     ;
 
 permissionOrAll returns [Set<Permission> perms]
-    : K_ALL ( K_PERMISSIONS )?       { $perms = Permission.ALL_DATA; }
+    : K_ALL ( K_PERMISSIONS )?       { $perms = Permission.ALL; }
     | p=permission ( K_PERMISSION )? { $perms = EnumSet.of($p.perm); }
     ;
 
 resource returns [IResource res]
-    : r=dataResource { $res = $r.res; }
+    : d=dataResource { $res = $d.res; }
+    | r=roleResource { $res = $r.res; }
+    | f=functionResource { $res = $f.res; }
     ;
 
 dataResource returns [DataResource res]
     : K_ALL K_KEYSPACES { $res = DataResource.root(); }
     | K_KEYSPACE ks = keyspaceName { $res = DataResource.keyspace($ks.id); }
     | ( K_COLUMNFAMILY )? cf = columnFamilyName
-      { $res = DataResource.columnFamily($cf.name.getKeyspace(), $cf.name.getColumnFamily()); }
+      { $res = DataResource.table($cf.name.getKeyspace(), $cf.name.getColumnFamily()); }
+    ;
+
+roleResource returns [RoleResource res]
+    : K_ALL K_ROLES { $res = RoleResource.root(); }
+    | K_ROLE role = userOrRoleName { $res = RoleResource.role($role.name.getName()); }
+    ;
+
+functionResource returns [FunctionResource res]
+    @init {
+        List<CQL3Type.Raw> argsTypes = new ArrayList<>();
+    }
+    : K_ALL K_FUNCTIONS { $res = FunctionResource.root(); }
+    | K_ALL K_FUNCTIONS K_IN K_KEYSPACE ks = keyspaceName { $res = FunctionResource.keyspace($ks.id); }
+    // Arg types are mandatory for DCL statements on Functions
+    | K_FUNCTION fn=functionName
+      (
+        '('
+          (
+            v=comparatorType { argsTypes.add(v); }
+            ( ',' v=comparatorType { argsTypes.add(v); } )*
+          )?
+        ')'
+      )
+      { $res = FunctionResource.functionFromCql($fn.s.keyspace, $fn.s.name, argsTypes); }
     ;
 
 /**
  * CREATE USER [IF NOT EXISTS] <username> [WITH PASSWORD <password>] [SUPERUSER|NOSUPERUSER]
  */
-createUserStatement returns [CreateUserStatement stmt]
+createUserStatement returns [CreateRoleStatement stmt]
     @init {
-        UserOptions opts = new UserOptions();
+        RoleOptions opts = new RoleOptions();
+        opts.setOption(IRoleManager.Option.LOGIN, true);
         boolean superuser = false;
         boolean ifNotExists = false;
+        RoleName name = new RoleName();
     }
-    : K_CREATE K_USER (K_IF K_NOT K_EXISTS { ifNotExists = true; })? username
-      ( K_WITH userOptions[opts] )?
+    : K_CREATE K_USER (K_IF K_NOT K_EXISTS { ifNotExists = true; })? u=username { name.setName($u.text, true); }
+      ( K_WITH userPassword[opts] )?
       ( K_SUPERUSER { superuser = true; } | K_NOSUPERUSER { superuser = false; } )?
-      { $stmt = new CreateUserStatement($username.text, opts, superuser, ifNotExists); }
+      { opts.setOption(IRoleManager.Option.SUPERUSER, superuser);
+        $stmt = new CreateRoleStatement(name, opts, ifNotExists); }
     ;
 
 /**
  * ALTER USER <username> [WITH PASSWORD <password>] [SUPERUSER|NOSUPERUSER]
  */
-alterUserStatement returns [AlterUserStatement stmt]
+alterUserStatement returns [AlterRoleStatement stmt]
     @init {
-        UserOptions opts = new UserOptions();
-        Boolean superuser = null;
+        RoleOptions opts = new RoleOptions();
+        RoleName name = new RoleName();
     }
-    : K_ALTER K_USER username
-      ( K_WITH userOptions[opts] )?
-      ( K_SUPERUSER { superuser = true; } | K_NOSUPERUSER { superuser = false; } )?
-      { $stmt = new AlterUserStatement($username.text, opts, superuser); }
+    : K_ALTER K_USER u=username { name.setName($u.text, true); }
+      ( K_WITH userPassword[opts] )?
+      ( K_SUPERUSER { opts.setOption(IRoleManager.Option.SUPERUSER, true); }
+        | K_NOSUPERUSER { opts.setOption(IRoleManager.Option.SUPERUSER, false); } ) ?
+      {  $stmt = new AlterRoleStatement(name, opts); }
     ;
 
 /**
  * DROP USER [IF EXISTS] <username>
  */
-dropUserStatement returns [DropUserStatement stmt]
-    @init { boolean ifExists = false; }
-    : K_DROP K_USER (K_IF K_EXISTS { ifExists = true; })? username { $stmt = new DropUserStatement($username.text, ifExists); }
+dropUserStatement returns [DropRoleStatement stmt]
+    @init {
+        boolean ifExists = false;
+        RoleName name = new RoleName();
+    }
+    : K_DROP K_USER (K_IF K_EXISTS { ifExists = true; })? u=username { name.setName($u.text, true); $stmt = new DropRoleStatement(name, ifExists); }
     ;
 
 /**
  * LIST USERS
  */
-listUsersStatement returns [ListUsersStatement stmt]
+listUsersStatement returns [ListRolesStatement stmt]
     : K_LIST K_USERS { $stmt = new ListUsersStatement(); }
     ;
 
-userOptions[UserOptions opts]
-    : userOption[opts]
+/**
+ * CREATE ROLE [IF NOT EXISTS] <rolename> [ [WITH] option [ [AND] option ]* ]
+ *
+ * where option can be:
+ *  PASSWORD = '<password>'
+ *  SUPERUSER = (true|false)
+ *  LOGIN = (true|false)
+ *  OPTIONS = { 'k1':'v1', 'k2':'v2'}
+ */
+createRoleStatement returns [CreateRoleStatement stmt]
+    @init {
+        RoleOptions opts = new RoleOptions();
+        boolean ifNotExists = false;
+    }
+    : K_CREATE K_ROLE (K_IF K_NOT K_EXISTS { ifNotExists = true; })? name=userOrRoleName
+      ( K_WITH roleOptions[opts] )?
+      {
+        // set defaults if they weren't explictly supplied
+        if (!opts.getLogin().isPresent())
+        {
+            opts.setOption(IRoleManager.Option.LOGIN, false);
+        }
+        if (!opts.getSuperuser().isPresent())
+        {
+            opts.setOption(IRoleManager.Option.SUPERUSER, false);
+        }
+        $stmt = new CreateRoleStatement(name, opts, ifNotExists);
+      }
     ;
 
-userOption[UserOptions opts]
-    : k=K_PASSWORD v=STRING_LITERAL { opts.put($k.text, $v.text); }
+/**
+ * ALTER ROLE <rolename> [ [WITH] option [ [AND] option ]* ]
+ *
+ * where option can be:
+ *  PASSWORD = '<password>'
+ *  SUPERUSER = (true|false)
+ *  LOGIN = (true|false)
+ *  OPTIONS = { 'k1':'v1', 'k2':'v2'}
+ */
+alterRoleStatement returns [AlterRoleStatement stmt]
+    @init {
+        RoleOptions opts = new RoleOptions();
+    }
+    : K_ALTER K_ROLE name=userOrRoleName
+      ( K_WITH roleOptions[opts] )?
+      {  $stmt = new AlterRoleStatement(name, opts); }
+    ;
+
+/**
+ * DROP ROLE [IF EXISTS] <rolename>
+ */
+dropRoleStatement returns [DropRoleStatement stmt]
+    @init {
+        boolean ifExists = false;
+    }
+    : K_DROP K_ROLE (K_IF K_EXISTS { ifExists = true; })? name=userOrRoleName
+      { $stmt = new DropRoleStatement(name, ifExists); }
+    ;
+
+/**
+ * LIST ROLES [OF <rolename>] [NORECURSIVE]
+ */
+listRolesStatement returns [ListRolesStatement stmt]
+    @init {
+        boolean recursive = true;
+        RoleName grantee = new RoleName();
+    }
+    : K_LIST K_ROLES
+      ( K_OF roleName[grantee])?
+      ( K_NORECURSIVE { recursive = false; } )?
+      { $stmt = new ListRolesStatement(grantee, recursive); }
+    ;
+
+roleOptions[RoleOptions opts]
+    : roleOption[opts] (K_AND roleOption[opts])*
+    ;
+
+roleOption[RoleOptions opts]
+    :  K_PASSWORD '=' v=STRING_LITERAL { opts.setOption(IRoleManager.Option.PASSWORD, $v.text); }
+    |  K_OPTIONS '=' m=mapLiteral { opts.setOption(IRoleManager.Option.OPTIONS, convertPropertyMap(m)); }
+    |  K_SUPERUSER '=' b=BOOLEAN { opts.setOption(IRoleManager.Option.SUPERUSER, Boolean.valueOf($b.text)); }
+    |  K_LOGIN '=' b=BOOLEAN { opts.setOption(IRoleManager.Option.LOGIN, Boolean.valueOf($b.text)); }
+    ;
+
+// for backwards compatibility in CREATE/ALTER USER, this has no '='
+userPassword[RoleOptions opts]
+    :  K_PASSWORD v=STRING_LITERAL { opts.setOption(IRoleManager.Option.PASSWORD, $v.text); }
     ;
 
 /** DEFINITIONS **/
@@ -853,6 +1134,11 @@
     : (ks=ident '.')? ut=non_type_ident { return new UTName(ks, ut); }
     ;
 
+userOrRoleName returns [RoleName name]
+    @init { $name = new RoleName(); }
+    : roleName[name] {return $name;}
+    ;
+
 ksName[KeyspaceElementName name]
     : t=IDENT              { $name.setKeyspace($t.text, false);}
     | t=QUOTED_NAME        { $name.setKeyspace($t.text, true);}
@@ -874,6 +1160,14 @@
     | QMARK {addRecognitionError("Bind variables cannot be used for index names");}
     ;
 
+roleName[RoleName name]
+    : t=IDENT              { $name.setName($t.text, false); }
+    | s=STRING_LITERAL     { $name.setName($s.text, true); }
+    | t=QUOTED_NAME        { $name.setName($t.text, true); }
+    | k=unreserved_keyword { $name.setName(k, false); }
+    | QMARK {addRecognitionError("Bind variables cannot be used for role names");}
+    ;
+
 constant returns [Constants.Literal constant]
     : t=STRING_LITERAL { $constant = Constants.Literal.string($t.text); }
     | t=INTEGER        { $constant = Constants.Literal.integer($t.text); }
@@ -939,10 +1233,16 @@
     | QMARK         { $value = newBindVariables(null); }
     ;
 
-functionName returns [String s]
-    : f=IDENT                       { $s = $f.text; }
+functionName returns [FunctionName s]
+    : (ks=keyspaceName '.')? f=allowedFunctionName   { $s = new FunctionName(ks, f); }
+    ;
+
+allowedFunctionName returns [String s]
+    : f=IDENT                       { $s = $f.text.toLowerCase(); }
+    | f=QUOTED_NAME                 { $s = $f.text; }
     | u=unreserved_function_keyword { $s = u; }
     | K_TOKEN                       { $s = "token"; }
+    | K_COUNT                       { $s = "count"; }
     ;
 
 function returns [Term.Raw t]
@@ -1051,16 +1351,14 @@
 relation[List<Relation> clauses]
     : name=cident type=relationType t=term { $clauses.add(new SingleColumnRelation(name, type, t)); }
     | K_TOKEN l=tupleOfIdentifiers type=relationType t=term
-        {
-            for (ColumnIdentifier.Raw id : l)
-                $clauses.add(new SingleColumnRelation(id, type, t, true));
-        }
+        { $clauses.add(new TokenRelation(l, type, t)); }
     | name=cident K_IN marker=inMarker
         { $clauses.add(new SingleColumnRelation(name, Operator.IN, marker)); }
     | name=cident K_IN inValues=singleColumnInValues
         { $clauses.add(SingleColumnRelation.createInRelation($name.id, inValues)); }
     | name=cident K_CONTAINS { Operator rt = Operator.CONTAINS; } (K_KEY { rt = Operator.CONTAINS_KEY; })?
         t=term { $clauses.add(new SingleColumnRelation(name, rt, t)); }
+    | name=cident '[' key=term ']' type=relationType t=term { $clauses.add(new SingleColumnRelation(name, key, type, t)); }
     | ids=tupleOfIdentifiers
       ( K_IN
           ( '(' ')'
@@ -1155,12 +1453,16 @@
     | K_FLOAT     { $t = CQL3Type.Native.FLOAT; }
     | K_INET      { $t = CQL3Type.Native.INET;}
     | K_INT       { $t = CQL3Type.Native.INT; }
+    | K_SMALLINT  { $t = CQL3Type.Native.SMALLINT; }
     | K_TEXT      { $t = CQL3Type.Native.TEXT; }
     | K_TIMESTAMP { $t = CQL3Type.Native.TIMESTAMP; }
+    | K_TINYINT   { $t = CQL3Type.Native.TINYINT; }
     | K_UUID      { $t = CQL3Type.Native.UUID; }
     | K_VARCHAR   { $t = CQL3Type.Native.VARCHAR; }
     | K_VARINT    { $t = CQL3Type.Native.VARINT; }
     | K_TIMEUUID  { $t = CQL3Type.Native.TIMEUUID; }
+    | K_DATE      { $t = CQL3Type.Native.DATE; }
+    | K_TIME      { $t = CQL3Type.Native.TIME; }
     ;
 
 collection_type returns [CQL3Type.Raw pt]
@@ -1185,6 +1487,7 @@
 username
     : IDENT
     | STRING_LITERAL
+    | QUOTED_NAME { addRecognitionError("Quoted strings are are not supported for user names and USER is deprecated, please use ROLE");}
     ;
 
 // Basically the same as cident, but we need to exlude existing CQL3 types
@@ -1223,8 +1526,13 @@
         | K_ALL
         | K_USER
         | K_USERS
+        | K_ROLE
+        | K_ROLES
         | K_SUPERUSER
         | K_NOSUPERUSER
+        | K_LOGIN
+        | K_NOLOGIN
+        | K_OPTIONS
         | K_PASSWORD
         | K_EXISTS
         | K_CUSTOM
@@ -1234,10 +1542,21 @@
         | K_STATIC
         | K_FROZEN
         | K_TUPLE
+        | K_FUNCTION
+        | K_FUNCTIONS
+        | K_AGGREGATE
+        | K_SFUNC
+        | K_STYPE
+        | K_FINALFUNC
+        | K_INITCOND
+        | K_RETURNS
+        | K_LANGUAGE
+        | K_JSON
+        | K_CALLED
+        | K_INPUT
         ) { $str = $k.text; }
     ;
 
-
 // Case-insensitive keywords
 K_SELECT:      S E L E C T;
 K_FROM:        F R O M;
@@ -1246,6 +1565,7 @@
 K_AND:         A N D;
 K_KEY:         K E Y;
 K_KEYS:        K E Y S;
+K_ENTRIES:     E N T R I E S;
 K_FULL:        F U L L;
 K_INSERT:      I N S E R T;
 K_UPDATE:      U P D A T E;
@@ -1302,13 +1622,20 @@
 K_REVOKE:      R E V O K E;
 K_MODIFY:      M O D I F Y;
 K_AUTHORIZE:   A U T H O R I Z E;
+K_DESCRIBE:    D E S C R I B E;
+K_EXECUTE:     E X E C U T E;
 K_NORECURSIVE: N O R E C U R S I V E;
 
 K_USER:        U S E R;
 K_USERS:       U S E R S;
+K_ROLE:        R O L E;
+K_ROLES:       R O L E S;
 K_SUPERUSER:   S U P E R U S E R;
 K_NOSUPERUSER: N O S U P E R U S E R;
 K_PASSWORD:    P A S S W O R D;
+K_LOGIN:       L O G I N;
+K_NOLOGIN:     N O L O G I N;
+K_OPTIONS:     O P T I O N S;
 
 K_CLUSTERING:  C L U S T E R I N G;
 K_ASCII:       A S C I I;
@@ -1321,6 +1648,8 @@
 K_FLOAT:       F L O A T;
 K_INET:        I N E T;
 K_INT:         I N T;
+K_SMALLINT:    S M A L L I N T;
+K_TINYINT:     T I N Y I N T;
 K_TEXT:        T E X T;
 K_UUID:        U U I D;
 K_VARCHAR:     V A R C H A R;
@@ -1328,6 +1657,8 @@
 K_TIMEUUID:    T I M E U U I D;
 K_TOKEN:       T O K E N;
 K_WRITETIME:   W R I T E T I M E;
+K_DATE:        D A T E;
+K_TIME:        T I M E;
 
 K_NULL:        N U L L;
 K_NOT:         N O T;
@@ -1343,6 +1674,22 @@
 K_STATIC:      S T A T I C;
 K_FROZEN:      F R O Z E N;
 
+K_FUNCTION:    F U N C T I O N;
+K_FUNCTIONS:   F U N C T I O N S;
+K_AGGREGATE:   A G G R E G A T E;
+K_SFUNC:       S F U N C;
+K_STYPE:       S T Y P E;
+K_FINALFUNC:   F I N A L F U N C;
+K_INITCOND:    I N I T C O N D;
+K_RETURNS:     R E T U R N S;
+K_CALLED:      C A L L E D;
+K_INPUT:       I N P U T;
+K_LANGUAGE:    L A N G U A G E;
+K_OR:          O R;
+K_REPLACE:     R E P L A C E;
+
+K_JSON:        J S O N;
+
 // Case-insensitive alpha characters
 fragment A: ('a'|'A');
 fragment B: ('b'|'B');
@@ -1372,9 +1719,26 @@
 fragment Z: ('z'|'Z');
 
 STRING_LITERAL
-    @init{ StringBuilder b = new StringBuilder(); }
-    @after{ setText(b.toString()); }
-    : '\'' (c=~('\'') { b.appendCodePoint(c);} | '\'' '\'' { b.appendCodePoint('\''); })* '\''
+    @init{
+        StringBuilder txt = new StringBuilder(); // temporary to build pg-style-string
+    }
+    @after{ setText(txt.toString()); }
+    :
+      /* pg-style string literal */
+      (
+        '\$' '\$'
+        ( /* collect all input until '$$' is reached again */
+          {  (input.size() - input.index() > 1)
+               && !"$$".equals(input.substring(input.index(), input.index() + 1)) }?
+             => c=. { txt.appendCodePoint(c); }
+        )*
+        '\$' '\$'
+      )
+      |
+      /* conventional quoted string literal */
+      (
+        '\'' (c=~('\'') { txt.appendCodePoint(c);} | '\'' '\'' { txt.appendCodePoint('\''); })* '\''
+      )
     ;
 
 QUOTED_NAME
diff --git a/src/java/org/apache/cassandra/cql3/CustomPayloadMirroringQueryHandler.java b/src/java/org/apache/cassandra/cql3/CustomPayloadMirroringQueryHandler.java
new file mode 100644
index 0000000..02a6df9
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/CustomPayloadMirroringQueryHandler.java
@@ -0,0 +1,84 @@
+/*
+ * 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.cql3;
+
+import java.nio.ByteBuffer;
+import java.util.Map;
+
+import org.apache.cassandra.cql3.statements.BatchStatement;
+import org.apache.cassandra.cql3.statements.ParsedStatement;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.MD5Digest;
+
+/**
+ * Custom QueryHandler that sends custom request payloads back with the result.
+ * Used to facilitate testing.
+ * Enabled with system property cassandra.custom_query_handler_class.
+ */
+public class CustomPayloadMirroringQueryHandler implements QueryHandler
+{
+    static QueryProcessor queryProcessor = QueryProcessor.instance;
+
+    public ResultMessage process(String query,
+                                 QueryState state,
+                                 QueryOptions options,
+                                 Map<String, ByteBuffer> customPayload)
+    {
+        ResultMessage result = queryProcessor.process(query, state, options, customPayload);
+        result.setCustomPayload(customPayload);
+        return result;
+    }
+
+    public ResultMessage.Prepared prepare(String query, QueryState state, Map<String, ByteBuffer> customPayload)
+    {
+        ResultMessage.Prepared prepared = queryProcessor.prepare(query, state, customPayload);
+        prepared.setCustomPayload(customPayload);
+        return prepared;
+    }
+
+    public ParsedStatement.Prepared getPrepared(MD5Digest id)
+    {
+        return queryProcessor.getPrepared(id);
+    }
+
+    public ParsedStatement.Prepared getPreparedForThrift(Integer id)
+    {
+        return queryProcessor.getPreparedForThrift(id);
+    }
+
+    public ResultMessage processPrepared(CQLStatement statement,
+                                         QueryState state,
+                                         QueryOptions options,
+                                         Map<String, ByteBuffer> customPayload)
+    {
+        ResultMessage result = queryProcessor.processPrepared(statement, state, options, customPayload);
+        result.setCustomPayload(customPayload);
+        return result;
+    }
+
+    public ResultMessage processBatch(BatchStatement statement,
+                                      QueryState state,
+                                      BatchQueryOptions options,
+                                      Map<String, ByteBuffer> customPayload)
+    {
+        ResultMessage result = queryProcessor.processBatch(statement, state, options, customPayload);
+        result.setCustomPayload(customPayload);
+        return result;
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/ErrorCollector.java b/src/java/org/apache/cassandra/cql3/ErrorCollector.java
index 2137da2..f49cca4 100644
--- a/src/java/org/apache/cassandra/cql3/ErrorCollector.java
+++ b/src/java/org/apache/cassandra/cql3/ErrorCollector.java
@@ -91,14 +91,14 @@
     }
 
     /**
-     * Throws the last syntax error found by the lexer or the parser if it exists.
+     * Throws the first syntax error found by the lexer or the parser if it exists.
      *
      * @throws SyntaxException the syntax error.
      */
-    public void throwLastSyntaxError() throws SyntaxException
+    public void throwFirstSyntaxError() throws SyntaxException
     {
         if (!errorMsgs.isEmpty())
-            throw new SyntaxException(errorMsgs.getLast());
+            throw new SyntaxException(errorMsgs.getFirst());
     }
 
     /**
@@ -146,7 +146,9 @@
         if (!includeQueryStart)
             builder.append("...");
 
-        lines[lineIndex(to)] = lines[lineIndex(to)].substring(0, getLastCharPositionInLine(to));
+        String toLine = lines[lineIndex(to)];
+        int toEnd = getLastCharPositionInLine(to);
+        lines[lineIndex(to)] = toEnd >= toLine.length() ? toLine : toLine.substring(0, toEnd);
         lines[lineIndex(offending)] = highlightToken(lines[lineIndex(offending)], offending);
         lines[lineIndex(from)] = lines[lineIndex(from)].substring(from.getCharPositionInLine());
 
@@ -163,7 +165,8 @@
      * Checks if the specified tokens are valid.
      *
      * @param tokens the tokens to check
-     * @return <code>true</code> if all the specified tokens are valid ones, <code>false</code> otherwise.
+     * @return <code>true</code> if all the specified tokens are valid ones,
+     * <code>false</code> otherwise.
      */
     private static boolean areTokensValid(Token... tokens)
     {
diff --git a/src/java/org/apache/cassandra/cql3/Json.java b/src/java/org/apache/cassandra/cql3/Json.java
new file mode 100644
index 0000000..78fbd08
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/Json.java
@@ -0,0 +1,333 @@
+/*
+ * 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.cql3;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.serializers.MarshalException;
+import org.codehaus.jackson.io.JsonStringEncoder;
+import org.codehaus.jackson.map.ObjectMapper;
+
+/** Term-related classes for INSERT JSON support. */
+public class Json
+{
+    public static final ObjectMapper JSON_OBJECT_MAPPER = new ObjectMapper();
+
+    public static final ColumnIdentifier JSON_COLUMN_ID = new ColumnIdentifier("[json]", true);
+
+    /**
+     * Quotes string contents using standard JSON quoting.
+     */
+    public static String quoteAsJsonString(String s)
+    {
+        return new String(JsonStringEncoder.getInstance().quoteAsString(s));
+    }
+
+    public static Object decodeJson(String json)
+    {
+        try
+        {
+            return JSON_OBJECT_MAPPER.readValue(json, Object.class);
+        }
+        catch (IOException exc)
+        {
+            throw new MarshalException("Error decoding JSON string: " + exc.getMessage());
+        }
+    }
+
+    public interface Raw
+    {
+        public Prepared prepareAndCollectMarkers(CFMetaData metadata, Collection<ColumnDefinition> receivers, VariableSpecifications boundNames);
+    }
+
+    /**
+     * Represents a literal JSON string in an INSERT JSON statement.
+     * For example: INSERT INTO mytable (key, col) JSON '{"key": 0, "col": 0}';
+     */
+    public static class Literal implements Raw
+    {
+        private final String text;
+
+        public Literal(String text)
+        {
+            this.text = text;
+        }
+
+        public Prepared prepareAndCollectMarkers(CFMetaData metadata, Collection<ColumnDefinition> receivers, VariableSpecifications boundNames)
+        {
+            return new PreparedLiteral(metadata.ksName, parseJson(text, receivers));
+        }
+    }
+
+    /**
+     * Represents a marker for a JSON string in an INSERT JSON statement.
+     * For example: INSERT INTO mytable (key, col) JSON ?;
+     */
+    public static class Marker implements Raw
+    {
+        protected final int bindIndex;
+
+        public Marker(int bindIndex)
+        {
+            this.bindIndex = bindIndex;
+        }
+
+        public Prepared prepareAndCollectMarkers(CFMetaData metadata, Collection<ColumnDefinition> receivers, VariableSpecifications boundNames)
+        {
+            boundNames.add(bindIndex, makeReceiver(metadata));
+            return new PreparedMarker(metadata.ksName, bindIndex, receivers);
+        }
+
+        private ColumnSpecification makeReceiver(CFMetaData metadata)
+        {
+            return new ColumnSpecification(metadata.ksName, metadata.cfName, JSON_COLUMN_ID, UTF8Type.instance);
+        }
+    }
+
+    /**
+     * A prepared, full set of JSON values.
+     */
+    public static abstract class Prepared
+    {
+        private final String keyspace;
+
+        protected Prepared(String keyspace)
+        {
+            this.keyspace = keyspace;
+        }
+
+        protected abstract Term.Raw getRawTermForColumn(ColumnDefinition def);
+
+        public Term getPrimaryKeyValueForColumn(ColumnDefinition def)
+        {
+            // Note that we know we don't have to call collectMarkerSpecification since it has already been collected
+            return getRawTermForColumn(def).prepare(keyspace, def);
+        }
+
+        public Operation getSetOperationForColumn(ColumnDefinition def)
+        {
+            // Note that we know we don't have to call collectMarkerSpecification on the operation since we have
+            // already collected all we need.
+            return new Operation.SetValue(getRawTermForColumn(def)).prepare(keyspace, def);
+        }
+    }
+
+    /**
+     * A prepared literal set of JSON values
+     */
+    private static class PreparedLiteral extends Prepared
+    {
+        private final Map<ColumnIdentifier, Term> columnMap;
+
+        public PreparedLiteral(String keyspace, Map<ColumnIdentifier, Term> columnMap)
+        {
+            super(keyspace);
+            this.columnMap = columnMap;
+        }
+
+        protected Term.Raw getRawTermForColumn(ColumnDefinition def)
+        {
+            Term value = columnMap.get(def.name);
+            return value == null ? Constants.NULL_LITERAL : new ColumnValue(value);
+        }
+    }
+
+    /**
+     *  A prepared bind marker for a set of JSON values
+     */
+    private static class PreparedMarker extends Prepared
+    {
+        private final int bindIndex;
+        private final Collection<ColumnDefinition> columns;
+
+        public PreparedMarker(String keyspace, int bindIndex, Collection<ColumnDefinition> columns)
+        {
+            super(keyspace);
+            this.bindIndex = bindIndex;
+            this.columns = columns;
+        }
+
+        protected DelayedColumnValue getRawTermForColumn(ColumnDefinition def)
+        {
+            return new DelayedColumnValue(this, def);
+        }
+    }
+
+    /**
+     * A Terminal for a single column.
+     *
+     * Note that this is intrinsically an already prepared term, but this still implements Term.Raw so that we can
+     * easily use it to create raw operations.
+     */
+    private static class ColumnValue implements Term.Raw
+    {
+        private final Term term;
+
+        public ColumnValue(Term term)
+        {
+            this.term = term;
+        }
+
+        @Override
+        public Term prepare(String keyspace, ColumnSpecification receiver) throws InvalidRequestException
+        {
+            return term;
+        }
+
+        @Override
+        public TestResult testAssignment(String keyspace, ColumnSpecification receiver)
+        {
+            return TestResult.NOT_ASSIGNABLE;
+        }
+    }
+
+    /**
+     * A NonTerminal for a single column.
+     *
+     * As with {@code ColumnValue}, this is intrinsically a prepared term but implements Terms.Raw for convenience.
+     */
+    private static class DelayedColumnValue extends Term.NonTerminal implements Term.Raw
+    {
+        private final PreparedMarker marker;
+        private final ColumnDefinition column;
+
+        public DelayedColumnValue(PreparedMarker prepared, ColumnDefinition column)
+        {
+            this.marker = prepared;
+            this.column = column;
+        }
+
+        @Override
+        public Term prepare(String keyspace, ColumnSpecification receiver) throws InvalidRequestException
+        {
+            return this;
+        }
+
+        @Override
+        public TestResult testAssignment(String keyspace, ColumnSpecification receiver)
+        {
+            return TestResult.WEAKLY_ASSIGNABLE;
+        }
+
+        @Override
+        public void collectMarkerSpecification(VariableSpecifications boundNames)
+        {
+            // We've already collected what we should (and in practice this method is never called).
+        }
+
+        @Override
+        public boolean containsBindMarker()
+        {
+            return true;
+        }
+
+        @Override
+        public Terminal bind(QueryOptions options) throws InvalidRequestException
+        {
+            Term term = options.getJsonColumnValue(marker.bindIndex, column.name, marker.columns);
+            return term == null ? null : term.bind(options);
+        }
+
+        @Override
+        public Iterable<Function> getFunctions()
+        {
+            return Collections.emptyList();
+        }
+    }
+
+    /**
+     * Given a JSON string, return a map of columns to their values for the insert.
+     */
+    public static Map<ColumnIdentifier, Term> parseJson(String jsonString, Collection<ColumnDefinition> expectedReceivers)
+    {
+        try
+        {
+            Map<String, Object> valueMap = JSON_OBJECT_MAPPER.readValue(jsonString, Map.class);
+
+            if (valueMap == null)
+                throw new InvalidRequestException("Got null for INSERT JSON values");
+
+            handleCaseSensitivity(valueMap);
+
+            Map<ColumnIdentifier, Term> columnMap = new HashMap<>(expectedReceivers.size());
+            for (ColumnSpecification spec : expectedReceivers)
+            {
+                Object parsedJsonObject = valueMap.remove(spec.name.toString());
+                if (parsedJsonObject == null)
+                {
+                    columnMap.put(spec.name, null);
+                }
+                else
+                {
+                    try
+                    {
+                        columnMap.put(spec.name, spec.type.fromJSONObject(parsedJsonObject));
+                    }
+                    catch(MarshalException exc)
+                    {
+                        throw new InvalidRequestException(String.format("Error decoding JSON value for %s: %s", spec.name, exc.getMessage()));
+                    }
+                }
+            }
+
+            if (!valueMap.isEmpty())
+            {
+                throw new InvalidRequestException(String.format(
+                        "JSON values map contains unrecognized column: %s", valueMap.keySet().iterator().next()));
+            }
+
+            return columnMap;
+        }
+        catch (IOException exc)
+        {
+            throw new InvalidRequestException(String.format("Could not decode JSON string as a map: %s. (String was: %s)", exc.toString(), jsonString));
+        }
+        catch (MarshalException exc)
+        {
+            throw new InvalidRequestException(exc.getMessage());
+        }
+    }
+
+    /**
+     * Handles unquoting and case-insensitivity in map keys.
+     */
+    public static void handleCaseSensitivity(Map<String, Object> valueMap)
+    {
+        for (String mapKey : new ArrayList<>(valueMap.keySet()))
+        {
+            // if it's surrounded by quotes, remove them and preserve the case
+            if (mapKey.startsWith("\"") && mapKey.endsWith("\""))
+            {
+                valueMap.put(mapKey.substring(1, mapKey.length() - 1), valueMap.remove(mapKey));
+                continue;
+            }
+
+            // otherwise, lowercase it if needed
+            String lowered = mapKey.toLowerCase(Locale.US);
+            if (!mapKey.equals(lowered))
+                valueMap.put(lowered, valueMap.remove(mapKey));
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/Lists.java b/src/java/org/apache/cassandra/cql3/Lists.java
index a939a27..cc75476 100644
--- a/src/java/org/apache/cassandra/cql3/Lists.java
+++ b/src/java/org/apache/cassandra/cql3/Lists.java
@@ -17,16 +17,22 @@
  */
 package org.apache.cassandra.cql3;
 
+import static org.apache.cassandra.cql3.Constants.UNSET_VALUE;
+
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicReference;
 
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.db.Cell;
 import org.apache.cassandra.db.ColumnFamily;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.composites.Composite;
+import org.apache.cassandra.db.composites.CompositesBuilder;
 import org.apache.cassandra.db.marshal.Int32Type;
 import org.apache.cassandra.db.marshal.ListType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
@@ -36,16 +42,12 @@
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.UUIDGen;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * Static helper methods and classes for lists.
  */
 public abstract class Lists
 {
-    private static final Logger logger = LoggerFactory.getLogger(Lists.class);
-
     private Lists() {}
 
     public static ColumnSpecification indexSpecOf(ColumnSpecification column)
@@ -98,22 +100,22 @@
             ColumnSpecification valueSpec = Lists.valueSpecOf(receiver);
             for (Term.Raw rt : elements)
             {
-                if (!rt.isAssignableTo(keyspace, valueSpec))
+                if (!rt.testAssignment(keyspace, valueSpec).isAssignable())
                     throw new InvalidRequestException(String.format("Invalid list literal for %s: value %s is not of type %s", receiver.name, rt, valueSpec.type.asCQL3Type()));
             }
         }
 
-        public boolean isAssignableTo(String keyspace, ColumnSpecification receiver)
+        public AssignmentTestable.TestResult testAssignment(String keyspace, ColumnSpecification receiver)
         {
-            try
-            {
-                validateAssignableTo(keyspace, receiver);
-                return true;
-            }
-            catch (InvalidRequestException e)
-            {
-                return false;
-            }
+            if (!(receiver.type instanceof ListType))
+                return AssignmentTestable.TestResult.NOT_ASSIGNABLE;
+
+            // If there is no elements, we can't say it's an exact match (an empty list if fundamentally polymorphic).
+            if (elements.isEmpty())
+                return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
+
+            ColumnSpecification valueSpec = Lists.valueSpecOf(receiver);
+            return AssignmentTestable.TestResult.testAll(keyspace, valueSpec, elements);
         }
 
         @Override
@@ -123,7 +125,7 @@
         }
     }
 
-    public static class Value extends Term.MultiItemTerminal implements Term.CollectionTerminal
+    public static class Value extends Term.MultiItemTerminal
     {
         public final List<ByteBuffer> elements;
 
@@ -138,8 +140,8 @@
             {
                 // Collections have this small hack that validate cannot be called on a serialized object,
                 // but compose does the validation (so we're fine).
-                List<?> l = (List<?>)type.getSerializer().deserializeForNativeProtocol(value, version);
-                List<ByteBuffer> elements = new ArrayList<ByteBuffer>(l.size());
+                List<?> l = type.getSerializer().deserializeForNativeProtocol(value, version);
+                List<ByteBuffer> elements = new ArrayList<>(l.size());
                 for (Object element : l)
                     // elements can be null in lists that represent a set of IN values
                     elements.add(element == null ? null : type.getElementsType().decompose(element));
@@ -151,12 +153,7 @@
             }
         }
 
-        public ByteBuffer get(QueryOptions options)
-        {
-            return getWithProtocolVersion(options.getProtocolVersion());
-        }
-
-        public ByteBuffer getWithProtocolVersion(int protocolVersion)
+        public ByteBuffer get(int protocolVersion)
         {
             return CollectionSerializer.pack(elements, elements.size(), protocolVersion);
         }
@@ -207,7 +204,7 @@
         {
         }
 
-        public Value bind(QueryOptions options) throws InvalidRequestException
+        public Terminal bind(QueryOptions options) throws InvalidRequestException
         {
             List<ByteBuffer> buffers = new ArrayList<ByteBuffer>(elements.size());
             for (Term t : elements)
@@ -216,6 +213,8 @@
 
                 if (bytes == null)
                     throw new InvalidRequestException("null is not supported inside collections");
+                if (bytes == ByteBufferUtil.UNSET_BYTE_BUFFER)
+                    return UNSET_VALUE;
 
                 // We don't support value > 64K because the serialization format encode the length as an unsigned short.
                 if (bytes.remaining() > FBUtilities.MAX_UNSIGNED_SHORT)
@@ -227,6 +226,11 @@
             }
             return new Value(buffers);
         }
+
+        public Iterable<Function> getFunctions()
+        {
+            return Terms.getFunctions(elements);
+        }
     }
 
     /**
@@ -240,10 +244,14 @@
             assert receiver.type instanceof ListType;
         }
 
-        public Value bind(QueryOptions options) throws InvalidRequestException
+        public Terminal bind(QueryOptions options) throws InvalidRequestException
         {
             ByteBuffer value = options.getValues().get(bindIndex);
-            return value == null ? null : Value.fromSerialized(value, (ListType)receiver.type, options.getProtocolVersion());
+            if (value == null)
+                return null;
+            if (value == ByteBufferUtil.UNSET_BYTE_BUFFER)
+                return UNSET_VALUE;
+            return Value.fromSerialized(value, (ListType)receiver.type, options.getProtocolVersion());
         }
     }
 
@@ -259,7 +267,7 @@
     {
         // Our reference time (1 jan 2010, 00:00:00) in milliseconds.
         private static final long REFERENCE_TIME = 1262304000000L;
-        private static final AtomicReference<PrecisionTime> last = new AtomicReference<PrecisionTime>(new PrecisionTime(Long.MAX_VALUE, 0));
+        private static final AtomicReference<PrecisionTime> last = new AtomicReference<>(new PrecisionTime(Long.MAX_VALUE, 0));
 
         public final long millis;
         public final int nanos;
@@ -296,13 +304,15 @@
 
         public void execute(ByteBuffer rowKey, ColumnFamily cf, Composite prefix, UpdateParameters params) throws InvalidRequestException
         {
-            if (column.type.isMultiCell())
+            Term.Terminal value = t.bind(params.options);
+            if (column.type.isMultiCell() && value != UNSET_VALUE)
             {
                 // delete + append
                 CellName name = cf.getComparator().create(prefix, column);
                 cf.addAtom(params.makeTombstoneForOverwrite(name.slice()));
             }
-            Appender.doAppend(t, cf, prefix, column, params);
+            if (value != UNSET_VALUE)
+                Appender.doAppend(cf, prefix, column, params, value);
         }
     }
 
@@ -339,8 +349,10 @@
 
             if (index == null)
                 throw new InvalidRequestException("Invalid null value for list index");
+            if (index == ByteBufferUtil.UNSET_BYTE_BUFFER)
+                throw new InvalidRequestException("Invalid unset value for list index");
 
-            List<Cell> existingList = params.getPrefetchedList(rowKey, column.name);
+            List<Cell> existingList = params.getPrefetchedList(rowKey, column.name, cf);
             int idx = ByteBufferUtil.toInt(index);
             if (existingList == null || existingList.size() == 0)
                 throw new InvalidRequestException("Attempted to set an element on a list which is null");
@@ -352,7 +364,7 @@
             {
                 cf.addColumn(params.makeTombstone(elementName));
             }
-            else
+            else if (value != ByteBufferUtil.UNSET_BYTE_BUFFER)
             {
                 // We don't support value > 64K because the serialization format encode the length as an unsigned short.
                 if (value.remaining() > FBUtilities.MAX_UNSIGNED_SHORT)
@@ -375,13 +387,13 @@
         public void execute(ByteBuffer rowKey, ColumnFamily cf, Composite prefix, UpdateParameters params) throws InvalidRequestException
         {
             assert column.type.isMultiCell() : "Attempted to append to a frozen list";
-            doAppend(t, cf, prefix, column, params);
+            Term.Terminal value = t.bind(params.options);
+            if (value != UNSET_VALUE)
+                doAppend(cf, prefix, column, params, value);
         }
 
-        static void doAppend(Term t, ColumnFamily cf, Composite prefix, ColumnDefinition column, UpdateParameters params) throws InvalidRequestException
+        static void doAppend(ColumnFamily cf, Composite prefix, ColumnDefinition column, UpdateParameters params, Term.Terminal value) throws InvalidRequestException
         {
-            Term.Terminal value = t.bind(params.options);
-            Lists.Value listValue = (Lists.Value)value;
             if (column.type.isMultiCell())
             {
                 // If we append null, do nothing. Note that for Setter, we've
@@ -389,11 +401,10 @@
                 if (value == null)
                     return;
 
-                List<ByteBuffer> toAdd = listValue.elements;
-                for (int i = 0; i < toAdd.size(); i++)
+                for (ByteBuffer buffer : ((Value) value).elements)
                 {
                     ByteBuffer uuid = ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes());
-                    cf.addColumn(params.makeColumn(cf.getComparator().create(prefix, column, uuid), toAdd.get(i)));
+                    cf.addColumn(params.makeColumn(cf.getComparator().create(prefix, column, uuid), buffer));
                 }
             }
             else
@@ -403,7 +414,7 @@
                 if (value == null)
                     cf.addAtom(params.makeTombstone(name));
                 else
-                    cf.addColumn(params.makeColumn(name, listValue.getWithProtocolVersion(Server.CURRENT_VERSION)));
+                    cf.addColumn(params.makeColumn(name, value.get(Server.CURRENT_VERSION)));
             }
         }
     }
@@ -419,13 +430,12 @@
         {
             assert column.type.isMultiCell() : "Attempted to prepend to a frozen list";
             Term.Terminal value = t.bind(params.options);
-            if (value == null)
+            if (value == null || value == UNSET_VALUE)
                 return;
 
-            assert value instanceof Lists.Value;
             long time = PrecisionTime.REFERENCE_TIME - (System.currentTimeMillis() - PrecisionTime.REFERENCE_TIME);
 
-            List<ByteBuffer> toAdd = ((Lists.Value)value).elements;
+            List<ByteBuffer> toAdd = ((Value) value).elements;
             for (int i = toAdd.size() - 1; i >= 0; i--)
             {
                 PrecisionTime pt = PrecisionTime.getNext(time);
@@ -451,7 +461,7 @@
         public void execute(ByteBuffer rowKey, ColumnFamily cf, Composite prefix, UpdateParameters params) throws InvalidRequestException
         {
             assert column.type.isMultiCell() : "Attempted to delete from a frozen list";
-            List<Cell> existingList = params.getPrefetchedList(rowKey, column.name);
+            List<Cell> existingList = params.getPrefetchedList(rowKey, column.name, cf);
             // We want to call bind before possibly returning to reject queries where the value provided is not a list.
             Term.Terminal value = t.bind(params.options);
 
@@ -460,16 +470,14 @@
             if (existingList.isEmpty())
                 return;
 
-            if (value == null)
+            if (value == null || value == UNSET_VALUE)
                 return;
 
-            assert value instanceof Lists.Value;
-
             // Note: below, we will call 'contains' on this toDiscard list for each element of existingList.
             // Meaning that if toDiscard is big, converting it to a HashSet might be more efficient. However,
             // the read-before-write this operation requires limits its usefulness on big lists, so in practice
             // toDiscard will be small and keeping a list will be more efficient.
-            List<ByteBuffer> toDiscard = ((Lists.Value)value).elements;
+            List<ByteBuffer> toDiscard = ((Value) value).elements;
             for (Cell cell : existingList)
             {
                 if (toDiscard.contains(cell.value()))
@@ -497,9 +505,12 @@
             Term.Terminal index = t.bind(params.options);
             if (index == null)
                 throw new InvalidRequestException("Invalid null value for list index");
+            if (index == Constants.UNSET_VALUE)
+                return;
 
-            List<Cell> existingList = params.getPrefetchedList(rowKey, column.name);
-            int idx = ByteBufferUtil.toInt(index.get(params.options));
+            List<Cell> existingList = params.getPrefetchedList(rowKey, column.name, cf);
+
+            int idx = ByteBufferUtil.toInt(index.get(params.options.getProtocolVersion()));
             if (existingList == null || existingList.size() == 0)
                 throw new InvalidRequestException("Attempted to delete an element from a list which is null");
             if (idx < 0 || idx >= existingList.size())
diff --git a/src/java/org/apache/cassandra/cql3/Maps.java b/src/java/org/apache/cassandra/cql3/Maps.java
index 8a64663..5bb3a48 100644
--- a/src/java/org/apache/cassandra/cql3/Maps.java
+++ b/src/java/org/apache/cassandra/cql3/Maps.java
@@ -17,17 +17,15 @@
  */
 package org.apache.cassandra.cql3;
 
+import static org.apache.cassandra.cql3.Constants.UNSET_VALUE;
+
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.TreeMap;
+import java.util.*;
+
+import com.google.common.collect.Iterables;
 
 import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.db.ColumnFamily;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.composites.Composite;
@@ -36,6 +34,7 @@
 import org.apache.cassandra.serializers.CollectionSerializer;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.transport.Server;
+import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Pair;
 
@@ -71,7 +70,7 @@
 
             ColumnSpecification keySpec = Maps.keySpecOf(receiver);
             ColumnSpecification valueSpec = Maps.valueSpecOf(receiver);
-            Map<Term, Term> values = new HashMap<Term, Term>(entries.size());
+            Map<Term, Term> values = new HashMap<>(entries.size());
             boolean allTerminal = true;
             for (Pair<Term.Raw, Term.Raw> entry : entries)
             {
@@ -99,24 +98,36 @@
             ColumnSpecification valueSpec = Maps.valueSpecOf(receiver);
             for (Pair<Term.Raw, Term.Raw> entry : entries)
             {
-                if (!entry.left.isAssignableTo(keyspace, keySpec))
+                if (!entry.left.testAssignment(keyspace, keySpec).isAssignable())
                     throw new InvalidRequestException(String.format("Invalid map literal for %s: key %s is not of type %s", receiver.name, entry.left, keySpec.type.asCQL3Type()));
-                if (!entry.right.isAssignableTo(keyspace, valueSpec))
+                if (!entry.right.testAssignment(keyspace, valueSpec).isAssignable())
                     throw new InvalidRequestException(String.format("Invalid map literal for %s: value %s is not of type %s", receiver.name, entry.right, valueSpec.type.asCQL3Type()));
             }
         }
 
-        public boolean isAssignableTo(String keyspace, ColumnSpecification receiver)
+        public AssignmentTestable.TestResult testAssignment(String keyspace, ColumnSpecification receiver)
         {
-            try
+            if (!(receiver.type instanceof MapType))
+                return AssignmentTestable.TestResult.NOT_ASSIGNABLE;
+
+            // If there is no elements, we can't say it's an exact match (an empty map if fundamentally polymorphic).
+            if (entries.isEmpty())
+                return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
+
+            ColumnSpecification keySpec = Maps.keySpecOf(receiver);
+            ColumnSpecification valueSpec = Maps.valueSpecOf(receiver);
+            // It's an exact match if all are exact match, but is not assignable as soon as any is non assignable.
+            AssignmentTestable.TestResult res = AssignmentTestable.TestResult.EXACT_MATCH;
+            for (Pair<Term.Raw, Term.Raw> entry : entries)
             {
-                validateAssignableTo(keyspace, receiver);
-                return true;
+                AssignmentTestable.TestResult t1 = entry.left.testAssignment(keyspace, keySpec);
+                AssignmentTestable.TestResult t2 = entry.right.testAssignment(keyspace, valueSpec);
+                if (t1 == AssignmentTestable.TestResult.NOT_ASSIGNABLE || t2 == AssignmentTestable.TestResult.NOT_ASSIGNABLE)
+                    return AssignmentTestable.TestResult.NOT_ASSIGNABLE;
+                if (t1 != AssignmentTestable.TestResult.EXACT_MATCH || t2 != AssignmentTestable.TestResult.EXACT_MATCH)
+                    res = AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
             }
-            catch (InvalidRequestException e)
-            {
-                return false;
-            }
+            return res;
         }
 
         @Override
@@ -134,7 +145,7 @@
         }
     }
 
-    public static class Value extends Term.Terminal implements Term.CollectionTerminal
+    public static class Value extends Term.Terminal
     {
         public final Map<ByteBuffer, ByteBuffer> map;
 
@@ -149,8 +160,8 @@
             {
                 // Collections have this small hack that validate cannot be called on a serialized object,
                 // but compose does the validation (so we're fine).
-                Map<?, ?> m = (Map<?, ?>)type.getSerializer().deserializeForNativeProtocol(value, version);
-                Map<ByteBuffer, ByteBuffer> map = new LinkedHashMap<ByteBuffer, ByteBuffer>(m.size());
+                Map<?, ?> m = type.getSerializer().deserializeForNativeProtocol(value, version);
+                Map<ByteBuffer, ByteBuffer> map = new LinkedHashMap<>(m.size());
                 for (Map.Entry<?, ?> entry : m.entrySet())
                     map.put(type.getKeysType().decompose(entry.getKey()), type.getValuesType().decompose(entry.getValue()));
                 return new Value(map);
@@ -161,12 +172,7 @@
             }
         }
 
-        public ByteBuffer get(QueryOptions options)
-        {
-            return getWithProtocolVersion(options.getProtocolVersion());
-        }
-
-        public ByteBuffer getWithProtocolVersion(int protocolVersion)
+        public ByteBuffer get(int protocolVersion)
         {
             List<ByteBuffer> buffers = new ArrayList<>(2 * map.size());
             for (Map.Entry<ByteBuffer, ByteBuffer> entry : map.entrySet())
@@ -219,7 +225,7 @@
         {
         }
 
-        public Value bind(QueryOptions options) throws InvalidRequestException
+        public Terminal bind(QueryOptions options) throws InvalidRequestException
         {
             Map<ByteBuffer, ByteBuffer> buffers = new TreeMap<ByteBuffer, ByteBuffer>(comparator);
             for (Map.Entry<Term, Term> entry : elements.entrySet())
@@ -228,6 +234,8 @@
                 ByteBuffer keyBytes = entry.getKey().bindAndGet(options);
                 if (keyBytes == null)
                     throw new InvalidRequestException("null is not supported inside collections");
+                if (keyBytes == ByteBufferUtil.UNSET_BYTE_BUFFER)
+                    throw new InvalidRequestException("unset value is not supported for map keys");
                 if (keyBytes.remaining() > FBUtilities.MAX_UNSIGNED_SHORT)
                     throw new InvalidRequestException(String.format("Map key is too long. Map keys are limited to %d bytes but %d bytes keys provided",
                                                                     FBUtilities.MAX_UNSIGNED_SHORT,
@@ -236,6 +244,9 @@
                 ByteBuffer valueBytes = entry.getValue().bindAndGet(options);
                 if (valueBytes == null)
                     throw new InvalidRequestException("null is not supported inside collections");
+                if (valueBytes == ByteBufferUtil.UNSET_BYTE_BUFFER)
+                    return UNSET_VALUE;
+
                 if (valueBytes.remaining() > FBUtilities.MAX_UNSIGNED_SHORT)
                     throw new InvalidRequestException(String.format("Map value is too long. Map values are limited to %d bytes but %d bytes value provided",
                                                                     FBUtilities.MAX_UNSIGNED_SHORT,
@@ -245,6 +256,12 @@
             }
             return new Value(buffers);
         }
+
+        public Iterable<Function> getFunctions()
+        {
+            return Iterables.concat(Terms.getFunctions(elements.keySet()),
+                                    Terms.getFunctions(elements.values()));
+        }
     }
 
     public static class Marker extends AbstractMarker
@@ -255,10 +272,14 @@
             assert receiver.type instanceof MapType;
         }
 
-        public Value bind(QueryOptions options) throws InvalidRequestException
+        public Terminal bind(QueryOptions options) throws InvalidRequestException
         {
             ByteBuffer value = options.getValues().get(bindIndex);
-            return value == null ? null : Value.fromSerialized(value, (MapType)receiver.type, options.getProtocolVersion());
+            if (value == null)
+                return null;
+            if (value == ByteBufferUtil.UNSET_BYTE_BUFFER)
+                return UNSET_VALUE;
+            return Value.fromSerialized(value, (MapType)receiver.type, options.getProtocolVersion());
         }
     }
 
@@ -271,13 +292,15 @@
 
         public void execute(ByteBuffer rowKey, ColumnFamily cf, Composite prefix, UpdateParameters params) throws InvalidRequestException
         {
-            if (column.type.isMultiCell())
+            Term.Terminal value = t.bind(params.options);
+            if (column.type.isMultiCell() && value != UNSET_VALUE)
             {
                 // delete + put
                 CellName name = cf.getComparator().create(prefix, column);
                 cf.addAtom(params.makeTombstoneForOverwrite(name.slice()));
             }
-            Putter.doPut(t, cf, prefix, column, params);
+            if (value != UNSET_VALUE)
+                Putter.doPut(cf, prefix, column, params, value);
         }
     }
 
@@ -305,6 +328,8 @@
             ByteBuffer value = t.bindAndGet(params.options);
             if (key == null)
                 throw new InvalidRequestException("Invalid null map key");
+            if (key == ByteBufferUtil.UNSET_BYTE_BUFFER)
+                throw new InvalidRequestException("Invalid unset map key");
 
             CellName cellName = cf.getComparator().create(prefix, column, key);
 
@@ -312,7 +337,7 @@
             {
                 cf.addColumn(params.makeTombstone(cellName));
             }
-            else
+            else if (value != ByteBufferUtil.UNSET_BYTE_BUFFER)
             {
                 // We don't support value > 64K because the serialization format encode the length as an unsigned short.
                 if (value.remaining() > FBUtilities.MAX_UNSIGNED_SHORT)
@@ -335,19 +360,20 @@
         public void execute(ByteBuffer rowKey, ColumnFamily cf, Composite prefix, UpdateParameters params) throws InvalidRequestException
         {
             assert column.type.isMultiCell() : "Attempted to add items to a frozen map";
-            doPut(t, cf, prefix, column, params);
+            Term.Terminal value = t.bind(params.options);
+            if (value != UNSET_VALUE)
+                doPut(cf, prefix, column, params, value);
         }
 
-        static void doPut(Term t, ColumnFamily cf, Composite prefix, ColumnDefinition column, UpdateParameters params) throws InvalidRequestException
+        static void doPut(ColumnFamily cf, Composite prefix, ColumnDefinition column, UpdateParameters params, Term.Terminal value) throws InvalidRequestException
         {
-            Term.Terminal value = t.bind(params.options);
-            Maps.Value mapValue = (Maps.Value) value;
             if (column.type.isMultiCell())
             {
                 if (value == null)
                     return;
 
-                for (Map.Entry<ByteBuffer, ByteBuffer> entry : mapValue.map.entrySet())
+                Map<ByteBuffer, ByteBuffer> elements = ((Value) value).map;
+                for (Map.Entry<ByteBuffer, ByteBuffer> entry : elements.entrySet())
                 {
                     CellName cellName = cf.getComparator().create(prefix, column, entry.getKey());
                     cf.addColumn(params.makeColumn(cellName, entry.getValue()));
@@ -360,7 +386,7 @@
                 if (value == null)
                     cf.addAtom(params.makeTombstone(cellName));
                 else
-                    cf.addColumn(params.makeColumn(cellName, mapValue.getWithProtocolVersion(Server.CURRENT_VERSION)));
+                    cf.addColumn(params.makeColumn(cellName, value.get(Server.CURRENT_VERSION)));
             }
         }
     }
@@ -378,8 +404,10 @@
             Term.Terminal key = t.bind(params.options);
             if (key == null)
                 throw new InvalidRequestException("Invalid null map key");
+            if (key == Constants.UNSET_VALUE)
+                throw new InvalidRequestException("Invalid unset map key");
 
-            CellName cellName = cf.getComparator().create(prefix, column, key.get(params.options));
+            CellName cellName = cf.getComparator().create(prefix, column, key.get(params.options.getProtocolVersion()));
             cf.addColumn(params.makeTombstone(cellName));
         }
     }
diff --git a/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java b/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java
index 37eb69e..b54bdd0 100644
--- a/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java
+++ b/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java
@@ -17,15 +17,31 @@
  */
 package org.apache.cassandra.cql3;
 
+import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.Term.MultiColumnRaw;
+import org.apache.cassandra.cql3.Term.Raw;
+import org.apache.cassandra.cql3.restrictions.MultiColumnRestriction;
+import org.apache.cassandra.cql3.restrictions.Restriction;
+import org.apache.cassandra.cql3.statements.Bound;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
+
 /**
  * A relation using the tuple notation, which typically affects multiple columns.
  * Examples:
+ * {@code
  *  - SELECT ... WHERE (a, b, c) > (1, 'a', 10)
  *  - SELECT ... WHERE (a, b, c) IN ((1, 2, 3), (4, 5, 6))
  *  - SELECT ... WHERE (a, b) < ?
  *  - SELECT ... WHERE (a, b) IN ?
+ * }
  */
 public class MultiColumnRelation extends Relation
 {
@@ -51,10 +67,13 @@
 
     /**
      * Creates a multi-column EQ, LT, LTE, GT, or GTE relation.
+     * {@code
      * For example: "SELECT ... WHERE (a, b) > (0, 1)"
+     * }
      * @param entities the columns on the LHS of the relation
      * @param relationType the relation operator
      * @param valuesOrMarker a Tuples.Literal instance or a Tuples.Raw marker
+     * @return a new <code>MultiColumnRelation</code> instance
      */
     public static MultiColumnRelation createNonInRelation(List<ColumnIdentifier.Raw> entities, Operator relationType, Term.MultiColumnRaw valuesOrMarker)
     {
@@ -67,6 +86,7 @@
      * For example: "SELECT ... WHERE (a, b) IN ((0, 1), (2, 3))"
      * @param entities the columns on the LHS of the relation
      * @param inValues a list of Tuples.Literal instances or a Tuples.Raw markers
+     * @return a new <code>MultiColumnRelation</code> instance
      */
     public static MultiColumnRelation createInRelation(List<ColumnIdentifier.Raw> entities, List<? extends Term.MultiColumnRaw> inValues)
     {
@@ -78,6 +98,7 @@
      * For example: "SELECT ... WHERE (a, b) IN ?"
      * @param entities the columns on the LHS of the relation
      * @param inMarker a single IN marker
+     * @return a new <code>MultiColumnRelation</code> instance
      */
     public static MultiColumnRelation createSingleMarkerInRelation(List<ColumnIdentifier.Raw> entities, Tuples.INRaw inMarker)
     {
@@ -91,54 +112,106 @@
 
     /**
      * For non-IN relations, returns the Tuples.Literal or Tuples.Raw marker for a single tuple.
+     * @return a Tuples.Literal for non-IN relations or Tuples.Raw marker for a single tuple.
      */
-    public Term.MultiColumnRaw getValue()
+    private Term.MultiColumnRaw getValue()
     {
-        assert relationType != Operator.IN;
-        return valuesOrMarker;
+        return relationType == Operator.IN ? inMarker : valuesOrMarker;
     }
 
-    /**
-     * For IN relations, returns the list of Tuples.Literal instances or Tuples.Raw markers.
-     * If a single IN marker was used, this will return null;
-     */
-    public List<? extends Term.MultiColumnRaw> getInValues()
-    {
-
-        return inValues;
-    }
-
-    /**
-     * For IN relations, returns the single marker for the IN values if there is one, otherwise null.
-     */
-    public Tuples.INRaw getInMarker()
-    {
-        return inMarker;
-    }
-
+    @Override
     public boolean isMultiColumn()
     {
         return true;
     }
 
     @Override
+    protected Restriction newEQRestriction(CFMetaData cfm,
+                                           VariableSpecifications boundNames) throws InvalidRequestException
+    {
+        List<ColumnDefinition> receivers = receivers(cfm);
+        Term term = toTerm(receivers, getValue(), cfm.ksName, boundNames);
+        return new MultiColumnRestriction.EQ(receivers, term);
+    }
+
+    @Override
+    protected Restriction newINRestriction(CFMetaData cfm,
+                                           VariableSpecifications boundNames) throws InvalidRequestException
+    {
+        List<ColumnDefinition> receivers = receivers(cfm);
+        List<Term> terms = toTerms(receivers, inValues, cfm.ksName, boundNames);
+        if (terms == null)
+        {
+            Term term = toTerm(receivers, getValue(), cfm.ksName, boundNames);
+            return new MultiColumnRestriction.InWithMarker(receivers, (AbstractMarker) term);
+        }
+        return new MultiColumnRestriction.InWithValues(receivers, terms);
+    }
+
+    @Override
+    protected Restriction newSliceRestriction(CFMetaData cfm,
+                                              VariableSpecifications boundNames,
+                                              Bound bound,
+                                              boolean inclusive) throws InvalidRequestException
+    {
+        List<ColumnDefinition> receivers = receivers(cfm);
+        Term term = toTerm(receivers(cfm), getValue(), cfm.ksName, boundNames);
+        return new MultiColumnRestriction.Slice(receivers, bound, inclusive, term);
+    }
+
+    @Override
+    protected Restriction newContainsRestriction(CFMetaData cfm,
+                                                 VariableSpecifications boundNames,
+                                                 boolean isKey) throws InvalidRequestException
+    {
+        throw invalidRequest("%s cannot be used for Multi-column relations", operator());
+    }
+
+    @Override
+    protected Term toTerm(List<? extends ColumnSpecification> receivers,
+                          Raw raw,
+                          String keyspace,
+                          VariableSpecifications boundNames) throws InvalidRequestException
+    {
+        Term term = ((MultiColumnRaw) raw).prepare(keyspace, receivers);
+        term.collectMarkerSpecification(boundNames);
+        return term;
+    }
+
+    protected List<ColumnDefinition> receivers(CFMetaData cfm) throws InvalidRequestException
+    {
+        List<ColumnDefinition> names = new ArrayList<>(getEntities().size());
+        int previousPosition = -1;
+        for (ColumnIdentifier.Raw raw : getEntities())
+        {
+            ColumnDefinition def = toColumnDefinition(cfm, raw);
+            checkTrue(def.isClusteringColumn(), "Multi-column relations can only be applied to clustering columns but was applied to: %s", def.name);
+            checkFalse(names.contains(def), "Column \"%s\" appeared twice in a relation: %s", def.name, this);
+
+            // check that no clustering columns were skipped
+            checkFalse(previousPosition != -1 && def.position() != previousPosition + 1,
+                       "Clustering columns must appear in the PRIMARY KEY order in multi-column relations: %s", this);
+
+            names.add(def);
+            previousPosition = def.position();
+        }
+        return names;
+    }
+
+    @Override
     public String toString()
     {
-        if (relationType == Operator.IN)
+        StringBuilder builder = new StringBuilder(Tuples.tupleToString(entities));
+        if (isIN())
         {
-            StringBuilder sb = new StringBuilder(Tuples.tupleToString(entities));
-            sb.append(" IN ");
-            sb.append(inMarker != null ? '?' : Tuples.tupleToString(inValues));
-            return sb.toString();
+            return builder.append(" IN ")
+                          .append(inMarker != null ? '?' : Tuples.tupleToString(inValues))
+                          .toString();
         }
-        else
-        {
-            StringBuilder sb = new StringBuilder(Tuples.tupleToString(entities));
-            sb.append(" ");
-            sb.append(relationType);
-            sb.append(" ");
-            sb.append(valuesOrMarker);
-            return sb.toString();
-        }
+        return builder.append(" ")
+                      .append(relationType)
+                      .append(" ")
+                      .append(valuesOrMarker)
+                      .toString();
     }
 }
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/cql3/Operation.java b/src/java/org/apache/cassandra/cql3/Operation.java
index ac25e29..4701a96 100644
--- a/src/java/org/apache/cassandra/cql3/Operation.java
+++ b/src/java/org/apache/cassandra/cql3/Operation.java
@@ -18,9 +18,10 @@
 package org.apache.cassandra.cql3;
 
 import java.nio.ByteBuffer;
+import java.util.Collections;
 
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.db.ColumnFamily;
 import org.apache.cassandra.db.composites.Composite;
 import org.apache.cassandra.db.marshal.*;
@@ -56,6 +57,11 @@
         this.t = t;
     }
 
+    public Iterable<Function> getFunctions()
+    {
+        return t != null ? t.getFunctions() : Collections.<Function>emptySet();
+    }
+
     /**
      * @return whether the operation requires a read of the previous value to be executed
      * (only lists setterByIdx, discard and discardByIdx requires that).
diff --git a/src/java/org/apache/cassandra/cql3/Operator.java b/src/java/org/apache/cassandra/cql3/Operator.java
index 359fcb8..86bcbd3 100644
--- a/src/java/org/apache/cassandra/cql3/Operator.java
+++ b/src/java/org/apache/cassandra/cql3/Operator.java
@@ -23,7 +23,92 @@
 
 public enum Operator
 {
-    EQ(0), LT(4), LTE(3), GTE(1), GT(2), IN(7), CONTAINS(5), CONTAINS_KEY(6), NEQ(8);
+    EQ(0)
+    {
+        @Override
+        public String toString()
+        {
+            return "=";
+        }
+    },
+    LT(4)
+    {
+        @Override
+        public String toString()
+        {
+            return "<";
+        }
+
+        @Override
+        public Operator reverse()
+        {
+            return GT;
+        }
+    },
+    LTE(3)
+    {
+        @Override
+        public String toString()
+        {
+            return "<=";
+        }
+
+        @Override
+        public Operator reverse()
+        {
+            return GTE;
+        }
+    },
+    GTE(1)
+    {
+        @Override
+        public String toString()
+        {
+            return ">=";
+        }
+
+        @Override
+        public Operator reverse()
+        {
+            return LTE;
+        }
+    },
+    GT(2)
+    {
+        @Override
+        public String toString()
+        {
+            return ">";
+        }
+
+        @Override
+        public Operator reverse()
+        {
+            return LT;
+        }
+    },
+    IN(7)
+    {
+    },
+    CONTAINS(5)
+    {
+    },
+    CONTAINS_KEY(6)
+    {
+        @Override
+        public String toString()
+        {
+            return "CONTAINS KEY";
+        }
+    },
+    NEQ(8)
+    {
+        @Override
+        public String toString()
+        {
+            return "!=";
+        }
+    };
 
     /**
      * The binary representation of this <code>Enum</code> value.
@@ -70,24 +155,16 @@
     @Override
     public String toString()
     {
-        switch (this)
-        {
-            case EQ:
-                return "=";
-            case LT:
-                return "<";
-            case LTE:
-                return "<=";
-            case GT:
-                return ">";
-            case GTE:
-                return ">=";
-            case NEQ:
-                return "!=";
-            case CONTAINS_KEY:
-                return "CONTAINS KEY";
-            default:
-                return this.name();
-        }
+         return this.name();
     }
-}
\ No newline at end of file
+
+    /**
+     * Returns the reverse operator if this one.
+     *
+     * @return the reverse operator of this one.
+     */
+    public Operator reverse()
+    {
+        return this;
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/QueryHandler.java b/src/java/org/apache/cassandra/cql3/QueryHandler.java
index d42d90e..3c11c0e 100644
--- a/src/java/org/apache/cassandra/cql3/QueryHandler.java
+++ b/src/java/org/apache/cassandra/cql3/QueryHandler.java
@@ -17,6 +17,9 @@
  */
 package org.apache.cassandra.cql3;
 
+import java.nio.ByteBuffer;
+import java.util.Map;
+
 import org.apache.cassandra.cql3.statements.BatchStatement;
 import org.apache.cassandra.cql3.statements.ParsedStatement;
 import org.apache.cassandra.exceptions.RequestExecutionException;
@@ -27,10 +30,26 @@
 
 public interface QueryHandler
 {
-    public ResultMessage process(String query, QueryState state, QueryOptions options) throws RequestExecutionException, RequestValidationException;
-    public ResultMessage.Prepared prepare(String query, QueryState state) throws RequestValidationException;
-    public ParsedStatement.Prepared getPrepared(MD5Digest id);
-    public ParsedStatement.Prepared getPreparedForThrift(Integer id);
-    public ResultMessage processPrepared(CQLStatement statement, QueryState state, QueryOptions options) throws RequestExecutionException, RequestValidationException;
-    public ResultMessage processBatch(BatchStatement statement, QueryState state, BatchQueryOptions options) throws RequestExecutionException, RequestValidationException;
+    ResultMessage process(String query,
+                          QueryState state,
+                          QueryOptions options,
+                          Map<String, ByteBuffer> customPayload) throws RequestExecutionException, RequestValidationException;
+
+    ResultMessage.Prepared prepare(String query,
+                                   QueryState state,
+                                   Map<String, ByteBuffer> customPayload) throws RequestValidationException;
+
+    ParsedStatement.Prepared getPrepared(MD5Digest id);
+
+    ParsedStatement.Prepared getPreparedForThrift(Integer id);
+
+    ResultMessage processPrepared(CQLStatement statement,
+                                  QueryState state,
+                                  QueryOptions options,
+                                  Map<String, ByteBuffer> customPayload) throws RequestExecutionException, RequestValidationException;
+
+    ResultMessage processBatch(BatchStatement statement,
+                               QueryState state,
+                               BatchQueryOptions options,
+                               Map<String, ByteBuffer> customPayload) throws RequestExecutionException, RequestValidationException;
 }
diff --git a/src/java/org/apache/cassandra/cql3/QueryOptions.java b/src/java/org/apache/cassandra/cql3/QueryOptions.java
index b2569e7..be773e1 100644
--- a/src/java/org/apache/cassandra/cql3/QueryOptions.java
+++ b/src/java/org/apache/cassandra/cql3/QueryOptions.java
@@ -18,19 +18,21 @@
 package org.apache.cassandra.cql3;
 
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.EnumSet;
-import java.util.List;
+import java.util.*;
 
+import com.google.common.collect.ImmutableList;
 import io.netty.buffer.ByteBuf;
 
+import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.service.pager.PagingState;
 import org.apache.cassandra.transport.CBCodec;
 import org.apache.cassandra.transport.CBUtil;
 import org.apache.cassandra.transport.ProtocolException;
+import org.apache.cassandra.transport.Server;
 import org.apache.cassandra.utils.Pair;
 
 /**
@@ -42,33 +44,41 @@
                                                                        Collections.<ByteBuffer>emptyList(),
                                                                        false,
                                                                        SpecificOptions.DEFAULT,
-                                                                       3);
+                                                                       Server.CURRENT_VERSION);
 
     public static final CBCodec<QueryOptions> codec = new Codec();
 
+    // A cache of bind values parsed as JSON, see getJsonColumnValue for details.
+    private List<Map<ColumnIdentifier, Term>> jsonValuesCache;
+
     public static QueryOptions fromProtocolV1(ConsistencyLevel consistency, List<ByteBuffer> values)
     {
-        return new DefaultQueryOptions(consistency, values, false, SpecificOptions.DEFAULT, 1);
+        return new DefaultQueryOptions(consistency, values, false, SpecificOptions.DEFAULT, Server.VERSION_1);
     }
 
     public static QueryOptions fromProtocolV2(ConsistencyLevel consistency, List<ByteBuffer> values)
     {
-        return new DefaultQueryOptions(consistency, values, false, SpecificOptions.DEFAULT, 2);
+        return new DefaultQueryOptions(consistency, values, false, SpecificOptions.DEFAULT, Server.VERSION_2);
     }
 
     public static QueryOptions forInternalCalls(ConsistencyLevel consistency, List<ByteBuffer> values)
     {
-        return new DefaultQueryOptions(consistency, values, false, SpecificOptions.DEFAULT, 3);
+        return new DefaultQueryOptions(consistency, values, false, SpecificOptions.DEFAULT, Server.VERSION_3);
     }
 
     public static QueryOptions forInternalCalls(List<ByteBuffer> values)
     {
-        return new DefaultQueryOptions(ConsistencyLevel.ONE, values, false, SpecificOptions.DEFAULT, 3);
+        return new DefaultQueryOptions(ConsistencyLevel.ONE, values, false, SpecificOptions.DEFAULT, Server.VERSION_3);
     }
 
     public static QueryOptions fromPreV3Batch(ConsistencyLevel consistency)
     {
-        return new DefaultQueryOptions(consistency, Collections.<ByteBuffer>emptyList(), false, SpecificOptions.DEFAULT, 2);
+        return new DefaultQueryOptions(consistency, Collections.<ByteBuffer>emptyList(), false, SpecificOptions.DEFAULT, Server.VERSION_2);
+    }
+
+    public static QueryOptions forProtocolVersion(int protocolVersion)
+    {
+        return new DefaultQueryOptions(null, null, true, null, protocolVersion);
     }
 
     public static QueryOptions create(ConsistencyLevel consistency, List<ByteBuffer> values, boolean skipMetadata, int pageSize, PagingState pagingState, ConsistencyLevel serialConsistency)
@@ -76,10 +86,82 @@
         return new DefaultQueryOptions(consistency, values, skipMetadata, new SpecificOptions(pageSize, pagingState, serialConsistency, -1L), 0);
     }
 
+    public static QueryOptions addColumnSpecifications(QueryOptions options, List<ColumnSpecification> columnSpecs)
+    {
+        return new OptionsWithColumnSpecifications(options, columnSpecs);
+    }
+
     public abstract ConsistencyLevel getConsistency();
     public abstract List<ByteBuffer> getValues();
     public abstract boolean skipMetadata();
 
+    /**
+     * Returns the term corresponding to column {@code columnName} in the JSON value of bind index {@code bindIndex}.
+     *
+     * This is functionally equivalent to:
+     *   {@code Json.parseJson(UTF8Type.instance.getSerializer().deserialize(getValues().get(bindIndex)), expectedReceivers).get(columnName)}
+     * but this cache the result of parsing the JSON so that while this might be called for multiple columns on the same {@code bindIndex}
+     * value, the underlying JSON value is only parsed/processed once.
+     *
+     * Note: this is a bit more involved in CQL specifics than this class generally is but we as we need to cache this per-query and in an object
+     * that is available when we bind values, this is the easier place to have this.
+     *
+     * @param bindIndex the index of the bind value that should be interpreted as a JSON value.
+     * @param columnName the name of the column we want the value of.
+     * @param expectedReceivers the columns expected in the JSON value at index {@code bindIndex}. This is only used when parsing the
+     * json initially and no check is done afterwards. So in practice, any call of this method on the same QueryOptions object and with the same
+     * {@code bindIndx} values should use the same value for this parameter, but this isn't validated in any way.
+     *
+     * @return the value correspong to column {@code columnName} in the (JSON) bind value at index {@code bindIndex}. This may return null if the
+     * JSON value has no value for this column.
+     */
+    public Term getJsonColumnValue(int bindIndex, ColumnIdentifier columnName, Collection<ColumnDefinition> expectedReceivers) throws InvalidRequestException
+    {
+        if (jsonValuesCache == null)
+            jsonValuesCache = new ArrayList<>(Collections.<Map<ColumnIdentifier, Term>>nCopies(getValues().size(), null));
+
+        Map<ColumnIdentifier, Term> jsonValue = jsonValuesCache.get(bindIndex);
+        if (jsonValue == null)
+        {
+            ByteBuffer value = getValues().get(bindIndex);
+            if (value == null)
+                throw new InvalidRequestException("Got null for INSERT JSON values");
+
+            jsonValue = Json.parseJson(UTF8Type.instance.getSerializer().deserialize(value), expectedReceivers);
+            jsonValuesCache.set(bindIndex, jsonValue);
+        }
+
+        return jsonValue.get(columnName);
+    }
+
+    /**
+     * Tells whether or not this <code>QueryOptions</code> contains the column specifications for the bound variables.
+     * <p>The column specifications will be present only for prepared statements.</p>
+     * @return <code>true</code> this <code>QueryOptions</code> contains the column specifications for the bound
+     * variables, <code>false</code> otherwise.
+     */
+    public boolean hasColumnSpecifications()
+    {
+        return false;
+    }
+
+    /**
+     * Returns the column specifications for the bound variables (<i>optional operation</i>).
+     *
+     * <p>The column specifications will be present only for prepared statements.</p>
+     *
+     * <p>Invoke the {@link hasColumnSpecifications} method before invoking this method in order to ensure that this
+     * <code>QueryOptions</code> contains the column specifications.</p>
+     *
+     * @return the option names
+     * @throws UnsupportedOperationException If this <code>QueryOptions</code> does not contains the column
+     * specifications.
+     */
+    public ImmutableList<ColumnSpecification> getColumnSpecifications()
+    {
+        throw new UnsupportedOperationException();
+    }
+
     /**  The pageSize for this query. Will be <= 0 if not relevant for the query.  */
     public int getPageSize()
     {
@@ -163,7 +245,7 @@
         }
     }
 
-    static abstract class QueryOptionsWrapper extends QueryOptions
+    static class QueryOptionsWrapper extends QueryOptions
     {
         protected final QueryOptions wrapped;
 
@@ -172,6 +254,11 @@
             this.wrapped = wrapped;
         }
 
+        public List<ByteBuffer> getValues()
+        {
+            return this.wrapped.getValues();
+        }
+
         public ConsistencyLevel getConsistency()
         {
             return wrapped.getConsistency();
@@ -200,6 +287,32 @@
         }
     }
 
+    /**
+     * <code>QueryOptions</code> decorator that provides access to the column specifications.
+     */
+    static class OptionsWithColumnSpecifications extends QueryOptionsWrapper
+    {
+        private final ImmutableList<ColumnSpecification> columnSpecs;
+
+        OptionsWithColumnSpecifications(QueryOptions wrapped, List<ColumnSpecification> columnSpecs)
+        {
+            super(wrapped);
+            this.columnSpecs = ImmutableList.copyOf(columnSpecs);
+        }
+
+        @Override
+        public boolean hasColumnSpecifications()
+        {
+            return true;
+        }
+
+        @Override
+        public ImmutableList<ColumnSpecification> getColumnSpecifications()
+        {
+            return columnSpecs;
+        }
+    }
+
     static class OptionsWithNames extends QueryOptionsWrapper
     {
         private final List<String> names;
@@ -232,6 +345,7 @@
             return this;
         }
 
+        @Override
         public List<ByteBuffer> getValues()
         {
             assert orderedValues != null; // We should have called prepare first!
@@ -306,13 +420,13 @@
             {
                 if (flags.contains(Flag.NAMES_FOR_VALUES))
                 {
-                    Pair<List<String>, List<ByteBuffer>> namesAndValues = CBUtil.readNameAndValueList(body);
+                    Pair<List<String>, List<ByteBuffer>> namesAndValues = CBUtil.readNameAndValueList(body, version);
                     names = namesAndValues.left;
                     values = namesAndValues.right;
                 }
                 else
                 {
-                    values = CBUtil.readValueList(body);
+                    values = CBUtil.readValueList(body, version);
                 }
             }
 
diff --git a/src/java/org/apache/cassandra/cql3/QueryProcessor.java b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
index 4340d42..c702679 100644
--- a/src/java/org/apache/cassandra/cql3/QueryProcessor.java
+++ b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
@@ -18,53 +18,46 @@
 package org.apache.cassandra.cql3;
 
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-import java.util.concurrent.*;
+import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Iterators;
 import com.google.common.primitives.Ints;
-
-import org.apache.cassandra.service.MigrationListener;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap;
 import com.googlecode.concurrentlinkedhashmap.EntryWeigher;
 import com.googlecode.concurrentlinkedhashmap.EvictionListener;
-
-import org.apache.cassandra.concurrent.ScheduledExecutors;
 import org.antlr.runtime.*;
+import org.apache.cassandra.concurrent.ScheduledExecutors;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.functions.FunctionName;
+import org.apache.cassandra.cql3.functions.Functions;
 import org.apache.cassandra.cql3.statements.*;
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.composites.CType;
-import org.apache.cassandra.db.composites.CellName;
-import org.apache.cassandra.db.composites.CellNameType;
-import org.apache.cassandra.db.composites.Composite;
+import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.RequestExecutionException;
-import org.apache.cassandra.exceptions.RequestValidationException;
-import org.apache.cassandra.exceptions.SyntaxException;
+import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.metrics.CQLMetrics;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.MigrationManager;
-import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.*;
 import org.apache.cassandra.service.pager.QueryPager;
 import org.apache.cassandra.service.pager.QueryPagers;
 import org.apache.cassandra.thrift.ThriftClientState;
 import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.transport.messages.ResultMessage;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.MD5Digest;
-import org.apache.cassandra.utils.SemanticVersion;
+import org.apache.cassandra.utils.*;
 import org.github.jamm.MemoryMeter;
 
 public class QueryProcessor implements QueryHandler
 {
-    public static final SemanticVersion CQL_VERSION = new SemanticVersion("3.2.1");
+    public static final CassandraVersion CQL_VERSION = new CassandraVersion("3.3.1");
 
     public static final QueryProcessor instance = new QueryProcessor();
 
@@ -72,7 +65,7 @@
     private static final MemoryMeter meter = new MemoryMeter().withGuessing(MemoryMeter.Guess.FALLBACK_BEST).ignoreKnownSingletons();
     private static final long MAX_CACHE_PREPARED_MEMORY = Runtime.getRuntime().maxMemory() / 256;
 
-    private static EntryWeigher<MD5Digest, ParsedStatement.Prepared> cqlMemoryUsageWeigher = new EntryWeigher<MD5Digest, ParsedStatement.Prepared>()
+    private static final EntryWeigher<MD5Digest, ParsedStatement.Prepared> cqlMemoryUsageWeigher = new EntryWeigher<MD5Digest, ParsedStatement.Prepared>()
     {
         @Override
         public int weightOf(MD5Digest key, ParsedStatement.Prepared value)
@@ -81,7 +74,7 @@
         }
     };
 
-    private static EntryWeigher<Integer, ParsedStatement.Prepared> thriftMemoryUsageWeigher = new EntryWeigher<Integer, ParsedStatement.Prepared>()
+    private static final EntryWeigher<Integer, ParsedStatement.Prepared> thriftMemoryUsageWeigher = new EntryWeigher<Integer, ParsedStatement.Prepared>()
     {
         @Override
         public int weightOf(Integer key, ParsedStatement.Prepared value)
@@ -158,14 +151,7 @@
         InternalStateInstance()
         {
             ClientState state = ClientState.forInternalCalls();
-            try
-            {
-                state.setKeyspace(Keyspace.SYSTEM_KS);
-            }
-            catch (InvalidRequestException e)
-            {
-                throw new RuntimeException();
-            }
+            state.setKeyspace(SystemKeyspace.NAME);
             this.queryState = new QueryState(state);
         }
     }
@@ -196,6 +182,8 @@
         {
             throw new InvalidRequestException("Key may not be empty");
         }
+        if (key == ByteBufferUtil.UNSET_BYTE_BUFFER)
+            throw new InvalidRequestException("Key may not be unset");
 
         // check that key can be handled by FBUtilities.writeShortByteArray
         if (key.remaining() > FBUtilities.MAX_UNSIGNED_SHORT)
@@ -245,6 +233,15 @@
         return instance.process(queryString, queryState, QueryOptions.forInternalCalls(cl, Collections.<ByteBuffer>emptyList()));
     }
 
+    public ResultMessage process(String query,
+                                 QueryState state,
+                                 QueryOptions options,
+                                 Map<String, ByteBuffer> customPayload)
+                                         throws RequestExecutionException, RequestValidationException
+    {
+        return process(query, state, options);
+    }
+
     public ResultMessage process(String queryString, QueryState queryState, QueryOptions options)
     throws RequestExecutionException, RequestValidationException
     {
@@ -267,18 +264,16 @@
 
     public static UntypedResultSet process(String query, ConsistencyLevel cl) throws RequestExecutionException
     {
-        try
-        {
-            ResultMessage result = instance.process(query, QueryState.forInternalCalls(), QueryOptions.forInternalCalls(cl, Collections.<ByteBuffer>emptyList()));
-            if (result instanceof ResultMessage.Rows)
-                return UntypedResultSet.create(((ResultMessage.Rows)result).result);
-            else
-                return null;
-        }
-        catch (RequestValidationException e)
-        {
-            throw new RuntimeException(e);
-        }
+        return process(query, cl, Collections.<ByteBuffer>emptyList());
+    }
+
+    public static UntypedResultSet process(String query, ConsistencyLevel cl, List<ByteBuffer> values) throws RequestExecutionException
+    {
+        ResultMessage result = instance.process(query, QueryState.forInternalCalls(), QueryOptions.forInternalCalls(cl, values));
+        if (result instanceof ResultMessage.Rows)
+            return UntypedResultSet.create(((ResultMessage.Rows)result).result);
+        else
+            return null;
     }
 
     private static QueryOptions makeInternalOptions(ParsedStatement.Prepared prepared, Object[] values)
@@ -311,41 +306,23 @@
 
     public static UntypedResultSet executeInternal(String query, Object... values)
     {
-        try
-        {
-            ParsedStatement.Prepared prepared = prepareInternal(query);
-            ResultMessage result = prepared.statement.executeInternal(internalQueryState(), makeInternalOptions(prepared, values));
-            if (result instanceof ResultMessage.Rows)
-                return UntypedResultSet.create(((ResultMessage.Rows)result).result);
-            else
-                return null;
-        }
-        catch (RequestExecutionException e)
-        {
-            throw new RuntimeException(e);
-        }
-        catch (RequestValidationException e)
-        {
-            throw new RuntimeException("Error validating " + query, e);
-        }
+        ParsedStatement.Prepared prepared = prepareInternal(query);
+        ResultMessage result = prepared.statement.executeInternal(internalQueryState(), makeInternalOptions(prepared, values));
+        if (result instanceof ResultMessage.Rows)
+            return UntypedResultSet.create(((ResultMessage.Rows)result).result);
+        else
+            return null;
     }
 
     public static UntypedResultSet executeInternalWithPaging(String query, int pageSize, Object... values)
     {
-        try
-        {
-            ParsedStatement.Prepared prepared = prepareInternal(query);
-            if (!(prepared.statement instanceof SelectStatement))
-                throw new IllegalArgumentException("Only SELECTs can be paged");
+        ParsedStatement.Prepared prepared = prepareInternal(query);
+        if (!(prepared.statement instanceof SelectStatement))
+            throw new IllegalArgumentException("Only SELECTs can be paged");
 
-            SelectStatement select = (SelectStatement)prepared.statement;
-            QueryPager pager = QueryPagers.localPager(select.getPageableCommand(makeInternalOptions(prepared, values)));
-            return UntypedResultSet.create(select, pager, pageSize);
-        }
-        catch (RequestValidationException e)
-        {
-            throw new RuntimeException("Error validating query" + e);
-        }
+        SelectStatement select = (SelectStatement)prepared.statement;
+        QueryPager pager = QueryPagers.localPager(select.getPageableCommand(makeInternalOptions(prepared, values)));
+        return UntypedResultSet.create(select, pager, pageSize);
     }
 
     /**
@@ -354,24 +331,13 @@
      */
     public static UntypedResultSet executeOnceInternal(String query, Object... values)
     {
-        try
-        {
-            ParsedStatement.Prepared prepared = parseStatement(query, internalQueryState());
-            prepared.statement.validate(internalQueryState().getClientState());
-            ResultMessage result = prepared.statement.executeInternal(internalQueryState(), makeInternalOptions(prepared, values));
-            if (result instanceof ResultMessage.Rows)
-                return UntypedResultSet.create(((ResultMessage.Rows)result).result);
-            else
-                return null;
-        }
-        catch (RequestExecutionException e)
-        {
-            throw new RuntimeException(e);
-        }
-        catch (RequestValidationException e)
-        {
-            throw new RuntimeException("Error validating query " + query, e);
-        }
+        ParsedStatement.Prepared prepared = parseStatement(query, internalQueryState());
+        prepared.statement.validate(internalQueryState().getClientState());
+        ResultMessage result = prepared.statement.executeInternal(internalQueryState(), makeInternalOptions(prepared, values));
+        if (result instanceof ResultMessage.Rows)
+            return UntypedResultSet.create(((ResultMessage.Rows)result).result);
+        else
+            return null;
     }
 
     /**
@@ -407,27 +373,25 @@
 
     public static UntypedResultSet resultify(String query, List<Row> rows)
     {
-        try
-        {
-            SelectStatement ss = (SelectStatement) getStatement(query, null).statement;
-            ResultSet cqlRows = ss.process(rows);
-            return UntypedResultSet.create(cqlRows);
-        }
-        catch (RequestValidationException e)
-        {
-            throw new AssertionError(e);
-        }
+        SelectStatement ss = (SelectStatement) getStatement(query, null).statement;
+        ResultSet cqlRows = ss.process(rows);
+        return UntypedResultSet.create(cqlRows);
+    }
+
+    public ResultMessage.Prepared prepare(String query,
+                                          QueryState state,
+                                          Map<String, ByteBuffer> customPayload) throws RequestValidationException
+    {
+        return prepare(query, state);
     }
 
     public ResultMessage.Prepared prepare(String queryString, QueryState queryState)
-    throws RequestValidationException
     {
         ClientState cState = queryState.getClientState();
         return prepare(queryString, cState, cState instanceof ThriftClientState);
     }
 
     public static ResultMessage.Prepared prepare(String queryString, ClientState clientState, boolean forThrift)
-    throws RequestValidationException
     {
         ResultMessage.Prepared existing = getStoredPreparedStatement(queryString, clientState.getRawKeyspace(), forThrift);
         if (existing != null)
@@ -496,6 +460,15 @@
         }
     }
 
+    public ResultMessage processPrepared(CQLStatement statement,
+                                         QueryState state,
+                                         QueryOptions options,
+                                         Map<String, ByteBuffer> customPayload)
+                                                 throws RequestExecutionException, RequestValidationException
+    {
+        return processPrepared(statement, state, options);
+    }
+
     public ResultMessage processPrepared(CQLStatement statement, QueryState queryState, QueryOptions options)
     throws RequestExecutionException, RequestValidationException
     {
@@ -519,6 +492,15 @@
         return processStatement(statement, queryState, options);
     }
 
+    public ResultMessage processBatch(BatchStatement statement,
+                                      QueryState state,
+                                      BatchQueryOptions options,
+                                      Map<String, ByteBuffer> customPayload)
+                                              throws RequestExecutionException, RequestValidationException
+    {
+        return processBatch(statement, state, options);
+    }
+
     public ResultMessage processBatch(BatchStatement batch, QueryState queryState, BatchQueryOptions options)
     throws RequestExecutionException, RequestValidationException
     {
@@ -562,10 +544,14 @@
 
             // The errorCollector has queue up any errors that the lexer and parser may have encountered
             // along the way, if necessary, we turn the last error into exceptions here.
-            errorCollector.throwLastSyntaxError();
+            errorCollector.throwFirstSyntaxError();
 
             return statement;
         }
+        catch (CassandraException ce)
+        {
+            throw ce;
+        }
         catch (RuntimeException re)
         {
             logger.error(String.format("The statement: [%s] could not be parsed.", queryStr), re);
@@ -585,6 +571,15 @@
         return meter.measureDeep(key);
     }
 
+    /**
+     * Clear our internal statmeent cache for test purposes.
+     */
+    @VisibleForTesting
+    public static void clearInternalStatementsCache()
+    {
+        internalStatements.clear();
+    }
+
     private static class MigrationSubscriber extends MigrationListener
     {
         private void removeInvalidPreparedStatements(String ksName, String cfName)
@@ -638,25 +633,99 @@
             return ksName.equals(statementKsName) && (cfName == null || cfName.equals(statementCfName));
         }
 
+        public void onCreateFunction(String ksName, String functionName, List<AbstractType<?>> argTypes)
+        {
+            if (Functions.getOverloadCount(new FunctionName(ksName, functionName)) > 1)
+            {
+                // in case there are other overloads, we have to remove all overloads since argument type
+                // matching may change (due to type casting)
+                removeAllInvalidPreparedStatementsForFunction(ksName, functionName);
+            }
+        }
+
+        public void onCreateAggregate(String ksName, String aggregateName, List<AbstractType<?>> argTypes)
+        {
+            if (Functions.getOverloadCount(new FunctionName(ksName, aggregateName)) > 1)
+            {
+                // in case there are other overloads, we have to remove all overloads since argument type
+                // matching may change (due to type casting)
+                removeAllInvalidPreparedStatementsForFunction(ksName, aggregateName);
+            }
+        }
+
         public void onUpdateColumnFamily(String ksName, String cfName, boolean columnsDidChange)
         {
+            logger.trace("Column definitions for {}.{} changed, invalidating related prepared statements", ksName, cfName);
             if (columnsDidChange)
-            {
-                logger.debug("Column definitions for {}.{} changed, invalidating related prepared statements", ksName, cfName);
                 removeInvalidPreparedStatements(ksName, cfName);
-            }
+        }
+
+        public void onUpdateFunction(String ksName, String functionName, List<AbstractType<?>> argTypes)
+        {
+            // Updating a function may imply we've changed the body of the function, so we need to invalid statements so that
+            // the new definition is picked (the function is resolved at preparation time).
+            // TODO: if the function has multiple overload, we could invalidate only the statement refering to the overload
+            // that was updated. This requires a few changes however and probably doesn't matter much in practice.
+            removeAllInvalidPreparedStatementsForFunction(ksName, functionName);
+        }
+
+        public void onUpdateAggregate(String ksName, String aggregateName, List<AbstractType<?>> argTypes)
+        {
+            // Updating a function may imply we've changed the body of the function, so we need to invalid statements so that
+            // the new definition is picked (the function is resolved at preparation time).
+            // TODO: if the function has multiple overload, we could invalidate only the statement refering to the overload
+            // that was updated. This requires a few changes however and probably doesn't matter much in practice.
+            removeAllInvalidPreparedStatementsForFunction(ksName, aggregateName);
         }
 
         public void onDropKeyspace(String ksName)
         {
-            logger.debug("Keyspace {} was dropped, invalidating related prepared statements", ksName);
+            logger.trace("Keyspace {} was dropped, invalidating related prepared statements", ksName);
             removeInvalidPreparedStatements(ksName, null);
         }
 
         public void onDropColumnFamily(String ksName, String cfName)
         {
-            logger.debug("Table {}.{} was dropped, invalidating related prepared statements", ksName, cfName);
+            logger.trace("Table {}.{} was dropped, invalidating related prepared statements", ksName, cfName);
             removeInvalidPreparedStatements(ksName, cfName);
         }
-	}
+
+        public void onDropFunction(String ksName, String functionName, List<AbstractType<?>> argTypes)
+        {
+            removeAllInvalidPreparedStatementsForFunction(ksName, functionName);
+        }
+
+        public void onDropAggregate(String ksName, String aggregateName, List<AbstractType<?>> argTypes)
+        {
+            removeAllInvalidPreparedStatementsForFunction(ksName, aggregateName);
+        }
+
+        private void removeAllInvalidPreparedStatementsForFunction(String ksName, String functionName)
+        {
+            removeInvalidPreparedStatementsForFunction(internalStatements.values().iterator(), ksName, functionName);
+            removeInvalidPreparedStatementsForFunction(preparedStatements.values().iterator(), ksName, functionName);
+            removeInvalidPreparedStatementsForFunction(thriftPreparedStatements.values().iterator(), ksName, functionName);
+        }
+
+        private static void removeInvalidPreparedStatementsForFunction(Iterator<ParsedStatement.Prepared> statements,
+                                                                       final String ksName,
+                                                                       final String functionName)
+        {
+            final Predicate<Function> matchesFunction = new Predicate<Function>()
+            {
+                public boolean apply(Function f)
+                {
+                    return ksName.equals(f.name().keyspace) && functionName.equals(f.name().name);
+                }
+            };
+
+            Iterators.removeIf(statements, new Predicate<ParsedStatement.Prepared>()
+            {
+                public boolean apply(ParsedStatement.Prepared statement)
+                {
+                    return Iterables.any(statement.statement.getFunctions(), matchesFunction);
+                }
+            });
+        }
+    }
 }
diff --git a/src/java/org/apache/cassandra/cql3/Relation.java b/src/java/org/apache/cassandra/cql3/Relation.java
index d13a5d9..1337096 100644
--- a/src/java/org/apache/cassandra/cql3/Relation.java
+++ b/src/java/org/apache/cassandra/cql3/Relation.java
@@ -17,6 +17,18 @@
  */
 package org.apache.cassandra.cql3;
 
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.restrictions.Restriction;
+import org.apache.cassandra.cql3.statements.Bound;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.exceptions.UnrecognizedEntityException;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
+
 public abstract class Relation {
 
     protected Operator relationType;
@@ -26,10 +38,212 @@
         return relationType;
     }
 
-    public abstract boolean isMultiColumn();
-
-    public boolean isOnToken()
+    /**
+     * Checks if this relation apply to multiple columns.
+     *
+     * @return <code>true</code> if this relation apply to multiple columns, <code>false</code> otherwise.
+     */
+    public boolean isMultiColumn()
     {
         return false;
     }
+
+    /**
+     * Checks if this relation is a token relation (e.g. <pre>token(a) = token(1)</pre>).
+     *
+     * @return <code>true</code> if this relation is a token relation, <code>false</code> otherwise.
+     */
+    public boolean onToken()
+    {
+        return false;
+    }
+
+    /**
+     * Checks if the operator of this relation is a <code>CONTAINS</code>.
+     * @return <code>true</code>  if the operator of this relation is a <code>CONTAINS</code>, <code>false</code>
+     * otherwise.
+     */
+    public final boolean isContains()
+    {
+        return relationType == Operator.CONTAINS;
+    }
+
+    /**
+     * Checks if the operator of this relation is a <code>CONTAINS_KEY</code>.
+     * @return <code>true</code>  if the operator of this relation is a <code>CONTAINS_KEY</code>, <code>false</code>
+     * otherwise.
+     */
+    public final boolean isContainsKey()
+    {
+        return relationType == Operator.CONTAINS_KEY;
+    }
+
+    /**
+     * Checks if the operator of this relation is a <code>IN</code>.
+     * @return <code>true</code>  if the operator of this relation is a <code>IN</code>, <code>false</code>
+     * otherwise.
+     */
+    public final boolean isIN()
+    {
+        return relationType == Operator.IN;
+    }
+
+    /**
+     * Checks if the operator of this relation is a <code>EQ</code>.
+     * @return <code>true</code>  if the operator of this relation is a <code>EQ</code>, <code>false</code>
+     * otherwise.
+     */
+    public final boolean isEQ()
+    {
+        return relationType == Operator.EQ;
+    }
+
+    /**
+     * Checks if the operator of this relation is a <code>Slice</code> (GT, GTE, LTE, LT).
+     *
+     * @return <code>true</code> if the operator of this relation is a <code>Slice</code>, <code>false</code> otherwise.
+     */
+    public final boolean isSlice()
+    {
+        return relationType == Operator.GT
+                || relationType == Operator.GTE
+                || relationType == Operator.LTE
+                || relationType == Operator.LT;
+    }
+
+    /**
+     * Converts this <code>Relation</code> into a <code>Restriction</code>.
+     *
+     * @param cfm the Column Family meta data
+     * @param boundNames the variables specification where to collect the bind variables
+     * @return the <code>Restriction</code> corresponding to this <code>Relation</code>
+     * @throws InvalidRequestException if this <code>Relation</code> is not valid
+     */
+    public final Restriction toRestriction(CFMetaData cfm,
+                                           VariableSpecifications boundNames) throws InvalidRequestException
+    {
+        switch (relationType)
+        {
+            case EQ: return newEQRestriction(cfm, boundNames);
+            case LT: return newSliceRestriction(cfm, boundNames, Bound.END, false);
+            case LTE: return newSliceRestriction(cfm, boundNames, Bound.END, true);
+            case GTE: return newSliceRestriction(cfm, boundNames, Bound.START, true);
+            case GT: return newSliceRestriction(cfm, boundNames, Bound.START, false);
+            case IN: return newINRestriction(cfm, boundNames);
+            case CONTAINS: return newContainsRestriction(cfm, boundNames, false);
+            case CONTAINS_KEY: return newContainsRestriction(cfm, boundNames, true);
+            default: throw invalidRequest("Unsupported \"!=\" relation: %s", this);
+        }
+    }
+
+    /**
+     * Creates a new EQ restriction instance.
+     *
+     * @param cfm the Column Family meta data
+     * @param boundNames the variables specification where to collect the bind variables
+     * @return a new EQ restriction instance.
+     * @throws InvalidRequestException if the relation cannot be converted into an EQ restriction.
+     */
+    protected abstract Restriction newEQRestriction(CFMetaData cfm,
+                                                    VariableSpecifications boundNames) throws InvalidRequestException;
+
+    /**
+     * Creates a new IN restriction instance.
+     *
+     * @param cfm the Column Family meta data
+     * @param boundNames the variables specification where to collect the bind variables
+     * @return a new IN restriction instance
+     * @throws InvalidRequestException if the relation cannot be converted into an IN restriction.
+     */
+    protected abstract Restriction newINRestriction(CFMetaData cfm,
+                                                    VariableSpecifications boundNames) throws InvalidRequestException;
+
+    /**
+     * Creates a new Slice restriction instance.
+     *
+     * @param cfm the Column Family meta data
+     * @param boundNames the variables specification where to collect the bind variables
+     * @param bound the slice bound
+     * @param inclusive <code>true</code> if the bound is included.
+     * @return a new slice restriction instance
+     * @throws InvalidRequestException if the <code>Relation</code> is not valid
+     */
+    protected abstract Restriction newSliceRestriction(CFMetaData cfm,
+                                                       VariableSpecifications boundNames,
+                                                       Bound bound,
+                                                       boolean inclusive) throws InvalidRequestException;
+
+    /**
+     * Creates a new Contains restriction instance.
+     *
+     * @param cfm the Column Family meta data
+     * @param boundNames the variables specification where to collect the bind variables
+     * @param isKey <code>true</code> if the restriction to create is a CONTAINS KEY
+     * @return a new Contains <code>Restriction</code> instance
+     * @throws InvalidRequestException if the <code>Relation</code> is not valid
+     */
+    protected abstract Restriction newContainsRestriction(CFMetaData cfm,
+                                                          VariableSpecifications boundNames,
+                                                          boolean isKey) throws InvalidRequestException;
+
+    /**
+     * Converts the specified <code>Raw</code> into a <code>Term</code>.
+     * @param receivers the columns to which the values must be associated at
+     * @param raw the raw term to convert
+     * @param keyspace the keyspace name
+     * @param boundNames the variables specification where to collect the bind variables
+     *
+     * @return the <code>Term</code> corresponding to the specified <code>Raw</code>
+     * @throws InvalidRequestException if the <code>Raw</code> term is not valid
+     */
+    protected abstract Term toTerm(List<? extends ColumnSpecification> receivers,
+                                   Term.Raw raw,
+                                   String keyspace,
+                                   VariableSpecifications boundNames)
+                                   throws InvalidRequestException;
+
+    /**
+     * Converts the specified <code>Raw</code> terms into a <code>Term</code>s.
+     * @param receivers the columns to which the values must be associated at
+     * @param raws the raw terms to convert
+     * @param keyspace the keyspace name
+     * @param boundNames the variables specification where to collect the bind variables
+     *
+     * @return the <code>Term</code>s corresponding to the specified <code>Raw</code> terms
+     * @throws InvalidRequestException if the <code>Raw</code> terms are not valid
+     */
+    protected final List<Term> toTerms(List<? extends ColumnSpecification> receivers,
+                                       List<? extends Term.Raw> raws,
+                                       String keyspace,
+                                       VariableSpecifications boundNames) throws InvalidRequestException
+    {
+        if (raws == null)
+            return null;
+
+        List<Term> terms = new ArrayList<>();
+        for (int i = 0, m = raws.size(); i < m; i++)
+            terms.add(toTerm(receivers, raws.get(i), keyspace, boundNames));
+
+        return terms;
+    }
+
+    /**
+     * Converts the specified entity into a column definition.
+     *
+     * @param cfm the column family meta data
+     * @param entity the entity to convert
+     * @return the column definition corresponding to the specified entity
+     * @throws InvalidRequestException if the entity cannot be recognized
+     */
+    protected final ColumnDefinition toColumnDefinition(CFMetaData cfm,
+                                                        ColumnIdentifier.Raw entity) throws InvalidRequestException
+    {
+        ColumnIdentifier identifier = entity.prepare(cfm);
+        ColumnDefinition def = cfm.getColumnDefinition(identifier);
+
+        if (def == null)
+            throw new UnrecognizedEntityException(identifier, this);
+
+        return def;
+    }
 }
diff --git a/src/java/org/apache/cassandra/cql3/ResultSet.java b/src/java/org/apache/cassandra/cql3/ResultSet.java
index 85cba57..028691f 100644
--- a/src/java/org/apache/cassandra/cql3/ResultSet.java
+++ b/src/java/org/apache/cassandra/cql3/ResultSet.java
@@ -24,7 +24,6 @@
 
 import org.apache.cassandra.transport.*;
 import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.db.marshal.LongType;
 import org.apache.cassandra.db.marshal.ReversedType;
 import org.apache.cassandra.thrift.Column;
 import org.apache.cassandra.thrift.CqlMetadata;
@@ -37,17 +36,16 @@
 public class ResultSet
 {
     public static final Codec codec = new Codec();
-    public static final ColumnIdentifier COUNT_COLUMN = new ColumnIdentifier("count", false);
 
-    public final Metadata metadata;
+    public final ResultMetadata metadata;
     public final List<List<ByteBuffer>> rows;
 
     public ResultSet(List<ColumnSpecification> metadata)
     {
-        this(new Metadata(metadata), new ArrayList<List<ByteBuffer>>());
+        this(new ResultMetadata(metadata), new ArrayList<List<ByteBuffer>>());
     }
 
-    public ResultSet(Metadata metadata, List<List<ByteBuffer>> rows)
+    public ResultSet(ResultMetadata metadata, List<List<ByteBuffer>> rows)
     {
         this.metadata = metadata;
         this.rows = rows;
@@ -58,6 +56,11 @@
         return rows.size();
     }
 
+    public boolean isEmpty()
+    {
+        return size() == 0;
+    }
+
     public void addRow(List<ByteBuffer> row)
     {
         assert row.size() == metadata.valueCount();
@@ -92,27 +95,6 @@
         }
     }
 
-    public ResultSet makeCountResult(ColumnIdentifier alias)
-    {
-        assert metadata.names != null;
-        String ksName = metadata.names.get(0).ksName;
-        String cfName = metadata.names.get(0).cfName;
-        long count = rows.size();
-        return makeCountResult(ksName, cfName, count, alias);
-    }
-
-    public static ResultSet.Metadata makeCountMetadata(String ksName, String cfName, ColumnIdentifier alias)
-    {
-        ColumnSpecification spec = new ColumnSpecification(ksName, cfName, alias == null ? COUNT_COLUMN : alias, LongType.instance);
-        return new Metadata(Collections.singletonList(spec));
-    }
-
-    public static ResultSet makeCountResult(String ksName, String cfName, long count, ColumnIdentifier alias)
-    {
-        List<List<ByteBuffer>> newRows = Collections.singletonList(Collections.singletonList(ByteBufferUtil.bytes(count)));
-        return new ResultSet(makeCountMetadata(ksName, cfName, alias), newRows);
-    }
-
     public CqlResult toThriftResult()
     {
         assert metadata.names != null;
@@ -197,7 +179,7 @@
          */
         public ResultSet decode(ByteBuf body, int version)
         {
-            Metadata m = Metadata.codec.decode(body, version);
+            ResultMetadata m = ResultMetadata.codec.decode(body, version);
             int rowCount = body.readInt();
             ResultSet rs = new ResultSet(m, new ArrayList<List<ByteBuffer>>(rowCount));
 
@@ -211,12 +193,12 @@
 
         public void encode(ResultSet rs, ByteBuf dest, int version)
         {
-            Metadata.codec.encode(rs.metadata, dest, version);
+            ResultMetadata.codec.encode(rs.metadata, dest, version);
             dest.writeInt(rs.rows.size());
             for (List<ByteBuffer> row : rs.rows)
             {
                 // Note that we do only want to serialize only the first columnCount values, even if the row
-                // as more: see comment on Metadata.names field.
+                // as more: see comment on ResultMetadata.names field.
                 for (int i = 0; i < rs.metadata.columnCount; i++)
                     CBUtil.writeValue(row.get(i), dest);
             }
@@ -224,7 +206,7 @@
 
         public int encodedSize(ResultSet rs, int version)
         {
-            int size = Metadata.codec.encodedSize(rs.metadata, version) + 4;
+            int size = ResultMetadata.codec.encodedSize(rs.metadata, version) + 4;
             for (List<ByteBuffer> row : rs.rows)
             {
                 for (int i = 0; i < rs.metadata.columnCount; i++)
@@ -234,11 +216,14 @@
         }
     }
 
-    public static class Metadata
+    /**
+     * The metadata for the results of executing a query or prepared statement.
+     */
+    public static class ResultMetadata
     {
-        public static final CBCodec<Metadata> codec = new Codec();
+        public static final CBCodec<ResultMetadata> codec = new Codec();
 
-        public static final Metadata EMPTY = new Metadata(EnumSet.of(Flag.NO_METADATA), null, 0, null);
+        public static final ResultMetadata EMPTY = new ResultMetadata(EnumSet.of(Flag.NO_METADATA), null, 0, null);
 
         private final EnumSet<Flag> flags;
         // Please note that columnCount can actually be smaller than names, even if names is not null. This is
@@ -249,14 +234,14 @@
         private final int columnCount;
         private PagingState pagingState;
 
-        public Metadata(List<ColumnSpecification> names)
+        public ResultMetadata(List<ColumnSpecification> names)
         {
             this(EnumSet.noneOf(Flag.class), names, names.size(), null);
-            if (!names.isEmpty() && allInSameCF())
+            if (!names.isEmpty() && ColumnSpecification.allInSameTable(names))
                 flags.add(Flag.GLOBAL_TABLES_SPEC);
         }
 
-        private Metadata(EnumSet<Flag> flags, List<ColumnSpecification> names, int columnCount, PagingState pagingState)
+        private ResultMetadata(EnumSet<Flag> flags, List<ColumnSpecification> names, int columnCount, PagingState pagingState)
         {
             this.flags = flags;
             this.names = names;
@@ -264,9 +249,9 @@
             this.pagingState = pagingState;
         }
 
-        public Metadata copy()
+        public ResultMetadata copy()
         {
-            return new Metadata(EnumSet.copyOf(flags), names, columnCount, pagingState);
+            return new ResultMetadata(EnumSet.copyOf(flags), names, columnCount, pagingState);
         }
 
         /**
@@ -296,24 +281,6 @@
             names.add(name);
         }
 
-        private boolean allInSameCF()
-        {
-            if (names == null)
-                return false;
-
-            assert !names.isEmpty();
-
-            Iterator<ColumnSpecification> iter = names.iterator();
-            ColumnSpecification first = iter.next();
-            while (iter.hasNext())
-            {
-                ColumnSpecification name = iter.next();
-                if (!name.ksName.equals(first.ksName) || !name.cfName.equals(first.cfName))
-                    return false;
-            }
-            return true;
-        }
-
         public void setHasMorePages(PagingState pagingState)
         {
             this.pagingState = pagingState;
@@ -341,7 +308,7 @@
             {
                 for (ColumnSpecification name : names)
                 {
-                    sb.append("[").append(name.name.toString());
+                    sb.append("[").append(name.name);
                     sb.append("(").append(name.ksName).append(", ").append(name.cfName).append(")");
                     sb.append(", ").append(name.type).append("]");
                 }
@@ -351,9 +318,9 @@
             return sb.toString();
         }
 
-        private static class Codec implements CBCodec<Metadata>
+        private static class Codec implements CBCodec<ResultMetadata>
         {
-            public Metadata decode(ByteBuf body, int version)
+            public ResultMetadata decode(ByteBuf body, int version)
             {
                 // flags & column count
                 int iflags = body.readInt();
@@ -366,7 +333,7 @@
                     state = PagingState.deserialize(CBUtil.readValue(body));
 
                 if (flags.contains(Flag.NO_METADATA))
-                    return new Metadata(flags, null, columnCount, state);
+                    return new ResultMetadata(flags, null, columnCount, state);
 
                 boolean globalTablesSpec = flags.contains(Flag.GLOBAL_TABLES_SPEC);
 
@@ -388,16 +355,16 @@
                     AbstractType type = DataType.toType(DataType.codec.decodeOne(body, version));
                     names.add(new ColumnSpecification(ksName, cfName, colName, type));
                 }
-                return new Metadata(flags, names, names.size(), state);
+                return new ResultMetadata(flags, names, names.size(), state);
             }
 
-            public void encode(Metadata m, ByteBuf dest, int version)
+            public void encode(ResultMetadata m, ByteBuf dest, int version)
             {
                 boolean noMetadata = m.flags.contains(Flag.NO_METADATA);
                 boolean globalTablesSpec = m.flags.contains(Flag.GLOBAL_TABLES_SPEC);
                 boolean hasMorePages = m.flags.contains(Flag.HAS_MORE_PAGES);
 
-                assert version > 1 || (!m.flags.contains(Flag.HAS_MORE_PAGES) && !noMetadata): "version = " + version + ", flags = " + m.flags;
+                assert version > 1 || (!hasMorePages && !noMetadata): "version = " + version + ", flags = " + m.flags;
 
                 dest.writeInt(Flag.serialize(m.flags));
                 dest.writeInt(m.columnCount);
@@ -427,7 +394,7 @@
                 }
             }
 
-            public int encodedSize(Metadata m, int version)
+            public int encodedSize(ResultMetadata m, int version)
             {
                 boolean noMetadata = m.flags.contains(Flag.NO_METADATA);
                 boolean globalTablesSpec = m.flags.contains(Flag.GLOBAL_TABLES_SPEC);
@@ -462,6 +429,185 @@
         }
     }
 
+    /**
+     * The metadata for the query parameters in a prepared statement.
+     */
+    public static class PreparedMetadata
+    {
+        public static final CBCodec<PreparedMetadata> codec = new Codec();
+
+        private final EnumSet<Flag> flags;
+        public final List<ColumnSpecification> names;
+        private final Short[] partitionKeyBindIndexes;
+
+        public PreparedMetadata(List<ColumnSpecification> names, Short[] partitionKeyBindIndexes)
+        {
+            this(EnumSet.noneOf(Flag.class), names, partitionKeyBindIndexes);
+            if (!names.isEmpty() && ColumnSpecification.allInSameTable(names))
+                flags.add(Flag.GLOBAL_TABLES_SPEC);
+        }
+
+        private PreparedMetadata(EnumSet<Flag> flags, List<ColumnSpecification> names, Short[] partitionKeyBindIndexes)
+        {
+            this.flags = flags;
+            this.names = names;
+            this.partitionKeyBindIndexes = partitionKeyBindIndexes;
+        }
+
+        public PreparedMetadata copy()
+        {
+            return new PreparedMetadata(EnumSet.copyOf(flags), names, partitionKeyBindIndexes);
+        }
+
+        @Override
+        public boolean equals(Object other)
+        {
+            if (!(other instanceof PreparedMetadata))
+                return false;
+
+            PreparedMetadata that = (PreparedMetadata) other;
+            return this.names.equals(that.names) &&
+                   this.flags.equals(that.flags) &&
+                   Arrays.equals(this.partitionKeyBindIndexes, that.partitionKeyBindIndexes);
+        }
+
+        @Override
+        public String toString()
+        {
+            StringBuilder sb = new StringBuilder();
+            for (ColumnSpecification name : names)
+            {
+                sb.append("[").append(name.name);
+                sb.append("(").append(name.ksName).append(", ").append(name.cfName).append(")");
+                sb.append(", ").append(name.type).append("]");
+            }
+
+            sb.append(", bindIndexes=[");
+            if (partitionKeyBindIndexes != null)
+            {
+                for (int i = 0; i < partitionKeyBindIndexes.length; i++)
+                {
+                    if (i > 0)
+                        sb.append(", ");
+                    sb.append(partitionKeyBindIndexes[i]);
+                }
+            }
+            sb.append("]");
+            return sb.toString();
+        }
+
+        private static class Codec implements CBCodec<PreparedMetadata>
+        {
+            public PreparedMetadata decode(ByteBuf body, int version)
+            {
+                // flags & column count
+                int iflags = body.readInt();
+                int columnCount = body.readInt();
+
+                EnumSet<Flag> flags = Flag.deserialize(iflags);
+
+                Short[] partitionKeyBindIndexes = null;
+                if (version >= Server.VERSION_4)
+                {
+                    int numPKNames = body.readInt();
+                    if (numPKNames > 0)
+                    {
+                        partitionKeyBindIndexes = new Short[numPKNames];
+                        for (int i = 0; i < numPKNames; i++)
+                            partitionKeyBindIndexes[i] = body.readShort();
+                    }
+                }
+
+                boolean globalTablesSpec = flags.contains(Flag.GLOBAL_TABLES_SPEC);
+
+                String globalKsName = null;
+                String globalCfName = null;
+                if (globalTablesSpec)
+                {
+                    globalKsName = CBUtil.readString(body);
+                    globalCfName = CBUtil.readString(body);
+                }
+
+                // metadata (names/types)
+                List<ColumnSpecification> names = new ArrayList<>(columnCount);
+                for (int i = 0; i < columnCount; i++)
+                {
+                    String ksName = globalTablesSpec ? globalKsName : CBUtil.readString(body);
+                    String cfName = globalTablesSpec ? globalCfName : CBUtil.readString(body);
+                    ColumnIdentifier colName = new ColumnIdentifier(CBUtil.readString(body), true);
+                    AbstractType type = DataType.toType(DataType.codec.decodeOne(body, version));
+                    names.add(new ColumnSpecification(ksName, cfName, colName, type));
+                }
+                return new PreparedMetadata(flags, names, partitionKeyBindIndexes);
+            }
+
+            public void encode(PreparedMetadata m, ByteBuf dest, int version)
+            {
+                boolean globalTablesSpec = m.flags.contains(Flag.GLOBAL_TABLES_SPEC);
+                dest.writeInt(Flag.serialize(m.flags));
+                dest.writeInt(m.names.size());
+
+                if (version >= Server.VERSION_4)
+                {
+                    // there's no point in providing partition key bind indexes if the statements affect multiple tables
+                    if (m.partitionKeyBindIndexes == null || !globalTablesSpec)
+                    {
+                        dest.writeInt(0);
+                    }
+                    else
+                    {
+                        dest.writeInt(m.partitionKeyBindIndexes.length);
+                        for (Short bindIndex : m.partitionKeyBindIndexes)
+                            dest.writeShort(bindIndex);
+                    }
+                }
+
+                if (globalTablesSpec)
+                {
+                    CBUtil.writeString(m.names.get(0).ksName, dest);
+                    CBUtil.writeString(m.names.get(0).cfName, dest);
+                }
+
+                for (ColumnSpecification name : m.names)
+                {
+                    if (!globalTablesSpec)
+                    {
+                        CBUtil.writeString(name.ksName, dest);
+                        CBUtil.writeString(name.cfName, dest);
+                    }
+                    CBUtil.writeString(name.name.toString(), dest);
+                    DataType.codec.writeOne(DataType.fromType(name.type, version), dest, version);
+                }
+            }
+
+            public int encodedSize(PreparedMetadata m, int version)
+            {
+                boolean globalTablesSpec = m.flags.contains(Flag.GLOBAL_TABLES_SPEC);
+                int size = 8;
+                if (globalTablesSpec)
+                {
+                    size += CBUtil.sizeOfString(m.names.get(0).ksName);
+                    size += CBUtil.sizeOfString(m.names.get(0).cfName);
+                }
+
+                if (m.partitionKeyBindIndexes != null && version >= 4)
+                    size += 4 + 2 * m.partitionKeyBindIndexes.length;
+
+                for (ColumnSpecification name : m.names)
+                {
+                    if (!globalTablesSpec)
+                    {
+                        size += CBUtil.sizeOfString(name.ksName);
+                        size += CBUtil.sizeOfString(name.cfName);
+                    }
+                    size += CBUtil.sizeOfString(name.name.toString());
+                    size += DataType.codec.oneSerializedSize(DataType.fromType(name.type, version), version);
+                }
+                return size;
+            }
+        }
+    }
+
     public static enum Flag
     {
         // The order of that enum matters!!
diff --git a/src/java/org/apache/cassandra/cql3/RoleName.java b/src/java/org/apache/cassandra/cql3/RoleName.java
new file mode 100644
index 0000000..ce81fa9
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/RoleName.java
@@ -0,0 +1,46 @@
+/*
+ * 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.cql3;
+
+import java.util.Locale;
+
+public class RoleName
+{
+    private String name;
+
+    public void setName(String name, boolean keepCase)
+    {
+        this.name = keepCase ? name : name.toLowerCase(Locale.US);
+    }
+
+    public boolean hasName()
+    {
+        return name != null;
+    }
+
+    public String getName()
+    {
+        return name;
+    }
+
+    @Override
+    public String toString()
+    {
+        return name;
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/Sets.java b/src/java/org/apache/cassandra/cql3/Sets.java
index a7e146f..093f1dc 100644
--- a/src/java/org/apache/cassandra/cql3/Sets.java
+++ b/src/java/org/apache/cassandra/cql3/Sets.java
@@ -17,12 +17,15 @@
  */
 package org.apache.cassandra.cql3;
 
+import static org.apache.cassandra.cql3.Constants.UNSET_VALUE;
+
 import java.nio.ByteBuffer;
 import java.util.*;
 
 import com.google.common.base.Joiner;
 
 import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.db.ColumnFamily;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.composites.Composite;
@@ -67,7 +70,7 @@
                 return new Maps.Value(Collections.<ByteBuffer, ByteBuffer>emptyMap());
 
             ColumnSpecification valueSpec = Sets.valueSpecOf(receiver);
-            Set<Term> values = new HashSet<Term>(elements.size());
+            Set<Term> values = new HashSet<>(elements.size());
             boolean allTerminal = true;
             for (Term.Raw rt : elements)
             {
@@ -100,22 +103,28 @@
             ColumnSpecification valueSpec = Sets.valueSpecOf(receiver);
             for (Term.Raw rt : elements)
             {
-                if (!rt.isAssignableTo(keyspace, valueSpec))
+                if (!rt.testAssignment(keyspace, valueSpec).isAssignable())
                     throw new InvalidRequestException(String.format("Invalid set literal for %s: value %s is not of type %s", receiver.name, rt, valueSpec.type.asCQL3Type()));
             }
         }
 
-        public boolean isAssignableTo(String keyspace, ColumnSpecification receiver)
+        public AssignmentTestable.TestResult testAssignment(String keyspace, ColumnSpecification receiver)
         {
-            try
+            if (!(receiver.type instanceof SetType))
             {
-                validateAssignableTo(keyspace, receiver);
-                return true;
+                // We've parsed empty maps as a set literal to break the ambiguity so handle that case now
+                if (receiver.type instanceof MapType && elements.isEmpty())
+                    return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
+
+                return AssignmentTestable.TestResult.NOT_ASSIGNABLE;
             }
-            catch (InvalidRequestException e)
-            {
-                return false;
-            }
+
+            // If there is no elements, we can't say it's an exact match (an empty set if fundamentally polymorphic).
+            if (elements.isEmpty())
+                return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
+
+            ColumnSpecification valueSpec = Sets.valueSpecOf(receiver);
+            return AssignmentTestable.TestResult.testAll(keyspace, valueSpec, elements);
         }
 
         @Override
@@ -125,7 +134,7 @@
         }
     }
 
-    public static class Value extends Term.Terminal implements Term.CollectionTerminal
+    public static class Value extends Term.Terminal
     {
         public final SortedSet<ByteBuffer> elements;
 
@@ -140,8 +149,8 @@
             {
                 // Collections have this small hack that validate cannot be called on a serialized object,
                 // but compose does the validation (so we're fine).
-                Set<?> s = (Set<?>)type.getSerializer().deserializeForNativeProtocol(value, version);
-                SortedSet<ByteBuffer> elements = new TreeSet<ByteBuffer>(type.getElementsType());
+                Set<?> s = type.getSerializer().deserializeForNativeProtocol(value, version);
+                SortedSet<ByteBuffer> elements = new TreeSet<>(type.getElementsType());
                 for (Object element : s)
                     elements.add(type.getElementsType().decompose(element));
                 return new Value(elements);
@@ -152,14 +161,9 @@
             }
         }
 
-        public ByteBuffer get(QueryOptions options)
+        public ByteBuffer get(int protocolVersion)
         {
-            return getWithProtocolVersion(options.getProtocolVersion());
-        }
-
-        public ByteBuffer getWithProtocolVersion(int protocolVersion)
-        {
-            return CollectionSerializer.pack(new ArrayList<>(elements), elements.size(), protocolVersion);
+            return CollectionSerializer.pack(elements, elements.size(), protocolVersion);
         }
 
         public boolean equals(SetType st, Value v)
@@ -200,7 +204,7 @@
         {
         }
 
-        public Value bind(QueryOptions options) throws InvalidRequestException
+        public Terminal bind(QueryOptions options) throws InvalidRequestException
         {
             SortedSet<ByteBuffer> buffers = new TreeSet<>(comparator);
             for (Term t : elements)
@@ -209,6 +213,8 @@
 
                 if (bytes == null)
                     throw new InvalidRequestException("null is not supported inside collections");
+                if (bytes == ByteBufferUtil.UNSET_BYTE_BUFFER)
+                    return UNSET_VALUE;
 
                 // We don't support value > 64K because the serialization format encode the length as an unsigned short.
                 if (bytes.remaining() > FBUtilities.MAX_UNSIGNED_SHORT)
@@ -220,6 +226,11 @@
             }
             return new Value(buffers);
         }
+
+        public Iterable<Function> getFunctions()
+        {
+            return Terms.getFunctions(elements);
+        }
     }
 
     public static class Marker extends AbstractMarker
@@ -230,10 +241,14 @@
             assert receiver.type instanceof SetType;
         }
 
-        public Value bind(QueryOptions options) throws InvalidRequestException
+        public Terminal bind(QueryOptions options) throws InvalidRequestException
         {
             ByteBuffer value = options.getValues().get(bindIndex);
-            return value == null ? null : Value.fromSerialized(value, (SetType)receiver.type, options.getProtocolVersion());
+            if (value == null)
+                return null;
+            if (value == ByteBufferUtil.UNSET_BYTE_BUFFER)
+                return UNSET_VALUE;
+            return Value.fromSerialized(value, (SetType)receiver.type, options.getProtocolVersion());
         }
     }
 
@@ -246,13 +261,15 @@
 
         public void execute(ByteBuffer rowKey, ColumnFamily cf, Composite prefix, UpdateParameters params) throws InvalidRequestException
         {
-            if (column.type.isMultiCell())
+            Term.Terminal value = t.bind(params.options);
+            if (column.type.isMultiCell() && value != UNSET_VALUE)
             {
                 // delete + add
                 CellName name = cf.getComparator().create(prefix, column);
                 cf.addAtom(params.makeTombstoneForOverwrite(name.slice()));
             }
-            Adder.doAdd(t, cf, prefix, column, params);
+            if (value != UNSET_VALUE)
+                Adder.doAdd(cf, prefix, column, params, value);
         }
     }
 
@@ -266,22 +283,22 @@
         public void execute(ByteBuffer rowKey, ColumnFamily cf, Composite prefix, UpdateParameters params) throws InvalidRequestException
         {
             assert column.type.isMultiCell() : "Attempted to add items to a frozen set";
-
-            doAdd(t, cf, prefix, column, params);
+            Term.Terminal value = t.bind(params.options);
+            if (value != UNSET_VALUE)
+                doAdd(cf, prefix, column, params, value);
         }
 
-        static void doAdd(Term t, ColumnFamily cf, Composite prefix, ColumnDefinition column, UpdateParameters params) throws InvalidRequestException
+        static void doAdd(ColumnFamily cf, Composite prefix, ColumnDefinition column, UpdateParameters params, Term.Terminal value) throws InvalidRequestException
         {
-            Term.Terminal value = t.bind(params.options);
-            Sets.Value setValue = (Sets.Value)value;
             if (column.type.isMultiCell())
             {
                 if (value == null)
                     return;
 
-                Set<ByteBuffer> toAdd = setValue.elements;
-                for (ByteBuffer bb : toAdd)
+                for (ByteBuffer bb : ((Value) value).elements)
                 {
+                    if (bb == ByteBufferUtil.UNSET_BYTE_BUFFER)
+                        continue;
                     CellName cellName = cf.getComparator().create(prefix, column, bb);
                     cf.addColumn(params.makeColumn(cellName, ByteBufferUtil.EMPTY_BYTE_BUFFER));
                 }
@@ -293,7 +310,7 @@
                 if (value == null)
                     cf.addAtom(params.makeTombstone(cellName));
                 else
-                    cf.addColumn(params.makeColumn(cellName, ((Value) value).getWithProtocolVersion(Server.CURRENT_VERSION)));
+                    cf.addColumn(params.makeColumn(cellName, value.get(Server.CURRENT_VERSION)));
             }
         }
     }
@@ -311,18 +328,16 @@
             assert column.type.isMultiCell() : "Attempted to remove items from a frozen set";
 
             Term.Terminal value = t.bind(params.options);
-            if (value == null)
+            if (value == null || value == UNSET_VALUE)
                 return;
 
             // This can be either a set or a single element
             Set<ByteBuffer> toDiscard = value instanceof Sets.Value
                                       ? ((Sets.Value)value).elements
-                                      : Collections.singleton(value.get(params.options));
+                                      : Collections.singleton(value.get(params.options.getProtocolVersion()));
 
             for (ByteBuffer bb : toDiscard)
-            {
                 cf.addColumn(params.makeTombstone(cf.getComparator().create(prefix, column, bb)));
-            }
         }
     }
 
@@ -340,7 +355,7 @@
             if (elt == null)
                 throw new InvalidRequestException("Invalid null set element");
 
-            CellName cellName = cf.getComparator().create(prefix, column, elt.get(params.options));
+            CellName cellName = cf.getComparator().create(prefix, column, elt.get(params.options.getProtocolVersion()));
             cf.addColumn(params.makeTombstone(cellName));
         }
     }
diff --git a/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java b/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java
index 676064d..dbae5f0 100644
--- a/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java
+++ b/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java
@@ -17,27 +17,56 @@
  */
 package org.apache.cassandra.cql3;
 
+import java.util.Collections;
+import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.Term.Raw;
+import org.apache.cassandra.cql3.restrictions.Restriction;
+import org.apache.cassandra.cql3.restrictions.SingleColumnRestriction;
+import org.apache.cassandra.cql3.statements.Bound;
+import org.apache.cassandra.db.marshal.CollectionType;
+import org.apache.cassandra.db.marshal.ListType;
+import org.apache.cassandra.db.marshal.MapType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+
 /**
  * Relations encapsulate the relationship between an entity of some kind, and
  * a value (term). For example, <key> > "start" or "colname1" = "somevalue".
  *
  */
-public class SingleColumnRelation extends Relation
+public final class SingleColumnRelation extends Relation
 {
     private final ColumnIdentifier.Raw entity;
+    private final Term.Raw mapKey;
     private final Term.Raw value;
     private final List<Term.Raw> inValues;
-    public final boolean onToken;
 
-    private SingleColumnRelation(ColumnIdentifier.Raw entity, Operator type, Term.Raw value, List<Term.Raw> inValues, boolean onToken)
+    private SingleColumnRelation(ColumnIdentifier.Raw entity, Term.Raw mapKey, Operator type, Term.Raw value, List<Term.Raw> inValues)
     {
         this.entity = entity;
+        this.mapKey = mapKey;
         this.relationType = type;
         this.value = value;
         this.inValues = inValues;
-        this.onToken = onToken;
+    }
+
+    /**
+     * Creates a new relation.
+     *
+     * @param entity the kind of relation this is; what the term is being compared to.
+     * @param mapKey the key into the entity identifying the value the term is being compared to.
+     * @param type the type that describes how this entity relates to the value.
+     * @param value the value being compared.
+     */
+    public SingleColumnRelation(ColumnIdentifier.Raw entity, Term.Raw mapKey, Operator type, Term.Raw value)
+    {
+        this(entity, mapKey, type, value, null);
     }
 
     /**
@@ -49,17 +78,12 @@
      */
     public SingleColumnRelation(ColumnIdentifier.Raw entity, Operator type, Term.Raw value)
     {
-        this(entity, type, value, null, false);
-    }
-
-    public SingleColumnRelation(ColumnIdentifier.Raw entity, Operator type, Term.Raw value, boolean onToken)
-    {
-        this(entity, type, value, null, onToken);
+        this(entity, null, type, value);
     }
 
     public static SingleColumnRelation createInRelation(ColumnIdentifier.Raw entity, List<Term.Raw> inValues)
     {
-        return new SingleColumnRelation(entity, Operator.IN, null, inValues, false);
+        return new SingleColumnRelation(entity, null, Operator.IN, null, inValues);
     }
 
     public ColumnIdentifier.Raw getEntity()
@@ -67,26 +91,23 @@
         return entity;
     }
 
-    public Term.Raw getValue()
+    public Term.Raw getMapKey()
     {
-        assert relationType != Operator.IN || value == null || value instanceof AbstractMarker.INRaw;
-        return value;
+        return mapKey;
     }
 
-    public List<Term.Raw> getInValues()
+    @Override
+    protected Term toTerm(List<? extends ColumnSpecification> receivers,
+                          Raw raw,
+                          String keyspace,
+                          VariableSpecifications boundNames)
+                          throws InvalidRequestException
     {
-        assert relationType == Operator.IN;
-        return inValues;
-    }
+        assert receivers.size() == 1;
 
-    public boolean isMultiColumn()
-    {
-        return false;
-    }
-
-    public boolean isOnToken()
-    {
-        return onToken;
+        Term term = raw.prepare(keyspace, receivers.get(0));
+        term.collectMarkerSpecification(boundNames);
+        return term;
     }
 
     public SingleColumnRelation withNonStrictOperator()
@@ -94,7 +115,7 @@
         switch (relationType)
         {
             case GT: return new SingleColumnRelation(entity, Operator.GTE, value);
-            case LT:  return new SingleColumnRelation(entity, Operator.LTE, value);
+            case LT: return new SingleColumnRelation(entity, Operator.LTE, value);
             default: return this;
         }
     }
@@ -102,11 +123,152 @@
     @Override
     public String toString()
     {
-        if (relationType == Operator.IN)
-            return String.format("%s IN %s", entity, inValues);
-        else if (onToken)
-            return String.format("token(%s) %s %s", entity, relationType, value);
-        else
-            return String.format("%s %s %s", entity, relationType, value);
+        String entityAsString = entity.toString();
+        if (mapKey != null)
+            entityAsString = String.format("%s[%s]", entityAsString, mapKey);
+
+        if (isIN())
+            return String.format("%s IN %s", entityAsString, inValues);
+
+        return String.format("%s %s %s", entityAsString, relationType, value);
+    }
+
+    @Override
+    protected Restriction newEQRestriction(CFMetaData cfm,
+                                           VariableSpecifications boundNames) throws InvalidRequestException
+    {
+        ColumnDefinition columnDef = toColumnDefinition(cfm, entity);
+        if (mapKey == null)
+        {
+            Term term = toTerm(toReceivers(columnDef), value, cfm.ksName, boundNames);
+            return new SingleColumnRestriction.EQ(columnDef, term);
+        }
+        List<? extends ColumnSpecification> receivers = toReceivers(columnDef);
+        Term entryKey = toTerm(Collections.singletonList(receivers.get(0)), mapKey, cfm.ksName, boundNames);
+        Term entryValue = toTerm(Collections.singletonList(receivers.get(1)), value, cfm.ksName, boundNames);
+        return new SingleColumnRestriction.Contains(columnDef, entryKey, entryValue);
+    }
+
+    @Override
+    protected Restriction newINRestriction(CFMetaData cfm,
+                                           VariableSpecifications boundNames) throws InvalidRequestException
+    {
+        ColumnDefinition columnDef = toColumnDefinition(cfm, entity);
+        List<? extends ColumnSpecification> receivers = toReceivers(columnDef);
+        List<Term> terms = toTerms(receivers, inValues, cfm.ksName, boundNames);
+        if (terms == null)
+        {
+            Term term = toTerm(receivers, value, cfm.ksName, boundNames);
+            return new SingleColumnRestriction.InWithMarker(columnDef, (Lists.Marker) term);
+        }
+        return new SingleColumnRestriction.InWithValues(columnDef, terms);
+    }
+
+    @Override
+    protected Restriction newSliceRestriction(CFMetaData cfm,
+                                              VariableSpecifications boundNames,
+                                              Bound bound,
+                                              boolean inclusive) throws InvalidRequestException
+    {
+        ColumnDefinition columnDef = toColumnDefinition(cfm, entity);
+        Term term = toTerm(toReceivers(columnDef), value, cfm.ksName, boundNames);
+        return new SingleColumnRestriction.Slice(columnDef, bound, inclusive, term);
+    }
+
+    @Override
+    protected Restriction newContainsRestriction(CFMetaData cfm,
+                                                 VariableSpecifications boundNames,
+                                                 boolean isKey) throws InvalidRequestException
+    {
+        ColumnDefinition columnDef = toColumnDefinition(cfm, entity);
+        Term term = toTerm(toReceivers(columnDef), value, cfm.ksName, boundNames);
+        return new SingleColumnRestriction.Contains(columnDef, term, isKey);
+    }
+
+    /**
+     * Returns the receivers for this relation.
+     * @param columnDef the column definition
+     *
+     * @return the receivers for the specified relation.
+     * @throws InvalidRequestException if the relation is invalid
+     */
+    private List<? extends ColumnSpecification> toReceivers(ColumnDefinition columnDef) throws InvalidRequestException
+    {
+        ColumnSpecification receiver = columnDef;
+
+        if (isIN())
+        {
+            // We only allow IN on the row key and the clustering key so far, never on non-PK columns, and this even if
+            // there's an index
+            // Note: for backward compatibility reason, we conside a IN of 1 value the same as a EQ, so we let that
+            // slide.
+            checkFalse(!columnDef.isPrimaryKeyColumn() && !canHaveOnlyOneValue(),
+                       "IN predicates on non-primary-key columns (%s) is not yet supported", columnDef.name);
+        }
+        else if (isSlice())
+        {
+            // Non EQ relation is not supported without token(), even if we have a 2ndary index (since even those
+            // are ordered by partitioner).
+            // Note: In theory we could allow it for 2ndary index queries with ALLOW FILTERING, but that would
+            // probably require some special casing
+            // Note bis: This is also why we don't bother handling the 'tuple' notation of #4851 for keys. If we
+            // lift the limitation for 2ndary
+            // index with filtering, we'll need to handle it though.
+            checkFalse(columnDef.isPartitionKey(), "Only EQ and IN relation are supported on the partition key (unless you use the token() function)");
+        }
+
+        checkFalse(isContainsKey() && !(receiver.type instanceof MapType), "Cannot use CONTAINS KEY on non-map column %s", receiver.name);
+
+        if (mapKey != null)
+        {
+            checkFalse(receiver.type instanceof ListType, "Indexes on list entries (%s[index] = value) are not currently supported.", receiver.name);
+            checkTrue(receiver.type instanceof MapType, "Column %s cannot be used as a map", receiver.name);
+            checkTrue(receiver.type.isMultiCell(), "Map-entry equality predicates on frozen map column %s are not supported", receiver.name);
+            checkTrue(isEQ(), "Only EQ relations are supported on map entries");
+        }
+
+        if (receiver.type.isCollection())
+        {
+            // We don't support relations against entire collections (unless they're frozen), like "numbers = {1, 2, 3}"
+            checkFalse(receiver.type.isMultiCell() && !isLegalRelationForNonFrozenCollection(),
+                       "Collection column '%s' (%s) cannot be restricted by a '%s' relation",
+                       receiver.name,
+                       receiver.type.asCQL3Type(),
+                       operator());
+
+            if (isContainsKey() || isContains())
+            {
+                receiver = makeCollectionReceiver(receiver, isContainsKey());
+            }
+            else if (receiver.type.isMultiCell() && mapKey != null && isEQ())
+            {
+                List<ColumnSpecification> receivers = new ArrayList<>(2);
+                receivers.add(makeCollectionReceiver(receiver, true));
+                receivers.add(makeCollectionReceiver(receiver, false));
+                return receivers;
+            }
+        }
+
+        return Collections.singletonList(receiver);
+    }
+
+    private static ColumnSpecification makeCollectionReceiver(ColumnSpecification receiver, boolean forKey)
+    {
+        return ((CollectionType<?>) receiver.type).makeCollectionReceiver(receiver, forKey);
+    }
+
+    private boolean isLegalRelationForNonFrozenCollection()
+    {
+        return isContainsKey() || isContains() || isMapEntryEquality();
+    }
+
+    private boolean isMapEntryEquality()
+    {
+        return mapKey != null && isEQ();
+    }
+
+    private boolean canHaveOnlyOneValue()
+    {
+        return isEQ() || (isIN() && inValues != null && inValues.size() == 1);
     }
 }
diff --git a/src/java/org/apache/cassandra/cql3/Term.java b/src/java/org/apache/cassandra/cql3/Term.java
index 1587df1..6fa0c76 100644
--- a/src/java/org/apache/cassandra/cql3/Term.java
+++ b/src/java/org/apache/cassandra/cql3/Term.java
@@ -18,8 +18,11 @@
 package org.apache.cassandra.cql3;
 
 import java.nio.ByteBuffer;
+import java.util.Collections;
 import java.util.List;
+import java.util.Set;
 
+import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 
 /**
@@ -67,6 +70,8 @@
      */
     public abstract boolean containsBindMarker();
 
+    Iterable<Function> getFunctions();
+
     /**
      * A parsed, non prepared (thus untyped) term.
      *
@@ -76,7 +81,7 @@
      *   - a function call
      *   - a marker
      */
-    public interface Raw extends AssignementTestable
+    public interface Raw extends AssignmentTestable
     {
         /**
          * This method validates this RawTerm is valid for provided column
@@ -115,6 +120,11 @@
         public void collectMarkerSpecification(VariableSpecifications boundNames) {}
         public Terminal bind(QueryOptions options) { return this; }
 
+        public Set<Function> getFunctions()
+        {
+            return Collections.emptySet();
+        }
+
         // While some NonTerminal may not have bind markers, no Term can be Terminal
         // with a bind marker
         public boolean containsBindMarker()
@@ -124,12 +134,13 @@
 
         /**
          * @return the serialized value of this terminal.
+         * @param protocolVersion
          */
-        public abstract ByteBuffer get(QueryOptions options);
+        public abstract ByteBuffer get(int protocolVersion) throws InvalidRequestException;
 
         public ByteBuffer bindAndGet(QueryOptions options) throws InvalidRequestException
         {
-            return get(options);
+            return get(options.getProtocolVersion());
         }
     }
 
@@ -138,12 +149,6 @@
         public abstract List<ByteBuffer> getElements();
     }
 
-    public interface CollectionTerminal
-    {
-        /** Gets the value of the collection when serialized with the given protocol version format */
-        public ByteBuffer getWithProtocolVersion(int protocolVersion);
-    }
-
     /**
      * A non terminal term, i.e. a term that can only be reduce to a byte buffer
      * at execution time.
@@ -159,7 +164,7 @@
         public ByteBuffer bindAndGet(QueryOptions options) throws InvalidRequestException
         {
             Terminal t = bind(options);
-            return t == null ? null : t.get(options);
+            return t == null ? null : t.get(options.getProtocolVersion());
         }
     }
 }
diff --git a/src/java/org/apache/cassandra/cql3/Terms.java b/src/java/org/apache/cassandra/cql3/Terms.java
new file mode 100644
index 0000000..0b049b9
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/Terms.java
@@ -0,0 +1,45 @@
+/*
+ * 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.cql3;
+
+import java.util.Collections;
+
+import com.google.common.collect.Iterables;
+
+import org.apache.cassandra.cql3.functions.Function;
+
+public class Terms
+{
+
+    private static com.google.common.base.Function<Term, Iterable<Function>> TO_FUNCTION_ITERABLE =
+    new com.google.common.base.Function<Term, Iterable<Function>>()
+    {
+        public Iterable<Function> apply(Term term)
+        {
+            return term.getFunctions();
+        }
+    };
+
+    public static Iterable<Function> getFunctions(Iterable<Term> terms)
+    {
+        if (terms == null)
+            return Collections.emptySet();
+
+        return Iterables.concat(Iterables.transform(terms, TO_FUNCTION_ITERABLE));
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/TokenRelation.java b/src/java/org/apache/cassandra/cql3/TokenRelation.java
new file mode 100644
index 0000000..46a812c
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/TokenRelation.java
@@ -0,0 +1,164 @@
+/*
+ * 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.cql3;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import com.google.common.base.Joiner;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.Term.Raw;
+import org.apache.cassandra.cql3.restrictions.Restriction;
+import org.apache.cassandra.cql3.restrictions.TokenRestriction;
+import org.apache.cassandra.cql3.statements.Bound;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.service.StorageService;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkContainsNoDuplicates;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkContainsOnly;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
+
+/**
+ * A relation using the token function.
+ * Examples:
+ * <ul>
+ * <li>SELECT ... WHERE token(a) &gt; token(1)</li>
+ * <li>SELECT ... WHERE token(a, b) &gt; token(1, 3)</li>
+ * </ul>
+ */
+public final class TokenRelation extends Relation
+{
+    private final List<ColumnIdentifier.Raw> entities;
+
+    private final Term.Raw value;
+
+    public TokenRelation(List<ColumnIdentifier.Raw> entities, Operator type, Term.Raw value)
+    {
+        this.entities = entities;
+        this.relationType = type;
+        this.value = value;
+    }
+
+    @Override
+    public boolean onToken()
+    {
+        return true;
+    }
+
+    @Override
+    protected Restriction newEQRestriction(CFMetaData cfm, VariableSpecifications boundNames) throws InvalidRequestException
+    {
+        List<ColumnDefinition> columnDefs = getColumnDefinitions(cfm);
+        Term term = toTerm(toReceivers(cfm, columnDefs), value, cfm.ksName, boundNames);
+        return new TokenRestriction.EQ(cfm.getKeyValidatorAsCType(), columnDefs, term);
+    }
+
+    @Override
+    protected Restriction newINRestriction(CFMetaData cfm, VariableSpecifications boundNames) throws InvalidRequestException
+    {
+        throw invalidRequest("%s cannot be used with the token function", operator());
+    }
+
+    @Override
+    protected Restriction newSliceRestriction(CFMetaData cfm,
+                                              VariableSpecifications boundNames,
+                                              Bound bound,
+                                              boolean inclusive) throws InvalidRequestException
+    {
+        List<ColumnDefinition> columnDefs = getColumnDefinitions(cfm);
+        Term term = toTerm(toReceivers(cfm, columnDefs), value, cfm.ksName, boundNames);
+        return new TokenRestriction.Slice(cfm.getKeyValidatorAsCType(), columnDefs, bound, inclusive, term);
+    }
+
+    @Override
+    protected Restriction newContainsRestriction(CFMetaData cfm, VariableSpecifications boundNames, boolean isKey) throws InvalidRequestException
+    {
+        throw invalidRequest("%s cannot be used with the token function", operator());
+    }
+
+    @Override
+    protected Term toTerm(List<? extends ColumnSpecification> receivers,
+                          Raw raw,
+                          String keyspace,
+                          VariableSpecifications boundNames) throws InvalidRequestException
+    {
+        Term term = raw.prepare(keyspace, receivers.get(0));
+        term.collectMarkerSpecification(boundNames);
+        return term;
+    }
+
+    @Override
+    public String toString()
+    {
+        return String.format("token%s %s %s", Tuples.tupleToString(entities), relationType, value);
+    }
+
+    /**
+     * Returns the definition of the columns to which apply the token restriction.
+     *
+     * @param cfm the column family metadata
+     * @return the definition of the columns to which apply the token restriction.
+     * @throws InvalidRequestException if the entity cannot be resolved
+     */
+    private List<ColumnDefinition> getColumnDefinitions(CFMetaData cfm) throws InvalidRequestException
+    {
+        List<ColumnDefinition> columnDefs = new ArrayList<>();
+        for ( ColumnIdentifier.Raw raw : entities)
+        {
+            columnDefs.add(toColumnDefinition(cfm, raw));
+        }
+        return columnDefs;
+    }
+
+    /**
+     * Returns the receivers for this relation.
+     *
+     * @param cfm the Column Family meta data
+     * @param columnDefs the column definitions
+     * @return the receivers for the specified relation.
+     * @throws InvalidRequestException if the relation is invalid
+     */
+    private static List<? extends ColumnSpecification> toReceivers(CFMetaData cfm,
+                                                                   List<ColumnDefinition> columnDefs)
+                                                                   throws InvalidRequestException
+    {
+
+        if (!columnDefs.equals(cfm.partitionKeyColumns()))
+        {
+            checkTrue(columnDefs.containsAll(cfm.partitionKeyColumns()),
+                      "The token() function must be applied to all partition key components or none of them");
+
+            checkContainsNoDuplicates(columnDefs, "The token() function contains duplicate partition key components");
+
+            checkContainsOnly(columnDefs, cfm.partitionKeyColumns(), "The token() function must contains only partition key components");
+
+            throw invalidRequest("The token function arguments must be in the partition key order: %s",
+                                 Joiner.on(", ").join(ColumnDefinition.toIdentifiers(cfm.partitionKeyColumns())));
+        }
+
+        ColumnDefinition firstColumn = columnDefs.get(0);
+        return Collections.singletonList(new ColumnSpecification(firstColumn.ksName,
+                                                                 firstColumn.cfName,
+                                                                 new ColumnIdentifier("partition key token", true),
+                                                                 StorageService.getPartitioner().getTokenValidator()));
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/Tuples.java b/src/java/org/apache/cassandra/cql3/Tuples.java
index 4f1f141..89fecd0 100644
--- a/src/java/org/apache/cassandra/cql3/Tuples.java
+++ b/src/java/org/apache/cassandra/cql3/Tuples.java
@@ -18,14 +18,19 @@
 package org.apache.cassandra.cql3;
 
 import java.nio.ByteBuffer;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.cql3.Term.MultiColumnRaw;
+import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.serializers.MarshalException;
+import org.apache.cassandra.utils.ByteBufferUtil;
 
 /**
  * Static helper methods and classes for tuples.
@@ -110,21 +115,21 @@
 
                 Term.Raw value = elements.get(i);
                 ColumnSpecification spec = componentSpecOf(receiver, i);
-                if (!value.isAssignableTo(keyspace, spec))
+                if (!value.testAssignment(keyspace, spec).isAssignable())
                     throw new InvalidRequestException(String.format("Invalid tuple literal for %s: component %d is not of type %s", receiver.name, i, spec.type.asCQL3Type()));
             }
         }
 
-        public boolean isAssignableTo(String keyspace, ColumnSpecification receiver)
+        public AssignmentTestable.TestResult testAssignment(String keyspace, ColumnSpecification receiver)
         {
             try
             {
                 validateAssignableTo(keyspace, receiver);
-                return true;
+                return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
             }
             catch (InvalidRequestException e)
             {
-                return false;
+                return AssignmentTestable.TestResult.NOT_ASSIGNABLE;
             }
         }
 
@@ -152,7 +157,7 @@
             return new Value(type.split(bytes));
         }
 
-        public ByteBuffer get(QueryOptions options)
+        public ByteBuffer get(int protocolVersion)
         {
             return TupleType.buildValue(elements);
         }
@@ -200,6 +205,9 @@
             for (int i = 0; i < elements.size(); i++)
             {
                 buffers[i] = elements.get(i).bindAndGet(options);
+                // Since A tuple value is always written in its entirety Cassandra can't preserve a pre-existing value by 'not setting' the new value. Reject the query.
+                if (buffers[i] == ByteBufferUtil.UNSET_BYTE_BUFFER)
+                    throw new InvalidRequestException(String.format("Invalid unset value for tuple field number %d", i));
                 // Inside tuples, we must force the serialization of collections to v3 whatever protocol
                 // version is in use since we're going to store directly that serialized value.
                 if (version < 3 && type.type(i).isCollection())
@@ -225,6 +233,11 @@
         {
             return tupleToString(elements);
         }
+
+        public Iterable<Function> getFunctions()
+        {
+            return Terms.getFunctions(elements);
+        }
     }
 
     /**
@@ -246,7 +259,7 @@
             {
                 // Collections have this small hack that validate cannot be called on a serialized object,
                 // but the deserialization does the validation (so we're fine).
-                List<?> l = (List<?>)type.getSerializer().deserializeForNativeProtocol(value, options.getProtocolVersion());
+                List<?> l = type.getSerializer().deserializeForNativeProtocol(value, options.getProtocolVersion());
 
                 assert type.getElementsType() instanceof TupleType;
                 TupleType tupleType = (TupleType) type.getElementsType();
@@ -263,7 +276,7 @@
             }
         }
 
-        public ByteBuffer get(QueryOptions options)
+        public ByteBuffer get(int protocolVersion)
         {
             throw new UnsupportedOperationException();
         }
@@ -276,7 +289,9 @@
 
     /**
      * A raw placeholder for a tuple of values for different multiple columns, each of which may have a different type.
+     * {@code
      * For example, "SELECT ... WHERE (col1, col2) > ?".
+     * }
      */
     public static class Raw extends AbstractMarker.Raw implements Term.MultiColumnRaw
     {
@@ -285,7 +300,7 @@
             super(bindIndex);
         }
 
-        private static ColumnSpecification makeReceiver(List<? extends ColumnSpecification> receivers) throws InvalidRequestException
+        private static ColumnSpecification makeReceiver(List<? extends ColumnSpecification> receivers)
         {
             List<AbstractType<?>> types = new ArrayList<>(receivers.size());
             StringBuilder inName = new StringBuilder("(");
@@ -319,7 +334,7 @@
     /**
      * A raw marker for an IN list of tuples, like "SELECT ... WHERE (a, b, c) IN ?"
      */
-    public static class INRaw extends AbstractMarker.Raw
+    public static class INRaw extends AbstractMarker.Raw implements MultiColumnRaw
     {
         public INRaw(int bindIndex)
         {
@@ -362,7 +377,9 @@
     }
 
     /**
+     * {@code
      * Represents a marker for a single tuple, like "SELECT ... WHERE (a, b, c) > ?"
+     * }
      */
     public static class Marker extends AbstractMarker
     {
@@ -374,6 +391,8 @@
         public Value bind(QueryOptions options) throws InvalidRequestException
         {
             ByteBuffer value = options.getValues().get(bindIndex);
+            if (value == ByteBufferUtil.UNSET_BYTE_BUFFER)
+                throw new InvalidRequestException(String.format("Invalid unset value for tuple %s", receiver.name));
             return value == null ? null : Value.fromSerialized(value, (TupleType)receiver.type);
         }
     }
@@ -392,6 +411,8 @@
         public InValue bind(QueryOptions options) throws InvalidRequestException
         {
             ByteBuffer value = options.getValues().get(bindIndex);
+            if (value == ByteBufferUtil.UNSET_BYTE_BUFFER)
+                throw new InvalidRequestException(String.format("Invalid unset value for %s", receiver.name));
             return value == null ? null : InValue.fromSerialized(value, (ListType)receiver.type, options);
         }
     }
diff --git a/src/java/org/apache/cassandra/cql3/TypeCast.java b/src/java/org/apache/cassandra/cql3/TypeCast.java
index e325e4d..561a158 100644
--- a/src/java/org/apache/cassandra/cql3/TypeCast.java
+++ b/src/java/org/apache/cassandra/cql3/TypeCast.java
@@ -17,6 +17,7 @@
  */
 package org.apache.cassandra.cql3;
 
+import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 
 public class TypeCast implements Term.Raw
@@ -32,11 +33,11 @@
 
     public Term prepare(String keyspace, ColumnSpecification receiver) throws InvalidRequestException
     {
-        if (!term.isAssignableTo(keyspace, castedSpecOf(keyspace, receiver)))
+        if (!term.testAssignment(keyspace, castedSpecOf(keyspace, receiver)).isAssignable())
             throw new InvalidRequestException(String.format("Cannot cast value %s to type %s", term, type));
 
-        if (!isAssignableTo(keyspace, receiver))
-            throw new InvalidRequestException(String.format("Cannot assign value %s to %s of type %s", this, receiver, receiver.type.asCQL3Type()));
+        if (!testAssignment(keyspace, receiver).isAssignable())
+            throw new InvalidRequestException(String.format("Cannot assign value %s to %s of type %s", this, receiver.name, receiver.type.asCQL3Type()));
 
         return term.prepare(keyspace, receiver);
     }
@@ -46,16 +47,15 @@
         return new ColumnSpecification(receiver.ksName, receiver.cfName, new ColumnIdentifier(toString(), true), type.prepare(keyspace).getType());
     }
 
-    public boolean isAssignableTo(String keyspace, ColumnSpecification receiver) throws InvalidRequestException
+    public AssignmentTestable.TestResult testAssignment(String keyspace, ColumnSpecification receiver)
     {
-        try
-        {
-            return receiver.type.isValueCompatibleWith(type.prepare(keyspace).getType());
-        }
-        catch (InvalidRequestException e)
-        {
-            throw new AssertionError();
-        }
+        AbstractType<?> castedType = type.prepare(keyspace).getType();
+        if (receiver.type.equals(castedType))
+            return AssignmentTestable.TestResult.EXACT_MATCH;
+        else if (receiver.type.isValueCompatibleWith(castedType))
+            return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
+        else
+            return AssignmentTestable.TestResult.NOT_ASSIGNABLE;
     }
 
     @Override
diff --git a/src/java/org/apache/cassandra/cql3/UntypedResultSet.java b/src/java/org/apache/cassandra/cql3/UntypedResultSet.java
index a0b6ae7..e8d610d 100644
--- a/src/java/org/apache/cassandra/cql3/UntypedResultSet.java
+++ b/src/java/org/apache/cassandra/cql3/UntypedResultSet.java
@@ -26,7 +26,6 @@
 
 import org.apache.cassandra.cql3.statements.SelectStatement;
 import org.apache.cassandra.db.marshal.*;
-import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.service.pager.QueryPager;
 
 /** a utility for doing internal cql-based queries */
@@ -74,8 +73,8 @@
 
         public Row one()
         {
-            if (cqlRows.rows.size() != 1)
-                throw new IllegalStateException("One row required, " + cqlRows.rows.size() + " found");
+            if (cqlRows.size() != 1)
+                throw new IllegalStateException("One row required, " + cqlRows.size() + " found");
             return new Row(cqlRows.metadata.requestNames(), cqlRows.rows.get(0));
         }
 
@@ -175,17 +174,13 @@
 
                 protected Row computeNext()
                 {
-                    try {
-                        while (currentPage == null || !currentPage.hasNext())
-                        {
-                            if (pager.isExhausted())
-                                return endOfData();
-                            currentPage = select.process(pager.fetchPage(pageSize)).rows.iterator();
-                        }
-                        return new Row(metadata, currentPage.next());
-                    } catch (RequestValidationException | RequestExecutionException e) {
-                        throw new RuntimeException(e);
+                    while (currentPage == null || !currentPage.hasNext())
+                    {
+                        if (pager.isExhausted())
+                            return endOfData();
+                        currentPage = select.process(pager.fetchPage(pageSize)).rows.iterator();
                     }
+                    return new Row(metadata, currentPage.next());
                 }
             };
         }
@@ -234,6 +229,16 @@
             return BooleanType.instance.compose(data.get(column));
         }
 
+        public byte getByte(String column)
+        {
+            return ByteType.instance.compose(data.get(column));
+        }
+
+        public short getShort(String column)
+        {
+            return ShortType.instance.compose(data.get(column));
+        }
+
         public int getInt(String column)
         {
             return Int32Type.instance.compose(data.get(column));
diff --git a/src/java/org/apache/cassandra/cql3/UpdateParameters.java b/src/java/org/apache/cassandra/cql3/UpdateParameters.java
index e412585..65edef7 100644
--- a/src/java/org/apache/cassandra/cql3/UpdateParameters.java
+++ b/src/java/org/apache/cassandra/cql3/UpdateParameters.java
@@ -91,16 +91,39 @@
         return new RangeTombstone(slice.start, slice.finish, timestamp - 1, localDeletionTime);
     }
 
-    public List<Cell> getPrefetchedList(ByteBuffer rowKey, ColumnIdentifier cql3ColumnName)
+    /**
+     * Returns the prefetched list with the already performed modifications.
+     * <p>If no modification have yet been performed this method will return the fetched list.
+     * If some modifications (updates or deletions) have already been done the list returned
+     * will be the result of the merge of the fetched list and of the pending mutations.</p>
+     *
+     * @param rowKey the row key
+     * @param cql3ColumnName the column name
+     * @param cf the pending modifications
+     * @return the prefetched list with the already performed modifications
+     */
+    public List<Cell> getPrefetchedList(ByteBuffer rowKey, ColumnIdentifier cql3ColumnName, ColumnFamily cf)
     {
         if (prefetchedLists == null)
             return Collections.emptyList();
 
         CQL3Row row = prefetchedLists.get(rowKey);
-        if (row == null)
-            return Collections.<Cell>emptyList();
 
-        List<Cell> cql3List = row.getMultiCellColumn(cql3ColumnName);
+        List<Cell> cql3List = row == null ? Collections.<Cell>emptyList() : row.getMultiCellColumn(cql3ColumnName);
+
+        if (!cf.isEmpty())
+        {
+            ColumnFamily currentCf = cf.cloneMe();
+
+            for (Cell c : cql3List)
+                currentCf.addColumn(c);
+
+            CFMetaData cfm = currentCf.metadata();
+            CQL3Row.RowIterator iterator = cfm.comparator.CQL3RowBuilder(cfm, timestamp).group(currentCf.iterator());
+            // We can only update one CQ3Row per partition key at a time (we don't allow IN for clustering key)
+            cql3List = iterator.hasNext() ? iterator.next().getMultiCellColumn(cql3ColumnName) : null;
+        }
+
         return (cql3List == null) ? Collections.<Cell>emptyList() : cql3List;
     }
 }
diff --git a/src/java/org/apache/cassandra/cql3/UserOptions.java b/src/java/org/apache/cassandra/cql3/UserOptions.java
deleted file mode 100644
index 701debd..0000000
--- a/src/java/org/apache/cassandra/cql3/UserOptions.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * 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.cql3;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.cassandra.auth.IAuthenticator;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.utils.FBUtilities;
-
-public class UserOptions
-{
-    private final Map<IAuthenticator.Option, Object> options = new HashMap<IAuthenticator.Option, Object>();
-
-    public void put(String name, Object value)
-    {
-        options.put(IAuthenticator.Option.valueOf(name.toUpperCase()), value);
-    }
-
-    public boolean isEmpty()
-    {
-        return options.isEmpty();
-    }
-
-    public Map<IAuthenticator.Option, Object> getOptions()
-    {
-        return options;
-    }
-
-    public void validate() throws InvalidRequestException
-    {
-        for (IAuthenticator.Option option : options.keySet())
-        {
-            if (!DatabaseDescriptor.getAuthenticator().supportedOptions().contains(option))
-                throw new InvalidRequestException(String.format("%s doesn't support %s option",
-                                                                DatabaseDescriptor.getAuthenticator().getClass().getName(),
-                                                                option));
-        }
-    }
-
-    public String toString()
-    {
-        return FBUtilities.toString(options);
-    }
-}
diff --git a/src/java/org/apache/cassandra/cql3/UserTypes.java b/src/java/org/apache/cassandra/cql3/UserTypes.java
index c92bc49..de3f545 100644
--- a/src/java/org/apache/cassandra/cql3/UserTypes.java
+++ b/src/java/org/apache/cassandra/cql3/UserTypes.java
@@ -20,11 +20,13 @@
 import java.nio.ByteBuffer;
 import java.util.*;
 
+import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.db.marshal.CollectionType;
-import org.apache.cassandra.db.marshal.UserType;
 import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.db.marshal.UserType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.transport.Server;
+import org.apache.cassandra.utils.ByteBufferUtil;
 
 /**
  * Static helper methods and classes for user types.
@@ -100,21 +102,21 @@
                     continue;
 
                 ColumnSpecification fieldSpec = fieldSpecOf(receiver, i);
-                if (!value.isAssignableTo(keyspace, fieldSpec))
+                if (!value.testAssignment(keyspace, fieldSpec).isAssignable())
                     throw new InvalidRequestException(String.format("Invalid user type literal for %s: field %s is not of type %s", receiver, field, fieldSpec.type.asCQL3Type()));
             }
         }
 
-        public boolean isAssignableTo(String keyspace, ColumnSpecification receiver)
+        public AssignmentTestable.TestResult testAssignment(String keyspace, ColumnSpecification receiver)
         {
             try
             {
                 validateAssignableTo(keyspace, receiver);
-                return true;
+                return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
             }
             catch (InvalidRequestException e)
             {
-                return false;
+                return AssignmentTestable.TestResult.NOT_ASSIGNABLE;
             }
         }
 
@@ -148,6 +150,11 @@
             this.values = values;
         }
 
+        public Iterable<Function> getFunctions()
+        {
+            return Terms.getFunctions(values);
+        }
+
         public boolean containsBindMarker()
         {
             for (Term t : values)
@@ -170,6 +177,9 @@
             for (int i = 0; i < type.size(); i++)
             {
                 buffers[i] = values.get(i).bindAndGet(options);
+                // Since A UDT value is always written in its entirety Cassandra can't preserve a pre-existing value by 'not setting' the new value. Reject the query.
+                if (buffers[i] == ByteBufferUtil.UNSET_BYTE_BUFFER)
+                    throw new InvalidRequestException(String.format("Invalid unset value for field '%s' of user defined type %s", type.fieldNameAsString(i), type.getNameAsString()));
                 // Inside UDT values, we must force the serialization of collections to v3 whatever protocol
                 // version is in use since we're going to store directly that serialized value.
                 if (version < Server.VERSION_3 && type.fieldType(i).isCollection() && buffers[i] != null)
diff --git a/src/java/org/apache/cassandra/cql3/VariableSpecifications.java b/src/java/org/apache/cassandra/cql3/VariableSpecifications.java
index ef78619..5304350 100644
--- a/src/java/org/apache/cassandra/cql3/VariableSpecifications.java
+++ b/src/java/org/apache/cassandra/cql3/VariableSpecifications.java
@@ -17,18 +17,33 @@
  */
 package org.apache.cassandra.cql3;
 
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 
 public class VariableSpecifications
 {
     private final List<ColumnIdentifier> variableNames;
     private final ColumnSpecification[] specs;
+    private final ColumnDefinition[] targetColumns;
 
     public VariableSpecifications(List<ColumnIdentifier> variableNames)
     {
         this.variableNames = variableNames;
         this.specs = new ColumnSpecification[variableNames.size()];
+        this.targetColumns = new ColumnDefinition[variableNames.size()];
+    }
+
+    /**
+     * Returns an empty instance of <code>VariableSpecifications</code>.
+     * @return an empty instance of <code>VariableSpecifications</code>
+     */
+    public static VariableSpecifications empty()
+    {
+        return new VariableSpecifications(Collections.<ColumnIdentifier> emptyList());
     }
 
     public int size()
@@ -41,12 +56,44 @@
         return Arrays.asList(specs);
     }
 
+    /**
+     * Returns an array with the same length as the number of partition key columns for the table corresponding
+     * to cfm.  Each short in the array represents the bind index of the marker that holds the value for that
+     * partition key column.  If there are no bind markers for any of the partition key columns, null is returned.
+     *
+     * Callers of this method should ensure that all statements operate on the same table.
+     */
+    public Short[] getPartitionKeyBindIndexes(CFMetaData cfm)
+    {
+        Short[] partitionKeyPositions = new Short[cfm.partitionKeyColumns().size()];
+        for (int i = 0; i < targetColumns.length; i++)
+        {
+            ColumnDefinition targetColumn = targetColumns[i];
+            if (targetColumn != null && targetColumn.isPartitionKey())
+            {
+                assert targetColumn.ksName.equals(cfm.ksName) && targetColumn.cfName.equals(cfm.cfName);
+                partitionKeyPositions[targetColumn.position()] = (short) i;
+            }
+        }
+
+        for (Short bindIndex : partitionKeyPositions)
+        {
+            if (bindIndex == null)
+                return null;
+        }
+
+        return partitionKeyPositions;
+    }
+
     public void add(int bindIndex, ColumnSpecification spec)
     {
-        ColumnIdentifier name = variableNames.get(bindIndex);
+        if (spec instanceof ColumnDefinition)
+            targetColumns[bindIndex] = (ColumnDefinition) spec;
+
+        ColumnIdentifier bindMarkerName = variableNames.get(bindIndex);
         // Use the user name, if there is one
-        if (name != null)
-            spec = new ColumnSpecification(spec.ksName, spec.cfName, name, spec.type);
+        if (bindMarkerName != null)
+            spec = new ColumnSpecification(spec.ksName, spec.cfName, bindMarkerName, spec.type);
         specs[bindIndex] = spec;
     }
 
diff --git a/src/java/org/apache/cassandra/cql3/functions/AbstractFunction.java b/src/java/org/apache/cassandra/cql3/functions/AbstractFunction.java
index 5dbfbf3..b77f4d5 100644
--- a/src/java/org/apache/cassandra/cql3/functions/AbstractFunction.java
+++ b/src/java/org/apache/cassandra/cql3/functions/AbstractFunction.java
@@ -17,32 +17,39 @@
  */
 package org.apache.cassandra.cql3.functions;
 
-import java.util.Arrays;
 import java.util.List;
 
+import com.google.common.base.Objects;
+import com.google.common.collect.ImmutableSet;
+
+import org.apache.cassandra.cql3.AssignmentTestable;
+import org.apache.cassandra.cql3.ColumnSpecification;
 import org.apache.cassandra.db.marshal.AbstractType;
 
+/**
+ * Base class for our native/hardcoded functions.
+ */
 public abstract class AbstractFunction implements Function
 {
-    public final String name;
-    public final List<AbstractType<?>> argsType;
-    public final AbstractType<?> returnType;
+    protected final FunctionName name;
+    protected final List<AbstractType<?>> argTypes;
+    protected final AbstractType<?> returnType;
 
-    protected AbstractFunction(String name, AbstractType<?> returnType, AbstractType<?>... argsType)
+    protected AbstractFunction(FunctionName name, List<AbstractType<?>> argTypes, AbstractType<?> returnType)
     {
         this.name = name;
-        this.argsType = Arrays.asList(argsType);
+        this.argTypes = argTypes;
         this.returnType = returnType;
     }
 
-    public String name()
+    public FunctionName name()
     {
         return name;
     }
 
-    public List<AbstractType<?>> argsType()
+    public List<AbstractType<?>> argTypes()
     {
-        return argsType;
+        return argTypes;
     }
 
     public AbstractType<?> returnType()
@@ -50,23 +57,63 @@
         return returnType;
     }
 
-    // Most of our functions are pure, the other ones should override this
-    public boolean isPure()
+    @Override
+    public boolean equals(Object o)
     {
-        return true;
+        if (!(o instanceof AbstractFunction))
+            return false;
+
+        AbstractFunction that = (AbstractFunction)o;
+        return Objects.equal(this.name, that.name)
+            && Objects.equal(this.argTypes, that.argTypes)
+            && Objects.equal(this.returnType, that.returnType);
     }
 
-    /**
-     * Creates a trivial factory that always return the provided function.
-     */
-    public static Function.Factory factory(final Function fun)
+    public Iterable<Function> getFunctions()
     {
-        return new Function.Factory()
+        return ImmutableSet.<Function>of(this);
+    }
+
+    public boolean hasReferenceTo(Function function)
+    {
+        return false;
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hashCode(name, argTypes, returnType);
+    }
+
+    public final AssignmentTestable.TestResult testAssignment(String keyspace, ColumnSpecification receiver)
+    {
+        // We should ignore the fact that the receiver type is frozen in our comparison as functions do not support
+        // frozen types for return type
+        AbstractType<?> returnType = returnType();
+        if (receiver.type.isFrozenCollection())
+            returnType = returnType.freeze();
+
+        if (receiver.type.equals(returnType))
+            return AssignmentTestable.TestResult.EXACT_MATCH;
+
+        if (receiver.type.isValueCompatibleWith(returnType))
+            return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
+
+        return AssignmentTestable.TestResult.NOT_ASSIGNABLE;
+    }
+
+    @Override
+    public String toString()
+    {
+        StringBuilder sb = new StringBuilder();
+        sb.append(name).append(" : (");
+        for (int i = 0; i < argTypes.size(); i++)
         {
-            public Function create(String ksName, String cfName)
-            {
-                return fun;
-            }
-        };
+            if (i > 0)
+                sb.append(", ");
+            sb.append(argTypes.get(i).asCQL3Type());
+        }
+        sb.append(") -> ").append(returnType.asCQL3Type());
+        return sb.toString();
     }
 }
diff --git a/src/java/org/apache/cassandra/cql3/functions/AggregateFcts.java b/src/java/org/apache/cassandra/cql3/functions/AggregateFcts.java
new file mode 100644
index 0000000..cca6156
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/AggregateFcts.java
@@ -0,0 +1,859 @@
+/*
+ * 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.cql3.functions;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.ByteType;
+import org.apache.cassandra.db.marshal.CounterColumnType;
+import org.apache.cassandra.db.marshal.DecimalType;
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.FloatType;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.db.marshal.IntegerType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.ShortType;
+
+/**
+ * Factory methods for aggregate functions.
+ */
+public abstract class AggregateFcts
+{
+    /**
+     * Checks if the specified function is the count rows (e.g. COUNT(*) or COUNT(1)) function.
+     *
+     * @param function the function to check
+     * @return <code>true</code> if the specified function is the count rows one, <code>false</code> otherwise.
+     */
+    public static boolean isCountRows(Function function)
+    {
+        return function == countRowsFunction;
+    }
+
+    /**
+     * The function used to count the number of rows of a result set. This function is called when COUNT(*) or COUNT(1)
+     * is specified.
+     */
+    public static final AggregateFunction countRowsFunction =
+            new NativeAggregateFunction("countRows", LongType.instance)
+            {
+                public Aggregate newAggregate()
+                {
+                    return new Aggregate()
+                    {
+                        private long count;
+
+                        public void reset()
+                        {
+                            count = 0;
+                        }
+
+                        public ByteBuffer compute(int protocolVersion)
+                        {
+                            return ((LongType) returnType()).decompose(Long.valueOf(count));
+                        }
+
+                        public void addInput(int protocolVersion, List<ByteBuffer> values)
+                        {
+                            count++;
+                        }
+                    };
+                }
+            };
+
+    /**
+     * The SUM function for decimal values.
+     */
+    public static final AggregateFunction sumFunctionForDecimal =
+            new NativeAggregateFunction("sum", DecimalType.instance, DecimalType.instance)
+            {
+                @Override
+                public Aggregate newAggregate()
+                {
+                    return new Aggregate()
+                    {
+                        private BigDecimal sum = BigDecimal.ZERO;
+
+                        public void reset()
+                        {
+                            sum = BigDecimal.ZERO;
+                        }
+
+                        public ByteBuffer compute(int protocolVersion)
+                        {
+                            return ((DecimalType) returnType()).decompose(sum);
+                        }
+
+                        public void addInput(int protocolVersion, List<ByteBuffer> values)
+                        {
+                            ByteBuffer value = values.get(0);
+
+                            if (value == null)
+                                return;
+
+                            BigDecimal number = ((BigDecimal) argTypes().get(0).compose(value));
+                            sum = sum.add(number);
+                        }
+                    };
+                }
+            };
+
+    /**
+     * The AVG function for decimal values.
+     */
+    public static final AggregateFunction avgFunctionForDecimal =
+            new NativeAggregateFunction("avg", DecimalType.instance, DecimalType.instance)
+            {
+                public Aggregate newAggregate()
+                {
+                    return new Aggregate()
+                    {
+                        private BigDecimal sum = BigDecimal.ZERO;
+
+                        private int count;
+
+                        public void reset()
+                        {
+                            count = 0;
+                            sum = BigDecimal.ZERO;
+                        }
+
+                        public ByteBuffer compute(int protocolVersion)
+                        {
+                            if (count == 0)
+                                return ((DecimalType) returnType()).decompose(BigDecimal.ZERO);
+
+                            return ((DecimalType) returnType()).decompose(sum.divide(BigDecimal.valueOf(count)));
+                        }
+
+                        public void addInput(int protocolVersion, List<ByteBuffer> values)
+                        {
+                            ByteBuffer value = values.get(0);
+
+                            if (value == null)
+                                return;
+
+                            count++;
+                            BigDecimal number = ((BigDecimal) argTypes().get(0).compose(value));
+                            sum = sum.add(number);
+                        }
+                    };
+                }
+            };
+
+    /**
+     * The SUM function for varint values.
+     */
+    public static final AggregateFunction sumFunctionForVarint =
+            new NativeAggregateFunction("sum", IntegerType.instance, IntegerType.instance)
+            {
+                public Aggregate newAggregate()
+                {
+                    return new Aggregate()
+                    {
+                        private BigInteger sum = BigInteger.ZERO;
+
+                        public void reset()
+                        {
+                            sum = BigInteger.ZERO;
+                        }
+
+                        public ByteBuffer compute(int protocolVersion)
+                        {
+                            return ((IntegerType) returnType()).decompose(sum);
+                        }
+
+                        public void addInput(int protocolVersion, List<ByteBuffer> values)
+                        {
+                            ByteBuffer value = values.get(0);
+
+                            if (value == null)
+                                return;
+
+                            BigInteger number = ((BigInteger) argTypes().get(0).compose(value));
+                            sum = sum.add(number);
+                        }
+                    };
+                }
+            };
+
+    /**
+     * The AVG function for varint values.
+     */
+    public static final AggregateFunction avgFunctionForVarint =
+            new NativeAggregateFunction("avg", IntegerType.instance, IntegerType.instance)
+            {
+                public Aggregate newAggregate()
+                {
+                    return new Aggregate()
+                    {
+                        private BigInteger sum = BigInteger.ZERO;
+
+                        private int count;
+
+                        public void reset()
+                        {
+                            count = 0;
+                            sum = BigInteger.ZERO;
+                        }
+
+                        public ByteBuffer compute(int protocolVersion)
+                        {
+                            if (count == 0)
+                                return ((IntegerType) returnType()).decompose(BigInteger.ZERO);
+
+                            return ((IntegerType) returnType()).decompose(sum.divide(BigInteger.valueOf(count)));
+                        }
+
+                        public void addInput(int protocolVersion, List<ByteBuffer> values)
+                        {
+                            ByteBuffer value = values.get(0);
+
+                            if (value == null)
+                                return;
+
+                            count++;
+                            BigInteger number = ((BigInteger) argTypes().get(0).compose(value));
+                            sum = sum.add(number);
+                        }
+                    };
+                }
+            };
+
+    /**
+     * The SUM function for byte values (tinyint).
+     */
+    public static final AggregateFunction sumFunctionForByte =
+            new NativeAggregateFunction("sum", ByteType.instance, ByteType.instance)
+            {
+                public Aggregate newAggregate()
+                {
+                    return new Aggregate()
+                    {
+                        private byte sum;
+
+                        public void reset()
+                        {
+                            sum = 0;
+                        }
+
+                        public ByteBuffer compute(int protocolVersion)
+                        {
+                            return ((ByteType) returnType()).decompose(sum);
+                        }
+
+                        public void addInput(int protocolVersion, List<ByteBuffer> values)
+                        {
+                            ByteBuffer value = values.get(0);
+
+                            if (value == null)
+                                return;
+
+                            Number number = ((Number) argTypes().get(0).compose(value));
+                            sum += number.byteValue();
+                        }
+                    };
+                }
+            };
+
+    /**
+     * AVG function for byte values (tinyint).
+     */
+    public static final AggregateFunction avgFunctionForByte =
+            new NativeAggregateFunction("avg", ByteType.instance, ByteType.instance)
+            {
+                public Aggregate newAggregate()
+                {
+                    return new Aggregate()
+                    {
+                        private byte sum;
+
+                        private int count;
+
+                        public void reset()
+                        {
+                            count = 0;
+                            sum = 0;
+                        }
+
+                        public ByteBuffer compute(int protocolVersion)
+                        {
+                            int avg = count == 0 ? 0 : sum / count;
+
+                            return ((ByteType) returnType()).decompose((byte) avg);
+                        }
+
+                        public void addInput(int protocolVersion, List<ByteBuffer> values)
+                        {
+                            ByteBuffer value = values.get(0);
+
+                            if (value == null)
+                                return;
+
+                            count++;
+                            Number number = ((Number) argTypes().get(0).compose(value));
+                            sum += number.byteValue();
+                        }
+                    };
+                }
+            };
+
+    /**
+     * The SUM function for short values (smallint).
+     */
+    public static final AggregateFunction sumFunctionForShort =
+            new NativeAggregateFunction("sum", ShortType.instance, ShortType.instance)
+            {
+                public Aggregate newAggregate()
+                {
+                    return new Aggregate()
+                    {
+                        private short sum;
+
+                        public void reset()
+                        {
+                            sum = 0;
+                        }
+
+                        public ByteBuffer compute(int protocolVersion)
+                        {
+                            return ((ShortType) returnType()).decompose(sum);
+                        }
+
+                        public void addInput(int protocolVersion, List<ByteBuffer> values)
+                        {
+                            ByteBuffer value = values.get(0);
+
+                            if (value == null)
+                                return;
+
+                            Number number = ((Number) argTypes().get(0).compose(value));
+                            sum += number.shortValue();
+                        }
+                    };
+                }
+            };
+
+    /**
+     * AVG function for for short values (smallint).
+     */
+    public static final AggregateFunction avgFunctionForShort =
+            new NativeAggregateFunction("avg", ShortType.instance, ShortType.instance)
+            {
+                public Aggregate newAggregate()
+                {
+                    return new Aggregate()
+                    {
+                        private short sum;
+
+                        private int count;
+
+                        public void reset()
+                        {
+                            count = 0;
+                            sum = 0;
+                        }
+
+                        public ByteBuffer compute(int protocolVersion)
+                        {
+                            int avg = count == 0 ? 0 : sum / count;
+
+                            return ((ShortType) returnType()).decompose((short) avg);
+                        }
+
+                        public void addInput(int protocolVersion, List<ByteBuffer> values)
+                        {
+                            ByteBuffer value = values.get(0);
+
+                            if (value == null)
+                                return;
+
+                            count++;
+                            Number number = ((Number) argTypes().get(0).compose(value));
+                            sum += number.shortValue();
+                        }
+                    };
+                }
+            };
+
+    /**
+     * The SUM function for int32 values.
+     */
+    public static final AggregateFunction sumFunctionForInt32 =
+            new NativeAggregateFunction("sum", Int32Type.instance, Int32Type.instance)
+            {
+                public Aggregate newAggregate()
+                {
+                    return new Aggregate()
+                    {
+                        private int sum;
+
+                        public void reset()
+                        {
+                            sum = 0;
+                        }
+
+                        public ByteBuffer compute(int protocolVersion)
+                        {
+                            return ((Int32Type) returnType()).decompose(sum);
+                        }
+
+                        public void addInput(int protocolVersion, List<ByteBuffer> values)
+                        {
+                            ByteBuffer value = values.get(0);
+
+                            if (value == null)
+                                return;
+
+                            Number number = ((Number) argTypes().get(0).compose(value));
+                            sum += number.intValue();
+                        }
+                    };
+                }
+            };
+
+    /**
+     * AVG function for int32 values.
+     */
+    public static final AggregateFunction avgFunctionForInt32 =
+            new NativeAggregateFunction("avg", Int32Type.instance, Int32Type.instance)
+            {
+                public Aggregate newAggregate()
+                {
+                    return new Aggregate()
+                    {
+                        private int sum;
+
+                        private int count;
+
+                        public void reset()
+                        {
+                            count = 0;
+                            sum = 0;
+                        }
+
+                        public ByteBuffer compute(int protocolVersion)
+                        {
+                            int avg = count == 0 ? 0 : sum / count;
+
+                            return ((Int32Type) returnType()).decompose(avg);
+                        }
+
+                        public void addInput(int protocolVersion, List<ByteBuffer> values)
+                        {
+                            ByteBuffer value = values.get(0);
+
+                            if (value == null)
+                                return;
+
+                            count++;
+                            Number number = ((Number) argTypes().get(0).compose(value));
+                            sum += number.intValue();
+                        }
+                    };
+                }
+            };
+
+    /**
+     * The SUM function for long values.
+     */
+    public static final AggregateFunction sumFunctionForLong =
+            new NativeAggregateFunction("sum", LongType.instance, LongType.instance)
+            {
+                public Aggregate newAggregate()
+                {
+                    return new LongSumAggregate();
+                }
+            };
+
+    /**
+     * AVG function for long values.
+     */
+    public static final AggregateFunction avgFunctionForLong =
+            new NativeAggregateFunction("avg", LongType.instance, LongType.instance)
+            {
+                public Aggregate newAggregate()
+                {
+                    return new LongAvgAggregate();
+                }
+            };
+
+    /**
+     * The SUM function for float values.
+     */
+    public static final AggregateFunction sumFunctionForFloat =
+            new NativeAggregateFunction("sum", FloatType.instance, FloatType.instance)
+            {
+                public Aggregate newAggregate()
+                {
+                    return new Aggregate()
+                    {
+                        private float sum;
+
+                        public void reset()
+                        {
+                            sum = 0;
+                        }
+
+                        public ByteBuffer compute(int protocolVersion)
+                        {
+                            return ((FloatType) returnType()).decompose(sum);
+                        }
+
+                        public void addInput(int protocolVersion, List<ByteBuffer> values)
+                        {
+                            ByteBuffer value = values.get(0);
+
+                            if (value == null)
+                                return;
+
+                            Number number = ((Number) argTypes().get(0).compose(value));
+                            sum += number.floatValue();
+                        }
+                    };
+                }
+            };
+
+    /**
+     * AVG function for float values.
+     */
+    public static final AggregateFunction avgFunctionForFloat =
+            new NativeAggregateFunction("avg", FloatType.instance, FloatType.instance)
+            {
+                public Aggregate newAggregate()
+                {
+                    return new Aggregate()
+                    {
+                        private float sum;
+
+                        private int count;
+
+                        public void reset()
+                        {
+                            count = 0;
+                            sum = 0;
+                        }
+
+                        public ByteBuffer compute(int protocolVersion)
+                        {
+                            float avg = count == 0 ? 0 : sum / count;
+
+                            return ((FloatType) returnType()).decompose(avg);
+                        }
+
+                        public void addInput(int protocolVersion, List<ByteBuffer> values)
+                        {
+                            ByteBuffer value = values.get(0);
+
+                            if (value == null)
+                                return;
+
+                            count++;
+                            Number number = ((Number) argTypes().get(0).compose(value));
+                            sum += number.floatValue();
+                        }
+                    };
+                }
+            };
+
+    /**
+     * The SUM function for double values.
+     */
+    public static final AggregateFunction sumFunctionForDouble =
+            new NativeAggregateFunction("sum", DoubleType.instance, DoubleType.instance)
+            {
+                public Aggregate newAggregate()
+                {
+                    return new Aggregate()
+                    {
+                        private double sum;
+
+                        public void reset()
+                        {
+                            sum = 0;
+                        }
+
+                        public ByteBuffer compute(int protocolVersion)
+                        {
+                            return ((DoubleType) returnType()).decompose(sum);
+                        }
+
+                        public void addInput(int protocolVersion, List<ByteBuffer> values)
+                        {
+                            ByteBuffer value = values.get(0);
+
+                            if (value == null)
+                                return;
+
+                            Number number = ((Number) argTypes().get(0).compose(value));
+                            sum += number.doubleValue();
+                        }
+                    };
+                }
+            };
+
+    /**
+     * AVG function for double values.
+     */
+    public static final AggregateFunction avgFunctionForDouble =
+            new NativeAggregateFunction("avg", DoubleType.instance, DoubleType.instance)
+            {
+                public Aggregate newAggregate()
+                {
+                    return new Aggregate()
+                    {
+                        private double sum;
+
+                        private int count;
+
+                        public void reset()
+                        {
+                            count = 0;
+                            sum = 0;
+                        }
+
+                        public ByteBuffer compute(int protocolVersion)
+                        {
+                            double avg = count == 0 ? 0 : sum / count;
+
+                            return ((DoubleType) returnType()).decompose(avg);
+                        }
+
+                        public void addInput(int protocolVersion, List<ByteBuffer> values)
+                        {
+                            ByteBuffer value = values.get(0);
+
+                            if (value == null)
+                                return;
+
+                            count++;
+                            Number number = ((Number) argTypes().get(0).compose(value));
+                            sum += number.doubleValue();
+                        }
+                    };
+                }
+            };
+
+    /**
+     * The SUM function for counter column values.
+     */
+    public static final AggregateFunction sumFunctionForCounter =
+            new NativeAggregateFunction("sum", CounterColumnType.instance, CounterColumnType.instance)
+            {
+                public Aggregate newAggregate()
+                {
+                    return new LongSumAggregate();
+                }
+            };
+
+    /**
+     * AVG function for counter column values.
+     */
+    public static final AggregateFunction avgFunctionForCounter =
+            new NativeAggregateFunction("avg", CounterColumnType.instance, CounterColumnType.instance)
+            {
+                public Aggregate newAggregate()
+                {
+                    return new LongAvgAggregate();
+                }
+            };
+
+    /**
+     * Creates a MAX function for the specified type.
+     *
+     * @param inputType the function input and output type
+     * @return a MAX function for the specified type.
+     */
+    public static AggregateFunction makeMaxFunction(final AbstractType<?> inputType)
+    {
+        return new NativeAggregateFunction("max", inputType, inputType)
+        {
+            public Aggregate newAggregate()
+            {
+                return new Aggregate()
+                {
+                    private ByteBuffer max;
+
+                    public void reset()
+                    {
+                        max = null;
+                    }
+
+                    public ByteBuffer compute(int protocolVersion)
+                    {
+                        return max;
+                    }
+
+                    public void addInput(int protocolVersion, List<ByteBuffer> values)
+                    {
+                        ByteBuffer value = values.get(0);
+
+                        if (value == null)
+                            return;
+
+                        if (max == null || returnType().compare(max, value) < 0)
+                            max = value;
+                    }
+                };
+            }
+        };
+    }
+
+    /**
+     * Creates a MIN function for the specified type.
+     *
+     * @param inputType the function input and output type
+     * @return a MIN function for the specified type.
+     */
+    public static AggregateFunction makeMinFunction(final AbstractType<?> inputType)
+    {
+        return new NativeAggregateFunction("min", inputType, inputType)
+        {
+            public Aggregate newAggregate()
+            {
+                return new Aggregate()
+                {
+                    private ByteBuffer min;
+
+                    public void reset()
+                    {
+                        min = null;
+                    }
+
+                    public ByteBuffer compute(int protocolVersion)
+                    {
+                        return min;
+                    }
+
+                    public void addInput(int protocolVersion, List<ByteBuffer> values)
+                    {
+                        ByteBuffer value = values.get(0);
+
+                        if (value == null)
+                            return;
+
+                        if (min == null || returnType().compare(min, value) > 0)
+                            min = value;
+                    }
+                };
+            }
+        };
+    }
+
+    /**
+     * Creates a COUNT function for the specified type.
+     *
+     * @param inputType the function input type
+     * @return a COUNT function for the specified type.
+     */
+    public static AggregateFunction makeCountFunction(AbstractType<?> inputType)
+    {
+        return new NativeAggregateFunction("count", LongType.instance, inputType)
+        {
+            public Aggregate newAggregate()
+            {
+                return new Aggregate()
+                {
+                    private long count;
+
+                    public void reset()
+                    {
+                        count = 0;
+                    }
+
+                    public ByteBuffer compute(int protocolVersion)
+                    {
+                        return ((LongType) returnType()).decompose(count);
+                    }
+
+                    public void addInput(int protocolVersion, List<ByteBuffer> values)
+                    {
+                        ByteBuffer value = values.get(0);
+
+                        if (value == null)
+                            return;
+
+                        count++;
+                    }
+                };
+            }
+        };
+    }
+
+    private static class LongSumAggregate implements AggregateFunction.Aggregate
+    {
+        private long sum;
+
+        public void reset()
+        {
+            sum = 0;
+        }
+
+        public ByteBuffer compute(int protocolVersion)
+        {
+            return LongType.instance.decompose(sum);
+        }
+
+        public void addInput(int protocolVersion, List<ByteBuffer> values)
+        {
+            ByteBuffer value = values.get(0);
+
+            if (value == null)
+                return;
+
+            Number number = LongType.instance.compose(value);
+            sum += number.longValue();
+        }
+    }
+
+    private static class LongAvgAggregate implements AggregateFunction.Aggregate
+    {
+        private long sum;
+
+        private int count;
+
+        public void reset()
+        {
+            count = 0;
+            sum = 0;
+        }
+
+        public ByteBuffer compute(int protocolVersion)
+        {
+            long avg = count == 0 ? 0 : sum / count;
+
+            return LongType.instance.decompose(avg);
+        }
+
+        public void addInput(int protocolVersion, List<ByteBuffer> values)
+        {
+            ByteBuffer value = values.get(0);
+
+            if (value == null)
+                return;
+
+            count++;
+            Number number = LongType.instance.compose(value);
+            sum += number.longValue();
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/functions/AggregateFunction.java b/src/java/org/apache/cassandra/cql3/functions/AggregateFunction.java
new file mode 100644
index 0000000..ddbc9d1
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/AggregateFunction.java
@@ -0,0 +1,63 @@
+/*
+ * 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.cql3.functions;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+/**
+ * Performs a calculation on a set of values and return a single value.
+ */
+public interface AggregateFunction extends Function
+{
+    /**
+     * Creates a new <code>Aggregate</code> instance.
+     *
+     * @return a new <code>Aggregate</code> instance.
+     */
+    public Aggregate newAggregate() throws InvalidRequestException;
+
+    /**
+     * An aggregation operation.
+     */
+    interface Aggregate
+    {
+        /**
+         * Adds the specified input to this aggregate.
+         *
+         * @param protocolVersion native protocol version
+         * @param values the values to add to the aggregate.
+         */
+        public void addInput(int protocolVersion, List<ByteBuffer> values) throws InvalidRequestException;
+
+        /**
+         * Computes and returns the aggregate current value.
+         *
+         * @param protocolVersion native protocol version
+         * @return the aggregate current value.
+         */
+        public ByteBuffer compute(int protocolVersion) throws InvalidRequestException;
+
+        /**
+         * Reset this aggregate.
+         */
+        public void reset();
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/functions/BytesConversionFcts.java b/src/java/org/apache/cassandra/cql3/functions/BytesConversionFcts.java
index e3023db..ddb33fc 100644
--- a/src/java/org/apache/cassandra/cql3/functions/BytesConversionFcts.java
+++ b/src/java/org/apache/cassandra/cql3/functions/BytesConversionFcts.java
@@ -34,9 +34,9 @@
     public static Function makeToBlobFunction(AbstractType<?> fromType)
     {
         String name = fromType.asCQL3Type() + "asblob";
-        return new AbstractFunction(name, BytesType.instance, fromType)
+        return new NativeScalarFunction(name, BytesType.instance, fromType)
         {
-            public ByteBuffer execute(List<ByteBuffer> parameters)
+            public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
             {
                 return parameters.get(0);
             }
@@ -46,9 +46,9 @@
     public static Function makeFromBlobFunction(final AbstractType<?> toType)
     {
         final String name = "blobas" + toType.asCQL3Type();
-        return new AbstractFunction(name, toType, BytesType.instance)
+        return new NativeScalarFunction(name, toType, BytesType.instance)
         {
-            public ByteBuffer execute(List<ByteBuffer> parameters) throws InvalidRequestException
+            public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters) throws InvalidRequestException
             {
                 ByteBuffer val = parameters.get(0);
                 try
@@ -66,17 +66,17 @@
         };
     }
 
-    public static final Function VarcharAsBlobFct = new AbstractFunction("varcharasblob", BytesType.instance, UTF8Type.instance)
+    public static final Function VarcharAsBlobFct = new NativeScalarFunction("varcharasblob", BytesType.instance, UTF8Type.instance)
     {
-        public ByteBuffer execute(List<ByteBuffer> parameters)
+        public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
         {
             return parameters.get(0);
         }
     };
 
-    public static final Function BlobAsVarcharFact = new AbstractFunction("blobasvarchar", UTF8Type.instance, BytesType.instance)
+    public static final Function BlobAsVarcharFact = new NativeScalarFunction("blobasvarchar", UTF8Type.instance, BytesType.instance)
     {
-        public ByteBuffer execute(List<ByteBuffer> parameters)
+        public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
         {
             return parameters.get(0);
         }
diff --git a/src/java/org/apache/cassandra/cql3/functions/FromJsonFct.java b/src/java/org/apache/cassandra/cql3/functions/FromJsonFct.java
new file mode 100644
index 0000000..2b9e8c6
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/FromJsonFct.java
@@ -0,0 +1,78 @@
+/*
+ * 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.cql3.functions;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.cassandra.cql3.Json;
+
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.exceptions.FunctionExecutionException;
+import org.apache.cassandra.serializers.MarshalException;
+
+public class FromJsonFct extends NativeScalarFunction
+{
+    public static final FunctionName NAME = FunctionName.nativeFunction("fromjson");
+
+    private static final Map<AbstractType<?>, FromJsonFct> instances = new ConcurrentHashMap<>();
+
+    public static FromJsonFct getInstance(AbstractType<?> returnType)
+    {
+        FromJsonFct func = instances.get(returnType);
+        if (func == null)
+        {
+            func = new FromJsonFct(returnType);
+            instances.put(returnType, func);
+        }
+        return func;
+    }
+
+    private FromJsonFct(AbstractType<?> returnType)
+    {
+        super("fromjson", returnType, UTF8Type.instance);
+    }
+
+    public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
+    {
+        assert parameters.size() == 1 : "Unexpectedly got " + parameters.size() + " arguments for fromJson()";
+        ByteBuffer argument = parameters.get(0);
+        if (argument == null)
+            return null;
+
+        String jsonArg = UTF8Type.instance.getSerializer().deserialize(argument);
+        try
+        {
+            Object object = Json.JSON_OBJECT_MAPPER.readValue(jsonArg, Object.class);
+            if (object == null)
+                return null;
+            return returnType.fromJSONObject(object).bindAndGet(QueryOptions.forProtocolVersion(protocolVersion));
+        }
+        catch (IOException exc)
+        {
+            throw new FunctionExecutionException(NAME, Collections.singletonList("text"), String.format("Could not decode JSON string '%s': %s", jsonArg, exc.toString()));
+        }
+        catch (MarshalException exc)
+        {
+            throw FunctionExecutionException.create(this, exc);
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/functions/Function.java b/src/java/org/apache/cassandra/cql3/functions/Function.java
index b278389..ed6e2a7 100644
--- a/src/java/org/apache/cassandra/cql3/functions/Function.java
+++ b/src/java/org/apache/cassandra/cql3/functions/Function.java
@@ -17,31 +17,34 @@
  */
 package org.apache.cassandra.cql3.functions;
 
-import java.nio.ByteBuffer;
 import java.util.List;
 
+import org.apache.cassandra.cql3.AssignmentTestable;
 import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.github.jamm.Unmetered;
 
 @Unmetered
-public interface Function
+public interface Function extends AssignmentTestable
 {
-    public String name();
-    public List<AbstractType<?>> argsType();
+    public FunctionName name();
+    public List<AbstractType<?>> argTypes();
     public AbstractType<?> returnType();
 
-    public ByteBuffer execute(List<ByteBuffer> parameters) throws InvalidRequestException;
+    /**
+     * Checks whether the function is a native/hard coded one or not.
+     *
+     * @return <code>true</code> if the function is a native/hard coded one, <code>false</code> otherwise.
+     */
+    public boolean isNative();
 
-    // Whether the function is a pure function (as in doesn't depend on, nor produce side effects).
-    public boolean isPure();
+    /**
+     * Checks whether the function is an aggregate function or not.
+     *
+     * @return <code>true</code> if the function is an aggregate function, <code>false</code> otherwise.
+     */
+    public boolean isAggregate();
 
-    public interface Factory
-    {
-        // We allow the function to be parametered by the keyspace it is part of because the
-        // "token" function needs it (the argument depends on the keyValidator). However,
-        // for most function, the factory will just always the same function object (see
-        // AbstractFunction).
-        public Function create(String ksName, String cfName);
-    }
+    public Iterable<Function> getFunctions();
+
+    public boolean hasReferenceTo(Function function);
 }
diff --git a/src/java/org/apache/cassandra/cql3/functions/FunctionCall.java b/src/java/org/apache/cassandra/cql3/functions/FunctionCall.java
index 4ae7c98..4f53c98 100644
--- a/src/java/org/apache/cassandra/cql3/functions/FunctionCall.java
+++ b/src/java/org/apache/cassandra/cql3/functions/FunctionCall.java
@@ -21,27 +21,31 @@
 import java.util.ArrayList;
 import java.util.List;
 
+import com.google.common.collect.Iterables;
+
 import org.apache.cassandra.cql3.*;
-import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.db.marshal.CollectionType;
-import org.apache.cassandra.db.marshal.ListType;
-import org.apache.cassandra.db.marshal.MapType;
-import org.apache.cassandra.db.marshal.SetType;
+import org.apache.cassandra.cql3.statements.RequestValidations;
+import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.serializers.MarshalException;
+import org.apache.cassandra.utils.ByteBufferUtil;
 
 public class FunctionCall extends Term.NonTerminal
 {
-    private final Function fun;
+    private final ScalarFunction fun;
     private final List<Term> terms;
 
-    private FunctionCall(Function fun, List<Term> terms)
+    private FunctionCall(ScalarFunction fun, List<Term> terms)
     {
         this.fun = fun;
         this.terms = terms;
     }
 
+    public Iterable<Function> getFunctions()
+    {
+        return Iterables.concat(Terms.getFunctions(terms), fun.getFunctions());
+    }
+
     public void collectMarkerSpecification(VariableSpecifications boundNames)
     {
         for (Term t : terms)
@@ -55,22 +59,19 @@
 
     public ByteBuffer bindAndGet(QueryOptions options) throws InvalidRequestException
     {
-        List<ByteBuffer> buffers = new ArrayList<ByteBuffer>(terms.size());
+        List<ByteBuffer> buffers = new ArrayList<>(terms.size());
         for (Term t : terms)
         {
-            // For now, we don't allow nulls as argument as no existing function needs it and it
-            // simplify things.
-            ByteBuffer val = t.bindAndGet(options);
-            if (val == null)
-                throw new InvalidRequestException(String.format("Invalid null value for argument to %s", fun));
-            buffers.add(val);
+            ByteBuffer functionArg = t.bindAndGet(options);
+            RequestValidations.checkBindValueSet(functionArg, "Invalid unset value for argument in call to function %s", fun.name().name);
+            buffers.add(functionArg);
         }
-        return executeInternal(fun, buffers);
+        return executeInternal(options.getProtocolVersion(), fun, buffers);
     }
 
-    private static ByteBuffer executeInternal(Function fun, List<ByteBuffer> params) throws InvalidRequestException
+    private static ByteBuffer executeInternal(int protocolVersion, ScalarFunction fun, List<ByteBuffer> params) throws InvalidRequestException
     {
-        ByteBuffer result = fun.execute(params);
+        ByteBuffer result = fun.execute(protocolVersion, params);
         try
         {
             // Check the method didn't lied on it's declared return type
@@ -80,8 +81,8 @@
         }
         catch (MarshalException e)
         {
-            throw new RuntimeException(String.format("Return of function %s (%s) is not a valid value for its declared return type %s", 
-                                                     fun, ByteBufferUtil.bytesToHex(result), fun.returnType().asCQL3Type()));
+            throw new RuntimeException(String.format("Return of function %s (%s) is not a valid value for its declared return type %s",
+                                                     fun, ByteBufferUtil.bytesToHex(result), fun.returnType().asCQL3Type()), e);
         }
     }
 
@@ -111,67 +112,84 @@
 
     public static class Raw implements Term.Raw
     {
-        private final String functionName;
+        private FunctionName name;
         private final List<Term.Raw> terms;
 
-        public Raw(String functionName, List<Term.Raw> terms)
+        public Raw(FunctionName name, List<Term.Raw> terms)
         {
-            this.functionName = functionName;
+            this.name = name;
             this.terms = terms;
         }
 
         public Term prepare(String keyspace, ColumnSpecification receiver) throws InvalidRequestException
         {
-            Function fun = Functions.get(keyspace, functionName, terms, receiver);
+            Function fun = Functions.get(keyspace, name, terms, receiver.ksName, receiver.cfName, receiver.type);
+            if (fun == null)
+                throw new InvalidRequestException(String.format("Unknown function %s called", name));
+            if (fun.isAggregate())
+                throw new InvalidRequestException("Aggregation function are not supported in the where clause");
 
-            List<Term> parameters = new ArrayList<Term>(terms.size());
-            boolean allTerminal = true;
+            ScalarFunction scalarFun = (ScalarFunction) fun;
+
+            // Functions.get() will complain if no function "name" type check with the provided arguments.
+            // We still have to validate that the return type matches however
+            if (!scalarFun.testAssignment(keyspace, receiver).isAssignable())
+                throw new InvalidRequestException(String.format("Type error: cannot assign result of function %s (type %s) to %s (type %s)",
+                                                                scalarFun.name(), scalarFun.returnType().asCQL3Type(),
+                                                                receiver.name, receiver.type.asCQL3Type()));
+
+            if (fun.argTypes().size() != terms.size())
+                throw new InvalidRequestException(String.format("Incorrect number of arguments specified for function %s (expected %d, found %d)",
+                                                                fun, fun.argTypes().size(), terms.size()));
+
+            List<Term> parameters = new ArrayList<>(terms.size());
             for (int i = 0; i < terms.size(); i++)
             {
-                Term t = terms.get(i).prepare(keyspace, Functions.makeArgSpec(receiver, fun, i));
-                if (t instanceof NonTerminal)
-                    allTerminal = false;
+                Term t = terms.get(i).prepare(keyspace, Functions.makeArgSpec(receiver.ksName, receiver.cfName, scalarFun, i));
                 parameters.add(t);
             }
 
-            // If all parameters are terminal and the function is pure, we can
-            // evaluate it now, otherwise we'd have to wait execution time
-            return allTerminal && fun.isPure()
-                ? makeTerminal(fun, execute(fun, parameters), QueryOptions.DEFAULT.getProtocolVersion())
-                : new FunctionCall(fun, parameters);
+            return new FunctionCall(scalarFun, parameters);
         }
 
-        // All parameters must be terminal
-        private static ByteBuffer execute(Function fun, List<Term> parameters) throws InvalidRequestException
+        public AssignmentTestable.TestResult testAssignment(String keyspace, ColumnSpecification receiver)
         {
-            List<ByteBuffer> buffers = new ArrayList<ByteBuffer>(parameters.size());
-            for (Term t : parameters)
+            // Note: Functions.get() will return null if the function doesn't exist, or throw is no function matching
+            // the arguments can be found. We may get one of those if an undefined/wrong function is used as argument
+            // of another, existing, function. In that case, we return true here because we'll throw a proper exception
+            // later with a more helpful error message that if we were to return false here.
+            try
             {
-                assert t instanceof Term.Terminal;
-                buffers.add(((Term.Terminal)t).get(QueryOptions.DEFAULT));
+                Function fun = Functions.get(keyspace, name, terms, receiver.ksName, receiver.cfName, receiver.type);
+
+                // Because fromJson() can return whatever type the receiver is, we'll always get EXACT_MATCH.  To
+                // handle potentially ambiguous function calls with fromJson() as an argument, always return
+                // WEAKLY_ASSIGNABLE to force the user to typecast if necessary
+                if (fun != null && fun.name().equals(FromJsonFct.NAME))
+                    return TestResult.WEAKLY_ASSIGNABLE;
+
+                if (fun != null && receiver.type.equals(fun.returnType()))
+                    return AssignmentTestable.TestResult.EXACT_MATCH;
+                else if (fun == null || receiver.type.isValueCompatibleWith(fun.returnType()))
+                    return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
+                else
+                    return AssignmentTestable.TestResult.NOT_ASSIGNABLE;
             }
-
-            return executeInternal(fun, buffers);
-        }
-
-        public boolean isAssignableTo(String keyspace, ColumnSpecification receiver)
-        {
-            AbstractType<?> returnType = Functions.getReturnType(functionName, receiver.ksName, receiver.cfName);
-            // Note: if returnType == null, it means the function doesn't exist. We may get this if an undefined function
-            // is used as argument of another, existing, function. In that case, we return true here because we'll catch
-            // the fact that the method is undefined latter anyway and with a more helpful error message that if we were
-            // to return false here.
-            return returnType == null || receiver.type.isValueCompatibleWith(returnType);
+            catch (InvalidRequestException e)
+            {
+                return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
+            }
         }
 
         @Override
         public String toString()
         {
             StringBuilder sb = new StringBuilder();
-            sb.append(functionName).append("(");
+            sb.append(name).append("(");
             for (int i = 0; i < terms.size(); i++)
             {
-                if (i > 0) sb.append(", ");
+                if (i > 0)
+                    sb.append(", ");
                 sb.append(terms.get(i));
             }
             return sb.append(")").toString();
diff --git a/src/java/org/apache/cassandra/cql3/functions/FunctionName.java b/src/java/org/apache/cassandra/cql3/functions/FunctionName.java
new file mode 100644
index 0000000..d732efa
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/FunctionName.java
@@ -0,0 +1,82 @@
+/*
+ * 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.cql3.functions;
+
+import com.google.common.base.Objects;
+
+import org.apache.cassandra.db.SystemKeyspace;
+
+public final class FunctionName
+{
+    public final String keyspace;
+    public final String name;
+
+    public static FunctionName nativeFunction(String name)
+    {
+        return new FunctionName(SystemKeyspace.NAME, name);
+    }
+
+    public FunctionName(String keyspace, String name)
+    {
+        assert name != null : "Name parameter must not be null";
+        this.keyspace = keyspace;
+        this.name = name;
+    }
+
+    public FunctionName asNativeFunction()
+    {
+        return FunctionName.nativeFunction(name);
+    }
+
+    public boolean hasKeyspace()
+    {
+        return keyspace != null;
+    }
+
+    @Override
+    public final int hashCode()
+    {
+        return Objects.hashCode(keyspace, name);
+    }
+
+    @Override
+    public final boolean equals(Object o)
+    {
+        if (!(o instanceof FunctionName))
+            return false;
+
+        FunctionName that = (FunctionName)o;
+        return Objects.equal(this.keyspace, that.keyspace)
+            && Objects.equal(this.name, that.name);
+    }
+
+    public final boolean equalsNativeFunction(FunctionName nativeFunction)
+    {
+        assert nativeFunction.keyspace.equals(SystemKeyspace.NAME);
+        if (this.hasKeyspace() && !this.keyspace.equals(SystemKeyspace.NAME))
+            return false;
+
+        return Objects.equal(this.name, nativeFunction.name);
+    }
+
+    @Override
+    public String toString()
+    {
+        return keyspace == null ? name : keyspace + "." + name;
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/functions/Functions.java b/src/java/org/apache/cassandra/cql3/functions/Functions.java
index 605e7b3..0f1af19 100644
--- a/src/java/org/apache/cassandra/cql3/functions/Functions.java
+++ b/src/java/org/apache/cassandra/cql3/functions/Functions.java
@@ -17,168 +17,389 @@
  */
 package org.apache.cassandra.cql3.functions;
 
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CopyOnWriteArrayList;
 
-import com.google.common.collect.ArrayListMultimap;
-
-import org.apache.cassandra.cql3.ColumnIdentifier;
-import org.apache.cassandra.cql3.ColumnSpecification;
-import org.apache.cassandra.cql3.CQL3Type;
-import org.apache.cassandra.cql3.AssignementTestable;
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.service.MigrationListener;
+import org.apache.cassandra.service.MigrationManager;
 
 public abstract class Functions
 {
+    // We special case the token function because that's the only function whose argument types actually
+    // depend on the table on which the function is called. Because it's the sole exception, it's easier
+    // to handle it as a special case.
+    private static final FunctionName TOKEN_FUNCTION_NAME = FunctionName.nativeFunction("token");
+
     private Functions() {}
 
-    // If we ever allow this to be populated at runtime, this will need to be thread safe.
-    private static final ArrayListMultimap<String, Function.Factory> declared = ArrayListMultimap.create();
+    private static final ConcurrentMap<FunctionName, List<Function>> declared = new ConcurrentHashMap<>();
+
     static
     {
-        // All method sharing the same name must have the same returnType. We could find a way to make that clear.
-        declared.put("token", TokenFct.factory);
-
-        declared.put("now", AbstractFunction.factory(TimeuuidFcts.nowFct));
-        declared.put("mintimeuuid", AbstractFunction.factory(TimeuuidFcts.minTimeuuidFct));
-        declared.put("maxtimeuuid", AbstractFunction.factory(TimeuuidFcts.maxTimeuuidFct));
-        declared.put("dateof", AbstractFunction.factory(TimeuuidFcts.dateOfFct));
-        declared.put("unixtimestampof", AbstractFunction.factory(TimeuuidFcts.unixTimestampOfFct));
-        declared.put("uuid", AbstractFunction.factory(UuidFcts.uuidFct));
+        declare(AggregateFcts.countRowsFunction);
+        declare(TimeFcts.nowFct);
+        declare(TimeFcts.minTimeuuidFct);
+        declare(TimeFcts.maxTimeuuidFct);
+        declare(TimeFcts.dateOfFct);
+        declare(TimeFcts.unixTimestampOfFct);
+        declare(TimeFcts.timeUuidtoDate);
+        declare(TimeFcts.timeUuidToTimestamp);
+        declare(TimeFcts.timeUuidToUnixTimestamp);
+        declare(TimeFcts.timestampToDate);
+        declare(TimeFcts.timestampToUnixTimestamp);
+        declare(TimeFcts.dateToTimestamp);
+        declare(TimeFcts.dateToUnixTimestamp);
+        declare(UuidFcts.uuidFct);
 
         for (CQL3Type type : CQL3Type.Native.values())
         {
-            // Note: because text and varchar ends up being synonimous, our automatic makeToBlobFunction doesn't work
+            // Note: because text and varchar ends up being synonymous, our automatic makeToBlobFunction doesn't work
             // for varchar, so we special case it below. We also skip blob for obvious reasons.
-            if (type == CQL3Type.Native.VARCHAR || type == CQL3Type.Native.BLOB)
-                continue;
-
-            Function toBlob = BytesConversionFcts.makeToBlobFunction(type.getType());
-            Function fromBlob = BytesConversionFcts.makeFromBlobFunction(type.getType());
-            declared.put(toBlob.name(), AbstractFunction.factory(toBlob));
-            declared.put(fromBlob.name(), AbstractFunction.factory(fromBlob));
+            if (type != CQL3Type.Native.VARCHAR && type != CQL3Type.Native.BLOB)
+            {
+                declare(BytesConversionFcts.makeToBlobFunction(type.getType()));
+                declare(BytesConversionFcts.makeFromBlobFunction(type.getType()));
+            }
         }
-        declared.put("varcharasblob", AbstractFunction.factory(BytesConversionFcts.VarcharAsBlobFct));
-        declared.put("blobasvarchar", AbstractFunction.factory(BytesConversionFcts.BlobAsVarcharFact));
-    }
+        declare(BytesConversionFcts.VarcharAsBlobFct);
+        declare(BytesConversionFcts.BlobAsVarcharFact);
 
-    public static AbstractType<?> getReturnType(String functionName, String ksName, String cfName)
-    {
-        List<Function.Factory> factories = declared.get(functionName.toLowerCase());
-        return factories.isEmpty()
-             ? null // That's ok, we'll complain later
-             : factories.get(0).create(ksName, cfName).returnType();
-    }
-
-    public static ColumnSpecification makeArgSpec(ColumnSpecification receiver, Function fun, int i)
-    {
-        return new ColumnSpecification(receiver.ksName,
-                receiver.cfName,
-                new ColumnIdentifier("arg" + i +  "(" + fun.name() + ")", true),
-                fun.argsType().get(i));
-    }
-
-    public static Function get(String keyspace, String name, List<? extends AssignementTestable> providedArgs, ColumnSpecification receiver) throws InvalidRequestException
-    {
-        List<Function.Factory> factories = declared.get(name.toLowerCase());
-        if (factories.isEmpty())
-            throw new InvalidRequestException(String.format("Unknown CQL3 function %s called", name));
-
-        // Fast path if there is not choice
-        if (factories.size() == 1)
+        for (CQL3Type type : CQL3Type.Native.values())
         {
-            Function fun = factories.get(0).create(receiver.ksName, receiver.cfName);
-            validateTypes(keyspace, fun, providedArgs, receiver);
+            // special case varchar to avoid duplicating functions for UTF8Type
+            if (type != CQL3Type.Native.VARCHAR)
+            {
+                declare(AggregateFcts.makeCountFunction(type.getType()));
+                declare(AggregateFcts.makeMaxFunction(type.getType()));
+                declare(AggregateFcts.makeMinFunction(type.getType()));
+            }
+        }
+        declare(AggregateFcts.sumFunctionForByte);
+        declare(AggregateFcts.sumFunctionForShort);
+        declare(AggregateFcts.sumFunctionForInt32);
+        declare(AggregateFcts.sumFunctionForLong);
+        declare(AggregateFcts.sumFunctionForFloat);
+        declare(AggregateFcts.sumFunctionForDouble);
+        declare(AggregateFcts.sumFunctionForDecimal);
+        declare(AggregateFcts.sumFunctionForVarint);
+        declare(AggregateFcts.sumFunctionForCounter);
+        declare(AggregateFcts.avgFunctionForByte);
+        declare(AggregateFcts.avgFunctionForShort);
+        declare(AggregateFcts.avgFunctionForInt32);
+        declare(AggregateFcts.avgFunctionForLong);
+        declare(AggregateFcts.avgFunctionForFloat);
+        declare(AggregateFcts.avgFunctionForDouble);
+        declare(AggregateFcts.avgFunctionForVarint);
+        declare(AggregateFcts.avgFunctionForDecimal);
+        declare(AggregateFcts.avgFunctionForCounter);
+
+        MigrationManager.instance.register(new FunctionsMigrationListener());
+    }
+
+    private static void declare(Function fun)
+    {
+        synchronized (declared)
+        {
+            List<Function> functions = declared.get(fun.name());
+            if (functions == null)
+            {
+                functions = new CopyOnWriteArrayList<>();
+                List<Function> existing = declared.putIfAbsent(fun.name(), functions);
+                if (existing != null)
+                    functions = existing;
+            }
+            functions.add(fun);
+        }
+    }
+
+    public static ColumnSpecification makeArgSpec(String receiverKs, String receiverCf, Function fun, int i)
+    {
+        return new ColumnSpecification(receiverKs,
+                                       receiverCf,
+                                       new ColumnIdentifier("arg" + i + '(' + fun.name().toString().toLowerCase() + ')', true),
+                                       fun.argTypes().get(i));
+    }
+
+    public static int getOverloadCount(FunctionName name)
+    {
+        return find(name).size();
+    }
+
+    /**
+     * @param keyspace the current keyspace
+     * @param name the name of the function
+     * @param providedArgs the arguments provided for the function call
+     * @param receiverKs the receiver's keyspace
+     * @param receiverCf the receiver's table
+     * @param receiverType if the receiver type is known (during inserts, for example), this should be the type of
+     *                     the receiver
+     * @throws InvalidRequestException
+     */
+    public static Function get(String keyspace,
+                               FunctionName name,
+                               List<? extends AssignmentTestable> providedArgs,
+                               String receiverKs,
+                               String receiverCf,
+                               AbstractType<?> receiverType)
+    throws InvalidRequestException
+    {
+        if (name.equalsNativeFunction(TOKEN_FUNCTION_NAME))
+            return new TokenFct(Schema.instance.getCFMetaData(receiverKs, receiverCf));
+
+        // The toJson() function can accept any type of argument, so instances of it are not pre-declared.  Instead,
+        // we create new instances as needed while handling selectors (which is the only place that toJson() is supported,
+        // due to needing to know the argument types in advance).
+        if (name.equalsNativeFunction(ToJsonFct.NAME))
+            throw new InvalidRequestException("toJson() may only be used within the selection clause of SELECT statements");
+
+        // Similarly, we can only use fromJson when we know the receiver type (such as inserts)
+        if (name.equalsNativeFunction(FromJsonFct.NAME))
+        {
+            if (receiverType == null)
+                throw new InvalidRequestException("fromJson() cannot be used in the selection clause of a SELECT statement");
+            return FromJsonFct.getInstance(receiverType);
+        }
+
+        List<Function> candidates;
+        if (!name.hasKeyspace())
+        {
+            // function name not fully qualified
+            candidates = new ArrayList<>();
+            // add 'SYSTEM' (native) candidates
+            candidates.addAll(find(name.asNativeFunction()));
+            // add 'current keyspace' candidates
+            candidates.addAll(find(new FunctionName(keyspace, name.name)));
+        }
+        else
+            // function name is fully qualified (keyspace + name)
+            candidates = find(name);
+
+        if (candidates.isEmpty())
+            return null;
+
+        // Fast path if there is only one choice
+        if (candidates.size() == 1)
+        {
+            Function fun = candidates.get(0);
+            validateTypes(keyspace, fun, providedArgs, receiverKs, receiverCf);
             return fun;
         }
 
-        Function candidate = null;
-        for (Function.Factory factory : factories)
+        List<Function> compatibles = null;
+        for (Function toTest : candidates)
         {
-            Function toTest = factory.create(receiver.ksName, receiver.cfName);
-            if (!isValidType(keyspace, toTest, providedArgs, receiver))
-                continue;
-
-            if (candidate == null)
-                candidate = toTest;
-            else
-                throw new InvalidRequestException(String.format("Ambiguous call to function %s (can match both type signature %s and %s): use type casts to disambiguate", name, signature(candidate), signature(toTest)));
+            AssignmentTestable.TestResult r = matchAguments(keyspace, toTest, providedArgs, receiverKs, receiverCf);
+            switch (r)
+            {
+                case EXACT_MATCH:
+                    // We always favor exact matches
+                    return toTest;
+                case WEAKLY_ASSIGNABLE:
+                    if (compatibles == null)
+                        compatibles = new ArrayList<>();
+                    compatibles.add(toTest);
+                    break;
+            }
         }
-        if (candidate == null)
-            throw new InvalidRequestException(String.format("Invalid call to function %s, none of its type signature matches (known type signatures: %s)", name, signatures(factories, receiver)));
-        return candidate;
+
+        if (compatibles == null || compatibles.isEmpty())
+            throw new InvalidRequestException(String.format("Invalid call to function %s, none of its type signatures match (known type signatures: %s)",
+                                                            name, toString(candidates)));
+
+        if (compatibles.size() > 1)
+            throw new InvalidRequestException(String.format("Ambiguous call to function %s (can be matched by following signatures: %s): use type casts to disambiguate",
+                        name, toString(compatibles)));
+
+        return compatibles.get(0);
     }
 
-    private static void validateTypes(String keyspace, Function fun, List<? extends AssignementTestable> providedArgs, ColumnSpecification receiver) throws InvalidRequestException
+    public static List<Function> find(FunctionName name)
     {
-        if (!receiver.type.isValueCompatibleWith(fun.returnType()))
-            throw new InvalidRequestException(String.format("Type error: cannot assign result of function %s (type %s) to %s (type %s)", fun.name(), fun.returnType().asCQL3Type(), receiver, receiver.type.asCQL3Type()));
+        List<Function> functions = declared.get(name);
+        return functions != null ? functions : Collections.<Function>emptyList();
+    }
 
-        if (providedArgs.size() != fun.argsType().size())
-            throw new InvalidRequestException(String.format("Invalid number of arguments in call to function %s: %d required but %d provided", fun.name(), fun.argsType().size(), providedArgs.size()));
+    public static Function find(FunctionName name, List<AbstractType<?>> argTypes)
+    {
+        assert name.hasKeyspace() : "function name not fully qualified";
+        for (Function f : find(name))
+        {
+            if (typeEquals(f.argTypes(), argTypes))
+                return f;
+        }
+        return null;
+    }
+
+    // This method and matchArguments are somewhat duplicate, but this method allows us to provide more precise errors in the common
+    // case where there is no override for a given function. This is thus probably worth the minor code duplication.
+    private static void validateTypes(String keyspace,
+                                      Function fun,
+                                      List<? extends AssignmentTestable> providedArgs,
+                                      String receiverKs,
+                                      String receiverCf)
+    throws InvalidRequestException
+    {
+        if (providedArgs.size() != fun.argTypes().size())
+            throw new InvalidRequestException(String.format("Invalid number of arguments in call to function %s: %d required but %d provided", fun.name(), fun.argTypes().size(), providedArgs.size()));
 
         for (int i = 0; i < providedArgs.size(); i++)
         {
-            AssignementTestable provided = providedArgs.get(i);
+            AssignmentTestable provided = providedArgs.get(i);
 
             // If the concrete argument is a bind variables, it can have any type.
             // We'll validate the actually provided value at execution time.
             if (provided == null)
                 continue;
 
-            ColumnSpecification expected = makeArgSpec(receiver, fun, i);
-            if (!provided.isAssignableTo(keyspace, expected))
+            ColumnSpecification expected = makeArgSpec(receiverKs, receiverCf, fun, i);
+            if (!provided.testAssignment(keyspace, expected).isAssignable())
                 throw new InvalidRequestException(String.format("Type error: %s cannot be passed as argument %d of function %s of type %s", provided, i, fun.name(), expected.type.asCQL3Type()));
         }
     }
 
-    private static boolean isValidType(String keyspace, Function fun, List<? extends AssignementTestable> providedArgs, ColumnSpecification receiver) throws InvalidRequestException
+    private static AssignmentTestable.TestResult matchAguments(String keyspace,
+                                                               Function fun,
+                                                               List<? extends AssignmentTestable> providedArgs,
+                                                               String receiverKs,
+                                                               String receiverCf)
     {
-        if (!receiver.type.isValueCompatibleWith(fun.returnType()))
-            return false;
+        if (providedArgs.size() != fun.argTypes().size())
+            return AssignmentTestable.TestResult.NOT_ASSIGNABLE;
 
-        if (providedArgs.size() != fun.argsType().size())
-            return false;
-
+        // It's an exact match if all are exact match, but is not assignable as soon as any is non assignable.
+        AssignmentTestable.TestResult res = AssignmentTestable.TestResult.EXACT_MATCH;
         for (int i = 0; i < providedArgs.size(); i++)
         {
-            AssignementTestable provided = providedArgs.get(i);
-
-            // If the concrete argument is a bind variables, it can have any type.
-            // We'll validate the actually provided value at execution time.
+            AssignmentTestable provided = providedArgs.get(i);
             if (provided == null)
+            {
+                res = AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
                 continue;
+            }
 
-            ColumnSpecification expected = makeArgSpec(receiver, fun, i);
-            if (!provided.isAssignableTo(keyspace, expected))
-                return false;
+            ColumnSpecification expected = makeArgSpec(receiverKs, receiverCf, fun, i);
+            AssignmentTestable.TestResult argRes = provided.testAssignment(keyspace, expected);
+            if (argRes == AssignmentTestable.TestResult.NOT_ASSIGNABLE)
+                return AssignmentTestable.TestResult.NOT_ASSIGNABLE;
+            if (argRes == AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE)
+                res = AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
         }
+        return res;
+    }
+
+    private static String toString(List<Function> funs)
+    {
+        StringBuilder sb = new StringBuilder();
+        for (int i = 0; i < funs.size(); i++)
+        {
+            if (i > 0) sb.append(", ");
+            sb.append(funs.get(i));
+        }
+        return sb.toString();
+    }
+
+    public static void addOrReplaceFunction(AbstractFunction fun)
+    {
+        // We shouldn't get there unless that function don't exist
+        removeFunction(fun.name(), fun.argTypes());
+        declare(fun);
+    }
+
+    // Same remarks than for addFunction
+    public static void removeFunction(FunctionName name, List<AbstractType<?>> argTypes)
+    {
+        assert name.hasKeyspace() : "function name " + name + " not fully qualified";
+        synchronized (declared)
+        {
+            List<Function> functions = find(name);
+            for (int i = 0; i < functions.size(); i++)
+            {
+                Function f = functions.get(i);
+                if (!typeEquals(f.argTypes(), argTypes))
+                    continue;
+                assert !f.isNative();
+                functions.remove(i);
+                if (functions.isEmpty())
+                    declared.remove(name);
+                return;
+            }
+        }
+    }
+
+    public static List<Function> getReferencesTo(Function old)
+    {
+        List<Function> references = new ArrayList<>();
+        for (List<Function> functions : declared.values())
+            for (Function function : functions)
+                if (function.hasReferenceTo(old))
+                    references.add(function);
+        return references;
+    }
+
+    public static Collection<Function> all()
+    {
+        List<Function> all = new ArrayList<>();
+        for (List<Function> functions : declared.values())
+            all.addAll(functions);
+        return all;
+    }
+
+    /*
+     * We need to compare the CQL3 representation of the type because comparing
+     * the AbstractType will fail for example if a UDT has been changed.
+     * Reason is that UserType.equals() takes the field names and types into account.
+     * Example CQL sequence that would fail when comparing AbstractType:
+     *    CREATE TYPE foo ...
+     *    CREATE FUNCTION bar ( par foo ) RETURNS foo ...
+     *    ALTER TYPE foo ADD ...
+     * or
+     *    ALTER TYPE foo ALTER ...
+     * or
+     *    ALTER TYPE foo RENAME ...
+     */
+    public static boolean typeEquals(AbstractType<?> t1, AbstractType<?> t2)
+    {
+        return t1.asCQL3Type().toString().equals(t2.asCQL3Type().toString());
+    }
+
+    public static boolean typeEquals(List<AbstractType<?>> t1, List<AbstractType<?>> t2)
+    {
+        if (t1.size() != t2.size())
+            return false;
+        for (int i = 0; i < t1.size(); i ++)
+            if (!typeEquals(t1.get(i), t2.get(i)))
+                return false;
         return true;
     }
 
-    private static String signature(Function fun)
+    public static int typeHashCode(AbstractType<?> t)
     {
-        List<AbstractType<?>> args = fun.argsType();
-        StringBuilder sb = new StringBuilder();
-        sb.append("(");
-        for (int i = 0; i < args.size(); i++)
-        {
-            if (i > 0) sb.append(", ");
-            sb.append(args.get(i).asCQL3Type());
-        }
-        sb.append(") -> ");
-        sb.append(fun.returnType().asCQL3Type());
-        return sb.toString();
+        return t.asCQL3Type().toString().hashCode();
     }
 
-    private static String signatures(List<Function.Factory> factories, ColumnSpecification receiver)
+    public static int typeHashCode(List<AbstractType<?>> types)
     {
-        StringBuilder sb = new StringBuilder();
-        for (int i = 0; i < factories.size(); i++)
-        {
-            if (i > 0) sb.append(", ");
-            sb.append(signature(factories.get(i).create(receiver.ksName, receiver.cfName)));
+        int h = 0;
+        for (AbstractType<?> type : types)
+            h = h * 31 + typeHashCode(type);
+        return h;
+    }
+
+    private static class FunctionsMigrationListener extends MigrationListener
+    {
+        public void onUpdateUserType(String ksName, String typeName) {
+            for (Function function : all())
+                if (function instanceof UDFunction)
+                    ((UDFunction)function).userTypeUpdated(ksName, typeName);
         }
-        return sb.toString();
     }
 }
diff --git a/src/java/org/apache/cassandra/cql3/functions/JavaSourceUDFFactory.java b/src/java/org/apache/cassandra/cql3/functions/JavaSourceUDFFactory.java
new file mode 100644
index 0000000..515c947
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/JavaSourceUDFFactory.java
@@ -0,0 +1,536 @@
+/*
+ * 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.cql3.functions;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.lang.invoke.MethodHandle;
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.MethodType;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.StringTokenizer;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.io.ByteStreams;
+
+import org.apache.cassandra.utils.FBUtilities;
+import org.eclipse.jdt.core.compiler.IProblem;
+import org.eclipse.jdt.internal.compiler.*;
+import org.eclipse.jdt.internal.compiler.Compiler;
+import org.eclipse.jdt.internal.compiler.classfmt.ClassFileReader;
+import org.eclipse.jdt.internal.compiler.classfmt.ClassFormatException;
+import org.eclipse.jdt.internal.compiler.env.ICompilationUnit;
+import org.eclipse.jdt.internal.compiler.env.INameEnvironment;
+import org.eclipse.jdt.internal.compiler.env.NameEnvironmentAnswer;
+import org.eclipse.jdt.internal.compiler.impl.CompilerOptions;
+import org.eclipse.jdt.internal.compiler.problem.DefaultProblemFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.datastax.driver.core.DataType;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+/**
+ * Java source UDF code generation.
+ */
+public final class JavaSourceUDFFactory
+{
+    private static final String GENERATED_PACKAGE = "org.apache.cassandra.cql3.udf.gen";
+
+    static final Logger logger = LoggerFactory.getLogger(JavaSourceUDFFactory.class);
+
+    private static final AtomicInteger classSequence = new AtomicInteger();
+
+    private static final ClassLoader baseClassLoader = Thread.currentThread().getContextClassLoader();
+    private static final EcjTargetClassLoader targetClassLoader = new EcjTargetClassLoader();
+    private static final IErrorHandlingPolicy errorHandlingPolicy = DefaultErrorHandlingPolicies.proceedWithAllProblems();
+    private static final IProblemFactory problemFactory = new DefaultProblemFactory(Locale.ENGLISH);
+    private static final CompilerOptions compilerOptions;
+
+    /**
+     * Poor man's template - just a text file splitted at '#' chars.
+     * Each string at an even index is a constant string (just copied),
+     * each string at an odd index is an 'instruction'.
+     */
+    private static final String[] javaSourceTemplate;
+
+    static
+    {
+        Map<String, String> settings = new HashMap<>();
+        settings.put(CompilerOptions.OPTION_LineNumberAttribute,
+                     CompilerOptions.GENERATE);
+        settings.put(CompilerOptions.OPTION_SourceFileAttribute,
+                     CompilerOptions.DISABLED);
+        settings.put(CompilerOptions.OPTION_ReportDeprecation,
+                     CompilerOptions.IGNORE);
+        settings.put(CompilerOptions.OPTION_Source,
+                     CompilerOptions.VERSION_1_7);
+        settings.put(CompilerOptions.OPTION_TargetPlatform,
+                     CompilerOptions.VERSION_1_7);
+
+        compilerOptions = new CompilerOptions(settings);
+        compilerOptions.parseLiteralExpressionsAsConstants = true;
+
+        try (InputStream input = JavaSourceUDFFactory.class.getResource("JavaSourceUDF.txt").openConnection().getInputStream())
+        {
+            ByteArrayOutputStream output = new ByteArrayOutputStream();
+            FBUtilities.copy(input, output, Long.MAX_VALUE);
+            String template = output.toString();
+
+            StringTokenizer st = new StringTokenizer(template, "#");
+            javaSourceTemplate = new String[st.countTokens()];
+            for (int i = 0; st.hasMoreElements(); i++)
+                javaSourceTemplate[i] = st.nextToken();
+        }
+        catch (IOException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    static UDFunction buildUDF(FunctionName name,
+                               List<ColumnIdentifier> argNames,
+                               List<AbstractType<?>> argTypes,
+                               AbstractType<?> returnType,
+                               boolean calledOnNullInput,
+                               String body)
+    throws InvalidRequestException
+    {
+        // argDataTypes is just the C* internal argTypes converted to the Java Driver DataType
+        DataType[] argDataTypes = UDHelper.driverTypes(argTypes);
+        // returnDataType is just the C* internal returnType converted to the Java Driver DataType
+        DataType returnDataType = UDHelper.driverType(returnType);
+        // javaParamTypes is just the Java representation for argTypes resp. argDataTypes
+        Class<?>[] javaParamTypes = UDHelper.javaTypes(argDataTypes, calledOnNullInput);
+        // javaReturnType is just the Java representation for returnType resp. returnDataType
+        Class<?> javaReturnType = returnDataType.asJavaClass();
+
+        String clsName = generateClassName(name);
+
+        StringBuilder javaSourceBuilder = new StringBuilder();
+        int lineOffset = 1;
+        for (int i = 0; i < javaSourceTemplate.length; i++)
+        {
+            String s = javaSourceTemplate[i];
+
+            // strings at odd indexes are 'instructions'
+            if ((i & 1) == 1)
+            {
+                switch (s)
+                {
+                    case "class_name":
+                        s = clsName;
+                        break;
+                    case "body":
+                        lineOffset = countNewlines(javaSourceBuilder);
+                        s = body;
+                        break;
+                    case "arguments":
+                        s = generateArguments(javaParamTypes, argNames);
+                        break;
+                    case "argument_list":
+                        s = generateArgumentList(javaParamTypes, argNames);
+                        break;
+                    case "return_type":
+                        s = javaSourceName(javaReturnType);
+                        break;
+                }
+            }
+
+            javaSourceBuilder.append(s);
+        }
+
+        String targetClassName = GENERATED_PACKAGE + '.' + clsName;
+
+        String javaSource = javaSourceBuilder.toString();
+
+        logger.trace("Compiling Java source UDF '{}' as class '{}' using source:\n{}", name, targetClassName, javaSource);
+
+        try
+        {
+            EcjCompilationUnit compilationUnit = new EcjCompilationUnit(javaSource, targetClassName);
+
+            Compiler compiler = new Compiler(compilationUnit,
+                                             errorHandlingPolicy,
+                                             compilerOptions,
+                                             compilationUnit,
+                                             problemFactory);
+            compiler.compile(new ICompilationUnit[]{ compilationUnit });
+
+            if (compilationUnit.problemList != null && !compilationUnit.problemList.isEmpty())
+            {
+                boolean fullSource = false;
+                StringBuilder problems = new StringBuilder();
+                for (IProblem problem : compilationUnit.problemList)
+                {
+                    long ln = problem.getSourceLineNumber() - lineOffset;
+                    if (ln < 1L)
+                    {
+                        if (problem.isError())
+                        {
+                            // if generated source around UDF source provided by the user is buggy,
+                            // this code is appended.
+                            problems.append("GENERATED SOURCE ERROR: line ")
+                                    .append(problem.getSourceLineNumber())
+                                    .append(" (in generated source): ")
+                                    .append(problem.getMessage())
+                                    .append('\n');
+                            fullSource = true;
+                        }
+                    }
+                    else
+                    {
+                        problems.append("Line ")
+                                .append(Long.toString(ln))
+                                .append(": ")
+                                .append(problem.getMessage())
+                                .append('\n');
+                    }
+                }
+
+                if (fullSource)
+                    throw new InvalidRequestException("Java source compilation failed:\n" + problems + "\n generated source:\n" + javaSource);
+                else
+                    throw new InvalidRequestException("Java source compilation failed:\n" + problems);
+            }
+
+            Class cls = targetClassLoader.loadClass(targetClassName);
+
+            // Count only non-synthetic methods, so code coverage instrumentation doesn't cause a miscount
+            int nonSyntheticMethodCount = 0;
+            for (Method m : cls.getDeclaredMethods())
+            {
+                if (!m.isSynthetic())
+                {
+                    nonSyntheticMethodCount += 1;
+                }
+            }
+
+            if (nonSyntheticMethodCount != 2 || cls.getDeclaredConstructors().length != 1)
+                throw new InvalidRequestException("Check your source to not define additional Java methods or constructors");
+            MethodType methodType = MethodType.methodType(void.class)
+                                              .appendParameterTypes(FunctionName.class, List.class, List.class, DataType[].class,
+                                                                    AbstractType.class, DataType.class,
+                                                                    boolean.class, String.class);
+            MethodHandle ctor = MethodHandles.lookup().findConstructor(cls, methodType);
+            return (UDFunction) ctor.invokeWithArguments(name, argNames, argTypes, argDataTypes,
+                                                         returnType, returnDataType,
+                                                         calledOnNullInput, body);
+        }
+        catch (InvocationTargetException e)
+        {
+            // in case of an ITE, use the cause
+            throw new InvalidRequestException(String.format("Could not compile function '%s' from Java source: %s", name, e.getCause()));
+        }
+        catch (VirtualMachineError e)
+        {
+            throw e;
+        }
+        catch (Throwable e)
+        {
+            throw new InvalidRequestException(String.format("Could not compile function '%s' from Java source: %s", name, e));
+        }
+    }
+
+    private static int countNewlines(StringBuilder javaSource)
+    {
+        int ln = 0;
+        for (int i = 0; i < javaSource.length(); i++)
+            if (javaSource.charAt(i) == '\n')
+                ln++;
+        return ln;
+    }
+
+    private static String generateClassName(FunctionName name)
+    {
+        String qualifiedName = name.toString();
+
+        StringBuilder sb = new StringBuilder(qualifiedName.length() + 10);
+        sb.append('C');
+        for (int i = 0; i < qualifiedName.length(); i++)
+        {
+            char c = qualifiedName.charAt(i);
+            if (Character.isJavaIdentifierPart(c))
+                sb.append(c);
+        }
+        sb.append('_')
+          .append(classSequence.incrementAndGet());
+        return sb.toString();
+    }
+
+    private static String javaSourceName(Class<?> type)
+    {
+        String n = type.getName();
+        return n.startsWith("java.lang.") ? type.getSimpleName() : n;
+    }
+
+    private static String generateArgumentList(Class<?>[] paramTypes, List<ColumnIdentifier> argNames)
+    {
+        // initial builder size can just be a guess (prevent temp object allocations)
+        StringBuilder code = new StringBuilder(32 * paramTypes.length);
+        for (int i = 0; i < paramTypes.length; i++)
+        {
+            if (i > 0)
+                code.append(", ");
+            code.append(javaSourceName(paramTypes[i]))
+                .append(' ')
+                .append(argNames.get(i));
+        }
+        return code.toString();
+    }
+
+    private static String generateArguments(Class<?>[] paramTypes, List<ColumnIdentifier> argNames)
+    {
+        StringBuilder code = new StringBuilder(64 * paramTypes.length);
+        for (int i = 0; i < paramTypes.length; i++)
+        {
+            if (i > 0)
+                code.append(",\n");
+
+            if (logger.isTraceEnabled())
+                code.append("                /* parameter '").append(argNames.get(i)).append("' */\n");
+
+            code
+                // cast to Java type
+                .append("                (").append(javaSourceName(paramTypes[i])).append(") ")
+                // generate object representation of input parameter (call UDFunction.compose)
+                .append(composeMethod(paramTypes[i])).append("(protocolVersion, ").append(i).append(", params.get(").append(i).append("))");
+        }
+        return code.toString();
+    }
+
+    private static String composeMethod(Class<?> type)
+    {
+        return (type.isPrimitive()) ? ("compose_" + type.getName()) : "compose";
+    }
+
+    // Java source UDFs are a very simple compilation task, which allows us to let one class implement
+    // all interfaces required by ECJ.
+    static final class EcjCompilationUnit implements ICompilationUnit, ICompilerRequestor, INameEnvironment
+    {
+        List<IProblem> problemList;
+        private final String className;
+        private final char[] sourceCode;
+
+        EcjCompilationUnit(String sourceCode, String className)
+        {
+            this.className = className;
+            this.sourceCode = sourceCode.toCharArray();
+        }
+
+        // ICompilationUnit
+
+        @Override
+        public char[] getFileName()
+        {
+            return sourceCode;
+        }
+
+        @Override
+        public char[] getContents()
+        {
+            return sourceCode;
+        }
+
+        @Override
+        public char[] getMainTypeName()
+        {
+            int dot = className.lastIndexOf('.');
+            return ((dot > 0) ? className.substring(dot + 1) : className).toCharArray();
+        }
+
+        @Override
+        public char[][] getPackageName()
+        {
+            StringTokenizer izer = new StringTokenizer(className, ".");
+            char[][] result = new char[izer.countTokens() - 1][];
+            for (int i = 0; i < result.length; i++)
+                result[i] = izer.nextToken().toCharArray();
+            return result;
+        }
+
+        @Override
+        public boolean ignoreOptionalProblems()
+        {
+            return false;
+        }
+
+        // ICompilerRequestor
+
+        @Override
+        public void acceptResult(CompilationResult result)
+        {
+            if (result.hasErrors())
+            {
+                IProblem[] problems = result.getProblems();
+                if (problemList == null)
+                    problemList = new ArrayList<>(problems.length);
+                Collections.addAll(problemList, problems);
+            }
+            else
+            {
+                ClassFile[] classFiles = result.getClassFiles();
+                for (ClassFile classFile : classFiles)
+                    targetClassLoader.addClass(className, classFile.getBytes());
+            }
+        }
+
+        // INameEnvironment
+
+        @Override
+        public NameEnvironmentAnswer findType(char[][] compoundTypeName)
+        {
+            StringBuilder result = new StringBuilder();
+            for (int i = 0; i < compoundTypeName.length; i++)
+            {
+                if (i > 0)
+                    result.append('.');
+                result.append(compoundTypeName[i]);
+            }
+            return findType(result.toString());
+        }
+
+        @Override
+        public NameEnvironmentAnswer findType(char[] typeName, char[][] packageName)
+        {
+            StringBuilder result = new StringBuilder();
+            int i = 0;
+            for (; i < packageName.length; i++)
+            {
+                if (i > 0)
+                    result.append('.');
+                result.append(packageName[i]);
+            }
+            if (i > 0)
+                result.append('.');
+            result.append(typeName);
+            return findType(result.toString());
+        }
+
+        private NameEnvironmentAnswer findType(String className)
+        {
+            if (className.equals(this.className))
+            {
+                return new NameEnvironmentAnswer(this, null);
+            }
+
+            String resourceName = className.replace('.', '/') + ".class";
+
+            try (InputStream is = baseClassLoader.getResourceAsStream(resourceName))
+            {
+                if (is != null)
+                {
+                    byte[] classBytes = ByteStreams.toByteArray(is);
+                    char[] fileName = className.toCharArray();
+                    ClassFileReader classFileReader = new ClassFileReader(classBytes, fileName, true);
+                    return new NameEnvironmentAnswer(classFileReader, null);
+                }
+            }
+            catch (IOException | ClassFormatException exc)
+            {
+                throw new RuntimeException(exc);
+            }
+            return null;
+        }
+
+        private boolean isPackage(String result)
+        {
+            if (result.equals(this.className))
+                return false;
+            String resourceName = result.replace('.', '/') + ".class";
+            try (InputStream is = baseClassLoader.getResourceAsStream(resourceName))
+            {
+                return is == null;
+            }
+            catch (IOException e)
+            {
+                // we are here, since close on is failed. That means it was not null
+                return false;
+            }
+        }
+
+        @Override
+        public boolean isPackage(char[][] parentPackageName, char[] packageName)
+        {
+            StringBuilder result = new StringBuilder();
+            int i = 0;
+            if (parentPackageName != null)
+                for (; i < parentPackageName.length; i++)
+                {
+                    if (i > 0)
+                        result.append('.');
+                    result.append(parentPackageName[i]);
+                }
+
+            if (Character.isUpperCase(packageName[0]) && !isPackage(result.toString()))
+                return false;
+            if (i > 0)
+                result.append('.');
+            result.append(packageName);
+
+            return isPackage(result.toString());
+        }
+
+        @Override
+        public void cleanup()
+        {
+        }
+    }
+
+    static final class EcjTargetClassLoader extends ClassLoader
+    {
+        // This map is usually empty.
+        // It only contains data *during* UDF compilation but not during runtime.
+        //
+        // addClass() is invoked by ECJ after successful compilation of the generated Java source.
+        // loadClass(targetClassName) is invoked by buildUDF() after ECJ returned from successful compilation.
+        //
+        private final Map<String, byte[]> classes = new ConcurrentHashMap<>();
+
+        EcjTargetClassLoader()
+        {
+            super(baseClassLoader);
+        }
+
+        public void addClass(String className, byte[] classData)
+        {
+            classes.put(className, classData);
+        }
+
+        protected Class<?> findClass(String name) throws ClassNotFoundException
+        {
+            // remove the class binary - it's only used once - so it's wasting heap
+            byte[] classData = classes.remove(name);
+
+            return classData != null ? defineClass(name, classData, 0, classData.length)
+                                     : super.findClass(name);
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/functions/NativeAggregateFunction.java b/src/java/org/apache/cassandra/cql3/functions/NativeAggregateFunction.java
new file mode 100644
index 0000000..88aab4b
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/NativeAggregateFunction.java
@@ -0,0 +1,36 @@
+/*
+ * 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.cql3.functions;
+
+import org.apache.cassandra.db.marshal.AbstractType;
+
+/**
+ * Base class for the <code>AggregateFunction</code> native classes.
+ */
+public abstract class NativeAggregateFunction extends NativeFunction implements AggregateFunction
+{
+    protected NativeAggregateFunction(String name, AbstractType<?> returnType, AbstractType<?>... argTypes)
+    {
+        super(name, returnType, argTypes);
+    }
+
+    public final boolean isAggregate()
+    {
+        return true;
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/functions/NativeFunction.java b/src/java/org/apache/cassandra/cql3/functions/NativeFunction.java
new file mode 100644
index 0000000..df66ea0
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/NativeFunction.java
@@ -0,0 +1,38 @@
+/*
+ * 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.cql3.functions;
+
+import java.util.Arrays;
+
+import org.apache.cassandra.db.marshal.AbstractType;
+
+/**
+ * Base class for our native/hardcoded functions.
+ */
+public abstract class NativeFunction extends AbstractFunction
+{
+    protected NativeFunction(String name, AbstractType<?> returnType, AbstractType<?>... argTypes)
+    {
+        super(FunctionName.nativeFunction(name), Arrays.asList(argTypes), returnType);
+    }
+
+    public boolean isNative()
+    {
+        return true;
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/functions/NativeScalarFunction.java b/src/java/org/apache/cassandra/cql3/functions/NativeScalarFunction.java
new file mode 100644
index 0000000..3ae0607
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/NativeScalarFunction.java
@@ -0,0 +1,41 @@
+/*
+ * 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.cql3.functions;
+
+import org.apache.cassandra.db.marshal.AbstractType;
+
+/**
+ * Base class for the <code>ScalarFunction</code> native classes.
+ */
+public abstract class NativeScalarFunction extends NativeFunction implements ScalarFunction
+{
+    protected NativeScalarFunction(String name, AbstractType<?> returnType, AbstractType<?>... argsType)
+    {
+        super(name, returnType, argsType);
+    }
+
+    public boolean isCalledOnNullInput()
+    {
+        return true;
+    }
+
+    public final boolean isAggregate()
+    {
+        return false;
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/functions/ScalarFunction.java b/src/java/org/apache/cassandra/cql3/functions/ScalarFunction.java
new file mode 100644
index 0000000..ba258df
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/ScalarFunction.java
@@ -0,0 +1,41 @@
+/*
+ * 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.cql3.functions;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+/**
+ * Determines a single output value based on a single input value.
+ */
+public interface ScalarFunction extends Function
+{
+    public boolean isCalledOnNullInput();
+
+    /**
+     * Applies this function to the specified parameter.
+     *
+     * @param protocolVersion protocol version used for parameters and return value
+     * @param parameters the input parameters
+     * @return the result of applying this function to the parameter
+     * @throws InvalidRequestException if this function cannot not be applied to the parameter
+     */
+    public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters) throws InvalidRequestException;
+}
diff --git a/src/java/org/apache/cassandra/cql3/functions/ScriptBasedUDF.java b/src/java/org/apache/cassandra/cql3/functions/ScriptBasedUDF.java
new file mode 100644
index 0000000..2d46934
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/ScriptBasedUDF.java
@@ -0,0 +1,155 @@
+/*
+ * 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.cql3.functions;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import javax.script.Bindings;
+import javax.script.Compilable;
+import javax.script.CompiledScript;
+import javax.script.ScriptEngine;
+import javax.script.ScriptEngineFactory;
+import javax.script.ScriptEngineManager;
+import javax.script.ScriptException;
+import javax.script.SimpleBindings;
+
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.exceptions.FunctionExecutionException;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+public class ScriptBasedUDF extends UDFunction
+{
+    static final Map<String, Compilable> scriptEngines = new HashMap<>();
+
+    static {
+        ScriptEngineManager scriptEngineManager = new ScriptEngineManager();
+        for (ScriptEngineFactory scriptEngineFactory : scriptEngineManager.getEngineFactories())
+        {
+            ScriptEngine scriptEngine = scriptEngineFactory.getScriptEngine();
+            boolean compilable = scriptEngine instanceof Compilable;
+            if (compilable)
+            {
+                logger.info("Found scripting engine {} {} - {} {} - language names: {}",
+                            scriptEngineFactory.getEngineName(), scriptEngineFactory.getEngineVersion(),
+                            scriptEngineFactory.getLanguageName(), scriptEngineFactory.getLanguageVersion(),
+                            scriptEngineFactory.getNames());
+                for (String name : scriptEngineFactory.getNames())
+                    scriptEngines.put(name, (Compilable) scriptEngine);
+            }
+        }
+    }
+
+    private final CompiledScript script;
+
+    ScriptBasedUDF(FunctionName name,
+                   List<ColumnIdentifier> argNames,
+                   List<AbstractType<?>> argTypes,
+                   AbstractType<?> returnType,
+                   boolean calledOnNullInput,
+                   String language,
+                   String body)
+    throws InvalidRequestException
+    {
+        super(name, argNames, argTypes, returnType, calledOnNullInput, language, body);
+
+        if (!"JavaScript".equalsIgnoreCase(language))
+            logger.warn("Support for UDFs using '" + language + "' has been deprecated and removed in 3.0. If '" +
+                        language + "' actually is JavaScript, change the language used in CREATE/ALTER FUNCTION to " +
+                        "'javascript'.");
+
+        Compilable scriptEngine = scriptEngines.get(language);
+        if (scriptEngine == null)
+            throw new InvalidRequestException(String.format("Invalid language '%s' for function '%s'", language, name));
+
+        try
+        {
+            this.script = scriptEngine.compile(body);
+        }
+        catch (RuntimeException | ScriptException e)
+        {
+            logger.info("Failed to compile function '{}' for language {}: ", name, language, e);
+            throw new InvalidRequestException(
+                    String.format("Failed to compile function '%s' for language %s: %s", name, language, e));
+        }
+    }
+
+    public ByteBuffer executeUserDefined(int protocolVersion, List<ByteBuffer> parameters) throws InvalidRequestException
+    {
+        Object[] params = new Object[argTypes.size()];
+        for (int i = 0; i < params.length; i++)
+            params[i] = compose(protocolVersion, i, parameters.get(i));
+
+        try
+        {
+            Bindings bindings = new SimpleBindings();
+            for (int i = 0; i < params.length; i++)
+                bindings.put(argNames.get(i).toString(), params[i]);
+
+            Object result = script.eval(bindings);
+            if (result == null)
+                return null;
+
+            Class<?> javaReturnType = returnDataType.asJavaClass();
+            Class<?> resultType = result.getClass();
+            if (!javaReturnType.isAssignableFrom(resultType))
+            {
+                if (result instanceof Number)
+                {
+                    Number rNumber = (Number) result;
+                    if (javaReturnType == Integer.class)
+                        result = rNumber.intValue();
+                    else if (javaReturnType == Short.class)
+                        result = rNumber.shortValue();
+                    else if (javaReturnType == Byte.class)
+                        result = rNumber.byteValue();
+                    else if (javaReturnType == Long.class)
+                        result = rNumber.longValue();
+                    else if (javaReturnType == Float.class)
+                        result = rNumber.floatValue();
+                    else if (javaReturnType == Double.class)
+                        result = rNumber.doubleValue();
+                    else if (javaReturnType == BigInteger.class)
+                    {
+                        if (rNumber instanceof BigDecimal)
+                            result = ((BigDecimal)rNumber).toBigInteger();
+                        else if (rNumber instanceof Double || rNumber instanceof Float)
+                            result = new BigDecimal(rNumber.toString()).toBigInteger();
+                        else
+                            result = BigInteger.valueOf(rNumber.longValue());
+                    }
+                    else if (javaReturnType == BigDecimal.class)
+                        // String c'tor of BigDecimal is more accurate than valueOf(double)
+                        result = new BigDecimal(rNumber.toString());
+                }
+            }
+
+            return decompose(protocolVersion, result);
+        }
+        catch (RuntimeException | ScriptException e)
+        {
+            logger.trace("Execution of UDF '{}' failed", name, e);
+            throw FunctionExecutionException.create(this, e);
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/functions/TimeFcts.java b/src/java/org/apache/cassandra/cql3/functions/TimeFcts.java
new file mode 100644
index 0000000..a4623cd
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/TimeFcts.java
@@ -0,0 +1,229 @@
+/*
+ * 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.cql3.functions;
+
+import java.nio.ByteBuffer;
+import java.util.Date;
+import java.util.List;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.serializers.TimestampSerializer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.UUIDGen;
+
+public abstract class TimeFcts
+{
+    public static Logger logger = LoggerFactory.getLogger(TimeFcts.class);
+
+    public static final Function nowFct = new NativeScalarFunction("now", TimeUUIDType.instance)
+    {
+        public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
+        {
+            return ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes());
+        }
+    };
+
+    public static final Function minTimeuuidFct = new NativeScalarFunction("mintimeuuid", TimeUUIDType.instance, TimestampType.instance)
+    {
+        public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
+        {
+            ByteBuffer bb = parameters.get(0);
+            if (bb == null)
+                return null;
+
+            return ByteBuffer.wrap(UUIDGen.decompose(UUIDGen.minTimeUUID(TimestampType.instance.compose(bb).getTime())));
+        }
+    };
+
+    public static final Function maxTimeuuidFct = new NativeScalarFunction("maxtimeuuid", TimeUUIDType.instance, TimestampType.instance)
+    {
+        public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
+        {
+            ByteBuffer bb = parameters.get(0);
+            if (bb == null)
+                return null;
+
+            return ByteBuffer.wrap(UUIDGen.decompose(UUIDGen.maxTimeUUID(TimestampType.instance.compose(bb).getTime())));
+        }
+    };
+
+    /**
+     * Function that convert a value of <code>TIMEUUID</code> into a value of type <code>TIMESTAMP</code>.
+     * @deprecated Replaced by the {@link #timeUuidToTimestamp} function
+     */
+    public static final Function dateOfFct = new NativeScalarFunction("dateof", TimestampType.instance, TimeUUIDType.instance)
+    {
+        private volatile boolean hasLoggedDeprecationWarning;
+
+        public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
+        {
+            if (!hasLoggedDeprecationWarning)
+            {
+                hasLoggedDeprecationWarning = true;
+                logger.warn("The function 'dateof' is deprecated." +
+                            " Use the function 'toTimestamp' instead.");
+            }
+
+            ByteBuffer bb = parameters.get(0);
+            if (bb == null)
+                return null;
+
+            long timeInMillis = UUIDGen.unixTimestamp(UUIDGen.getUUID(bb));
+            return ByteBufferUtil.bytes(timeInMillis);
+        }
+    };
+
+    /**
+     * Function that convert a value of type <code>TIMEUUID</code> into an UNIX timestamp.
+     * @deprecated Replaced by the {@link #timeUuidToUnixTimestamp} function
+     */
+    public static final Function unixTimestampOfFct = new NativeScalarFunction("unixtimestampof", LongType.instance, TimeUUIDType.instance)
+    {
+        private volatile boolean hasLoggedDeprecationWarning;
+
+        public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
+        {
+            if (!hasLoggedDeprecationWarning)
+            {
+                hasLoggedDeprecationWarning = true;
+                logger.warn("The function 'unixtimestampof' is deprecated." +
+                            " Use the function 'toUnixTimestamp' instead.");
+            }
+
+            ByteBuffer bb = parameters.get(0);
+            if (bb == null)
+                return null;
+
+            return ByteBufferUtil.bytes(UUIDGen.unixTimestamp(UUIDGen.getUUID(bb)));
+        }
+    };
+
+    /**
+     * Function that convert a value of <code>TIMEUUID</code> into a value of type <code>DATE</code>.
+     */
+    public static final Function timeUuidtoDate = new NativeScalarFunction("todate", SimpleDateType.instance, TimeUUIDType.instance)
+    {
+        public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
+        {
+            ByteBuffer bb = parameters.get(0);
+            if (bb == null)
+                return null;
+
+            long timeInMillis = UUIDGen.unixTimestamp(UUIDGen.getUUID(bb));
+            return SimpleDateType.instance.fromTimeInMillis(timeInMillis);
+        }
+    };
+
+    /**
+     * Function that convert a value of type <code>TIMEUUID</code> into a value of type <code>TIMESTAMP</code>.
+     */
+    public static final Function timeUuidToTimestamp = new NativeScalarFunction("totimestamp", TimestampType.instance, TimeUUIDType.instance)
+    {
+        public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
+        {
+            ByteBuffer bb = parameters.get(0);
+            if (bb == null)
+                return null;
+
+            long timeInMillis = UUIDGen.unixTimestamp(UUIDGen.getUUID(bb));
+            return TimestampType.instance.fromTimeInMillis(timeInMillis);
+        }
+    };
+
+    /**
+     * Function that convert a value of type <code>TIMEUUID</code> into an UNIX timestamp.
+     */
+    public static final Function timeUuidToUnixTimestamp = new NativeScalarFunction("tounixtimestamp", LongType.instance, TimeUUIDType.instance)
+    {
+        public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
+        {
+            ByteBuffer bb = parameters.get(0);
+            if (bb == null)
+                return null;
+
+            return ByteBufferUtil.bytes(UUIDGen.unixTimestamp(UUIDGen.getUUID(bb)));
+        }
+    };
+
+    /**
+     * Function that convert a value of type <code>TIMESTAMP</code> into an UNIX timestamp.
+     */
+    public static final Function timestampToUnixTimestamp = new NativeScalarFunction("tounixtimestamp", LongType.instance, TimestampType.instance)
+    {
+        public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
+        {
+            ByteBuffer bb = parameters.get(0);
+            if (bb == null)
+                return null;
+
+            Date date = TimestampType.instance.compose(bb);
+            return date == null ? null : ByteBufferUtil.bytes(date.getTime());
+        }
+    };
+
+   /**
+    * Function that convert a value of type <code>TIMESTAMP</code> into a <code>DATE</code>.
+    */
+   public static final Function timestampToDate = new NativeScalarFunction("todate", SimpleDateType.instance, TimestampType.instance)
+   {
+       public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
+       {
+           ByteBuffer bb = parameters.get(0);
+           if (bb == null)
+               return null;
+
+           Date date = TimestampType.instance.compose(bb);
+           return date == null ? null : SimpleDateType.instance.fromTimeInMillis(date.getTime());
+       }
+   };
+
+   /**
+    * Function that convert a value of type <code>TIMESTAMP</code> into a <code>DATE</code>.
+    */
+   public static final Function dateToTimestamp = new NativeScalarFunction("totimestamp", TimestampType.instance, SimpleDateType.instance)
+   {
+       public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
+       {
+           ByteBuffer bb = parameters.get(0);
+           if (bb == null)
+               return null;
+
+           long millis = SimpleDateType.instance.toTimeInMillis(bb);
+           return TimestampType.instance.fromTimeInMillis(millis);
+       }
+   };
+
+   /**
+    * Function that convert a value of type <code>DATE</code> into an UNIX timestamp.
+    */
+   public static final Function dateToUnixTimestamp = new NativeScalarFunction("tounixtimestamp", LongType.instance, SimpleDateType.instance)
+   {
+       public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
+       {
+           ByteBuffer bb = parameters.get(0);
+           if (bb == null)
+               return null;
+
+           return ByteBufferUtil.bytes(SimpleDateType.instance.toTimeInMillis(bb));
+       }
+   };
+}
+
diff --git a/src/java/org/apache/cassandra/cql3/functions/TimeuuidFcts.java b/src/java/org/apache/cassandra/cql3/functions/TimeuuidFcts.java
deleted file mode 100644
index be20102..0000000
--- a/src/java/org/apache/cassandra/cql3/functions/TimeuuidFcts.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/*
- * 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.cql3.functions;
-
-import java.nio.ByteBuffer;
-import java.util.Date;
-import java.util.List;
-
-import org.apache.cassandra.db.marshal.TimestampType;
-import org.apache.cassandra.db.marshal.TimeUUIDType;
-import org.apache.cassandra.db.marshal.LongType;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.UUIDGen;
-
-public abstract class TimeuuidFcts
-{
-    public static final Function nowFct = new AbstractFunction("now", TimeUUIDType.instance)
-    {
-        public ByteBuffer execute(List<ByteBuffer> parameters)
-        {
-            return ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes());
-        }
-
-        @Override
-        public boolean isPure()
-        {
-            return false;
-        }
-    };
-
-    public static final Function minTimeuuidFct = new AbstractFunction("mintimeuuid", TimeUUIDType.instance, TimestampType.instance)
-    {
-        public ByteBuffer execute(List<ByteBuffer> parameters)
-        {
-            ByteBuffer bb = parameters.get(0);
-            if (bb == null)
-                return null;
-
-            return ByteBuffer.wrap(UUIDGen.decompose(UUIDGen.minTimeUUID(TimestampType.instance.compose(bb).getTime())));
-        }
-    };
-
-    public static final Function maxTimeuuidFct = new AbstractFunction("maxtimeuuid", TimeUUIDType.instance, TimestampType.instance)
-    {
-        public ByteBuffer execute(List<ByteBuffer> parameters)
-        {
-            ByteBuffer bb = parameters.get(0);
-            if (bb == null)
-                return null;
-
-            return ByteBuffer.wrap(UUIDGen.decompose(UUIDGen.maxTimeUUID(TimestampType.instance.compose(bb).getTime())));
-        }
-    };
-
-    public static final Function dateOfFct = new AbstractFunction("dateof", TimestampType.instance, TimeUUIDType.instance)
-    {
-        public ByteBuffer execute(List<ByteBuffer> parameters)
-        {
-            ByteBuffer bb = parameters.get(0);
-            if (bb == null)
-                return null;
-
-            return TimestampType.instance.decompose(new Date(UUIDGen.unixTimestamp(UUIDGen.getUUID(bb))));
-        }
-    };
-
-    public static final Function unixTimestampOfFct = new AbstractFunction("unixtimestampof", LongType.instance, TimeUUIDType.instance)
-    {
-        public ByteBuffer execute(List<ByteBuffer> parameters)
-        {
-            ByteBuffer bb = parameters.get(0);
-            if (bb == null)
-                return null;
-
-            return ByteBufferUtil.bytes(UUIDGen.unixTimestamp(UUIDGen.getUUID(bb)));
-        }
-    };
-}
-
diff --git a/src/java/org/apache/cassandra/cql3/functions/ToJsonFct.java b/src/java/org/apache/cassandra/cql3/functions/ToJsonFct.java
new file mode 100644
index 0000000..bcb4559
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/ToJsonFct.java
@@ -0,0 +1,67 @@
+/*
+ * 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.cql3.functions;
+
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class ToJsonFct extends NativeScalarFunction
+{
+    public static final FunctionName NAME = FunctionName.nativeFunction("tojson");
+
+    private static final Map<AbstractType<?>, ToJsonFct> instances = new ConcurrentHashMap<>();
+
+    public static ToJsonFct getInstance(List<AbstractType<?>> argTypes) throws InvalidRequestException
+    {
+        if (argTypes.size() != 1)
+            throw new InvalidRequestException(String.format("toJson() only accepts one argument (got %d)", argTypes.size()));
+
+        AbstractType<?> fromType = argTypes.get(0);
+        ToJsonFct func = instances.get(fromType);
+        if (func == null)
+        {
+            func = new ToJsonFct(fromType);
+            instances.put(fromType, func);
+        }
+        return func;
+    }
+
+    private ToJsonFct(AbstractType<?> argType)
+    {
+        super("tojson", UTF8Type.instance, argType);
+    }
+
+    public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters) throws InvalidRequestException
+    {
+        assert parameters.size() == 1 : "Expected 1 argument for toJson(), but got " + parameters.size();
+        ByteBuffer parameter = parameters.get(0);
+        if (parameter == null)
+            return ByteBufferUtil.bytes("null");
+
+        return ByteBufferUtil.bytes(argTypes.get(0).toJSONString(parameter, protocolVersion));
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/functions/TokenFct.java b/src/java/org/apache/cassandra/cql3/functions/TokenFct.java
index 5093a72..9d50a97 100644
--- a/src/java/org/apache/cassandra/cql3/functions/TokenFct.java
+++ b/src/java/org/apache/cassandra/cql3/functions/TokenFct.java
@@ -22,26 +22,17 @@
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.composites.CBuilder;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.service.StorageService;
 
-public class TokenFct extends AbstractFunction
+public class TokenFct extends NativeScalarFunction
 {
     // The actual token function depends on the partitioner used
     private static final IPartitioner partitioner = StorageService.getPartitioner();
 
-    public static final Function.Factory factory = new Function.Factory()
-    {
-        public Function create(String ksName, String cfName)
-        {
-            return new TokenFct(Schema.instance.getCFMetaData(ksName, cfName));
-        }
-    };
-
     private final CFMetaData cfm;
 
     public TokenFct(CFMetaData cfm)
@@ -59,7 +50,7 @@
         return types;
     }
 
-    public ByteBuffer execute(List<ByteBuffer> parameters) throws InvalidRequestException
+    public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters) throws InvalidRequestException
     {
         CBuilder builder = cfm.getKeyValidatorAsCType().builder();
         for (int i = 0; i < parameters.size(); i++)
diff --git a/src/java/org/apache/cassandra/cql3/functions/UDAggregate.java b/src/java/org/apache/cassandra/cql3/functions/UDAggregate.java
new file mode 100644
index 0000000..5f4d107
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/UDAggregate.java
@@ -0,0 +1,228 @@
+/*
+ * 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.cql3.functions;
+
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import com.google.common.base.Objects;
+import com.google.common.collect.ImmutableSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.tracing.Tracing;
+
+/**
+ * Base class for user-defined-aggregates.
+ */
+public class UDAggregate extends AbstractFunction implements AggregateFunction
+{
+    protected static final Logger logger = LoggerFactory.getLogger(UDAggregate.class);
+
+    protected final AbstractType<?> stateType;
+    protected final ByteBuffer initcond;
+    private final ScalarFunction stateFunction;
+    private final ScalarFunction finalFunction;
+
+    public UDAggregate(FunctionName name,
+                       List<AbstractType<?>> argTypes,
+                       AbstractType<?> returnType,
+                       ScalarFunction stateFunc,
+                       ScalarFunction finalFunc,
+                       ByteBuffer initcond)
+    {
+        super(name, argTypes, returnType);
+        this.stateFunction = stateFunc;
+        this.finalFunction = finalFunc;
+        this.stateType = stateFunc != null ? stateFunc.returnType() : null;
+        this.initcond = initcond;
+    }
+
+    public static UDAggregate create(FunctionName name,
+                                     List<AbstractType<?>> argTypes,
+                                     AbstractType<?> returnType,
+                                     FunctionName stateFunc,
+                                     FunctionName finalFunc,
+                                     AbstractType<?> stateType,
+                                     ByteBuffer initcond)
+    throws InvalidRequestException
+    {
+        List<AbstractType<?>> stateTypes = new ArrayList<>(argTypes.size() + 1);
+        stateTypes.add(stateType);
+        stateTypes.addAll(argTypes);
+        List<AbstractType<?>> finalTypes = Collections.<AbstractType<?>>singletonList(stateType);
+        return new UDAggregate(name,
+                               argTypes,
+                               returnType,
+                               resolveScalar(name, stateFunc, stateTypes),
+                               finalFunc != null ? resolveScalar(name, finalFunc, finalTypes) : null,
+                               initcond);
+    }
+
+    public static UDAggregate createBroken(FunctionName name,
+                                           List<AbstractType<?>> argTypes,
+                                           AbstractType<?> returnType,
+                                           ByteBuffer initcond,
+                                           final InvalidRequestException reason)
+    {
+        return new UDAggregate(name, argTypes, returnType, null, null, initcond)
+        {
+            public Aggregate newAggregate() throws InvalidRequestException
+            {
+                throw new InvalidRequestException(String.format("Aggregate '%s' exists but hasn't been loaded successfully for the following reason: %s. "
+                                                                + "Please see the server log for more details",
+                                                                this,
+                                                                reason.getMessage()));
+            }
+        };
+    }
+
+    public boolean hasReferenceTo(Function function)
+    {
+        return stateFunction == function || finalFunction == function;
+    }
+
+    public Iterable<Function> getFunctions()
+    {
+        if (stateFunction == null)
+            return Collections.emptySet();
+        if (finalFunction != null)
+            return ImmutableSet.of(this, stateFunction, finalFunction);
+        else
+            return ImmutableSet.of(this, stateFunction);
+    }
+
+    public boolean isAggregate()
+    {
+        return true;
+    }
+
+    public boolean isNative()
+    {
+        return false;
+    }
+
+    public ScalarFunction stateFunction()
+    {
+        return stateFunction;
+    }
+
+    public ScalarFunction finalFunction()
+    {
+        return finalFunction;
+    }
+
+    public ByteBuffer initialCondition()
+    {
+        return initcond;
+    }
+
+    public AbstractType<?> stateType()
+    {
+        return stateType;
+    }
+
+    public Aggregate newAggregate() throws InvalidRequestException
+    {
+        return new Aggregate()
+        {
+            private long stateFunctionCount;
+            private long stateFunctionDuration;
+
+            private ByteBuffer state;
+            {
+                reset();
+            }
+
+            public void addInput(int protocolVersion, List<ByteBuffer> values) throws InvalidRequestException
+            {
+                long startTime = System.nanoTime();
+                stateFunctionCount++;
+                List<ByteBuffer> fArgs = new ArrayList<>(values.size() + 1);
+                fArgs.add(state);
+                fArgs.addAll(values);
+                if (stateFunction instanceof UDFunction)
+                {
+                    UDFunction udf = (UDFunction)stateFunction;
+                    if (udf.isCallableWrtNullable(fArgs))
+                        state = udf.executeUserDefined(protocolVersion, fArgs);
+                }
+                else
+                {
+                    state = stateFunction.execute(protocolVersion, fArgs);
+                }
+                stateFunctionDuration += (System.nanoTime() - startTime) / 1000;
+            }
+
+            public ByteBuffer compute(int protocolVersion) throws InvalidRequestException
+            {
+                // final function is traced in UDFunction
+                Tracing.trace("Executed UDA {}: {} call(s) to state function {} in {}\u03bcs", name(), stateFunctionCount, stateFunction.name(), stateFunctionDuration);
+                if (finalFunction == null)
+                    return state;
+
+                List<ByteBuffer> fArgs = Collections.singletonList(state);
+                ByteBuffer result = finalFunction.execute(protocolVersion, fArgs);
+                return result;
+            }
+
+            public void reset()
+            {
+                state = initcond != null ? initcond.duplicate() : null;
+                stateFunctionDuration = 0;
+                stateFunctionCount = 0;
+            }
+        };
+    }
+
+    private static ScalarFunction resolveScalar(FunctionName aName, FunctionName fName, List<AbstractType<?>> argTypes) throws InvalidRequestException
+    {
+        Function func = Functions.find(fName, argTypes);
+        if (func == null)
+            throw new InvalidRequestException(String.format("Referenced state function '%s %s' for aggregate '%s' does not exist",
+                                                            fName, Arrays.toString(UDHelper.driverTypes(argTypes)), aName));
+        if (!(func instanceof ScalarFunction))
+            throw new InvalidRequestException(String.format("Referenced state function '%s %s' for aggregate '%s' is not a scalar function",
+                                                            fName, Arrays.toString(UDHelper.driverTypes(argTypes)), aName));
+        return (ScalarFunction) func;
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (!(o instanceof UDAggregate))
+            return false;
+
+        UDAggregate that = (UDAggregate) o;
+        return Objects.equal(name, that.name)
+            && Functions.typeEquals(argTypes, that.argTypes)
+            && Functions.typeEquals(returnType, that.returnType)
+            && Objects.equal(stateFunction, that.stateFunction)
+            && Objects.equal(finalFunction, that.finalFunction)
+            && Objects.equal(stateType, that.stateType)
+            && Objects.equal(initcond, that.initcond);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hashCode(name, Functions.typeHashCode(argTypes), Functions.typeHashCode(returnType), stateFunction, finalFunction, stateType, initcond);
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/functions/UDFunction.java b/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
new file mode 100644
index 0000000..1e5cea6
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
@@ -0,0 +1,317 @@
+/*
+ * 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.cql3.functions;
+
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import com.google.common.base.Objects;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.datastax.driver.core.DataType;
+import com.datastax.driver.core.ProtocolVersion;
+import com.datastax.driver.core.UserType;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.exceptions.*;
+import org.apache.cassandra.service.MigrationManager;
+import org.apache.cassandra.tracing.Tracing;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+/**
+ * Base class for User Defined Functions.
+ */
+public abstract class UDFunction extends AbstractFunction implements ScalarFunction
+{
+    protected static final Logger logger = LoggerFactory.getLogger(UDFunction.class);
+
+    protected final List<ColumnIdentifier> argNames;
+
+    protected final String language;
+    protected final String body;
+
+    protected final DataType[] argDataTypes;
+    protected final DataType returnDataType;
+    protected final boolean calledOnNullInput;
+
+    protected UDFunction(FunctionName name,
+                         List<ColumnIdentifier> argNames,
+                         List<AbstractType<?>> argTypes,
+                         AbstractType<?> returnType,
+                         boolean calledOnNullInput,
+                         String language,
+                         String body)
+    {
+        this(name, argNames, argTypes, UDHelper.driverTypes(argTypes), returnType,
+             UDHelper.driverType(returnType), calledOnNullInput, language, body);
+    }
+
+    protected UDFunction(FunctionName name,
+                         List<ColumnIdentifier> argNames,
+                         List<AbstractType<?>> argTypes,
+                         DataType[] argDataTypes,
+                         AbstractType<?> returnType,
+                         DataType returnDataType,
+                         boolean calledOnNullInput,
+                         String language,
+                         String body)
+    {
+        super(name, argTypes, returnType);
+        assert new HashSet<>(argNames).size() == argNames.size() : "duplicate argument names";
+        this.argNames = argNames;
+        this.language = language;
+        this.body = body;
+        this.argDataTypes = argDataTypes;
+        this.returnDataType = returnDataType;
+        this.calledOnNullInput = calledOnNullInput;
+    }
+
+    public static UDFunction create(FunctionName name,
+                                    List<ColumnIdentifier> argNames,
+                                    List<AbstractType<?>> argTypes,
+                                    AbstractType<?> returnType,
+                                    boolean calledOnNullInput,
+                                    String language,
+                                    String body)
+    throws InvalidRequestException
+    {
+        if (!DatabaseDescriptor.enableUserDefinedFunctions())
+            throw new InvalidRequestException("User-defined-functions are disabled in cassandra.yaml - set enable_user_defined_functions=true to enable if you are aware of the security risks");
+
+        switch (language)
+        {
+            case "java": return JavaSourceUDFFactory.buildUDF(name, argNames, argTypes, returnType, calledOnNullInput, body);
+            default: return new ScriptBasedUDF(name, argNames, argTypes, returnType, calledOnNullInput, language, body);
+        }
+    }
+
+    /**
+     * It can happen that a function has been declared (is listed in the scheam) but cannot
+     * be loaded (maybe only on some nodes). This is the case for instance if the class defining
+     * the class is not on the classpath for some of the node, or after a restart. In that case,
+     * we create a "fake" function so that:
+     *  1) the broken function can be dropped easily if that is what people want to do.
+     *  2) we return a meaningful error message if the function is executed (something more precise
+     *     than saying that the function doesn't exist)
+     */
+    public static UDFunction createBrokenFunction(FunctionName name,
+                                                  List<ColumnIdentifier> argNames,
+                                                  List<AbstractType<?>> argTypes,
+                                                  AbstractType<?> returnType,
+                                                  boolean calledOnNullInput,
+                                                  String language,
+                                                  String body,
+                                                  final InvalidRequestException reason)
+    {
+        return new UDFunction(name, argNames, argTypes, returnType, calledOnNullInput, language, body)
+        {
+            public ByteBuffer executeUserDefined(int protocolVersion, List<ByteBuffer> parameters) throws InvalidRequestException
+            {
+                throw new InvalidRequestException(String.format("Function '%s' exists but hasn't been loaded successfully "
+                                                                + "for the following reason: %s. Please see the server log for details",
+                                                                this,
+                                                                reason.getMessage()));
+            }
+        };
+    }
+
+    public final ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters) throws InvalidRequestException
+    {
+        if (!DatabaseDescriptor.enableUserDefinedFunctions())
+            throw new InvalidRequestException("User-defined-functions are disabled in cassandra.yaml - set enable_user_defined_functions=true to enable if you are aware of the security risks");
+
+        if (!isCallableWrtNullable(parameters))
+            return null;
+
+        long tStart = System.nanoTime();
+        ByteBuffer result = executeUserDefined(protocolVersion, parameters);
+        Tracing.trace("Executed UDF {} in {}\u03bcs", name(), (System.nanoTime() - tStart) / 1000);
+        return result;
+    }
+
+    public boolean isCallableWrtNullable(List<ByteBuffer> parameters)
+    {
+        if (!calledOnNullInput)
+            for (int i = 0; i < parameters.size(); i++)
+                if (UDHelper.isNullOrEmpty(argTypes.get(i), parameters.get(i)))
+                    return false;
+        return true;
+    }
+
+    protected abstract ByteBuffer executeUserDefined(int protocolVersion, List<ByteBuffer> parameters) throws InvalidRequestException;
+
+    public boolean isAggregate()
+    {
+        return false;
+    }
+
+    public boolean isNative()
+    {
+        return false;
+    }
+
+    public boolean isCalledOnNullInput()
+    {
+        return calledOnNullInput;
+    }
+
+    public List<ColumnIdentifier> argNames()
+    {
+        return argNames;
+    }
+
+    public String body()
+    {
+        return body;
+    }
+
+    public String language()
+    {
+        return language;
+    }
+
+    /**
+     * Used by UDF implementations (both Java code generated by {@link org.apache.cassandra.cql3.functions.JavaSourceUDFFactory}
+     * and script executor {@link org.apache.cassandra.cql3.functions.ScriptBasedUDF}) to convert the C*
+     * serialized representation to the Java object representation.
+     *
+     * @param protocolVersion the native protocol version used for serialization
+     * @param argIndex index of the UDF input argument
+     */
+    protected Object compose(int protocolVersion, int argIndex, ByteBuffer value)
+    {
+        return UDHelper.isNullOrEmpty(argTypes.get(argIndex), value) ? null : argDataTypes[argIndex].deserialize(value, ProtocolVersion.fromInt(protocolVersion));
+    }
+
+    // do not remove - used by generated Java UDFs
+    protected float compose_float(int protocolVersion, int argIndex, ByteBuffer value)
+    {
+        assert value != null && value.remaining() > 0;
+        return (float)DataType.cfloat().deserialize(value, ProtocolVersion.fromInt(protocolVersion));
+    }
+
+    // do not remove - used by generated Java UDFs
+    protected double compose_double(int protocolVersion, int argIndex, ByteBuffer value)
+    {
+        assert value != null && value.remaining() > 0;
+        return (double)DataType.cdouble().deserialize(value, ProtocolVersion.fromInt(protocolVersion));
+    }
+
+    // do not remove - used by generated Java UDFs
+    protected byte compose_byte(int protocolVersion, int argIndex, ByteBuffer value)
+    {
+        assert value != null && value.remaining() > 0;
+        return (byte)DataType.tinyint().deserialize(value, ProtocolVersion.fromInt(protocolVersion));
+    }
+
+    // do not remove - used by generated Java UDFs
+    protected short compose_short(int protocolVersion, int argIndex, ByteBuffer value)
+    {
+        assert value != null && value.remaining() > 0;
+        return (short)DataType.smallint().deserialize(value, ProtocolVersion.fromInt(protocolVersion));
+    }
+
+    // do not remove - used by generated Java UDFs
+    protected int compose_int(int protocolVersion, int argIndex, ByteBuffer value)
+    {
+        assert value != null && value.remaining() > 0;
+        return (int)DataType.cint().deserialize(value, ProtocolVersion.fromInt(protocolVersion));
+    }
+
+    // do not remove - used by generated Java UDFs
+    protected long compose_long(int protocolVersion, int argIndex, ByteBuffer value)
+    {
+        assert value != null && value.remaining() > 0;
+        return (long)DataType.bigint().deserialize(value, ProtocolVersion.fromInt(protocolVersion));
+    }
+
+    // do not remove - used by generated Java UDFs
+    protected boolean compose_boolean(int protocolVersion, int argIndex, ByteBuffer value)
+    {
+        assert value != null && value.remaining() > 0;
+        return (boolean) DataType.cboolean().deserialize(value, ProtocolVersion.fromInt(protocolVersion));
+    }
+
+    /**
+     * Used by UDF implementations (both Java code generated by {@link org.apache.cassandra.cql3.functions.JavaSourceUDFFactory}
+     * and script executor {@link org.apache.cassandra.cql3.functions.ScriptBasedUDF}) to convert the Java
+     * object representation for the return value to the C* serialized representation.
+     *
+     * @param protocolVersion the native protocol version used for serialization
+     */
+    protected ByteBuffer decompose(int protocolVersion, Object value)
+    {
+        return value == null ? null : returnDataType.serialize(value, ProtocolVersion.fromInt(protocolVersion));
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (!(o instanceof UDFunction))
+            return false;
+
+        UDFunction that = (UDFunction)o;
+        return Objects.equal(name, that.name)
+            && Objects.equal(argNames, that.argNames)
+            && Functions.typeEquals(argTypes, that.argTypes)
+            && Functions.typeEquals(returnType, that.returnType)
+            && Objects.equal(language, that.language)
+            && Objects.equal(body, that.body);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hashCode(name, Functions.typeHashCode(argTypes), Functions.typeHashCode(returnType), returnType, language, body);
+    }
+
+    public void userTypeUpdated(String ksName, String typeName)
+    {
+        boolean updated = false;
+
+        for (int i = 0; i < argDataTypes.length; i++)
+        {
+            DataType dataType = argDataTypes[i];
+            if (dataType instanceof UserType)
+            {
+                UserType userType = (UserType) dataType;
+                if (userType.getKeyspace().equals(ksName) && userType.getTypeName().equals(typeName))
+                {
+                    KSMetaData ksm = Schema.instance.getKSMetaData(ksName);
+                    assert ksm != null;
+
+                    org.apache.cassandra.db.marshal.UserType ut = ksm.userTypes.getType(ByteBufferUtil.bytes(typeName));
+
+                    DataType newUserType = UDHelper.driverType(ut);
+                    argDataTypes[i] = newUserType;
+
+                    argTypes.set(i, ut);
+
+                    updated = true;
+                }
+            }
+        }
+
+        if (updated)
+            MigrationManager.announceNewFunction(this, true);
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/functions/UDHelper.java b/src/java/org/apache/cassandra/cql3/functions/UDHelper.java
new file mode 100644
index 0000000..d1d12e6
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/functions/UDHelper.java
@@ -0,0 +1,129 @@
+/*
+ * 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.cql3.functions;
+
+import java.lang.invoke.MethodHandle;
+import java.lang.invoke.MethodHandles;
+import java.lang.reflect.Method;
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import com.datastax.driver.core.DataType;
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.db.marshal.AbstractType;
+
+/**
+ * Helper class for User Defined Functions + Aggregates.
+ */
+public final class UDHelper
+{
+    // TODO make these c'tors and methods public in Java-Driver - see https://datastax-oss.atlassian.net/browse/JAVA-502
+    static final MethodHandle methodParseOne;
+    static
+    {
+        try
+        {
+            Class<?> cls = Class.forName("com.datastax.driver.core.CassandraTypeParser");
+            Method m = cls.getDeclaredMethod("parseOne", String.class);
+            m.setAccessible(true);
+            methodParseOne = MethodHandles.lookup().unreflect(m);
+        }
+        catch (Exception e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    /**
+     * Construct an array containing the Java classes for the given Java Driver {@link com.datastax.driver.core.DataType}s.
+     *
+     * @param dataTypes  array with UDF argument types
+     * @param calledOnNullInput whether to allow {@code null} as an argument value
+     * @return array of same size with UDF arguments
+     */
+    public static Class<?>[] javaTypes(DataType[] dataTypes, boolean calledOnNullInput)
+    {
+        Class<?>[] paramTypes = new Class[dataTypes.length];
+        for (int i = 0; i < paramTypes.length; i++)
+        {
+            Class<?> clazz = dataTypes[i].asJavaClass();
+            if (!calledOnNullInput)
+            {
+                // only care about classes that can be used in a data type
+                if (clazz == Integer.class)
+                    clazz = int.class;
+                else if (clazz == Long.class)
+                    clazz = long.class;
+                else if (clazz == Byte.class)
+                    clazz = byte.class;
+                else if (clazz == Short.class)
+                    clazz = short.class;
+                else if (clazz == Float.class)
+                    clazz = float.class;
+                else if (clazz == Double.class)
+                    clazz = double.class;
+                else if (clazz == Boolean.class)
+                    clazz = boolean.class;
+            }
+            paramTypes[i] = clazz;
+        }
+        return paramTypes;
+    }
+
+    /**
+     * Construct an array containing the Java Driver {@link com.datastax.driver.core.DataType}s for the
+     * C* internal types.
+     *
+     * @param abstractTypes list with UDF argument types
+     * @return array with argument types as {@link com.datastax.driver.core.DataType}
+     */
+    public static DataType[] driverTypes(List<AbstractType<?>> abstractTypes)
+    {
+        DataType[] argDataTypes = new DataType[abstractTypes.size()];
+        for (int i = 0; i < argDataTypes.length; i++)
+            argDataTypes[i] = driverType(abstractTypes.get(i));
+        return argDataTypes;
+    }
+
+    /**
+     * Returns the Java Driver {@link com.datastax.driver.core.DataType} for the C* internal type.
+     */
+    public static DataType driverType(AbstractType abstractType)
+    {
+        CQL3Type cqlType = abstractType.asCQL3Type();
+        try
+        {
+            return (DataType) methodParseOne.invoke(cqlType.getType().toString());
+        }
+        catch (RuntimeException | Error e)
+        {
+            // immediately rethrow these...
+            throw e;
+        }
+        catch (Throwable e)
+        {
+            throw new RuntimeException("cannot parse driver type " + cqlType.getType().toString(), e);
+        }
+    }
+
+    public static boolean isNullOrEmpty(AbstractType<?> type, ByteBuffer bb)
+    {
+        return bb == null ||
+               (bb.remaining() == 0 && type.isEmptyValueMeaningless());
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/functions/UuidFcts.java b/src/java/org/apache/cassandra/cql3/functions/UuidFcts.java
index 718bcbc..0aa3ac4 100644
--- a/src/java/org/apache/cassandra/cql3/functions/UuidFcts.java
+++ b/src/java/org/apache/cassandra/cql3/functions/UuidFcts.java
@@ -26,17 +26,11 @@
 
 public abstract class UuidFcts
 {
-    public static final Function uuidFct = new AbstractFunction("uuid", UUIDType.instance)
+    public static final Function uuidFct = new NativeScalarFunction("uuid", UUIDType.instance)
     {
-        public ByteBuffer execute(List<ByteBuffer> parameters)
+        public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters)
         {
             return UUIDSerializer.instance.serialize(UUID.randomUUID());
         }
-
-        @Override
-        public boolean isPure()
-        {
-            return false;
-        }
     };
 }
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/AbstractPrimaryKeyRestrictions.java b/src/java/org/apache/cassandra/cql3/restrictions/AbstractPrimaryKeyRestrictions.java
new file mode 100644
index 0000000..51c3e26
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/restrictions/AbstractPrimaryKeyRestrictions.java
@@ -0,0 +1,61 @@
+/*
+ * 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.cql3.restrictions;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.statements.Bound;
+import org.apache.cassandra.db.composites.CType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+/**
+ * Base class for <code>PrimaryKeyRestrictions</code>.
+ */
+abstract class AbstractPrimaryKeyRestrictions extends AbstractRestriction implements PrimaryKeyRestrictions
+{
+    /**
+     * The composite type.
+     */
+    protected final CType ctype;
+
+    public AbstractPrimaryKeyRestrictions(CType ctype)
+    {
+        this.ctype = ctype;
+    }
+
+    @Override
+    public List<ByteBuffer> bounds(CFMetaData cfm, Bound b, QueryOptions options) throws InvalidRequestException
+    {
+        return values(cfm, options);
+    }
+
+    @Override
+    public final boolean isEmpty()
+    {
+        return getColumnDefs().isEmpty();
+    }
+
+    @Override
+    public final int size()
+    {
+        return getColumnDefs().size();
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/AbstractRestriction.java b/src/java/org/apache/cassandra/cql3/restrictions/AbstractRestriction.java
new file mode 100644
index 0000000..4093780
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/restrictions/AbstractRestriction.java
@@ -0,0 +1,119 @@
+/*
+ * 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.cql3.restrictions;
+
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.statements.Bound;
+import org.apache.cassandra.db.composites.CompositesBuilder;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkBindValueSet;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
+
+/**
+ * Base class for <code>Restriction</code>s
+ */
+abstract class AbstractRestriction  implements Restriction
+{
+    @Override
+    public  boolean isOnToken()
+    {
+        return false;
+    }
+
+    @Override
+    public boolean isMultiColumn()
+    {
+        return false;
+    }
+
+    @Override
+    public boolean isSlice()
+    {
+        return false;
+    }
+
+    @Override
+    public boolean isEQ()
+    {
+        return false;
+    }
+
+    @Override
+    public boolean isIN()
+    {
+        return false;
+    }
+
+    @Override
+    public boolean isContains()
+    {
+        return false;
+    }
+
+    @Override
+    public boolean hasBound(Bound b)
+    {
+        return true;
+    }
+
+    @Override
+    public CompositesBuilder appendBoundTo(CFMetaData cfm, CompositesBuilder builder, Bound bound, QueryOptions options)
+    {
+        return appendTo(cfm, builder, options);
+    }
+
+    @Override
+    public boolean isInclusive(Bound b)
+    {
+        return true;
+    }
+
+    public boolean isNotReturningAnyRows(CFMetaData cfm, QueryOptions options)
+    {
+        return false;
+    }
+
+    protected static ByteBuffer validateIndexedValue(ColumnSpecification columnSpec,
+                                                     ByteBuffer value)
+                                                     throws InvalidRequestException
+    {
+        checkNotNull(value, "Unsupported null value for column %s", columnSpec.name);
+        checkBindValueSet(value, "Unsupported unset value for column %s", columnSpec.name);
+        checkFalse(value.remaining() > 0xFFFF, "Index expression values may not be larger than 64K");
+        return value;
+    }
+
+    /**
+     * Reverses the specified bound if the column type is a reversed one.
+     *
+     * @param columnDefinition the column definition
+     * @param bound the bound
+     * @return the bound reversed if the column type was a reversed one or the original bound
+     */
+    protected static Bound reverseBoundIfNeeded(ColumnDefinition columnDefinition, Bound bound)
+    {
+        return columnDefinition.isReversedType() ? bound.reverse() : bound;
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/ForwardingPrimaryKeyRestrictions.java b/src/java/org/apache/cassandra/cql3/restrictions/ForwardingPrimaryKeyRestrictions.java
new file mode 100644
index 0000000..76d0233
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/restrictions/ForwardingPrimaryKeyRestrictions.java
@@ -0,0 +1,192 @@
+/*
+ * 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.cql3.restrictions;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.statements.Bound;
+import org.apache.cassandra.db.IndexExpression;
+import org.apache.cassandra.db.composites.Composite;
+import org.apache.cassandra.db.composites.CompositesBuilder;
+import org.apache.cassandra.db.index.SecondaryIndexManager;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+/**
+ * A <code>PrimaryKeyRestrictions</code> which forwards all its method calls to another 
+ * <code>PrimaryKeyRestrictions</code>. Subclasses should override one or more methods to modify the behavior 
+ * of the backing <code>PrimaryKeyRestrictions</code> as desired per the decorator pattern. 
+ */
+abstract class ForwardingPrimaryKeyRestrictions implements PrimaryKeyRestrictions
+{
+    /**
+     * Returns the backing delegate instance that methods are forwarded to.
+     * @return the backing delegate instance that methods are forwarded to.
+     */
+    protected abstract PrimaryKeyRestrictions getDelegate();
+
+    @Override
+    public Iterable<Function> getFunctions()
+    {
+        return getDelegate().getFunctions();
+    }
+
+    @Override
+    public List<ColumnDefinition> getColumnDefs()
+    {
+        return getDelegate().getColumnDefs();
+    }
+
+    @Override
+    public ColumnDefinition getFirstColumn()
+    {
+        return getDelegate().getFirstColumn();
+    }
+
+    @Override
+    public ColumnDefinition getLastColumn()
+    {
+        return getDelegate().getLastColumn();
+    }
+
+    @Override
+    public PrimaryKeyRestrictions mergeWith(Restriction restriction) throws InvalidRequestException
+    {
+        return getDelegate().mergeWith(restriction);
+    }
+
+    @Override
+    public boolean hasSupportingIndex(SecondaryIndexManager secondaryIndexManager)
+    {
+        return getDelegate().hasSupportingIndex(secondaryIndexManager);
+    }
+
+    @Override
+    public List<ByteBuffer> values(CFMetaData cfm, QueryOptions options) throws InvalidRequestException
+    {
+        return getDelegate().values(cfm, options);
+    }
+
+    @Override
+    public CompositesBuilder appendTo(CFMetaData cfm, CompositesBuilder builder, QueryOptions options)
+    {
+        return getDelegate().appendTo(cfm, builder, options);
+    }
+
+    @Override
+    public List<Composite> valuesAsComposites(CFMetaData cfm, QueryOptions options) throws InvalidRequestException
+    {
+        return getDelegate().valuesAsComposites(cfm, options);
+    }
+
+    @Override
+    public List<ByteBuffer> bounds(CFMetaData cfm, Bound bound, QueryOptions options) throws InvalidRequestException
+    {
+        return getDelegate().bounds(cfm, bound, options);
+    }
+
+    @Override
+    public List<Composite> boundsAsComposites(CFMetaData cfm, Bound bound, QueryOptions options) throws InvalidRequestException
+    {
+        return getDelegate().boundsAsComposites(cfm, bound, options);
+    }
+
+    @Override
+    public CompositesBuilder appendBoundTo(CFMetaData cfm, CompositesBuilder builder, Bound bound, QueryOptions options)
+    {
+        return getDelegate().appendBoundTo(cfm, builder, bound, options);
+    }
+
+    @Override
+    public boolean isInclusive(Bound bound)
+    {
+        return getDelegate().isInclusive(bound.reverse());
+    }
+
+    @Override
+    public boolean isEmpty()
+    {
+        return getDelegate().isEmpty();
+    }
+
+    @Override
+    public int size()
+    {
+        return getDelegate().size();
+    }
+
+    @Override
+    public boolean isOnToken()
+    {
+        return getDelegate().isOnToken();
+    }
+
+    @Override
+    public boolean isSlice()
+    {
+        return getDelegate().isSlice();
+    }
+
+    @Override
+    public boolean isEQ()
+    {
+        return getDelegate().isEQ();
+    }
+
+    @Override
+    public boolean isIN()
+    {
+        return getDelegate().isIN();
+    }
+
+    @Override
+    public boolean isContains()
+    {
+        return getDelegate().isContains();
+    }
+
+    @Override
+    public boolean isMultiColumn()
+    {
+        return getDelegate().isMultiColumn();
+    }
+
+    @Override
+    public boolean hasBound(Bound b)
+    {
+        return getDelegate().hasBound(b);
+    }
+
+    @Override
+    public void addIndexExpressionTo(List<IndexExpression> expressions,
+                                     SecondaryIndexManager indexManager,
+                                     QueryOptions options) throws InvalidRequestException
+    {
+        getDelegate().addIndexExpressionTo(expressions, indexManager, options);
+    }
+
+    @Override
+    public boolean isNotReturningAnyRows(CFMetaData cfm, QueryOptions options)
+    {
+        return getDelegate().isNotReturningAnyRows(cfm, options);
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/MultiColumnRestriction.java b/src/java/org/apache/cassandra/cql3/restrictions/MultiColumnRestriction.java
new file mode 100644
index 0000000..44f25ec
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/restrictions/MultiColumnRestriction.java
@@ -0,0 +1,553 @@
+/*
+ * 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.cql3.restrictions;
+
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.cql3.Term.Terminal;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.statements.Bound;
+import org.apache.cassandra.db.IndexExpression;
+import org.apache.cassandra.db.composites.CompositesBuilder;
+import org.apache.cassandra.db.index.SecondaryIndex;
+import org.apache.cassandra.db.index.SecondaryIndexManager;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
+
+public abstract class MultiColumnRestriction extends AbstractRestriction
+{
+    /**
+     * The columns to which the restriction apply.
+     */
+    protected final List<ColumnDefinition> columnDefs;
+
+    public MultiColumnRestriction(List<ColumnDefinition> columnDefs)
+    {
+        this.columnDefs = columnDefs;
+    }
+
+    @Override
+    public boolean isMultiColumn()
+    {
+        return true;
+    }
+
+    @Override
+    public ColumnDefinition getFirstColumn()
+    {
+        return columnDefs.get(0);
+    }
+
+    @Override
+    public ColumnDefinition getLastColumn()
+    {
+        return columnDefs.get(columnDefs.size() - 1);
+    }
+
+    @Override
+    public List<ColumnDefinition> getColumnDefs()
+    {
+        return columnDefs;
+    }
+
+    @Override
+    public final Restriction mergeWith(Restriction otherRestriction) throws InvalidRequestException
+    {
+        // We want to allow query like: (b,c) > (?, ?) AND b < ?
+        if (!otherRestriction.isMultiColumn()
+                && ((SingleColumnRestriction) otherRestriction).canBeConvertedToMultiColumnRestriction())
+        {
+            return doMergeWith(((SingleColumnRestriction) otherRestriction).toMultiColumnRestriction());
+        }
+
+        return doMergeWith(otherRestriction);
+    }
+
+    protected abstract Restriction doMergeWith(Restriction otherRestriction) throws InvalidRequestException;
+
+    /**
+     * Returns the names of the columns that are specified within this <code>Restrictions</code> and the other one
+     * as a comma separated <code>String</code>.
+     *
+     * @param otherRestriction the other restrictions
+     * @return the names of the columns that are specified within this <code>Restrictions</code> and the other one
+     * as a comma separated <code>String</code>.
+     */
+    protected final String getColumnsInCommons(Restriction otherRestriction)
+    {
+        Set<ColumnDefinition> commons = new HashSet<>(getColumnDefs());
+        commons.retainAll(otherRestriction.getColumnDefs());
+        StringBuilder builder = new StringBuilder();
+        for (ColumnDefinition columnDefinition : commons)
+        {
+            if (builder.length() != 0)
+                builder.append(" ,");
+            builder.append(columnDefinition.name);
+        }
+        return builder.toString();
+    }
+
+    @Override
+    public final boolean hasSupportingIndex(SecondaryIndexManager indexManager)
+    {
+        for (ColumnDefinition columnDef : columnDefs)
+        {
+            SecondaryIndex index = indexManager.getIndexForColumn(columnDef.name.bytes);
+            if (index != null && isSupportedBy(index))
+                return true;
+        }
+        return false;
+    }
+
+    /**
+     * Check if this type of restriction is supported for by the specified index.
+     * @param index the Secondary index
+     *
+     * @return <code>true</code> this type of restriction is supported by the specified index,
+     * <code>false</code> otherwise.
+     */
+    protected abstract boolean isSupportedBy(SecondaryIndex index);
+
+    public static class EQ  extends MultiColumnRestriction
+    {
+        protected final Term value;
+
+        public EQ(List<ColumnDefinition> columnDefs, Term value)
+        {
+            super(columnDefs);
+            this.value = value;
+        }
+
+        @Override
+        public Iterable<Function> getFunctions()
+        {
+            return value.getFunctions();
+        }
+
+        @Override
+        public String toString()
+        {
+            return String.format("EQ(%s)", value);
+        }
+
+        @Override
+        public Restriction doMergeWith(Restriction otherRestriction) throws InvalidRequestException
+        {
+            throw invalidRequest("%s cannot be restricted by more than one relation if it includes an Equal",
+                                 getColumnsInCommons(otherRestriction));
+        }
+
+        @Override
+        protected boolean isSupportedBy(SecondaryIndex index)
+        {
+            return index.supportsOperator(Operator.EQ);
+        }
+
+        @Override
+        public CompositesBuilder appendTo(CFMetaData cfm, CompositesBuilder builder, QueryOptions options)
+        {
+            Tuples.Value t = ((Tuples.Value) value.bind(options));
+            List<ByteBuffer> values = t.getElements();
+            for (int i = 0, m = values.size(); i < m; i++)
+            {
+                builder.addElementToAll(values.get(i));
+                checkFalse(builder.containsNull(), "Invalid null value for column %s", columnDefs.get(i).name);
+            }
+            return builder;
+        }
+
+        @Override
+        public final void addIndexExpressionTo(List<IndexExpression> expressions,
+                                               SecondaryIndexManager indexManager,
+                                               QueryOptions options) throws InvalidRequestException
+        {
+            Tuples.Value t = ((Tuples.Value) value.bind(options));
+            List<ByteBuffer> values = t.getElements();
+
+            for (int i = 0, m = columnDefs.size(); i < m; i++)
+            {
+                ColumnDefinition columnDef = columnDefs.get(i);
+                ByteBuffer component = validateIndexedValue(columnDef, values.get(i));
+                expressions.add(new IndexExpression(columnDef.name.bytes, Operator.EQ, component));
+            }
+        }
+
+        @Override
+        public boolean isNotReturningAnyRows(CFMetaData cfm, QueryOptions options)
+        {
+            // Dense non-compound tables do not accept empty ByteBuffers. By consequence, we know that
+            // any query with an EQ restriction containing an empty value will not return any results.
+            return !cfm.comparator.isCompound()
+                    && !((Tuples.Value) value.bind(options)).getElements().get(0).hasRemaining();
+        }
+    }
+
+    public abstract static class IN extends MultiColumnRestriction
+    {
+        /**
+         * {@inheritDoc}
+         */
+        @Override
+        public CompositesBuilder appendTo(CFMetaData cfm, CompositesBuilder builder, QueryOptions options)
+        {
+            List<List<ByteBuffer>> splitInValues = filterValuesIfNeeded(cfm, splitValues(options));
+            builder.addAllElementsToAll(splitInValues);
+
+            if (builder.containsNull())
+                throw invalidRequest("Invalid null value in condition for columns: %s", ColumnDefinition.toIdentifiers(columnDefs));
+            return builder;
+        }
+
+        private List<List<ByteBuffer>> filterValuesIfNeeded(CFMetaData cfm, List<List<ByteBuffer>> splitInValues)
+        {
+            if (cfm.comparator.isCompound())
+                return splitInValues;
+
+            // Dense non-compound tables do not accept empty ByteBuffers. By consequence, we know that we can
+            // ignore any IN value which is an empty byte buffer an which otherwise will trigger an error.
+
+            // As some List implementations do not support remove, we copy the list to be on the safe side.
+            List<List<ByteBuffer>> filteredValues = new ArrayList<>(splitInValues.size());
+            for (List<ByteBuffer> values : splitInValues)
+            {
+                if (values.get(0).hasRemaining())
+                    filteredValues.add(values);
+            }
+            return filteredValues;
+        }
+
+        public IN(List<ColumnDefinition> columnDefs)
+        {
+            super(columnDefs);
+        }
+
+        @Override
+        public boolean isIN()
+        {
+            return true;
+        }
+
+        @Override
+        public Restriction doMergeWith(Restriction otherRestriction) throws InvalidRequestException
+        {
+            throw invalidRequest("%s cannot be restricted by more than one relation if it includes a IN",
+                                 getColumnsInCommons(otherRestriction));
+        }
+
+        @Override
+        protected boolean isSupportedBy(SecondaryIndex index)
+        {
+            return index.supportsOperator(Operator.IN);
+        }
+
+        @Override
+        public final void addIndexExpressionTo(List<IndexExpression> expressions,
+                                               SecondaryIndexManager indexManager,
+                                               QueryOptions options) throws InvalidRequestException
+        {
+            List<List<ByteBuffer>> splitInValues = splitValues(options);
+            checkTrue(splitInValues.size() == 1, "IN restrictions are not supported on indexed columns");
+            List<ByteBuffer> values = splitInValues.get(0);
+
+            for (int i = 0, m = columnDefs.size(); i < m; i++)
+            {
+                ColumnDefinition columnDef = columnDefs.get(i);
+                ByteBuffer component = validateIndexedValue(columnDef, values.get(i));
+                expressions.add(new IndexExpression(columnDef.name.bytes, Operator.EQ, component));
+            }
+        }
+
+        protected abstract List<List<ByteBuffer>> splitValues(QueryOptions options) throws InvalidRequestException;
+    }
+
+    /**
+     * An IN restriction that has a set of terms for in values.
+     * For example: "SELECT ... WHERE (a, b, c) IN ((1, 2, 3), (4, 5, 6))" or "WHERE (a, b, c) IN (?, ?)"
+     */
+    public static class InWithValues extends MultiColumnRestriction.IN
+    {
+        protected final List<Term> values;
+
+        public InWithValues(List<ColumnDefinition> columnDefs, List<Term> values)
+        {
+            super(columnDefs);
+            this.values = values;
+        }
+
+        @Override
+        public Iterable<Function> getFunctions()
+        {
+            return Terms.getFunctions(values);
+        }
+
+        @Override
+        public String toString()
+        {
+            return String.format("IN(%s)", values);
+        }
+
+        @Override
+        protected List<List<ByteBuffer>> splitValues(QueryOptions options) throws InvalidRequestException
+        {
+            List<List<ByteBuffer>> buffers = new ArrayList<>(values.size());
+            for (Term value : values)
+            {
+                Term.MultiItemTerminal term = (Term.MultiItemTerminal) value.bind(options);
+                buffers.add(term.getElements());
+            }
+            return buffers;
+        }
+    }
+
+    /**
+     * An IN restriction that uses a single marker for a set of IN values that are tuples.
+     * For example: "SELECT ... WHERE (a, b, c) IN ?"
+     */
+    public static class InWithMarker extends MultiColumnRestriction.IN
+    {
+        protected final AbstractMarker marker;
+
+        public InWithMarker(List<ColumnDefinition> columnDefs, AbstractMarker marker)
+        {
+            super(columnDefs);
+            this.marker = marker;
+        }
+
+        @Override
+        public Iterable<Function> getFunctions()
+        {
+            return Collections.emptySet();
+        }
+
+        @Override
+        public String toString()
+        {
+            return "IN ?";
+        }
+
+        @Override
+        protected List<List<ByteBuffer>> splitValues(QueryOptions options) throws InvalidRequestException
+        {
+            Tuples.InMarker inMarker = (Tuples.InMarker) marker;
+            Tuples.InValue inValue = inMarker.bind(options);
+            checkNotNull(inValue, "Invalid null value for IN restriction");
+            return inValue.getSplitValues();
+        }
+    }
+
+    public static class Slice extends MultiColumnRestriction
+    {
+        private final TermSlice slice;
+
+        public Slice(List<ColumnDefinition> columnDefs, Bound bound, boolean inclusive, Term term)
+        {
+            this(columnDefs, TermSlice.newInstance(bound, inclusive, term));
+        }
+
+        Slice(List<ColumnDefinition> columnDefs, TermSlice slice)
+        {
+            super(columnDefs);
+            this.slice = slice;
+        }
+
+        @Override
+        public boolean isSlice()
+        {
+            return true;
+        }
+
+        @Override
+        public CompositesBuilder appendTo(CFMetaData cfm, CompositesBuilder builder, QueryOptions options)
+        {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public CompositesBuilder appendBoundTo(CFMetaData cfm, CompositesBuilder builder, Bound bound, QueryOptions options)
+        {
+            boolean reversed = getFirstColumn().isReversedType();
+
+            EnumMap<Bound, List<ByteBuffer>> componentBounds = new EnumMap<Bound, List<ByteBuffer>>(Bound.class);
+            componentBounds.put(Bound.START, componentBounds(Bound.START, options));
+            componentBounds.put(Bound.END, componentBounds(Bound.END, options));
+
+            List<List<ByteBuffer>> toAdd = new ArrayList<>();
+            List<ByteBuffer> values = new ArrayList<>();
+
+            for (int i = 0, m = columnDefs.size(); i < m; i++)
+            {
+                ColumnDefinition column = columnDefs.get(i);
+                Bound b = reverseBoundIfNeeded(column, bound);
+
+                // For mixed order columns, we need to create additional slices when 2 columns are in reverse order
+                if (reversed != column.isReversedType())
+                {
+                    reversed = column.isReversedType();
+                    // As we are switching direction we need to add the current composite
+                    toAdd.add(values);
+
+                    // The new bound side has no value for this component.  just stop
+                    if (!hasComponent(b, i, componentBounds))
+                        continue;
+
+                    // The other side has still some components. We need to end the slice that we have just open.
+                    if (hasComponent(b.reverse(), i, componentBounds))
+                        toAdd.add(values);
+
+                    // We need to rebuild where we are in this bound side
+                    values = new ArrayList<ByteBuffer>();
+
+                    List<ByteBuffer> vals = componentBounds.get(b);
+
+                    int n = Math.min(i, vals.size());
+                    for (int j = 0; j < n; j++)
+                    {
+                        ByteBuffer v = checkNotNull(vals.get(j),
+                                                    "Invalid null value in condition for column %s",
+                                                    columnDefs.get(j).name);
+                        values.add(v);
+                    }
+                }
+
+                if (!hasComponent(b, i, componentBounds))
+                    continue;
+
+                ByteBuffer v = checkNotNull(componentBounds.get(b).get(i), "Invalid null value in condition for column %s", columnDefs.get(i).name);
+                values.add(v);
+            }
+            toAdd.add(values);
+
+            if (bound.isEnd())
+                Collections.reverse(toAdd);
+
+            return builder.addAllElementsToAll(toAdd);
+        }
+
+        @Override
+        protected boolean isSupportedBy(SecondaryIndex index)
+        {
+            return slice.isSupportedBy(index);
+        }
+
+        @Override
+        public boolean hasBound(Bound bound)
+        {
+            return slice.hasBound(bound);
+        }
+
+        @Override
+        public Iterable<Function> getFunctions()
+        {
+            return slice.getFunctions();
+        }
+
+        @Override
+        public boolean isInclusive(Bound bound)
+        {
+            return slice.isInclusive(bound);
+        }
+
+        @Override
+        public Restriction doMergeWith(Restriction otherRestriction) throws InvalidRequestException
+        {
+            checkTrue(otherRestriction.isSlice(),
+                      "Column \"%s\" cannot be restricted by both an equality and an inequality relation",
+                      getColumnsInCommons(otherRestriction));
+
+            if (!getFirstColumn().equals(otherRestriction.getFirstColumn()))
+            {
+                ColumnDefinition column = getFirstColumn().position() > otherRestriction.getFirstColumn().position()
+                        ? getFirstColumn() : otherRestriction.getFirstColumn();
+
+                throw invalidRequest("Column \"%s\" cannot be restricted by two inequalities not starting with the same column",
+                                     column.name);
+            }
+
+            checkFalse(hasBound(Bound.START) && otherRestriction.hasBound(Bound.START),
+                       "More than one restriction was found for the start bound on %s",
+                       getColumnsInCommons(otherRestriction));
+            checkFalse(hasBound(Bound.END) && otherRestriction.hasBound(Bound.END),
+                       "More than one restriction was found for the end bound on %s",
+                       getColumnsInCommons(otherRestriction));
+
+            Slice otherSlice = (Slice) otherRestriction;
+            List<ColumnDefinition> newColumnDefs = columnDefs.size() >= otherSlice.columnDefs.size() ?  columnDefs : otherSlice.columnDefs;
+
+            return new Slice(newColumnDefs, slice.merge(otherSlice.slice));
+        }
+
+        @Override
+        public final void addIndexExpressionTo(List<IndexExpression> expressions,
+                                               SecondaryIndexManager indexManager,
+                                               QueryOptions options) throws InvalidRequestException
+        {
+            throw invalidRequest("Multi-column slice restrictions cannot be used for filtering.");
+        }
+
+        @Override
+        public String toString()
+        {
+            return "SLICE" + slice;
+        }
+
+        /**
+         * Similar to bounds(), but returns one ByteBuffer per-component in the bound instead of a single
+         * ByteBuffer to represent the entire bound.
+         * @param b the bound type
+         * @param options the query options
+         * @return one ByteBuffer per-component in the bound
+         * @throws InvalidRequestException if the components cannot be retrieved
+         */
+        private List<ByteBuffer> componentBounds(Bound b, QueryOptions options) throws InvalidRequestException
+        {
+            if (!slice.hasBound(b))
+                return Collections.emptyList();
+
+            Terminal terminal = slice.bound(b).bind(options);
+
+            if (terminal instanceof Tuples.Value)
+            {
+                return ((Tuples.Value) terminal).getElements();
+            }
+
+            return Collections.singletonList(terminal.get(options.getProtocolVersion()));
+        }
+
+        @Override
+        public boolean isNotReturningAnyRows(CFMetaData cfm, QueryOptions options)
+        {
+            // Dense non-compound tables do not accept empty ByteBuffers. By consequence, we know that
+            // any query with a slice restriction with an empty value for the END bound will not return any results.
+            return !cfm.comparator.isCompound()
+                    && hasBound(Bound.END)
+                    && !componentBounds(Bound.END, options).get(0).hasRemaining();
+        }
+
+        private boolean hasComponent(Bound b, int index, EnumMap<Bound, List<ByteBuffer>> componentBounds)
+        {
+            return componentBounds.get(b).size() > index;
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/PrimaryKeyRestrictionSet.java b/src/java/org/apache/cassandra/cql3/restrictions/PrimaryKeyRestrictionSet.java
new file mode 100644
index 0000000..2549bdf
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/restrictions/PrimaryKeyRestrictionSet.java
@@ -0,0 +1,417 @@
+/*
+ * 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.cql3.restrictions;
+
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.statements.Bound;
+import org.apache.cassandra.db.IndexExpression;
+import org.apache.cassandra.db.composites.*;
+import org.apache.cassandra.db.composites.Composite.EOC;
+import org.apache.cassandra.db.index.SecondaryIndexManager;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+/**
+ * A set of single column restrictions on a primary key part (partition key or clustering key).
+ */
+final class PrimaryKeyRestrictionSet extends AbstractPrimaryKeyRestrictions implements Iterable<Restriction>
+{
+    /**
+     * The restrictions.
+     */
+    private final RestrictionSet restrictions;
+
+    /**
+     * <code>true</code> if the restrictions are corresponding to an EQ, <code>false</code> otherwise.
+     */
+    private boolean eq;
+
+    /**
+     * <code>true</code> if the restrictions are corresponding to an IN, <code>false</code> otherwise.
+     */
+    private boolean in;
+
+    /**
+     * <code>true</code> if the restrictions are corresponding to a Slice, <code>false</code> otherwise.
+     */
+    private boolean slice;
+
+    /**
+     * <code>true</code> if the restrictions are corresponding to a Contains, <code>false</code> otherwise.
+     */
+    private boolean contains;
+
+    public PrimaryKeyRestrictionSet(CType ctype)
+    {
+        super(ctype);
+        this.restrictions = new RestrictionSet();
+        this.eq = true;
+    }
+
+    private PrimaryKeyRestrictionSet(PrimaryKeyRestrictionSet primaryKeyRestrictions,
+                                     Restriction restriction) throws InvalidRequestException
+    {
+        super(primaryKeyRestrictions.ctype);
+        this.restrictions = primaryKeyRestrictions.restrictions.addRestriction(restriction);
+
+        if (restriction.isSlice() || primaryKeyRestrictions.isSlice())
+            this.slice = true;
+        else if (restriction.isContains() || primaryKeyRestrictions.isContains())
+            this.contains = true;
+        else if (restriction.isIN() || primaryKeyRestrictions.isIN())
+            this.in = true;
+        else
+            this.eq = true;
+    }
+
+    @Override
+    public boolean isSlice()
+    {
+        return slice;
+    }
+
+    @Override
+    public boolean isEQ()
+    {
+        return eq;
+    }
+
+    @Override
+    public boolean isIN()
+    {
+        return in;
+    }
+
+    @Override
+    public boolean isOnToken()
+    {
+        return false;
+    }
+
+    @Override
+    public boolean isContains()
+    {
+        return contains;
+    }
+
+    @Override
+    public boolean isMultiColumn()
+    {
+        return false;
+    }
+
+    @Override
+    public Iterable<Function> getFunctions()
+    {
+        return restrictions.getFunctions();
+    }
+
+    @Override
+    public PrimaryKeyRestrictions mergeWith(Restriction restriction) throws InvalidRequestException
+    {
+        if (restriction.isOnToken())
+        {
+            if (isEmpty())
+                return (PrimaryKeyRestrictions) restriction;
+
+            return new TokenFilter(this, (TokenRestriction) restriction);
+        }
+
+        return new PrimaryKeyRestrictionSet(this, restriction);
+    }
+
+    @Override
+    public List<Composite> valuesAsComposites(CFMetaData cfm, QueryOptions options) throws InvalidRequestException
+    {
+        return filterAndSort(appendTo(cfm, new CompositesBuilder(ctype), options).build());
+    }
+
+    @Override
+    public CompositesBuilder appendTo(CFMetaData cfm, CompositesBuilder builder, QueryOptions options)
+    {
+        for (Restriction r : restrictions)
+        {
+            r.appendTo(cfm, builder, options);
+            if (builder.hasMissingElements())
+                break;
+        }
+        return builder;
+    }
+
+    @Override
+    public CompositesBuilder appendBoundTo(CFMetaData cfm, CompositesBuilder builder, Bound bound, QueryOptions options)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public List<Composite> boundsAsComposites(CFMetaData cfm, Bound bound, QueryOptions options) throws InvalidRequestException
+    {
+        CompositesBuilder builder = new CompositesBuilder(ctype);
+        // The end-of-component of composite doesn't depend on whether the
+        // component type is reversed or not (i.e. the ReversedType is applied
+        // to the component comparator but not to the end-of-component itself),
+        // it only depends on whether the slice is reversed
+        int keyPosition = 0;
+        for (Restriction r : restrictions)
+        {
+            ColumnDefinition def = r.getFirstColumn();
+
+            if (keyPosition != def.position() || r.isContains())
+                break;
+
+            if (r.isSlice())
+            {
+                r.appendBoundTo(cfm, builder, bound, options);
+
+                // Since CASSANDRA-7281, the composites might not end with the same components and it is possible
+                // that one of the composites is an empty one. Unfortunatly, AbstractCType will always sort
+                // Composites.EMPTY before all the other components due to its EOC, even if it is not the desired
+                // behaviour in some cases. To avoid that problem the code will use normal composites for the empty
+                // ones until the composites are properly sorted. They will then be replaced by Composites.EMPTY as
+                // it is what is expected by the intra-node serialization.
+                // It is clearly a hack but it does not make a lot of sense to refactor 2.2 for that as the problem is
+                // already solved in 3.0.
+                List<Composite> composites = filterAndSort(setEocs(r, bound, builder.build()));
+                return Lists.transform(composites, new com.google.common.base.Function<Composite, Composite>()
+                {
+                    @Override
+                    public Composite apply(Composite composite)
+                    {
+                        return composite.isEmpty() ? Composites.EMPTY: composite;
+                    }
+                });
+            }
+
+            r.appendBoundTo(cfm, builder, bound, options);
+
+            if (builder.hasMissingElements())
+                return Collections.emptyList();
+
+            keyPosition = r.getLastColumn().position() + 1;
+        }
+        // Means no relation at all or everything was an equal
+        // Note: if the builder is "full", there is no need to use the end-of-component bit. For columns selection,
+        // it would be harmless to do it. However, we use this method got the partition key too. And when a query
+        // with 2ndary index is done, and with the the partition provided with an EQ, we'll end up here, and in that
+        // case using the eoc would be bad, since for the random partitioner we have no guarantee that
+        // prefix.end() will sort after prefix (see #5240).
+        EOC eoc = !builder.hasRemaining() ? EOC.NONE : (bound.isEnd() ? EOC.END : EOC.START);
+        return filterAndSort(builder.buildWithEOC(eoc));
+    }
+
+    /**
+     * Removes duplicates and sort the specified composites.
+     *
+     * @param composites the composites to filter and sort
+     * @return the composites sorted and without duplicates
+     */
+    private List<Composite> filterAndSort(List<Composite> composites)
+    {
+        if (composites.size() <= 1)
+            return composites;
+
+        TreeSet<Composite> set = new TreeSet<Composite>(ctype);
+        set.addAll(composites);
+
+        return new ArrayList<>(set);
+    }
+
+    /**
+     * Sets EOCs for the composites returned by the specified slice restriction for the given bound.
+     *
+     * @param r the slice restriction
+     * @param bound the bound
+     * @param composites the composites
+     * @return the composites with their EOCs properly set
+     */
+    private List<Composite> setEocs(Restriction r, Bound bound, List<Composite> composites)
+    {
+        List<Composite> list = new ArrayList<>(composites.size());
+
+        // The first column of the slice might not be the first clustering column (e.g. clustering_0 = ? AND (clustering_1, clustering_2) >= (?, ?)
+        int offset = r.getFirstColumn().position();
+
+        for (int i = 0, m = composites.size(); i < m; i++)
+        {
+            Composite composite = composites.get(i);
+
+            // Handle the no bound case
+            if (composite.size() == offset)
+            {
+                list.add(composite.withEOC(bound.isEnd() ? EOC.END : EOC.START));
+                continue;
+            }
+
+            // In the case of mixed order columns, we will have some extra slices where the columns change directions.
+            // For example: if we have clustering_0 DESC and clustering_1 ASC a slice like (clustering_0, clustering_1) > (1, 2)
+            // will produce 2 slices: [EMPTY, 1.START] and [1.2.END, 1.END]
+            // So, the END bound will return 2 composite with the same values 1
+            if (composite.size() <= r.getLastColumn().position() && i < m - 1 && composite.equals(composites.get(i + 1)))
+            {
+                list.add(composite.withEOC(EOC.START));
+                list.add(composites.get(i++).withEOC(EOC.END));
+                continue;
+            }
+
+            // Handle the normal bounds
+            ColumnDefinition column = r.getColumnDefs().get(composite.size() - 1 - offset);
+            Bound b = reverseBoundIfNeeded(column, bound);
+
+            Composite.EOC eoc = eocFor(r, bound, b);
+            list.add(composite.withEOC(eoc));
+        }
+
+        return list;
+    }
+
+    @Override
+    public List<ByteBuffer> values(CFMetaData cfm, QueryOptions options) throws InvalidRequestException
+    {
+        return Composites.toByteBuffers(valuesAsComposites(cfm, options));
+    }
+
+    @Override
+    public List<ByteBuffer> bounds(CFMetaData cfm, Bound b, QueryOptions options) throws InvalidRequestException
+    {
+        return Composites.toByteBuffers(boundsAsComposites(cfm, b, options));
+    }
+
+    private static Composite.EOC eocFor(Restriction r, Bound eocBound, Bound inclusiveBound)
+    {
+        if (eocBound.isStart())
+            return r.isInclusive(inclusiveBound) ? Composite.EOC.NONE : Composite.EOC.END;
+
+        return r.isInclusive(inclusiveBound) ? Composite.EOC.END : Composite.EOC.START;
+    }
+
+    @Override
+    public boolean hasBound(Bound b)
+    {
+        if (isEmpty())
+            return false;
+        return restrictions.lastRestriction().hasBound(b);
+    }
+
+    @Override
+    public boolean isInclusive(Bound b)
+    {
+        if (isEmpty())
+            return false;
+        return restrictions.lastRestriction().isInclusive(b);
+    }
+
+    @Override
+    public boolean hasSupportingIndex(SecondaryIndexManager indexManager)
+    {
+        return restrictions.hasSupportingIndex(indexManager);
+    }
+
+    @Override
+    public void addIndexExpressionTo(List<IndexExpression> expressions,
+                                     SecondaryIndexManager indexManager,
+                                     QueryOptions options) throws InvalidRequestException
+    {
+        Boolean clusteringColumns = null;
+        int position = 0;
+
+        for (Restriction restriction : restrictions)
+        {
+            ColumnDefinition columnDef = restriction.getFirstColumn();
+
+            // PrimaryKeyRestrictionSet contains only one kind of column, either partition key or clustering columns.
+            // Therefore we only need to check the column kind once. All the other columns will be of the same kind.
+            if (clusteringColumns == null)
+                clusteringColumns = columnDef.isClusteringColumn() ? Boolean.TRUE : Boolean.FALSE;
+
+            // We ignore all the clustering columns that can be handled by slices.
+            if (!clusteringColumns || handleInFilter(restriction, position) || restriction.hasSupportingIndex(indexManager))
+            {
+                restriction.addIndexExpressionTo(expressions, indexManager, options);
+                continue;
+            }
+
+            if (!restriction.isSlice())
+                position = restriction.getLastColumn().position() + 1;
+        }
+    }
+
+    @Override
+    public List<ColumnDefinition> getColumnDefs()
+    {
+        return restrictions.getColumnDefs();
+    }
+
+    @Override
+    public ColumnDefinition getFirstColumn()
+    {
+        return restrictions.firstColumn();
+    }
+
+    @Override
+    public ColumnDefinition getLastColumn()
+    {
+        return restrictions.lastColumn();
+    }
+
+    public final boolean needsFiltering()
+    {
+        // Backported from ClusteringColumnRestrictions from CASSANDRA-11310 for 3.6
+        // As that suggests, this should only be called on clustering column
+        // and not partition key restrictions.
+        int position = 0;
+        for (Restriction restriction : restrictions)
+        {
+            if (handleInFilter(restriction, position))
+                return true;
+
+            if (!restriction.isSlice())
+                position = restriction.getLastColumn().position() + 1;
+        }
+
+        return false;
+    }
+
+    @Override
+    public boolean isNotReturningAnyRows(CFMetaData cfm, QueryOptions options)
+    {
+        for (Restriction restriction : restrictions)
+        {
+            if (restriction.isNotReturningAnyRows(cfm, options))
+                return true;
+        }
+        return false;
+    }
+
+    private boolean handleInFilter(Restriction restriction, int index)
+    {
+        return restriction.isContains() || index != restriction.getFirstColumn().position();
+    }
+
+    public Iterator<Restriction> iterator()
+    {
+        return restrictions.iterator();
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/PrimaryKeyRestrictions.java b/src/java/org/apache/cassandra/cql3/restrictions/PrimaryKeyRestrictions.java
new file mode 100644
index 0000000..5e5e3f5
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/restrictions/PrimaryKeyRestrictions.java
@@ -0,0 +1,45 @@
+/*
+ * 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.cql3.restrictions;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.statements.Bound;
+import org.apache.cassandra.db.composites.Composite;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+/**
+ * A set of restrictions on a primary key part (partition key or clustering key).
+ *
+ */
+interface PrimaryKeyRestrictions extends Restriction, Restrictions
+{
+    @Override
+    public PrimaryKeyRestrictions mergeWith(Restriction restriction) throws InvalidRequestException;
+
+    public List<ByteBuffer> values(CFMetaData cfm, QueryOptions options) throws InvalidRequestException;
+
+    public List<Composite> valuesAsComposites(CFMetaData cfm, QueryOptions options) throws InvalidRequestException;
+
+    public List<ByteBuffer> bounds(CFMetaData cfm, Bound b, QueryOptions options) throws InvalidRequestException;
+
+    public List<Composite> boundsAsComposites(CFMetaData cfm, Bound bound, QueryOptions options) throws InvalidRequestException;
+}
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/Restriction.java b/src/java/org/apache/cassandra/cql3/restrictions/Restriction.java
new file mode 100644
index 0000000..9df100a
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/restrictions/Restriction.java
@@ -0,0 +1,150 @@
+/*
+ * 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.cql3.restrictions;
+
+import java.util.List;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.statements.Bound;
+import org.apache.cassandra.db.IndexExpression;
+import org.apache.cassandra.db.composites.CompositesBuilder;
+import org.apache.cassandra.db.index.SecondaryIndexManager;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+/**
+ * A restriction/clause on a column.
+ * The goal of this class being to group all conditions for a column in a SELECT.
+ *
+ * <p>Implementation of this class must be immutable. See {@link #mergeWith(Restriction)} for more explanation.</p>
+ */
+public interface Restriction
+{
+    public boolean isOnToken();
+    public boolean isSlice();
+    public boolean isEQ();
+    public boolean isIN();
+    public boolean isContains();
+    public boolean isMultiColumn();
+
+    /**
+     * Returns the definition of the first column.
+     * @return the definition of the first column.
+     */
+    public ColumnDefinition getFirstColumn();
+
+    /**
+     * Returns the definition of the last column.
+     * @return the definition of the last column.
+     */
+    public ColumnDefinition getLastColumn();
+
+    /**
+     * Returns the column definitions in position order.
+     * @return the column definitions in position order.
+     */
+    public List<ColumnDefinition> getColumnDefs();
+
+    /**
+     * Return an Iterable over all of the functions (both native and user-defined) used by any component
+     * of the restriction
+     * @return functions all functions found (may contain duplicates)
+     */
+    public Iterable<Function> getFunctions();
+
+    /**
+     * Checks if the specified bound is set or not.
+     * @param b the bound type
+     * @return <code>true</code> if the specified bound is set, <code>false</code> otherwise
+     */
+    public boolean hasBound(Bound b);
+
+    /**
+     * Checks if the specified bound is inclusive or not.
+     * @param b the bound type
+     * @return <code>true</code> if the specified bound is inclusive, <code>false</code> otherwise
+     */
+    public boolean isInclusive(Bound b);
+
+    /**
+     * Merges this restriction with the specified one.
+     *
+     * <p>Restriction are immutable. Therefore merging two restrictions result in a new one.
+     * The reason behind this choice is that it allow a great flexibility in the way the merging can done while
+     * preventing any side effect.</p>
+     *
+     * @param otherRestriction the restriction to merge into this one
+     * @return the restriction resulting of the merge
+     * @throws InvalidRequestException if the restrictions cannot be merged
+     */
+    public Restriction mergeWith(Restriction otherRestriction) throws InvalidRequestException;
+
+    /**
+     * Check if the restriction is on indexed columns.
+     *
+     * @param indexManager the index manager
+     * @return <code>true</code> if the restriction is on indexed columns, <code>false</code>
+     */
+    public boolean hasSupportingIndex(SecondaryIndexManager indexManager);
+
+    /**
+     * Adds to the specified list the <code>IndexExpression</code>s corresponding to this <code>Restriction</code>.
+     *
+     * @param expressions the list to add the <code>IndexExpression</code>s to
+     * @param indexManager the secondary index manager
+     * @param options the query options
+     * @throws InvalidRequestException if this <code>Restriction</code> cannot be converted into 
+     * <code>IndexExpression</code>s
+     */
+    public void addIndexExpressionTo(List<IndexExpression> expressions,
+                                     SecondaryIndexManager indexManager,
+                                     QueryOptions options)
+                                     throws InvalidRequestException;
+
+    /**
+     * Appends the values of this <code>Restriction</code> to the specified builder.
+     *
+     * @param cfm the table metadata
+     * @param builder the <code>CompositesBuilder</code> to append to.
+     * @param options the query options
+     * @return the <code>CompositesBuilder</code>
+     */
+    public CompositesBuilder appendTo(CFMetaData cfm, CompositesBuilder builder, QueryOptions options);
+
+    /**
+     * Appends the values of the <code>Restriction</code> for the specified bound to the specified builder.
+     *
+     * @param cfm the table metadata
+     * @param builder the <code>CompositesBuilder</code> to append to.
+     * @param bound the bound
+     * @param options the query options
+     * @return the <code>CompositesBuilder</code>
+     */
+    public CompositesBuilder appendBoundTo(CFMetaData cfm, CompositesBuilder builder, Bound bound, QueryOptions options);
+
+    /**
+     * Checks if this restriction will prevent the query to return any rows.
+     *
+     * @param cfm the table metadata
+     * @param options the query options
+     * @return {@code true} if this restriction will prevent the query to return any rows, {@false} otherwise
+     */
+    public boolean isNotReturningAnyRows(CFMetaData cfm, QueryOptions options);
+}
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/RestrictionSet.java b/src/java/org/apache/cassandra/cql3/restrictions/RestrictionSet.java
new file mode 100644
index 0000000..676ed13
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/restrictions/RestrictionSet.java
@@ -0,0 +1,260 @@
+/*
+ * 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.cql3.restrictions;
+
+import java.util.*;
+
+import com.google.common.collect.Iterables;
+
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.restrictions.SingleColumnRestriction.Contains;
+import org.apache.cassandra.db.IndexExpression;
+import org.apache.cassandra.db.index.SecondaryIndexManager;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+/**
+ * Sets of column restrictions.
+ *
+ * <p>This class is immutable in order to be use within {@link PrimaryKeyRestrictionSet} which as
+ * an implementation of {@link Restriction} need to be immutable.
+ */
+final class RestrictionSet implements Restrictions, Iterable<Restriction>
+{
+    /**
+     * The comparator used to sort the <code>Restriction</code>s.
+     */
+    private static final Comparator<ColumnDefinition> COLUMN_DEFINITION_COMPARATOR = new Comparator<ColumnDefinition>()
+    {
+        @Override
+        public int compare(ColumnDefinition column, ColumnDefinition otherColumn)
+        {
+            int value = Integer.compare(column.position(), otherColumn.position());
+            return value != 0 ? value : column.name.bytes.compareTo(otherColumn.name.bytes);
+        }
+    };
+
+    /**
+     * The restrictions per column.
+     */
+    protected final TreeMap<ColumnDefinition, Restriction> restrictions;
+
+    public RestrictionSet()
+    {
+        this(new TreeMap<ColumnDefinition, Restriction>(COLUMN_DEFINITION_COMPARATOR));
+    }
+
+    private RestrictionSet(TreeMap<ColumnDefinition, Restriction> restrictions)
+    {
+        this.restrictions = restrictions;
+    }
+
+    @Override
+    public final void addIndexExpressionTo(List<IndexExpression> expressions,
+                                           SecondaryIndexManager indexManager,
+                                           QueryOptions options) throws InvalidRequestException
+    {
+        for (Restriction restriction : restrictions.values())
+            restriction.addIndexExpressionTo(expressions, indexManager, options);
+    }
+
+    @Override
+    public final List<ColumnDefinition> getColumnDefs()
+    {
+        return new ArrayList<>(restrictions.keySet());
+    }
+
+    @Override
+    public Iterable<Function> getFunctions()
+    {
+        com.google.common.base.Function<Restriction, Iterable<Function>> transform =
+            new com.google.common.base.Function<Restriction, Iterable<Function>>()
+        {
+            public Iterable<Function> apply(Restriction restriction)
+            {
+                return restriction.getFunctions();
+            }
+        };
+
+        return Iterables.concat(Iterables.transform(restrictions.values(), transform));
+    }
+
+    @Override
+    public final boolean isEmpty()
+    {
+        return getColumnDefs().isEmpty();
+    }
+
+    @Override
+    public final int size()
+    {
+        return getColumnDefs().size();
+    }
+
+    /**
+     * Adds the specified restriction to this set of restrictions.
+     *
+     * @param restriction the restriction to add
+     * @return the new set of restrictions
+     * @throws InvalidRequestException if the new restriction cannot be added
+     */
+    public RestrictionSet addRestriction(Restriction restriction) throws InvalidRequestException
+    {
+        // RestrictionSet is immutable so we need to clone the restrictions map.
+        TreeMap<ColumnDefinition, Restriction> newRestrictions = new TreeMap<>(this.restrictions);
+        return new RestrictionSet(mergeRestrictions(newRestrictions, restriction));
+    }
+
+    private TreeMap<ColumnDefinition, Restriction> mergeRestrictions(TreeMap<ColumnDefinition, Restriction> restrictions,
+                                                                     Restriction restriction)
+                                                                     throws InvalidRequestException
+    {
+        Collection<ColumnDefinition> columnDefs = restriction.getColumnDefs();
+        Set<Restriction> existingRestrictions = getRestrictions(columnDefs);
+
+        if (existingRestrictions.isEmpty())
+        {
+            for (ColumnDefinition columnDef : columnDefs)
+                restrictions.put(columnDef, restriction);
+        }
+        else
+        {
+            for (Restriction existing : existingRestrictions)
+            {
+                Restriction newRestriction = mergeRestrictions(existing, restriction);
+
+                for (ColumnDefinition columnDef : columnDefs)
+                    restrictions.put(columnDef, newRestriction);
+            }
+        }
+
+        return restrictions;
+    }
+
+    /**
+     * Returns all the restrictions applied to the specified columns.
+     *
+     * @param columnDefs the column definitions
+     * @return all the restrictions applied to the specified columns
+     */
+    private Set<Restriction> getRestrictions(Collection<ColumnDefinition> columnDefs)
+    {
+        Set<Restriction> set = new HashSet<>();
+        for (ColumnDefinition columnDef : columnDefs)
+        {
+            Restriction existing = restrictions.get(columnDef);
+            if (existing != null)
+                set.add(existing);
+        }
+        return set;
+    }
+
+    @Override
+    public final boolean hasSupportingIndex(SecondaryIndexManager indexManager)
+    {
+        for (Restriction restriction : restrictions.values())
+        {
+            if (restriction.hasSupportingIndex(indexManager))
+                return true;
+        }
+        return false;
+    }
+
+    /**
+     * Returns the column after the specified one.
+     *
+     * @param columnDef the column for which the next one need to be found
+     * @return the column after the specified one.
+     */
+    ColumnDefinition nextColumn(ColumnDefinition columnDef)
+    {
+        return restrictions.tailMap(columnDef, false).firstKey();
+    }
+
+    /**
+     * Returns the definition of the first column.
+     *
+     * @return the definition of the first column.
+     */
+    ColumnDefinition firstColumn()
+    {
+        return isEmpty() ? null : this.restrictions.firstKey();
+    }
+
+    /**
+     * Returns the definition of the last column.
+     *
+     * @return the definition of the last column.
+     */
+    ColumnDefinition lastColumn()
+    {
+        return isEmpty() ? null : this.restrictions.lastKey();
+    }
+
+    /**
+     * Returns the last restriction.
+     *
+     * @return the last restriction.
+     */
+    Restriction lastRestriction()
+    {
+        return isEmpty() ? null : this.restrictions.lastEntry().getValue();
+    }
+
+    /**
+     * Merges the two specified restrictions.
+     *
+     * @param restriction the first restriction
+     * @param otherRestriction the second restriction
+     * @return the merged restriction
+     * @throws InvalidRequestException if the two restrictions cannot be merged
+     */
+    private static Restriction mergeRestrictions(Restriction restriction,
+                                                 Restriction otherRestriction) throws InvalidRequestException
+    {
+        return restriction == null ? otherRestriction
+                                   : restriction.mergeWith(otherRestriction);
+    }
+
+    /**
+     * Checks if the restrictions contains multiple contains, contains key, or map[key] = value.
+     *
+     * @return <code>true</code> if the restrictions contains multiple contains, contains key, or ,
+     * map[key] = value; <code>false</code> otherwise
+     */
+    public final boolean hasMultipleContains()
+    {
+        int numberOfContains = 0;
+        for (Restriction restriction : restrictions.values())
+        {
+            if (restriction.isContains())
+            {
+                Contains contains = (Contains) restriction;
+                numberOfContains += (contains.numberOfValues() + contains.numberOfKeys() + contains.numberOfEntries());
+            }
+        }
+        return numberOfContains > 1;
+    }
+
+    @Override
+    public Iterator<Restriction> iterator()
+    {
+        return new LinkedHashSet<>(restrictions.values()).iterator();
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/Restrictions.java b/src/java/org/apache/cassandra/cql3/restrictions/Restrictions.java
new file mode 100644
index 0000000..ab81bf7
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/restrictions/Restrictions.java
@@ -0,0 +1,83 @@
+/*
+ * 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.cql3.restrictions;
+
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.db.IndexExpression;
+import org.apache.cassandra.db.index.SecondaryIndexManager;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+/**
+ * Sets of restrictions
+ */
+interface Restrictions
+{
+    /**
+     * Returns the column definitions in position order.
+     * @return the column definitions in position order.
+     */
+    public Collection<ColumnDefinition> getColumnDefs();
+
+    /**
+     * Return an Iterable over all of the functions (both native and user-defined) used by any component
+     * of the restrictions
+     * @return functions all functions found (may contain duplicates)
+     */
+    public Iterable<Function> getFunctions();
+
+    /**
+     * Check if the restriction is on indexed columns.
+     *
+     * @param indexManager the index manager
+     * @return <code>true</code> if the restriction is on indexed columns, <code>false</code>
+     */
+    public boolean hasSupportingIndex(SecondaryIndexManager indexManager);
+
+    /**
+     * Adds to the specified list the <code>IndexExpression</code>s corresponding to this <code>Restriction</code>.
+     *
+     * @param expressions the list to add the <code>IndexExpression</code>s to
+     * @param indexManager the secondary index manager
+     * @param options the query options
+     * @throws InvalidRequestException if this <code>Restriction</code> cannot be converted into
+     * <code>IndexExpression</code>s
+     */
+    public void addIndexExpressionTo(List<IndexExpression> expressions,
+                                     SecondaryIndexManager indexManager,
+                                     QueryOptions options)
+                                     throws InvalidRequestException;
+
+    /**
+     * Checks if this <code>PrimaryKeyRestrictionSet</code> is empty or not.
+     *
+     * @return <code>true</code> if this <code>PrimaryKeyRestrictionSet</code> is empty, <code>false</code> otherwise.
+     */
+    boolean isEmpty();
+
+    /**
+     * Returns the number of columns that have a restriction.
+     *
+     * @return the number of columns that have a restriction.
+     */
+    public int size();
+}
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/ReversedPrimaryKeyRestrictions.java b/src/java/org/apache/cassandra/cql3/restrictions/ReversedPrimaryKeyRestrictions.java
new file mode 100644
index 0000000..9107acd
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/restrictions/ReversedPrimaryKeyRestrictions.java
@@ -0,0 +1,78 @@
+/*
+ * 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.cql3.restrictions;
+
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.statements.Bound;
+import org.apache.cassandra.db.composites.Composite;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+/**
+ * <code>PrimaryKeyRestrictions</code> decorator that reverse the slices.
+ */
+final class ReversedPrimaryKeyRestrictions extends ForwardingPrimaryKeyRestrictions
+{
+    /**
+     * The decorated restrictions.
+     */
+    private PrimaryKeyRestrictions restrictions;
+
+    public ReversedPrimaryKeyRestrictions(PrimaryKeyRestrictions restrictions)
+    {
+        this.restrictions = restrictions;
+    }
+
+    @Override
+    public PrimaryKeyRestrictions mergeWith(Restriction restriction) throws InvalidRequestException
+    {
+        return new ReversedPrimaryKeyRestrictions(this.restrictions.mergeWith(restriction));
+    }
+
+    @Override
+    public List<ByteBuffer> bounds(CFMetaData cfm, Bound bound, QueryOptions options) throws InvalidRequestException
+    {
+        List<ByteBuffer> buffers = restrictions.bounds(cfm, bound.reverse(), options);
+        Collections.reverse(buffers);
+        return buffers;
+    }
+
+    @Override
+    public List<Composite> boundsAsComposites(CFMetaData cfm, Bound bound, QueryOptions options) throws InvalidRequestException
+    {
+        List<Composite> composites = restrictions.boundsAsComposites(cfm, bound.reverse(), options);
+        Collections.reverse(composites);
+        return composites;
+    }
+
+    @Override
+    public boolean isInclusive(Bound bound)
+    {
+        return this.restrictions.isInclusive(bound.reverse());
+    }
+
+    @Override
+    protected PrimaryKeyRestrictions getDelegate()
+    {
+        return this.restrictions;
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/SingleColumnRestriction.java b/src/java/org/apache/cassandra/cql3/restrictions/SingleColumnRestriction.java
new file mode 100644
index 0000000..1f4960b
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/restrictions/SingleColumnRestriction.java
@@ -0,0 +1,669 @@
+/*
+ * 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.cql3.restrictions;
+
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import com.google.common.collect.Iterables;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.cql3.Term.Terminal;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.statements.Bound;
+import org.apache.cassandra.db.IndexExpression;
+import org.apache.cassandra.db.composites.CompositesBuilder;
+import org.apache.cassandra.db.index.SecondaryIndex;
+import org.apache.cassandra.db.index.SecondaryIndexManager;
+import org.apache.cassandra.db.marshal.CompositeType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkBindValueSet;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
+
+public abstract class SingleColumnRestriction extends AbstractRestriction
+{
+    /**
+     * The definition of the column to which apply the restriction.
+     */
+    protected final ColumnDefinition columnDef;
+
+    public SingleColumnRestriction(ColumnDefinition columnDef)
+    {
+        this.columnDef = columnDef;
+    }
+
+    @Override
+    public List<ColumnDefinition> getColumnDefs()
+    {
+        return Collections.singletonList(columnDef);
+    }
+
+    @Override
+    public ColumnDefinition getFirstColumn()
+    {
+        return columnDef;
+    }
+
+    @Override
+    public ColumnDefinition getLastColumn()
+    {
+        return columnDef;
+    }
+
+    @Override
+    public boolean hasSupportingIndex(SecondaryIndexManager indexManager)
+    {
+        SecondaryIndex index = indexManager.getIndexForColumn(columnDef.name.bytes);
+        return index != null && isSupportedBy(index);
+    }
+
+    @Override
+    public final Restriction mergeWith(Restriction otherRestriction) throws InvalidRequestException
+    {
+        // We want to allow query like: b > ? AND (b,c) < (?, ?)
+        if (otherRestriction.isMultiColumn() && canBeConvertedToMultiColumnRestriction())
+        {
+            return toMultiColumnRestriction().mergeWith(otherRestriction);
+        }
+
+        return doMergeWith(otherRestriction);
+    }
+
+    protected abstract Restriction doMergeWith(Restriction otherRestriction) throws InvalidRequestException;
+
+    /**
+     * Converts this <code>SingleColumnRestriction</code> into a {@link MultiColumnRestriction}
+     *
+     * @return the <code>MultiColumnRestriction</code> corresponding to this
+     */
+    abstract MultiColumnRestriction toMultiColumnRestriction();
+
+    /**
+     * Checks if this <code>Restriction</code> can be converted into a {@link MultiColumnRestriction}
+     *
+     * @return <code>true</code> if this <code>Restriction</code> can be converted into a
+     * {@link MultiColumnRestriction}, <code>false</code> otherwise.
+     */
+    boolean canBeConvertedToMultiColumnRestriction()
+    {
+        return true;
+    }
+
+    /**
+     * Check if this type of restriction is supported by the specified index.
+     *
+     * @param index the Secondary index
+     * @return <code>true</code> this type of restriction is supported by the specified index,
+     * <code>false</code> otherwise.
+     */
+    protected abstract boolean isSupportedBy(SecondaryIndex index);
+
+    public static final class EQ extends SingleColumnRestriction
+    {
+        private final Term value;
+
+        public EQ(ColumnDefinition columnDef, Term value)
+        {
+            super(columnDef);
+            this.value = value;
+        }
+
+        @Override
+        public Iterable<Function> getFunctions()
+        {
+            return value.getFunctions();
+        }
+
+        @Override
+        public boolean isEQ()
+        {
+            return true;
+        }
+
+        @Override
+        MultiColumnRestriction toMultiColumnRestriction()
+        {
+            return new MultiColumnRestriction.EQ(Collections.singletonList(columnDef), value);
+        }
+
+        @Override
+        public void addIndexExpressionTo(List<IndexExpression> expressions,
+                                         SecondaryIndexManager indexManager,
+                                         QueryOptions options) throws InvalidRequestException
+        {
+            ByteBuffer buffer = validateIndexedValue(columnDef, value.bindAndGet(options));
+            expressions.add(new IndexExpression(columnDef.name.bytes, Operator.EQ, buffer));
+        }
+
+        @Override
+        public CompositesBuilder appendTo(CFMetaData cfm, CompositesBuilder builder, QueryOptions options)
+        {
+            builder.addElementToAll(value.bindAndGet(options));
+            checkFalse(builder.containsNull(), "Invalid null value in condition for column %s", columnDef.name);
+            checkFalse(builder.containsUnset(), "Invalid unset value for column %s", columnDef.name);
+            return builder;
+        }
+
+        @Override
+        public String toString()
+        {
+            return String.format("EQ(%s)", value);
+        }
+
+        @Override
+        public Restriction doMergeWith(Restriction otherRestriction) throws InvalidRequestException
+        {
+            throw invalidRequest("%s cannot be restricted by more than one relation if it includes an Equal", columnDef.name);
+        }
+
+        @Override
+        protected boolean isSupportedBy(SecondaryIndex index)
+        {
+            return index.supportsOperator(Operator.EQ);
+        }
+
+        @Override
+        public boolean isNotReturningAnyRows(CFMetaData cfm, QueryOptions options)
+        {
+            assert columnDef.isClusteringColumn();
+
+            // Dense non-compound tables do not accept empty ByteBuffers. By consequence, we know that
+            // any query with an EQ restriction containing an empty value will not return any results.
+            return !cfm.comparator.isCompound() && !value.bindAndGet(options).hasRemaining();
+        }
+    }
+
+    public static abstract class IN extends SingleColumnRestriction
+    {
+        public IN(ColumnDefinition columnDef)
+        {
+            super(columnDef);
+        }
+
+        @Override
+        public final boolean isIN()
+        {
+            return true;
+        }
+
+        @Override
+        public final Restriction doMergeWith(Restriction otherRestriction) throws InvalidRequestException
+        {
+            throw invalidRequest("%s cannot be restricted by more than one relation if it includes a IN", columnDef.name);
+        }
+
+        @Override
+        public CompositesBuilder appendTo(CFMetaData cfm, CompositesBuilder builder, QueryOptions options)
+        {
+            List<ByteBuffer> values = filterValuesIfNeeded(cfm, getValues(options));
+
+            builder.addEachElementToAll(values);
+            checkFalse(builder.containsNull(), "Invalid null value in condition for column %s", columnDef.name);
+            checkFalse(builder.containsUnset(), "Invalid unset value for column %s", columnDef.name);
+            return builder;
+        }
+
+        private List<ByteBuffer> filterValuesIfNeeded(CFMetaData cfm, List<ByteBuffer> values)
+        {
+            if (!columnDef.isClusteringColumn() || cfm.comparator.isCompound())
+                return values;
+
+            // Dense non-compound tables do not accept empty ByteBuffers. By consequence, we know that we can
+            // ignore any IN value which is an empty byte buffer an which otherwise will trigger an error.
+
+            // As some List implementations do not support remove, we copy the list to be on the safe side.
+            List<ByteBuffer> filteredValues = new ArrayList<>(values.size());
+            for (ByteBuffer value : values)
+            {
+                if (value.hasRemaining())
+                    filteredValues.add(value);
+            }
+            return filteredValues;
+        }
+
+        @Override
+        public void addIndexExpressionTo(List<IndexExpression> expressions,
+                                         SecondaryIndexManager indexManager,
+                                         QueryOptions options) throws InvalidRequestException
+        {
+            List<ByteBuffer> values = getValues(options);
+            checkTrue(values.size() == 1, "IN restrictions are not supported on indexed columns");
+
+            ByteBuffer value = validateIndexedValue(columnDef, values.get(0));
+            expressions.add(new IndexExpression(columnDef.name.bytes, Operator.EQ, value));
+        }
+
+        @Override
+        protected final boolean isSupportedBy(SecondaryIndex index)
+        {
+            return index.supportsOperator(Operator.IN);
+        }
+
+        protected abstract List<ByteBuffer> getValues(QueryOptions options) throws InvalidRequestException;
+    }
+
+    public static class InWithValues extends IN
+    {
+        protected final List<Term> values;
+
+        public InWithValues(ColumnDefinition columnDef, List<Term> values)
+        {
+            super(columnDef);
+            this.values = values;
+        }
+
+        @Override
+        MultiColumnRestriction toMultiColumnRestriction()
+        {
+            return new MultiColumnRestriction.InWithValues(Collections.singletonList(columnDef), values);
+        }
+
+        @Override
+        public Iterable<Function> getFunctions()
+        {
+            return Terms.getFunctions(values);
+        }
+
+        @Override
+        protected List<ByteBuffer> getValues(QueryOptions options) throws InvalidRequestException
+        {
+            List<ByteBuffer> buffers = new ArrayList<>(values.size());
+            for (Term value : values)
+                buffers.add(value.bindAndGet(options));
+            return buffers;
+        }
+
+        @Override
+        public String toString()
+        {
+            return String.format("IN(%s)", values);
+        }
+    }
+
+    public static class InWithMarker extends IN
+    {
+        protected final AbstractMarker marker;
+
+        public InWithMarker(ColumnDefinition columnDef, AbstractMarker marker)
+        {
+            super(columnDef);
+            this.marker = marker;
+        }
+
+        @Override
+        public Iterable<Function> getFunctions()
+        {
+            return Collections.emptySet();
+        }
+
+        @Override
+        MultiColumnRestriction toMultiColumnRestriction()
+        {
+            return new MultiColumnRestriction.InWithMarker(Collections.singletonList(columnDef), marker);
+        }
+
+        @Override
+        protected List<ByteBuffer> getValues(QueryOptions options) throws InvalidRequestException
+        {
+            Terminal term = marker.bind(options);
+            checkNotNull(term, "Invalid null value for column %s", columnDef.name);
+            checkFalse(term == Constants.UNSET_VALUE, "Invalid unset value for column %s", columnDef.name);
+            Term.MultiItemTerminal lval = (Term.MultiItemTerminal) term;
+            return lval.getElements();
+        }
+
+        @Override
+        public String toString()
+        {
+            return "IN ?";
+        }
+    }
+
+    public static final class Slice extends SingleColumnRestriction
+    {
+        private final TermSlice slice;
+
+        public Slice(ColumnDefinition columnDef, Bound bound, boolean inclusive, Term term)
+        {
+            super(columnDef);
+            slice = TermSlice.newInstance(bound, inclusive, term);
+        }
+
+        @Override
+        public Iterable<Function> getFunctions()
+        {
+            return slice.getFunctions();
+        }
+
+        @Override
+        MultiColumnRestriction toMultiColumnRestriction()
+        {
+            return new MultiColumnRestriction.Slice(Collections.singletonList(columnDef), slice);
+        }
+
+        @Override
+        public boolean isSlice()
+        {
+            return true;
+        }
+
+        @Override
+        public CompositesBuilder appendTo(CFMetaData cfm, CompositesBuilder builder, QueryOptions options)
+        {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public boolean hasBound(Bound b)
+        {
+            return slice.hasBound(b);
+        }
+
+        @Override
+        public CompositesBuilder appendBoundTo(CFMetaData cfm, CompositesBuilder builder, Bound bound, QueryOptions options)
+        {
+            Bound b = reverseBoundIfNeeded(getFirstColumn(), bound);
+
+            if (!hasBound(b))
+                return builder;
+
+            ByteBuffer value = slice.bound(b).bindAndGet(options);
+            checkBindValueSet(value, "Invalid unset value for column %s", columnDef.name);
+            return builder.addElementToAll(value);
+
+        }
+
+        @Override
+        public boolean isInclusive(Bound b)
+        {
+            return slice.isInclusive(b);
+        }
+
+        @Override
+        public Restriction doMergeWith(Restriction otherRestriction) throws InvalidRequestException
+        {
+            checkTrue(otherRestriction.isSlice(),
+                      "Column \"%s\" cannot be restricted by both an equality and an inequality relation",
+                      columnDef.name);
+
+            SingleColumnRestriction.Slice otherSlice = (SingleColumnRestriction.Slice) otherRestriction;
+
+            checkFalse(hasBound(Bound.START) && otherSlice.hasBound(Bound.START),
+                       "More than one restriction was found for the start bound on %s", columnDef.name);
+
+            checkFalse(hasBound(Bound.END) && otherSlice.hasBound(Bound.END),
+                       "More than one restriction was found for the end bound on %s", columnDef.name);
+
+            return new Slice(columnDef,  slice.merge(otherSlice.slice));
+        }
+
+        @Override
+        public void addIndexExpressionTo(List<IndexExpression> expressions,
+                                         SecondaryIndexManager indexManager,
+                                         QueryOptions options) throws InvalidRequestException
+        {
+            for (Bound b : Bound.values())
+            {
+                if (hasBound(b))
+                {
+                    ByteBuffer value = validateIndexedValue(columnDef, slice.bound(b).bindAndGet(options));
+                    Operator op = slice.getIndexOperator(b);
+                    // If the underlying comparator for name is reversed, we need to reverse the IndexOperator: user operation
+                    // always refer to the "forward" sorting even if the clustering order is reversed, but the 2ndary code does
+                    // use the underlying comparator as is.
+                    op = columnDef.isReversedType() ? op.reverse() : op;
+                    expressions.add(new IndexExpression(columnDef.name.bytes, op, value));
+                }
+            }
+        }
+
+        @Override
+        protected boolean isSupportedBy(SecondaryIndex index)
+        {
+            return slice.isSupportedBy(index);
+        }
+
+        @Override
+        public String toString()
+        {
+            return String.format("SLICE%s", slice);
+        }
+
+        @Override
+        public boolean isNotReturningAnyRows(CFMetaData cfm, QueryOptions options)
+        {
+            assert columnDef.isClusteringColumn();
+
+            // Dense non-compound tables do not accept empty ByteBuffers. By consequence, we know that
+            // any query with a slice restriction with an empty value for the END bound will not return any results.
+            return !cfm.comparator.isCompound()
+                    && hasBound(Bound.END)
+                    && !slice.bound(Bound.END).bindAndGet(options).hasRemaining();
+        }
+
+        private Slice(ColumnDefinition columnDef, TermSlice slice)
+        {
+            super(columnDef);
+            this.slice = slice;
+        }
+    }
+
+    // This holds CONTAINS, CONTAINS_KEY, and map[key] = value restrictions because we might want to have any combination of them.
+    public static final class Contains extends SingleColumnRestriction
+    {
+        private List<Term> values = new ArrayList<>(); // for CONTAINS
+        private List<Term> keys = new ArrayList<>(); // for CONTAINS_KEY
+        private List<Term> entryKeys = new ArrayList<>(); // for map[key] = value
+        private List<Term> entryValues = new ArrayList<>(); // for map[key] = value
+
+        public Contains(ColumnDefinition columnDef, Term t, boolean isKey)
+        {
+            super(columnDef);
+            if (isKey)
+                keys.add(t);
+            else
+                values.add(t);
+        }
+
+        public Contains(ColumnDefinition columnDef, Term mapKey, Term mapValue)
+        {
+            super(columnDef);
+            entryKeys.add(mapKey);
+            entryValues.add(mapValue);
+        }
+
+        @Override
+        MultiColumnRestriction toMultiColumnRestriction()
+        {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        boolean canBeConvertedToMultiColumnRestriction()
+        {
+            return false;
+        }
+
+        @Override
+        public CompositesBuilder appendTo(CFMetaData cfm, CompositesBuilder builder, QueryOptions options)
+        {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public boolean isContains()
+        {
+            return true;
+        }
+
+        @Override
+        public Restriction doMergeWith(Restriction otherRestriction) throws InvalidRequestException
+        {
+            checkTrue(otherRestriction.isContains(),
+                      "Collection column %s can only be restricted by CONTAINS, CONTAINS KEY, or map-entry equality",
+                      columnDef.name);
+
+            SingleColumnRestriction.Contains newContains = new Contains(columnDef);
+
+            copyKeysAndValues(this, newContains);
+            copyKeysAndValues((Contains) otherRestriction, newContains);
+
+            return newContains;
+        }
+
+        @Override
+        public void addIndexExpressionTo(List<IndexExpression> expressions,
+                                         SecondaryIndexManager indexManager,
+                                         QueryOptions options)
+                                         throws InvalidRequestException
+        {
+            addExpressionsFor(expressions, bindAndGet(values, options), Operator.CONTAINS);
+            addExpressionsFor(expressions, bindAndGet(keys, options), Operator.CONTAINS_KEY);
+            addExpressionsFor(expressions, entries(options), Operator.EQ);
+        }
+
+        private void addExpressionsFor(List<IndexExpression> target, List<ByteBuffer> values,
+                                       Operator op) throws InvalidRequestException
+        {
+            for (ByteBuffer value : values)
+            {
+                validateIndexedValue(columnDef, value);
+                target.add(new IndexExpression(columnDef.name.bytes, op, value));
+            }
+        }
+
+        @Override
+        protected boolean isSupportedBy(SecondaryIndex index)
+        {
+            boolean supported = false;
+
+            if (numberOfValues() > 0)
+                supported |= index.supportsOperator(Operator.CONTAINS);
+
+            if (numberOfKeys() > 0)
+                supported |= index.supportsOperator(Operator.CONTAINS_KEY);
+
+            if (numberOfEntries() > 0)
+                supported |= index.supportsOperator(Operator.EQ);
+
+            return supported;
+        }
+
+        public int numberOfValues()
+        {
+            return values.size();
+        }
+
+        public int numberOfKeys()
+        {
+            return keys.size();
+        }
+
+        public int numberOfEntries()
+        {
+            return entryKeys.size();
+        }
+
+        @Override
+        public Iterable<Function> getFunctions()
+        {
+            return Iterables.concat(Terms.getFunctions(values),
+                                    Terms.getFunctions(keys),
+                                    Terms.getFunctions(entryKeys),
+                                    Terms.getFunctions(entryValues));
+        }
+
+        @Override
+        public String toString()
+        {
+            return String.format("CONTAINS(values=%s, keys=%s, entryKeys=%s, entryValues=%s)", values, keys, entryKeys, entryValues);
+        }
+
+        @Override
+        public boolean hasBound(Bound b)
+        {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public CompositesBuilder appendBoundTo(CFMetaData cfm, CompositesBuilder builder, Bound bound, QueryOptions options)
+        {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public boolean isInclusive(Bound b)
+        {
+            throw new UnsupportedOperationException();
+        }
+
+        private List<ByteBuffer> entries(QueryOptions options) throws InvalidRequestException
+        {
+            List<ByteBuffer> entryBuffers = new ArrayList<>(entryKeys.size());
+            List<ByteBuffer> keyBuffers = bindAndGet(entryKeys, options);
+            List<ByteBuffer> valueBuffers = bindAndGet(entryValues, options);
+            for (int i = 0; i < entryKeys.size(); i++)
+            {
+                if (valueBuffers.get(i) == null)
+                    throw new InvalidRequestException("Unsupported null value for map-entry equality");
+                entryBuffers.add(CompositeType.build(keyBuffers.get(i), valueBuffers.get(i)));
+            }
+            return entryBuffers;
+        }
+
+        /**
+         * Binds the query options to the specified terms and returns the resulting values.
+         *
+         * @param terms the terms
+         * @param options the query options
+         * @return the value resulting from binding the query options to the specified terms
+         * @throws InvalidRequestException if a problem occurs while binding the query options
+         */
+        private static List<ByteBuffer> bindAndGet(List<Term> terms, QueryOptions options) throws InvalidRequestException
+        {
+            List<ByteBuffer> buffers = new ArrayList<>(terms.size());
+            for (Term value : terms)
+                buffers.add(value.bindAndGet(options));
+            return buffers;
+        }
+
+        /**
+         * Copies the keys and value from the first <code>Contains</code> to the second one.
+         *
+         * @param from the <code>Contains</code> to copy from
+         * @param to the <code>Contains</code> to copy to
+         */
+        private static void copyKeysAndValues(Contains from, Contains to)
+        {
+            to.values.addAll(from.values);
+            to.keys.addAll(from.keys);
+            to.entryKeys.addAll(from.entryKeys);
+            to.entryValues.addAll(from.entryValues);
+        }
+
+        private Contains(ColumnDefinition columnDef)
+        {
+            super(columnDef);
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java b/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
new file mode 100644
index 0000000..2c396c4
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
@@ -0,0 +1,673 @@
+/*
+ * 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.cql3.restrictions;
+
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.Iterables;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.statements.Bound;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.composites.Composite;
+import org.apache.cassandra.db.index.SecondaryIndexManager;
+import org.apache.cassandra.dht.*;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.config.ColumnDefinition.toIdentifiers;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
+import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
+
+/**
+ * The restrictions corresponding to the relations specified on the where-clause of CQL query.
+ */
+public final class StatementRestrictions
+{
+    public static final String REQUIRES_ALLOW_FILTERING_MESSAGE =
+            "Cannot execute this query as it might involve data filtering and " +
+            "thus may have unpredictable performance. If you want to execute " +
+            "this query despite the performance unpredictability, use ALLOW FILTERING";
+
+    /**
+     * The Column Family meta data
+     */
+    public final CFMetaData cfm;
+
+    /**
+     * Restrictions on partitioning columns
+     */
+    private PrimaryKeyRestrictions partitionKeyRestrictions;
+
+    /**
+     * Restrictions on clustering columns
+     */
+    private PrimaryKeyRestrictions clusteringColumnsRestrictions;
+
+    /**
+     * Restriction on non-primary key columns (i.e. secondary index restrictions)
+     */
+    private RestrictionSet nonPrimaryKeyRestrictions;
+
+    /**
+     * <code>true</code> if nonPrimaryKeyRestrictions contains restriction on a regular column,
+     * <code>false</code> otherwise.
+     */
+    private boolean hasRegularColumnsRestriction = false;
+
+    /**
+     * The restrictions used to build the index expressions
+     */
+    private final List<Restrictions> indexRestrictions = new ArrayList<>();
+
+    /**
+     * <code>true</code> if the secondary index need to be queried, <code>false</code> otherwise
+     */
+    private boolean usesSecondaryIndexing;
+
+    /**
+     * Specify if the query will return a range of partition keys.
+     */
+    private boolean isKeyRange;
+
+    /**
+     * Creates a new empty <code>StatementRestrictions</code>.
+     *
+     * @param cfm the column family meta data
+     * @return a new empty <code>StatementRestrictions</code>.
+     */
+    public static StatementRestrictions empty(CFMetaData cfm)
+    {
+        return new StatementRestrictions(cfm);
+    }
+
+    private StatementRestrictions(CFMetaData cfm)
+    {
+        this.cfm = cfm;
+        this.partitionKeyRestrictions = new PrimaryKeyRestrictionSet(cfm.getKeyValidatorAsCType());
+        this.clusteringColumnsRestrictions = new PrimaryKeyRestrictionSet(cfm.comparator);
+        this.nonPrimaryKeyRestrictions = new RestrictionSet();
+    }
+
+    public StatementRestrictions(CFMetaData cfm,
+                                 List<Relation> whereClause,
+                                 VariableSpecifications boundNames,
+                                 boolean selectsOnlyStaticColumns,
+                                 boolean selectACollection,
+                                 boolean allowFiltering)
+    {
+        this.cfm = cfm;
+        this.partitionKeyRestrictions = new PrimaryKeyRestrictionSet(cfm.getKeyValidatorAsCType());
+        this.clusteringColumnsRestrictions = new PrimaryKeyRestrictionSet(cfm.comparator);
+        this.nonPrimaryKeyRestrictions = new RestrictionSet();
+
+        /*
+         * WHERE clause. For a given entity, rules are: - EQ relation conflicts with anything else (including a 2nd EQ)
+         * - Can't have more than one LT(E) relation (resp. GT(E) relation) - IN relation are restricted to row keys
+         * (for now) and conflicts with anything else (we could allow two IN for the same entity but that doesn't seem
+         * very useful) - The value_alias cannot be restricted in any way (we don't support wide rows with indexed value
+         * in CQL so far)
+         */
+        for (Relation relation : whereClause)
+            addRestriction(relation.toRestriction(cfm, boundNames));
+
+        SecondaryIndexManager secondaryIndexManager = Keyspace.open(cfm.ksName).getColumnFamilyStore(cfm.cfName).indexManager;
+        boolean hasQueriableClusteringColumnIndex = clusteringColumnsRestrictions.hasSupportingIndex(secondaryIndexManager);
+        boolean hasQueriableIndex = hasQueriableClusteringColumnIndex
+                || partitionKeyRestrictions.hasSupportingIndex(secondaryIndexManager)
+                || nonPrimaryKeyRestrictions.hasSupportingIndex(secondaryIndexManager);
+
+        // At this point, the select statement if fully constructed, but we still have a few things to validate
+        processPartitionKeyRestrictions(hasQueriableIndex);
+
+        // Some but not all of the partition key columns have been specified;
+        // hence we need turn these restrictions into index expressions.
+        if (usesSecondaryIndexing)
+            indexRestrictions.add(partitionKeyRestrictions);
+
+        checkFalse(selectsOnlyStaticColumns && hasClusteringColumnsRestriction(),
+                   "Cannot restrict clustering columns when selecting only static columns");
+
+        processClusteringColumnsRestrictions(hasQueriableIndex, selectACollection);
+
+        // Covers indexes on the first clustering column (among others).
+        if (isKeyRange && hasQueriableClusteringColumnIndex)
+            usesSecondaryIndexing = true;
+
+        usesSecondaryIndexing = usesSecondaryIndexing || clusteringColumnsRestrictions.isContains();
+
+        if (usesSecondaryIndexing)
+            indexRestrictions.add(clusteringColumnsRestrictions);
+
+        // Even if usesSecondaryIndexing is false at this point, we'll still have to use one if
+        // there is restrictions not covered by the PK.
+        if (!nonPrimaryKeyRestrictions.isEmpty())
+        {
+            if (!hasQueriableIndex)
+            {
+                // Filtering for non-index query is only supported for thrift static CFs
+                if (cfm.comparator.isDense() ||  cfm.comparator.isCompound())
+                    throw invalidRequest("Predicates on non-primary-key columns (%s) are not yet supported for non secondary index queries",
+                                         Joiner.on(", ").join(toIdentifiers(nonPrimaryKeyRestrictions.getColumnDefs())));
+
+                if (!allowFiltering)
+                    throw invalidRequest(REQUIRES_ALLOW_FILTERING_MESSAGE);
+            }
+            usesSecondaryIndexing = true;
+            indexRestrictions.add(nonPrimaryKeyRestrictions);
+        }
+
+        if (usesSecondaryIndexing)
+            validateSecondaryIndexSelections(selectsOnlyStaticColumns);
+    }
+
+    private void addRestriction(Restriction restriction) throws InvalidRequestException
+    {
+        if (restriction.isMultiColumn())
+            clusteringColumnsRestrictions = clusteringColumnsRestrictions.mergeWith(restriction);
+        else if (restriction.isOnToken())
+            partitionKeyRestrictions = partitionKeyRestrictions.mergeWith(restriction);
+        else
+            addSingleColumnRestriction((SingleColumnRestriction) restriction);
+    }
+
+    public Iterable<Function> getFunctions()
+    {
+        return Iterables.concat(partitionKeyRestrictions.getFunctions(),
+                                clusteringColumnsRestrictions.getFunctions(),
+                                nonPrimaryKeyRestrictions.getFunctions());
+    }
+
+    private void addSingleColumnRestriction(SingleColumnRestriction restriction) throws InvalidRequestException
+    {
+        ColumnDefinition def = restriction.columnDef;
+        if (def.isPartitionKey())
+            partitionKeyRestrictions = partitionKeyRestrictions.mergeWith(restriction);
+        else if (def.isClusteringColumn())
+            clusteringColumnsRestrictions = clusteringColumnsRestrictions.mergeWith(restriction);
+        else
+        {
+            if (restriction.columnDef.kind == ColumnDefinition.Kind.REGULAR)
+            {
+                hasRegularColumnsRestriction = true;
+            }
+            nonPrimaryKeyRestrictions = nonPrimaryKeyRestrictions.addRestriction(restriction);
+        }
+    }
+
+    /**
+     * Checks if the restrictions on the partition key is an IN restriction.
+     *
+     * @return <code>true</code> the restrictions on the partition key is an IN restriction, <code>false</code>
+     * otherwise.
+     */
+    public boolean keyIsInRelation()
+    {
+        return partitionKeyRestrictions.isIN();
+    }
+
+    /**
+     * Checks if the query request a range of partition keys.
+     *
+     * @return <code>true</code> if the query request a range of partition keys, <code>false</code> otherwise.
+     */
+    public boolean isKeyRange()
+    {
+        return this.isKeyRange;
+    }
+
+    public boolean hasRegularColumnsRestriction()
+    {
+        return hasRegularColumnsRestriction;
+    }
+
+    /**
+     * Checks if the secondary index need to be queried.
+     *
+     * @return <code>true</code> if the secondary index need to be queried, <code>false</code> otherwise.
+     */
+    public boolean usesSecondaryIndexing()
+    {
+        return this.usesSecondaryIndexing;
+    }
+
+    private void processPartitionKeyRestrictions(boolean hasQueriableIndex) throws InvalidRequestException
+    {
+        // If there is a queriable index, no special condition are required on the other restrictions.
+        // But we still need to know 2 things:
+        // - If we don't have a queriable index, is the query ok
+        // - Is it queriable without 2ndary index, which is always more efficient
+        // If a component of the partition key is restricted by a relation, all preceding
+        // components must have a EQ. Only the last partition key component can be in IN relation.
+        if (partitionKeyRestrictions.isOnToken())
+            isKeyRange = true;
+
+        if (hasPartitionKeyUnrestrictedComponents())
+        {
+            if (!partitionKeyRestrictions.isEmpty())
+            {
+                if (!hasQueriableIndex)
+                    throw invalidRequest("Partition key parts: %s must be restricted as other parts are",
+                                         Joiner.on(", ").join(getPartitionKeyUnrestrictedComponents()));
+            }
+
+            isKeyRange = true;
+            usesSecondaryIndexing = hasQueriableIndex;
+        }
+    }
+
+    /**
+     * Checks if the partition key has some unrestricted components.
+     * @return <code>true</code> if the partition key has some unrestricted components, <code>false</code> otherwise.
+     */
+    private boolean hasPartitionKeyUnrestrictedComponents()
+    {
+        return partitionKeyRestrictions.size() <  cfm.partitionKeyColumns().size();
+    }
+
+    public boolean hasPartitionKeyRestrictions()
+    {
+        return !partitionKeyRestrictions.isEmpty();
+    }
+
+    /**
+     * Checks if the restrictions contain any non-primary key restrictions
+     * @return <code>true</code> if the restrictions contain any non-primary key restrictions, <code>false</code> otherwise.
+     */
+    public boolean hasNonPrimaryKeyRestrictions()
+    {
+        return !nonPrimaryKeyRestrictions.isEmpty();
+    }
+
+    /**
+     * Returns the partition key components that are not restricted.
+     * @return the partition key components that are not restricted.
+     */
+    private List<ColumnIdentifier> getPartitionKeyUnrestrictedComponents()
+    {
+        List<ColumnDefinition> list = new ArrayList<>(cfm.partitionKeyColumns());
+        list.removeAll(partitionKeyRestrictions.getColumnDefs());
+        return ColumnDefinition.toIdentifiers(list);
+    }
+
+    /**
+     * Processes the clustering column restrictions.
+     *
+     * @param hasQueriableIndex <code>true</code> if some of the queried data are indexed, <code>false</code> otherwise
+     * @param selectACollection <code>true</code> if the query should return a collection column
+     * @throws InvalidRequestException if the request is invalid
+     */
+    private void processClusteringColumnsRestrictions(boolean hasQueriableIndex,
+                                                      boolean selectACollection) throws InvalidRequestException
+    {
+        validateClusteringRestrictions(hasQueriableIndex);
+
+        checkFalse(clusteringColumnsRestrictions.isIN() && selectACollection,
+                   "Cannot restrict clustering columns by IN relations when a collection is selected by the query");
+        checkFalse(clusteringColumnsRestrictions.isContains() && !hasQueriableIndex,
+                   "Cannot restrict clustering columns by a CONTAINS relation without a secondary index");
+
+        if (hasClusteringColumnsRestriction() && clusteringRestrictionsNeedFiltering())
+        {
+            if (hasQueriableIndex)
+            {
+                usesSecondaryIndexing = true;
+                return;
+            }
+
+            List<ColumnDefinition> clusteringColumns = cfm.clusteringColumns();
+            List<ColumnDefinition> restrictedColumns = new LinkedList<>(clusteringColumnsRestrictions.getColumnDefs());
+
+            for (int i = 0, m = restrictedColumns.size(); i < m; i++)
+            {
+                ColumnDefinition clusteringColumn = clusteringColumns.get(i);
+                ColumnDefinition restrictedColumn = restrictedColumns.get(i);
+
+                if (!clusteringColumn.equals(restrictedColumn))
+                {
+                    throw invalidRequest(
+                              "PRIMARY KEY column \"%s\" cannot be restricted as preceding column \"%s\" is not restricted",
+                              restrictedColumn.name,
+                              clusteringColumn.name);
+                }
+            }
+        }
+    }
+
+    /**
+     * Validates whether or not restrictions are allowed for execution when secondary index is not used.
+     */
+    public final void validateClusteringRestrictions(boolean hasQueriableIndex)
+    {
+        assert clusteringColumnsRestrictions instanceof PrimaryKeyRestrictionSet;
+
+        // If there's a queriable index, filtering will take care of clustering restrictions
+        if (hasQueriableIndex)
+            return;
+
+        Iterator<Restriction> iter = ((PrimaryKeyRestrictionSet)clusteringColumnsRestrictions).iterator();
+        Restriction previousRestriction = null;
+
+        while (iter.hasNext())
+        {
+            Restriction restriction = iter.next();
+
+            if (previousRestriction != null)
+            {
+                ColumnDefinition lastRestrictionStart = previousRestriction.getFirstColumn();
+                ColumnDefinition newRestrictionStart = restriction.getFirstColumn();
+
+                if (previousRestriction.isSlice() && newRestrictionStart.position() > lastRestrictionStart.position())
+                    throw invalidRequest("Clustering column \"%s\" cannot be restricted (preceding column \"%s\" is restricted by a non-EQ relation)",
+                                         newRestrictionStart.name,
+                                         lastRestrictionStart.name);
+            }
+            previousRestriction = restriction;
+        }
+    }
+
+    public final boolean clusteringRestrictionsNeedFiltering()
+    {
+        assert clusteringColumnsRestrictions instanceof PrimaryKeyRestrictionSet;
+        return ((PrimaryKeyRestrictionSet) clusteringColumnsRestrictions).needsFiltering();
+    }
+
+    public List<IndexExpression> getIndexExpressions(SecondaryIndexManager indexManager,
+                                                     QueryOptions options) throws InvalidRequestException
+    {
+        if (!usesSecondaryIndexing || indexRestrictions.isEmpty())
+            return Collections.emptyList();
+
+        List<IndexExpression> expressions = new ArrayList<>();
+        for (Restrictions restrictions : indexRestrictions)
+            restrictions.addIndexExpressionTo(expressions, indexManager, options);
+
+        return expressions;
+    }
+
+    /**
+     * Returns the partition keys for which the data is requested.
+     *
+     * @param options the query options
+     * @return the partition keys for which the data is requested.
+     * @throws InvalidRequestException if the partition keys cannot be retrieved
+     */
+    public Collection<ByteBuffer> getPartitionKeys(final QueryOptions options) throws InvalidRequestException
+    {
+        return partitionKeyRestrictions.values(cfm, options);
+    }
+
+    /**
+     * Returns the specified bound of the partition key.
+     *
+     * @param b the boundary type
+     * @param options the query options
+     * @return the specified bound of the partition key
+     * @throws InvalidRequestException if the boundary cannot be retrieved
+     */
+    private ByteBuffer getPartitionKeyBound(Bound b, QueryOptions options) throws InvalidRequestException
+    {
+        // Deal with unrestricted partition key components (special-casing is required to deal with 2i queries on the
+        // first
+        // component of a composite partition key).
+        if (hasPartitionKeyUnrestrictedComponents())
+            return ByteBufferUtil.EMPTY_BYTE_BUFFER;
+
+        // We deal with IN queries for keys in other places, so we know buildBound will return only one result
+        return partitionKeyRestrictions.bounds(cfm, b, options).get(0);
+    }
+
+    /**
+     * Returns the partition key bounds.
+     *
+     * @param options the query options
+     * @return the partition key bounds
+     * @throws InvalidRequestException if the query is invalid
+     */
+    public AbstractBounds<RowPosition> getPartitionKeyBounds(QueryOptions options) throws InvalidRequestException
+    {
+        IPartitioner p = StorageService.getPartitioner();
+
+        if (partitionKeyRestrictions.isOnToken())
+        {
+            return getPartitionKeyBoundsForTokenRestrictions(p, options);
+        }
+
+        return getPartitionKeyBounds(p, options);
+    }
+
+    private AbstractBounds<RowPosition> getPartitionKeyBounds(IPartitioner p,
+                                                              QueryOptions options) throws InvalidRequestException
+    {
+        ByteBuffer startKeyBytes = getPartitionKeyBound(Bound.START, options);
+        ByteBuffer finishKeyBytes = getPartitionKeyBound(Bound.END, options);
+
+        RowPosition startKey = RowPosition.ForKey.get(startKeyBytes, p);
+        RowPosition finishKey = RowPosition.ForKey.get(finishKeyBytes, p);
+
+        if (startKey.compareTo(finishKey) > 0 && !finishKey.isMinimum())
+            return null;
+
+        if (partitionKeyRestrictions.isInclusive(Bound.START))
+        {
+            return partitionKeyRestrictions.isInclusive(Bound.END)
+                    ? new Bounds<>(startKey, finishKey)
+                    : new IncludingExcludingBounds<>(startKey, finishKey);
+        }
+
+        return partitionKeyRestrictions.isInclusive(Bound.END)
+                ? new Range<>(startKey, finishKey)
+                : new ExcludingBounds<>(startKey, finishKey);
+    }
+
+    private AbstractBounds<RowPosition> getPartitionKeyBoundsForTokenRestrictions(IPartitioner p,
+                                                                                  QueryOptions options)
+                                                                                          throws InvalidRequestException
+    {
+        Token startToken = getTokenBound(Bound.START, options, p);
+        Token endToken = getTokenBound(Bound.END, options, p);
+
+        boolean includeStart = partitionKeyRestrictions.isInclusive(Bound.START);
+        boolean includeEnd = partitionKeyRestrictions.isInclusive(Bound.END);
+
+        /*
+         * If we ask SP.getRangeSlice() for (token(200), token(200)], it will happily return the whole ring.
+         * However, wrapping range doesn't really make sense for CQL, and we want to return an empty result in that
+         * case (CASSANDRA-5573). So special case to create a range that is guaranteed to be empty.
+         *
+         * In practice, we want to return an empty result set if either startToken > endToken, or both are equal but
+         * one of the bound is excluded (since [a, a] can contains something, but not (a, a], [a, a) or (a, a)).
+         * Note though that in the case where startToken or endToken is the minimum token, then this special case
+         * rule should not apply.
+         */
+        int cmp = startToken.compareTo(endToken);
+        if (!startToken.isMinimum() && !endToken.isMinimum()
+                && (cmp > 0 || (cmp == 0 && (!includeStart || !includeEnd))))
+            return null;
+
+        RowPosition start = includeStart ? startToken.minKeyBound() : startToken.maxKeyBound();
+        RowPosition end = includeEnd ? endToken.maxKeyBound() : endToken.minKeyBound();
+
+        return new Range<>(start, end);
+    }
+
+    private Token getTokenBound(Bound b, QueryOptions options, IPartitioner p) throws InvalidRequestException
+    {
+        if (!partitionKeyRestrictions.hasBound(b))
+            return p.getMinimumToken();
+
+        ByteBuffer value = partitionKeyRestrictions.bounds(cfm, b, options).get(0);
+        checkNotNull(value, "Invalid null token value");
+        return p.getTokenFactory().fromByteArray(value);
+    }
+
+    /**
+     * Checks if the query does not contains any restriction on the clustering columns.
+     *
+     * @return <code>true</code> if the query does not contains any restriction on the clustering columns,
+     * <code>false</code> otherwise.
+     */
+    public boolean hasNoClusteringColumnsRestriction()
+    {
+        return clusteringColumnsRestrictions.isEmpty();
+    }
+
+    /**
+     * Checks if the query has some restrictions on the clustering columns.
+     *
+     * @return <code>true</code> if the query has some restrictions on the clustering columns,
+     * <code>false</code> otherwise.
+     */
+    public boolean hasClusteringColumnsRestriction()
+    {
+        return !clusteringColumnsRestrictions.isEmpty();
+    }
+
+    // For non-composite slices, we don't support internally the difference between exclusive and
+    // inclusive bounds, so we deal with it manually.
+    public boolean isNonCompositeSliceWithExclusiveBounds()
+    {
+        return !cfm.comparator.isCompound()
+                && clusteringColumnsRestrictions.isSlice()
+                && (!clusteringColumnsRestrictions.isInclusive(Bound.START) || !clusteringColumnsRestrictions.isInclusive(Bound.END));
+    }
+
+    /**
+     * Returns the requested clustering columns as <code>Composite</code>s.
+     *
+     * @param options the query options
+     * @return the requested clustering columns as <code>Composite</code>s
+     * @throws InvalidRequestException if the query is not valid
+     */
+    public List<Composite> getClusteringColumnsAsComposites(QueryOptions options) throws InvalidRequestException
+    {
+        return clusteringColumnsRestrictions.valuesAsComposites(cfm, options);
+    }
+
+    /**
+     * Returns the bounds (start or end) of the clustering columns as <code>Composites</code>.
+     *
+     * @param b the bound type
+     * @param options the query options
+     * @return the bounds (start or end) of the clustering columns as <code>Composites</code>
+     * @throws InvalidRequestException if the request is not valid
+     */
+    public List<Composite> getClusteringColumnsBoundsAsComposites(Bound b,
+                                                                  QueryOptions options) throws InvalidRequestException
+    {
+        List<Composite> bounds = clusteringColumnsRestrictions.boundsAsComposites(cfm, b, options);
+        for (Composite c : bounds) {
+            if (!c.isEmpty())
+                QueryProcessor.validateComposite(c, cfm.comparator);
+        }
+        return bounds;
+    }
+
+    /**
+     * Returns the bounds (start or end) of the clustering columns.
+     *
+     * @param b the bound type
+     * @param options the query options
+     * @return the bounds (start or end) of the clustering columns
+     * @throws InvalidRequestException if the request is not valid
+     */
+    public List<ByteBuffer> getClusteringColumnsBounds(Bound b, QueryOptions options) throws InvalidRequestException
+    {
+        return clusteringColumnsRestrictions.bounds(cfm, b, options);
+    }
+
+    /**
+     * Checks if the bounds (start or end) of the clustering columns are inclusive.
+     *
+     * @param bound the bound type
+     * @return <code>true</code> if the bounds (start or end) of the clustering columns are inclusive,
+     * <code>false</code> otherwise
+     */
+    public boolean areRequestedBoundsInclusive(Bound bound)
+    {
+        return clusteringColumnsRestrictions.isInclusive(bound);
+    }
+
+    /**
+     * Checks if the query returns a range of columns.
+     *
+     * @return <code>true</code> if the query returns a range of columns, <code>false</code> otherwise.
+     */
+    public boolean isColumnRange()
+    {
+        // Due to CASSANDRA-5762, we always do a slice for CQL3 tables (not dense, composite).
+        // Static CF (non dense but non composite) never entails a column slice however
+        if (!cfm.comparator.isDense())
+            return cfm.comparator.isCompound();
+
+        // Otherwise (i.e. for compact table where we don't have a row marker anyway and thus don't care about
+        // CASSANDRA-5762),
+        // it is a range query if it has at least one the column alias for which no relation is defined or is not EQ.
+        return clusteringColumnsRestrictions.size() < cfm.clusteringColumns().size() || clusteringColumnsRestrictions.isSlice();
+    }
+
+    /**
+     * Checks if the query need to use filtering.
+     * @return <code>true</code> if the query need to use filtering, <code>false</code> otherwise.
+     */
+    public boolean needFiltering()
+    {
+        int numberOfRestrictedColumns = 0;
+        for (Restrictions restrictions : indexRestrictions)
+            numberOfRestrictedColumns += restrictions.size();
+
+        return numberOfRestrictedColumns > 1
+                || (numberOfRestrictedColumns == 0 && !clusteringColumnsRestrictions.isEmpty())
+                || (numberOfRestrictedColumns != 0
+                        && nonPrimaryKeyRestrictions.hasMultipleContains());
+    }
+
+    private void validateSecondaryIndexSelections(boolean selectsOnlyStaticColumns) throws InvalidRequestException
+    {
+        checkFalse(keyIsInRelation(),
+                   "Select on indexed columns and with IN clause for the PRIMARY KEY are not supported");
+        // When the user only select static columns, the intent is that we don't query the whole partition but just
+        // the static parts. But 1) we don't have an easy way to do that with 2i and 2) since we don't support index on
+        // static columns
+        // so far, 2i means that you've restricted a non static column, so the query is somewhat non-sensical.
+        checkFalse(selectsOnlyStaticColumns, "Queries using 2ndary indexes don't support selecting only static columns");
+    }
+
+    public void reverse()
+    {
+        clusteringColumnsRestrictions = new ReversedPrimaryKeyRestrictions(clusteringColumnsRestrictions);
+    }
+
+    /**
+     * Checks if the query will never return any rows.
+     *
+     * @param options the query options
+     * @return {@code true} if the query will never return any rows, {@false} otherwise
+     */
+    public boolean isNotReturningAnyRows(QueryOptions options)
+    {
+        return clusteringColumnsRestrictions.isNotReturningAnyRows(cfm, options);
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/TermSlice.java b/src/java/org/apache/cassandra/cql3/restrictions/TermSlice.java
new file mode 100644
index 0000000..d082cc3
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/restrictions/TermSlice.java
@@ -0,0 +1,184 @@
+/*
+ * 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.cql3.restrictions;
+
+import java.util.Collections;
+
+import com.google.common.collect.Iterables;
+
+import org.apache.cassandra.cql3.Operator;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.statements.Bound;
+import org.apache.cassandra.db.index.SecondaryIndex;
+
+final class TermSlice
+{
+    /**
+     * The slice boundaries.
+     */
+    private final Term[] bounds;
+
+    /**
+     * Specifies if a slice boundary is inclusive or not.
+     */
+    private final boolean[] boundInclusive;
+
+    /**
+     * Creates a new <code>TermSlice</code> with the specified boundaries.
+     *
+     * @param start the lower boundary
+     * @param includeStart <code>true</code> if the lower boundary is inclusive
+     * @param end the upper boundary
+     * @param includeEnd <code>true</code> if the upper boundary is inclusive
+     */
+    private TermSlice(Term start, boolean includeStart, Term end, boolean includeEnd)
+    {
+        bounds = new Term[]{start, end};
+        boundInclusive = new boolean[]{includeStart, includeEnd};
+    }
+
+    /**
+     * Creates a new <code>TermSlice</code> with the specified boundary.
+     *
+     * @param bound the boundary type
+     * @param include <code>true</code> if the boundary is inclusive
+     * @param term the value
+     * @return a new <code>TermSlice</code> instance
+     */
+    public static TermSlice newInstance(Bound bound, boolean include, Term term)
+    {
+        return  bound.isStart() ? new TermSlice(term, include, null, false) 
+                                : new TermSlice(null, false, term, include);
+    }
+
+    /**
+     * Returns the boundary value.
+     *
+     * @param bound the boundary type
+     * @return the boundary value
+     */
+    public Term bound(Bound bound)
+    {
+        return bounds[bound.idx];
+    }
+
+    /**
+     * Checks if this slice has a boundary for the specified type.
+     *
+     * @param b the boundary type
+     * @return <code>true</code> if this slice has a boundary for the specified type, <code>false</code> otherwise.
+     */
+    public boolean hasBound(Bound b)
+    {
+        return bounds[b.idx] != null;
+    }
+
+    /**
+     * Checks if this slice boundary is inclusive for the specified type.
+     *
+     * @param b the boundary type
+     * @return <code>true</code> if this slice boundary is inclusive for the specified type,
+     * <code>false</code> otherwise.
+     */
+    public boolean isInclusive(Bound b)
+    {
+        return bounds[b.idx] == null || boundInclusive[b.idx];
+    }
+
+    /**
+     * Merges this slice with the specified one.
+     *
+     * @param otherSlice the slice to merge to
+     * @return the new slice resulting from the merge
+     */
+    public TermSlice merge(TermSlice otherSlice)
+    {
+        if (hasBound(Bound.START))
+        {
+            assert !otherSlice.hasBound(Bound.START);
+
+            return new TermSlice(bound(Bound.START), 
+                                  isInclusive(Bound.START),
+                                  otherSlice.bound(Bound.END),
+                                  otherSlice.isInclusive(Bound.END));
+        }
+        assert !otherSlice.hasBound(Bound.END);
+
+        return new TermSlice(otherSlice.bound(Bound.START), 
+                              otherSlice.isInclusive(Bound.START),
+                              bound(Bound.END),
+                              isInclusive(Bound.END));
+    }
+
+    @Override
+    public String toString()
+    {
+        return String.format("(%s %s, %s %s)", boundInclusive[0] ? ">=" : ">",
+                             bounds[0],
+                             boundInclusive[1] ? "<=" : "<",
+                             bounds[1]);
+    }
+
+    /**
+     * Returns the index operator corresponding to the specified boundary.
+     *
+     * @param b the boundary type
+     * @return the index operator corresponding to the specified boundary
+     */
+    public Operator getIndexOperator(Bound b)
+    {
+        if (b.isStart())
+            return boundInclusive[b.idx] ? Operator.GTE : Operator.GT;
+
+        return boundInclusive[b.idx] ? Operator.LTE : Operator.LT;
+    }
+
+    /**
+     * Check if this <code>TermSlice</code> is supported by the specified index.
+     *
+     * @param index the Secondary index
+     * @return <code>true</code> this type of <code>TermSlice</code> is supported by the specified index,
+     * <code>false</code> otherwise.
+     */
+    public boolean isSupportedBy(SecondaryIndex index)
+    {
+        boolean supported = false;
+
+        if (hasBound(Bound.START))
+            supported |= isInclusive(Bound.START) ? index.supportsOperator(Operator.GTE)
+                    : index.supportsOperator(Operator.GT);
+        if (hasBound(Bound.END))
+            supported |= isInclusive(Bound.END) ? index.supportsOperator(Operator.LTE)
+                    : index.supportsOperator(Operator.LT);
+
+        return supported;
+    }
+
+    public Iterable<Function> getFunctions()
+    {
+        if (hasBound(Bound.START) && hasBound(Bound.END))
+            return Iterables.concat(bound(Bound.START).getFunctions(), bound(Bound.END).getFunctions());
+        else if (hasBound(Bound.START))
+            return bound(Bound.START).getFunctions();
+        else if (hasBound(Bound.END))
+            return bound(Bound.END).getFunctions();
+        else
+            return Collections.emptySet();
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/TokenFilter.java b/src/java/org/apache/cassandra/cql3/restrictions/TokenFilter.java
new file mode 100644
index 0000000..18444ec
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/restrictions/TokenFilter.java
@@ -0,0 +1,245 @@
+/*
+ * 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.cql3.restrictions;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import com.google.common.collect.BoundType;
+import com.google.common.collect.ImmutableRangeSet;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeSet;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.statements.Bound;
+import org.apache.cassandra.db.composites.Composite;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.service.StorageService;
+
+import static org.apache.cassandra.cql3.statements.Bound.END;
+import static org.apache.cassandra.cql3.statements.Bound.START;
+
+/**
+ * <code>Restriction</code> decorator used to merge non-token restriction and token restriction on partition keys.
+ */
+final class TokenFilter extends ForwardingPrimaryKeyRestrictions
+{
+    /**
+     * The decorated restriction
+     */
+    private PrimaryKeyRestrictions restrictions;
+
+    /**
+     * The restriction on the token
+     */
+    private TokenRestriction tokenRestriction;
+
+    /**
+     * The partitioner
+     */
+    private static final IPartitioner partitioner = StorageService.getPartitioner();
+
+    @Override
+    protected PrimaryKeyRestrictions getDelegate()
+    {
+        return restrictions;
+    }
+
+    @Override
+    public boolean isOnToken()
+    {
+        // if all partition key columns have non-token restrictions, we can simply use the token range to filter
+        // those restrictions and then ignore the token range
+        return restrictions.size() < tokenRestriction.size();
+    }
+
+    public TokenFilter(PrimaryKeyRestrictions restrictions, TokenRestriction tokenRestriction)
+    {
+        this.restrictions = restrictions;
+        this.tokenRestriction = tokenRestriction;
+    }
+
+    @Override
+    public List<ByteBuffer> values(CFMetaData cfm, QueryOptions options) throws InvalidRequestException
+    {
+        return filter(cfm, restrictions.values(cfm, options), options);
+    }
+
+    @Override
+    public List<Composite> valuesAsComposites(CFMetaData cfm, QueryOptions options) throws InvalidRequestException
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public PrimaryKeyRestrictions mergeWith(Restriction restriction) throws InvalidRequestException
+    {
+        if (restriction.isOnToken())
+            return new TokenFilter(restrictions, (TokenRestriction) tokenRestriction.mergeWith(restriction));
+
+        return new TokenFilter(super.mergeWith(restriction), tokenRestriction);
+    }
+
+    @Override
+    public boolean isInclusive(Bound bound)
+    {
+        return tokenRestriction.isInclusive(bound);
+    }
+
+    @Override
+    public boolean hasBound(Bound b)
+    {
+        return tokenRestriction.hasBound(b);
+    }
+
+    @Override
+    public List<ByteBuffer> bounds(CFMetaData cfm, Bound bound, QueryOptions options) throws InvalidRequestException
+    {
+        return tokenRestriction.bounds(cfm, bound, options);
+    }
+
+    @Override
+    public List<Composite> boundsAsComposites(CFMetaData cfm, Bound bound, QueryOptions options) throws InvalidRequestException
+    {
+        return tokenRestriction.boundsAsComposites(cfm, bound, options);
+    }
+
+    /**
+     * Filter the values returned by the restriction.
+     *
+     * @param cfm the table metadata
+     * @param values the values returned by the decorated restriction
+     * @param options the query options
+     * @return the values matching the token restriction
+     * @throws InvalidRequestException if the request is invalid
+     */
+    private List<ByteBuffer> filter(CFMetaData cfm, List<ByteBuffer> values, QueryOptions options) throws InvalidRequestException
+    {
+        RangeSet<Token> rangeSet = tokenRestriction.isSlice() ? toRangeSet(cfm, tokenRestriction, options)
+                                                              : toRangeSet(tokenRestriction.values(cfm, options));
+
+        return filterWithRangeSet(rangeSet, values);
+    }
+
+    /**
+     * Filter out the values for which the tokens are not included within the specified range.
+     *
+     * @param tokens the tokens range
+     * @param values the restricted values
+     * @return the values for which the tokens are not included within the specified range.
+     */
+    private static List<ByteBuffer> filterWithRangeSet(RangeSet<Token> tokens, List<ByteBuffer> values)
+    {
+        List<ByteBuffer> remaining = new ArrayList<>();
+
+        for (ByteBuffer value : values)
+        {
+            Token token = partitioner.getToken(value);
+
+            if (!tokens.contains(token))
+                continue;
+
+            remaining.add(value);
+        }
+        return remaining;
+    }
+
+    /**
+     * Converts the specified list into a range set.
+     *
+     * @param buffers the token restriction values
+     * @return the range set corresponding to the specified list
+     */
+    private static RangeSet<Token> toRangeSet(List<ByteBuffer> buffers)
+    {
+        ImmutableRangeSet.Builder<Token> builder = ImmutableRangeSet.builder();
+
+        for (ByteBuffer buffer : buffers)
+            builder.add(Range.singleton(deserializeToken(buffer)));
+
+        return builder.build();
+    }
+
+    /**
+     * Converts the specified slice into a range set.
+     *
+     * @param cfm the table metadata
+     * @param slice the slice to convert
+     * @param options the query option
+     * @return the range set corresponding to the specified slice
+     * @throws InvalidRequestException if the request is invalid
+     */
+    private static RangeSet<Token> toRangeSet(CFMetaData cfm, TokenRestriction slice, QueryOptions options) throws InvalidRequestException
+    {
+        if (slice.hasBound(START))
+        {
+            Token start = deserializeToken(slice.bounds(cfm, START, options).get(0));
+
+            BoundType startBoundType = toBoundType(slice.isInclusive(START));
+
+            if (slice.hasBound(END))
+            {
+                BoundType endBoundType = toBoundType(slice.isInclusive(END));
+                Token end = deserializeToken(slice.bounds(cfm, END, options).get(0));
+
+                if (start.equals(end) && (BoundType.OPEN == startBoundType || BoundType.OPEN == endBoundType))
+                    return ImmutableRangeSet.of();
+
+                if (start.compareTo(end) <= 0)
+                    return ImmutableRangeSet.of(Range.range(start,
+                                                            startBoundType,
+                                                            end,
+                                                            endBoundType));
+
+                return ImmutableRangeSet.<Token> builder()
+                                        .add(Range.upTo(end, endBoundType))
+                                        .add(Range.downTo(start, startBoundType))
+                                        .build();
+            }
+            return ImmutableRangeSet.of(Range.downTo(start,
+                                                     startBoundType));
+        }
+        Token end = deserializeToken(slice.bounds(cfm, END, options).get(0));
+        return ImmutableRangeSet.of(Range.upTo(end, toBoundType(slice.isInclusive(END))));
+    }
+
+    public boolean isNotReturningAnyRows(CFMetaData cfm, QueryOptions options)
+    {
+        return false;
+    }
+
+    /**
+     * Deserializes the token corresponding to the specified buffer.
+     *
+     * @param buffer the buffer
+     * @return the token corresponding to the specified buffer
+     */
+    private static Token deserializeToken(ByteBuffer buffer)
+    {
+        return partitioner.getTokenFactory().fromByteArray(buffer);
+    }
+
+    private static BoundType toBoundType(boolean inclusive)
+    {
+        return inclusive ? BoundType.CLOSED : BoundType.OPEN;
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/restrictions/TokenRestriction.java b/src/java/org/apache/cassandra/cql3/restrictions/TokenRestriction.java
new file mode 100644
index 0000000..97c55c4
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/restrictions/TokenRestriction.java
@@ -0,0 +1,274 @@
+/*
+ * 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.cql3.restrictions;
+
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+
+import com.google.common.base.Joiner;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.statements.Bound;
+import org.apache.cassandra.db.IndexExpression;
+import org.apache.cassandra.db.composites.CType;
+import org.apache.cassandra.db.composites.Composite;
+import org.apache.cassandra.db.composites.CompositesBuilder;
+import org.apache.cassandra.db.index.SecondaryIndexManager;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
+
+/**
+ * <code>Restriction</code> using the token function.
+ */
+public abstract class TokenRestriction extends AbstractPrimaryKeyRestrictions
+{
+    /**
+     * The definition of the columns to which apply the token restriction.
+     */
+    protected final List<ColumnDefinition> columnDefs;
+
+    /**
+     * Creates a new <code>TokenRestriction</code> that apply to the specified columns.
+     *
+     * @param ctype the composite type
+     * @param columnDefs the definition of the columns to which apply the token restriction
+     */
+    public TokenRestriction(CType ctype, List<ColumnDefinition> columnDefs)
+    {
+        super(ctype);
+        this.columnDefs = columnDefs;
+    }
+
+    @Override
+    public  boolean isOnToken()
+    {
+        return true;
+    }
+
+    @Override
+    public List<ColumnDefinition> getColumnDefs()
+    {
+        return columnDefs;
+    }
+
+    @Override
+    public ColumnDefinition getFirstColumn()
+    {
+        return columnDefs.get(0);
+    }
+
+    @Override
+    public ColumnDefinition getLastColumn()
+    {
+        return columnDefs.get(columnDefs.size() - 1);
+    }
+
+    @Override
+    public boolean hasSupportingIndex(SecondaryIndexManager secondaryIndexManager)
+    {
+        return false;
+    }
+
+    @Override
+    public final void addIndexExpressionTo(List<IndexExpression> expressions,
+                                     SecondaryIndexManager indexManager,
+                                     QueryOptions options)
+    {
+        throw new UnsupportedOperationException("Index expression cannot be created for token restriction");
+    }
+
+    @Override
+    public CompositesBuilder appendTo(CFMetaData cfm, CompositesBuilder builder, QueryOptions options)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public List<Composite> valuesAsComposites(CFMetaData cfm, QueryOptions options) throws InvalidRequestException
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public List<Composite> boundsAsComposites(CFMetaData cfm, Bound bound, QueryOptions options) throws InvalidRequestException
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    /**
+     * Returns the column names as a comma separated <code>String</code>.
+     *
+     * @return the column names as a comma separated <code>String</code>.
+     */
+    protected final String getColumnNamesAsString()
+    {
+        return Joiner.on(", ").join(ColumnDefinition.toIdentifiers(columnDefs));
+    }
+
+    @Override
+    public final PrimaryKeyRestrictions mergeWith(Restriction otherRestriction) throws InvalidRequestException
+    {
+        if (!otherRestriction.isOnToken())
+            return new TokenFilter(toPrimaryKeyRestriction(otherRestriction), this);
+
+        return doMergeWith((TokenRestriction) otherRestriction);
+    }
+
+    /**
+     * Merges this restriction with the specified <code>TokenRestriction</code>.
+     * @param otherRestriction the <code>TokenRestriction</code> to merge with.
+     */
+    protected abstract PrimaryKeyRestrictions doMergeWith(TokenRestriction otherRestriction) throws InvalidRequestException;
+
+    /**
+     * Converts the specified restriction into a <code>PrimaryKeyRestrictions</code>.
+     *
+     * @param restriction the restriction to convert
+     * @return a <code>PrimaryKeyRestrictions</code>
+     * @throws InvalidRequestException if a problem occurs while converting the restriction
+     */
+    private PrimaryKeyRestrictions toPrimaryKeyRestriction(Restriction restriction) throws InvalidRequestException
+    {
+        if (restriction instanceof PrimaryKeyRestrictions)
+            return (PrimaryKeyRestrictions) restriction;
+
+        return new PrimaryKeyRestrictionSet(ctype).mergeWith(restriction);
+    }
+
+    public static final class EQ extends TokenRestriction
+    {
+        private final Term value;
+
+        public EQ(CType ctype, List<ColumnDefinition> columnDefs, Term value)
+        {
+            super(ctype, columnDefs);
+            this.value = value;
+        }
+
+        @Override
+        public boolean isEQ()
+        {
+            return true;
+        }
+
+        @Override
+        public Iterable<Function> getFunctions()
+        {
+            return value.getFunctions();
+        }
+
+        @Override
+        protected PrimaryKeyRestrictions doMergeWith(TokenRestriction otherRestriction) throws InvalidRequestException
+        {
+            throw invalidRequest("%s cannot be restricted by more than one relation if it includes an Equal",
+                                 Joiner.on(", ").join(ColumnDefinition.toIdentifiers(columnDefs)));
+        }
+
+        @Override
+        public List<ByteBuffer> values(CFMetaData cfm, QueryOptions options) throws InvalidRequestException
+        {
+            return Collections.singletonList(value.bindAndGet(options));
+        }
+    }
+
+    public static class Slice extends TokenRestriction
+    {
+        private final TermSlice slice;
+
+        public Slice(CType ctype, List<ColumnDefinition> columnDefs, Bound bound, boolean inclusive, Term term)
+        {
+            super(ctype, columnDefs);
+            slice = TermSlice.newInstance(bound, inclusive, term);
+        }
+
+        @Override
+        public boolean isSlice()
+        {
+            return true;
+        }
+
+        @Override
+        public List<ByteBuffer> values(CFMetaData cfm, QueryOptions options) throws InvalidRequestException
+        {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public boolean hasBound(Bound b)
+        {
+            return slice.hasBound(b);
+        }
+
+        @Override
+        public List<ByteBuffer> bounds(CFMetaData cfm, Bound b, QueryOptions options) throws InvalidRequestException
+        {
+            return Collections.singletonList(slice.bound(b).bindAndGet(options));
+        }
+
+        @Override
+        public Iterable<Function> getFunctions()
+        {
+            return slice.getFunctions();
+        }
+
+        @Override
+        public boolean isInclusive(Bound b)
+        {
+            return slice.isInclusive(b);
+        }
+
+        @Override
+        protected PrimaryKeyRestrictions doMergeWith(TokenRestriction otherRestriction)
+        throws InvalidRequestException
+        {
+            if (!otherRestriction.isSlice())
+                throw invalidRequest("Columns \"%s\" cannot be restricted by both an equality and an inequality relation",
+                                     getColumnNamesAsString());
+
+            TokenRestriction.Slice otherSlice = (TokenRestriction.Slice) otherRestriction;
+
+            if (hasBound(Bound.START) && otherSlice.hasBound(Bound.START))
+                throw invalidRequest("More than one restriction was found for the start bound on %s",
+                                     getColumnNamesAsString());
+
+            if (hasBound(Bound.END) && otherSlice.hasBound(Bound.END))
+                throw invalidRequest("More than one restriction was found for the end bound on %s",
+                                     getColumnNamesAsString());
+
+            return new Slice(ctype, columnDefs,  slice.merge(otherSlice.slice));
+        }
+
+        @Override
+        public String toString()
+        {
+            return String.format("SLICE%s", slice);
+        }
+
+        private Slice(CType ctype, List<ColumnDefinition> columnDefs, TermSlice slice)
+        {
+            super(ctype, columnDefs);
+            this.slice = slice;
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java b/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java
new file mode 100644
index 0000000..bf1234f
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java
@@ -0,0 +1,137 @@
+/*
+ * 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.cql3.selection;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.List;
+
+import com.google.common.collect.Iterables;
+
+import org.apache.commons.lang3.text.StrBuilder;
+
+import org.apache.cassandra.cql3.functions.AggregateFcts;
+
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+abstract class AbstractFunctionSelector<T extends Function> extends Selector
+{
+    protected final T fun;
+
+    /**
+     * The list used to pass the function arguments is recycled to avoid the cost of instantiating a new list
+     * with each function call.
+     */
+    protected final List<ByteBuffer> args;
+    protected final List<Selector> argSelectors;
+
+    public static Factory newFactory(final Function fun, final SelectorFactories factories) throws InvalidRequestException
+    {
+        if (fun.isAggregate())
+        {
+            if (factories.doesAggregation())
+                throw new InvalidRequestException("aggregate functions cannot be used as arguments of aggregate functions");
+        }
+
+        return new Factory()
+        {
+            protected String getColumnName()
+            {
+                if (AggregateFcts.isCountRows(fun))
+                    return "count";
+
+                return new StrBuilder(fun.name().toString()).append('(')
+                                                            .appendWithSeparators(factories.getColumnNames(), ", ")
+                                                            .append(')')
+                                                            .toString();
+            }
+
+            protected AbstractType<?> getReturnType()
+            {
+                return fun.returnType();
+            }
+
+            protected void addColumnMapping(SelectionColumnMapping mapping, ColumnSpecification resultsColumn)
+            {
+                SelectionColumnMapping tmpMapping = SelectionColumnMapping.newMapping();
+                for (Factory factory : factories)
+                   factory.addColumnMapping(tmpMapping, resultsColumn);
+
+                if (tmpMapping.getMappings().get(resultsColumn).isEmpty())
+                    // add a null mapping for cases where there are no
+                    // further selectors, such as no-arg functions and count
+                    mapping.addMapping(resultsColumn, (ColumnDefinition)null);
+                else
+                    // collate the mapped columns from the child factories & add those
+                    mapping.addMapping(resultsColumn, tmpMapping.getMappings().values());
+            }
+
+            public Iterable<Function> getFunctions()
+            {
+                return Iterables.concat(fun.getFunctions(), factories.getFunctions());
+            }
+
+            public Selector newInstance() throws InvalidRequestException
+            {
+                return fun.isAggregate() ? new AggregateFunctionSelector(fun, factories.newInstances())
+                                         : new ScalarFunctionSelector(fun, factories.newInstances());
+            }
+
+            public boolean isWritetimeSelectorFactory()
+            {
+                return factories.containsWritetimeSelectorFactory();
+            }
+
+            public boolean isTTLSelectorFactory()
+            {
+                return factories.containsTTLSelectorFactory();
+            }
+
+            public boolean isAggregateSelectorFactory()
+            {
+                return fun.isAggregate() || factories.doesAggregation();
+            }
+        };
+    }
+
+    protected AbstractFunctionSelector(T fun, List<Selector> argSelectors)
+    {
+        this.fun = fun;
+        this.argSelectors = argSelectors;
+        this.args = Arrays.asList(new ByteBuffer[argSelectors.size()]);
+    }
+
+    public AbstractType<?> getType()
+    {
+        return fun.returnType();
+    }
+
+    @Override
+    public String toString()
+    {
+        return new StrBuilder().append(fun.name())
+                               .append("(")
+                               .appendWithSeparators(argSelectors, ", ")
+                               .append(")")
+                               .toString();
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/selection/AggregateFunctionSelector.java b/src/java/org/apache/cassandra/cql3/selection/AggregateFunctionSelector.java
new file mode 100644
index 0000000..27a8294
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/selection/AggregateFunctionSelector.java
@@ -0,0 +1,66 @@
+/*
+ * 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.cql3.selection;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.cassandra.cql3.functions.AggregateFunction;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+final class AggregateFunctionSelector extends AbstractFunctionSelector<AggregateFunction>
+{
+    private final AggregateFunction.Aggregate aggregate;
+
+    public boolean isAggregate()
+    {
+        return true;
+    }
+
+    public void addInput(int protocolVersion, ResultSetBuilder rs) throws InvalidRequestException
+    {
+        // Aggregation of aggregation is not supported
+        for (int i = 0, m = argSelectors.size(); i < m; i++)
+        {
+            Selector s = argSelectors.get(i);
+            s.addInput(protocolVersion, rs);
+            args.set(i, s.getOutput(protocolVersion));
+            s.reset();
+        }
+        this.aggregate.addInput(protocolVersion, args);
+    }
+
+    public ByteBuffer getOutput(int protocolVersion) throws InvalidRequestException
+    {
+        return aggregate.compute(protocolVersion);
+    }
+
+    public void reset()
+    {
+        aggregate.reset();
+    }
+
+    AggregateFunctionSelector(Function fun, List<Selector> argSelectors) throws InvalidRequestException
+    {
+        super((AggregateFunction) fun, argSelectors);
+
+        this.aggregate = this.fun.newAggregate();
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/selection/FieldSelector.java b/src/java/org/apache/cassandra/cql3/selection/FieldSelector.java
new file mode 100644
index 0000000..63b6cc6
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/selection/FieldSelector.java
@@ -0,0 +1,109 @@
+/*
+ * 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.cql3.selection;
+
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.db.marshal.UserType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+final class FieldSelector extends Selector
+{
+    private final UserType type;
+    private final int field;
+    private final Selector selected;
+
+    public static Factory newFactory(final UserType type, final int field, final Selector.Factory factory)
+    {
+        return new Factory()
+        {
+            protected String getColumnName()
+            {
+                return String.format("%s.%s",
+                                     factory.getColumnName(),
+                                     UTF8Type.instance.getString(type.fieldName(field)));
+            }
+
+            protected AbstractType<?> getReturnType()
+            {
+                return type.fieldType(field);
+            }
+
+            protected void addColumnMapping(SelectionColumnMapping mapping, ColumnSpecification resultsColumn)
+            {
+                factory.addColumnMapping(mapping, resultsColumn);
+            }
+
+            public Selector newInstance() throws InvalidRequestException
+            {
+                return new FieldSelector(type, field, factory.newInstance());
+            }
+
+            public boolean isAggregateSelectorFactory()
+            {
+                return factory.isAggregateSelectorFactory();
+            }
+        };
+    }
+
+    public boolean isAggregate()
+    {
+        return false;
+    }
+
+    public void addInput(int protocolVersion, ResultSetBuilder rs) throws InvalidRequestException
+    {
+        selected.addInput(protocolVersion, rs);
+    }
+
+    public ByteBuffer getOutput(int protocolVersion) throws InvalidRequestException
+    {
+        ByteBuffer value = selected.getOutput(protocolVersion);
+        if (value == null)
+            return null;
+        ByteBuffer[] buffers = type.split(value);
+        return field < buffers.length ? buffers[field] : null;
+    }
+
+    public AbstractType<?> getType()
+    {
+        return type.fieldType(field);
+    }
+
+    public void reset()
+    {
+        selected.reset();
+    }
+
+    @Override
+    public String toString()
+    {
+        return String.format("%s.%s", selected, UTF8Type.instance.getString(type.fieldName(field)));
+    }
+
+    private FieldSelector(UserType type, int field, Selector selected)
+    {
+        this.type = type;
+        this.field = field;
+        this.selected = selected;
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/selection/RawSelector.java b/src/java/org/apache/cassandra/cql3/selection/RawSelector.java
new file mode 100644
index 0000000..7d5543f
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/selection/RawSelector.java
@@ -0,0 +1,61 @@
+/*
+ * 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.cql3.selection;
+
+import java.util.List;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Lists;
+
+public class RawSelector
+{
+    public final Selectable.Raw selectable;
+    public final ColumnIdentifier alias;
+
+    public RawSelector(Selectable.Raw selectable, ColumnIdentifier alias)
+    {
+        this.selectable = selectable;
+        this.alias = alias;
+    }
+
+    /**
+     * Converts the specified list of <code>RawSelector</code>s into a list of <code>Selectable</code>s.
+     *
+     * @param raws the <code>RawSelector</code>s to converts.
+     * @return a list of <code>Selectable</code>s
+     */
+    public static List<Selectable> toSelectables(List<RawSelector> raws, final CFMetaData cfm)
+    {
+        return Lists.transform(raws, new Function<RawSelector, Selectable>()
+        {
+            public Selectable apply(RawSelector raw)
+            {
+                return raw.selectable.prepare(cfm);
+            }
+        });
+    }
+
+    public boolean processesSelection()
+    {
+        return selectable.processesSelection();
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/selection/ScalarFunctionSelector.java b/src/java/org/apache/cassandra/cql3/selection/ScalarFunctionSelector.java
new file mode 100644
index 0000000..bb56bb8
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/selection/ScalarFunctionSelector.java
@@ -0,0 +1,67 @@
+/*
+ * 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.cql3.selection;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.functions.ScalarFunction;
+import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+final class ScalarFunctionSelector extends AbstractFunctionSelector<ScalarFunction>
+{
+    public boolean isAggregate()
+    {
+        // We cannot just return true as it is possible to have a scalar function wrapping an aggregation function
+        if (argSelectors.isEmpty())
+            return false;
+
+        return argSelectors.get(0).isAggregate();
+    }
+
+    public void addInput(int protocolVersion, ResultSetBuilder rs) throws InvalidRequestException
+    {
+        for (int i = 0, m = argSelectors.size(); i < m; i++)
+        {
+            Selector s = argSelectors.get(i);
+            s.addInput(protocolVersion, rs);
+        }
+    }
+
+    public void reset()
+    {
+    }
+
+    public ByteBuffer getOutput(int protocolVersion) throws InvalidRequestException
+    {
+        for (int i = 0, m = argSelectors.size(); i < m; i++)
+        {
+            Selector s = argSelectors.get(i);
+            args.set(i, s.getOutput(protocolVersion));
+            s.reset();
+        }
+        return fun.execute(protocolVersion, args);
+    }
+
+    ScalarFunctionSelector(Function fun, List<Selector> argSelectors)
+    {
+        super((ScalarFunction) fun, argSelectors);
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/cql3/selection/Selectable.java b/src/java/org/apache/cassandra/cql3/selection/Selectable.java
new file mode 100644
index 0000000..ee134ee
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/selection/Selectable.java
@@ -0,0 +1,251 @@
+/*
+ * 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.cql3.selection;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.lang3.text.StrBuilder;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.functions.*;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.UserType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+public abstract class Selectable
+{
+    public abstract Selector.Factory newSelectorFactory(CFMetaData cfm, List<ColumnDefinition> defs)
+            throws InvalidRequestException;
+
+    protected static int addAndGetIndex(ColumnDefinition def, List<ColumnDefinition> l)
+    {
+        int idx = l.indexOf(def);
+        if (idx < 0)
+        {
+            idx = l.size();
+            l.add(def);
+        }
+        return idx;
+    }
+
+    public static interface Raw
+    {
+        public Selectable prepare(CFMetaData cfm);
+
+        /**
+         * Returns true if any processing is performed on the selected column.
+         **/
+        public boolean processesSelection();
+    }
+
+    public static class WritetimeOrTTL extends Selectable
+    {
+        public final ColumnIdentifier id;
+        public final boolean isWritetime;
+
+        public WritetimeOrTTL(ColumnIdentifier id, boolean isWritetime)
+        {
+            this.id = id;
+            this.isWritetime = isWritetime;
+        }
+
+        @Override
+        public String toString()
+        {
+            return (isWritetime ? "writetime" : "ttl") + "(" + id + ")";
+        }
+
+        public Selector.Factory newSelectorFactory(CFMetaData cfm,
+                                                   List<ColumnDefinition> defs) throws InvalidRequestException
+        {
+            ColumnDefinition def = cfm.getColumnDefinition(id);
+            if (def == null)
+                throw new InvalidRequestException(String.format("Undefined name %s in selection clause", id));
+            if (def.isPrimaryKeyColumn())
+                throw new InvalidRequestException(
+                        String.format("Cannot use selection function %s on PRIMARY KEY part %s",
+                                      isWritetime ? "writeTime" : "ttl",
+                                      def.name));
+            if (def.type.isCollection())
+                throw new InvalidRequestException(String.format("Cannot use selection function %s on collections",
+                                                                isWritetime ? "writeTime" : "ttl"));
+
+            return WritetimeOrTTLSelector.newFactory(def, addAndGetIndex(def, defs), isWritetime);
+        }
+
+        public static class Raw implements Selectable.Raw
+        {
+            private final ColumnIdentifier.Raw id;
+            private final boolean isWritetime;
+
+            public Raw(ColumnIdentifier.Raw id, boolean isWritetime)
+            {
+                this.id = id;
+                this.isWritetime = isWritetime;
+            }
+
+            public WritetimeOrTTL prepare(CFMetaData cfm)
+            {
+                return new WritetimeOrTTL(id.prepare(cfm), isWritetime);
+            }
+
+            public boolean processesSelection()
+            {
+                return true;
+            }
+        }
+    }
+
+    public static class WithFunction extends Selectable
+    {
+        public final FunctionName functionName;
+        public final List<Selectable> args;
+
+        public WithFunction(FunctionName functionName, List<Selectable> args)
+        {
+            this.functionName = functionName;
+            this.args = args;
+        }
+
+        @Override
+        public String toString()
+        {
+            return new StrBuilder().append(functionName)
+                                   .append("(")
+                                   .appendWithSeparators(args, ", ")
+                                   .append(")")
+                                   .toString();
+        }
+
+        public Selector.Factory newSelectorFactory(CFMetaData cfm,
+                                                   List<ColumnDefinition> defs) throws InvalidRequestException
+        {
+            SelectorFactories factories  =
+                    SelectorFactories.createFactoriesAndCollectColumnDefinitions(args, cfm, defs);
+
+            // We need to circumvent the normal function lookup process for toJson() because instances of the function
+            // are not pre-declared (because it can accept any type of argument).
+            Function fun;
+            if (functionName.equalsNativeFunction(ToJsonFct.NAME))
+                fun = ToJsonFct.getInstance(factories.getReturnTypes());
+            else
+                fun = Functions.get(cfm.ksName, functionName, factories.newInstances(), cfm.ksName, cfm.cfName, null);
+
+            if (fun == null)
+                throw new InvalidRequestException(String.format("Unknown function '%s'", functionName));
+            if (fun.returnType() == null)
+                throw new InvalidRequestException(String.format("Unknown function %s called in selection clause",
+                                                                functionName));
+
+            return AbstractFunctionSelector.newFactory(fun, factories);
+        }
+
+        public static class Raw implements Selectable.Raw
+        {
+            private final FunctionName functionName;
+            private final List<Selectable.Raw> args;
+
+            public Raw(FunctionName functionName, List<Selectable.Raw> args)
+            {
+                this.functionName = functionName;
+                this.args = args;
+            }
+
+            public WithFunction prepare(CFMetaData cfm)
+            {
+                List<Selectable> preparedArgs = new ArrayList<>(args.size());
+                for (Selectable.Raw arg : args)
+                    preparedArgs.add(arg.prepare(cfm));
+                return new WithFunction(functionName, preparedArgs);
+            }
+
+            public boolean processesSelection()
+            {
+                return true;
+            }
+        }
+    }
+
+    public static class WithFieldSelection extends Selectable
+    {
+        public final Selectable selected;
+        public final ColumnIdentifier field;
+
+        public WithFieldSelection(Selectable selected, ColumnIdentifier field)
+        {
+            this.selected = selected;
+            this.field = field;
+        }
+
+        @Override
+        public String toString()
+        {
+            return String.format("%s.%s", selected, field);
+        }
+
+        public Selector.Factory newSelectorFactory(CFMetaData cfm,
+                                                   List<ColumnDefinition> defs) throws InvalidRequestException
+        {
+            Selector.Factory factory = selected.newSelectorFactory(cfm, defs);
+            AbstractType<?> type = factory.newInstance().getType();
+            if (!(type instanceof UserType))
+                throw new InvalidRequestException(
+                        String.format("Invalid field selection: %s of type %s is not a user type",
+                                      selected,
+                                      type.asCQL3Type()));
+
+            UserType ut = (UserType) type;
+            for (int i = 0; i < ut.size(); i++)
+            {
+                if (!ut.fieldName(i).equals(field.bytes))
+                    continue;
+                return FieldSelector.newFactory(ut, i, factory);
+            }
+            throw new InvalidRequestException(String.format("%s of type %s has no field %s",
+                                                            selected,
+                                                            type.asCQL3Type(),
+                                                            field));
+        }
+
+        public static class Raw implements Selectable.Raw
+        {
+            private final Selectable.Raw selected;
+            private final ColumnIdentifier.Raw field;
+
+            public Raw(Selectable.Raw selected, ColumnIdentifier.Raw field)
+            {
+                this.selected = selected;
+                this.field = field;
+            }
+
+            public WithFieldSelection prepare(CFMetaData cfm)
+            {
+                return new WithFieldSelection(selected.prepare(cfm), field.prepare(cfm));
+            }
+
+            public boolean processesSelection()
+            {
+                return true;
+            }
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/selection/Selection.java b/src/java/org/apache/cassandra/cql3/selection/Selection.java
new file mode 100644
index 0000000..6016059
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/selection/Selection.java
@@ -0,0 +1,566 @@
+/*
+ * 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.cql3.selection;
+
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Iterators;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.db.Cell;
+import org.apache.cassandra.db.CounterCell;
+import org.apache.cassandra.db.ExpiringCell;
+import org.apache.cassandra.db.context.CounterContext;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public abstract class Selection
+{
+    /**
+     * A predicate that returns <code>true</code> for static columns.
+     */
+    private static final Predicate<ColumnDefinition> STATIC_COLUMN_FILTER = new Predicate<ColumnDefinition>()
+    {
+        public boolean apply(ColumnDefinition def)
+        {
+            return def.isStatic();
+        }
+    };
+
+    private final CFMetaData cfm;
+    private final List<ColumnDefinition> columns;
+    private final SelectionColumnMapping columnMapping;
+    private final ResultSet.ResultMetadata metadata;
+    private final boolean collectTimestamps;
+    private final boolean collectTTLs;
+
+    protected Selection(CFMetaData cfm,
+                        List<ColumnDefinition> columns,
+                        SelectionColumnMapping columnMapping,
+                        boolean collectTimestamps,
+                        boolean collectTTLs)
+    {
+        this.cfm = cfm;
+        this.columns = columns;
+        this.columnMapping = columnMapping;
+        this.metadata = new ResultSet.ResultMetadata(columnMapping.getColumnSpecifications());
+        this.collectTimestamps = collectTimestamps;
+        this.collectTTLs = collectTTLs;
+    }
+
+    // Overriden by SimpleSelection when appropriate.
+    public boolean isWildcard()
+    {
+        return false;
+    }    
+
+    /**
+     * Checks if this selection contains static columns.
+     * @return <code>true</code> if this selection contains static columns, <code>false</code> otherwise;
+     */
+    public boolean containsStaticColumns()
+    {
+        if (!cfm.hasStaticColumns())
+            return false;
+
+        if (isWildcard())
+            return true;
+
+        return !Iterables.isEmpty(Iterables.filter(columns, STATIC_COLUMN_FILTER));
+    }
+
+    /**
+     * Checks if this selection contains only static columns.
+     * @return <code>true</code> if this selection contains only static columns, <code>false</code> otherwise;
+     */
+    public boolean containsOnlyStaticColumns()
+    {
+        if (!containsStaticColumns())
+            return false;
+
+        if (isWildcard())
+            return false;
+
+        for (ColumnDefinition def : getColumns())
+        {
+            if (!def.isPartitionKey() && !def.isStatic())
+                return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * Checks if this selection contains a collection.
+     *
+     * @return <code>true</code> if this selection contains a collection, <code>false</code> otherwise.
+     */
+    public boolean containsACollection()
+    {
+        if (!cfm.comparator.hasCollections())
+            return false;
+
+        for (ColumnDefinition def : getColumns())
+            if (def.type.isCollection() && def.type.isMultiCell())
+                return true;
+
+        return false;
+    }
+
+    public ResultSet.ResultMetadata getResultMetadata(boolean isJson)
+    {
+        if (!isJson)
+            return metadata;
+
+        ColumnSpecification firstColumn = metadata.names.get(0);
+        ColumnSpecification jsonSpec = new ColumnSpecification(firstColumn.ksName, firstColumn.cfName, Json.JSON_COLUMN_ID, UTF8Type.instance);
+        return new ResultSet.ResultMetadata(Arrays.asList(jsonSpec));
+    }
+
+    public static Selection wildcard(CFMetaData cfm)
+    {
+        List<ColumnDefinition> all = new ArrayList<>(cfm.allColumns().size());
+        Iterators.addAll(all, cfm.allColumnsInSelectOrder());
+        return new SimpleSelection(cfm, all, true);
+    }
+
+    public static Selection forColumns(CFMetaData cfm, List<ColumnDefinition> columns)
+    {
+        return new SimpleSelection(cfm, columns, false);
+    }
+
+    public int addColumnForOrdering(ColumnDefinition c)
+    {
+        columns.add(c);
+        metadata.addNonSerializedColumn(c);
+        return columns.size() - 1;
+    }
+
+    public Iterable<Function> getFunctions()
+    {
+        return Collections.emptySet();
+    }
+
+    private static boolean processesSelection(List<RawSelector> rawSelectors)
+    {
+        for (RawSelector rawSelector : rawSelectors)
+        {
+            if (rawSelector.processesSelection())
+                return true;
+        }
+        return false;
+    }
+
+    public static Selection fromSelectors(CFMetaData cfm, List<RawSelector> rawSelectors) throws InvalidRequestException
+    {
+        List<ColumnDefinition> defs = new ArrayList<>();
+
+        SelectorFactories factories =
+                SelectorFactories.createFactoriesAndCollectColumnDefinitions(RawSelector.toSelectables(rawSelectors, cfm), cfm, defs);
+        SelectionColumnMapping mapping = collectColumnMappings(cfm, rawSelectors, factories);
+
+        return (processesSelection(rawSelectors) || rawSelectors.size() != defs.size())
+               ? new SelectionWithProcessing(cfm, defs, mapping, factories)
+               : new SimpleSelection(cfm, defs, mapping, false);
+    }
+
+    /**
+     * Returns the index of the specified column within the resultset
+     * @param c the column
+     * @return the index of the specified column within the resultset or -1
+     */
+    public int getResultSetIndex(ColumnDefinition c)
+    {
+        return getColumnIndex(c);
+    }
+
+    /**
+     * Returns the index of the specified column
+     * @param c the column
+     * @return the index of the specified column or -1
+     */
+    protected final int getColumnIndex(ColumnDefinition c)
+    {
+        for (int i = 0, m = columns.size(); i < m; i++)
+            if (columns.get(i).name.equals(c.name))
+                return i;
+        return -1;
+    }
+
+    private static SelectionColumnMapping collectColumnMappings(CFMetaData cfm,
+                                                                List<RawSelector> rawSelectors,
+                                                                SelectorFactories factories)
+    {
+        SelectionColumnMapping selectionColumns = SelectionColumnMapping.newMapping();
+        Iterator<RawSelector> iter = rawSelectors.iterator();
+        for (Selector.Factory factory : factories)
+        {
+            ColumnSpecification colSpec = factory.getColumnSpecification(cfm);
+            ColumnIdentifier alias = iter.next().alias;
+            factory.addColumnMapping(selectionColumns,
+                                     alias == null ? colSpec : colSpec.withAlias(alias));
+        }
+        return selectionColumns;
+    }
+
+    protected abstract Selectors newSelectors() throws InvalidRequestException;
+
+    /**
+     * @return the list of CQL3 columns value this SelectionClause needs.
+     */
+    public List<ColumnDefinition> getColumns()
+    {
+        return columns;
+    }
+
+    /**
+     * @return the mappings between resultset columns and the underlying columns
+     */
+    public SelectionColumns getColumnMapping()
+    {
+        return columnMapping;
+    }
+
+    public ResultSetBuilder resultSetBuilder(long now, boolean isJson) throws InvalidRequestException
+    {
+        return new ResultSetBuilder(now, isJson);
+    }
+
+    public abstract boolean isAggregate();
+
+    @Override
+    public String toString()
+    {
+        return Objects.toStringHelper(this)
+                .add("columns", columns)
+                .add("columnMapping", columnMapping)
+                .add("metadata", metadata)
+                .add("collectTimestamps", collectTimestamps)
+                .add("collectTTLs", collectTTLs)
+                .toString();
+    }
+
+    public class ResultSetBuilder
+    {
+        private final ResultSet resultSet;
+
+        /**
+         * As multiple thread can access a <code>Selection</code> instance each <code>ResultSetBuilder</code> will use
+         * its own <code>Selectors</code> instance.
+         */
+        private final Selectors selectors;
+
+        /*
+         * We'll build CQL3 row one by one.
+         * The currentRow is the values for the (CQL3) columns we've fetched.
+         * We also collect timestamps and ttls for the case where the writetime and
+         * ttl functions are used. Note that we might collect timestamp and/or ttls
+         * we don't care about, but since the array below are allocated just once,
+         * it doesn't matter performance wise.
+         */
+        List<ByteBuffer> current;
+        final long[] timestamps;
+        final int[] ttls;
+        final long now;
+
+        private final boolean isJson;
+
+        private ResultSetBuilder(long now, boolean isJson) throws InvalidRequestException
+        {
+            this.resultSet = new ResultSet(getResultMetadata(isJson).copy(), new ArrayList<List<ByteBuffer>>());
+            this.selectors = newSelectors();
+            this.timestamps = collectTimestamps ? new long[columns.size()] : null;
+            this.ttls = collectTTLs ? new int[columns.size()] : null;
+            this.now = now;
+            this.isJson = isJson;
+        }
+
+        public void add(ByteBuffer v)
+        {
+            current.add(v);
+        }
+
+        public void add(Cell c)
+        {
+            current.add(isDead(c) ? null : value(c));
+            if (timestamps != null)
+            {
+                timestamps[current.size() - 1] = isDead(c) ? Long.MIN_VALUE : c.timestamp();
+            }
+            if (ttls != null)
+            {
+                int ttl = -1;
+                if (!isDead(c) && c instanceof ExpiringCell)
+                    ttl = c.getLocalDeletionTime() - (int) (now / 1000);
+                ttls[current.size() - 1] = ttl;
+            }
+        }
+
+        private boolean isDead(Cell c)
+        {
+            return c == null || !c.isLive(now);
+        }
+
+        public void newRow(int protocolVersion) throws InvalidRequestException
+        {
+            if (current != null)
+            {
+                selectors.addInputRow(protocolVersion, this);
+                if (!selectors.isAggregate())
+                {
+                    resultSet.addRow(getOutputRow(protocolVersion));
+                    selectors.reset();
+                }
+            }
+            current = new ArrayList<>(columns.size());
+        }
+
+        public ResultSet build(int protocolVersion) throws InvalidRequestException
+        {
+            if (current != null)
+            {
+                selectors.addInputRow(protocolVersion, this);
+                resultSet.addRow(getOutputRow(protocolVersion));
+                selectors.reset();
+                current = null;
+            }
+
+            if (resultSet.isEmpty() && selectors.isAggregate())
+                resultSet.addRow(getOutputRow(protocolVersion));
+            return resultSet;
+        }
+
+        private List<ByteBuffer> getOutputRow(int protocolVersion)
+        {
+            List<ByteBuffer> outputRow = selectors.getOutputRow(protocolVersion);
+            return isJson ? rowToJson(outputRow, protocolVersion)
+                          : outputRow;
+        }
+
+        private List<ByteBuffer> rowToJson(List<ByteBuffer> row, int protocolVersion)
+        {
+            StringBuilder sb = new StringBuilder("{");
+            for (int i = 0; i < metadata.names.size(); i++)
+            {
+                if (i > 0)
+                    sb.append(", ");
+
+                ColumnSpecification spec = metadata.names.get(i);
+                String columnName = spec.name.toString();
+                if (!columnName.equals(columnName.toLowerCase(Locale.US)))
+                    columnName = "\"" + columnName + "\"";
+
+                ByteBuffer buffer = row.get(i);
+                sb.append('"');
+                sb.append(Json.quoteAsJsonString(columnName));
+                sb.append("\": ");
+                if (buffer == null)
+                    sb.append("null");
+                else
+                    sb.append(spec.type.toJSONString(buffer, protocolVersion));
+            }
+            sb.append("}");
+            return Collections.singletonList(UTF8Type.instance.getSerializer().serialize(sb.toString()));
+        }
+
+        private ByteBuffer value(Cell c)
+        {
+            return (c instanceof CounterCell)
+                ? ByteBufferUtil.bytes(CounterContext.instance().total(c.value()))
+                : c.value();
+        }
+    }
+
+    private static interface Selectors
+    {
+        public boolean isAggregate();
+
+        /**
+         * Adds the current row of the specified <code>ResultSetBuilder</code>.
+         *
+         * @param rs the <code>ResultSetBuilder</code>
+         * @throws InvalidRequestException
+         */
+        public void addInputRow(int protocolVersion, ResultSetBuilder rs) throws InvalidRequestException;
+
+        public List<ByteBuffer> getOutputRow(int protocolVersion) throws InvalidRequestException;
+
+        public void reset();
+    }
+
+    // Special cased selection for when no function is used (this save some allocations).
+    private static class SimpleSelection extends Selection
+    {
+        private final boolean isWildcard;
+
+        public SimpleSelection(CFMetaData cfm, List<ColumnDefinition> columns, boolean isWildcard)
+        {
+            this(cfm, columns, SelectionColumnMapping.simpleMapping(columns), isWildcard);
+        }
+
+        public SimpleSelection(CFMetaData cfm,
+                               List<ColumnDefinition> columns,
+                               SelectionColumnMapping metadata,
+                               boolean isWildcard)
+        {
+            /*
+             * In theory, even a simple selection could have multiple time the same column, so we
+             * could filter those duplicate out of columns. But since we're very unlikely to
+             * get much duplicate in practice, it's more efficient not to bother.
+             */
+            super(cfm, columns, metadata, false, false);
+            this.isWildcard = isWildcard;
+        }
+
+        @Override
+        public boolean isWildcard()
+        {
+            return isWildcard;
+        }
+
+        public boolean isAggregate()
+        {
+            return false;
+        }
+
+        protected Selectors newSelectors()
+        {
+            return new Selectors()
+            {
+                private List<ByteBuffer> current;
+
+                public void reset()
+                {
+                    current = null;
+                }
+
+                public List<ByteBuffer> getOutputRow(int protocolVersion)
+                {
+                    return current;
+                }
+
+                public void addInputRow(int protocolVersion, ResultSetBuilder rs) throws InvalidRequestException
+                {
+                    current = rs.current;
+                }
+
+                public boolean isAggregate()
+                {
+                    return false;
+                }
+            };
+        }
+    }
+
+    private static class SelectionWithProcessing extends Selection
+    {
+        private final SelectorFactories factories;
+
+        public SelectionWithProcessing(CFMetaData cfm,
+                                       List<ColumnDefinition> columns,
+                                       SelectionColumnMapping metadata,
+                                       SelectorFactories factories) throws InvalidRequestException
+        {
+            super(cfm,
+                  columns,
+                  metadata,
+                  factories.containsWritetimeSelectorFactory(),
+                  factories.containsTTLSelectorFactory());
+
+            this.factories = factories;
+        }
+
+        @Override
+        public Iterable<Function> getFunctions()
+        {
+            return factories.getFunctions();
+        }
+
+        @Override
+        public int getResultSetIndex(ColumnDefinition c)
+        {
+            int index = getColumnIndex(c);
+
+            if (index < 0)
+                return -1;
+
+            for (int i = 0, m = factories.size(); i < m; i++)
+                if (factories.get(i).isSimpleSelectorFactory(index))
+                    return i;
+
+            return -1;
+        }
+
+        @Override
+        public int addColumnForOrdering(ColumnDefinition c)
+        {
+            int index = super.addColumnForOrdering(c);
+            factories.addSelectorForOrdering(c, index);
+            return factories.size() - 1;
+        }
+
+        public boolean isAggregate()
+        {
+            return factories.doesAggregation();
+        }
+
+        protected Selectors newSelectors() throws InvalidRequestException
+        {
+            return new Selectors()
+            {
+                private final List<Selector> selectors = factories.newInstances();
+
+                public void reset()
+                {
+                    for (Selector selector : selectors)
+                        selector.reset();
+                }
+
+                public boolean isAggregate()
+                {
+                    return factories.doesAggregation();
+                }
+
+                public List<ByteBuffer> getOutputRow(int protocolVersion) throws InvalidRequestException
+                {
+                    List<ByteBuffer> outputRow = new ArrayList<>(selectors.size());
+
+                    for (Selector selector: selectors)
+                        outputRow.add(selector.getOutput(protocolVersion));
+
+                    return outputRow;
+                }
+
+                public void addInputRow(int protocolVersion, ResultSetBuilder rs) throws InvalidRequestException
+                {
+                    for (Selector selector : selectors)
+                        selector.addInput(protocolVersion, rs);
+                }
+            };
+        }
+
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/selection/SelectionColumnMapping.java b/src/java/org/apache/cassandra/cql3/selection/SelectionColumnMapping.java
new file mode 100644
index 0000000..8636f19
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/selection/SelectionColumnMapping.java
@@ -0,0 +1,152 @@
+/*
+ *
+ * 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.cql3.selection;
+
+import java.util.*;
+
+import com.google.common.base.Function;
+import com.google.common.base.Joiner;
+import com.google.common.base.Objects;
+import com.google.common.collect.*;
+
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.ColumnSpecification;
+
+/**
+ * Separately maintains the ColumnSpecifications and their mappings to underlying
+ * columns as we may receive null mappings. This occurs where a query result
+ * includes a column specification which does not map to any particular real
+ * column, e.g. COUNT queries or where no-arg functions like now() are used
+ */
+public class SelectionColumnMapping implements SelectionColumns
+{
+    private final ArrayList<ColumnSpecification> columnSpecifications;
+    private final HashMultimap<ColumnSpecification, ColumnDefinition> columnMappings;
+
+    private SelectionColumnMapping()
+    {
+        this.columnSpecifications = new ArrayList<>();
+        this.columnMappings = HashMultimap.create();
+    }
+
+    protected static SelectionColumnMapping newMapping()
+    {
+        return new SelectionColumnMapping();
+    }
+
+    protected static SelectionColumnMapping simpleMapping(Iterable<ColumnDefinition> columnDefinitions)
+    {
+        SelectionColumnMapping mapping = new SelectionColumnMapping();
+        for (ColumnDefinition def: columnDefinitions)
+            mapping.addMapping(def, def);
+        return mapping;
+    }
+
+    protected SelectionColumnMapping addMapping(ColumnSpecification colSpec, ColumnDefinition column)
+    {
+        columnSpecifications.add(colSpec);
+        // functions without arguments do not map to any column, so don't
+        // record any mapping in that case
+        if (column != null)
+            columnMappings.put(colSpec, column);
+        return this;
+    }
+
+    protected SelectionColumnMapping addMapping(ColumnSpecification colSpec, Iterable<ColumnDefinition> columns)
+    {
+        columnSpecifications.add(colSpec);
+        columnMappings.putAll(colSpec, columns);
+        return this;
+    }
+
+    public List<ColumnSpecification> getColumnSpecifications()
+    {
+        // return a mutable copy as we may add extra columns
+        // for ordering (CASSANDRA-4911 & CASSANDRA-8286)
+        return Lists.newArrayList(columnSpecifications);
+    }
+
+    public Multimap<ColumnSpecification, ColumnDefinition> getMappings()
+    {
+        return Multimaps.unmodifiableMultimap(columnMappings);
+    }
+
+    public boolean equals(Object obj)
+    {
+        if (obj == null)
+            return false;
+
+        if (!(obj instanceof SelectionColumnMapping))
+            return false;
+
+        SelectionColumns other = (SelectionColumns)obj;
+        return Objects.equal(columnMappings, other.getMappings())
+            && Objects.equal(columnSpecifications, other.getColumnSpecifications());
+    }
+
+    public int hashCode()
+    {
+        return Objects.hashCode(columnMappings);
+    }
+
+    public String toString()
+    {
+        final Function<ColumnDefinition, String> getDefName = new Function<ColumnDefinition, String>()
+        {
+            public String apply(ColumnDefinition def)
+            {
+                return def.name.toString();
+            }
+        };
+        Function<Map.Entry<ColumnSpecification, Collection<ColumnDefinition>>, String> mappingEntryToString =
+        new Function<Map.Entry<ColumnSpecification, Collection<ColumnDefinition>>, String>(){
+            public String apply(Map.Entry<ColumnSpecification, Collection<ColumnDefinition>> entry)
+            {
+                StringBuilder builder = new StringBuilder();
+                builder.append(entry.getKey().name.toString());
+                builder.append(":[");
+                builder.append(Joiner.on(',').join(Iterables.transform(entry.getValue(), getDefName)));
+                builder.append("]");
+                return builder.toString();
+            }
+        };
+
+        Function<ColumnSpecification, String> colSpecToString = new Function<ColumnSpecification, String>()
+        {
+            public String apply(ColumnSpecification columnSpecification)
+            {
+                return columnSpecification.name.toString();
+            }
+        };
+
+        StringBuilder builder = new StringBuilder();
+        builder.append("{ Columns:[");
+        builder.append(Joiner.on(",")
+                             .join(Iterables.transform(columnSpecifications, colSpecToString)));
+        builder.append("], Mappings:[");
+        builder.append(Joiner.on(", ")
+                             .join(Iterables.transform(columnMappings.asMap().entrySet(),
+                                                       mappingEntryToString)));
+        builder.append("] }");
+        return builder.toString();
+    }
+
+}
diff --git a/src/java/org/apache/cassandra/cql3/selection/SelectionColumns.java b/src/java/org/apache/cassandra/cql3/selection/SelectionColumns.java
new file mode 100644
index 0000000..151a2f3
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/selection/SelectionColumns.java
@@ -0,0 +1,38 @@
+/*
+ *
+ * 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.cql3.selection;
+
+import java.util.List;
+
+import com.google.common.collect.Multimap;
+
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.ColumnSpecification;
+
+/**
+ * Represents a mapping between the actual columns used to satisfy a Selection
+ * and the column definitions included in the resultset metadata for the query.
+ */
+public interface SelectionColumns
+{
+    List<ColumnSpecification> getColumnSpecifications();
+    Multimap<ColumnSpecification, ColumnDefinition> getMappings();
+}
diff --git a/src/java/org/apache/cassandra/cql3/selection/Selector.java b/src/java/org/apache/cassandra/cql3/selection/Selector.java
new file mode 100644
index 0000000..7b818b5
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/selection/Selector.java
@@ -0,0 +1,207 @@
+/*
+ * 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.cql3.selection;
+
+import java.nio.ByteBuffer;
+import java.util.Collections;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.cql3.AssignmentTestable;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.ReversedType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+/**
+ * A <code>Selector</code> is used to convert the data returned by the storage engine into the data requested by the 
+ * user. They correspond to the &lt;selector&gt; elements from the select clause.
+ * <p>Since the introduction of aggregation, <code>Selector</code>s cannot be called anymore by multiple threads 
+ * as they have an internal state.</p>
+ */
+public abstract class Selector implements AssignmentTestable
+{
+    /**
+     * A factory for <code>Selector</code> instances.
+     */
+    public static abstract class Factory
+    {
+        public Iterable<Function> getFunctions()
+        {
+            return Collections.emptySet();
+        }
+
+        /**
+         * Returns the column specification corresponding to the output value of the selector instances created by
+         * this factory.
+         *
+         * @param cfm the column family meta data
+         * @return a column specification
+         */
+        public final ColumnSpecification getColumnSpecification(CFMetaData cfm)
+        {
+            return new ColumnSpecification(cfm.ksName,
+                                           cfm.cfName,
+                                           new ColumnIdentifier(getColumnName(), true),
+                                           getReturnType());
+        }
+
+        /**
+         * Creates a new <code>Selector</code> instance.
+         *
+         * @return a new <code>Selector</code> instance
+         */
+        public abstract Selector newInstance() throws InvalidRequestException;
+
+        /**
+         * Checks if this factory creates selectors instances that creates aggregates.
+         *
+         * @return <code>true</code> if this factory creates selectors instances that creates aggregates,
+         * <code>false</code> otherwise
+         */
+        public boolean isAggregateSelectorFactory()
+        {
+            return false;
+        }
+
+        /**
+         * Checks if this factory creates <code>writetime</code> selectors instances.
+         *
+         * @return <code>true</code> if this factory creates <code>writetime</code> selectors instances,
+         * <code>false</code> otherwise
+         */
+        public boolean isWritetimeSelectorFactory()
+        {
+            return false;
+        }
+
+        /**
+         * Checks if this factory creates <code>TTL</code> selectors instances.
+         *
+         * @return <code>true</code> if this factory creates <code>TTL</code> selectors instances,
+         * <code>false</code> otherwise
+         */
+        public boolean isTTLSelectorFactory()
+        {
+            return false;
+        }
+
+        /**
+         * Checks if this factory creates <code>Selector</code>s that simply return the specified column.
+         *
+         * @param index the column index
+         * @return <code>true</code> if this factory creates <code>Selector</code>s that simply return
+         * the specified column, <code>false</code> otherwise.
+         */
+        public boolean isSimpleSelectorFactory(int index)
+        {
+            return false;
+        }
+
+        /**
+         * Returns the name of the column corresponding to the output value of the selector instances created by
+         * this factory.
+         *
+         * @return a column name
+         */
+        protected abstract String getColumnName();
+
+        /**
+         * Returns the type of the values returned by the selector instances created by this factory.
+         *
+         * @return the selector output type
+         */
+        protected abstract AbstractType<?> getReturnType();
+
+        /**
+         * Record a mapping between the ColumnDefinitions that are used by the selector
+         * instances created by this factory and a column in the ResultSet.Metadata
+         * returned with a query. In most cases, this is likely to be a 1:1 mapping,
+         * but some selector instances may utilise multiple columns (or none at all)
+         * to produce a value (i.e. functions).
+         *
+         * @param mapping the instance of the column mapping belonging to the current query's Selection
+         * @param resultsColumn the column in the ResultSet.Metadata to which the ColumnDefinitions used
+         *                      by the Selector are to be mapped
+         */
+        protected abstract void addColumnMapping(SelectionColumnMapping mapping, ColumnSpecification resultsColumn);
+    }
+
+    /**
+     * Add the current value from the specified <code>ResultSetBuilder</code>.
+     *
+     * @param protocolVersion protocol version used for serialization
+     * @param rs the <code>ResultSetBuilder</code>
+     * @throws InvalidRequestException if a problem occurs while add the input value
+     */
+    public abstract void addInput(int protocolVersion, ResultSetBuilder rs) throws InvalidRequestException;
+
+    /**
+     * Returns the selector output.
+     *
+     * @param protocolVersion protocol version used for serialization
+     * @return the selector output
+     * @throws InvalidRequestException if a problem occurs while computing the output value
+     */
+    public abstract ByteBuffer getOutput(int protocolVersion) throws InvalidRequestException;
+
+    /**
+     * Returns the <code>Selector</code> output type.
+     *
+     * @return the <code>Selector</code> output type.
+     */
+    public abstract AbstractType<?> getType();
+
+    /**
+     * Checks if this <code>Selector</code> is creating aggregates.
+     *
+     * @return <code>true</code> if this <code>Selector</code> is creating aggregates <code>false</code>
+     * otherwise.
+     */
+    public boolean isAggregate()
+    {
+        return false;
+    }
+
+    /**
+     * Reset the internal state of this <code>Selector</code>.
+     */
+    public abstract void reset();
+
+    public final AssignmentTestable.TestResult testAssignment(String keyspace, ColumnSpecification receiver)
+    {
+        // We should ignore the fact that the output type is frozen in our comparison as functions do not support
+        // frozen types for arguments
+        AbstractType<?> receiverType = receiver.type;
+        if (getType().isFrozenCollection())
+            receiverType = receiverType.freeze();
+
+        if (getType().isReversed())
+            receiverType = ReversedType.getInstance(receiverType);
+
+        if (receiverType.equals(getType()))
+            return AssignmentTestable.TestResult.EXACT_MATCH;
+
+        if (receiverType.isValueCompatibleWith(getType()))
+            return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
+
+        return AssignmentTestable.TestResult.NOT_ASSIGNABLE;
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/selection/SelectorFactories.java b/src/java/org/apache/cassandra/cql3/selection/SelectorFactories.java
new file mode 100644
index 0000000..fbbfbb5
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/selection/SelectorFactories.java
@@ -0,0 +1,214 @@
+/*
+ * 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.cql3.selection;
+
+import java.util.*;
+
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.selection.Selector.Factory;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+/**
+ * A set of <code>Selector</code> factories.
+ */
+final class SelectorFactories implements Iterable<Selector.Factory>
+{
+    /**
+     * The <code>Selector</code> factories.
+     */
+    private final List<Selector.Factory> factories;
+
+    /**
+     * <code>true</code> if one of the factory creates writetime selectors.
+     */
+    private boolean containsWritetimeFactory;
+
+    /**
+     * <code>true</code> if one of the factory creates TTL selectors.
+     */
+    private boolean containsTTLFactory;
+
+    /**
+     * The number of factories creating aggregates.
+     */
+    private int numberOfAggregateFactories;
+
+    /**
+     * Creates a new <code>SelectorFactories</code> instance and collect the column definitions.
+     *
+     * @param selectables the <code>Selectable</code>s for which the factories must be created
+     * @param cfm the Column Family Definition
+     * @param defs the collector parameter for the column definitions
+     * @return a new <code>SelectorFactories</code> instance
+     * @throws InvalidRequestException if a problem occurs while creating the factories
+     */
+    public static SelectorFactories createFactoriesAndCollectColumnDefinitions(List<Selectable> selectables,
+                                                                               CFMetaData cfm,
+                                                                               List<ColumnDefinition> defs)
+                                                                               throws InvalidRequestException
+    {
+        return new SelectorFactories(selectables, cfm, defs);
+    }
+
+    private SelectorFactories(List<Selectable> selectables,
+                              CFMetaData cfm,
+                              List<ColumnDefinition> defs)
+                              throws InvalidRequestException
+    {
+        factories = new ArrayList<>(selectables.size());
+
+        for (Selectable selectable : selectables)
+        {
+            Factory factory = selectable.newSelectorFactory(cfm, defs);
+            containsWritetimeFactory |= factory.isWritetimeSelectorFactory();
+            containsTTLFactory |= factory.isTTLSelectorFactory();
+            if (factory.isAggregateSelectorFactory())
+                ++numberOfAggregateFactories;
+            factories.add(factory);
+        }
+    }
+
+    public Iterable<Function> getFunctions()
+    {
+        Iterable<Function> functions = Collections.emptySet();
+        for (Factory factory : factories)
+            if (factory != null)
+                functions = Iterables.concat(functions, factory.getFunctions());
+        return functions;
+    }
+
+    /**
+     * Returns the factory with the specified index.
+     *
+     * @param i the factory index
+     * @return the factory with the specified index
+     */
+    public Selector.Factory get(int i)
+    {
+        return factories.get(i);
+    }
+
+    /**
+     * Adds a new <code>Selector.Factory</code> for a column that is needed only for ORDER BY purposes.
+     * @param def the column that is needed for ordering
+     * @param index the index of the column definition in the Selection's list of columns
+     */
+    public void addSelectorForOrdering(ColumnDefinition def, int index)
+    {
+        factories.add(SimpleSelector.newFactory(def, index));
+    }
+
+    /**
+     * Whether the selector built by this factory does aggregation or not (either directly or in a sub-selector).
+     *
+     * @return <code>true</code> if the selector built by this factor does aggregation, <code>false</code> otherwise.
+     */
+    public boolean doesAggregation()
+    {
+        return numberOfAggregateFactories > 0;
+    }
+
+    /**
+     * Checks if this <code>SelectorFactories</code> contains at least one factory for writetime selectors.
+     *
+     * @return <code>true</code> if this <code>SelectorFactories</code> contains at least one factory for writetime
+     * selectors, <code>false</code> otherwise.
+     */
+    public boolean containsWritetimeSelectorFactory()
+    {
+        return containsWritetimeFactory;
+    }
+
+    /**
+     * Checks if this <code>SelectorFactories</code> contains at least one factory for TTL selectors.
+     *
+     * @return <code>true</code> if this <code>SelectorFactories</code> contains at least one factory for TTL
+     * selectors, <code>false</code> otherwise.
+     */
+    public boolean containsTTLSelectorFactory()
+    {
+        return containsTTLFactory;
+    }
+
+    /**
+     * Creates a list of new <code>Selector</code> instances.
+     * @return a list of new <code>Selector</code> instances.
+     */
+    public List<Selector> newInstances() throws InvalidRequestException
+    {
+        List<Selector> selectors = new ArrayList<>(factories.size());
+        for (Selector.Factory factory : factories)
+        {
+            selectors.add(factory.newInstance());
+        }
+        return selectors;
+    }
+
+    public Iterator<Factory> iterator()
+    {
+        return factories.iterator();
+    }
+
+    /**
+     * Returns the names of the columns corresponding to the output values of the selector instances created by
+     * these factories.
+     *
+     * @return a list of column names
+     */
+    public List<String> getColumnNames()
+    {
+        return Lists.transform(factories, new com.google.common.base.Function<Selector.Factory, String>()
+        {
+            public String apply(Selector.Factory factory)
+            {
+                return factory.getColumnName();
+            }
+        });
+    }
+
+    /**
+     * Returns a list of the return types of the selector instances created by these factories.
+     *
+     * @return a list of types
+     */
+    public List<AbstractType<?>> getReturnTypes()
+    {
+        return Lists.transform(factories, new com.google.common.base.Function<Selector.Factory, AbstractType<?>>()
+        {
+            public AbstractType<?> apply(Selector.Factory factory)
+            {
+                return factory.getReturnType();
+            }
+        });
+    }
+
+    /**
+     * Returns the number of factories.
+     * @return the number of factories
+     */
+    public int size()
+    {
+        return factories.size();
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/selection/SimpleSelector.java b/src/java/org/apache/cassandra/cql3/selection/SimpleSelector.java
new file mode 100644
index 0000000..e4040fa
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/selection/SimpleSelector.java
@@ -0,0 +1,112 @@
+/*
+ * 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.cql3.selection;
+
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+public final class SimpleSelector extends Selector
+{
+    private final String columnName;
+    private final int idx;
+    private final AbstractType<?> type;
+    private ByteBuffer current;
+    private boolean isSet;
+
+    public static Factory newFactory(final ColumnDefinition def, final int idx)
+    {
+        return new Factory()
+        {
+            @Override
+            protected String getColumnName()
+            {
+                return def.name.toString();
+            }
+
+            @Override
+            protected AbstractType<?> getReturnType()
+            {
+                return def.type;
+            }
+
+            protected void addColumnMapping(SelectionColumnMapping mapping, ColumnSpecification resultColumn)
+            {
+               mapping.addMapping(resultColumn, def);
+            }
+
+            @Override
+            public Selector newInstance()
+            {
+                return new SimpleSelector(def.name.toString(), idx, def.type);
+            }
+
+            @Override
+            public boolean isSimpleSelectorFactory(int index)
+            {
+                return index == idx;
+            }
+        };
+    }
+
+    @Override
+    public void addInput(int protocolVersion, ResultSetBuilder rs) throws InvalidRequestException
+    {
+        if (!isSet)
+        {
+            isSet = true;
+            current = rs.current.get(idx);
+        }
+    }
+
+    @Override
+    public ByteBuffer getOutput(int protocolVersion) throws InvalidRequestException
+    {
+        return current;
+    }
+
+    @Override
+    public void reset()
+    {
+        isSet = false;
+        current = null;
+    }
+
+    @Override
+    public AbstractType<?> getType()
+    {
+        return type;
+    }
+
+    @Override
+    public String toString()
+    {
+        return columnName;
+    }
+
+    private SimpleSelector(String columnName, int idx, AbstractType<?> type)
+    {
+        this.columnName = columnName;
+        this.idx = idx;
+        this.type = type;
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/selection/WritetimeOrTTLSelector.java b/src/java/org/apache/cassandra/cql3/selection/WritetimeOrTTLSelector.java
new file mode 100644
index 0000000..131827f
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/selection/WritetimeOrTTLSelector.java
@@ -0,0 +1,122 @@
+/*
+ * 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.cql3.selection;
+
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.selection.Selection.ResultSetBuilder;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+final class WritetimeOrTTLSelector extends Selector
+{
+    private final String columnName;
+    private final int idx;
+    private final boolean isWritetime;
+    private ByteBuffer current;
+    private boolean isSet;
+
+    public static Factory newFactory(final ColumnDefinition def, final int idx, final boolean isWritetime)
+    {
+        return new Factory()
+        {
+            protected String getColumnName()
+            {
+                return String.format("%s(%s)", isWritetime ? "writetime" : "ttl", def.name.toString());
+            }
+
+            protected AbstractType<?> getReturnType()
+            {
+                return isWritetime ? LongType.instance : Int32Type.instance;
+            }
+
+            protected void addColumnMapping(SelectionColumnMapping mapping, ColumnSpecification resultsColumn)
+            {
+               mapping.addMapping(resultsColumn, def);
+            }
+
+            public Selector newInstance()
+            {
+                return new WritetimeOrTTLSelector(def.name.toString(), idx, isWritetime);
+            }
+
+            public boolean isWritetimeSelectorFactory()
+            {
+                return isWritetime;
+            }
+
+            public boolean isTTLSelectorFactory()
+            {
+                return !isWritetime;
+            }
+        };
+    }
+
+    public void addInput(int protocolVersion, ResultSetBuilder rs)
+    {
+        if (isSet)
+            return;
+
+        isSet = true;
+
+        if (isWritetime)
+        {
+            long ts = rs.timestamps[idx];
+            current = ts != Long.MIN_VALUE ? ByteBufferUtil.bytes(ts) : null;
+        }
+        else
+        {
+            int ttl = rs.ttls[idx];
+            current = ttl > 0 ? ByteBufferUtil.bytes(ttl) : null;
+        }
+    }
+
+    public ByteBuffer getOutput(int protocolVersion)
+    {
+        return current;
+    }
+
+    public void reset()
+    {
+        isSet = false;
+        current = null;
+    }
+
+    public AbstractType<?> getType()
+    {
+        return isWritetime ? LongType.instance : Int32Type.instance;
+    }
+
+    @Override
+    public String toString()
+    {
+        return columnName;
+    }
+
+    private WritetimeOrTTLSelector(String columnName, int idx, boolean isWritetime)
+    {
+        this.columnName = columnName;
+        this.idx = idx;
+        this.isWritetime = isWritetime;
+    }
+
+}
diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterKeyspaceStatement.java b/src/java/org/apache/cassandra/cql3/statements/AlterKeyspaceStatement.java
index e65a51e..50c3f00 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AlterKeyspaceStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterKeyspaceStatement.java
@@ -21,7 +21,7 @@
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.locator.AbstractReplicationStrategy;
 import org.apache.cassandra.service.ClientState;
@@ -57,7 +57,7 @@
         KSMetaData ksm = Schema.instance.getKSMetaData(name);
         if (ksm == null)
             throw new InvalidRequestException("Unknown keyspace " + name);
-        if (ksm.name.equalsIgnoreCase(Keyspace.SYSTEM_KS))
+        if (ksm.name.equalsIgnoreCase(SystemKeyspace.NAME))
             throw new InvalidRequestException("Cannot alter system keyspace");
 
         attrs.validate();
diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterRoleStatement.java b/src/java/org/apache/cassandra/cql3/statements/AlterRoleStatement.java
new file mode 100644
index 0000000..6134741
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterRoleStatement.java
@@ -0,0 +1,90 @@
+/*
+ * 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.cql3.statements;
+
+import org.apache.cassandra.auth.*;
+import org.apache.cassandra.auth.IRoleManager.Option;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.RoleName;
+import org.apache.cassandra.exceptions.*;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.transport.messages.ResultMessage;
+
+public class AlterRoleStatement extends AuthenticationStatement
+{
+    private final RoleResource role;
+    private final RoleOptions opts;
+
+    public AlterRoleStatement(RoleName name, RoleOptions opts)
+    {
+        this.role = RoleResource.role(name.getName());
+        this.opts = opts;
+    }
+
+    public void validate(ClientState state) throws RequestValidationException
+    {
+        opts.validate();
+
+        if (opts.isEmpty())
+            throw new InvalidRequestException("ALTER [ROLE|USER] can't be empty");
+
+        // validate login here before checkAccess to avoid leaking user existence to anonymous users.
+        state.ensureNotAnonymous();
+        if (!DatabaseDescriptor.getRoleManager().isExistingRole(role))
+            throw new InvalidRequestException(String.format("%s doesn't exist", role.getRoleName()));
+    }
+
+    public void checkAccess(ClientState state) throws UnauthorizedException
+    {
+        AuthenticatedUser user = state.getUser();
+        boolean isSuper = user.isSuper();
+
+        if (opts.getSuperuser().isPresent() && user.getRoles().contains(role))
+            throw new UnauthorizedException("You aren't allowed to alter your own superuser " +
+                                            "status or that of a role granted to you");
+
+        if (opts.getSuperuser().isPresent() && !isSuper)
+            throw new UnauthorizedException("Only superusers are allowed to alter superuser status");
+
+        // superusers can do whatever else they like
+        if (isSuper)
+            return;
+
+        // a role may only modify the subset of its own attributes as determined by IRoleManager#alterableOptions
+        if (user.getName().equals(role.getRoleName()))
+        {
+            for (Option option : opts.getOptions().keySet())
+            {
+                if (!DatabaseDescriptor.getRoleManager().alterableOptions().contains(option))
+                    throw new UnauthorizedException(String.format("You aren't allowed to alter %s", option));
+            }
+        }
+        else
+        {
+            // if not attempting to alter another role, ensure we have ALTER permissions on it
+            super.checkPermission(state, Permission.ALTER, role);
+        }
+    }
+
+    public ResultMessage execute(ClientState state) throws RequestValidationException, RequestExecutionException
+    {
+        if (!opts.isEmpty())
+            DatabaseDescriptor.getRoleManager().alterRole(state.getUser(), role, opts);
+        return null;
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
index 42b7f7f..f4a7b39 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
@@ -252,7 +252,10 @@
                         for (ColumnDefinition columnDef : cfm.regularAndStaticColumns())
                         {
                             if (columnDef.name.equals(columnName))
+                            {
                                 toDelete = columnDef;
+                                break;
+                            }
                         }
                         assert toDelete != null;
                         cfm.removeColumnDefinition(toDelete);
@@ -262,7 +265,7 @@
                 break;
             case OPTS:
                 if (cfProps == null)
-                    throw new InvalidRequestException(String.format("ALTER COLUMNFAMILY WITH invoked, but no parameters found"));
+                    throw new InvalidRequestException("ALTER TABLE WITH invoked, but no parameters found");
 
                 cfProps.validate();
 
@@ -281,7 +284,7 @@
                 break;
         }
 
-        MigrationManager.announceColumnFamilyUpdate(cfm, false, isLocalOnly);
+        MigrationManager.announceColumnFamilyUpdate(cfm, isLocalOnly);
         return true;
     }
 
diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterTypeStatement.java b/src/java/org/apache/cassandra/cql3/statements/AlterTypeStatement.java
index 6459e6b..9203cf9 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AlterTypeStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterTypeStatement.java
@@ -113,7 +113,7 @@
             for (ColumnDefinition def : copy.allColumns())
                 modified |= updateDefinition(copy, def, toUpdate.keyspace, toUpdate.name, updated);
             if (modified)
-                MigrationManager.announceColumnFamilyUpdate(copy, false, isLocalOnly);
+                MigrationManager.announceColumnFamilyUpdate(copy, isLocalOnly);
         }
 
         // Other user types potentially using the updated type
diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterUserStatement.java b/src/java/org/apache/cassandra/cql3/statements/AlterUserStatement.java
deleted file mode 100644
index 7a6e96b..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/AlterUserStatement.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/*
- * 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.cql3.statements;
-
-import org.apache.cassandra.auth.Auth;
-import org.apache.cassandra.auth.AuthenticatedUser;
-import org.apache.cassandra.auth.IAuthenticator;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.cql3.UserOptions;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.RequestExecutionException;
-import org.apache.cassandra.exceptions.RequestValidationException;
-import org.apache.cassandra.exceptions.UnauthorizedException;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.transport.messages.ResultMessage;
-
-public class AlterUserStatement extends AuthenticationStatement
-{
-    private final String username;
-    private final UserOptions opts;
-    private final Boolean superuser;
-
-    public AlterUserStatement(String username, UserOptions opts, Boolean superuser)
-    {
-        this.username = username;
-        this.opts = opts;
-        this.superuser = superuser;
-    }
-
-    public void validate(ClientState state) throws RequestValidationException
-    {
-        opts.validate();
-
-        if (superuser == null && opts.isEmpty())
-            throw new InvalidRequestException("ALTER USER can't be empty");
-
-        // validate login here before checkAccess to avoid leaking user existence to anonymous users.
-        state.ensureNotAnonymous();
-
-        if (!Auth.isExistingUser(username))
-            throw new InvalidRequestException(String.format("User %s doesn't exist", username));
-    }
-
-    public void checkAccess(ClientState state) throws UnauthorizedException
-    {
-        AuthenticatedUser user = state.getUser();
-
-        boolean isSuper = user.isSuper();
-
-        if (superuser != null && user.getName().equals(username))
-            throw new UnauthorizedException("You aren't allowed to alter your own superuser status");
-
-        if (superuser != null && !isSuper)
-            throw new UnauthorizedException("Only superusers are allowed to alter superuser status");
-
-        if (!user.isSuper() && !user.getName().equals(username))
-            throw new UnauthorizedException("You aren't allowed to alter this user");
-
-        if (!isSuper)
-        {
-            for (IAuthenticator.Option option : opts.getOptions().keySet())
-            {
-                if (!DatabaseDescriptor.getAuthenticator().alterableOptions().contains(option))
-                    throw new UnauthorizedException(String.format("You aren't allowed to alter %s option", option));
-            }
-        }
-    }
-
-    public ResultMessage execute(ClientState state) throws RequestValidationException, RequestExecutionException
-    {
-        if (!opts.isEmpty())
-            DatabaseDescriptor.getAuthenticator().alter(username, opts.getOptions());
-        if (superuser != null)
-            Auth.insertUser(username, superuser.booleanValue());
-        return null;
-    }
-}
diff --git a/src/java/org/apache/cassandra/cql3/statements/AuthenticationStatement.java b/src/java/org/apache/cassandra/cql3/statements/AuthenticationStatement.java
index b47dd92..151e4f0 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AuthenticationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AuthenticationStatement.java
@@ -17,9 +17,13 @@
  */
 package org.apache.cassandra.cql3.statements;
 
+import org.apache.cassandra.auth.Permission;
+import org.apache.cassandra.auth.RoleResource;
 import org.apache.cassandra.cql3.CQLStatement;
 import org.apache.cassandra.cql3.QueryOptions;
-import org.apache.cassandra.exceptions.*;
+import org.apache.cassandra.exceptions.RequestExecutionException;
+import org.apache.cassandra.exceptions.RequestValidationException;
+import org.apache.cassandra.exceptions.UnauthorizedException;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.transport.messages.ResultMessage;
@@ -50,5 +54,20 @@
         // executeInternal is for local query only, thus altering users doesn't make sense and is not supported
         throw new UnsupportedOperationException();
     }
+
+    public void checkPermission(ClientState state, Permission required, RoleResource resource) throws UnauthorizedException
+    {
+        try
+        {
+            state.ensureHasPermission(required, resource);
+        }
+        catch (UnauthorizedException e)
+        {
+            // Catch and rethrow with a more friendly message
+            throw new UnauthorizedException(String.format("User %s does not have sufficient privileges " +
+                                                          "to perform the requested operation",
+                                                          state.getUser().getName()));
+        }
+    }
 }
 
diff --git a/src/java/org/apache/cassandra/cql3/statements/AuthorizationStatement.java b/src/java/org/apache/cassandra/cql3/statements/AuthorizationStatement.java
index 2c7f2cb..098e22c 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AuthorizationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AuthorizationStatement.java
@@ -19,9 +19,12 @@
 
 
 import org.apache.cassandra.auth.DataResource;
+import org.apache.cassandra.auth.IResource;
 import org.apache.cassandra.cql3.CQLStatement;
 import org.apache.cassandra.cql3.QueryOptions;
-import org.apache.cassandra.exceptions.*;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.exceptions.RequestExecutionException;
+import org.apache.cassandra.exceptions.RequestValidationException;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.transport.messages.ResultMessage;
@@ -53,10 +56,14 @@
         throw new UnsupportedOperationException();
     }
 
-    public static DataResource maybeCorrectResource(DataResource resource, ClientState state) throws InvalidRequestException
+    public static IResource maybeCorrectResource(IResource resource, ClientState state) throws InvalidRequestException
     {
-        if (resource.isColumnFamilyLevel() && resource.getKeyspace() == null)
-            return DataResource.columnFamily(state.getKeyspace(), resource.getColumnFamily());
+        if (DataResource.class.isInstance(resource))
+        {
+            DataResource dataResource = (DataResource) resource;
+            if (dataResource.isTableLevel() && dataResource.getKeyspace() == null)
+                return DataResource.table(state.getKeyspace(), dataResource.getTable());
+        }
         return resource;
     }
 }
diff --git a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
index ada8d91..76e389b 100644
--- a/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/BatchStatement.java
@@ -22,21 +22,23 @@
 import java.util.concurrent.TimeUnit;
 
 import com.google.common.base.Function;
-import com.google.common.collect.*;
-
-import org.apache.cassandra.config.DatabaseDescriptor;
+import com.google.common.collect.Iterables;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.slf4j.helpers.MessageFormatter;
 
 import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.composites.Composite;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.ClientWarn;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.service.StorageProxy;
+import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.transport.messages.ResultMessage;
 import org.apache.cassandra.utils.NoSpamLogger;
 import org.apache.cassandra.utils.Pair;
@@ -81,6 +83,14 @@
         this.hasConditions = hasConditions;
     }
 
+    public Iterable<org.apache.cassandra.cql3.functions.Function> getFunctions()
+    {
+        Iterable<org.apache.cassandra.cql3.functions.Function> functions = attrs.getFunctions();
+        for (ModificationStatement statement : statements)
+            functions = Iterables.concat(functions, statement.getFunctions());
+        return functions;
+    }
+
     public int getBoundTerms()
     {
         return boundTerms;
@@ -235,10 +245,11 @@
      *
      * @param cfs ColumnFamilies that will store the batch's mutations.
      */
-    public static void verifyBatchSize(Iterable<ColumnFamily> cfs)
+    public static void verifyBatchSize(Iterable<ColumnFamily> cfs) throws InvalidRequestException
     {
         long size = 0;
         long warnThreshold = DatabaseDescriptor.getBatchSizeWarnThreshold();
+        long failThreshold = DatabaseDescriptor.getBatchSizeFailThreshold();
 
         for (ColumnFamily cf : cfs)
             size += cf.dataSize();
@@ -249,8 +260,18 @@
             for (ColumnFamily cf : cfs)
                 ksCfPairs.add(String.format("%s.%s", cf.metadata().ksName, cf.metadata().cfName));
 
-            String format = "Batch of prepared statements for {} is of size {}, exceeding specified threshold of {} by {}.";
-            logger.warn(format, ksCfPairs, size, warnThreshold, size - warnThreshold);
+            String format = "Batch of prepared statements for {} is of size {}, exceeding specified threshold of {} by {}.{}";
+            if (size > failThreshold)
+            {
+                Tracing.trace(format, ksCfPairs, size, failThreshold, size - failThreshold, " (see batch_size_fail_threshold_in_kb)");
+                logger.error(format, ksCfPairs, size, failThreshold, size - failThreshold, " (see batch_size_fail_threshold_in_kb)");
+                throw new InvalidRequestException("Batch too large");
+            }
+            else if (logger.isWarnEnabled())
+            {
+                logger.warn(format, ksCfPairs, size, warnThreshold, size - warnThreshold, "");
+            }
+            ClientWarn.instance.warn(MessageFormatter.arrayFormat(format, new Object[]{ ksCfPairs, size, warnThreshold, size - warnThreshold, "" }).getMessage());
         }
     }
 
@@ -275,6 +296,13 @@
             {
                 NoSpamLogger.log(logger, NoSpamLogger.Level.WARN, 1, TimeUnit.MINUTES, unloggedBatchWarning,
                                  keySet.size(), ksCfPairs.size() == 1 ? "" : "s", ksCfPairs);
+
+                ClientWarn.instance.warn(MessageFormatter.arrayFormat(unloggedBatchWarning,
+                                                                  new Object[]{
+                                                                              keySet.size(),
+                                                                              ksCfPairs.size() == 1 ? "" : "s",
+                                                                              ksCfPairs
+                                                                  }).getMessage());
             }
         }
     }
@@ -348,7 +376,6 @@
     }
 
     private Pair<CQL3CasRequest,Set<ColumnDefinition>> makeCasRequest(BatchQueryOptions options, QueryState state)
-    throws InvalidRequestException
     {
         long now = state.getTimestamp();
         ByteBuffer key = null;
@@ -464,9 +491,25 @@
         {
             VariableSpecifications boundNames = getBoundVariables();
 
+            String firstKS = null;
+            String firstCF = null;
+            boolean haveMultipleCFs = false;
+
             List<ModificationStatement> statements = new ArrayList<>(parsedStatements.size());
             for (ModificationStatement.Parsed parsed : parsedStatements)
+            {
+                if (firstKS == null)
+                {
+                    firstKS = parsed.keyspace();
+                    firstCF = parsed.columnFamily();
+                }
+                else if (!haveMultipleCFs)
+                {
+                    haveMultipleCFs = !firstKS.equals(parsed.keyspace()) || !firstCF.equals(parsed.columnFamily());
+                }
+
                 statements.add(parsed.prepare(boundNames));
+            }
 
             Attributes prepAttrs = attrs.prepare("[batch]", "[batch]");
             prepAttrs.collectMarkerSpecification(boundNames);
@@ -474,7 +517,12 @@
             BatchStatement batchStatement = new BatchStatement(boundNames.size(), type, statements, prepAttrs);
             batchStatement.validate();
 
-            return new ParsedStatement.Prepared(batchStatement, boundNames);
+            // Use the CFMetadata of the first statement for partition key bind indexes.  If the statements affect
+            // multiple tables, we won't send partition key bind indexes.
+            Short[] partitionKeyBindIndexes = (haveMultipleCFs || batchStatement.statements.isEmpty())? null
+                                                              : boundNames.getPartitionKeyBindIndexes(batchStatement.statements.get(0).cfm);
+
+            return new ParsedStatement.Prepared(batchStatement, boundNames, partitionKeyBindIndexes);
         }
     }
 }
diff --git a/src/java/org/apache/cassandra/cql3/statements/Bound.java b/src/java/org/apache/cassandra/cql3/statements/Bound.java
index 8aa3a37..7742642 100644
--- a/src/java/org/apache/cassandra/cql3/statements/Bound.java
+++ b/src/java/org/apache/cassandra/cql3/statements/Bound.java
@@ -28,8 +28,18 @@
         this.idx = idx;
     }
 
-    public static Bound reverse(Bound b)
+    public Bound reverse()
     {
-        return b == START ? END : START;
+        return isStart() ? END : START;
+    }
+
+    public boolean isStart()
+    {
+        return this == START;
+    }
+
+    public boolean isEnd()
+    {
+        return this == END;
     }
 }
diff --git a/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java b/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java
index 17edd6d..c02e78e 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java
@@ -26,6 +26,7 @@
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.io.compress.CompressionParameters;
+import org.apache.cassandra.utils.BloomCalculations;
 
 public class CFPropDefs extends PropertyDefinitions
 {
@@ -147,7 +148,7 @@
     {
         Map<String, String> compactionOptions = getMap(KW_COMPACTION);
         if (compactionOptions == null)
-            return new HashMap<>();
+            return Collections.emptyMap();
         return compactionOptions;
     }
 
@@ -155,7 +156,7 @@
     {
         Map<String, String> compressionOptions = getMap(KW_COMPRESSION);
         if (compressionOptions == null)
-            return new HashMap<>();
+            return Collections.emptyMap();
         return compressionOptions;
     }
     public CachingOptions getCachingOptions() throws SyntaxException, ConfigurationException
@@ -191,7 +192,7 @@
             cfm.comment(getString(KW_COMMENT, ""));
 
         cfm.readRepairChance(getDouble(KW_READREPAIRCHANCE, cfm.getReadRepairChance()));
-        cfm.dcLocalReadRepairChance(getDouble(KW_DCLOCALREADREPAIRCHANCE, cfm.getDcLocalReadRepair()));
+        cfm.dcLocalReadRepairChance(getDouble(KW_DCLOCALREADREPAIRCHANCE, cfm.getDcLocalReadRepairChance()));
         cfm.gcGraceSeconds(getInt(KW_GCGRACESECONDS, cfm.getGcGraceSeconds()));
         int minCompactionThreshold = toInt(KW_MINCOMPACTIONTHRESHOLD, getCompactionOptions().get(KW_MINCOMPACTIONTHRESHOLD), cfm.getMinCompactionThreshold());
         int maxCompactionThreshold = toInt(KW_MAXCOMPACTIONTHRESHOLD, getCompactionOptions().get(KW_MAXCOMPACTIONTHRESHOLD), cfm.getMaxCompactionThreshold());
@@ -211,7 +212,17 @@
             cfm.compactionStrategyOptions(new HashMap<>(getCompactionOptions()));
         }
 
-        cfm.bloomFilterFpChance(getDouble(KW_BF_FP_CHANCE, cfm.getBloomFilterFpChance()));
+        double bloomFilterFpChance = getDouble(KW_BF_FP_CHANCE, cfm.getBloomFilterFpChance());
+        double minBloomFilterFpChanceValue = BloomCalculations.minSupportedBloomFilterFpChance();
+        if (bloomFilterFpChance <=  minBloomFilterFpChanceValue || bloomFilterFpChance > 1)
+        {
+            throw new ConfigurationException(String.format(
+                    "%s must be larger than %s and less than or equal to 1.0 (got %s)",
+                    KW_BF_FP_CHANCE,
+                    minBloomFilterFpChanceValue,
+                    bloomFilterFpChance));
+        }
+        cfm.bloomFilterFpChance(bloomFilterFpChance);
 
         if (!getCompressionOptions().isEmpty())
             cfm.compressionParameters(CompressionParameters.create(getCompressionOptions()));
diff --git a/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java b/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java
index 3ea7cd8..081a14e 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java
@@ -27,7 +27,6 @@
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.composites.Composite;
 import org.apache.cassandra.db.filter.*;
-import org.apache.cassandra.db.marshal.CompositeType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.service.CASRequest;
 import org.apache.cassandra.utils.Pair;
@@ -82,7 +81,7 @@
     {
         RowCondition previous = conditions.put(prefix, new ExistCondition(prefix, now));
         // this should be prevented by the parser, but it doesn't hurt to check
-        if (previous != null && previous instanceof NotExistCondition)
+        if (previous instanceof NotExistCondition)
             throw new InvalidRequestException("Cannot mix IF EXISTS and IF NOT EXISTS conditions for the same row");
     }
 
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateAggregateStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateAggregateStatement.java
new file mode 100644
index 0000000..5ee7e33
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateAggregateStatement.java
@@ -0,0 +1,255 @@
+/*
+ * 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.cql3.statements;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.cassandra.auth.*;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.cql3.functions.*;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.exceptions.*;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.MigrationManager;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.thrift.ThriftValidation;
+import org.apache.cassandra.transport.Event;
+
+/**
+ * A {@code CREATE AGGREGATE} statement parsed from a CQL query.
+ */
+public final class CreateAggregateStatement extends SchemaAlteringStatement
+{
+    private final boolean orReplace;
+    private final boolean ifNotExists;
+    private FunctionName functionName;
+    private FunctionName stateFunc;
+    private FunctionName finalFunc;
+    private final CQL3Type.Raw stateTypeRaw;
+
+    private final List<CQL3Type.Raw> argRawTypes;
+    private final Term.Raw ival;
+
+    private UDAggregate udAggregate;
+    private boolean replaced;
+
+    private List<AbstractType<?>> argTypes;
+    private AbstractType<?> returnType;
+    private ScalarFunction stateFunction;
+    private ScalarFunction finalFunction;
+    private ByteBuffer initcond;
+
+    public CreateAggregateStatement(FunctionName functionName,
+                                    List<CQL3Type.Raw> argRawTypes,
+                                    String stateFunc,
+                                    CQL3Type.Raw stateType,
+                                    String finalFunc,
+                                    Term.Raw ival,
+                                    boolean orReplace,
+                                    boolean ifNotExists)
+    {
+        this.functionName = functionName;
+        this.argRawTypes = argRawTypes;
+        this.stateFunc = new FunctionName(functionName.keyspace, stateFunc);
+        this.finalFunc = finalFunc != null ? new FunctionName(functionName.keyspace, finalFunc) : null;
+        this.stateTypeRaw = stateType;
+        this.ival = ival;
+        this.orReplace = orReplace;
+        this.ifNotExists = ifNotExists;
+    }
+
+    public Prepared prepare()
+    {
+        argTypes = new ArrayList<>(argRawTypes.size());
+        for (CQL3Type.Raw rawType : argRawTypes)
+            argTypes.add(prepareType("arguments", rawType));
+
+        AbstractType<?> stateType = prepareType("state type", stateTypeRaw);
+
+        List<AbstractType<?>> stateArgs = stateArguments(stateType, argTypes);
+
+        Function f = Functions.find(stateFunc, stateArgs);
+        if (!(f instanceof ScalarFunction))
+            throw new InvalidRequestException("State function " + stateFuncSig(stateFunc, stateTypeRaw, argRawTypes) + " does not exist or is not a scalar function");
+        stateFunction = (ScalarFunction)f;
+
+        AbstractType<?> stateReturnType = stateFunction.returnType();
+        if (!stateReturnType.equals(stateType))
+            throw new InvalidRequestException("State function " + stateFuncSig(stateFunction.name(), stateTypeRaw, argRawTypes) + " return type must be the same as the first argument type - check STYPE, argument and return types");
+
+        if (finalFunc != null)
+        {
+            List<AbstractType<?>> finalArgs = Collections.<AbstractType<?>>singletonList(stateType);
+            f = Functions.find(finalFunc, finalArgs);
+            if (!(f instanceof ScalarFunction))
+                throw new InvalidRequestException("Final function " + finalFunc + '(' + stateTypeRaw + ") does not exist or is not a scalar function");
+            finalFunction = (ScalarFunction) f;
+            returnType = finalFunction.returnType();
+        }
+        else
+        {
+            returnType = stateReturnType;
+        }
+
+        if (ival != null)
+        {
+            ColumnSpecification receiver = new ColumnSpecification(functionName.keyspace, "--dummy--", new ColumnIdentifier("(aggregate_initcond)", true), stateType);
+            initcond = ival.prepare(functionName.keyspace, receiver).bindAndGet(QueryOptions.DEFAULT);
+            if (Constants.NULL_LITERAL != ival && UDHelper.isNullOrEmpty(stateType, initcond))
+                throw new InvalidRequestException("INITCOND must not be empty for all types except TEXT, ASCII, BLOB");
+        }
+
+        return super.prepare();
+    }
+
+    private AbstractType<?> prepareType(String typeName, CQL3Type.Raw rawType)
+    {
+        if (rawType.isFrozen())
+            throw new InvalidRequestException(String.format("The function %s should not be frozen; remove the frozen<> modifier", typeName));
+
+        // UDT are not supported non frozen but we do not allow the frozen keyword for argument. So for the moment we
+        // freeze them here
+        if (!rawType.canBeNonFrozen())
+            rawType.freeze();
+
+        AbstractType<?> type = rawType.prepare(functionName.keyspace).getType();
+        return type;
+    }
+
+    public void prepareKeyspace(ClientState state) throws InvalidRequestException
+    {
+        if (!functionName.hasKeyspace() && state.getRawKeyspace() != null)
+            functionName = new FunctionName(state.getKeyspace(), functionName.name);
+
+        if (!functionName.hasKeyspace())
+            throw new InvalidRequestException("Functions must be fully qualified with a keyspace name if a keyspace is not set for the session");
+
+        ThriftValidation.validateKeyspaceNotSystem(functionName.keyspace);
+
+        stateFunc = new FunctionName(functionName.keyspace, stateFunc.name);
+        if (finalFunc != null)
+            finalFunc = new FunctionName(functionName.keyspace, finalFunc.name);
+    }
+
+    protected void grantPermissionsToCreator(QueryState state)
+    {
+        try
+        {
+            IResource resource = FunctionResource.function(functionName.keyspace, functionName.name, argTypes);
+            DatabaseDescriptor.getAuthorizer().grant(AuthenticatedUser.SYSTEM_USER,
+                                                     resource.applicablePermissions(),
+                                                     resource,
+                                                     RoleResource.role(state.getClientState().getUser().getName()));
+        }
+        catch (RequestExecutionException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
+    {
+        if (Functions.find(functionName, argTypes) != null && orReplace)
+            state.ensureHasPermission(Permission.ALTER, FunctionResource.function(functionName.keyspace,
+                                                                                  functionName.name,
+                                                                                  argTypes));
+        else
+            state.ensureHasPermission(Permission.CREATE, FunctionResource.keyspace(functionName.keyspace));
+
+        for (Function referencedFunction : stateFunction.getFunctions())
+            state.ensureHasPermission(Permission.EXECUTE, referencedFunction);
+
+        if (finalFunction != null)
+            for (Function referencedFunction : finalFunction.getFunctions())
+                state.ensureHasPermission(Permission.EXECUTE, referencedFunction);
+    }
+
+    public void validate(ClientState state) throws InvalidRequestException
+    {
+        if (ifNotExists && orReplace)
+            throw new InvalidRequestException("Cannot use both 'OR REPLACE' and 'IF NOT EXISTS' directives");
+
+        if (Schema.instance.getKSMetaData(functionName.keyspace) == null)
+            throw new InvalidRequestException(String.format("Cannot add aggregate '%s' to non existing keyspace '%s'.", functionName.name, functionName.keyspace));
+    }
+
+    public Event.SchemaChange changeEvent()
+    {
+        return new Event.SchemaChange(replaced ? Event.SchemaChange.Change.UPDATED : Event.SchemaChange.Change.CREATED,
+                                      Event.SchemaChange.Target.AGGREGATE,
+                                      udAggregate.name().keyspace, udAggregate.name().name, AbstractType.asCQLTypeStringList(udAggregate.argTypes()));
+    }
+
+    public boolean announceMigration(boolean isLocalOnly) throws RequestValidationException
+    {
+        Function old = Functions.find(functionName, argTypes);
+        if (old != null)
+        {
+            if (ifNotExists)
+                return false;
+            if (!orReplace)
+                throw new InvalidRequestException(String.format("Function %s already exists", old));
+            if (!(old instanceof AggregateFunction))
+                throw new InvalidRequestException(String.format("Aggregate %s can only replace an aggregate", old));
+
+            // Means we're replacing the function. We still need to validate that 1) it's not a native function and 2) that the return type
+            // matches (or that could break existing code badly)
+            if (old.isNative())
+                throw new InvalidRequestException(String.format("Cannot replace native aggregate %s", old));
+            if (!old.returnType().isValueCompatibleWith(returnType))
+                throw new InvalidRequestException(String.format("Cannot replace aggregate %s, the new return type %s is not compatible with the return type %s of existing function",
+                                                                functionName, returnType.asCQL3Type(), old.returnType().asCQL3Type()));
+        }
+
+        if (!stateFunction.isCalledOnNullInput() && initcond == null)
+            throw new InvalidRequestException(String.format("Cannot create aggregate %s without INITCOND because state function %s does not accept 'null' arguments", functionName, stateFunc));
+
+        udAggregate = new UDAggregate(functionName, argTypes, returnType,
+                                                  stateFunction,
+                                                  finalFunction,
+                                                  initcond);
+        replaced = old != null;
+
+        MigrationManager.announceNewAggregate(udAggregate, isLocalOnly);
+
+        return true;
+    }
+
+    private static String stateFuncSig(FunctionName stateFuncName, CQL3Type.Raw stateTypeRaw, List<CQL3Type.Raw> argRawTypes)
+    {
+        StringBuilder sb = new StringBuilder();
+        sb.append(stateFuncName.toString()).append('(').append(stateTypeRaw);
+        for (CQL3Type.Raw argRawType : argRawTypes)
+            sb.append(", ").append(argRawType);
+        sb.append(')');
+        return sb.toString();
+    }
+
+    private static List<AbstractType<?>> stateArguments(AbstractType<?> stateType, List<AbstractType<?>> argTypes)
+    {
+        List<AbstractType<?>> r = new ArrayList<>(argTypes.size() + 1);
+        r.add(stateType);
+        r.addAll(argTypes);
+        return r;
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateFunctionStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateFunctionStatement.java
new file mode 100644
index 0000000..77e41ed
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateFunctionStatement.java
@@ -0,0 +1,194 @@
+/*
+ * 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.cql3.statements;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+
+import org.apache.cassandra.auth.*;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.functions.*;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.exceptions.*;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.MigrationManager;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.thrift.ThriftValidation;
+import org.apache.cassandra.transport.Event;
+
+/**
+ * A {@code CREATE FUNCTION} statement parsed from a CQL query.
+ */
+public final class CreateFunctionStatement extends SchemaAlteringStatement
+{
+    private final boolean orReplace;
+    private final boolean ifNotExists;
+    private FunctionName functionName;
+    private final String language;
+    private final String body;
+
+    private final List<ColumnIdentifier> argNames;
+    private final List<CQL3Type.Raw> argRawTypes;
+    private final CQL3Type.Raw rawReturnType;
+    private final boolean calledOnNullInput;
+
+    private List<AbstractType<?>> argTypes;
+    private AbstractType<?> returnType;
+    private UDFunction udFunction;
+    private boolean replaced;
+
+    public CreateFunctionStatement(FunctionName functionName,
+                                   String language,
+                                   String body,
+                                   List<ColumnIdentifier> argNames,
+                                   List<CQL3Type.Raw> argRawTypes,
+                                   CQL3Type.Raw rawReturnType,
+                                   boolean calledOnNullInput,
+                                   boolean orReplace,
+                                   boolean ifNotExists)
+    {
+        this.functionName = functionName;
+        this.language = language;
+        this.body = body;
+        this.argNames = argNames;
+        this.argRawTypes = argRawTypes;
+        this.rawReturnType = rawReturnType;
+        this.calledOnNullInput = calledOnNullInput;
+        this.orReplace = orReplace;
+        this.ifNotExists = ifNotExists;
+    }
+
+    public Prepared prepare() throws InvalidRequestException
+    {
+        if (new HashSet<>(argNames).size() != argNames.size())
+            throw new InvalidRequestException(String.format("duplicate argument names for given function %s with argument names %s",
+                                                            functionName, argNames));
+
+        argTypes = new ArrayList<>(argRawTypes.size());
+        for (CQL3Type.Raw rawType : argRawTypes)
+            argTypes.add(prepareType("arguments", rawType));
+
+        returnType = prepareType("return type", rawReturnType);
+        return super.prepare();
+    }
+
+    public void prepareKeyspace(ClientState state) throws InvalidRequestException
+    {
+        if (!functionName.hasKeyspace() && state.getRawKeyspace() != null)
+            functionName = new FunctionName(state.getRawKeyspace(), functionName.name);
+
+        if (!functionName.hasKeyspace())
+            throw new InvalidRequestException("Functions must be fully qualified with a keyspace name if a keyspace is not set for the session");
+
+        ThriftValidation.validateKeyspaceNotSystem(functionName.keyspace);
+    }
+
+    protected void grantPermissionsToCreator(QueryState state)
+    {
+        try
+        {
+            IResource resource = FunctionResource.function(functionName.keyspace, functionName.name, argTypes);
+            DatabaseDescriptor.getAuthorizer().grant(AuthenticatedUser.SYSTEM_USER,
+                                                     resource.applicablePermissions(),
+                                                     resource,
+                                                     RoleResource.role(state.getClientState().getUser().getName()));
+        }
+        catch (RequestExecutionException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
+    {
+        if (Functions.find(functionName, argTypes) != null && orReplace)
+            state.ensureHasPermission(Permission.ALTER, FunctionResource.function(functionName.keyspace,
+                                                                                  functionName.name,
+                                                                                  argTypes));
+        else
+            state.ensureHasPermission(Permission.CREATE, FunctionResource.keyspace(functionName.keyspace));
+    }
+
+    public void validate(ClientState state) throws InvalidRequestException
+    {
+        if (!DatabaseDescriptor.enableUserDefinedFunctions())
+            throw new InvalidRequestException("User-defined-functions are disabled in cassandra.yaml - set enable_user_defined_functions=true to enable if you are aware of the security risks");
+
+        if (ifNotExists && orReplace)
+            throw new InvalidRequestException("Cannot use both 'OR REPLACE' and 'IF NOT EXISTS' directives");
+
+        if (Schema.instance.getKSMetaData(functionName.keyspace) == null)
+            throw new InvalidRequestException(String.format("Cannot add function '%s' to non existing keyspace '%s'.", functionName.name, functionName.keyspace));
+    }
+
+    public Event.SchemaChange changeEvent()
+    {
+        return new Event.SchemaChange(replaced ? Event.SchemaChange.Change.UPDATED : Event.SchemaChange.Change.CREATED,
+                                      Event.SchemaChange.Target.FUNCTION,
+                                      udFunction.name().keyspace, udFunction.name().name, AbstractType.asCQLTypeStringList(udFunction.argTypes()));
+    }
+
+    public boolean announceMigration(boolean isLocalOnly) throws RequestValidationException
+    {
+        Function old = Functions.find(functionName, argTypes);
+        if (old != null)
+        {
+            if (ifNotExists)
+                return false;
+            if (!orReplace)
+                throw new InvalidRequestException(String.format("Function %s already exists", old));
+            if (!(old instanceof ScalarFunction))
+                throw new InvalidRequestException(String.format("Function %s can only replace a function", old));
+            if (calledOnNullInput != ((ScalarFunction) old).isCalledOnNullInput())
+                throw new InvalidRequestException(String.format("Function %s can only be replaced with %s", old,
+                                                                calledOnNullInput ? "CALLED ON NULL INPUT" : "RETURNS NULL ON NULL INPUT"));
+
+            if (!Functions.typeEquals(old.returnType(), returnType))
+                throw new InvalidRequestException(String.format("Cannot replace function %s, the new return type %s is not compatible with the return type %s of existing function",
+                                                                functionName, returnType.asCQL3Type(), old.returnType().asCQL3Type()));
+        }
+
+        this.udFunction = UDFunction.create(functionName, argNames, argTypes, returnType, calledOnNullInput, language, body);
+        this.replaced = old != null;
+
+        // add function to registry to prevent duplicate compilation on coordinator during migration
+        Functions.addOrReplaceFunction(udFunction);
+
+        MigrationManager.announceNewFunction(udFunction, isLocalOnly);
+
+        return true;
+    }
+
+    private AbstractType<?> prepareType(String typeName, CQL3Type.Raw rawType)
+    {
+        if (rawType.isFrozen())
+            throw new InvalidRequestException(String.format("The function %s should not be frozen; remove the frozen<> modifier", typeName));
+
+        // UDT are not supported non frozen but we do not allow the frozen keyword for argument. So for the moment we
+        // freeze them here
+        if (!rawType.canBeNonFrozen())
+            rawType.freeze();
+
+        AbstractType<?> type = rawType.prepare(functionName.keyspace).getType();
+        return type;
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
index 72174fb..d93c0a7 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java
@@ -29,7 +29,6 @@
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.config.IndexType;
 import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.db.index.SecondaryIndex;
 import org.apache.cassandra.db.marshal.MapType;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.cql3.*;
@@ -80,34 +79,28 @@
 
         boolean isMap = cd.type instanceof MapType;
         boolean isFrozenCollection = cd.type.isCollection() && !cd.type.isMultiCell();
-        if (target.isCollectionKeys)
+
+        if (isFrozenCollection)
         {
-            if (!isMap)
-                throw new InvalidRequestException("Cannot create index on keys of column " + target + " with non-map type");
-            if (!cd.type.isMultiCell())
-                throw new InvalidRequestException("Cannot create index on keys of frozen<map> column " + target);
+            validateForFrozenCollection(target);
         }
-        else if (target.isFullCollection)
+        else
         {
-            if (!isFrozenCollection)
-                throw new InvalidRequestException("full() indexes can only be created on frozen collections");
-        }
-        else if (isFrozenCollection)
-        {
-            throw new InvalidRequestException("Frozen collections currently only support full-collection indexes. " +
-                                              "For example, 'CREATE INDEX ON <table>(full(<columnName>))'.");
+            validateNotFullIndex(target);
+            validateIsValuesIndexIfTargetColumnNotCollection(cd, target);
+            validateTargetColumnIsMapIfIndexInvolvesKeys(isMap, target);
         }
 
         if (cd.getIndexType() != null)
         {
-            boolean previousIsKeys = cd.hasIndexOption(SecondaryIndex.INDEX_KEYS_OPTION_NAME);
-            if (isMap && target.isCollectionKeys != previousIsKeys)
+            IndexTarget.TargetType prevType = IndexTarget.TargetType.fromColumnDefinition(cd);
+            if (isMap && target.type != prevType)
             {
-                String msg = "Cannot create index on %s %s, an index on %s %s already exists and indexing "
-                           + "a map on both keys and values at the same time is not currently supported";
+                String msg = "Cannot create index on %s(%s): an index on %s(%s) already exists and indexing " +
+                             "a map on more than one dimension at the same time is not currently supported";
                 throw new InvalidRequestException(String.format(msg,
-                                                                target.column, target.isCollectionKeys ? "keys" : "values",
-                                                                target.column, previousIsKeys ? "keys" : "values"));
+                                                                target.type, target.column,
+                                                                prevType, target.column));
             }
 
             if (ifNotExists)
@@ -138,11 +131,40 @@
             throw new InvalidRequestException(String.format("Cannot create secondary index on partition key column %s", target.column));
     }
 
+    private void validateForFrozenCollection(IndexTarget target) throws InvalidRequestException
+    {
+        if (target.type != IndexTarget.TargetType.FULL)
+            throw new InvalidRequestException(String.format("Cannot create index on %s of frozen<map> column %s", target.type, target.column));
+    }
+
+    private void validateNotFullIndex(IndexTarget target) throws InvalidRequestException
+    {
+        if (target.type == IndexTarget.TargetType.FULL)
+            throw new InvalidRequestException("full() indexes can only be created on frozen collections");
+    }
+
+    private void validateIsValuesIndexIfTargetColumnNotCollection(ColumnDefinition cd, IndexTarget target) throws InvalidRequestException
+    {
+        if (!cd.type.isCollection() && target.type != IndexTarget.TargetType.VALUES)
+            throw new InvalidRequestException(String.format("Cannot create index on %s of column %s; only non-frozen collections support %s indexes",
+                                                            target.type, target.column, target.type));
+    }
+
+    private void validateTargetColumnIsMapIfIndexInvolvesKeys(boolean isMap, IndexTarget target) throws InvalidRequestException
+    {
+        if (target.type == IndexTarget.TargetType.KEYS || target.type == IndexTarget.TargetType.KEYS_AND_VALUES)
+        {
+            if (!isMap)
+                throw new InvalidRequestException(String.format("Cannot create index on %s of column %s with non-map type",
+                                                                target.type, target.column));
+        }
+    }
+
     public boolean announceMigration(boolean isLocalOnly) throws RequestValidationException
     {
         CFMetaData cfm = Schema.instance.getCFMetaData(keyspace(), columnFamily()).copy();
         IndexTarget target = rawTarget.prepare(cfm);
-        logger.debug("Updating column {} definition for index {}", target.column, indexName);
+        logger.trace("Updating column {} definition for index {}", target.column, indexName);
         ColumnDefinition cd = cfm.getColumnDefinition(target.column);
 
         if (cd.getIndexType() != null && ifNotExists)
@@ -159,8 +181,7 @@
             // to also index map keys, so we record that this is the values we index to make our
             // lives easier then.
             if (cd.type.isCollection() && cd.type.isMultiCell())
-                options = ImmutableMap.of(target.isCollectionKeys ? SecondaryIndex.INDEX_KEYS_OPTION_NAME
-                                                                  : SecondaryIndex.INDEX_VALUES_OPTION_NAME, "");
+                options = ImmutableMap.of(target.type.indexOption(), "");
             cd.setIndexType(IndexType.COMPOSITES, options);
         }
         else
@@ -170,7 +191,7 @@
 
         cd.setIndexName(indexName);
         cfm.addDefaultIndexNames();
-        MigrationManager.announceColumnFamilyUpdate(cfm, false, isLocalOnly);
+        MigrationManager.announceColumnFamilyUpdate(cfm, isLocalOnly);
         return true;
     }
 
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateKeyspaceStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateKeyspaceStatement.java
index 8281cbd..a3e27e4 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateKeyspaceStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateKeyspaceStatement.java
@@ -17,18 +17,12 @@
  */
 package org.apache.cassandra.cql3.statements;
 
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.auth.Permission;
+import org.apache.cassandra.auth.*;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.exceptions.AlreadyExistsException;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.RequestValidationException;
-import org.apache.cassandra.exceptions.UnauthorizedException;
+import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.locator.AbstractReplicationStrategy;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.MigrationManager;
-import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.service.*;
 import org.apache.cassandra.thrift.ThriftValidation;
 import org.apache.cassandra.transport.Event;
 
@@ -116,4 +110,26 @@
     {
         return new Event.SchemaChange(Event.SchemaChange.Change.CREATED, keyspace());
     }
+
+    protected void grantPermissionsToCreator(QueryState state)
+    {
+        try
+        {
+            RoleResource role = RoleResource.role(state.getClientState().getUser().getName());
+            DataResource keyspace = DataResource.keyspace(keyspace());
+            DatabaseDescriptor.getAuthorizer().grant(AuthenticatedUser.SYSTEM_USER,
+                                                     keyspace.applicablePermissions(),
+                                                     keyspace,
+                                                     role);
+            FunctionResource functions = FunctionResource.keyspace(keyspace());
+            DatabaseDescriptor.getAuthorizer().grant(AuthenticatedUser.SYSTEM_USER,
+                                                     functions.applicablePermissions(),
+                                                     functions,
+                                                     role);
+        }
+        catch (RequestExecutionException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
 }
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateRoleStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateRoleStatement.java
new file mode 100644
index 0000000..9be4c89
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateRoleStatement.java
@@ -0,0 +1,102 @@
+/*
+ * 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.cql3.statements;
+
+import org.apache.cassandra.auth.*;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.RoleName;
+import org.apache.cassandra.exceptions.*;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.transport.messages.ResultMessage;
+
+public class CreateRoleStatement extends AuthenticationStatement
+{
+    private final RoleResource role;
+    private final RoleOptions opts;
+    private final boolean ifNotExists;
+
+    public CreateRoleStatement(RoleName name, RoleOptions options, boolean ifNotExists)
+    {
+        this.role = RoleResource.role(name.getName());
+        this.opts = options;
+        this.ifNotExists = ifNotExists;
+    }
+
+    public void checkAccess(ClientState state) throws UnauthorizedException
+    {
+        super.checkPermission(state, Permission.CREATE, RoleResource.root());
+        if (opts.getSuperuser().isPresent())
+        {
+            if (opts.getSuperuser().get() && !state.getUser().isSuper())
+                throw new UnauthorizedException("Only superusers can create a role with superuser status");
+        }
+    }
+
+    public void validate(ClientState state) throws RequestValidationException
+    {
+        opts.validate();
+
+        if (role.getRoleName().isEmpty())
+            throw new InvalidRequestException("Role name can't be an empty string");
+
+        // validate login here before checkAccess to avoid leaking role existence to anonymous users.
+        state.ensureNotAnonymous();
+
+        if (!ifNotExists && DatabaseDescriptor.getRoleManager().isExistingRole(role))
+            throw new InvalidRequestException(String.format("%s already exists", role.getRoleName()));
+    }
+
+    public ResultMessage execute(ClientState state) throws RequestExecutionException, RequestValidationException
+    {
+        // not rejected in validate()
+        if (ifNotExists && DatabaseDescriptor.getRoleManager().isExistingRole(role))
+            return null;
+
+        DatabaseDescriptor.getRoleManager().createRole(state.getUser(), role, opts);
+        grantPermissionsToCreator(state);
+        return null;
+    }
+
+    /**
+     * Grant all applicable permissions on the newly created role to the user performing the request
+     * see also: SchemaAlteringStatement#grantPermissionsToCreator and the overridden implementations
+     * of it in subclasses CreateKeyspaceStatement & CreateTableStatement.
+     * @param state
+     */
+    private void grantPermissionsToCreator(ClientState state)
+    {
+        // The creator of a Role automatically gets ALTER/DROP/AUTHORIZE permissions on it if:
+        // * the user is not anonymous
+        // * the configured IAuthorizer supports granting of permissions (not all do, AllowAllAuthorizer doesn't and
+        //   custom external implementations may not)
+        if (!state.getUser().isAnonymous())
+        {
+            try
+            {
+                DatabaseDescriptor.getAuthorizer().grant(AuthenticatedUser.SYSTEM_USER,
+                                                         role.applicablePermissions(),
+                                                         role,
+                                                         RoleResource.role(state.getUser().getName()));
+            }
+            catch (UnsupportedOperationException e)
+            {
+                // not a problem, grant is an optional method on IAuthorizer
+            }
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
index 2c4b23b..e761674 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
@@ -20,23 +20,23 @@
 import java.nio.ByteBuffer;
 import java.util.*;
 
-import org.apache.cassandra.exceptions.*;
-import org.apache.commons.lang3.StringUtils;
 import com.google.common.collect.HashMultiset;
 import com.google.common.collect.Multiset;
+import org.apache.commons.lang3.StringUtils;
 
-import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.cql3.*;
-import org.apache.cassandra.db.composites.*;
+import org.apache.cassandra.auth.*;
+import org.apache.cassandra.config.*;
+import org.apache.cassandra.cql3.CFName;
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.db.ColumnFamilyType;
+import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.db.marshal.*;
-import org.apache.cassandra.exceptions.AlreadyExistsException;
+import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.io.compress.CompressionParameters;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.MigrationManager;
+import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.transport.Event;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
@@ -74,19 +74,12 @@
         this.staticColumns = staticColumns;
         this.id = id;
 
-        try
-        {
-            if (!this.properties.hasProperty(CFPropDefs.KW_COMPRESSION) && CFMetaData.DEFAULT_COMPRESSOR != null)
-                this.properties.addProperty(CFPropDefs.KW_COMPRESSION,
-                                            new HashMap<String, String>()
-                                            {{
-                                                put(CompressionParameters.SSTABLE_COMPRESSION, CFMetaData.DEFAULT_COMPRESSOR);
-                                            }});
-        }
-        catch (SyntaxException e)
-        {
-            throw new AssertionError(e);
-        }
+        if (!this.properties.hasProperty(CFPropDefs.KW_COMPRESSION) && CFMetaData.DEFAULT_COMPRESSOR != null)
+            this.properties.addProperty(CFPropDefs.KW_COMPRESSION,
+                                        new HashMap<String, String>()
+                                        {{
+                                            put(CompressionParameters.SSTABLE_COMPRESSION, CFMetaData.DEFAULT_COMPRESSOR);
+                                        }});
     }
 
     public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
@@ -135,6 +128,22 @@
         return new Event.SchemaChange(Event.SchemaChange.Change.CREATED, Event.SchemaChange.Target.TABLE, keyspace(), columnFamily());
     }
 
+    protected void grantPermissionsToCreator(QueryState state)
+    {
+        try
+        {
+            IResource resource = DataResource.table(keyspace(), columnFamily());
+            DatabaseDescriptor.getAuthorizer().grant(AuthenticatedUser.SYSTEM_USER,
+                                                     resource.applicablePermissions(),
+                                                     resource,
+                                                     RoleResource.role(state.getClientState().getUser().getName()));
+        }
+        catch (RequestExecutionException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
     /**
      * Returns a CFMetaData instance based on the parameters parsed from this
      * <code>CREATE</code> statement, or defaults where applicable.
@@ -161,14 +170,32 @@
             .addAllColumnDefinitions(getColumns(cfmd))
             .isDense(isDense);
 
-        cfmd.addColumnMetadataFromAliases(keyAliases, keyValidator, ColumnDefinition.Kind.PARTITION_KEY);
-        cfmd.addColumnMetadataFromAliases(columnAliases, comparator.asAbstractType(), ColumnDefinition.Kind.CLUSTERING_COLUMN);
+        addColumnMetadataFromAliases(cfmd, keyAliases, keyValidator, ColumnDefinition.Kind.PARTITION_KEY);
+        addColumnMetadataFromAliases(cfmd, columnAliases, comparator.asAbstractType(), ColumnDefinition.Kind.CLUSTERING_COLUMN);
         if (valueAlias != null)
-            cfmd.addColumnMetadataFromAliases(Collections.<ByteBuffer>singletonList(valueAlias), defaultValidator, ColumnDefinition.Kind.COMPACT_VALUE);
+            addColumnMetadataFromAliases(cfmd, Collections.singletonList(valueAlias), defaultValidator, ColumnDefinition.Kind.COMPACT_VALUE);
 
         properties.applyToCFMetadata(cfmd);
     }
 
+    private void addColumnMetadataFromAliases(CFMetaData cfm, List<ByteBuffer> aliases, AbstractType<?> comparator, ColumnDefinition.Kind kind)
+    {
+        if (comparator instanceof CompositeType)
+        {
+            CompositeType ct = (CompositeType)comparator;
+            for (int i = 0; i < aliases.size(); ++i)
+                if (aliases.get(i) != null)
+                    cfm.addOrReplaceColumnDefinition(new ColumnDefinition(cfm, aliases.get(i), ct.types.get(i), i, kind));
+        }
+        else
+        {
+            assert aliases.size() <= 1;
+            if (!aliases.isEmpty() && aliases.get(0) != null)
+                cfm.addOrReplaceColumnDefinition(new ColumnDefinition(cfm, aliases.get(0), comparator, null, kind));
+        }
+    }
+
+
     public static class RawStatement extends CFStatement
     {
         private final Map<ColumnIdentifier, CQL3Type.Raw> definitions = new HashMap<>();
@@ -197,9 +224,9 @@
         {
             // Column family name
             if (!columnFamily().matches("\\w+"))
-                throw new InvalidRequestException(String.format("\"%s\" is not a valid column family name (must be alphanumeric character only: [0-9A-Za-z]+)", columnFamily()));
+                throw new InvalidRequestException(String.format("\"%s\" is not a valid table name (must be alphanumeric character or underscore only: [a-zA-Z_0-9]+)", columnFamily()));
             if (columnFamily().length() > Schema.NAME_LENGTH)
-                throw new InvalidRequestException(String.format("Column family names shouldn't be more than %s characters long (got \"%s\")", Schema.NAME_LENGTH, columnFamily()));
+                throw new InvalidRequestException(String.format("Table names shouldn't be more than %s characters long (got \"%s\")", Schema.NAME_LENGTH, columnFamily()));
 
             for (Multiset.Entry<ColumnIdentifier> entry : definedNames.entrySet())
                 if (entry.getCount() > 1)
@@ -241,7 +268,7 @@
             {
                 stmt.keyAliases.add(alias.bytes);
                 AbstractType<?> t = getTypeAndRemove(stmt.columns, alias);
-                if (t instanceof CounterColumnType)
+                if (t.asCQL3Type().getType() instanceof CounterColumnType)
                     throw new InvalidRequestException(String.format("counter type is not supported for PRIMARY KEY part %s", alias));
                 if (staticColumns.contains(alias))
                     throw new InvalidRequestException(String.format("Static column %s cannot be part of the PRIMARY KEY", alias));
@@ -289,7 +316,7 @@
 
                     stmt.columnAliases.add(alias.bytes);
                     AbstractType<?> at = getTypeAndRemove(stmt.columns, alias);
-                    if (at instanceof CounterColumnType)
+                    if (at.asCQL3Type().getType() instanceof CounterColumnType)
                         throw new InvalidRequestException(String.format("counter type is not supported for PRIMARY KEY part %s", stmt.columnAliases.get(0)));
                     stmt.comparator = new SimpleDenseCellNameType(at);
                 }
@@ -301,7 +328,7 @@
                         stmt.columnAliases.add(t.bytes);
 
                         AbstractType<?> type = getTypeAndRemove(stmt.columns, t);
-                        if (type instanceof CounterColumnType)
+                        if (type.asCQL3Type().getType() instanceof CounterColumnType)
                             throw new InvalidRequestException(String.format("counter type is not supported for PRIMARY KEY part %s", t));
                         if (staticColumns.contains(t))
                             throw new InvalidRequestException(String.format("Static column %s cannot be part of the PRIMARY KEY", t));
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateTriggerStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateTriggerStatement.java
index 6ebe0d3..ef2f263 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateTriggerStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateTriggerStatement.java
@@ -78,7 +78,7 @@
         {
             cfm.addTriggerDefinition(triggerDefinition);
             logger.info("Adding trigger with name {} and class {}", triggerName, triggerClass);
-            MigrationManager.announceColumnFamilyUpdate(cfm, false, isLocalOnly);
+            MigrationManager.announceColumnFamilyUpdate(cfm, isLocalOnly);
             return true;
         }
         return false;
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateUserStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateUserStatement.java
deleted file mode 100644
index 0903659..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/CreateUserStatement.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * 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.cql3.statements;
-
-import org.apache.cassandra.auth.Auth;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.cql3.UserOptions;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.RequestExecutionException;
-import org.apache.cassandra.exceptions.RequestValidationException;
-import org.apache.cassandra.exceptions.UnauthorizedException;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.transport.messages.ResultMessage;
-
-public class CreateUserStatement extends AuthenticationStatement
-{
-    private final String username;
-    private final UserOptions opts;
-    private final boolean superuser;
-    private final boolean ifNotExists;
-
-    public CreateUserStatement(String username, UserOptions opts, boolean superuser, boolean ifNotExists)
-    {
-        this.username = username;
-        this.opts = opts;
-        this.superuser = superuser;
-        this.ifNotExists = ifNotExists;
-    }
-
-    public void validate(ClientState state) throws RequestValidationException
-    {
-        if (username.isEmpty())
-            throw new InvalidRequestException("Username can't be an empty string");
-
-        opts.validate();
-
-        // validate login here before checkAccess to avoid leaking user existence to anonymous users.
-        state.ensureNotAnonymous();
-
-        if (!ifNotExists && Auth.isExistingUser(username))
-            throw new InvalidRequestException(String.format("User %s already exists", username));
-    }
-
-    public void checkAccess(ClientState state) throws UnauthorizedException
-    {
-        if (!state.getUser().isSuper())
-            throw new UnauthorizedException("Only superusers are allowed to perform CREATE USER queries");
-    }
-
-    public ResultMessage execute(ClientState state) throws RequestValidationException, RequestExecutionException
-    {
-        // not rejected in validate()
-        if (ifNotExists && Auth.isExistingUser(username))
-            return null;
-
-        DatabaseDescriptor.getAuthenticator().create(username, opts.getOptions());
-        Auth.insertUser(username, superuser);
-        return null;
-    }
-}
diff --git a/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java b/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
index d8fa467..397928d 100644
--- a/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/DeleteStatement.java
@@ -23,6 +23,7 @@
 import com.google.common.collect.Iterators;
 
 import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.cql3.restrictions.Restriction;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.*;
diff --git a/src/java/org/apache/cassandra/cql3/statements/DropAggregateStatement.java b/src/java/org/apache/cassandra/cql3/statements/DropAggregateStatement.java
new file mode 100644
index 0000000..2d5ea70
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/DropAggregateStatement.java
@@ -0,0 +1,156 @@
+/*
+ * 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.cql3.statements;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.auth.Permission;
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.functions.*;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.exceptions.RequestValidationException;
+import org.apache.cassandra.exceptions.UnauthorizedException;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.MigrationManager;
+import org.apache.cassandra.thrift.ThriftValidation;
+import org.apache.cassandra.transport.Event;
+
+/**
+ * A <code>DROP AGGREGATE</code> statement parsed from a CQL query.
+ */
+public final class DropAggregateStatement extends SchemaAlteringStatement
+{
+    private FunctionName functionName;
+    private final boolean ifExists;
+    private final List<CQL3Type.Raw> argRawTypes;
+    private final boolean argsPresent;
+
+    private Function old;
+
+    public DropAggregateStatement(FunctionName functionName,
+                                  List<CQL3Type.Raw> argRawTypes,
+                                  boolean argsPresent,
+                                  boolean ifExists)
+    {
+        this.functionName = functionName;
+        this.argRawTypes = argRawTypes;
+        this.argsPresent = argsPresent;
+        this.ifExists = ifExists;
+    }
+
+    public void prepareKeyspace(ClientState state) throws InvalidRequestException
+    {
+        if (!functionName.hasKeyspace() && state.getRawKeyspace() != null)
+            functionName = new FunctionName(state.getKeyspace(), functionName.name);
+
+        if (!functionName.hasKeyspace())
+            throw new InvalidRequestException("Functions must be fully qualified with a keyspace name if a keyspace is not set for the session");
+
+        ThriftValidation.validateKeyspaceNotSystem(functionName.keyspace);
+    }
+
+    public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
+    {
+        // TODO CASSANDRA-7557 (function DDL permission)
+
+        state.hasKeyspaceAccess(functionName.keyspace, Permission.DROP);
+    }
+
+    public void validate(ClientState state) throws RequestValidationException
+    {
+    }
+
+    public Event.SchemaChange changeEvent()
+    {
+        return new Event.SchemaChange(Event.SchemaChange.Change.DROPPED, Event.SchemaChange.Target.AGGREGATE,
+                                      old.name().keyspace, old.name().name, AbstractType.asCQLTypeStringList(old.argTypes()));
+    }
+
+    public boolean announceMigration(boolean isLocalOnly) throws RequestValidationException
+    {
+        List<Function> olds = Functions.find(functionName);
+
+        if (!argsPresent && olds != null && olds.size() > 1)
+            throw new InvalidRequestException(String.format("'DROP AGGREGATE %s' matches multiple function definitions; " +
+                                                            "specify the argument types by issuing a statement like " +
+                                                            "'DROP AGGREGATE %s (type, type, ...)'. Hint: use cqlsh " +
+                                                            "'DESCRIBE AGGREGATE %s' command to find all overloads",
+                                                            functionName, functionName, functionName));
+
+        List<AbstractType<?>> argTypes = new ArrayList<>(argRawTypes.size());
+        for (CQL3Type.Raw rawType : argRawTypes)
+            argTypes.add(prepareType("arguments", rawType));
+
+        Function old;
+        if (argsPresent)
+        {
+            old = Functions.find(functionName, argTypes);
+            if (old == null || !(old instanceof AggregateFunction))
+            {
+                if (ifExists)
+                    return false;
+                // just build a nicer error message
+                StringBuilder sb = new StringBuilder();
+                for (CQL3Type.Raw rawType : argRawTypes)
+                {
+                    if (sb.length() > 0)
+                        sb.append(", ");
+                    sb.append(rawType);
+                }
+                throw new InvalidRequestException(String.format("Cannot drop non existing aggregate '%s(%s)'",
+                                                                functionName, sb));
+            }
+        }
+        else
+        {
+            if (olds == null || olds.isEmpty() || !(olds.get(0) instanceof AggregateFunction))
+            {
+                if (ifExists)
+                    return false;
+                throw new InvalidRequestException(String.format("Cannot drop non existing aggregate '%s'", functionName));
+            }
+            old = olds.get(0);
+        }
+
+        if (old.isNative())
+            throw new InvalidRequestException(String.format("Cannot drop aggregate '%s' because it is a " +
+                                                            "native (built-in) function", functionName));
+
+        this.old = old;
+
+        MigrationManager.announceAggregateDrop((UDAggregate)old, isLocalOnly);
+
+        return true;
+    }
+
+    private AbstractType<?> prepareType(String typeName, CQL3Type.Raw rawType)
+    {
+        if (rawType.isFrozen())
+            throw new InvalidRequestException(String.format("The function %s should not be frozen; remove the frozen<> modifier", typeName));
+
+        // UDT are not supported non frozen but we do not allow the frozen keyword for argument. So for the moment we
+        // freeze them here
+        if (!rawType.canBeNonFrozen())
+            rawType.freeze();
+
+        AbstractType<?> type = rawType.prepare(functionName.keyspace).getType();
+        return type;
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/statements/DropFunctionStatement.java b/src/java/org/apache/cassandra/cql3/statements/DropFunctionStatement.java
new file mode 100644
index 0000000..edd8a65
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/DropFunctionStatement.java
@@ -0,0 +1,194 @@
+/*
+ * 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.cql3.statements;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import com.google.common.base.Joiner;
+
+import org.apache.cassandra.auth.FunctionResource;
+import org.apache.cassandra.auth.Permission;
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.functions.*;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.exceptions.RequestValidationException;
+import org.apache.cassandra.exceptions.UnauthorizedException;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.MigrationManager;
+import org.apache.cassandra.thrift.ThriftValidation;
+import org.apache.cassandra.transport.Event;
+
+/**
+ * A <code>DROP FUNCTION</code> statement parsed from a CQL query.
+ */
+public final class DropFunctionStatement extends SchemaAlteringStatement
+{
+    private FunctionName functionName;
+    private final boolean ifExists;
+    private final List<CQL3Type.Raw> argRawTypes;
+    private final boolean argsPresent;
+
+    private Function old;
+    private List<AbstractType<?>> argTypes;
+
+    public DropFunctionStatement(FunctionName functionName,
+                                 List<CQL3Type.Raw> argRawTypes,
+                                 boolean argsPresent,
+                                 boolean ifExists)
+    {
+        this.functionName = functionName;
+        this.argRawTypes = argRawTypes;
+        this.argsPresent = argsPresent;
+        this.ifExists = ifExists;
+    }
+
+    @Override
+    public Prepared prepare() throws InvalidRequestException
+    {
+        argTypes = new ArrayList<>(argRawTypes.size());
+        for (CQL3Type.Raw rawType : argRawTypes)
+        {
+            if (rawType.isFrozen())
+                throw new InvalidRequestException("The function arguments should not be frozen; remove the frozen<> modifier");
+
+            // UDT are not supported non frozen but we do not allow the frozen keyword for argument. So for the moment we
+            // freeze them here
+            if (!rawType.canBeNonFrozen())
+                rawType.freeze();
+
+            argTypes.add(rawType.prepare(functionName.keyspace).getType());
+        }
+        return super.prepare();
+    }
+
+    @Override
+    public void prepareKeyspace(ClientState state) throws InvalidRequestException
+    {
+        if (!functionName.hasKeyspace() && state.getRawKeyspace() != null)
+            functionName = new FunctionName(state.getKeyspace(), functionName.name);
+
+        if (!functionName.hasKeyspace())
+            throw new InvalidRequestException("Functions must be fully qualified with a keyspace name if a keyspace is not set for the session");
+
+        ThriftValidation.validateKeyspaceNotSystem(functionName.keyspace);
+    }
+
+    @Override
+    public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
+    {
+        Function function = findFunction();
+        if (function == null)
+        {
+            if (!ifExists)
+                throw new InvalidRequestException(String.format("Unconfigured function %s.%s(%s)",
+                                                                functionName.keyspace,
+                                                                functionName.name,
+                                                                Joiner.on(",").join(argRawTypes)));
+        }
+        else
+        {
+            state.ensureHasPermission(Permission.DROP, FunctionResource.function(function.name().keyspace,
+                                                                                 function.name().name,
+                                                                                 function.argTypes()));
+        }
+    }
+
+    @Override
+    public void validate(ClientState state)
+    {
+        List<Function> olds = Functions.find(functionName);
+
+        if (!argsPresent && olds != null && olds.size() > 1)
+            throw new InvalidRequestException(String.format("'DROP FUNCTION %s' matches multiple function definitions; " +
+                                                            "specify the argument types by issuing a statement like " +
+                                                            "'DROP FUNCTION %s (type, type, ...)'. Hint: use cqlsh " +
+                                                            "'DESCRIBE FUNCTION %s' command to find all overloads",
+                                                            functionName, functionName, functionName));
+    }
+
+    @Override
+    public Event.SchemaChange changeEvent()
+    {
+        return new Event.SchemaChange(Event.SchemaChange.Change.DROPPED, Event.SchemaChange.Target.FUNCTION,
+                                      old.name().keyspace, old.name().name, AbstractType.asCQLTypeStringList(old.argTypes()));
+    }
+
+    @Override
+    public boolean announceMigration(boolean isLocalOnly) throws RequestValidationException
+    {
+        old = findFunction();
+        if (old == null)
+        {
+            if (ifExists)
+                return false;
+            else
+                throw new InvalidRequestException(getMissingFunctionError());
+        }
+
+        List<Function> references = Functions.getReferencesTo(old);
+        if (!references.isEmpty())
+            throw new InvalidRequestException(String.format("Function '%s' still referenced by %s", old, references));
+
+        MigrationManager.announceFunctionDrop((UDFunction) old, isLocalOnly);
+
+        return true;
+    }
+
+    private String getMissingFunctionError()
+    {
+        // just build a nicer error message
+        StringBuilder sb = new StringBuilder("Cannot drop non existing function '");
+        sb.append(functionName);
+        if (argsPresent)
+            sb.append(Joiner.on(", ").join(argRawTypes));
+        sb.append("'");
+        return sb.toString();
+    }
+
+    private String typeKeyspace(CQL3Type.Raw rawType)
+    {
+        String ks = rawType.keyspace();
+        if (ks != null)
+            return ks;
+        return functionName.keyspace;
+    }
+
+    private Function findFunction()
+    {
+        Function old;
+        if (argsPresent)
+        {
+            old = Functions.find(functionName, argTypes);
+            if (old == null || !(old instanceof ScalarFunction))
+            {
+                return null;
+            }
+        }
+        else
+        {
+            List<Function> olds = Functions.find(functionName);
+            if (olds == null || olds.isEmpty() || !(olds.get(0) instanceof ScalarFunction))
+                return null;
+
+            old = olds.get(0);
+        }
+        return old;
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/statements/DropIndexStatement.java b/src/java/org/apache/cassandra/cql3/statements/DropIndexStatement.java
index c6c0244..0d33e57 100644
--- a/src/java/org/apache/cassandra/cql3/statements/DropIndexStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/DropIndexStatement.java
@@ -97,7 +97,7 @@
 
         CFMetaData updatedCfm = updateCFMetadata(cfm);
         indexedCF = updatedCfm.cfName;
-        MigrationManager.announceColumnFamilyUpdate(updatedCfm, false, isLocalOnly);
+        MigrationManager.announceColumnFamilyUpdate(updatedCfm, isLocalOnly);
         return true;
     }
 
diff --git a/src/java/org/apache/cassandra/cql3/statements/DropRoleStatement.java b/src/java/org/apache/cassandra/cql3/statements/DropRoleStatement.java
new file mode 100644
index 0000000..55fa83a
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/DropRoleStatement.java
@@ -0,0 +1,75 @@
+/*
+ * 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.cql3.statements;
+
+import org.apache.cassandra.auth.*;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.RoleName;
+import org.apache.cassandra.exceptions.*;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.transport.messages.ResultMessage;
+
+public class DropRoleStatement extends AuthenticationStatement
+{
+    private final RoleResource role;
+    private final boolean ifExists;
+
+    public DropRoleStatement(RoleName name, boolean ifExists)
+    {
+        this.role = RoleResource.role(name.getName());
+        this.ifExists = ifExists;
+    }
+
+    public void checkAccess(ClientState state) throws UnauthorizedException
+    {
+        super.checkPermission(state, Permission.DROP, role);
+
+        // We only check superuser status for existing roles to avoid
+        // caching info about roles which don't exist (CASSANDRA-9189)
+        if (DatabaseDescriptor.getRoleManager().isExistingRole(role)
+            && Roles.hasSuperuserStatus(role)
+            && !state.getUser().isSuper())
+            throw new UnauthorizedException("Only superusers can drop a role with superuser status");
+    }
+
+    public void validate(ClientState state) throws RequestValidationException
+    {
+        // validate login here before checkAccess to avoid leaking user existence to anonymous users.
+        state.ensureNotAnonymous();
+
+        if (!ifExists && !DatabaseDescriptor.getRoleManager().isExistingRole(role))
+            throw new InvalidRequestException(String.format("%s doesn't exist", role.getRoleName()));
+
+        AuthenticatedUser user = state.getUser();
+        if (user != null && user.getName().equals(role.getRoleName()))
+            throw new InvalidRequestException("Cannot DROP primary role for current login");
+    }
+
+    public ResultMessage execute(ClientState state) throws RequestValidationException, RequestExecutionException
+    {
+        // not rejected in validate()
+        if (ifExists && !DatabaseDescriptor.getRoleManager().isExistingRole(role))
+            return null;
+
+        // clean up grants and permissions of/on the dropped role.
+        DatabaseDescriptor.getRoleManager().dropRole(state.getUser(), role);
+        DatabaseDescriptor.getAuthorizer().revokeAllFrom(role);
+        DatabaseDescriptor.getAuthorizer().revokeAllOn(role);
+        return null;
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/statements/DropTriggerStatement.java b/src/java/org/apache/cassandra/cql3/statements/DropTriggerStatement.java
index e3db1e1..8267b4e 100644
--- a/src/java/org/apache/cassandra/cql3/statements/DropTriggerStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/DropTriggerStatement.java
@@ -63,7 +63,7 @@
         if (cfm.removeTrigger(triggerName))
         {
             logger.info("Dropping trigger with name {}", triggerName);
-            MigrationManager.announceColumnFamilyUpdate(cfm, false, isLocalOnly);
+            MigrationManager.announceColumnFamilyUpdate(cfm, isLocalOnly);
             return true;
         }
         if (!ifExists)
diff --git a/src/java/org/apache/cassandra/cql3/statements/DropTypeStatement.java b/src/java/org/apache/cassandra/cql3/statements/DropTypeStatement.java
index bc6005d..6993ea3 100644
--- a/src/java/org/apache/cassandra/cql3/statements/DropTypeStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/DropTypeStatement.java
@@ -20,6 +20,8 @@
 import org.apache.cassandra.auth.Permission;
 import org.apache.cassandra.config.*;
 import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.functions.Functions;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.service.ClientState;
@@ -77,6 +79,15 @@
         // we drop and 2) existing tables referencing the type (maybe in a nested
         // way).
 
+        for (Function function : Functions.all())
+        {
+            if (isUsedBy(function.returnType()))
+                throw new InvalidRequestException(String.format("Cannot drop user type %s as it is still used by function %s", name, function));
+            for (AbstractType<?> argType : function.argTypes())
+                if (isUsedBy(argType))
+                    throw new InvalidRequestException(String.format("Cannot drop user type %s as it is still used by function %s", name, function));
+        }
+
         for (KSMetaData ksm2 : Schema.instance.getKeyspaceDefinitions())
         {
             for (UserType ut : ksm2.userTypes.getAllTypes().values())
diff --git a/src/java/org/apache/cassandra/cql3/statements/DropUserStatement.java b/src/java/org/apache/cassandra/cql3/statements/DropUserStatement.java
deleted file mode 100644
index 9e92b26..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/DropUserStatement.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * 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.cql3.statements;
-
-import org.apache.cassandra.auth.Auth;
-import org.apache.cassandra.auth.AuthenticatedUser;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.RequestExecutionException;
-import org.apache.cassandra.exceptions.RequestValidationException;
-import org.apache.cassandra.exceptions.UnauthorizedException;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.transport.messages.ResultMessage;
-
-public class DropUserStatement extends AuthenticationStatement
-{
-    private final String username;
-    private final boolean ifExists;
-
-    public DropUserStatement(String username, boolean ifExists)
-    {
-        this.username = username;
-        this.ifExists = ifExists;
-    }
-
-    public void validate(ClientState state) throws RequestValidationException
-    {
-        // validate login here before checkAccess to avoid leaking user existence to anonymous users.
-        state.ensureNotAnonymous();
-
-        if (!ifExists && !Auth.isExistingUser(username))
-            throw new InvalidRequestException(String.format("User %s doesn't exist", username));
-
-        AuthenticatedUser user = state.getUser();
-        if (user != null && user.getName().equals(username))
-            throw new InvalidRequestException("Users aren't allowed to DROP themselves");
-    }
-
-    public void checkAccess(ClientState state) throws UnauthorizedException
-    {
-        if (!state.getUser().isSuper())
-            throw new UnauthorizedException("Only superusers are allowed to perform DROP USER queries");
-    }
-
-    public ResultMessage execute(ClientState state) throws RequestValidationException, RequestExecutionException
-    {
-        // not rejected in validate()
-        if (ifExists && !Auth.isExistingUser(username))
-            return null;
-
-        // clean up permissions after the dropped user.
-        DatabaseDescriptor.getAuthorizer().revokeAll(username);
-        Auth.deleteUser(username);
-        DatabaseDescriptor.getAuthenticator().drop(username);
-        return null;
-    }
-}
diff --git a/src/java/org/apache/cassandra/cql3/statements/GrantPermissionsStatement.java b/src/java/org/apache/cassandra/cql3/statements/GrantPermissionsStatement.java
new file mode 100644
index 0000000..06a53e2
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/GrantPermissionsStatement.java
@@ -0,0 +1,43 @@
+/*
+ * 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.cql3.statements;
+
+import java.util.Set;
+
+import org.apache.cassandra.auth.IResource;
+import org.apache.cassandra.auth.Permission;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.RoleName;
+import org.apache.cassandra.exceptions.RequestExecutionException;
+import org.apache.cassandra.exceptions.RequestValidationException;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.transport.messages.ResultMessage;
+
+public class GrantPermissionsStatement extends PermissionsManagementStatement
+{
+    public GrantPermissionsStatement(Set<Permission> permissions, IResource resource, RoleName grantee)
+    {
+        super(permissions, resource, grantee);
+    }
+
+    public ResultMessage execute(ClientState state) throws RequestValidationException, RequestExecutionException
+    {
+        DatabaseDescriptor.getAuthorizer().grant(state.getUser(), permissions, resource, grantee);
+        return null;
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/statements/GrantRoleStatement.java b/src/java/org/apache/cassandra/cql3/statements/GrantRoleStatement.java
new file mode 100644
index 0000000..a22a99a
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/GrantRoleStatement.java
@@ -0,0 +1,39 @@
+/*
+ * 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.cql3.statements;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.RoleName;
+import org.apache.cassandra.exceptions.RequestExecutionException;
+import org.apache.cassandra.exceptions.RequestValidationException;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.transport.messages.ResultMessage;
+
+public class GrantRoleStatement extends RoleManagementStatement
+{
+    public GrantRoleStatement(RoleName name, RoleName grantee)
+    {
+        super(name, grantee);
+    }
+
+    public ResultMessage execute(ClientState state) throws RequestValidationException, RequestExecutionException
+    {
+        DatabaseDescriptor.getRoleManager().grantRole(state.getUser(), role, grantee);
+        return null;
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/statements/GrantStatement.java b/src/java/org/apache/cassandra/cql3/statements/GrantStatement.java
deleted file mode 100644
index 67b651c..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/GrantStatement.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/**
- * 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.cql3.statements;
-
-import java.util.Set;
-
-import org.apache.cassandra.auth.IResource;
-import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.exceptions.RequestExecutionException;
-import org.apache.cassandra.exceptions.RequestValidationException;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.transport.messages.ResultMessage;
-
-public class GrantStatement extends PermissionAlteringStatement
-{
-    public GrantStatement(Set<Permission> permissions, IResource resource, String username)
-    {
-        super(permissions, resource, username);
-    }
-
-    public ResultMessage execute(ClientState state) throws RequestValidationException, RequestExecutionException
-    {
-        DatabaseDescriptor.getAuthorizer().grant(state.getUser(), permissions, resource, username);
-        return null;
-    }
-}
diff --git a/src/java/org/apache/cassandra/cql3/statements/IndexTarget.java b/src/java/org/apache/cassandra/cql3/statements/IndexTarget.java
index eeee907..d602388 100644
--- a/src/java/org/apache/cassandra/cql3/statements/IndexTarget.java
+++ b/src/java/org/apache/cassandra/cql3/statements/IndexTarget.java
@@ -17,53 +17,98 @@
  */
 package org.apache.cassandra.cql3.statements;
 
+import java.util.Map;
+
 import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.db.index.SecondaryIndex;
 
 public class IndexTarget
 {
     public final ColumnIdentifier column;
-    public final boolean isCollectionKeys;
-    public final boolean isFullCollection;
+    public final TargetType type;
 
-    private IndexTarget(ColumnIdentifier column, boolean isCollectionKeys, boolean isFullCollection)
+    private IndexTarget(ColumnIdentifier column, TargetType type)
     {
         this.column = column;
-        this.isCollectionKeys = isCollectionKeys;
-        this.isFullCollection = isFullCollection;
+        this.type = type;
     }
 
     public static class Raw
     {
         private final ColumnIdentifier.Raw column;
-        private final boolean isCollectionKeys;
-        private final boolean isFullCollection;
+        private final TargetType type;
 
-        private Raw(ColumnIdentifier.Raw column, boolean isCollectionKeys, boolean isFullCollection)
+        private Raw(ColumnIdentifier.Raw column, TargetType type)
         {
             this.column = column;
-            this.isCollectionKeys = isCollectionKeys;
-            this.isFullCollection = isFullCollection;
+            this.type = type;
         }
 
         public static Raw valuesOf(ColumnIdentifier.Raw c)
         {
-            return new Raw(c, false, false);
+            return new Raw(c, TargetType.VALUES);
         }
 
         public static Raw keysOf(ColumnIdentifier.Raw c)
         {
-            return new Raw(c, true, false);
+            return new Raw(c, TargetType.KEYS);
+        }
+
+        public static Raw keysAndValuesOf(ColumnIdentifier.Raw c)
+        {
+            return new Raw(c, TargetType.KEYS_AND_VALUES);
         }
 
         public static Raw fullCollection(ColumnIdentifier.Raw c)
         {
-            return new Raw(c, false, true);
+            return new Raw(c, TargetType.FULL);
         }
 
         public IndexTarget prepare(CFMetaData cfm)
         {
-            return new IndexTarget(column.prepare(cfm), isCollectionKeys, isFullCollection);
+            return new IndexTarget(column.prepare(cfm), type);
+        }
+    }
+
+    public static enum TargetType
+    {
+        VALUES, KEYS, KEYS_AND_VALUES, FULL;
+
+        public String toString()
+        {
+            switch (this)
+            {
+                case KEYS: return "keys";
+                case KEYS_AND_VALUES: return "entries";
+                case FULL: return "full";
+                default: return "values";
+            }
+        }
+
+        public String indexOption()
+        {
+            switch (this)
+            {
+                case KEYS: return SecondaryIndex.INDEX_KEYS_OPTION_NAME;
+                case KEYS_AND_VALUES: return SecondaryIndex.INDEX_ENTRIES_OPTION_NAME;
+                case VALUES: return SecondaryIndex.INDEX_VALUES_OPTION_NAME;
+                default: throw new AssertionError();
+            }
+        }
+
+        public static TargetType fromColumnDefinition(ColumnDefinition cd)
+        {
+            Map<String, String> options = cd.getIndexOptions();
+            if (options.containsKey(SecondaryIndex.INDEX_KEYS_OPTION_NAME))
+                return KEYS;
+            else if (options.containsKey(SecondaryIndex.INDEX_ENTRIES_OPTION_NAME))
+                return KEYS_AND_VALUES;
+            else if (cd.type.isCollection() && !cd.type.isMultiCell())
+                return FULL;
+            else
+                return VALUES;
         }
     }
 }
diff --git a/src/java/org/apache/cassandra/cql3/statements/ListPermissionsStatement.java b/src/java/org/apache/cassandra/cql3/statements/ListPermissionsStatement.java
index d8db871..58f8e9c 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ListPermissionsStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ListPermissionsStatement.java
@@ -21,9 +21,7 @@
 
 import org.apache.cassandra.auth.*;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.cql3.ColumnIdentifier;
-import org.apache.cassandra.cql3.ColumnSpecification;
-import org.apache.cassandra.cql3.ResultSet;
+import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.RequestExecutionException;
@@ -33,7 +31,7 @@
 
 public class ListPermissionsStatement extends AuthorizationStatement
 {
-    private static final String KS = Auth.AUTH_KS;
+    private static final String KS = AuthKeyspace.NAME;
     private static final String CF = "permissions"; // virtual cf to use for now.
 
     private static final List<ColumnSpecification> metadata;
@@ -41,23 +39,24 @@
     static
     {
         List<ColumnSpecification> columns = new ArrayList<ColumnSpecification>(4);
+        columns.add(new ColumnSpecification(KS, CF, new ColumnIdentifier("role", true), UTF8Type.instance));
         columns.add(new ColumnSpecification(KS, CF, new ColumnIdentifier("username", true), UTF8Type.instance));
         columns.add(new ColumnSpecification(KS, CF, new ColumnIdentifier("resource", true), UTF8Type.instance));
         columns.add(new ColumnSpecification(KS, CF, new ColumnIdentifier("permission", true), UTF8Type.instance));
         metadata = Collections.unmodifiableList(columns);
     }
 
-    private final Set<Permission> permissions;
-    private DataResource resource;
-    private final String username;
-    private final boolean recursive;
+    protected final Set<Permission> permissions;
+    protected IResource resource;
+    protected final boolean recursive;
+    private final RoleResource grantee;
 
-    public ListPermissionsStatement(Set<Permission> permissions, IResource resource, String username, boolean recursive)
+    public ListPermissionsStatement(Set<Permission> permissions, IResource resource, RoleName grantee, boolean recursive)
     {
         this.permissions = permissions;
-        this.resource = (DataResource) resource;
-        this.username = username;
+        this.resource = resource;
         this.recursive = recursive;
+        this.grantee = grantee.hasName()? RoleResource.role(grantee.getName()) : null;
     }
 
     public void validate(ClientState state) throws RequestValidationException
@@ -65,16 +64,16 @@
         // a check to ensure the existence of the user isn't being leaked by user existence check.
         state.ensureNotAnonymous();
 
-        if (username != null && !Auth.isExistingUser(username))
-            throw new InvalidRequestException(String.format("User %s doesn't exist", username));
-
         if (resource != null)
         {
             resource = maybeCorrectResource(resource, state);
             if (!resource.exists())
                 throw new InvalidRequestException(String.format("%s doesn't exist", resource));
         }
-    }
+
+        if ((grantee != null) && !DatabaseDescriptor.getRoleManager().isExistingRole(grantee))
+            throw new InvalidRequestException(String.format("%s doesn't exist", grantee));
+   }
 
     public void checkAccess(ClientState state)
     {
@@ -100,6 +99,19 @@
         return resultMessage(details);
     }
 
+    private Set<PermissionDetails> list(ClientState state, IResource resource)
+    throws RequestValidationException, RequestExecutionException
+    {
+        try
+        {
+            return DatabaseDescriptor.getAuthorizer().list(state.getUser(), permissions, resource, grantee);
+        }
+        catch (UnsupportedOperationException e)
+        {
+            throw new InvalidRequestException(e.getMessage());
+        }
+    }
+
     private ResultMessage resultMessage(List<PermissionDetails> details)
     {
         if (details.isEmpty())
@@ -108,16 +120,11 @@
         ResultSet result = new ResultSet(metadata);
         for (PermissionDetails pd : details)
         {
-            result.addColumnValue(UTF8Type.instance.decompose(pd.username));
+            result.addColumnValue(UTF8Type.instance.decompose(pd.grantee));
+            result.addColumnValue(UTF8Type.instance.decompose(pd.grantee));
             result.addColumnValue(UTF8Type.instance.decompose(pd.resource.toString()));
             result.addColumnValue(UTF8Type.instance.decompose(pd.permission.toString()));
         }
         return new ResultMessage.Rows(result);
     }
-
-    private Set<PermissionDetails> list(ClientState state, IResource resource)
-    throws RequestValidationException, RequestExecutionException
-    {
-        return DatabaseDescriptor.getAuthorizer().list(state.getUser(), permissions, resource, username);
-    }
 }
diff --git a/src/java/org/apache/cassandra/cql3/statements/ListRolesStatement.java b/src/java/org/apache/cassandra/cql3/statements/ListRolesStatement.java
new file mode 100644
index 0000000..477aedc
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/ListRolesStatement.java
@@ -0,0 +1,126 @@
+/*
+ * 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.cql3.statements;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.auth.*;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.MapType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.exceptions.*;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.transport.messages.ResultMessage;
+
+public class ListRolesStatement extends AuthorizationStatement
+{
+    // pseudo-virtual cf as the actual datasource is dependent on the IRoleManager impl
+    private static final String KS = AuthKeyspace.NAME;
+    private static final String CF = AuthKeyspace.ROLES;
+
+    private static final MapType optionsType = MapType.getInstance(UTF8Type.instance, UTF8Type.instance, false);
+    private static final List<ColumnSpecification> metadata =
+        ImmutableList.of(new ColumnSpecification(KS, CF, new ColumnIdentifier("role", true), UTF8Type.instance),
+                         new ColumnSpecification(KS, CF, new ColumnIdentifier("super", true), BooleanType.instance),
+                         new ColumnSpecification(KS, CF, new ColumnIdentifier("login", true), BooleanType.instance),
+                         new ColumnSpecification(KS, CF, new ColumnIdentifier("options", true), optionsType));
+
+    private final RoleResource grantee;
+    private final boolean recursive;
+
+    public ListRolesStatement()
+    {
+        this(new RoleName(), false);
+    }
+
+    public ListRolesStatement(RoleName grantee, boolean recursive)
+    {
+        this.grantee = grantee.hasName() ? RoleResource.role(grantee.getName()) : null;
+        this.recursive = recursive;
+    }
+
+    public void validate(ClientState state) throws UnauthorizedException, InvalidRequestException
+    {
+        state.ensureNotAnonymous();
+
+        if ((grantee != null) && !DatabaseDescriptor.getRoleManager().isExistingRole(grantee))
+            throw new InvalidRequestException(String.format("%s doesn't exist", grantee));
+    }
+
+    public void checkAccess(ClientState state) throws InvalidRequestException
+    {
+    }
+
+    public ResultMessage execute(ClientState state) throws RequestValidationException, RequestExecutionException
+    {
+        // If the executing user has DESCRIBE permission on the root roles resource, let them list any and all roles
+        boolean hasRootLevelSelect = DatabaseDescriptor.getAuthorizer()
+                                                       .authorize(state.getUser(), RoleResource.root())
+                                                       .contains(Permission.DESCRIBE);
+        if (hasRootLevelSelect)
+        {
+            if (grantee == null)
+                return resultMessage(DatabaseDescriptor.getRoleManager().getAllRoles());
+            else
+                return resultMessage(DatabaseDescriptor.getRoleManager().getRoles(grantee, recursive));
+        }
+        else
+        {
+            RoleResource currentUser = RoleResource.role(state.getUser().getName());
+            if (grantee == null)
+                return resultMessage(DatabaseDescriptor.getRoleManager().getRoles(currentUser, recursive));
+            if (DatabaseDescriptor.getRoleManager().getRoles(currentUser, true).contains(grantee))
+                return resultMessage(DatabaseDescriptor.getRoleManager().getRoles(grantee, recursive));
+            else
+                throw new UnauthorizedException(String.format("You are not authorized to view roles granted to %s ", grantee.getRoleName()));
+        }
+    }
+
+    private ResultMessage resultMessage(Set<RoleResource> roles)
+    {
+        if (roles.isEmpty())
+            return new ResultMessage.Void();
+
+        List<RoleResource> sorted = Lists.newArrayList(roles);
+        Collections.sort(sorted);
+        return formatResults(sorted);
+    }
+
+    // overridden in ListUsersStatement to include legacy metadata
+    protected ResultMessage formatResults(List<RoleResource> sortedRoles)
+    {
+        ResultSet result = new ResultSet(metadata);
+
+        IRoleManager roleManager = DatabaseDescriptor.getRoleManager();
+        for (RoleResource role : sortedRoles)
+        {
+            result.addColumnValue(UTF8Type.instance.decompose(role.getRoleName()));
+            result.addColumnValue(BooleanType.instance.decompose(roleManager.isSuper(role)));
+            result.addColumnValue(BooleanType.instance.decompose(roleManager.canLogin(role)));
+            result.addColumnValue(optionsType.decompose(roleManager.getCustomOptions(role)));
+        }
+        return new ResultMessage.Rows(result);
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/statements/ListUsersStatement.java b/src/java/org/apache/cassandra/cql3/statements/ListUsersStatement.java
index 8acbcab..7251980 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ListUsersStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ListUsersStatement.java
@@ -17,31 +17,42 @@
  */
 package org.apache.cassandra.cql3.statements;
 
-import org.apache.cassandra.auth.Auth;
-import org.apache.cassandra.cql3.QueryProcessor;
-import org.apache.cassandra.db.ConsistencyLevel;
-import org.apache.cassandra.exceptions.RequestExecutionException;
-import org.apache.cassandra.exceptions.RequestValidationException;
-import org.apache.cassandra.exceptions.UnauthorizedException;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.QueryState;
+import java.util.List;
+
+import com.google.common.collect.ImmutableList;
+
+import org.apache.cassandra.auth.*;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.ResultSet;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.transport.messages.ResultMessage;
 
-public class ListUsersStatement extends AuthenticationStatement
+public class ListUsersStatement extends ListRolesStatement
 {
-    public void validate(ClientState state)
-    {
-    }
+    // pseudo-virtual cf as the actual datasource is dependent on the IRoleManager impl
+    private static final String KS = AuthKeyspace.NAME;
+    private static final String CF = "users";
 
-    public void checkAccess(ClientState state) throws UnauthorizedException
-    {
-        state.ensureNotAnonymous();
-    }
+    private static final List<ColumnSpecification> metadata =
+        ImmutableList.of(new ColumnSpecification(KS, CF, new ColumnIdentifier("name", true), UTF8Type.instance),
+                         new ColumnSpecification(KS, CF, new ColumnIdentifier("super", true), BooleanType.instance));
 
-    public ResultMessage execute(ClientState state) throws RequestValidationException, RequestExecutionException
+    @Override
+    protected ResultMessage formatResults(List<RoleResource> sortedRoles)
     {
-        return QueryProcessor.process(String.format("SELECT * FROM %s.%s", Auth.AUTH_KS, Auth.USERS_CF),
-                                      ConsistencyLevel.QUORUM,
-                                      QueryState.forInternalCalls());
+        ResultSet result = new ResultSet(metadata);
+
+        IRoleManager roleManager = DatabaseDescriptor.getRoleManager();
+        for (RoleResource role : sortedRoles)
+        {
+            if (!roleManager.canLogin(role))
+                continue;
+            result.addColumnValue(UTF8Type.instance.decompose(role.getRoleName()));
+            result.addColumnValue(BooleanType.instance.decompose(Roles.hasSuperuserStatus(role)));
+        }
+        return new ResultMessage.Rows(result);
     }
 }
diff --git a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
index f84188a..3d134b5 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@ -20,17 +20,21 @@
 import java.nio.ByteBuffer;
 import java.util.*;
 
-import com.google.common.base.Function;
 import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
 
 import org.apache.cassandra.auth.Permission;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.restrictions.Restriction;
+import org.apache.cassandra.cql3.restrictions.SingleColumnRestriction;
+import org.apache.cassandra.cql3.selection.Selection;
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.composites.CBuilder;
 import org.apache.cassandra.db.composites.Composite;
+import org.apache.cassandra.db.composites.CompositesBuilder;
 import org.apache.cassandra.db.filter.ColumnSlice;
 import org.apache.cassandra.db.filter.SliceQueryFilter;
 import org.apache.cassandra.db.marshal.BooleanType;
@@ -44,6 +48,9 @@
 import org.apache.cassandra.triggers.TriggerExecutor;
 import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.UUIDGen;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
+import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
 
 /*
  * Abstract parent class of individual modifications, i.e. INSERT, UPDATE and DELETE.
@@ -73,7 +80,8 @@
     private boolean setsStaticColumns;
     private boolean setsRegularColumns;
 
-    private final Function<ColumnCondition, ColumnDefinition> getColumnForCondition = new Function<ColumnCondition, ColumnDefinition>()
+    private final com.google.common.base.Function<ColumnCondition, ColumnDefinition> getColumnForCondition =
+      new com.google.common.base.Function<ColumnCondition, ColumnDefinition>()
     {
         public ColumnDefinition apply(ColumnCondition cond)
         {
@@ -89,6 +97,27 @@
         this.attrs = attrs;
     }
 
+    public Iterable<Function> getFunctions()
+    {
+        List<Iterable<Function>> iterables = new LinkedList<>();
+        for (Restriction restriction : processedKeys.values())
+            iterables.add(restriction.getFunctions());
+
+        if (columnOperations != null)
+            for (Operation operation : columnOperations)
+                iterables.add(operation.getFunctions());
+
+        if (columnConditions != null)
+            for (ColumnCondition condition : columnConditions)
+                iterables.add(condition.getFunctions());
+
+        if (staticConditions != null)
+            for (ColumnCondition condition : staticConditions)
+                iterables.add(condition.getFunctions());
+
+        return Iterables.concat(iterables);
+    }
+
     public abstract boolean requireFullClusteringKey();
     public abstract void addUpdateForKey(ColumnFamily updates, ByteBuffer key, Composite prefix, UpdateParameters params) throws InvalidRequestException;
 
@@ -134,6 +163,9 @@
         // CAS updates can be used to simulate a SELECT query, so should require Permission.SELECT as well.
         if (hasConditions())
             state.hasColumnFamilyAccess(keyspace(), columnFamily(), Permission.SELECT);
+
+        for (Function function : getFunctions())
+            state.ensureHasPermission(Permission.EXECUTE, function);
     }
 
     public void validate(ClientState state) throws InvalidRequestException
@@ -231,7 +263,7 @@
 
     public void addKeyValue(ColumnDefinition def, Term value) throws InvalidRequestException
     {
-        addKeyValues(def, new SingleColumnRestriction.EQ(value, false));
+        addKeyValues(def, new SingleColumnRestriction.EQ(def, value));
     }
 
     public void processWhereClause(List<Relation> whereClause, VariableSpecifications names) throws InvalidRequestException
@@ -245,7 +277,7 @@
             }
             SingleColumnRelation rel = (SingleColumnRelation) relation;
 
-            if (rel.onToken)
+            if (rel.onToken())
                 throw new InvalidRequestException(String.format("The token function cannot be used in WHERE clauses for UPDATE and DELETE statements: %s", relation));
 
             ColumnIdentifier id = rel.getEntity().prepare(cfm);
@@ -259,31 +291,9 @@
                 case CLUSTERING_COLUMN:
                     Restriction restriction;
 
-                    if (rel.operator() == Operator.EQ)
+                    if (rel.isEQ() || (def.isPartitionKey() && rel.isIN()))
                     {
-                        Term t = rel.getValue().prepare(keyspace(), def);
-                        t.collectMarkerSpecification(names);
-                        restriction = new SingleColumnRestriction.EQ(t, false);
-                    }
-                    else if (def.kind == ColumnDefinition.Kind.PARTITION_KEY && rel.operator() == Operator.IN)
-                    {
-                        if (rel.getValue() != null)
-                        {
-                            Term t = rel.getValue().prepare(keyspace(), def);
-                            t.collectMarkerSpecification(names);
-                            restriction = new SingleColumnRestriction.InWithMarker((Lists.Marker)t);
-                        }
-                        else
-                        {
-                            List<Term> values = new ArrayList<Term>(rel.getInValues().size());
-                            for (Term.Raw raw : rel.getInValues())
-                            {
-                                Term t = raw.prepare(keyspace(), def);
-                                t.collectMarkerSpecification(names);
-                                values.add(t);
-                            }
-                            restriction = new SingleColumnRestriction.InWithValues(values);
-                        }
+                        restriction = rel.toRestriction(cfm, names);
                     }
                     else
                     {
@@ -301,38 +311,23 @@
     public List<ByteBuffer> buildPartitionKeyNames(QueryOptions options)
     throws InvalidRequestException
     {
-        CBuilder keyBuilder = cfm.getKeyValidatorAsCType().builder();
-        List<ByteBuffer> keys = new ArrayList<ByteBuffer>();
+        CompositesBuilder keyBuilder = new CompositesBuilder(cfm.getKeyValidatorAsCType());
         for (ColumnDefinition def : cfm.partitionKeyColumns())
         {
-            Restriction r = processedKeys.get(def.name);
-            if (r == null)
-                throw new InvalidRequestException(String.format("Missing mandatory PRIMARY KEY part %s", def.name));
-
-            List<ByteBuffer> values = r.values(options);
-
-            if (keyBuilder.remainingCount() == 1)
-            {
-                for (ByteBuffer val : values)
-                {
-                    if (val == null)
-                        throw new InvalidRequestException(String.format("Invalid null value for partition key part %s", def.name));
-                    ByteBuffer key = keyBuilder.buildWith(val).toByteBuffer();
-                    ThriftValidation.validateKey(cfm, key);
-                    keys.add(key);
-                }
-            }
-            else
-            {
-                if (values.size() != 1)
-                    throw new InvalidRequestException("IN is only supported on the last column of the partition key");
-                ByteBuffer val = values.get(0);
-                if (val == null)
-                    throw new InvalidRequestException(String.format("Invalid null value for partition key part %s", def.name));
-                keyBuilder.add(val);
-            }
+            Restriction r = checkNotNull(processedKeys.get(def.name), "Missing mandatory PRIMARY KEY part %s", def.name);
+            r.appendTo(cfm, keyBuilder, options);
         }
-        return keys;
+
+        return Lists.transform(filterAndSort(keyBuilder.build()), new com.google.common.base.Function<Composite, ByteBuffer>()
+        {
+            @Override
+            public ByteBuffer apply(Composite composite)
+            {
+                ByteBuffer byteBuffer = composite.toByteBuffer();
+                ThriftValidation.validateKey(cfm, byteBuffer);
+                return byteBuffer;
+            }
+        });
     }
 
     public Composite createClusteringPrefix(QueryOptions options)
@@ -394,7 +389,7 @@
     private Composite createClusteringPrefixBuilderInternal(QueryOptions options)
     throws InvalidRequestException
     {
-        CBuilder builder = cfm.comparator.prefixBuilder();
+        CompositesBuilder builder = new CompositesBuilder(cfm.comparator);
         ColumnDefinition firstEmptyKey = null;
         for (ColumnDefinition def : cfm.clusteringColumns())
         {
@@ -402,24 +397,35 @@
             if (r == null)
             {
                 firstEmptyKey = def;
-                if (requireFullClusteringKey() && !cfm.comparator.isDense() && cfm.comparator.isCompound())
-                    throw new InvalidRequestException(String.format("Missing mandatory PRIMARY KEY part %s", def.name));
+                checkFalse(requireFullClusteringKey() && !cfm.comparator.isDense() && cfm.comparator.isCompound(), 
+                           "Missing mandatory PRIMARY KEY part %s", def.name);
             }
             else if (firstEmptyKey != null)
             {
-                throw new InvalidRequestException(String.format("Missing PRIMARY KEY part %s since %s is set", firstEmptyKey.name, def.name));
+                throw invalidRequest("Missing PRIMARY KEY part %s since %s is set", firstEmptyKey.name, def.name);
             }
             else
             {
-                List<ByteBuffer> values = r.values(options);
-                assert values.size() == 1; // We only allow IN for row keys so far
-                ByteBuffer val = values.get(0);
-                if (val == null)
-                    throw new InvalidRequestException(String.format("Invalid null value for clustering key part %s", def.name));
-                builder.add(val);
+                r.appendTo(cfm, builder, options);
             }
         }
-        return builder.build();
+        return builder.build().get(0); // We only allow IN for row keys so far
+    }
+
+    /**
+     * Removes duplicates and sort the specified composites.
+     *
+     * @param composites the composites to filter and sort
+     * @return the composites sorted and without duplicates
+     */
+    private List<Composite> filterAndSort(List<Composite> composites)
+    {
+        if (composites.size() <= 1)
+            return composites;
+
+        TreeSet<Composite> set = new TreeSet<Composite>(cfm.getKeyValidatorAsCType());
+        set.addAll(composites);
+        return new ArrayList<>(set);
     }
 
     protected ColumnDefinition getFirstEmptyKey()
@@ -545,7 +551,6 @@
     }
 
     private CQL3CasRequest makeCasRequest(QueryState queryState, QueryOptions options)
-    throws InvalidRequestException
     {
         List<ByteBuffer> keys = buildPartitionKeyNames(options);
         // We don't support IN for CAS operation so far
@@ -595,7 +600,7 @@
         boolean success = cf == null;
 
         ColumnSpecification spec = new ColumnSpecification(ksName, cfName, CAS_RESULT_COLUMN, BooleanType.instance);
-        ResultSet.Metadata metadata = new ResultSet.Metadata(Collections.singletonList(spec));
+        ResultSet.ResultMetadata metadata = new ResultSet.ResultMetadata(Collections.singletonList(spec));
         List<List<ByteBuffer>> rows = Collections.singletonList(Collections.singletonList(BooleanType.instance.decompose(success)));
 
         ResultSet rs = new ResultSet(metadata, rows);
@@ -622,7 +627,7 @@
             row.addAll(right.rows.get(i));
             rows.add(row);
         }
-        return new ResultSet(new ResultSet.Metadata(specs), rows);
+        return new ResultSet(new ResultSet.ResultMetadata(specs), rows);
     }
 
     private static ResultSet buildCasFailureResultSet(ByteBuffer key, ColumnFamily cf, Iterable<ColumnDefinition> columnsWithConditions, boolean isBatch, QueryOptions options)
@@ -632,7 +637,7 @@
         Selection selection;
         if (columnsWithConditions == null)
         {
-            selection = Selection.wildcard(cfm, false, null);
+            selection = Selection.wildcard(cfm);
         }
         else
         {
@@ -648,14 +653,15 @@
             }
             for (ColumnDefinition def : columnsWithConditions)
                 defs.add(def);
-            selection = Selection.forColumns(new ArrayList<>(defs));
+            selection = Selection.forColumns(cfm, new ArrayList<>(defs));
+
         }
 
         long now = System.currentTimeMillis();
-        Selection.ResultSetBuilder builder = selection.resultSetBuilder(now);
+        Selection.ResultSetBuilder builder = selection.resultSetBuilder(now, false);
         SelectStatement.forSelection(cfm, selection).processColumnFamily(key, cf, options, now, builder);
 
-        return builder.build();
+        return builder.build(options.getProtocolVersion());
     }
 
     public ResultMessage executeInternal(QueryState queryState, QueryOptions options) throws RequestValidationException, RequestExecutionException
@@ -687,7 +693,6 @@
     }
 
     static ColumnFamily casInternal(CQL3CasRequest request, QueryState state)
-    throws InvalidRequestException
     {
         UUID ballot = UUIDGen.getTimeUUIDFromMicros(state.getTimestamp());
         CFMetaData metadata = Schema.instance.getCFMetaData(request.cfm.ksName, request.cfm.cfName);
@@ -786,7 +791,8 @@
         {
             VariableSpecifications boundNames = getBoundVariables();
             ModificationStatement statement = prepare(boundNames);
-            return new ParsedStatement.Prepared(statement, boundNames);
+            CFMetaData cfm = ThriftValidation.validateColumnFamily(keyspace(), columnFamily());
+            return new ParsedStatement.Prepared(statement, boundNames, boundNames.getPartitionKeyBindIndexes(cfm));
         }
 
         public ModificationStatement prepare(VariableSpecifications boundNames) throws InvalidRequestException
diff --git a/src/java/org/apache/cassandra/cql3/statements/MultiColumnRestriction.java b/src/java/org/apache/cassandra/cql3/statements/MultiColumnRestriction.java
deleted file mode 100644
index e2ba2a1..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/MultiColumnRestriction.java
+++ /dev/null
@@ -1,143 +0,0 @@
-/*
- * 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.cql3.statements;
-
-import org.apache.cassandra.cql3.AbstractMarker;
-import org.apache.cassandra.cql3.QueryOptions;
-import org.apache.cassandra.cql3.Term;
-import org.apache.cassandra.cql3.Tuples;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.cassandra.cql3.Term.Terminal;
-
-public interface MultiColumnRestriction extends Restriction
-{
-    public static class EQ extends SingleColumnRestriction.EQ implements MultiColumnRestriction
-    {
-        public EQ(Term value, boolean onToken)
-        {
-            super(value, onToken);
-        }
-
-        public boolean isMultiColumn()
-        {
-            return true;
-        }
-
-        public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
-        {
-            Tuples.Value t = (Tuples.Value)value.bind(options);
-            return t.getElements();
-        }
-    }
-
-    public interface IN extends MultiColumnRestriction
-    {
-        public List<List<ByteBuffer>> splitValues(QueryOptions options) throws InvalidRequestException;
-    }
-
-    /**
-     * An IN restriction that has a set of terms for in values.
-     * For example: "SELECT ... WHERE (a, b, c) IN ((1, 2, 3), (4, 5, 6))" or "WHERE (a, b, c) IN (?, ?)"
-     */
-    public static class InWithValues extends SingleColumnRestriction.InWithValues implements MultiColumnRestriction.IN
-    {
-        public InWithValues(List<? extends Term> values)
-        {
-            super(values);
-        }
-
-        public boolean isMultiColumn()
-        {
-            return true;
-        }
-
-        public List<List<ByteBuffer>> splitValues(QueryOptions options) throws InvalidRequestException
-        {
-            List<List<ByteBuffer>> buffers = new ArrayList<>(values.size());
-            for (Term value : values)
-            {
-                Term.MultiItemTerminal term = (Term.MultiItemTerminal)value.bind(options);
-                buffers.add(term.getElements());
-            }
-            return buffers;
-        }
-    }
-
-    /**
-     * An IN restriction that uses a single marker for a set of IN values that are tuples.
-     * For example: "SELECT ... WHERE (a, b, c) IN ?"
-     */
-    public static class InWithMarker extends SingleColumnRestriction.InWithMarker implements MultiColumnRestriction.IN
-    {
-        public InWithMarker(AbstractMarker marker)
-        {
-            super(marker);
-        }
-
-        public boolean isMultiColumn()
-        {
-            return true;
-        }
-
-        public List<List<ByteBuffer>> splitValues(QueryOptions options) throws InvalidRequestException
-        {
-            Tuples.InMarker inMarker = (Tuples.InMarker)marker;
-            Tuples.InValue inValue = inMarker.bind(options);
-            if (inValue == null)
-                throw new InvalidRequestException("Invalid null value for IN restriction");
-            return inValue.getSplitValues();
-        }
-    }
-
-    public static class Slice extends SingleColumnRestriction.Slice implements MultiColumnRestriction
-    {
-        public Slice(boolean onToken)
-        {
-            super(onToken);
-        }
-
-        public boolean isMultiColumn()
-        {
-            return true;
-        }
-
-        public ByteBuffer bound(Bound b, QueryOptions options) throws InvalidRequestException
-        {
-            throw new UnsupportedOperationException("Multicolumn slice restrictions do not support bound()");
-        }
-
-        /**
-         * Similar to bounds(), but returns one ByteBuffer per-component in the bound instead of a single
-         * ByteBuffer to represent the entire bound.
-         */
-        public List<ByteBuffer> componentBounds(Bound b, QueryOptions options) throws InvalidRequestException
-        {
-            Terminal terminal = bounds[b.idx].bind(options);
-            if (terminal instanceof Tuples.Value)
-                return ((Tuples.Value) terminal).getElements();
-
-            return Collections.singletonList(terminal.get(options));
-        }
-    }
-}
diff --git a/src/java/org/apache/cassandra/cql3/statements/ParsedStatement.java b/src/java/org/apache/cassandra/cql3/statements/ParsedStatement.java
index d048327..539a957 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ParsedStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ParsedStatement.java
@@ -21,6 +21,7 @@
 import java.util.List;
 
 import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.exceptions.RequestValidationException;
 
 public abstract class ParsedStatement
@@ -44,21 +45,28 @@
     {
         public final CQLStatement statement;
         public final List<ColumnSpecification> boundNames;
+        public final Short[] partitionKeyBindIndexes;
 
-        public Prepared(CQLStatement statement, List<ColumnSpecification> boundNames)
+        protected Prepared(CQLStatement statement, List<ColumnSpecification> boundNames, Short[] partitionKeyBindIndexes)
         {
             this.statement = statement;
             this.boundNames = boundNames;
+            this.partitionKeyBindIndexes = partitionKeyBindIndexes;
         }
 
-        public Prepared(CQLStatement statement, VariableSpecifications names)
+        public Prepared(CQLStatement statement, VariableSpecifications names, Short[] partitionKeyBindIndexes)
         {
-            this(statement, names.getSpecifications());
+            this(statement, names.getSpecifications(), partitionKeyBindIndexes);
         }
 
         public Prepared(CQLStatement statement)
         {
-            this(statement, Collections.<ColumnSpecification>emptyList());
+            this(statement, Collections.<ColumnSpecification>emptyList(), null);
         }
     }
+
+    public Iterable<Function> getFunctions()
+    {
+        return Collections.emptyList();
+    }
 }
diff --git a/src/java/org/apache/cassandra/cql3/statements/PermissionAlteringStatement.java b/src/java/org/apache/cassandra/cql3/statements/PermissionAlteringStatement.java
deleted file mode 100644
index 5f3b70c..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/PermissionAlteringStatement.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * 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.cql3.statements;
-
-import java.util.Set;
-
-import org.apache.cassandra.auth.Auth;
-import org.apache.cassandra.auth.DataResource;
-import org.apache.cassandra.auth.IResource;
-import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.RequestValidationException;
-import org.apache.cassandra.exceptions.UnauthorizedException;
-import org.apache.cassandra.service.ClientState;
-
-public abstract class PermissionAlteringStatement extends AuthorizationStatement
-{
-    protected final Set<Permission> permissions;
-    protected DataResource resource;
-    protected final String username;
-
-    protected PermissionAlteringStatement(Set<Permission> permissions, IResource resource, String username)
-    {
-        this.permissions = permissions;
-        this.resource = (DataResource) resource;
-        this.username = username;
-    }
-
-    public void validate(ClientState state) throws RequestValidationException
-    {
-        // validate login here before checkAccess to avoid leaking user existence to anonymous users.
-        state.ensureNotAnonymous();
-
-        if (!Auth.isExistingUser(username))
-            throw new InvalidRequestException(String.format("User %s doesn't exist", username));
-
-        // if a keyspace is omitted when GRANT/REVOKE ON TABLE <table>, we need to correct the resource.
-        resource = maybeCorrectResource(resource, state);
-        if (!resource.exists())
-            throw new InvalidRequestException(String.format("%s doesn't exist", resource));
-    }
-
-    public void checkAccess(ClientState state) throws UnauthorizedException
-    {
-        // check that the user has AUTHORIZE permission on the resource or its parents, otherwise reject GRANT/REVOKE.
-        state.ensureHasPermission(Permission.AUTHORIZE, resource);
-        // check that the user has [a single permission or all in case of ALL] on the resource or its parents.
-        for (Permission p : permissions)
-            state.ensureHasPermission(p, resource);
-    }
-}
diff --git a/src/java/org/apache/cassandra/cql3/statements/PermissionsManagementStatement.java b/src/java/org/apache/cassandra/cql3/statements/PermissionsManagementStatement.java
new file mode 100644
index 0000000..56a2f26
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/PermissionsManagementStatement.java
@@ -0,0 +1,79 @@
+/*
+ * 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.cql3.statements;
+
+import java.util.Set;
+
+import org.apache.cassandra.auth.*;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.RoleName;
+import org.apache.cassandra.db.SystemKeyspace;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.exceptions.RequestValidationException;
+import org.apache.cassandra.exceptions.UnauthorizedException;
+import org.apache.cassandra.service.ClientState;
+
+public abstract class PermissionsManagementStatement extends AuthorizationStatement
+{
+    protected final Set<Permission> permissions;
+    protected IResource resource;
+    protected final RoleResource grantee;
+
+    protected PermissionsManagementStatement(Set<Permission> permissions, IResource resource, RoleName grantee)
+    {
+        this.permissions = permissions;
+        this.resource = resource;
+        this.grantee = RoleResource.role(grantee.getName());
+    }
+
+    public void validate(ClientState state) throws RequestValidationException
+    {
+        // validate login here before checkAccess to avoid leaking user existence to anonymous users.
+        state.ensureNotAnonymous();
+
+        if (!DatabaseDescriptor.getRoleManager().isExistingRole(grantee))
+            throw new InvalidRequestException(String.format("Role %s doesn't exist", grantee.getRoleName()));
+
+        // if a keyspace is omitted when GRANT/REVOKE ON TABLE <table>, we need to correct the resource.
+        // called both here and in checkAccess(), as in some cases we do not call the latter.
+        resource = maybeCorrectResource(resource, state);
+
+        // altering permissions on builtin functions is not supported
+        if (resource instanceof FunctionResource
+            && SystemKeyspace.NAME.equals(((FunctionResource)resource).getKeyspace()))
+        {
+            throw new InvalidRequestException("Altering permissions on builtin functions is not supported");
+        }
+
+        if (!resource.exists())
+            throw new InvalidRequestException(String.format("Resource %s doesn't exist", resource));
+    }
+
+    public void checkAccess(ClientState state) throws UnauthorizedException
+    {
+        // if a keyspace is omitted when GRANT/REVOKE ON TABLE <table>, we need to correct the resource.
+        resource = maybeCorrectResource(resource, state);
+
+        // check that the user has AUTHORIZE permission on the resource or its parents, otherwise reject GRANT/REVOKE.
+        state.ensureHasPermission(Permission.AUTHORIZE, resource);
+
+        // check that the user has [a single permission or all in case of ALL] on the resource or its parents.
+        for (Permission p : permissions)
+            state.ensureHasPermission(p, resource);
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/statements/PropertyDefinitions.java b/src/java/org/apache/cassandra/cql3/statements/PropertyDefinitions.java
index eb4f074..793285b 100644
--- a/src/java/org/apache/cassandra/cql3/statements/PropertyDefinitions.java
+++ b/src/java/org/apache/cassandra/cql3/statements/PropertyDefinitions.java
@@ -94,8 +94,8 @@
         return (value == null) ? defaultValue : value.toLowerCase().matches("(1|true|yes)");
     }
 
-    // Return a property value, typed as a Double
-    public Double getDouble(String key, Double defaultValue) throws SyntaxException
+    // Return a property value, typed as a double
+    public double getDouble(String key, double defaultValue) throws SyntaxException
     {
         String value = getSimple(key);
         if (value == null)
@@ -106,7 +106,7 @@
         {
             try
             {
-                return Double.valueOf(value);
+                return Double.parseDouble(value);
             }
             catch (NumberFormatException e)
             {
diff --git a/src/java/org/apache/cassandra/cql3/statements/RawSelector.java b/src/java/org/apache/cassandra/cql3/statements/RawSelector.java
deleted file mode 100644
index c2d4e20..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/RawSelector.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.cql3.statements;
-
-import org.apache.cassandra.cql3.ColumnIdentifier;
-
-public class RawSelector
-{
-    public final Selectable.Raw selectable;
-    public final ColumnIdentifier alias;
-
-    public RawSelector(Selectable.Raw selectable, ColumnIdentifier alias)
-    {
-        this.selectable = selectable;
-        this.alias = alias;
-    }
-
-    public boolean processesSelection()
-    {
-        return selectable.processesSelection();
-    }
-}
diff --git a/src/java/org/apache/cassandra/cql3/statements/RequestValidations.java b/src/java/org/apache/cassandra/cql3/statements/RequestValidations.java
new file mode 100644
index 0000000..fc07878
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/RequestValidations.java
@@ -0,0 +1,209 @@
+/*
+ * 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.cql3.statements;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.commons.lang3.ArrayUtils.EMPTY_OBJECT_ARRAY;
+
+/**
+ * Utility methods use to perform request validation.
+ */
+public final class RequestValidations
+{
+    /**
+     * Checks that the specified expression is <code>true</code>. If not an <code>InvalidRequestException</code> will
+     * be thrown.
+     *
+     * @param expression the expression to test
+     * @param message the error message
+     * @throws InvalidRequestException if the specified expression is <code>false</code>.
+     */
+    public static void checkTrue(boolean expression, String message) throws InvalidRequestException
+    {
+        checkTrue(expression, message, EMPTY_OBJECT_ARRAY);
+    }
+
+    /**
+     * Checks that the specified expression is <code>true</code>. If not an <code>InvalidRequestException</code> will
+     * be thrown.
+     *
+     * @param expression the expression to test
+     * @param messageTemplate the template used to build the error message
+     * @param messageArgs the message arguments
+     * @throws InvalidRequestException if the specified expression is <code>false</code>.
+     */
+    public static void checkTrue(boolean expression,
+                                 String messageTemplate,
+                                 Object... messageArgs)
+                                 throws InvalidRequestException
+    {
+        if (!expression)
+            throw invalidRequest(messageTemplate, messageArgs);
+    }
+
+    /**
+     * Checks that the specified list does not contains duplicates.
+     *
+     * @param list the list to test
+     * @param message the error message
+     * @throws InvalidRequestException if the specified list contains duplicates.
+     */
+    public static void checkContainsNoDuplicates(List<?> list, String message) throws InvalidRequestException
+    {
+        if (new HashSet<>(list).size() != list.size())
+            throw invalidRequest(message);
+    }
+
+    /**
+     * Checks that the specified list contains only the specified elements.
+     *
+     * @param list the list to test
+     * @param expectedElements the expected elements
+     * @param message the error message
+     * @throws InvalidRequestException if the specified list contains duplicates.
+     */
+    public static <E> void checkContainsOnly(List<E> list,
+                                             List<E> expectedElements,
+                                             String message) throws InvalidRequestException
+    {
+        List<E> copy = new ArrayList<>(list);
+        copy.removeAll(expectedElements);
+        if (!copy.isEmpty())
+            throw invalidRequest(message);
+    }
+
+    /**
+     * Checks that the specified expression is <code>false</code>. If not an <code>InvalidRequestException</code> will
+     * be thrown.
+     *
+     * @param expression the expression to test
+     * @param messageTemplate the template used to build the error message
+     * @param messageArgs the message arguments
+     * @throws InvalidRequestException if the specified expression is <code>true</code>.
+     */
+    public static void checkFalse(boolean expression,
+                                  String messageTemplate,
+                                  Object... messageArgs)
+                                  throws InvalidRequestException
+    {
+        checkTrue(!expression, messageTemplate, messageArgs);
+    }
+
+    /**
+     * Checks that the specified expression is <code>false</code>. If not an <code>InvalidRequestException</code> will
+     * be thrown.
+     *
+     * @param expression the expression to test
+     * @param message the error message
+     * @throws InvalidRequestException if the specified expression is <code>true</code>.
+     */
+    public static void checkFalse(boolean expression, String message) throws InvalidRequestException
+    {
+        checkTrue(!expression, message);
+    }
+
+    /**
+     * Checks that the specified object is NOT <code>null</code>.
+     * If it is an <code>InvalidRequestException</code> will be throws.
+     *
+     * @param object the object to test
+     * @param messageTemplate the template used to build the error message
+     * @param messageArgs the message arguments
+     * @return the object
+     * @throws InvalidRequestException if the specified object is <code>null</code>.
+     */
+    public static <T> T checkNotNull(T object, String messageTemplate, Object... messageArgs)
+            throws InvalidRequestException
+    {
+        checkTrue(object != null, messageTemplate, messageArgs);
+        return object;
+    }
+
+    /**
+     * Checks that the specified bind marker value is set to a meaningful value.
+     * If it is not a <code>InvalidRequestException</code> will be thrown.
+     *
+     * @param b the <code>ByteBuffer</code> to test
+     * @param messageTemplate the template used to build the error message
+     * @param messageArgs the message arguments
+     * @throws InvalidRequestException if the specified bind marker value is not set to a meaningful value.
+     */
+    public static void checkBindValueSet(ByteBuffer b, String messageTemplate, Object... messageArgs)
+            throws InvalidRequestException
+    {
+        checkTrue(b != ByteBufferUtil.UNSET_BYTE_BUFFER, messageTemplate, messageArgs);
+    }
+
+    /**
+     * Checks that the specified object is <code>null</code>.
+     * If it is not an <code>InvalidRequestException</code> will be throws.
+     *
+     * @param object the object to test
+     * @param messageTemplate the template used to build the error message
+     * @param messageArgs the message arguments
+     * @return the object
+     * @throws InvalidRequestException if the specified object is not <code>null</code>.
+     */
+    public static <T> T checkNull(T object, String messageTemplate, Object... messageArgs)
+            throws InvalidRequestException
+    {
+        checkTrue(object == null, messageTemplate, messageArgs);
+        return object;
+    }
+
+    /**
+     * Checks that the specified object is <code>null</code>.
+     * If it is not an <code>InvalidRequestException</code> will be throws.
+     *
+     * @param object the object to test
+     * @param message the error message
+     * @return the object
+     * @throws InvalidRequestException if the specified object is not <code>null</code>.
+     */
+    public static <T> T checkNull(T object, String message) throws InvalidRequestException
+    {
+        return checkNull(object, message, EMPTY_OBJECT_ARRAY);
+    }
+
+    /**
+     * Returns an <code>InvalidRequestException</code> with the specified message.
+     *
+     * @param messageTemplate the template used to build the error message
+     * @param messageArgs the message arguments
+     * @return an <code>InvalidRequestException</code> with the specified message.
+     */
+    public static InvalidRequestException invalidRequest(String messageTemplate, Object... messageArgs)
+    {
+        return new InvalidRequestException(String.format(messageTemplate, messageArgs));
+    }
+
+    /**
+     * This class must not be instantiated as it only contains static methods.
+     */
+    private RequestValidations()
+    {
+
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/statements/Restriction.java b/src/java/org/apache/cassandra/cql3/statements/Restriction.java
deleted file mode 100644
index bd0277c..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/Restriction.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * 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.cql3.statements;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-
-import org.apache.cassandra.cql3.Operator;
-import org.apache.cassandra.cql3.QueryOptions;
-import org.apache.cassandra.cql3.Term;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-
-/**
- * A restriction/clause on a column.
- * The goal of this class being to group all conditions for a column in a SELECT.
- */
-public interface Restriction
-{
-    public boolean isOnToken();
-
-    public boolean isSlice();
-    public boolean isEQ();
-    public boolean isIN();
-    public boolean isContains();
-    public boolean isMultiColumn();
-
-    /**
-     * Returns true if, when applied to a clustering column, this restriction can be handled through one or more slices
-     * alone without filtering.  For example, EQ restrictions can be represented as a slice, but CONTAINS cannot.
-     */
-    public boolean canEvaluateWithSlices();
-
-    // Not supported by Slice, but it's convenient to have here
-    public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException;
-
-    public static interface EQ extends Restriction {}
-
-    public static interface IN extends Restriction
-    {
-        public boolean canHaveOnlyOneValue();
-    }
-
-    public static interface Slice extends Restriction
-    {
-        public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException;
-
-        /** Returns true if the start or end bound (depending on the argument) is set, false otherwise */
-        public boolean hasBound(Bound b);
-
-        public Term bound(Bound b);
-
-        public ByteBuffer bound(Bound b, QueryOptions options) throws InvalidRequestException;
-
-        /** Returns true if the start or end bound (depending on the argument) is inclusive, false otherwise */
-        public boolean isInclusive(Bound b);
-
-        public Operator getRelation(Bound eocBound, Bound inclusiveBound);
-
-        public Operator getIndexOperator(Bound b);
-
-        public void setBound(Operator type, Term t) throws InvalidRequestException;
-
-        public void setBound(Slice restriction) throws InvalidRequestException;
-    }
-}
diff --git a/src/java/org/apache/cassandra/cql3/statements/RevokePermissionsStatement.java b/src/java/org/apache/cassandra/cql3/statements/RevokePermissionsStatement.java
new file mode 100644
index 0000000..9acc685
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/RevokePermissionsStatement.java
@@ -0,0 +1,43 @@
+/*
+ * 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.cql3.statements;
+
+import java.util.Set;
+
+import org.apache.cassandra.auth.IResource;
+import org.apache.cassandra.auth.Permission;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.RoleName;
+import org.apache.cassandra.exceptions.RequestExecutionException;
+import org.apache.cassandra.exceptions.RequestValidationException;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.transport.messages.ResultMessage;
+
+public class RevokePermissionsStatement extends PermissionsManagementStatement
+{
+    public RevokePermissionsStatement(Set<Permission> permissions, IResource resource, RoleName grantee)
+    {
+        super(permissions, resource, grantee);
+    }
+
+    public ResultMessage execute(ClientState state) throws RequestValidationException, RequestExecutionException
+    {
+        DatabaseDescriptor.getAuthorizer().revoke(state.getUser(), permissions, resource, grantee);
+        return null;
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/statements/RevokeRoleStatement.java b/src/java/org/apache/cassandra/cql3/statements/RevokeRoleStatement.java
new file mode 100644
index 0000000..4de905f
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/RevokeRoleStatement.java
@@ -0,0 +1,39 @@
+/*
+ * 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.cql3.statements;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.RoleName;
+import org.apache.cassandra.exceptions.RequestExecutionException;
+import org.apache.cassandra.exceptions.RequestValidationException;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.transport.messages.ResultMessage;
+
+public class RevokeRoleStatement extends RoleManagementStatement
+{
+    public RevokeRoleStatement(RoleName name, RoleName grantee)
+    {
+        super(name, grantee);
+    }
+
+    public ResultMessage execute(ClientState state) throws RequestValidationException, RequestExecutionException
+    {
+        DatabaseDescriptor.getRoleManager().revokeRole(state.getUser(), role, grantee);
+        return null;
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/statements/RevokeStatement.java b/src/java/org/apache/cassandra/cql3/statements/RevokeStatement.java
deleted file mode 100644
index e81a385..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/RevokeStatement.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/**
- * 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.cql3.statements;
-
-import java.util.Set;
-
-import org.apache.cassandra.auth.IResource;
-import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.exceptions.RequestExecutionException;
-import org.apache.cassandra.exceptions.RequestValidationException;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.transport.messages.ResultMessage;
-
-public class RevokeStatement extends PermissionAlteringStatement
-{
-    public RevokeStatement(Set<Permission> permissions, IResource resource, String username)
-    {
-        super(permissions, resource, username);
-    }
-
-    public ResultMessage execute(ClientState state) throws RequestValidationException, RequestExecutionException
-    {
-        DatabaseDescriptor.getAuthorizer().revoke(state.getUser(), permissions, resource, username);
-        return null;
-    }
-}
diff --git a/src/java/org/apache/cassandra/cql3/statements/RoleManagementStatement.java b/src/java/org/apache/cassandra/cql3/statements/RoleManagementStatement.java
new file mode 100644
index 0000000..e12b626
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/RoleManagementStatement.java
@@ -0,0 +1,55 @@
+/*
+ * 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.cql3.statements;
+
+import org.apache.cassandra.auth.Permission;
+import org.apache.cassandra.auth.RoleResource;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.RoleName;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.exceptions.RequestValidationException;
+import org.apache.cassandra.exceptions.UnauthorizedException;
+import org.apache.cassandra.service.ClientState;
+
+public abstract class RoleManagementStatement extends AuthenticationStatement
+{
+    protected final RoleResource role;
+    protected final RoleResource grantee;
+
+    public RoleManagementStatement(RoleName name, RoleName grantee)
+    {
+        this.role = RoleResource.role(name.getName());
+        this.grantee = RoleResource.role(grantee.getName());
+    }
+
+    public void checkAccess(ClientState state) throws UnauthorizedException
+    {
+        super.checkPermission(state, Permission.AUTHORIZE, role);
+    }
+
+    public void validate(ClientState state) throws RequestValidationException
+    {
+        state.ensureNotAnonymous();
+
+        if (!DatabaseDescriptor.getRoleManager().isExistingRole(role))
+            throw new InvalidRequestException(String.format("%s doesn't exist", role.getRoleName()));
+
+        if (!DatabaseDescriptor.getRoleManager().isExistingRole(grantee))
+            throw new InvalidRequestException(String.format("%s doesn't exist", grantee.getRoleName()));
+    }
+}
diff --git a/src/java/org/apache/cassandra/cql3/statements/SchemaAlteringStatement.java b/src/java/org/apache/cassandra/cql3/statements/SchemaAlteringStatement.java
index 8882871..a477df6 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SchemaAlteringStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SchemaAlteringStatement.java
@@ -17,10 +17,12 @@
  */
 package org.apache.cassandra.cql3.statements;
 
+import org.apache.cassandra.auth.AuthenticatedUser;
 import org.apache.cassandra.cql3.CFName;
 import org.apache.cassandra.cql3.CQLStatement;
 import org.apache.cassandra.cql3.QueryOptions;
-import org.apache.cassandra.exceptions.*;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.exceptions.RequestValidationException;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.transport.Event;
@@ -66,6 +68,17 @@
     public abstract Event.SchemaChange changeEvent();
 
     /**
+     * Schema alteration may result in a new database object (keyspace, table, role, function) being created capable of
+     * having permissions GRANTed on it. The creator of the object (the primary role assigned to the AuthenticatedUser
+     * performing the operation) is automatically granted ALL applicable permissions on the object. This is a hook for
+     * subclasses to override in order to perform that grant when the statement is executed.
+     */
+    protected void grantPermissionsToCreator(QueryState state)
+    {
+        // no-op by default
+    }
+
+    /**
      * Announces the migration to other nodes in the cluster.
      * @return true if the execution of this statement resulted in a schema change, false otherwise (when IF NOT EXISTS
      * is used, for example)
@@ -78,19 +91,39 @@
         // If an IF [NOT] EXISTS clause was used, this may not result in an actual schema change.  To avoid doing
         // extra work in the drivers to handle schema changes, we return an empty message in this case. (CASSANDRA-7600)
         boolean didChangeSchema = announceMigration(false);
-        return didChangeSchema ? new ResultMessage.SchemaChange(changeEvent()) : new ResultMessage.Void();
+        if (!didChangeSchema)
+            return new ResultMessage.Void();
+
+        Event.SchemaChange ce = changeEvent();
+
+        // when a schema alteration results in a new db object being created, we grant permissions on the new
+        // object to the user performing the request if:
+        // * the user is not anonymous
+        // * the configured IAuthorizer supports granting of permissions (not all do, AllowAllAuthorizer doesn't and
+        //   custom external implementations may not)
+        AuthenticatedUser user = state.getClientState().getUser();
+        if (user != null && !user.isAnonymous() && ce != null && ce.change == Event.SchemaChange.Change.CREATED)
+        {
+            try
+            {
+                grantPermissionsToCreator(state);
+            }
+            catch (UnsupportedOperationException e)
+            {
+                // not a problem, grant is an optional method on IAuthorizer
+            }
+        }
+
+        return ce == null ? new ResultMessage.Void() : new ResultMessage.SchemaChange(ce);
     }
 
     public ResultMessage executeInternal(QueryState state, QueryOptions options)
     {
-        try
-        {
-            boolean didChangeSchema = announceMigration(true);
-            return didChangeSchema ? new ResultMessage.SchemaChange(changeEvent()) : new ResultMessage.Void();
-        }
-        catch (RequestValidationException e)
-        {
-            throw new RuntimeException(e);
-        }
+        boolean didChangeSchema = announceMigration(true);
+        if (!didChangeSchema)
+            return new ResultMessage.Void();
+
+        Event.SchemaChange ce = changeEvent();
+        return ce == null ? new ResultMessage.Void() : new ResultMessage.SchemaChange(ce);
     }
 }
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 74fb201..13276c7 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -20,42 +20,50 @@
 import java.nio.ByteBuffer;
 import java.util.*;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Joiner;
 import com.google.common.base.Objects;
 import com.google.common.base.Predicate;
-import com.google.common.collect.AbstractIterator;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Iterators;
 
-import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.cql3.*;
-import org.apache.cassandra.cql3.statements.Restriction.Slice;
-import org.apache.cassandra.cql3.statements.SingleColumnRestriction.Contains;
-import org.apache.cassandra.db.composites.*;
-import org.apache.cassandra.transport.messages.ResultMessage;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.filter.*;
-import org.apache.cassandra.db.index.SecondaryIndex;
-import org.apache.cassandra.db.index.SecondaryIndexManager;
-import org.apache.cassandra.db.marshal.*;
-import org.apache.cassandra.dht.*;
-import org.apache.cassandra.exceptions.*;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.QueryState;
-import org.apache.cassandra.service.StorageProxy;
-import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.service.pager.*;
-import org.apache.cassandra.db.ConsistencyLevel;
-import org.apache.cassandra.thrift.ThriftValidation;
-import org.apache.cassandra.serializers.MarshalException;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.FBUtilities;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.auth.Permission;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.restrictions.StatementRestrictions;
+import org.apache.cassandra.cql3.selection.RawSelector;
+import org.apache.cassandra.cql3.selection.Selection;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.composites.*;
+import org.apache.cassandra.db.filter.*;
+import org.apache.cassandra.db.index.SecondaryIndexManager;
+import org.apache.cassandra.db.marshal.CollectionType;
+import org.apache.cassandra.db.marshal.CompositeType;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.exceptions.*;
+import org.apache.cassandra.serializers.MarshalException;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.ClientWarn;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.StorageProxy;
+import org.apache.cassandra.service.pager.Pageable;
+import org.apache.cassandra.service.pager.QueryPager;
+import org.apache.cassandra.service.pager.QueryPagers;
+import org.apache.cassandra.thrift.ThriftValidation;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkTrue;
+import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
+import static org.apache.cassandra.utils.ByteBufferUtil.UNSET_BYTE_BUFFER;
+
 /**
  * Encapsulates a completely parsed SELECT query, including the target
  * column family, expression, result count, and ordering clause.
@@ -77,80 +85,42 @@
     private final Selection selection;
     private final Term limit;
 
-    /** Restrictions on partitioning columns */
-    private final Restriction[] keyRestrictions;
+    private final StatementRestrictions restrictions;
 
-    /** Restrictions on clustering columns */
-    private final Restriction[] columnRestrictions;
+    private final boolean isReversed;
 
-    /** Restrictions on non-primary key columns (i.e. secondary index restrictions) */
-    private final Map<ColumnIdentifier, Restriction> metadataRestrictions = new HashMap<ColumnIdentifier, Restriction>();
-
-    // The map keys are the name of the columns that must be converted into IndexExpressions if a secondary index need
-    // to be used. The value specify if the column has an index that can be used to for the relation in which the column
-    // is specified.
-    private final Map<ColumnDefinition, Boolean> restrictedColumns = new HashMap<ColumnDefinition, Boolean>();
-    private Restriction.Slice sliceRestriction;
-
-    private boolean isReversed;
-    private boolean onToken;
-    private boolean isKeyRange;
-    private boolean keyIsInRelation;
-    private boolean usesSecondaryIndexing;
-
-    private Map<ColumnIdentifier, Integer> orderingIndexes;
-
-    private boolean selectsStaticColumns;
-    private boolean selectsOnlyStaticColumns;
+    /**
+     * The comparator used to orders results when multiple keys are selected (using IN).
+     */
+    private final Comparator<List<ByteBuffer>> orderingComparator;
 
     // Used by forSelection below
-    private static final Parameters defaultParameters = new Parameters(Collections.<ColumnIdentifier.Raw, Boolean>emptyMap(), false, false, null, false);
+    private static final Parameters defaultParameters = new Parameters(Collections.<ColumnIdentifier.Raw, Boolean>emptyMap(), false, false, false);
 
-    private static final Predicate<ColumnDefinition> isStaticFilter = new Predicate<ColumnDefinition>()
-    {
-        public boolean apply(ColumnDefinition def)
-        {
-            return def.isStatic();
-        }
-    };
-
-    public SelectStatement(CFMetaData cfm, int boundTerms, Parameters parameters, Selection selection, Term limit)
+    public SelectStatement(CFMetaData cfm,
+                           int boundTerms,
+                           Parameters parameters,
+                           Selection selection,
+                           StatementRestrictions restrictions,
+                           boolean isReversed,
+                           Comparator<List<ByteBuffer>> orderingComparator,
+                           Term limit)
     {
         this.cfm = cfm;
         this.boundTerms = boundTerms;
         this.selection = selection;
-        this.keyRestrictions = new Restriction[cfm.partitionKeyColumns().size()];
-        this.columnRestrictions = new Restriction[cfm.clusteringColumns().size()];
+        this.restrictions = restrictions;
+        this.isReversed = isReversed;
+        this.orderingComparator = orderingComparator;
         this.parameters = parameters;
         this.limit = limit;
-
-        // Now gather a few info on whether we should bother with static columns or not for this statement
-        initStaticColumnsInfo();
     }
 
-    private void initStaticColumnsInfo()
+    public Iterable<Function> getFunctions()
     {
-        if (!cfm.hasStaticColumns())
-            return;
-
-        // If it's a wildcard, we do select static but not only them
-        if (selection.isWildcard())
-        {
-            selectsStaticColumns = true;
-            return;
-        }
-
-        // Otherwise, check the selected columns
-        selectsStaticColumns = !Iterables.isEmpty(Iterables.filter(selection.getColumns(), isStaticFilter));
-        selectsOnlyStaticColumns = true;
-        for (ColumnDefinition def : selection.getColumns())
-        {
-            if (def.kind != ColumnDefinition.Kind.PARTITION_KEY && def.kind != ColumnDefinition.Kind.STATIC)
-            {
-                selectsOnlyStaticColumns = false;
-                break;
-            }
-        }
+        return Iterables.concat(selection.getFunctions(),
+                                restrictions.getFunctions(),
+                                limit != null ? limit.getFunctions() : Collections.<Function>emptySet());
     }
 
     // Creates a simple select based on the given selection.
@@ -158,35 +128,31 @@
     // queried data through processColumnFamily.
     static SelectStatement forSelection(CFMetaData cfm, Selection selection)
     {
-        return new SelectStatement(cfm, 0, defaultParameters, selection, null);
+        return new SelectStatement(cfm,
+                                   0,
+                                   defaultParameters,
+                                   selection,
+                                   StatementRestrictions.empty(cfm),
+                                   false,
+                                   null,
+                                   null);
     }
 
-    public ResultSet.Metadata getResultMetadata()
+    public ResultSet.ResultMetadata getResultMetadata()
     {
-        return parameters.isCount
-             ? ResultSet.makeCountMetadata(keyspace(), columnFamily(), parameters.countAlias)
-             : selection.getResultMetadata();
+        return selection.getResultMetadata(parameters.isJson);
     }
 
-    /**
-     * May be used by custom QueryHandler implementations
-     */
     public int getBoundTerms()
     {
         return boundTerms;
     }
 
-    /**
-     * May be used by custom QueryHandler implementations
-     */
-    public Selection getSelection()
-    {
-        return selection;
-    }
-
     public void checkAccess(ClientState state) throws InvalidRequestException, UnauthorizedException
     {
         state.hasColumnFamilyAccess(keyspace(), columnFamily(), Permission.SELECT);
+        for (Function function : getFunctions())
+            state.ensureHasPermission(Permission.EXECUTE, function);
     }
 
     public void validate(ClientState state) throws InvalidRequestException
@@ -197,107 +163,55 @@
     public ResultMessage.Rows execute(QueryState state, QueryOptions options) throws RequestExecutionException, RequestValidationException
     {
         ConsistencyLevel cl = options.getConsistency();
-        if (cl == null)
-            throw new InvalidRequestException("Invalid empty consistency level");
+        checkNotNull(cl, "Invalid empty consistency level");
 
         cl.validateForRead(keyspace());
 
         int limit = getLimit(options);
         long now = System.currentTimeMillis();
         Pageable command = getPageableCommand(options, limit, now);
-
-        int pageSize = options.getPageSize();
-        // A count query will never be paged for the user, but we always page it internally to avoid OOM.
-        // If we user provided a pageSize we'll use that to page internally (because why not), otherwise we use our default
-        // Note that if there are some nodes in the cluster with a version less than 2.0, we can't use paging (CASSANDRA-6707).
-        if (parameters.isCount && pageSize <= 0)
-            pageSize = DEFAULT_COUNT_PAGE_SIZE;
+        int pageSize = getPageSize(options);
 
         if (pageSize <= 0 || command == null || !QueryPagers.mayNeedPaging(command, pageSize))
-        {
             return execute(command, options, limit, now, state);
-        }
-        else
-        {
-            QueryPager pager = QueryPagers.pager(command, cl, state.getClientState(), options.getPagingState());
-            if (parameters.isCount)
-                return pageCountQuery(pager, options, pageSize, now, limit);
 
-            // We can't properly do post-query ordering if we page (see #6722)
-            if (needsPostQueryOrdering())
-                throw new InvalidRequestException("Cannot page queries with both ORDER BY and a IN restriction on the partition key; you must either remove the "
-                                                + "ORDER BY or the IN and sort client side, or disable paging for this query");
-
-            List<Row> page = pager.fetchPage(pageSize);
-            ResultMessage.Rows msg = processResults(page, options, limit, now);
-
-            if (!pager.isExhausted())
-                msg.result.metadata.setHasMorePages(pager.state());
-
-            return msg;
-        }
+        QueryPager pager = QueryPagers.pager(command, cl, state.getClientState(), options.getPagingState());
+        return execute(pager, options, limit, now, pageSize);
     }
 
     private Pageable getPageableCommand(QueryOptions options, int limit, long now) throws RequestValidationException
     {
-        if (isNotReturningAnyRows(options))
+        if (restrictions.isNotReturningAnyRows(options))
             return null;
 
         int limitForQuery = updateLimitForQuery(limit);
-        if (isKeyRange || usesSecondaryIndexing)
+        if (restrictions.isKeyRange() || restrictions.usesSecondaryIndexing())
             return getRangeCommand(options, limitForQuery, now);
 
         List<ReadCommand> commands = getSliceCommands(options, limitForQuery, now);
         return commands == null ? null : new Pageable.ReadCommands(commands, limitForQuery);
     }
 
-    /**
-     * Checks if the query will never return any rows.
-     *
-     * @param options the query options
-     * @return {@code true} if the query will never return any rows, {@false} otherwise
-     * @throws InvalidRequestException if the request is invalid
-     */
-    private boolean isNotReturningAnyRows(QueryOptions options) throws InvalidRequestException
-    {
-        // Dense non-compound tables do not accept empty ByteBuffers. By consequence, we know that:
-        // - any query with an EQ restriction containing an empty value will not return any result
-        // - any query with a slice restriction with an empty value for the END bound will not return any result
-
-        if (cfm.comparator.isDense() && !cfm.comparator.isCompound())
-        {
-            for (Restriction restriction : columnRestrictions)
-            {
-                if (restriction != null)
-                {
-                    if (restriction.isEQ())
-                    {
-                        for (ByteBuffer value : restriction.values(options))
-                        {
-                            if (!value.hasRemaining())
-                                return true;
-                        }
-                    }
-                    else if (restriction.isSlice() && ((Slice) restriction).hasBound(Bound.END))
-                    {
-                        ByteBuffer value = restriction.isMultiColumn()
-                                ? ((MultiColumnRestriction.Slice) restriction).componentBounds(Bound.END, options).get(0)
-                                : ((Slice) restriction).bound(Bound.END, options);
-
-                        return !value.hasRemaining();
-                    }
-                }
-            }
-        }
-        return false;
-    }
-
     public Pageable getPageableCommand(QueryOptions options) throws RequestValidationException
     {
         return getPageableCommand(options, getLimit(options), System.currentTimeMillis());
     }
 
-    private ResultMessage.Rows execute(Pageable command, QueryOptions options, int limit, long now, QueryState state) throws RequestValidationException, RequestExecutionException
+    private int getPageSize(QueryOptions options)
+    {
+        int pageSize = options.getPageSize();
+
+        // An aggregation query will never be paged for the user, but we always page it internally to avoid OOM.
+        // If we user provided a pageSize we'll use that to page internally (because why not), otherwise we use our default
+        // Note that if there are some nodes in the cluster with a version less than 2.0, we can't use paging (CASSANDRA-6707).
+        if (selection.isAggregate() && pageSize <= 0)
+            pageSize = DEFAULT_COUNT_PAGE_SIZE;
+
+        return  pageSize;
+    }
+
+    private ResultMessage.Rows execute(Pageable command, QueryOptions options, int limit, long now, QueryState state)
+    throws RequestValidationException, RequestExecutionException
     {
         List<Row> rows;
         if (command == null)
@@ -314,28 +228,58 @@
         return processResults(rows, options, limit, now);
     }
 
-    private ResultMessage.Rows pageCountQuery(QueryPager pager, QueryOptions options, int pageSize, long now, int limit) throws RequestValidationException, RequestExecutionException
+    private ResultMessage.Rows execute(QueryPager pager, QueryOptions options, int limit, long now, int pageSize)
+    throws RequestValidationException, RequestExecutionException
     {
-        int count = 0;
-        while (!pager.isExhausted())
+        if (selection.isAggregate())
+            return pageAggregateQuery(pager, options, pageSize, now);
+
+        // We can't properly do post-query ordering if we page (see #6722)
+        checkFalse(needsPostQueryOrdering(),
+                   "Cannot page queries with both ORDER BY and a IN restriction on the partition key;"
+                   + " you must either remove the ORDER BY or the IN and sort client side, or disable paging for this query");
+
+        List<Row> page = pager.fetchPage(pageSize);
+        ResultMessage.Rows msg = processResults(page, options, limit, now);
+
+        if (!pager.isExhausted())
+            msg.result.metadata.setHasMorePages(pager.state());
+
+        return msg;
+    }
+
+    private ResultMessage.Rows pageAggregateQuery(QueryPager pager, QueryOptions options, int pageSize, long now)
+            throws RequestValidationException, RequestExecutionException
+    {
+        if (!restrictions.hasPartitionKeyRestrictions())
         {
-            int maxLimit = pager.maxRemaining();
-            logger.debug("New maxLimit for paged count query is {}", maxLimit);
-            ResultSet rset = process(pager.fetchPage(pageSize), options, maxLimit, now);
-            count += rset.rows.size();
+            logger.warn("Aggregation query used without partition key");
+            ClientWarn.instance.warn("Aggregation query used without partition key");
+        }
+        else if (restrictions.keyIsInRelation())
+        {
+            logger.warn("Aggregation query used on multiple partition keys (IN restriction)");
+            ClientWarn.instance.warn("Aggregation query used on multiple partition keys (IN restriction)");
         }
 
-        // We sometimes query one more result than the user limit asks to handle exclusive bounds with compact tables (see updateLimitForQuery).
-        // So do make sure the count is not greater than what the user asked for.
-        ResultSet result = ResultSet.makeCountResult(keyspace(), columnFamily(), Math.min(count, limit), parameters.countAlias);
-        return new ResultMessage.Rows(result);
+        Selection.ResultSetBuilder result = selection.resultSetBuilder(now, parameters.isJson);
+        while (!pager.isExhausted())
+        {
+            for (Row row : pager.fetchPage(pageSize))
+            {
+                // Not columns match the query, skip
+                if (row.cf == null)
+                    continue;
+
+                processColumnFamily(row.key.getKey(), row.cf, options, now, result);
+            }
+        }
+        return new ResultMessage.Rows(result.build(options.getProtocolVersion()));
     }
 
     public ResultMessage.Rows processResults(List<Row> rows, QueryOptions options, int limit, long now) throws RequestValidationException
     {
-        // Even for count, we need to process the result as it'll group some column together in sparse column families
         ResultSet rset = process(rows, options, limit, now);
-        rset = parameters.isCount ? rset.makeCountResult(parameters.countAlias) : rset;
         return new ResultMessage.Rows(rset);
     }
 
@@ -357,10 +301,7 @@
     {
         int limit = getLimit(options);
         Pageable command = getPageableCommand(options, limit, now);
-
-        int pageSize = options.getPageSize();
-        if (parameters.isCount && pageSize <= 0)
-            pageSize = DEFAULT_COUNT_PAGE_SIZE;
+        int pageSize = getPageSize(options);
 
         if (pageSize <= 0 || command == null || !QueryPagers.mayNeedPaging(command, pageSize))
         {
@@ -372,30 +313,13 @@
 
             return processResults(rows, options, limit, now);
         }
-        else
-        {
-            QueryPager pager = QueryPagers.localPager(command);
-            if (parameters.isCount)
-                return pageCountQuery(pager, options, pageSize, now, limit);
 
-            // We can't properly do post-query ordering if we page (see #6722)
-            if (needsPostQueryOrdering())
-                throw new InvalidRequestException("Cannot page queries with both ORDER BY and a IN restriction on the partition key; you must either remove the "
-                                                  + "ORDER BY or the IN and sort client side, or disable paging for this query");
-
-            List<Row> page = pager.fetchPage(pageSize);
-            ResultMessage.Rows msg = processResults(page, options, limit, now);
-
-            if (!pager.isExhausted())
-                msg.result.metadata.setHasMorePages(pager.state());
-
-            return msg;
-        }
+        QueryPager pager = QueryPagers.localPager(command);
+        return execute(pager, options, limit, now, pageSize);
     }
 
     public ResultSet process(List<Row> rows) throws InvalidRequestException
     {
-        assert !parameters.isCount; // not yet needed
         QueryOptions options = QueryOptions.DEFAULT;
         return process(rows, options, getLimit(options), System.currentTimeMillis());
     }
@@ -410,11 +334,25 @@
         return cfm.cfName;
     }
 
+    /**
+     * May be used by custom QueryHandler implementations
+     */
+    public Selection getSelection()
+    {
+        return selection;
+    }
+
+    /**
+     * May be used by custom QueryHandler implementations
+     */
+    public StatementRestrictions getRestrictions()
+    {
+        return restrictions;
+    }
+
     private List<ReadCommand> getSliceCommands(QueryOptions options, int limit, long now) throws RequestValidationException
     {
-        Collection<ByteBuffer> keys = getKeys(options);
-        if (keys.isEmpty()) // in case of IN () for (the last column of) the partition key.
-            return null;
+        Collection<ByteBuffer> keys = restrictions.getPartitionKeys(options);
 
         List<ReadCommand> commands = new ArrayList<>(keys.size());
 
@@ -445,72 +383,12 @@
         List<IndexExpression> expressions = getValidatedIndexExpressions(options);
         // The LIMIT provided by the user is the number of CQL row he wants returned.
         // We want to have getRangeSlice to count the number of columns, not the number of keys.
-        AbstractBounds<RowPosition> keyBounds = getKeyBounds(options);
+        AbstractBounds<RowPosition> keyBounds = restrictions.getPartitionKeyBounds(options);
         return keyBounds == null
              ? null
              : new RangeSliceCommand(keyspace(), columnFamily(), now,  filter, keyBounds, expressions, limit, !parameters.isDistinct, false);
     }
 
-    /**
-     * May be used by custom QueryHandler implementations
-     */
-    public AbstractBounds<RowPosition> getKeyBounds(QueryOptions options) throws InvalidRequestException
-    {
-        IPartitioner p = StorageService.getPartitioner();
-
-        if (onToken)
-        {
-            Token startToken = getTokenBound(Bound.START, options, p);
-            Token endToken = getTokenBound(Bound.END, options, p);
-
-            boolean includeStart = includeKeyBound(Bound.START);
-            boolean includeEnd = includeKeyBound(Bound.END);
-
-            /*
-             * If we ask SP.getRangeSlice() for (token(200), token(200)], it will happily return the whole ring.
-             * However, wrapping range doesn't really make sense for CQL, and we want to return an empty result
-             * in that case (CASSANDRA-5573). So special case to create a range that is guaranteed to be empty.
-             *
-             * In practice, we want to return an empty result set if either startToken > endToken, or both are
-             * equal but one of the bound is excluded (since [a, a] can contains something, but not (a, a], [a, a)
-             * or (a, a)). Note though that in the case where startToken or endToken is the minimum token, then
-             * this special case rule should not apply.
-             */
-            int cmp = startToken.compareTo(endToken);
-            if (!startToken.isMinimum() && !endToken.isMinimum() && (cmp > 0 || (cmp == 0 && (!includeStart || !includeEnd))))
-                return null;
-
-            RowPosition start = includeStart ? startToken.minKeyBound() : startToken.maxKeyBound();
-            RowPosition end = includeEnd ? endToken.maxKeyBound() : endToken.minKeyBound();
-
-            return new Range<RowPosition>(start, end);
-        }
-        else
-        {
-            ByteBuffer startKeyBytes = getKeyBound(Bound.START, options);
-            ByteBuffer finishKeyBytes = getKeyBound(Bound.END, options);
-
-            RowPosition startKey = RowPosition.ForKey.get(startKeyBytes, p);
-            RowPosition finishKey = RowPosition.ForKey.get(finishKeyBytes, p);
-
-            if (startKey.compareTo(finishKey) > 0 && !finishKey.isMinimum(p))
-                return null;
-
-            if (includeKeyBound(Bound.START))
-            {
-                return includeKeyBound(Bound.END)
-                     ? new Bounds<RowPosition>(startKey, finishKey)
-                     : new IncludingExcludingBounds<RowPosition>(startKey, finishKey);
-            }
-            else
-            {
-                return includeKeyBound(Bound.END)
-                     ? new Range<RowPosition>(startKey, finishKey)
-                     : new ExcludingBounds<RowPosition>(startKey, finishKey);
-            }
-        }
-    }
-
     private ColumnSlice makeStaticSlice()
     {
         // Note: we could use staticPrefix.start() for the start bound, but EMPTY gives us the
@@ -532,19 +410,19 @@
 
             // See the comments on IGNORE_TOMBSTONED_PARTITIONS and CASSANDRA-8490 for why we use a special value for
             // DISTINCT queries on the partition key only.
-            toGroup = selectsStaticColumns ? toGroup : SliceQueryFilter.IGNORE_TOMBSTONED_PARTITIONS;
+            toGroup = selection.containsStaticColumns() ? toGroup : SliceQueryFilter.IGNORE_TOMBSTONED_PARTITIONS;
             return new SliceQueryFilter(ColumnSlice.ALL_COLUMNS_ARRAY, false, 1, toGroup);
         }
-        else if (isColumnRange())
+        else if (restrictions.isColumnRange())
         {
-            List<Composite> startBounds = getRequestedBound(Bound.START, options);
-            List<Composite> endBounds = getRequestedBound(Bound.END, options);
+            List<Composite> startBounds = restrictions.getClusteringColumnsBoundsAsComposites(Bound.START, options);
+            List<Composite> endBounds = restrictions.getClusteringColumnsBoundsAsComposites(Bound.END, options);
             assert startBounds.size() == endBounds.size();
 
             // Handles fetching static columns. Note that for 2i, the filter is just used to restrict
             // the part of the index to query so adding the static slice would be useless and confusing.
             // For 2i, static columns are retrieve in CompositesSearcher with each index hit.
-            ColumnSlice staticSlice = selectsStaticColumns && !usesSecondaryIndexing
+            ColumnSlice staticSlice = selection.containsStaticColumns() && !restrictions.usesSecondaryIndexing()
                                     ? makeStaticSlice()
                                     : null;
 
@@ -642,280 +520,51 @@
      */
     public int getLimit(QueryOptions options) throws InvalidRequestException
     {
-        int l = Integer.MAX_VALUE;
         if (limit != null)
         {
-            ByteBuffer b = limit.bindAndGet(options);
-            if (b == null)
-                throw new InvalidRequestException("Invalid null value of limit");
-
+            ByteBuffer b = checkNotNull(limit.bindAndGet(options), "Invalid null value of limit");
+            // treat UNSET limit value as 'unlimited'
+            if (b == UNSET_BYTE_BUFFER)
+                return Integer.MAX_VALUE;
             try
             {
                 Int32Type.instance.validate(b);
-                l = Int32Type.instance.compose(b);
+                int l = Int32Type.instance.compose(b);
+                checkTrue(l > 0, "LIMIT must be strictly positive");
+                return l;
             }
             catch (MarshalException e)
             {
                 throw new InvalidRequestException("Invalid limit value");
             }
         }
-
-        if (l <= 0)
-            throw new InvalidRequestException("LIMIT must be strictly positive");
-
-        return l;
+        return Integer.MAX_VALUE;
     }
 
     private int updateLimitForQuery(int limit)
     {
+        // If the query is for an aggregate, we do not want to limit the number of rows retrieved. The LIMIT
+        // clause apply to the number of rows returned to the user and not to the number of rows retrieved.
+        if (selection.isAggregate())
+            return Integer.MAX_VALUE;
         // Internally, we don't support exclusive bounds for slices. Instead, we query one more element if necessary
         // and exclude it later (in processColumnFamily)
-        return sliceRestriction != null && (!sliceRestriction.isInclusive(Bound.START) || !sliceRestriction.isInclusive(Bound.END)) && limit != Integer.MAX_VALUE
+        return restrictions.isNonCompositeSliceWithExclusiveBounds() && limit != Integer.MAX_VALUE
              ? limit + 1
              : limit;
     }
 
-    private NavigableSet<ByteBuffer> getKeys(final QueryOptions options) throws InvalidRequestException
-    {
-        TreeSet<ByteBuffer> sortedKeys = new TreeSet<>(cfm.getKeyValidator());
-        CBuilder builder = cfm.getKeyValidatorAsCType().builder();
-        for (ColumnDefinition def : cfm.partitionKeyColumns())
-        {
-            Restriction r = keyRestrictions[def.position()];
-            assert r != null && !r.isSlice();
-
-            List<ByteBuffer> values = r.values(options);
-
-            if (builder.remainingCount() == 1)
-            {
-                for (ByteBuffer val : values)
-                {
-                    if (val == null)
-                        throw new InvalidRequestException(String.format("Invalid null value for partition key part %s", def.name));
-
-                    ByteBuffer keyBuffer = builder.buildWith(val).toByteBuffer();
-                    validateKey(keyBuffer);
-                    sortedKeys.add(keyBuffer);
-                }
-            }
-            else
-            {
-                // Note: for backward compatibility reasons, we let INs with 1 value slide
-                if (values.size() != 1)
-                    throw new InvalidRequestException("IN is only supported on the last column of the partition key");
-                ByteBuffer val = values.get(0);
-                if (val == null)
-                    throw new InvalidRequestException(String.format("Invalid null value for partition key part %s", def.name));
-                builder.add(val);
-            }
-        }
-        return sortedKeys;
-    }
-
-    private void validateKey(ByteBuffer keyBuffer) throws InvalidRequestException
-    {
-        if (keyBuffer == null || keyBuffer.remaining() == 0)
-            throw new InvalidRequestException("Key may not be empty");
-
-        try
-        {
-            cfm.getKeyValidator().validate(keyBuffer);
-        }
-        catch (MarshalException exc)
-        {
-            throw new InvalidRequestException("Partition key IN clause contained invalid value: " + exc);
-        }
-    }
-
-    private ByteBuffer getKeyBound(Bound b, QueryOptions options) throws InvalidRequestException
-    {
-        // Deal with unrestricted partition key components (special-casing is required to deal with 2i queries on the first
-        // component of a composite partition key).
-        for (int i = 0; i < keyRestrictions.length; i++)
-            if (keyRestrictions[i] == null)
-                return ByteBufferUtil.EMPTY_BYTE_BUFFER;
-
-        // We deal with IN queries for keys in other places, so we know buildBound will return only one result
-        return buildBound(b, cfm.partitionKeyColumns(), keyRestrictions, false, cfm.getKeyValidatorAsCType(), options).get(0).toByteBuffer();
-    }
-
-    private Token getTokenBound(Bound b, QueryOptions options, IPartitioner p) throws InvalidRequestException
-    {
-        assert onToken;
-
-        Restriction restriction = keyRestrictions[0];
-
-        assert !restriction.isMultiColumn() : "Unexpectedly got a multi-column restriction on a partition key for a range query";
-        SingleColumnRestriction keyRestriction = (SingleColumnRestriction)restriction;
-
-        ByteBuffer value;
-        if (keyRestriction.isEQ())
-        {
-            value = keyRestriction.values(options).get(0);
-        }
-        else
-        {
-            SingleColumnRestriction.Slice slice = (SingleColumnRestriction.Slice)keyRestriction;
-            if (!slice.hasBound(b))
-                return p.getMinimumToken();
-
-            value = slice.bound(b, options);
-        }
-
-        if (value == null)
-            throw new InvalidRequestException("Invalid null token value");
-        return p.getTokenFactory().fromByteArray(value);
-    }
-
-    private boolean includeKeyBound(Bound b)
-    {
-        for (Restriction r : keyRestrictions)
-        {
-            if (r == null)
-                return true;
-            else if (r.isSlice())
-            {
-                assert !r.isMultiColumn() : "Unexpectedly got multi-column restriction on partition key";
-                return ((SingleColumnRestriction.Slice)r).isInclusive(b);
-            }
-        }
-        // All equality
-        return true;
-    }
-
-    private boolean isColumnRange()
-    {
-        // Due to CASSANDRA-5762, we always do a slice for CQL3 tables (not dense, composite).
-        // Static CF (non dense but non composite) never entails a column slice however
-        if (!cfm.comparator.isDense())
-            return cfm.comparator.isCompound();
-
-        // Otherwise (i.e. for compact table where we don't have a row marker anyway and thus don't care about CASSANDRA-5762),
-        // it is a range query if it has at least one the column alias for which no relation is defined or is not EQ.
-        for (Restriction r : columnRestrictions)
-        {
-            if (r == null || r.isSlice())
-                return true;
-        }
-        return false;
-    }
-
     private SortedSet<CellName> getRequestedColumns(QueryOptions options) throws InvalidRequestException
     {
         // Note: getRequestedColumns don't handle static columns, but due to CASSANDRA-5762
         // we always do a slice for CQL3 tables, so it's ok to ignore them here
-        assert !isColumnRange();
-
-        CBuilder builder = cfm.comparator.prefixBuilder();
-
-        Iterator<ColumnDefinition> columnIter = cfm.clusteringColumns().iterator();
-        while (columnIter.hasNext())
-        {
-            ColumnDefinition def = columnIter.next();
-            Restriction r = columnRestrictions[def.position()];
-            assert r != null && !r.isSlice();
-
-            if (r.isEQ())
-            {
-                List<ByteBuffer> values = r.values(options);
-                if (r.isMultiColumn())
-                {
-                    for (int i = 0, m = values.size(); i < m; i++)
-                    {
-                        ByteBuffer val = values.get(i);
-
-                        if (i != 0)
-                            columnIter.next();
-
-                        if (val == null)
-                            throw new InvalidRequestException(String.format("Invalid null value for clustering key part %s", def.name));
-                        builder.add(val);
-                    }
-                }
-                else
-                {
-                    ByteBuffer val = r.values(options).get(0);
-                    if (val == null)
-                        throw new InvalidRequestException(String.format("Invalid null value for clustering key part %s", def.name));
-                    builder.add(val);
-                }
-            }
-            else
-            {
-                if (!r.isMultiColumn())
-                {
-                    List<ByteBuffer> values = r.values(options);
-                    // We have a IN, which we only support for the last column.
-                    // If compact, just add all values and we're done. Otherwise,
-                    // for each value of the IN, creates all the columns corresponding to the selection.
-                    if (values.isEmpty())
-                        return null;
-                    SortedSet<CellName> columns = new TreeSet<CellName>(cfm.comparator);
-                    Iterator<ByteBuffer> iter = values.iterator();
-                    while (iter.hasNext())
-                    {
-                        ByteBuffer val = iter.next();
-                        if (val == null)
-                            throw new InvalidRequestException(String.format("Invalid null value for clustering key part %s", def.name));
-
-                        if (ignoreInValue(val))
-                            continue;
-
-                        Composite prefix = builder.buildWith(val);
-                        columns.addAll(addSelectedColumns(prefix));
-                    }
-                    return columns;
-                }
-
-                // we have a multi-column IN restriction
-                List<List<ByteBuffer>> values = ((MultiColumnRestriction.IN) r).splitValues(options);
-                TreeSet<CellName> inValues = new TreeSet<>(cfm.comparator);
-                for (List<ByteBuffer> components : values)
-                {
-                    for (int i = 0; i < components.size(); i++)
-                        if (components.get(i) == null)
-                            throw new InvalidRequestException("Invalid null value in condition for column "
-                                    + cfm.clusteringColumns().get(i + def.position()).name);
-
-                    if (ignoreInValue(components))
-                        continue;
-
-                    Composite prefix = builder.buildWith(components);
-                    inValues.addAll(addSelectedColumns(prefix));
-                }
-                return inValues;
-            }
-        }
-
-        return addSelectedColumns(builder.build());
+        assert !restrictions.isColumnRange();
+        SortedSet<CellName> columns = new TreeSet<CellName>(cfm.comparator);
+        for (Composite composite : restrictions.getClusteringColumnsAsComposites(options))
+            columns.addAll(addSelectedColumns(composite));
+        return columns;
     }
 
-    /**
-     * Checks if we should ignore the specified IN value for a clustering column as it will not return any result.
-     *
-     * @param val the IN value to check
-     * @return {@code true} if we should ignore the value, {@code false} otherwise.
-     */
-    private boolean ignoreInValue(ByteBuffer val)
-    {
-        // Dense non-compound tables do not accept empty ByteBuffers. By consequence, we know that we can
-        // ignore any IN value which is an empty byte buffer an which otherwise will trigger an error.
-        return !cfm.comparator.isCompound() && !val.hasRemaining();
-    }
-
-   /**
-    * Checks if we should ignore the specified IN components for a clustering column as it will not return any result.
-    *
-    * @param components the IN components to check
-    * @return {@code true} if we should ignore the value, {@code false} otherwise.
-    */
-   private boolean ignoreInValue(List<ByteBuffer> components)
-   {
-       // Dense non-compound tables do not accept empty ByteBuffers. By consequence, we know that we can
-       // ignore any IN value which is an empty byte buffer an which otherwise will trigger an error.
-       return !cfm.comparator.isCompound() && !components.get(0).hasRemaining();
-   }
-
     private SortedSet<CellName> addSelectedColumns(Composite prefix)
     {
         if (cfm.comparator.isDense())
@@ -924,10 +573,6 @@
         }
         else
         {
-            // Collections require doing a slice query because a given collection is a
-            // non-know set of columns, so we shouldn't get there
-            assert !selectACollection();
-
             SortedSet<CellName> columns = new TreeSet<CellName>(cfm.comparator);
 
             // We need to query the selected column as well as the marker
@@ -941,7 +586,7 @@
 
                 // selected columns
                 for (ColumnDefinition def : selection.getColumns())
-                    if (def.kind == ColumnDefinition.Kind.REGULAR || def.kind == ColumnDefinition.Kind.STATIC)
+                    if (def.isRegular() || def.isStatic())
                         columns.add(cfm.comparator.create(prefix, def));
             }
             else
@@ -954,378 +599,77 @@
         }
     }
 
-    /** Returns true if a non-frozen collection is selected, false otherwise. */
-    private boolean selectACollection()
-    {
-        if (!cfm.comparator.hasCollections())
-            return false;
-
-        for (ColumnDefinition def : selection.getColumns())
-        {
-            if (def.type.isCollection() && def.type.isMultiCell())
-                return true;
-        }
-
-        return false;
-    }
-
-    @VisibleForTesting
-    static List<Composite> buildBound(Bound bound,
-                                      List<ColumnDefinition> defs,
-                                      Restriction[] restrictions,
-                                      boolean isReversed,
-                                      CType type,
-                                      QueryOptions options) throws InvalidRequestException
-    {
-        CBuilder builder = type.builder();
-
-        // The end-of-component of composite doesn't depend on whether the
-        // component type is reversed or not (i.e. the ReversedType is applied
-        // to the component comparator but not to the end-of-component itself),
-        // it only depends on whether the slice is reversed
-        Bound eocBound = isReversed ? Bound.reverse(bound) : bound;
-        for (int i = 0, m = defs.size(); i < m; i++)
-        {
-            ColumnDefinition def = defs.get(i);
-
-            // In a restriction, we always have Bound.START < Bound.END for the "base" comparator.
-            // So if we're doing a reverse slice, we must inverse the bounds when giving them as start and end of the slice filter.
-            // But if the actual comparator itself is reversed, we must inversed the bounds too.
-            Bound b = isReversed == isReversedType(def) ? bound : Bound.reverse(bound);
-            Restriction r = restrictions[def.position()];
-            if (isNullRestriction(r, b) || !r.canEvaluateWithSlices())
-            {
-                // There wasn't any non EQ relation on that key, we select all records having the preceding component as prefix.
-                // For composites, if there was preceding component and we're computing the end, we must change the last component
-                // End-Of-Component, otherwise we would be selecting only one record.
-                Composite prefix = builder.build();
-                return Collections.singletonList(eocBound == Bound.END ? prefix.end() : prefix.start());
-            }
-            if (r.isSlice())
-            {
-                if (r.isMultiColumn())
-                {
-                    MultiColumnRestriction.Slice slice = (MultiColumnRestriction.Slice) r;
-
-                    if (!slice.hasBound(b))
-                    {
-                        Composite prefix = builder.build();
-                        return Collections.singletonList(builder.remainingCount() > 0 && eocBound == Bound.END
-                                ? prefix.end()
-                                : prefix);
-                    }
-
-                    List<ByteBuffer> vals = slice.componentBounds(b, options);
-
-                    for (int j = 0, n = vals.size(); j < n; j++)
-                        addValue(builder, defs.get(i + j), vals.get(j)) ;
-                }
-                else
-                {
-                    builder.add(getSliceValue(r, b, options));
-                }
-                Operator relType = ((Restriction.Slice)r).getRelation(eocBound, b);
-                return Collections.singletonList(builder.build().withEOC(eocForRelation(relType)));
-            }
-
-            if (r.isIN())
-            {
-                // The IN query might not have listed the values in comparator order, so we need to re-sort
-                // the bounds lists to make sure the slices works correctly (also, to avoid duplicates).
-                TreeSet<Composite> inValues = new TreeSet<>(isReversed ? type.reverseComparator() : type);
-
-                if (r.isMultiColumn())
-                {
-                    List<List<ByteBuffer>> splitInValues = ((MultiColumnRestriction.IN) r).splitValues(options);
-
-                    for (List<ByteBuffer> components : splitInValues)
-                    {
-                        for (int j = 0; j < components.size(); j++)
-                            if (components.get(j) == null)
-                                throw new InvalidRequestException("Invalid null value in condition for column " + defs.get(i + j).name);
-
-                        Composite prefix = builder.buildWith(components);
-                        inValues.add(builder.remainingCount() == 0 ? prefix : addEOC(prefix, eocBound));
-                    }
-                    return new ArrayList<>(inValues);
-                }
-
-                List<ByteBuffer> values = r.values(options);
-                if (values.size() != 1)
-                {
-                    // IN query, we only support it on the clustering columns
-                    assert def.position() == defs.size() - 1;
-                    for (ByteBuffer val : values)
-                    {
-                        if (val == null)
-                            throw new InvalidRequestException(String.format("Invalid null value in condition for column %s",
-                                                                            def.name));
-                        Composite prefix = builder.buildWith(val);
-                        // See below for why this
-                        inValues.add(builder.remainingCount() == 0 ? prefix : addEOC(prefix, eocBound));
-                    }
-                    return new ArrayList<>(inValues);
-                }
-            }
-
-            List<ByteBuffer> values = r.values(options);
-
-            if (r.isMultiColumn())
-            {
-                for (int j = 0; j < values.size(); j++)
-                    addValue(builder, defs.get(i + j), values.get(j));
-                i += values.size() - 1; // skips the processed columns
-            }
-            else
-            {
-                addValue(builder, def, values.get(0));
-            }
-        }
-        // Means no relation at all or everything was an equal
-        // Note: if the builder is "full", there is no need to use the end-of-component bit. For columns selection,
-        // it would be harmless to do it. However, we use this method got the partition key too. And when a query
-        // with 2ndary index is done, and with the the partition provided with an EQ, we'll end up here, and in that
-        // case using the eoc would be bad, since for the random partitioner we have no guarantee that
-        // prefix.end() will sort after prefix (see #5240).
-        Composite prefix = builder.build();
-        return Collections.singletonList(builder.remainingCount() == 0 ? prefix : addEOC(prefix, eocBound));
-    }
-
-    /**
-     * Adds an EOC to the specified Composite.
-     *
-     * @param composite the composite
-     * @param eocBound the EOC bound
-     * @return a new <code>Composite</code> with the EOC corresponding to the eocBound
-     */
-    private static Composite addEOC(Composite composite, Bound eocBound)
-    {
-        return eocBound == Bound.END ? composite.end() : composite.start();
-    }
-
-    /**
-     * Adds the specified value to the specified builder
-     *
-     * @param builder the CBuilder to which the value must be added
-     * @param def the column associated to the value
-     * @param value the value to add
-     * @throws InvalidRequestException if the value is null
-     */
-    private static void addValue(CBuilder builder, ColumnDefinition def, ByteBuffer value) throws InvalidRequestException
-    {
-        if (value == null)
-            throw new InvalidRequestException(String.format("Invalid null value in condition for column %s", def.name));
-        builder.add(value);
-    }
-
-    private static Composite.EOC eocForRelation(Operator op)
-    {
-        switch (op)
-        {
-            case LT:
-                // < X => using startOf(X) as finish bound
-                return Composite.EOC.START;
-            case GT:
-            case LTE:
-                // > X => using endOf(X) as start bound
-                // <= X => using endOf(X) as finish bound
-                return Composite.EOC.END;
-            default:
-                // >= X => using X as start bound (could use START_OF too)
-                // = X => using X
-                return Composite.EOC.NONE;
-        }
-    }
-
-    private static boolean isNullRestriction(Restriction r, Bound b)
-    {
-        return r == null || (r.isSlice() && !((Restriction.Slice)r).hasBound(b));
-    }
-
-    private static ByteBuffer getSliceValue(Restriction r, Bound b, QueryOptions options) throws InvalidRequestException
-    {
-        Restriction.Slice slice = (Restriction.Slice)r;
-        assert slice.hasBound(b);
-        ByteBuffer val = slice.bound(b, options);
-        if (val == null)
-            throw new InvalidRequestException(String.format("Invalid null clustering key part %s", r));
-        return val;
-    }
-
-    private List<Composite> getRequestedBound(Bound b, QueryOptions options) throws InvalidRequestException
-    {
-        assert isColumnRange();
-        List<Composite> bound = buildBound(b, cfm.clusteringColumns(), columnRestrictions, isReversed, cfm.comparator, options);
-        for (Composite c : bound) {
-            if (!c.isEmpty())
-                QueryProcessor.validateComposite(c, cfm.comparator);
-        }
-        return bound;
-    }
-
     /**
      * May be used by custom QueryHandler implementations
      */
     public List<IndexExpression> getValidatedIndexExpressions(QueryOptions options) throws InvalidRequestException
     {
-        if (!usesSecondaryIndexing || restrictedColumns.isEmpty())
+        if (!restrictions.usesSecondaryIndexing())
             return Collections.emptyList();
 
-        List<IndexExpression> expressions = new ArrayList<IndexExpression>();
-        for (ColumnDefinition def : restrictedColumns.keySet())
-        {
-            Restriction restriction;
-            switch (def.kind)
-            {
-                case PARTITION_KEY:
-                    restriction = keyRestrictions[def.position()];
-                    break;
-                case CLUSTERING_COLUMN:
-                    restriction = columnRestrictions[def.position()];
-                    break;
-                case REGULAR:
-                case STATIC:
-                    restriction = metadataRestrictions.get(def.name);
-                    break;
-                default:
-                    // We don't allow restricting a COMPACT_VALUE for now in prepare.
-                    throw new AssertionError();
-            }
+        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(columnFamily());
+        SecondaryIndexManager secondaryIndexManager = cfs.indexManager;
 
-            if (restriction.isSlice())
-            {
-                Restriction.Slice slice = (Restriction.Slice)restriction;
-                for (Bound b : Bound.values())
-                {
-                    if (slice.hasBound(b))
-                    {
-                        ByteBuffer value = validateIndexedValue(def, slice.bound(b, options));
-                        Operator op = slice.getIndexOperator(b);
-                        // If the underlying comparator for name is reversed, we need to reverse the IndexOperator: user operation
-                        // always refer to the "forward" sorting even if the clustering order is reversed, but the 2ndary code does
-                        // use the underlying comparator as is.
-                        if (def.type instanceof ReversedType)
-                            op = reverse(op);
-                        expressions.add(new IndexExpression(def.name.bytes, op, value));
-                    }
-                }
-            }
-            else if (restriction.isContains())
-            {
-                SingleColumnRestriction.Contains contains = (SingleColumnRestriction.Contains)restriction;
-                for (ByteBuffer value : contains.values(options))
-                {
-                    validateIndexedValue(def, value);
-                    expressions.add(new IndexExpression(def.name.bytes, Operator.CONTAINS, value));
-                }
-                for (ByteBuffer key : contains.keys(options))
-                {
-                    validateIndexedValue(def, key);
-                    expressions.add(new IndexExpression(def.name.bytes, Operator.CONTAINS_KEY, key));
-                }
-            }
-            else
-            {
-                ByteBuffer value;
-                if (restriction.isMultiColumn())
-                {
-                    List<ByteBuffer> values = restriction.values(options);
-                    value = values.get(def.position());
-                }
-                else
-                {
-                    List<ByteBuffer> values = restriction.values(options);
-                    if (values.size() != 1)
-                        throw new InvalidRequestException("IN restrictions are not supported on indexed columns");
+        List<IndexExpression> expressions = restrictions.getIndexExpressions(secondaryIndexManager, options);
+        secondaryIndexManager.validateIndexSearchersForQuery(expressions);
 
-                    value = values.get(0);
-                }
-
-                validateIndexedValue(def, value);
-                expressions.add(new IndexExpression(def.name.bytes, Operator.EQ, value));
-            }
-        }
-
-        if (usesSecondaryIndexing)
-        {
-            ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(columnFamily());
-            SecondaryIndexManager secondaryIndexManager = cfs.indexManager;
-            secondaryIndexManager.validateIndexSearchersForQuery(expressions);
-        }
-        
         return expressions;
     }
 
-    private static ByteBuffer validateIndexedValue(ColumnDefinition def, ByteBuffer value) throws InvalidRequestException
-    {
-        if (value == null)
-            throw new InvalidRequestException(String.format("Unsupported null value for indexed column %s", def.name));
-        if (value.remaining() > 0xFFFF)
-            throw new InvalidRequestException("Index expression values may not be larger than 64K");
-        return value;
-    }
-
     private CellName makeExclusiveSliceBound(Bound bound, CellNameType type, QueryOptions options) throws InvalidRequestException
     {
-        if (sliceRestriction.isInclusive(bound))
+        // clusteringColumnBounds may reverse bound if clustering order is reversed
+        // but areRequestedBoundsInclusive checks for Restriction::isInclusive and never
+        // reverses the order. In order to avoid inconsistencies and check inclusive
+        // bounds correctly, we need to check for column order and reverse it. See CASSANDRA-10988
+        if (restrictions.areRequestedBoundsInclusive(reverseBoundIfNeeded(bound)))
             return null;
 
-        // We can only reach that point if cfm.comparator.isCompound() = false and the table has some clustering columns.
+        // We can only reach that if the table is dense non-compound tables.
         // By consequence, we know that the table is a COMPACT table with only one clustering column.
-        ByteBuffer value = sliceRestriction.isMultiColumn() ? ((MultiColumnRestriction.Slice) sliceRestriction).componentBounds(bound, options).get(0)
-                                                            : sliceRestriction.bound(bound, options);
+        ByteBuffer value = restrictions.getClusteringColumnsBounds(bound, options).get(0);
 
         // Dense non-compound tables do not accept empty ByteBuffers. By consequence, if the slice value is empty
         // we know that we can treat the slice as inclusive.
         return value.hasRemaining() ? type.makeCellName(value) : null;
     }
 
+    /**
+     * Reverses the specified bound if the non-compound clustering column is a reversed one.
+     * @param bound bound to reverse
+     * @return the bound reversed if the column type was a reversed one or the original bound
+     */
+    private Bound reverseBoundIfNeeded(Bound bound)
+    {
+        assert !cfm.comparator.isCompound();
+
+        List<ColumnDefinition> columnDefs = cfm.clusteringColumns();
+        return columnDefs.get(columnDefs.size() - 1).isReversedType() ? bound.reverse() : bound;
+    }
 
     private Iterator<Cell> applySliceRestriction(final Iterator<Cell> cells, final QueryOptions options) throws InvalidRequestException
     {
-        assert sliceRestriction != null;
-
         final CellNameType type = cfm.comparator;
+
         final CellName excludedStart = makeExclusiveSliceBound(Bound.START, type, options);
         final CellName excludedEnd = makeExclusiveSliceBound(Bound.END, type, options);
 
-        return new AbstractIterator<Cell>()
+        return Iterators.filter(cells, new Predicate<Cell>()
         {
-            protected Cell computeNext()
+            public boolean apply(Cell c)
             {
-                while (cells.hasNext())
-                {
-                    Cell c = cells.next();
-
-                    // For dynamic CF, the column could be out of the requested bounds (because we don't support strict bounds internally (unless
-                    // the comparator is composite that is)), filter here
-                    if ( (excludedStart != null && type.compare(c.name(), excludedStart) == 0)
-                      || (excludedEnd != null && type.compare(c.name(), excludedEnd) == 0) )
-                        continue;
-
-                    return c;
-                }
-                return endOfData();
+                // For dynamic CF, the column could be out of the requested bounds (because we don't support strict bounds internally (unless
+                // the comparator is composite that is)), filter here
+                return !((excludedStart != null && type.compare(c.name(), excludedStart) == 0)
+                            || (excludedEnd != null && type.compare(c.name(), excludedEnd) == 0));
             }
-        };
-    }
-
-    private static Operator reverse(Operator op)
-    {
-        switch (op)
-        {
-            case LT:  return Operator.GT;
-            case LTE: return Operator.GTE;
-            case GT:  return Operator.LT;
-            case GTE: return Operator.LTE;
-            default: return op;
-        }
+        });
     }
 
     private ResultSet process(List<Row> rows, QueryOptions options, int limit, long now) throws InvalidRequestException
     {
-        Selection.ResultSetBuilder result = selection.resultSetBuilder(now);
-        for (org.apache.cassandra.db.Row row : rows)
+        Selection.ResultSetBuilder result = selection.resultSetBuilder(now, parameters.isJson);
+        for (Row row : rows)
         {
             // Not columns match the query, skip
             if (row.cf == null)
@@ -1334,7 +678,7 @@
             processColumnFamily(row.key.getKey(), row.cf, options, now, result);
         }
 
-        ResultSet cqlRows = result.build();
+        ResultSet cqlRows = result.build(options.getProtocolVersion());
 
         orderResults(cqlRows);
 
@@ -1363,18 +707,19 @@
         }
 
         Iterator<Cell> cells = cf.getSortedColumns().iterator();
-        if (sliceRestriction != null)
+        if (restrictions.isNonCompositeSliceWithExclusiveBounds())
             cells = applySliceRestriction(cells, options);
 
+        int protocolVersion = options.getProtocolVersion();
         CQL3Row.RowIterator iter = cfm.comparator.CQL3RowBuilder(cfm, now).group(cells);
 
         // If there is static columns but there is no non-static row,
         // and the select was a full partition selection (i.e. there was no condition on clustering or regular columns),
         // we want to include the static columns in the result set (and we're done).
         CQL3Row staticRow = iter.getStaticRow();
-        if (staticRow != null && !iter.hasNext() && hasNoClusteringColumnsRestriction() && hasNoRegularColumnsRestriction())
+        if (staticRow != null && !iter.hasNext() && !restrictions.hasClusteringColumnsRestriction() && !restrictions.hasRegularColumnsRestriction())
         {
-            result.newRow();
+            result.newRow(protocolVersion);
             for (ColumnDefinition def : selection.getColumns())
             {
                 switch (def.kind)
@@ -1397,7 +742,7 @@
             CQL3Row cql3Row = iter.next();
 
             // Respect requested order
-            result.newRow();
+            result.newRow(protocolVersion);
             // Respect selection order
             for (ColumnDefinition def : selection.getColumns())
             {
@@ -1444,138 +789,25 @@
         result.add(row.getColumn(def.name));
     }
 
-    private boolean hasNoClusteringColumnsRestriction()
-    {
-        for (int i = 0; i < columnRestrictions.length; i++)
-            if (columnRestrictions[i] != null)
-                return false;
-        return true;
-    }
-
-    private boolean hasNoRegularColumnsRestriction()
-    {
-        for (ColumnDefinition def : restrictedColumns.keySet())
-        {
-            if (def.kind == ColumnDefinition.Kind.REGULAR)
-                return false;
-        }
-        return true;
-    }
-
     private boolean needsPostQueryOrdering()
     {
         // We need post-query ordering only for queries with IN on the partition key and an ORDER BY.
-        return keyIsInRelation && !parameters.orderings.isEmpty();
+        return restrictions.keyIsInRelation() && !parameters.orderings.isEmpty();
     }
 
     /**
      * Orders results when multiple keys are selected (using IN)
      */
-    private void orderResults(ResultSet cqlRows) throws InvalidRequestException
+    private void orderResults(ResultSet cqlRows)
     {
         if (cqlRows.size() == 0 || !needsPostQueryOrdering())
             return;
 
-        assert orderingIndexes != null;
-
-        List<Integer> idToSort = new ArrayList<Integer>();
-        List<Comparator<ByteBuffer>> sorters = new ArrayList<Comparator<ByteBuffer>>();
-
-        for (ColumnIdentifier.Raw identifier : parameters.orderings.keySet())
-        {
-            ColumnDefinition orderingColumn = cfm.getColumnDefinition(identifier.prepare(cfm));
-            idToSort.add(orderingIndexes.get(orderingColumn.name));
-            sorters.add(orderingColumn.type);
-        }
-
-        Comparator<List<ByteBuffer>> comparator = idToSort.size() == 1
-                                                ? new SingleColumnComparator(idToSort.get(0), sorters.get(0))
-                                                : new CompositeComparator(sorters, idToSort);
-        Collections.sort(cqlRows.rows, comparator);
-    }
-
-    private static boolean isReversedType(ColumnDefinition def)
-    {
-        return def.type instanceof ReversedType;
-    }
-
-    private boolean columnFilterIsIdentity()
-    {
-        for (Restriction r : columnRestrictions)
-        {
-            if (r != null)
-                return false;
-        }
-        return true;
-    }
-
-    /**
-     * May be used by custom QueryHandler implementations
-     */
-    public boolean hasClusteringColumnsRestriction()
-    {
-        for (int i = 0; i < columnRestrictions.length; i++)
-            if (columnRestrictions[i] != null)
-                return true;
-        return false;
-    }
-
-    /**
-     * May be used by custom QueryHandler implementations
-     */
-    public boolean hasPartitionKeyRestriction()
-    {
-        for (int i = 0; i < keyRestrictions.length; i++)
-            if (keyRestrictions[i] != null)
-                return true;
-        return false;
-    }
-
-    private void validateDistinctSelection()
-    throws InvalidRequestException
-    {
-        Collection<ColumnDefinition> requestedColumns = selection.getColumns();
-        for (ColumnDefinition def : requestedColumns)
-            if (def.kind != ColumnDefinition.Kind.PARTITION_KEY && def.kind != ColumnDefinition.Kind.STATIC)
-                throw new InvalidRequestException(String.format("SELECT DISTINCT queries must only request partition key columns and/or static columns (not %s)", def.name));
-
-        // If it's a key range, we require that all partition key columns are selected so we don't have to bother with post-query grouping.
-        if (!isKeyRange)
-            return;
-
-        for (ColumnDefinition def : cfm.partitionKeyColumns())
-            if (!requestedColumns.contains(def))
-                throw new InvalidRequestException(String.format("SELECT DISTINCT queries must request all the partition key columns (missing %s)", def.name));
-    }
-
-    /**
-     * Checks if the specified column is restricted by multiple contains or contains key.
-     *
-     * @param columnDef the definition of the column to check
-     * @return <code>true</code> the specified column is restricted by multiple contains or contains key,
-     * <code>false</code> otherwise
-     */
-    private boolean isRestrictedByMultipleContains(ColumnDefinition columnDef)
-    {
-        if (!columnDef.type.isCollection())
-            return false;
-
-        Restriction restriction = metadataRestrictions.get(columnDef.name);
-
-        if (!(restriction instanceof Contains))
-            return false;
-
-        Contains contains = (Contains) restriction;
-        return (contains.numberOfValues() + contains.numberOfKeys()) > 1;
+        Collections.sort(cqlRows.rows, orderingComparator);
     }
 
     public static class RawStatement extends CFStatement
     {
-        /**
-         * Checks to ensure that the warning for missing allow filtering is only logged once.
-         */
-        private static volatile boolean hasLoggedMissingAllowFilteringWarning = false;
-
         private final Parameters parameters;
         private final List<RawSelector> selectClause;
         private final List<Relation> whereClause;
@@ -1595,147 +827,70 @@
             CFMetaData cfm = ThriftValidation.validateColumnFamily(keyspace(), columnFamily());
             VariableSpecifications boundNames = getBoundVariables();
 
-            // Select clause
-            if (parameters.isCount && !selectClause.isEmpty())
-                throw new InvalidRequestException("Only COUNT(*) and COUNT(1) operations are currently supported.");
-
             Selection selection = selectClause.isEmpty()
-                                ? Selection.wildcard(cfm, parameters.isCount, parameters.countAlias)
-                                : Selection.fromSelectors(cfm, selectClause);
+                                  ? Selection.wildcard(cfm)
+                                  : Selection.fromSelectors(cfm, selectClause);
 
-            SelectStatement stmt = new SelectStatement(cfm, boundNames.size(), parameters, selection, prepareLimit(boundNames));
-
-            /*
-             * WHERE clause. For a given entity, rules are:
-             *   - EQ relation conflicts with anything else (including a 2nd EQ)
-             *   - Can't have more than one LT(E) relation (resp. GT(E) relation)
-             *   - IN relation are restricted to row keys (for now) and conflicts with anything else
-             *     (we could allow two IN for the same entity but that doesn't seem very useful)
-             *   - The value_alias cannot be restricted in any way (we don't support wide rows with indexed value in CQL so far)
-             */
-            boolean hasQueriableIndex = false;
-            boolean hasQueriableClusteringColumnIndex = false;
-
-            ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(columnFamily());
-            SecondaryIndexManager indexManager = cfs.indexManager;
-
-            for (Relation relation : whereClause)
-            {
-                if (relation.isMultiColumn())
-                {
-                    MultiColumnRelation rel = (MultiColumnRelation) relation;
-                    List<ColumnDefinition> names = new ArrayList<>(rel.getEntities().size());
-                    for (ColumnIdentifier.Raw rawEntity : rel.getEntities())
-                    {
-                        ColumnIdentifier entity = rawEntity.prepare(cfm);
-                        ColumnDefinition def = cfm.getColumnDefinition(entity);
-                        boolean[] queriable = processRelationEntity(stmt, indexManager, relation, entity, def);
-                        hasQueriableIndex |= queriable[0];
-                        hasQueriableClusteringColumnIndex |= queriable[1];
-                        names.add(def);
-                    }
-                    updateRestrictionsForRelation(stmt, names, rel, boundNames);
-                }
-                else
-                {
-                    SingleColumnRelation rel = (SingleColumnRelation) relation;
-                    ColumnIdentifier entity = rel.getEntity().prepare(cfm);
-                    ColumnDefinition def = cfm.getColumnDefinition(entity);
-                    boolean[] queriable = processRelationEntity(stmt, indexManager, relation, entity, def);
-                    hasQueriableIndex |= queriable[0];
-                    hasQueriableClusteringColumnIndex |= queriable[1];
-                    updateRestrictionsForRelation(stmt, def, rel, boundNames);
-                }
-            }
-
-             // At this point, the select statement if fully constructed, but we still have a few things to validate
-            processPartitionKeyRestrictions(stmt, hasQueriableIndex, cfm);
-
-            // All (or none) of the partition key columns have been specified;
-            // hence there is no need to turn these restrictions into index expressions.
-            if (!stmt.usesSecondaryIndexing)
-                stmt.restrictedColumns.keySet().removeAll(cfm.partitionKeyColumns());
-
-            if (stmt.selectsOnlyStaticColumns && stmt.hasClusteringColumnsRestriction())
-                throw new InvalidRequestException("Cannot restrict clustering columns when selecting only static columns");
-
-            processColumnRestrictions(stmt, hasQueriableIndex, cfm);
-
-            // Covers indexes on the first clustering column (among others).
-            if (stmt.isKeyRange && hasQueriableClusteringColumnIndex)
-                stmt.usesSecondaryIndexing = true;
-
-            int numberOfRestrictionsEvaluatedWithSlices = 0;
-
-            Restriction lastSliceRestriction = null;
-            for (ColumnDefinition def : cfm.clusteringColumns())
-            {
-                // Remove clustering column restrictions that can be handled by slices; the remainder will be
-                // handled by filters (which may require a secondary index).
-                Boolean indexed = stmt.restrictedColumns.get(def);
-                if (indexed == null)
-                    break;
-
-                Restriction restriction = stmt.columnRestrictions[def.position()];
-                if (lastSliceRestriction != null && !restriction.equals(lastSliceRestriction))
-                    break;
-
-                if (!(indexed && stmt.usesSecondaryIndexing) && restriction.canEvaluateWithSlices())
-                {
-                    stmt.restrictedColumns.remove(def);
-                    numberOfRestrictionsEvaluatedWithSlices++;
-                    if (restriction.isSlice())
-                        lastSliceRestriction = restriction;
-                }
-            }
-
-            // Even if usesSecondaryIndexing is false at this point, we'll still have to use one if
-            // there are restrictions not covered by the PK.
-            if (!stmt.metadataRestrictions.isEmpty())
-                stmt.usesSecondaryIndexing = true;
-
-            if (stmt.usesSecondaryIndexing)
-                validateSecondaryIndexSelections(stmt);
-
-            if (!stmt.parameters.orderings.isEmpty())
-                processOrderingClause(stmt, cfm);
-
-            checkNeedsFiltering(stmt, numberOfRestrictionsEvaluatedWithSlices);
+            StatementRestrictions restrictions = prepareRestrictions(cfm, boundNames, selection);
 
             if (parameters.isDistinct)
-                stmt.validateDistinctSelection();
+                validateDistinctSelection(cfm, selection, restrictions);
 
-            return new ParsedStatement.Prepared(stmt, boundNames);
-        }
+            Comparator<List<ByteBuffer>> orderingComparator = null;
+            boolean isReversed = false;
 
-        /** Returns a pair of (hasQueriableIndex, hasQueriableClusteringColumnIndex) */
-        private boolean[] processRelationEntity(SelectStatement stmt,
-                                                SecondaryIndexManager indexManager,
-                                                Relation relation,
-                                                ColumnIdentifier entity,
-                                                ColumnDefinition def) throws InvalidRequestException
-        {
-            if (def == null)
-                handleUnrecognizedEntity(entity, relation);
-
-            SecondaryIndex index = indexManager.getIndexForColumn(def.name.bytes);
-            if (index != null && index.supportsOperator(relation.operator()))
+            if (!parameters.orderings.isEmpty())
             {
-                stmt.restrictedColumns.put(def, Boolean.TRUE);
-                return new boolean[]{true, def.kind == ColumnDefinition.Kind.CLUSTERING_COLUMN};
+                verifyOrderingIsAllowed(restrictions);
+                orderingComparator = getOrderingComparator(cfm, selection, restrictions);
+                isReversed = isReversed(cfm);
             }
 
-            stmt.restrictedColumns.put(def, Boolean.FALSE);
-            return new boolean[]{false, false};
+            if (isReversed)
+                restrictions.reverse();
+
+            checkNeedsFiltering(restrictions);
+
+            SelectStatement stmt = new SelectStatement(cfm,
+                                                        boundNames.size(),
+                                                        parameters,
+                                                        selection,
+                                                        restrictions,
+                                                        isReversed,
+                                                        orderingComparator,
+                                                        prepareLimit(boundNames));
+
+            return new ParsedStatement.Prepared(stmt, boundNames, boundNames.getPartitionKeyBindIndexes(cfm));
         }
 
-        /** Throws an InvalidRequestException for an unrecognized identifier in the WHERE clause */
-        private void handleUnrecognizedEntity(ColumnIdentifier entity, Relation relation) throws InvalidRequestException
+        /**
+         * Prepares the restrictions.
+         *
+         * @param cfm the column family meta data
+         * @param boundNames the variable specifications
+         * @param selection the selection
+         * @return the restrictions
+         * @throws InvalidRequestException if a problem occurs while building the restrictions
+         */
+        private StatementRestrictions prepareRestrictions(CFMetaData cfm,
+                                                          VariableSpecifications boundNames,
+                                                          Selection selection) throws InvalidRequestException
         {
-            if (containsAlias(entity))
-                throw new InvalidRequestException(String.format("Aliases aren't allowed in the where clause ('%s')", relation));
-            else
-                throw new InvalidRequestException(String.format("Undefined name %s in where clause ('%s')", entity, relation));
+            try
+            {
+                return new StatementRestrictions(cfm,
+                                                 whereClause,
+                                                 boundNames,
+                                                 selection.containsOnlyStaticColumns(),
+                                                 selection.containsACollection(),
+                                                 parameters.allowFiltering);
+            }
+            catch (UnrecognizedEntityException e)
+            {
+                if (containsAlias(e.entity))
+                    throw invalidRequest("Aliases aren't allowed in the where clause ('%s')", e.relation);
+                throw e;
+            }
         }
 
         /** Returns a Term for the limit or null if no limit is set */
@@ -1749,580 +904,92 @@
             return prepLimit;
         }
 
-        private void updateRestrictionsForRelation(SelectStatement stmt, List<ColumnDefinition> defs, MultiColumnRelation relation, VariableSpecifications boundNames) throws InvalidRequestException
+        private static void verifyOrderingIsAllowed(StatementRestrictions restrictions) throws InvalidRequestException
         {
-            List<ColumnDefinition> restrictedColumns = new ArrayList<>();
-            Set<ColumnDefinition> seen = new HashSet<>();
-            Restriction existing = null;
-
-            int previousPosition = defs.get(0).position() - 1;
-            for (int i = 0, m = defs.size(); i < m; i++)
-            {
-                ColumnDefinition def = defs.get(i);
-
-                // ensure multi-column restriction only applies to clustering columns
-                if (def.kind != ColumnDefinition.Kind.CLUSTERING_COLUMN)
-                    throw new InvalidRequestException(String.format("Multi-column relations can only be applied to clustering columns: %s", def.name));
-
-                if (seen.contains(def))
-                    throw new InvalidRequestException(String.format("Column \"%s\" appeared twice in a relation: %s", def.name, relation));
-                seen.add(def);
-
-                // check that no clustering columns were skipped
-                if (def.position() != previousPosition + 1)
-                {
-                    if (previousPosition == -1)
-                        throw new InvalidRequestException(String.format(
-                                "Clustering columns may not be skipped in multi-column relations. " +
-                                "They should appear in the PRIMARY KEY order. Got %s", relation));
-
-                    throw new InvalidRequestException(String.format(
-                                "Clustering columns must appear in the PRIMARY KEY order in multi-column relations: %s",
-                                 relation));
-                }
-                previousPosition++;
-
-                Restriction previous = existing;
-                existing = getExistingRestriction(stmt, def);
-                Operator operator = relation.operator();
-                if (existing != null)
-                {
-                    if (operator == Operator.EQ || operator == Operator.IN)
-                    {
-                        throw new InvalidRequestException(String.format(
-                                "Column \"%s\" cannot be restricted by more than one relation if it is in an %s relation",
-                                def.name, operator));
-                    }
-                    else if (!existing.isSlice())
-                    {
-                        throw new InvalidRequestException(String.format(
-                                "Column \"%s\" cannot be restricted by an equality relation and an inequality relation",
-                                def.name));
-                    }
-                    else
-                    {
-                        boolean existingRestrictionStartBefore =
-                            (i == 0 && def.position() != 0 && stmt.columnRestrictions[def.position() - 1] == existing);
-
-                        boolean existingRestrictionStartAfter = (i != 0 && previous != existing);
-
-                        if (existingRestrictionStartBefore || existingRestrictionStartAfter)
-                        {
-                            throw new InvalidRequestException(String.format(
-                                    "Column \"%s\" cannot be restricted by two inequalities not starting with the same column: %s",
-                                    def.name, relation));
-                        }
-
-                        checkBound(existing, def, operator);
-                    }
-                }
-                restrictedColumns.add(def);
-            }
-
-            switch (relation.operator())
-            {
-                case EQ:
-                {
-                    Term t = relation.getValue().prepare(keyspace(), defs);
-                    t.collectMarkerSpecification(boundNames);
-                    Restriction restriction = new MultiColumnRestriction.EQ(t, false);
-                    for (ColumnDefinition def : restrictedColumns)
-                        stmt.columnRestrictions[def.position()] = restriction;
-                    break;
-                }
-                case IN:
-                {
-                    Restriction restriction;
-                    List<? extends Term.MultiColumnRaw> inValues = relation.getInValues();
-                    if (inValues != null)
-                    {
-                        // we have something like "(a, b, c) IN ((1, 2, 3), (4, 5, 6), ...) or
-                        // "(a, b, c) IN (?, ?, ?)
-                        List<Term> terms = new ArrayList<>(inValues.size());
-                        for (Term.MultiColumnRaw tuple : inValues)
-                        {
-                            Term t = tuple.prepare(keyspace(), defs);
-                            t.collectMarkerSpecification(boundNames);
-                            terms.add(t);
-                        }
-                         restriction = new MultiColumnRestriction.InWithValues(terms);
-                    }
-                    else
-                    {
-                        Tuples.INRaw rawMarker = relation.getInMarker();
-                        AbstractMarker t = rawMarker.prepare(keyspace(), defs);
-                        t.collectMarkerSpecification(boundNames);
-                        restriction = new MultiColumnRestriction.InWithMarker(t);
-                    }
-                    for (ColumnDefinition def : restrictedColumns)
-                        stmt.columnRestrictions[def.position()] = restriction;
-
-                    break;
-                }
-                case LT:
-                case LTE:
-                case GT:
-                case GTE:
-                {
-                    Term t = relation.getValue().prepare(keyspace(), defs);
-                    t.collectMarkerSpecification(boundNames);
-
-                    Restriction.Slice existingRestriction = (Restriction.Slice) getExistingRestriction(stmt, defs.get(0));
-                    Restriction.Slice restriction;
-                    if (existingRestriction == null)
-                    {
-                        restriction = new MultiColumnRestriction.Slice(false);
-                    }
-                    else if (!existingRestriction.isMultiColumn())
-                    {
-                        restriction = new MultiColumnRestriction.Slice(false);
-                        restriction.setBound(existingRestriction);
-                    }
-                    else
-                    {
-                        restriction = existingRestriction;
-                    }
-                    restriction.setBound(relation.operator(), t);
-
-                    for (ColumnDefinition def : defs)
-                    {
-                        stmt.columnRestrictions[def.position()] = restriction;
-                    }
-                    break;
-                }
-                case NEQ:
-                    throw new InvalidRequestException(String.format("Unsupported \"!=\" relation: %s", relation));
-            }
+            checkFalse(restrictions.usesSecondaryIndexing(), "ORDER BY with 2ndary indexes is not supported.");
+            checkFalse(restrictions.isKeyRange(), "ORDER BY is only supported when the partition key is restricted by an EQ or an IN.");
         }
 
-        /**
-         * Checks that the operator for the specified column is compatible with the bounds of the existing restriction.
-         *
-         * @param existing the existing restriction
-         * @param def the column definition
-         * @param operator the operator
-         * @throws InvalidRequestException if the operator is not compatible with the bounds of the existing restriction
-         */
-        private static void checkBound(Restriction existing, ColumnDefinition def, Operator operator) throws InvalidRequestException
+        private static void validateDistinctSelection(CFMetaData cfm,
+                                                      Selection selection,
+                                                      StatementRestrictions restrictions)
+                                                      throws InvalidRequestException
         {
-            Restriction.Slice existingSlice = (Restriction.Slice) existing;
+            checkFalse(restrictions.hasClusteringColumnsRestriction() || restrictions.hasNonPrimaryKeyRestrictions(),
+                       "SELECT DISTINCT with WHERE clause only supports restriction by partition key.");
 
-            if (existingSlice.hasBound(Bound.START) && (operator == Operator.GT || operator == Operator.GTE))
-                throw new InvalidRequestException(String.format(
-                            "More than one restriction was found for the start bound on %s", def.name));
+            Collection<ColumnDefinition> requestedColumns = selection.getColumns();
+            for (ColumnDefinition def : requestedColumns)
+                checkFalse(!def.isPartitionKey() && !def.isStatic(),
+                           "SELECT DISTINCT queries must only request partition key columns and/or static columns (not %s)",
+                           def.name);
 
-            if (existingSlice.hasBound(Bound.END) && (operator == Operator.LT || operator == Operator.LTE))
-                throw new InvalidRequestException(String.format(
-                            "More than one restriction was found for the end bound on %s", def.name));
-        }
+            // If it's a key range, we require that all partition key columns are selected so we don't have to bother
+            // with post-query grouping.
+            if (!restrictions.isKeyRange())
+                return;
 
-        private static Restriction getExistingRestriction(SelectStatement stmt, ColumnDefinition def)
-        {
-            switch (def.kind)
-            {
-                case PARTITION_KEY:
-                    return stmt.keyRestrictions[def.position()];
-                case CLUSTERING_COLUMN:
-                    return stmt.columnRestrictions[def.position()];
-                case REGULAR:
-                case STATIC:
-                    return stmt.metadataRestrictions.get(def.name);
-                default:
-                    throw new AssertionError();
-            }
-        }
-
-        private void updateRestrictionsForRelation(SelectStatement stmt, ColumnDefinition def, SingleColumnRelation relation, VariableSpecifications names) throws InvalidRequestException
-        {
-            switch (def.kind)
-            {
-                case PARTITION_KEY:
-                {
-                    Restriction existingRestriction = stmt.keyRestrictions[def.position()];
-                    Restriction previousRestriction = def.position() == 0 ? null : stmt.keyRestrictions[def.position() - 1];
-                    stmt.keyRestrictions[def.position()] = updateSingleColumnRestriction(def, existingRestriction, previousRestriction, relation, names);
-                    break;
-                }
-                case CLUSTERING_COLUMN:
-                {
-                    Restriction existingRestriction = stmt.columnRestrictions[def.position()];
-                    Restriction previousRestriction = def.position() == 0 ? null : stmt.columnRestrictions[def.position() - 1];
-                    stmt.columnRestrictions[def.position()] = updateSingleColumnRestriction(def, existingRestriction, previousRestriction, relation, names);
-                    break;
-                }
-                case COMPACT_VALUE:
-                {
-                    throw new InvalidRequestException(String.format("Predicates on the non-primary-key column (%s) of a COMPACT table are not yet supported", def.name));
-                }
-                case REGULAR:
-                case STATIC:
-                {
-                    // We only all IN on the row key and last clustering key so far, never on non-PK columns, and this even if there's an index
-                    Restriction r = updateSingleColumnRestriction(def, stmt.metadataRestrictions.get(def.name), null, relation, names);
-
-                    if (r.isIN() && !((Restriction.IN)r).canHaveOnlyOneValue())
-                        // Note: for backward compatibility reason, we conside a IN of 1 value the same as a EQ, so we let that slide.
-                        throw new InvalidRequestException(String.format("IN predicates on non-primary-key columns (%s) is not yet supported", def.name));
-                    stmt.metadataRestrictions.put(def.name, r);
-                    break;
-                }
-            }
-        }
-
-        Restriction updateSingleColumnRestriction(ColumnDefinition def, Restriction existingRestriction, Restriction previousRestriction, SingleColumnRelation newRel, VariableSpecifications boundNames) throws InvalidRequestException
-        {
-            ColumnSpecification receiver = def;
-            if (newRel.onToken)
-            {
-                if (def.kind != ColumnDefinition.Kind.PARTITION_KEY)
-                    throw new InvalidRequestException(String.format("The token() function is only supported on the partition key, found on %s", def.name));
-
-                receiver = new ColumnSpecification(def.ksName,
-                                                   def.cfName,
-                                                   new ColumnIdentifier("partition key token", true),
-                                                   StorageService.getPartitioner().getTokenValidator());
-            }
-
-            // We don't support relations against entire collections (unless they're frozen), like "numbers = {1, 2, 3}"
-            if (receiver.type.isCollection() && receiver.type.isMultiCell() && !(newRel.operator() == Operator.CONTAINS_KEY || newRel.operator() == Operator.CONTAINS))
-            {
-                throw new InvalidRequestException(String.format("Collection column '%s' (%s) cannot be restricted by a '%s' relation",
-                                                                def.name, receiver.type.asCQL3Type(), newRel.operator()));
-            }
-
-            switch (newRel.operator())
-            {
-                case EQ:
-                {
-                    if (existingRestriction != null)
-                        throw new InvalidRequestException(String.format("%s cannot be restricted by more than one relation if it includes an Equal", def.name));
-                    Term t = newRel.getValue().prepare(keyspace(), receiver);
-                    t.collectMarkerSpecification(boundNames);
-                    existingRestriction = new SingleColumnRestriction.EQ(t, newRel.onToken);
-                }
-                break;
-                case IN:
-                    if (existingRestriction != null)
-                        throw new InvalidRequestException(String.format("%s cannot be restricted by more than one relation if it includes a IN", def.name));
-
-                    if (newRel.getInValues() == null)
-                    {
-                        // Means we have a "SELECT ... IN ?"
-                        assert newRel.getValue() != null;
-                        Term t = newRel.getValue().prepare(keyspace(), receiver);
-                        t.collectMarkerSpecification(boundNames);
-                        existingRestriction = new SingleColumnRestriction.InWithMarker((Lists.Marker)t);
-                    }
-                    else
-                    {
-                        List<Term> inValues = new ArrayList<>(newRel.getInValues().size());
-                        for (Term.Raw raw : newRel.getInValues())
-                        {
-                            Term t = raw.prepare(keyspace(), receiver);
-                            t.collectMarkerSpecification(boundNames);
-                            inValues.add(t);
-                        }
-                        existingRestriction = new SingleColumnRestriction.InWithValues(inValues);
-                    }
-                    break;
-                case NEQ:
-                    throw new InvalidRequestException(String.format("Unsupported \"!=\" relation on column \"%s\"", def.name));
-                case GT:
-                case GTE:
-                case LT:
-                case LTE:
-                    {
-                        // A slice restriction can be merged with another one under some conditions:
-                        // 1) both restrictions are on a token function or non of them are
-                        // (e.g. token(partitionKey) > token(?) AND token(partitionKey) <= token(?) or clustering1 > 1 AND clustering1 <= 2).
-                        // 2) both restrictions needs to start with the same column (e.g clustering1 > 0 AND (clustering1, clustering2) <= (2, 1)).
-                        if (existingRestriction == null)
-                            existingRestriction = new SingleColumnRestriction.Slice(newRel.onToken);
-                        else if (!existingRestriction.isSlice())
-                            throw new InvalidRequestException(String.format("Column \"%s\" cannot be restricted by both an equality and an inequality relation", def.name));
-                        else if (existingRestriction.isOnToken() != newRel.onToken)
-                            // For partition keys, we shouldn't have slice restrictions without token(). And while this is rejected later by
-                            // processPartitionKeysRestrictions, we shouldn't update the existing restriction by the new one if the old one was using token()
-                            // and the new one isn't since that would bypass that later test.
-                            throw new InvalidRequestException("Only EQ and IN relation are supported on the partition key (unless you use the token() function)");
-
-                        checkBound(existingRestriction, def, newRel.operator());
-
-                        if (def.position() != 0 && previousRestriction == existingRestriction)
-                            throw new InvalidRequestException(String.format("Column \"%s\" cannot be restricted by two inequalities not starting with the same column: %s",
-                                                                            def.name,
-                                                                            newRel));
-
-                        Term t = newRel.getValue().prepare(keyspace(), receiver);
-                        t.collectMarkerSpecification(boundNames);
-                        ((SingleColumnRestriction.Slice) existingRestriction).setBound(newRel.operator(), t);
-                    }
-                    break;
-                case CONTAINS_KEY:
-                    if (!(receiver.type instanceof MapType))
-                        throw new InvalidRequestException(String.format("Cannot use CONTAINS KEY on non-map column %s", def.name));
-                    // Fallthrough on purpose
-                case CONTAINS:
-                {
-                    if (!receiver.type.isCollection())
-                        throw new InvalidRequestException(String.format("Cannot use %s relation on non collection column %s", newRel.operator(), def.name));
-
-                    if (existingRestriction == null)
-                        existingRestriction = new SingleColumnRestriction.Contains();
-                    else if (!existingRestriction.isContains())
-                        throw new InvalidRequestException(String.format("Collection column %s can only be restricted by CONTAINS or CONTAINS KEY", def.name));
-
-                    boolean isKey = newRel.operator() == Operator.CONTAINS_KEY;
-                    receiver = makeCollectionReceiver(receiver, isKey);
-                    Term t = newRel.getValue().prepare(keyspace(), receiver);
-
-                    t.collectMarkerSpecification(boundNames);
-                    ((SingleColumnRestriction.Contains)existingRestriction).add(t, isKey);
-                    break;
-                }
-            }
-            return existingRestriction;
-        }
-
-        private void processPartitionKeyRestrictions(SelectStatement stmt, boolean hasQueriableIndex, CFMetaData cfm) throws InvalidRequestException
-        {
-            // If there is a queriable index, no special condition are required on the other restrictions.
-            // But we still need to know 2 things:
-            //   - If we don't have a queriable index, is the query ok
-            //   - Is it queriable without 2ndary index, which is always more efficient
-            // If a component of the partition key is restricted by a relation, all preceding
-            // components must have a EQ. Only the last partition key component can be in IN relation.
-            boolean canRestrictFurtherComponents = true;
-            ColumnDefinition previous = null;
-            stmt.keyIsInRelation = false;
-            Iterator<ColumnDefinition> iter = cfm.partitionKeyColumns().iterator();
-            for (int i = 0; i < stmt.keyRestrictions.length; i++)
-            {
-                ColumnDefinition cdef = iter.next();
-                Restriction restriction = stmt.keyRestrictions[i];
-
-                if (restriction == null)
-                {
-                    if (stmt.onToken)
-                        throw new InvalidRequestException("The token() function must be applied to all partition key components or none of them");
-
-                    // The only time not restricting a key part is allowed is if none are restricted or an index is used.
-                    if (i > 0 && stmt.keyRestrictions[i - 1] != null)
-                    {
-                        if (hasQueriableIndex)
-                        {
-                            stmt.usesSecondaryIndexing = true;
-                            stmt.isKeyRange = true;
-                            break;
-                        }
-                        throw new InvalidRequestException(String.format("Partition key part %s must be restricted since preceding part is", cdef.name));
-                    }
-
-                    stmt.isKeyRange = true;
-                    canRestrictFurtherComponents = false;
-                }
-                else if (!canRestrictFurtherComponents)
-                {
-                    if (hasQueriableIndex)
-                    {
-                        stmt.usesSecondaryIndexing = true;
-                        break;
-                    }
-                    throw new InvalidRequestException(String.format(
-                            "Partitioning column \"%s\" cannot be restricted because the preceding column (\"%s\") is " +
-                            "either not restricted or is restricted by a non-EQ relation", cdef.name, previous));
-                }
-                else if (restriction.isOnToken())
-                {
-                    // If this is a query on tokens, it's necessarily a range query (there can be more than one key per token).
-                    stmt.isKeyRange = true;
-                    stmt.onToken = true;
-                }
-                else if (stmt.onToken)
-                {
-                    throw new InvalidRequestException(String.format("The token() function must be applied to all partition key components or none of them"));
-                }
-                else if (!restriction.isSlice())
-                {
-                    if (restriction.isIN())
-                    {
-                        // We only support IN for the last name so far
-                        if (i != stmt.keyRestrictions.length - 1)
-                            throw new InvalidRequestException(String.format("Partition KEY part %s cannot be restricted by IN relation (only the last part of the partition key can)", cdef.name));
-                        stmt.keyIsInRelation = true;
-                    }
-                }
-                else
-                {
-                    // Non EQ relation is not supported without token(), even if we have a 2ndary index (since even those are ordered by partitioner).
-                    // Note: In theory we could allow it for 2ndary index queries with ALLOW FILTERING, but that would probably require some special casing
-                    // Note bis: This is also why we don't bother handling the 'tuple' notation of #4851 for keys. If we lift the limitation for 2ndary
-                    // index with filtering, we'll need to handle it though.
-                    throw new InvalidRequestException("Only EQ and IN relation are supported on the partition key (unless you use the token() function)");
-                }
-                previous = cdef;
-            }
-
-            if (stmt.onToken)
-                checkTokenFunctionArgumentsOrder(cfm);
-        }
-
-        /**
-         * Checks that the column identifiers used as argument for the token function have been specified in the
-         * partition key order.
-         * @param cfm the Column Family MetaData
-         * @throws InvalidRequestException if the arguments have not been provided in the proper order.
-         */
-        private void checkTokenFunctionArgumentsOrder(CFMetaData cfm) throws InvalidRequestException
-        {
-            Iterator<ColumnDefinition> iter = Iterators.cycle(cfm.partitionKeyColumns());
-            for (Relation relation : whereClause)
-            {
-                if (!relation.isOnToken())
-                    continue;
-
-                assert !relation.isMultiColumn() : "Unexpectedly got multi-column token relation";
-                SingleColumnRelation singleColumnRelation = (SingleColumnRelation) relation;
-                if (!cfm.getColumnDefinition(singleColumnRelation.getEntity().prepare(cfm)).equals(iter.next()))
-                    throw new InvalidRequestException(String.format("The token function arguments must be in the partition key order: %s",
-                                                                    Joiner.on(',').join(cfm.partitionKeyColumns())));
-            }
-        }
-
-        private void processColumnRestrictions(SelectStatement stmt, boolean hasQueriableIndex, CFMetaData cfm) throws InvalidRequestException
-        {
-            // If a clustering key column is restricted by a non-EQ relation, all preceding
-            // columns must have a EQ, and all following must have no restriction. Unless
-            // the column is indexed that is.
-            boolean canRestrictFurtherComponents = true;
-            ColumnDefinition previous = null;
-            Restriction previousRestriction = null;
-            Iterator<ColumnDefinition> iter = cfm.clusteringColumns().iterator();
-            for (int i = 0; i < stmt.columnRestrictions.length; i++)
-            {
-                ColumnDefinition cdef = iter.next();
-                Restriction restriction = stmt.columnRestrictions[i];
-
-                if (restriction == null)
-                {
-                    canRestrictFurtherComponents = false;
-                }
-                else if (!canRestrictFurtherComponents)
-                {
-                    // We're here if the previous clustering column was either not restricted, was a slice or an IN tulpe-notation.
-
-                    // we can continue if we are in the special case of a slice 'tuple' notation from #4851
-                    if (restriction != previousRestriction)
-                    {
-                        // if we have a 2ndary index, we need to use it
-                        if (hasQueriableIndex)
-                        {
-                            stmt.usesSecondaryIndexing = true;
-                            break;
-                        }
-
-                        if (previousRestriction == null)
-                            throw new InvalidRequestException(String.format(
-                                "PRIMARY KEY column \"%s\" cannot be restricted (preceding column \"%s\" is not restricted)", cdef.name, previous.name));
-
-                        if (previousRestriction.isMultiColumn() && previousRestriction.isIN())
-                            throw new InvalidRequestException(String.format(
-                                     "PRIMARY KEY column \"%s\" cannot be restricted (preceding column \"%s\" is restricted by an IN tuple notation)", cdef.name, previous.name));
-
-                        throw new InvalidRequestException(String.format(
-                                "PRIMARY KEY column \"%s\" cannot be restricted (preceding column \"%s\" is restricted by a non-EQ relation)", cdef.name, previous.name));
-                    }
-                }
-                else if (restriction.isSlice())
-                {
-                    canRestrictFurtherComponents = false;
-                    Restriction.Slice slice = (Restriction.Slice)restriction;
-                    // For non-composite slices, we don't support internally the difference between exclusive and
-                    // inclusive bounds, so we deal with it manually.
-                    if (!cfm.comparator.isCompound() && (!slice.isInclusive(Bound.START) || !slice.isInclusive(Bound.END)))
-                        stmt.sliceRestriction = slice;
-                }
-                else if (restriction.isIN())
-                {
-                    if (!restriction.isMultiColumn() && i != stmt.columnRestrictions.length - 1)
-                        throw new InvalidRequestException(String.format("Clustering column \"%s\" cannot be restricted by an IN relation", cdef.name));
-
-                    if (stmt.selectACollection())
-                        throw new InvalidRequestException(String.format("Cannot restrict column \"%s\" by IN relation as a collection is selected by the query", cdef.name));
-
-                    if (restriction.isMultiColumn())
-                        canRestrictFurtherComponents = false;
-                }
-                else if (restriction.isContains())
-                {
-                    if (!hasQueriableIndex)
-                        throw new InvalidRequestException(String.format("Cannot restrict column \"%s\" by a CONTAINS relation without a secondary index", cdef.name));
-                    stmt.usesSecondaryIndexing = true;
-                }
-
-                previous = cdef;
-                previousRestriction = restriction;
-            }
-        }
-
-        private void validateSecondaryIndexSelections(SelectStatement stmt) throws InvalidRequestException
-        {
-            if (stmt.keyIsInRelation)
-                throw new InvalidRequestException("Select on indexed columns and with IN clause for the PRIMARY KEY are not supported");
-            // When the user only select static columns, the intent is that we don't query the whole partition but just
-            // the static parts. But 1) we don't have an easy way to do that with 2i and 2) since we don't support index on static columns
-            // so far, 2i means that you've restricted a non static column, so the query is somewhat non-sensical.
-            if (stmt.selectsOnlyStaticColumns)
-                throw new InvalidRequestException("Queries using 2ndary indexes don't support selecting only static columns");            
-        }
-
-        private void verifyOrderingIsAllowed(SelectStatement stmt) throws InvalidRequestException
-        {
-            if (stmt.usesSecondaryIndexing)
-                throw new InvalidRequestException("ORDER BY with 2ndary indexes is not supported.");
-
-            if (stmt.isKeyRange)
-                throw new InvalidRequestException("ORDER BY is only supported when the partition key is restricted by an EQ or an IN.");
+            for (ColumnDefinition def : cfm.partitionKeyColumns())
+                checkTrue(requestedColumns.contains(def),
+                          "SELECT DISTINCT queries must request all the partition key columns (missing %s)", def.name);
         }
 
         private void handleUnrecognizedOrderingColumn(ColumnIdentifier column) throws InvalidRequestException
         {
-            if (containsAlias(column))
-                throw new InvalidRequestException(String.format("Aliases are not allowed in order by clause ('%s')", column));
-            else
-                throw new InvalidRequestException(String.format("Order by on unknown column %s", column));
+            checkFalse(containsAlias(column), "Aliases are not allowed in order by clause ('%s')", column);
+            checkFalse(true, "Order by on unknown column %s", column);
         }
 
-        private void processOrderingClause(SelectStatement stmt, CFMetaData cfm) throws InvalidRequestException
+        private Comparator<List<ByteBuffer>> getOrderingComparator(CFMetaData cfm,
+                                                                   Selection selection,
+                                                                   StatementRestrictions restrictions)
+                                                                   throws InvalidRequestException
         {
-            verifyOrderingIsAllowed(stmt);
+            if (!restrictions.keyIsInRelation())
+                return null;
 
-            // If we order post-query (see orderResults), the sorted column needs to be in the ResultSet for sorting, even if we don't
-            // ultimately ship them to the client (CASSANDRA-4911).
-            if (stmt.keyIsInRelation)
+            Map<ColumnIdentifier, Integer> orderingIndexes = getOrderingIndex(cfm, selection);
+
+            List<Integer> idToSort = new ArrayList<Integer>();
+            List<Comparator<ByteBuffer>> sorters = new ArrayList<Comparator<ByteBuffer>>();
+
+            for (ColumnIdentifier.Raw raw : parameters.orderings.keySet())
             {
-                stmt.orderingIndexes = new HashMap<>();
-                for (ColumnIdentifier.Raw rawColumn : stmt.parameters.orderings.keySet())
-                {
-                    ColumnIdentifier column = rawColumn.prepare(cfm);
-                    final ColumnDefinition def = cfm.getColumnDefinition(column);
-                    if (def == null)
-                        handleUnrecognizedOrderingColumn(column);
-
-                    int index = stmt.selection.getResultSetIndex(def);
-                    if (index < 0)
-                        index = stmt.selection.addColumnForOrdering(def);
-                    stmt.orderingIndexes.put(def.name, index);
-                }
+                ColumnIdentifier identifier = raw.prepare(cfm);
+                ColumnDefinition orderingColumn = cfm.getColumnDefinition(identifier);
+                idToSort.add(orderingIndexes.get(orderingColumn.name));
+                sorters.add(orderingColumn.type);
             }
-            stmt.isReversed = isReversed(stmt, cfm);
+            return idToSort.size() == 1 ? new SingleColumnComparator(idToSort.get(0), sorters.get(0))
+                    : new CompositeComparator(sorters, idToSort);
         }
 
-        private boolean isReversed(SelectStatement stmt, CFMetaData cfm) throws InvalidRequestException
+        private Map<ColumnIdentifier, Integer> getOrderingIndex(CFMetaData cfm, Selection selection)
+                throws InvalidRequestException
+        {
+            // If we order post-query (see orderResults), the sorted column needs to be in the ResultSet for sorting,
+            // even if we don't
+            // ultimately ship them to the client (CASSANDRA-4911).
+            Map<ColumnIdentifier, Integer> orderingIndexes = new HashMap<>();
+            for (ColumnIdentifier.Raw raw : parameters.orderings.keySet())
+            {
+                ColumnIdentifier column = raw.prepare(cfm);
+                final ColumnDefinition def = cfm.getColumnDefinition(column);
+                if (def == null)
+                    handleUnrecognizedOrderingColumn(column);
+                int index = selection.getResultSetIndex(def);
+                if (index < 0)
+                    index = selection.addColumnForOrdering(def);
+                orderingIndexes.put(def.name, index);
+            }
+            return orderingIndexes;
+        }
+
+        private boolean isReversed(CFMetaData cfm) throws InvalidRequestException
         {
             Boolean[] reversedMap = new Boolean[cfm.clusteringColumns().size()];
             int i = 0;
-            for (Map.Entry<ColumnIdentifier.Raw, Boolean> entry : stmt.parameters.orderings.entrySet())
+            for (Map.Entry<ColumnIdentifier.Raw, Boolean> entry : parameters.orderings.entrySet())
             {
                 ColumnIdentifier column = entry.getKey().prepare(cfm);
                 boolean reversed = entry.getValue();
@@ -2331,13 +998,13 @@
                 if (def == null)
                     handleUnrecognizedOrderingColumn(column);
 
-                if (def.kind != ColumnDefinition.Kind.CLUSTERING_COLUMN)
-                    throw new InvalidRequestException(String.format("Order by is currently only supported on the clustered columns of the PRIMARY KEY, got %s", column));
+                checkTrue(def.isClusteringColumn(),
+                          "Order by is currently only supported on the clustered columns of the PRIMARY KEY, got %s", column);
 
-                if (i++ != def.position())
-                    throw new InvalidRequestException(String.format("Order by currently only support the ordering of columns following their declared order in the PRIMARY KEY"));
+                checkTrue(i++ == def.position(),
+                          "Order by currently only support the ordering of columns following their declared order in the PRIMARY KEY");
 
-                reversedMap[def.position()] = (reversed != isReversedType(def));
+                reversedMap[def.position()] = (reversed != def.isReversedType());
             }
 
             // Check that all boolean in reversedMap, if set, agrees
@@ -2353,26 +1020,23 @@
                     isReversed = b;
                     continue;
                 }
-                if (!isReversed.equals(b))
-                    throw new InvalidRequestException(String.format("Unsupported order by relation"));
+                checkTrue(isReversed.equals(b), "Unsupported order by relation");
             }
             assert isReversed != null;
             return isReversed;
         }
 
         /** If ALLOW FILTERING was not specified, this verifies that it is not needed */
-        private void checkNeedsFiltering(SelectStatement stmt, int numberOfRestrictionsEvaluatedWithSlices) throws InvalidRequestException
+        private void checkNeedsFiltering(StatementRestrictions restrictions) throws InvalidRequestException
         {
             // non-key-range non-indexed queries cannot involve filtering underneath
-            if (!parameters.allowFiltering && (stmt.isKeyRange || stmt.usesSecondaryIndexing))
+            if (!parameters.allowFiltering && (restrictions.isKeyRange() || restrictions.usesSecondaryIndexing()))
             {
                 // We will potentially filter data if either:
                 //  - Have more than one IndexExpression
                 //  - Have no index expression and the column filter is not the identity
-                if (needFiltering(stmt, numberOfRestrictionsEvaluatedWithSlices))
-                    throw new InvalidRequestException("Cannot execute this query as it might involve data filtering and " +
-                                                      "thus may have unpredictable performance. If you want to execute " +
-                                                      "this query despite the performance unpredictability, use ALLOW FILTERING");
+                checkFalse(restrictions.needFiltering(),
+                           StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE);
             }
 
             // We don't internally support exclusive slice bounds on non-composite tables. To deal with it we do an
@@ -2385,49 +1049,15 @@
             // to bump the query limit by N where N is the number of rows we will return, but we don't know that in
             // advance. So, since we currently don't have a good way to handle such query, we refuse it (#7059) rather
             // than answering with something that is wrong.
-            if (stmt.sliceRestriction != null && stmt.isKeyRange && limit != null)
+            if (restrictions.isNonCompositeSliceWithExclusiveBounds() && restrictions.isKeyRange() && limit != null)
             {
-                SingleColumnRelation rel = findInclusiveClusteringRelationForCompact(stmt.cfm);
-                throw new InvalidRequestException(String.format("The query requests a restriction of rows with a strict bound (%s) over a range of partitions. "
-                                                              + "This is not supported by the underlying storage engine for COMPACT tables if a LIMIT is provided. "
-                                                              + "Please either make the condition non strict (%s) or remove the user LIMIT", rel, rel.withNonStrictOperator()));
+                SingleColumnRelation rel = findInclusiveClusteringRelationForCompact(restrictions.cfm);
+                throw invalidRequest("The query requests a restriction of rows with a strict bound (%s) over a range of partitions. "
+                                   + "This is not supported by the underlying storage engine for COMPACT tables if a LIMIT is provided. "
+                                   + "Please either make the condition non strict (%s) or remove the user LIMIT", rel, rel.withNonStrictOperator());
             }
         }
 
-        /**
-         * Checks if the specified statement will need to filter the data.
-         *
-         * @param stmt the statement to test.
-         * @param numberOfRestrictionsEvaluatedWithSlices the number of restrictions that can be evaluated with slices
-         * @return <code>true</code> if the specified statement will need to filter the data, <code>false</code>
-         * otherwise.
-         */
-        private static boolean needFiltering(SelectStatement stmt, int numberOfRestrictionsEvaluatedWithSlices)
-        {
-            boolean needFiltering = stmt.restrictedColumns.size() > 1
-                    || (stmt.restrictedColumns.isEmpty() && !stmt.columnFilterIsIdentity())
-                    || (!stmt.restrictedColumns.isEmpty()
-                            && stmt.isRestrictedByMultipleContains(Iterables.getOnlyElement(stmt.restrictedColumns.keySet())));
-
-            // For some secondary index queries, that were having some restrictions on non-indexed clustering columns,
-            // were not requiring ALLOW FILTERING as we should. The first time such a query is executed we will log a
-            // warning to notify the user (CASSANDRA-8418)
-            if (!needFiltering
-                    && !hasLoggedMissingAllowFilteringWarning
-                    && (stmt.restrictedColumns.size() + numberOfRestrictionsEvaluatedWithSlices) > 1)
-            {
-                hasLoggedMissingAllowFilteringWarning = true;
-
-                String msg = "Some secondary index queries with restrictions on non-indexed clustering columns "
-                           + "were executed without ALLOW FILTERING. In Cassandra 3.0, these queries will require "
-                           + "ALLOW FILTERING (see CASSANDRA-8418 for details).";
-
-                logger.warn(msg);
-            }
-
-            return needFiltering;
-        }
-
         private SingleColumnRelation findInclusiveClusteringRelationForCompact(CFMetaData cfm)
         {
             for (Relation r : whereClause)
@@ -2435,8 +1065,9 @@
                 // We only call this when sliceRestriction != null, i.e. for compact table with non composite comparator,
                 // so it can't be a MultiColumnRelation.
                 SingleColumnRelation rel = (SingleColumnRelation)r;
-                if (cfm.getColumnDefinition(rel.getEntity().prepare(cfm)).kind == ColumnDefinition.Kind.CLUSTERING_COLUMN
-                    && (rel.operator() == Operator.GT || rel.operator() == Operator.LT))
+
+                if (cfm.getColumnDefinition(rel.getEntity().prepare(cfm)).isClusteringColumn()
+                        && (rel.operator() == Operator.GT || rel.operator() == Operator.LT))
                     return rel;
             }
 
@@ -2460,23 +1091,6 @@
             return new ColumnSpecification(keyspace(), columnFamily(), new ColumnIdentifier("[limit]", true), Int32Type.instance);
         }
 
-        private static ColumnSpecification makeCollectionReceiver(ColumnSpecification collection, boolean isKey)
-        {
-            assert collection.type.isCollection();
-            switch (((CollectionType)collection.type).kind)
-            {
-                case LIST:
-                    assert !isKey;
-                    return Lists.valueSpecOf(collection);
-                case SET:
-                    assert !isKey;
-                    return Sets.valueSpecOf(collection);
-                case MAP:
-                    return isKey ? Maps.keySpecOf(collection) : Maps.valueSpecOf(collection);
-            }
-            throw new AssertionError();
-        }
-
         @Override
         public String toString()
         {
@@ -2485,7 +1099,6 @@
                           .add("selectClause", selectClause)
                           .add("whereClause", whereClause)
                           .add("isDistinct", parameters.isDistinct)
-                          .add("isCount", parameters.isCount)
                           .toString();
         }
     }
@@ -2495,28 +1108,36 @@
         // Public because CASSANDRA-9858
         public final Map<ColumnIdentifier.Raw, Boolean> orderings;
         public final boolean isDistinct;
-        public final boolean isCount;
-        public final ColumnIdentifier countAlias;
         public final boolean allowFiltering;
+        public final boolean isJson;
 
         public Parameters(Map<ColumnIdentifier.Raw, Boolean> orderings,
                           boolean isDistinct,
-                          boolean isCount,
-                          ColumnIdentifier countAlias,
-                          boolean allowFiltering)
+                          boolean allowFiltering,
+                          boolean isJson)
         {
             this.orderings = orderings;
             this.isDistinct = isDistinct;
-            this.isCount = isCount;
-            this.countAlias = countAlias;
             this.allowFiltering = allowFiltering;
+            this.isJson = isJson;
+        }
+    }
+
+    private static abstract class ColumnComparator<T> implements Comparator<T>
+    {
+        protected final int compare(Comparator<ByteBuffer> comparator, ByteBuffer aValue, ByteBuffer bValue)
+        {
+            if (aValue == null)
+                return bValue == null ? 0 : -1;
+
+            return bValue == null ? 1 : comparator.compare(aValue, bValue);
         }
     }
 
     /**
      * Used in orderResults(...) method when single 'ORDER BY' condition where given
      */
-    private static class SingleColumnComparator implements Comparator<List<ByteBuffer>>
+    private static class SingleColumnComparator extends ColumnComparator<List<ByteBuffer>>
     {
         private final int index;
         private final Comparator<ByteBuffer> comparator;
@@ -2529,14 +1150,14 @@
 
         public int compare(List<ByteBuffer> a, List<ByteBuffer> b)
         {
-            return comparator.compare(a.get(index), b.get(index));
+            return compare(comparator, a.get(index), b.get(index));
         }
     }
 
     /**
      * Used in orderResults(...) method when multiple 'ORDER BY' conditions where given
      */
-    private static class CompositeComparator implements Comparator<List<ByteBuffer>>
+    private static class CompositeComparator extends ColumnComparator<List<ByteBuffer>>
     {
         private final List<Comparator<ByteBuffer>> orderTypes;
         private final List<Integer> positions;
@@ -2554,10 +1175,7 @@
                 Comparator<ByteBuffer> type = orderTypes.get(i);
                 int columnPos = positions.get(i);
 
-                ByteBuffer aValue = a.get(columnPos);
-                ByteBuffer bValue = b.get(columnPos);
-
-                int comparison = type.compare(aValue, bValue);
+                int comparison = compare(type, a.get(columnPos), b.get(columnPos));
 
                 if (comparison != 0)
                     return comparison;
diff --git a/src/java/org/apache/cassandra/cql3/statements/Selectable.java b/src/java/org/apache/cassandra/cql3/statements/Selectable.java
deleted file mode 100644
index 09fc661..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/Selectable.java
+++ /dev/null
@@ -1,169 +0,0 @@
-/*
- * 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.cql3.statements;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.cql3.ColumnIdentifier;
-
-public interface Selectable
-{
-
-    public static interface Raw
-    {
-        public Selectable prepare(CFMetaData cfm);
-
-        /**
-         * Returns true if any processing is performed on the selected column.
-         **/
-        public boolean processesSelection();
-    }
-
-    public static class WritetimeOrTTL implements Selectable
-    {
-        public final ColumnIdentifier id;
-        public final boolean isWritetime;
-
-        public WritetimeOrTTL(ColumnIdentifier id, boolean isWritetime)
-        {
-            this.id = id;
-            this.isWritetime = isWritetime;
-        }
-
-        @Override
-        public String toString()
-        {
-            return (isWritetime ? "writetime" : "ttl") + "(" + id + ")";
-        }
-
-        public static class Raw implements Selectable.Raw
-        {
-            private final ColumnIdentifier.Raw id;
-            private final boolean isWritetime;
-
-            public Raw(ColumnIdentifier.Raw id, boolean isWritetime)
-            {
-                this.id = id;
-                this.isWritetime = isWritetime;
-            }
-
-            public WritetimeOrTTL prepare(CFMetaData cfm)
-            {
-                return new WritetimeOrTTL(id.prepare(cfm), isWritetime);
-            }
-
-            public boolean processesSelection()
-            {
-                return true;
-            }
-        }
-    }
-
-    public static class WithFunction implements Selectable
-    {
-        public final String functionName;
-        public final List<Selectable> args;
-
-        public WithFunction(String functionName, List<Selectable> args)
-        {
-            this.functionName = functionName;
-            this.args = args;
-        }
-
-        @Override
-        public String toString()
-        {
-            StringBuilder sb = new StringBuilder();
-            sb.append(functionName).append("(");
-            for (int i = 0; i < args.size(); i++)
-            {
-                if (i > 0) sb.append(", ");
-                sb.append(args.get(i));
-            }
-            return sb.append(")").toString();
-        }
-
-        public static class Raw implements Selectable.Raw
-        {
-            private final String functionName;
-            private final List<Selectable.Raw> args;
-
-            public Raw(String functionName, List<Selectable.Raw> args)
-            {
-                this.functionName = functionName;
-                this.args = args;
-            }
-
-            public WithFunction prepare(CFMetaData cfm)
-            {
-                List<Selectable> preparedArgs = new ArrayList<>(args.size());
-                for (Selectable.Raw arg : args)
-                    preparedArgs.add(arg.prepare(cfm));
-                return new WithFunction(functionName, preparedArgs);
-            }
-
-            public boolean processesSelection()
-            {
-                return true;
-            }
-        }
-    }
-
-    public static class WithFieldSelection implements Selectable
-    {
-        public final Selectable selected;
-        public final ColumnIdentifier field;
-
-        public WithFieldSelection(Selectable selected, ColumnIdentifier field)
-        {
-            this.selected = selected;
-            this.field = field;
-        }
-
-        @Override
-        public String toString()
-        {
-            return String.format("%s.%s", selected, field);
-        }
-
-        public static class Raw implements Selectable.Raw
-        {
-            private final Selectable.Raw selected;
-            private final ColumnIdentifier.Raw field;
-
-            public Raw(Selectable.Raw selected, ColumnIdentifier.Raw field)
-            {
-                this.selected = selected;
-                this.field = field;
-            }
-
-            public WithFieldSelection prepare(CFMetaData cfm)
-            {
-                return new WithFieldSelection(selected.prepare(cfm), field.prepare(cfm));
-            }
-
-            public boolean processesSelection()
-            {
-                return true;
-            }
-        }
-    }
-}
diff --git a/src/java/org/apache/cassandra/cql3/statements/Selection.java b/src/java/org/apache/cassandra/cql3/statements/Selection.java
deleted file mode 100644
index f50ec1b..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/Selection.java
+++ /dev/null
@@ -1,696 +0,0 @@
-/*
- * 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.cql3.statements;
-
-import java.nio.ByteBuffer;
-import java.util.*;
-
-import com.google.common.collect.Lists;
-
-import org.apache.cassandra.cql3.ColumnSpecification;
-
-import org.apache.cassandra.cql3.*;
-import org.apache.cassandra.cql3.functions.Function;
-import org.apache.cassandra.cql3.functions.Functions;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.db.Cell;
-import org.apache.cassandra.db.CounterCell;
-import org.apache.cassandra.db.ExpiringCell;
-import org.apache.cassandra.db.context.CounterContext;
-import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.db.marshal.Int32Type;
-import org.apache.cassandra.db.marshal.LongType;
-import org.apache.cassandra.db.marshal.UserType;
-import org.apache.cassandra.db.marshal.UTF8Type;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-public abstract class Selection
-{
-    private final List<ColumnDefinition> columns;
-    private final SelectionColumnMapping columnMapping;
-    private final ResultSet.Metadata metadata;
-    private final boolean collectTimestamps;
-    private final boolean collectTTLs;
-
-    protected Selection(List<ColumnDefinition> columns,
-                        SelectionColumnMapping columnMapping,
-                        boolean collectTimestamps,
-                        boolean collectTTLs)
-    {
-        this(columns, columnMapping, columnMapping.getColumnSpecifications(), collectTimestamps, collectTTLs);
-    }
-
-    // Alternative ctor for when we need to pass the List<ColumnSpecification> directly,
-    // rather than getting it from columnMapping. This is to handle COUNT correctly.
-    protected Selection(List<ColumnDefinition> columns,
-                        SelectionColumnMapping columnMapping,
-                        List<ColumnSpecification> columnSpecifications,
-                        boolean collectTimestamps,
-                        boolean collectTTLs)
-    {
-        this.columns = columns;
-        this.columnMapping = columnMapping;
-        this.metadata = new ResultSet.Metadata(columnSpecifications);
-        this.collectTimestamps = collectTimestamps;
-        this.collectTTLs = collectTTLs;
-    }
-
-    // Overriden by SimpleSelection when appropriate.
-    public boolean isWildcard()
-    {
-        return false;
-    }
-
-    public ResultSet.Metadata getResultMetadata()
-    {
-        return metadata;
-    }
-
-    public static Selection wildcard(CFMetaData cfm, boolean isCount, ColumnIdentifier countAlias)
-    {
-        List<ColumnDefinition> allColumns = Lists.newArrayList(cfm.allColumnsInSelectOrder());
-        SelectionColumnMapping columnMapping = isCount ? SelectionColumnMapping.countMapping(cfm, countAlias)
-                                                       : SelectionColumnMapping.simpleMapping(allColumns);
-
-        return new SimpleSelection(allColumns, columnMapping, true, isCount);
-    }
-
-    public static Selection forColumns(List<ColumnDefinition> columns)
-    {
-        return new SimpleSelection(columns);
-    }
-
-    public int addColumnForOrdering(ColumnDefinition c)
-    {
-        columns.add(c);
-        metadata.addNonSerializedColumn(c);
-        return columns.size() - 1;
-    }
-
-    /**
-     * Returns the index of the specified column within the resultset
-     * @param c the column
-     * @return the index of the specified column within the resultset or -1
-     */
-    public int getResultSetIndex(ColumnDefinition c)
-    {
-        return getColumnIndex(c);
-    }
-
-    /**
-     * Returns the index of the specified column
-     * @param c the column
-     * @return the index of the specified column or -1
-     */
-    protected final int getColumnIndex(ColumnDefinition c)
-    {
-        for (int i = 0, m = columns.size(); i < m; i++)
-            if (columns.get(i).name.equals(c.name))
-                return i;
-        return -1;
-    }
-
-    private static boolean requiresProcessing(List<RawSelector> rawSelectors)
-    {
-        for (RawSelector rawSelector : rawSelectors)
-        {
-            if (rawSelector.processesSelection())
-                return true;
-        }
-        return false;
-    }
-
-    private static int addAndGetIndex(ColumnDefinition def, List<ColumnDefinition> l)
-    {
-        int idx = l.indexOf(def);
-        if (idx < 0)
-        {
-            idx = l.size();
-            l.add(def);
-        }
-        return idx;
-    }
-
-    private static Selector makeSelector(CFMetaData cfm, RawSelector raw, List<ColumnDefinition> defs, SelectionColumnMapping columnMapping) throws InvalidRequestException
-    {
-        Selectable selectable = raw.selectable.prepare(cfm);
-        return makeSelector(cfm, selectable, raw.alias, defs, columnMapping);
-    }
-
-    private static Selector makeSelector(CFMetaData cfm, Selectable selectable, ColumnIdentifier alias, List<ColumnDefinition> defs, SelectionColumnMapping columnMapping) throws InvalidRequestException
-    {
-        if (selectable instanceof ColumnIdentifier)
-        {
-            ColumnDefinition def = cfm.getColumnDefinition((ColumnIdentifier) selectable);
-            if (def == null)
-                throw new InvalidRequestException(String.format("Undefined name %s in selection clause", selectable));
-
-            if (columnMapping != null)
-                columnMapping.addMapping(alias == null ? def : makeAliasSpec(cfm, def.type, alias), def);
-            return new SimpleSelector(def.name.toString(), addAndGetIndex(def, defs), def.type);
-        }
-        else if (selectable instanceof Selectable.WritetimeOrTTL)
-        {
-            Selectable.WritetimeOrTTL tot = (Selectable.WritetimeOrTTL)selectable;
-            ColumnDefinition def = cfm.getColumnDefinition(tot.id);
-            if (def == null)
-                throw new InvalidRequestException(String.format("Undefined name %s in selection clause", tot.id));
-            if (def.isPrimaryKeyColumn())
-                throw new InvalidRequestException(String.format("Cannot use selection function %s on PRIMARY KEY part %s", tot.isWritetime ? "writeTime" : "ttl", def.name));
-            if (def.type.isCollection())
-                throw new InvalidRequestException(String.format("Cannot use selection function %s on collections", tot.isWritetime ? "writeTime" : "ttl"));
-
-            if (columnMapping != null)
-                columnMapping.addMapping(makeWritetimeOrTTLSpec(cfm, tot, alias), def);
-            return new WritetimeOrTTLSelector(def.name.toString(), addAndGetIndex(def, defs), tot.isWritetime);
-        }
-        else if (selectable instanceof Selectable.WithFieldSelection)
-        {
-            Selectable.WithFieldSelection withField = (Selectable.WithFieldSelection)selectable;
-            // use a temporary columns mapping to collect the underlying column from the type selectable
-            SelectionColumnMapping tmpMapping = SelectionColumnMapping.newMapping();
-            Selector selected = makeSelector(cfm, withField.selected, null, defs, tmpMapping);
-            AbstractType<?> type = selected.getType();
-            if (!(type instanceof UserType))
-                throw new InvalidRequestException(String.format("Invalid field selection: %s of type %s is not a user type", withField.selected, type.asCQL3Type()));
-
-            UserType ut = (UserType)type;
-            for (int i = 0; i < ut.size(); i++)
-            {
-                if (!ut.fieldName(i).equals(withField.field.bytes))
-                    continue;
-
-                if (columnMapping != null)
-                    columnMapping.addMapping(makeFieldSelectSpec(cfm, withField, ut.fieldType(i), alias),
-                                             tmpMapping.getMappings().values());
-                return new FieldSelector(ut, i, selected);
-            }
-            throw new InvalidRequestException(String.format("%s of type %s has no field %s", withField.selected, type.asCQL3Type(), withField.field));
-        }
-        else
-        {
-            Selectable.WithFunction withFun = (Selectable.WithFunction)selectable;
-            List<Selector> args = new ArrayList<Selector>(withFun.args.size());
-            // use a temporary columns mapping to collate the columns used by all the function args
-            SelectionColumnMapping tmpMapping = SelectionColumnMapping.newMapping();
-            for (Selectable arg : withFun.args)
-                args.add(makeSelector(cfm, arg, null, defs, tmpMapping));
-
-            AbstractType<?> returnType = Functions.getReturnType(withFun.functionName, cfm.ksName, cfm.cfName);
-            if (returnType == null)
-                throw new InvalidRequestException(String.format("Unknown function '%s'", withFun.functionName));
-
-            ColumnSpecification spec = makeFunctionSpec(cfm, withFun, returnType, alias);
-            Function fun = Functions.get(cfm.ksName, withFun.functionName, args, spec);
-            if (columnMapping != null)
-                columnMapping.addMapping(spec, tmpMapping.getMappings().values());
-
-            return new FunctionSelector(fun, args);
-        }
-    }
-
-    private static ColumnSpecification makeWritetimeOrTTLSpec(CFMetaData cfm, Selectable.WritetimeOrTTL tot, ColumnIdentifier alias)
-    {
-        return new ColumnSpecification(cfm.ksName,
-                                       cfm.cfName,
-                                       alias == null ? new ColumnIdentifier(tot.toString(), true) : alias,
-                                       tot.isWritetime ? LongType.instance : Int32Type.instance);
-    }
-
-    private static ColumnSpecification makeFieldSelectSpec(CFMetaData cfm, Selectable.WithFieldSelection s, AbstractType<?> type, ColumnIdentifier alias)
-    {
-        return new ColumnSpecification(cfm.ksName,
-                                       cfm.cfName,
-                                       alias == null ? new ColumnIdentifier(s.toString(), true) : alias,
-                                       type);
-    }
-
-    private static ColumnSpecification makeFunctionSpec(CFMetaData cfm,
-                                                        Selectable.WithFunction fun,
-                                                        AbstractType<?> returnType,
-                                                        ColumnIdentifier alias) throws InvalidRequestException
-    {
-        if (returnType == null)
-            throw new InvalidRequestException(String.format("Unknown function %s called in selection clause", fun.functionName));
-
-        return new ColumnSpecification(cfm.ksName,
-                                       cfm.cfName,
-                                       alias == null ? new ColumnIdentifier(fun.toString(), true) : alias,
-                                       returnType);
-    }
-
-    private static ColumnSpecification makeAliasSpec(CFMetaData cfm, AbstractType<?> type, ColumnIdentifier alias)
-    {
-        return new ColumnSpecification(cfm.ksName, cfm.cfName, alias, type);
-    }
-
-    public static Selection fromSelectors(CFMetaData cfm, List<RawSelector> rawSelectors) throws InvalidRequestException
-    {
-        if (requiresProcessing(rawSelectors))
-        {
-            List<ColumnDefinition> defs = new ArrayList<ColumnDefinition>();
-            SelectionColumnMapping columnMapping = SelectionColumnMapping.newMapping();
-            List<Selector> selectors = new ArrayList<Selector>(rawSelectors.size());
-            boolean collectTimestamps = false;
-            boolean collectTTLs = false;
-            for (RawSelector rawSelector : rawSelectors)
-            {
-                Selector selector = makeSelector(cfm, rawSelector, defs, columnMapping);
-                selectors.add(selector);
-                collectTimestamps |= selector.usesTimestamps();
-                collectTTLs |= selector.usesTTLs();
-            }
-            return new SelectionWithProcessing(defs, columnMapping, selectors, collectTimestamps, collectTTLs);
-        }
-        else
-        {
-            List<ColumnDefinition> defs = new ArrayList<ColumnDefinition>(rawSelectors.size());
-            SelectionColumnMapping columnMapping = SelectionColumnMapping.newMapping();
-            for (RawSelector rawSelector : rawSelectors)
-            {
-                assert rawSelector.selectable instanceof ColumnIdentifier.Raw;
-                ColumnIdentifier id = (ColumnIdentifier) rawSelector.selectable.prepare(cfm);
-                ColumnDefinition def = cfm.getColumnDefinition(id);
-                if (def == null)
-                    throw new InvalidRequestException(String.format("Undefined name %s in selection clause", id));
-
-                defs.add(def);
-                columnMapping.addMapping(rawSelector.alias == null ? def : makeAliasSpec(cfm,
-                                                                                         def.type,
-                                                                                         rawSelector.alias),
-                                         def);
-            }
-            return new SimpleSelection(defs, columnMapping);
-        }
-    }
-
-    protected abstract List<ByteBuffer> handleRow(ResultSetBuilder rs) throws InvalidRequestException;
-
-    /**
-     * @return the list of CQL3 columns value this SelectionClause needs.
-     */
-    public List<ColumnDefinition> getColumns()
-    {
-        return columns;
-    }
-
-    /**
-     * @return the mappings between resultset columns and the underlying columns
-     */
-    public SelectionColumns getColumnMapping()
-    {
-        return columnMapping;
-    }
-
-    public ResultSetBuilder resultSetBuilder(long now)
-    {
-        return new ResultSetBuilder(now);
-    }
-
-    protected List<ColumnSpecification> getColumnSpecifications()
-    {
-        return columnMapping.getColumnSpecifications();
-    }
-
-    private static ByteBuffer value(Cell c)
-    {
-        return (c instanceof CounterCell)
-            ? ByteBufferUtil.bytes(CounterContext.instance().total(c.value()))
-            : c.value();
-    }
-
-    public class ResultSetBuilder
-    {
-        private final ResultSet resultSet;
-
-        /*
-         * We'll build CQL3 row one by one.
-         * The currentRow is the values for the (CQL3) columns we've fetched.
-         * We also collect timestamps and ttls for the case where the writetime and
-         * ttl functions are used. Note that we might collect timestamp and/or ttls
-         * we don't care about, but since the array below are allocated just once,
-         * it doesn't matter performance wise.
-         */
-        List<ByteBuffer> current;
-        final long[] timestamps;
-        final int[] ttls;
-        final long now;
-
-        private ResultSetBuilder(long now)
-        {
-            this.resultSet = new ResultSet(getResultMetadata().copy(), new ArrayList<List<ByteBuffer>>());
-            this.timestamps = collectTimestamps ? new long[columns.size()] : null;
-            this.ttls = collectTTLs ? new int[columns.size()] : null;
-            this.now = now;
-        }
-
-        public void add(ByteBuffer v)
-        {
-            current.add(v);
-        }
-
-        public void add(Cell c)
-        {
-            current.add(isDead(c) ? null : value(c));
-            if (timestamps != null)
-            {
-                timestamps[current.size() - 1] = isDead(c) ? Long.MIN_VALUE : c.timestamp();
-            }
-            if (ttls != null)
-            {
-                int ttl = -1;
-                if (!isDead(c) && c instanceof ExpiringCell)
-                    ttl = c.getLocalDeletionTime() - (int) (now / 1000);
-                ttls[current.size() - 1] = ttl;
-            }
-        }
-
-        private boolean isDead(Cell c)
-        {
-            return c == null || !c.isLive(now);
-        }
-
-        public void newRow() throws InvalidRequestException
-        {
-            if (current != null)
-                resultSet.addRow(handleRow(this));
-            current = new ArrayList<ByteBuffer>(columns.size());
-        }
-
-        public ResultSet build() throws InvalidRequestException
-        {
-            if (current != null)
-            {
-                resultSet.addRow(handleRow(this));
-                current = null;
-            }
-            return resultSet;
-        }
-    }
-
-    // Special cased selection for when no function is used (this save some allocations).
-    private static class SimpleSelection extends Selection
-    {
-        private final boolean isWildcard;
-        private final boolean isCount;
-
-        public SimpleSelection(List<ColumnDefinition> columns)
-        {
-            this(columns, SelectionColumnMapping.simpleMapping(columns), false, false);
-        }
-
-        public SimpleSelection(List<ColumnDefinition> columns, SelectionColumnMapping columnMapping)
-        {
-            this(columns, columnMapping, false, false);
-        }
-
-        public SimpleSelection(List<ColumnDefinition> columns,
-                               SelectionColumnMapping columnMapping,
-                               boolean wildcard,
-                               boolean isCount)
-        {
-            /*
-             * In theory, even a simple selection could have multiple time the same column, so we
-             * could filter those duplicate out of columns. But since we're very unlikely to
-             * get much duplicate in practice, it's more efficient not to bother.
-             */
-            // the List<ColumnSpecification> we pass depends on whether this is a COUNT or not
-            super(columns,
-                  columnMapping,
-                  isCount ? new ArrayList<ColumnSpecification>(columns) : columnMapping.getColumnSpecifications(),
-                  false,
-                  false);
-            this.isWildcard = wildcard;
-            this.isCount = isCount;
-        }
-
-        protected List<ByteBuffer> handleRow(ResultSetBuilder rs)
-        {
-            return rs.current;
-        }
-
-        @Override
-        public boolean isWildcard()
-        {
-            return isWildcard;
-        }
-    }
-
-    private static abstract class Selector implements AssignementTestable
-    {
-        public abstract ByteBuffer compute(ResultSetBuilder rs) throws InvalidRequestException;
-        public abstract AbstractType<?> getType();
-
-        public boolean isAssignableTo(String keyspace, ColumnSpecification receiver)
-        {
-            return receiver.type.isValueCompatibleWith(getType());
-        }
-
-        /** Returns true if the selector acts on a column's timestamp, false otherwise. */
-        public boolean usesTimestamps()
-        {
-            return false;
-        }
-
-        /** Returns true if the selector acts on a column's TTL, false otherwise. */
-        public boolean usesTTLs()
-        {
-            return false;
-        }
-    }
-
-    private static class SimpleSelector extends Selector
-    {
-        private final String columnName;
-        private final int idx;
-        private final AbstractType<?> type;
-
-        public SimpleSelector(String columnName, int idx, AbstractType<?> type)
-        {
-            this.columnName = columnName;
-            this.idx = idx;
-            this.type = type;
-        }
-
-        public ByteBuffer compute(ResultSetBuilder rs)
-        {
-            return rs.current.get(idx);
-        }
-
-        public AbstractType<?> getType()
-        {
-            return type;
-        }
-
-        @Override
-        public String toString()
-        {
-            return columnName;
-        }
-    }
-
-    private static class SelectionWithProcessing extends Selection
-    {
-        private final List<Selector> selectors;
-
-        public SelectionWithProcessing(List<ColumnDefinition> columns,
-                                       SelectionColumnMapping columnMapping,
-                                       List<Selector> selectors,
-                                       boolean collectTimestamps,
-                                       boolean collectTTLs)
-        {
-            super(columns, columnMapping, collectTimestamps, collectTTLs);
-            this.selectors = selectors;
-        }
-
-        protected List<ByteBuffer> handleRow(ResultSetBuilder rs) throws InvalidRequestException
-        {
-            List<ByteBuffer> result = new ArrayList<>();
-            for (Selector selector : selectors)
-                result.add(selector.compute(rs));
-            return result;
-        }
-
-        @Override
-        public int getResultSetIndex(ColumnDefinition c)
-        {
-            int index = getColumnIndex(c);
-
-            if (index < 0)
-                return -1;
-
-            for (int i = 0, m = selectors.size(); i < m; i++)
-            {
-                Selector selector = selectors.get(i);
-                if (selector instanceof SimpleSelector && ((SimpleSelector) selector).idx == index)
-                {
-                    return i;
-                }
-            }
-            return -1;
-        }
-
-        @Override
-        public int addColumnForOrdering(ColumnDefinition c)
-        {
-            int index = super.addColumnForOrdering(c);
-            selectors.add(new SimpleSelector(c.name.toString(), index, c.type));
-            return selectors.size() - 1;
-        }
-    }
-
-    private static class FunctionSelector extends Selector
-    {
-        private final Function fun;
-        private final List<Selector> argSelectors;
-
-        public FunctionSelector(Function fun, List<Selector> argSelectors)
-        {
-            this.fun = fun;
-            this.argSelectors = argSelectors;
-        }
-
-        public ByteBuffer compute(ResultSetBuilder rs) throws InvalidRequestException
-        {
-            List<ByteBuffer> args = new ArrayList<ByteBuffer>(argSelectors.size());
-            for (Selector s : argSelectors)
-                args.add(s.compute(rs));
-
-            return fun.execute(args);
-        }
-
-        public AbstractType<?> getType()
-        {
-            return fun.returnType();
-        }
-
-        public boolean usesTimestamps()
-        {
-            for (Selector s : argSelectors)
-                if (s.usesTimestamps())
-                    return true;
-            return false;
-        }
-
-        public boolean usesTTLs()
-        {
-            for (Selector s : argSelectors)
-                if (s.usesTTLs())
-                    return true;
-            return false;
-        }
-
-        @Override
-        public String toString()
-        {
-            StringBuilder sb = new StringBuilder();
-            sb.append(fun.name()).append("(");
-            for (int i = 0; i < argSelectors.size(); i++)
-            {
-                if (i > 0)
-                    sb.append(", ");
-                sb.append(argSelectors.get(i));
-            }
-            return sb.append(")").toString();
-        }
-    }
-
-    private static class FieldSelector extends Selector
-    {
-        private final UserType type;
-        private final int field;
-        private final Selector selected;
-
-        public FieldSelector(UserType type, int field, Selector selected)
-        {
-            this.type = type;
-            this.field = field;
-            this.selected = selected;
-        }
-
-        public ByteBuffer compute(ResultSetBuilder rs) throws InvalidRequestException
-        {
-            ByteBuffer value = selected.compute(rs);
-            if (value == null)
-                return null;
-            ByteBuffer[] buffers = type.split(value);
-            return field < buffers.length ? buffers[field] : null;
-        }
-
-        public AbstractType<?> getType()
-        {
-            return type.fieldType(field);
-        }
-
-        @Override
-        public String toString()
-        {
-            return String.format("%s.%s", selected, UTF8Type.instance.getString(type.fieldName(field)));
-        }
-    }
-
-    private static class WritetimeOrTTLSelector extends Selector
-    {
-        private final String columnName;
-        private final int idx;
-        private final boolean isWritetime;
-
-        public WritetimeOrTTLSelector(String columnName, int idx, boolean isWritetime)
-        {
-            this.columnName = columnName;
-            this.idx = idx;
-            this.isWritetime = isWritetime;
-        }
-
-        public ByteBuffer compute(ResultSetBuilder rs)
-        {
-            if (isWritetime)
-            {
-                long ts = rs.timestamps[idx];
-                return ts != Long.MIN_VALUE ? ByteBufferUtil.bytes(ts) : null;
-            }
-
-            int ttl = rs.ttls[idx];
-            return ttl > 0 ? ByteBufferUtil.bytes(ttl) : null;
-        }
-
-        public AbstractType<?> getType()
-        {
-            return isWritetime ? LongType.instance : Int32Type.instance;
-        }
-
-
-        public boolean usesTimestamps()
-        {
-            return isWritetime;
-        }
-
-        public boolean usesTTLs()
-        {
-            return !isWritetime;
-        }
-
-        @Override
-        public String toString()
-        {
-            return columnName;
-        }
-    }
-}
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectionColumnMapping.java b/src/java/org/apache/cassandra/cql3/statements/SelectionColumnMapping.java
deleted file mode 100644
index 39bd12c..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/SelectionColumnMapping.java
+++ /dev/null
@@ -1,161 +0,0 @@
-/*
- *
- * 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.cql3.statements;
-
-import java.util.*;
-
-import com.google.common.base.Function;
-import com.google.common.base.Joiner;
-import com.google.common.base.Objects;
-import com.google.common.collect.*;
-
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.cql3.ColumnIdentifier;
-import org.apache.cassandra.cql3.ColumnSpecification;
-import org.apache.cassandra.cql3.ResultSet;
-import org.apache.cassandra.db.marshal.LongType;
-
-public class SelectionColumnMapping implements SelectionColumns
-{
-    private final ArrayList<ColumnSpecification> columnSpecifications;
-    private final HashMultimap<ColumnSpecification, ColumnDefinition> columnMappings;
-
-    private SelectionColumnMapping()
-    {
-        this.columnSpecifications = new ArrayList<>();
-        this.columnMappings = HashMultimap.create();
-    }
-
-    protected static SelectionColumnMapping newMapping()
-    {
-        return new SelectionColumnMapping();
-    }
-
-    protected static SelectionColumnMapping countMapping(CFMetaData cfm, ColumnIdentifier countAlias)
-    {
-        ColumnSpecification spec = new ColumnSpecification(cfm.ksName,
-                                                           cfm.cfName,
-                                                           countAlias == null ? ResultSet.COUNT_COLUMN
-                                                                              : countAlias,
-                                                           LongType.instance);
-
-        return new SelectionColumnMapping().addMapping(spec, Collections.<ColumnDefinition>emptyList());
-    }
-
-    protected static SelectionColumnMapping simpleMapping(Iterable<ColumnDefinition> columnDefinitions)
-    {
-        SelectionColumnMapping mapping = new SelectionColumnMapping();
-        for (ColumnDefinition def: columnDefinitions)
-            mapping.addMapping(def, def);
-        return mapping;
-    }
-
-    protected SelectionColumnMapping addMapping(ColumnSpecification colSpec, ColumnDefinition column)
-    {
-        columnSpecifications.add(colSpec);
-        // functions without arguments do not map to any column, so don't
-        // record any mapping in that case
-        if (column != null)
-            columnMappings.put(colSpec, column);
-        return this;
-    }
-
-    protected SelectionColumnMapping addMapping(ColumnSpecification colSpec, Iterable<ColumnDefinition> columns)
-    {
-        columnSpecifications.add(colSpec);
-        columnMappings.putAll(colSpec, columns);
-        return this;
-    }
-
-    public List<ColumnSpecification> getColumnSpecifications()
-    {
-        // return a mutable copy as we may add extra columns
-        // for ordering (CASSANDRA-4911 & CASSANDRA-8286)
-        return Lists.newArrayList(columnSpecifications);
-    }
-
-    public Multimap<ColumnSpecification, ColumnDefinition> getMappings()
-    {
-        return Multimaps.unmodifiableMultimap(columnMappings);
-    }
-
-    public boolean equals(Object obj)
-    {
-        if (obj == null)
-            return false;
-
-        if (!(obj instanceof SelectionColumnMapping))
-            return false;
-
-        SelectionColumns other = (SelectionColumns)obj;
-        return Objects.equal(columnMappings, other.getMappings())
-            && Objects.equal(columnSpecifications, other.getColumnSpecifications());
-    }
-
-    public int hashCode()
-    {
-        return Objects.hashCode(columnMappings);
-    }
-
-    public String toString()
-    {
-        final Function<ColumnDefinition, String> getDefName = new Function<ColumnDefinition, String>()
-        {
-            public String apply(ColumnDefinition columnDefinition)
-            {
-                return columnDefinition.name.toString();
-            }
-        };
-        Function<Map.Entry<ColumnSpecification, Collection<ColumnDefinition>>, String> mappingEntryToString =
-        new Function<Map.Entry<ColumnSpecification, Collection<ColumnDefinition>>, String>(){
-            public String apply(Map.Entry<ColumnSpecification, Collection<ColumnDefinition>> entry)
-            {
-                StringBuilder builder = new StringBuilder();
-                builder.append(entry.getKey().name.toString());
-                builder.append(":[");
-                builder.append(Joiner.on(',').join(Iterables.transform(entry.getValue(), getDefName)));
-                builder.append("]");
-                return builder.toString();
-            }
-        };
-
-        Function<ColumnSpecification, String> colSpecToString = new Function<ColumnSpecification, String>()
-        {
-            public String apply(ColumnSpecification columnSpecification)
-            {
-                return columnSpecification.name.toString();
-            }
-        };
-
-        StringBuilder builder = new StringBuilder();
-        builder.append("{ Columns:[");
-        builder.append(Joiner.on(",")
-                             .join(Iterables.transform(columnSpecifications, colSpecToString )));
-        builder.append("], Mappings:[");
-        builder.append(Joiner.on(", ")
-                             .join(Iterables.transform(columnMappings.asMap().entrySet(),
-                                                       mappingEntryToString)));
-        builder.append("] }");
-        return builder.toString();
-    }
-
-}
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectionColumns.java b/src/java/org/apache/cassandra/cql3/statements/SelectionColumns.java
deleted file mode 100644
index fa6e214..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/SelectionColumns.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- *
- * 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.cql3.statements;
-
-import java.util.List;
-
-import com.google.common.collect.Multimap;
-
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.cql3.ColumnSpecification;
-
-/**
- * Represents a mapping between the actual columns used to satisfy a Selection
- * and the column definitions included in the resultset metadata for the query.
- */
-public interface SelectionColumns
-{
-    List<ColumnSpecification> getColumnSpecifications();
-    Multimap<ColumnSpecification, ColumnDefinition> getMappings();
-}
diff --git a/src/java/org/apache/cassandra/cql3/statements/SingleColumnRestriction.java b/src/java/org/apache/cassandra/cql3/statements/SingleColumnRestriction.java
deleted file mode 100644
index 17229a6..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/SingleColumnRestriction.java
+++ /dev/null
@@ -1,462 +0,0 @@
-/*
- * 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.cql3.statements;
-
-import org.apache.cassandra.cql3.*;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-public abstract class SingleColumnRestriction implements Restriction
-{
-    public boolean isMultiColumn()
-    {
-        return false;
-    }
-
-    public static class EQ extends SingleColumnRestriction implements Restriction.EQ
-    {
-        protected final Term value;
-        private final boolean onToken;
-
-        public EQ(Term value, boolean onToken)
-        {
-            this.value = value;
-            this.onToken = onToken;
-        }
-
-        public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
-        {
-            return Collections.singletonList(value.bindAndGet(options));
-        }
-
-        public boolean isSlice()
-        {
-            return false;
-        }
-
-        public boolean isEQ()
-        {
-            return true;
-        }
-
-        public boolean isIN()
-        {
-            return false;
-        }
-
-        public boolean isContains()
-        {
-            return false;
-        }
-
-        public boolean isOnToken()
-        {
-            return onToken;
-        }
-
-        public boolean canEvaluateWithSlices()
-        {
-            return true;
-        }
-
-        @Override
-        public String toString()
-        {
-            return String.format("EQ(%s)%s", value, onToken ? "*" : "");
-        }
-    }
-
-    public static class InWithValues extends SingleColumnRestriction implements Restriction.IN
-    {
-        protected final List<? extends Term> values;
-
-        public InWithValues(List<? extends Term> values)
-        {
-            this.values = values;
-        }
-
-        public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
-        {
-            List<ByteBuffer> buffers = new ArrayList<>(values.size());
-            for (Term value : values)
-                buffers.add(value.bindAndGet(options));
-            return buffers;
-        }
-
-        public boolean canHaveOnlyOneValue()
-        {
-            return values.size() == 1;
-        }
-
-        public boolean isSlice()
-        {
-            return false;
-        }
-
-        public boolean isEQ()
-        {
-            return false;
-        }
-
-        public boolean isIN()
-        {
-            return true;
-        }
-
-        public boolean isContains()
-        {
-            return false;
-        }
-
-        public boolean isOnToken()
-        {
-            return false;
-        }
-
-        public boolean canEvaluateWithSlices()
-        {
-            return true;
-        }
-
-        @Override
-        public String toString()
-        {
-            return String.format("IN(%s)", values);
-        }
-    }
-
-    public static class InWithMarker extends SingleColumnRestriction implements Restriction.IN
-    {
-        protected final AbstractMarker marker;
-
-        public InWithMarker(AbstractMarker marker)
-        {
-            this.marker = marker;
-        }
-
-        public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
-        {
-            Term.MultiItemTerminal lval = (Term.MultiItemTerminal)marker.bind(options);
-            if (lval == null)
-                throw new InvalidRequestException("Invalid null value for IN restriction");
-            return lval.getElements();
-        }
-
-        public boolean canHaveOnlyOneValue()
-        {
-            return false;
-        }
-
-        public boolean isSlice()
-        {
-            return false;
-        }
-
-        public boolean isEQ()
-        {
-            return false;
-        }
-
-        public boolean isIN()
-        {
-            return true;
-        }
-
-        public boolean isContains()
-        {
-            return false;
-        }
-
-        public boolean isOnToken()
-        {
-            return false;
-        }
-
-        public boolean canEvaluateWithSlices()
-        {
-            return true;
-        }
-
-        @Override
-        public String toString()
-        {
-            return "IN ?";
-        }
-    }
-
-    public static class Slice extends SingleColumnRestriction implements Restriction.Slice
-    {
-        protected final Term[] bounds;
-        protected final boolean[] boundInclusive;
-        protected final boolean onToken;
-
-        public Slice(boolean onToken)
-        {
-            this.bounds = new Term[2];
-            this.boundInclusive = new boolean[2];
-            this.onToken = onToken;
-        }
-
-        public boolean isSlice()
-        {
-            return true;
-        }
-
-        public boolean isEQ()
-        {
-            return false;
-        }
-
-        public boolean isIN()
-        {
-            return false;
-        }
-
-        public boolean isContains()
-        {
-            return false;
-        }
-
-        public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
-        {
-            throw new UnsupportedOperationException();
-        }
-
-        public boolean isOnToken()
-        {
-            return onToken;
-        }
-
-        public boolean canEvaluateWithSlices()
-        {
-            return true;
-        }
-
-        /** Returns true if the start or end bound (depending on the argument) is set, false otherwise */
-        public boolean hasBound(Bound b)
-        {
-            return bounds[b.idx] != null;
-        }
-
-        public Term bound(Bound b)
-        {
-            return bounds[b.idx];
-        }
-
-        public ByteBuffer bound(Bound b, QueryOptions options) throws InvalidRequestException
-        {
-            return bounds[b.idx].bindAndGet(options);
-        }
-
-        /** Returns true if the start or end bound (depending on the argument) is inclusive, false otherwise */
-        public boolean isInclusive(Bound b)
-        {
-            return bounds[b.idx] == null || boundInclusive[b.idx];
-        }
-
-        public Operator getRelation(Bound eocBound, Bound inclusiveBound)
-        {
-            switch (eocBound)
-            {
-                case START:
-                    return boundInclusive[inclusiveBound.idx] ? Operator.GTE : Operator.GT;
-                case END:
-                    return boundInclusive[inclusiveBound.idx] ? Operator.LTE : Operator.LT;
-            }
-            throw new AssertionError();
-        }
-
-        public Operator getIndexOperator(Bound b)
-        {
-            switch (b)
-            {
-                case START:
-                    return boundInclusive[b.idx] ? Operator.GTE : Operator.GT;
-                case END:
-                    return boundInclusive[b.idx] ? Operator.LTE : Operator.LT;
-            }
-            throw new AssertionError();
-        }
-
-        @Override
-        public final void setBound(Operator operator, Term t) throws InvalidRequestException
-        {
-            Bound b;
-            boolean inclusive;
-            switch (operator)
-            {
-                case GT:
-                    b = Bound.START;
-                    inclusive = false;
-                    break;
-                case GTE:
-                    b = Bound.START;
-                    inclusive = true;
-                    break;
-                case LT:
-                    b = Bound.END;
-                    inclusive = false;
-                    break;
-                case LTE:
-                    b = Bound.END;
-                    inclusive = true;
-                    break;
-                default:
-                    throw new AssertionError();
-            }
-
-            setBound(b, inclusive, t);
-        }
-
-        public void setBound(Restriction.Slice slice) throws InvalidRequestException
-        {
-            for (Bound bound : Bound.values())
-                if (slice.hasBound(bound))
-                    setBound(bound, slice.isInclusive(bound), slice.bound(bound));
-        }
-
-        private void setBound(Bound bound, boolean inclusive, Term term) throws InvalidRequestException {
-
-            assert bounds[bound.idx] == null;
-
-            bounds[bound.idx] = term;
-            boundInclusive[bound.idx] = inclusive;
-        }
-
-        @Override
-        public String toString()
-        {
-            return String.format("SLICE(%s %s, %s %s)%s", boundInclusive[0] ? ">=" : ">",
-                                 bounds[0],
-                                 boundInclusive[1] ? "<=" : "<",
-                                 bounds[1],
-                                 onToken ? "*" : "");
-        }
-    }
-
-    // This holds both CONTAINS and CONTAINS_KEY restriction because we might want to have both of them.
-    public static class Contains extends SingleColumnRestriction
-    {
-        private List<Term> values; // for CONTAINS
-        private List<Term> keys;   // for CONTAINS_KEY
-
-        public boolean hasContains()
-        {
-            return values != null;
-        }
-
-        public boolean hasContainsKey()
-        {
-            return keys != null;
-        }
-
-        public int numberOfValues()
-        {
-            return values == null ? 0 : values.size();
-        }
-
-        public int numberOfKeys()
-        {
-            return keys == null ? 0 : keys.size();
-        }
-
-        public void add(Term t, boolean isKey)
-        {
-            if (isKey)
-                addKey(t);
-            else
-                addValue(t);
-        }
-
-        public void addValue(Term t)
-        {
-            if (values == null)
-                values = new ArrayList<>();
-            values.add(t);
-        }
-
-        public void addKey(Term t)
-        {
-            if (keys == null)
-                keys = new ArrayList<>();
-            keys.add(t);
-        }
-
-        public List<ByteBuffer> values(QueryOptions options) throws InvalidRequestException
-        {
-            if (values == null)
-                return Collections.emptyList();
-
-            List<ByteBuffer> buffers = new ArrayList<ByteBuffer>(values.size());
-            for (Term value : values)
-                buffers.add(value.bindAndGet(options));
-            return buffers;
-        }
-
-        public List<ByteBuffer> keys(QueryOptions options) throws InvalidRequestException
-        {
-            if (keys == null)
-                return Collections.emptyList();
-
-            List<ByteBuffer> buffers = new ArrayList<ByteBuffer>(keys.size());
-            for (Term value : keys)
-                buffers.add(value.bindAndGet(options));
-            return buffers;
-        }
-
-        public boolean isSlice()
-        {
-            return false;
-        }
-
-        public boolean isEQ()
-        {
-            return false;
-        }
-
-        public boolean isIN()
-        {
-            return false;
-        }
-
-        public boolean isContains()
-        {
-            return true;
-        }
-
-        public boolean isOnToken()
-        {
-            return false;
-        }
-
-        public boolean canEvaluateWithSlices()
-        {
-            return false;
-        }
-
-        @Override
-        public String toString()
-        {
-            return String.format("CONTAINS(values=%s, keys=%s)", values, keys);
-        }
-    }
-}
diff --git a/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java b/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
index 09275e8..9234a79 100644
--- a/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
@@ -58,22 +58,13 @@
         ThriftValidation.validateColumnFamily(keyspace(), columnFamily());
     }
 
-    public ResultMessage execute(QueryState state, QueryOptions options)
-    throws InvalidRequestException, TruncateException
+    public ResultMessage execute(QueryState state, QueryOptions options) throws InvalidRequestException, TruncateException
     {
         try
         {
             StorageProxy.truncateBlocking(keyspace(), columnFamily());
         }
-        catch (UnavailableException e)
-        {
-            throw new TruncateException(e);
-        }
-        catch (TimeoutException e)
-        {
-            throw new TruncateException(e);
-        }
-        catch (IOException e)
+        catch (UnavailableException | TimeoutException | IOException e)
         {
             throw new TruncateException(e);
         }
@@ -81,7 +72,6 @@
     }
 
     public ResultMessage executeInternal(QueryState state, QueryOptions options)
-    throws TruncateException
     {
         try
         {
diff --git a/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java b/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
index 39e632a..517d842 100644
--- a/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/UpdateStatement.java
@@ -31,6 +31,8 @@
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.Pair;
 
+import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
+
 /**
  * An <code>UPDATE</code> statement parsed from a CQL query statement.
  *
@@ -84,7 +86,7 @@
         if (cfm.comparator.isDense())
         {
             if (prefix.isEmpty())
-                throw new InvalidRequestException(String.format("Missing PRIMARY KEY part %s", cfm.clusteringColumns().iterator().next().name));
+                throw new InvalidRequestException(String.format("Missing PRIMARY KEY part %s", cfm.clusteringColumns().get(0).name));
 
             // An empty name for the compact value is what we use to recognize the case where there is not column
             // outside the PK, see CreateStatement.
@@ -117,13 +119,13 @@
     }
 
     /**
-     * Checks that the value of the indexed columns is valid.
+     * Checks if the values of the indexed columns are valid.
      *
      * @param key row key for the column family
      * @param cf the column family
-     * @throws InvalidRequestException if one of the values is invalid
+     * @throws InvalidRequestException if one of the values of the indexed columns is not valid
      */
-    private void validateIndexedColumns(ByteBuffer key, ColumnFamily cf) throws InvalidRequestException
+    private void validateIndexedColumns(ByteBuffer key, ColumnFamily cf)
     {
         SecondaryIndexManager indexManager = Keyspace.open(cfm.ksName).getColumnFamilyStore(cfm.cfId).indexManager;
         if (indexManager.hasIndexes())
@@ -134,11 +136,11 @@
                 SecondaryIndex failedIndex = indexManager.validate(key, cell);
                 if (failedIndex != null)
                 {
-                    throw new InvalidRequestException(String.format("Can't index column value of size %d for index %s on %s.%s",
-                                                                    cell.value().remaining(),
-                                                                    failedIndex.getIndexName(),
-                                                                    cfm.ksName,
-                                                                    cfm.cfName));
+                    throw invalidRequest(String.format("Can't index column value of size %d for index %s on %s.%s",
+                                                       cell.value().remaining(),
+                                                       failedIndex.getIndexName(),
+                                                       cfm.ksName,
+                                                       cfm.cfName));
                 }
             }
             indexManager.validateRowLevelIndexes(key, cf);
@@ -154,13 +156,15 @@
          * A parsed <code>INSERT</code> statement.
          *
          * @param name column family being operated on
+         * @param attrs additional attributes for statement (CL, timestamp, timeToLive)
          * @param columnNames list of column names
          * @param columnValues list of column values (corresponds to names)
-         * @param attrs additional attributes for statement (CL, timestamp, timeToLive)
+         * @param ifNotExists true if an IF NOT EXISTS condition was specified, false otherwise
          */
         public ParsedInsert(CFName name,
                             Attributes.Raw attrs,
-                            List<ColumnIdentifier.Raw> columnNames, List<Term.Raw> columnValues,
+                            List<ColumnIdentifier.Raw> columnNames,
+                            List<Term.Raw> columnValues,
                             boolean ifNotExists)
         {
             super(name, attrs, null, ifNotExists, false);
@@ -170,16 +174,20 @@
 
         protected ModificationStatement prepareInternal(CFMetaData cfm, VariableSpecifications boundNames, Attributes attrs) throws InvalidRequestException
         {
-            UpdateStatement stmt = new UpdateStatement(ModificationStatement.StatementType.INSERT,boundNames.size(), cfm, attrs);
+            UpdateStatement stmt = new UpdateStatement(ModificationStatement.StatementType.INSERT, boundNames.size(), cfm, attrs);
 
             // Created from an INSERT
             if (stmt.isCounter())
-                throw new InvalidRequestException("INSERT statement are not allowed on counter tables, use UPDATE instead");
-            if (columnNames.size() != columnValues.size())
-                throw new InvalidRequestException("Unmatched column names/values");
+                throw new InvalidRequestException("INSERT statements are not allowed on counter tables, use UPDATE instead");
+
+            if (columnNames == null)
+                throw new InvalidRequestException("Column names for INSERT must be provided when using VALUES");
             if (columnNames.isEmpty())
                 throw new InvalidRequestException("No columns provided to INSERT");
+            if (columnNames.size() != columnValues.size())
+                throw new InvalidRequestException("Unmatched column names/values");
 
+            String ks = keyspace();
             for (int i = 0; i < columnNames.size(); i++)
             {
                 ColumnIdentifier id = columnNames.get(i).prepare(cfm);
@@ -195,22 +203,54 @@
                 }
 
                 Term.Raw value = columnValues.get(i);
-
-                switch (def.kind)
+                if (def.isPrimaryKeyColumn())
                 {
-                    case PARTITION_KEY:
-                    case CLUSTERING_COLUMN:
-                        Term t = value.prepare(keyspace(), def);
-                        t.collectMarkerSpecification(boundNames);
-                        stmt.addKeyValue(def, t);
-                        break;
-                    default:
-                        Operation operation = new Operation.SetValue(value).prepare(keyspace(), def);
-                        operation.collectMarkerSpecification(boundNames);
-                        stmt.addOperation(operation);
-                        break;
+                    Term t = value.prepare(ks, def);
+                    t.collectMarkerSpecification(boundNames);
+                    stmt.addKeyValue(def, t);
+                }
+                else
+                {
+                    Operation operation = new Operation.SetValue(value).prepare(ks, def);
+                    operation.collectMarkerSpecification(boundNames);
+                    stmt.addOperation(operation);
                 }
             }
+
+            return stmt;
+        }
+    }
+
+    /**
+     * A parsed INSERT JSON statement.
+     */
+    public static class ParsedInsertJson extends ModificationStatement.Parsed
+    {
+        private final Json.Raw jsonValue;
+
+        public ParsedInsertJson(CFName name, Attributes.Raw attrs, Json.Raw jsonValue, boolean ifNotExists)
+        {
+            super(name, attrs, null, ifNotExists, false);
+            this.jsonValue = jsonValue;
+        }
+
+        protected ModificationStatement prepareInternal(CFMetaData cfm, VariableSpecifications boundNames, Attributes attrs) throws InvalidRequestException
+        {
+            UpdateStatement stmt = new UpdateStatement(ModificationStatement.StatementType.INSERT, boundNames.size(), cfm, attrs);
+            if (stmt.isCounter())
+                throw new InvalidRequestException("INSERT statements are not allowed on counter tables, use UPDATE instead");
+
+            Collection<ColumnDefinition> defs = cfm.allColumns();
+            Json.Prepared prepared = jsonValue.prepareAndCollectMarkers(cfm, defs, boundNames);
+
+            for (ColumnDefinition def : defs)
+            {
+                if (def.isPrimaryKeyColumn())
+                    stmt.addKeyValue(def, prepared.getPrimaryKeyValueForColumn(def));
+                else
+                    stmt.addOperation(prepared.getSetOperationForColumn(def));
+            }
+
             return stmt;
         }
     }
diff --git a/src/java/org/apache/cassandra/db/AbstractCell.java b/src/java/org/apache/cassandra/db/AbstractCell.java
index 37d483f..bd63985 100644
--- a/src/java/org/apache/cassandra/db/AbstractCell.java
+++ b/src/java/org/apache/cassandra/db/AbstractCell.java
@@ -30,7 +30,7 @@
 import org.apache.cassandra.db.composites.CellNameType;
 import org.apache.cassandra.db.context.CounterContext;
 import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.format.Version;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.utils.FBUtilities;
 
@@ -39,7 +39,7 @@
     public static Iterator<OnDiskAtom> onDiskIterator(final DataInput in,
                                                       final ColumnSerializer.Flag flag,
                                                       final int expireBefore,
-                                                      final Descriptor.Version version,
+                                                      final Version version,
                                                       final CellNameType type)
     {
         return new AbstractIterator<OnDiskAtom>()
diff --git a/src/java/org/apache/cassandra/db/AbstractNativeCell.java b/src/java/org/apache/cassandra/db/AbstractNativeCell.java
index e01d860..207a972 100644
--- a/src/java/org/apache/cassandra/db/AbstractNativeCell.java
+++ b/src/java/org/apache/cassandra/db/AbstractNativeCell.java
@@ -37,6 +37,8 @@
 
 
 /**
+ * <pre>
+ * {@code
  * Packs a CellName AND a Cell into one off-heap representation.
  * Layout is:
  *
@@ -55,6 +57,8 @@
  * name extra:  lowest 2 bits indicate the clustering size delta (i.e. how many name items are NOT part of the clustering key)
  *              the next 2 bits indicate the CellNameType
  *              the next bit indicates if the column is a static or clustered/dynamic column
+ * }
+ * </pre>
  */
 public abstract class AbstractNativeCell extends AbstractCell implements CellName
 {
diff --git a/src/java/org/apache/cassandra/db/ArrayBackedSortedColumns.java b/src/java/org/apache/cassandra/db/ArrayBackedSortedColumns.java
index c53832b..1beb982 100644
--- a/src/java/org/apache/cassandra/db/ArrayBackedSortedColumns.java
+++ b/src/java/org/apache/cassandra/db/ArrayBackedSortedColumns.java
@@ -29,6 +29,7 @@
 import org.apache.cassandra.db.filter.ColumnSlice;
 import org.apache.cassandra.utils.BatchRemoveIterator;
 import org.apache.cassandra.utils.memory.AbstractAllocator;
+import org.apache.cassandra.utils.SearchIterator;
 
 /**
  * A ColumnFamily backed by an array.
@@ -282,10 +283,16 @@
     public void maybeAppendColumn(Cell cell, DeletionInfo.InOrderTester tester, int gcBefore)
     {
         if (cell.getLocalDeletionTime() >= gcBefore && !tester.isDeleted(cell))
-        {
-            internalAdd(cell);
-            sortedSize++;
-        }
+            appendColumn(cell);
+    }
+
+    /**
+     * Adds a cell, assuming that it sorts *strictly after* the current-last cell in the array.
+     */
+    public void appendColumn(Cell cell)
+    {
+        internalAdd(cell);
+        sortedSize++;
     }
 
     public void addColumn(Cell cell)
@@ -524,6 +531,59 @@
              : new SlicesIterator(slices, !reversed);
     }
 
+    public SearchIterator<CellName, Cell> searchIterator()
+    {
+        maybeSortCells();
+
+        return new SearchIterator<CellName, Cell>()
+        {
+            // the first index that we could find the next key at, i.e. one larger
+            // than the last key's location
+            private int i = 0;
+
+            // We assume a uniform distribution of keys,
+            // so we keep track of how many keys were skipped to satisfy last lookup, and only look at twice that
+            // many keys for next lookup initially, extending to whole range only if we couldn't find it in that subrange
+            private int range = size / 2;
+
+            public boolean hasNext()
+            {
+                return i < size;
+            }
+
+            public Cell next(CellName name)
+            {
+                if (!isSorted || !hasNext())
+                    throw new IllegalStateException();
+
+                // optimize for runs of sequential matches, as in CollationController
+                // checking to see if we've found the desired cells yet (CASSANDRA-6933)
+                int c = metadata.comparator.compare(name, cells[i].name());
+                if (c <= 0)
+                    return c < 0 ? null : cells[i++];
+
+                // use range to manually force a better bsearch "pivot" by breaking it into two calls:
+                // first for i..i+range, then i+range..size if necessary.
+                // https://issues.apache.org/jira/browse/CASSANDRA-6933?focusedCommentId=13958264&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13958264
+                int limit = Math.min(size, i + range);
+                int i2 = binarySearch(i + 1, limit, name, internalComparator());
+                if (-1 - i2 == limit)
+                    i2 = binarySearch(limit, size, name, internalComparator());
+                // i2 can't be zero since we already checked cells[i] above
+                if (i2 > 0)
+                {
+                    range = i2 - i;
+                    i = i2 + 1;
+                    return cells[i2];
+                }
+                i2 = -1 - i2;
+                range = i2 - i;
+                i = i2;
+                return null;
+            }
+        };
+    }
+
     private class SlicesIterator extends AbstractIterator<Cell>
     {
         private final ColumnSlice[] slices;
@@ -623,8 +683,17 @@
 
         public Cell next()
         {
-            shouldCallNext = false;
-            return cells[idx--];
+            try
+            {
+                shouldCallNext = false;
+                return cells[idx--];
+            }
+            catch (ArrayIndexOutOfBoundsException e)
+            {
+                NoSuchElementException ne = new NoSuchElementException(e.getMessage());
+                ne.initCause(e);
+                throw ne;
+            }
         }
 
         public void remove()
@@ -656,8 +725,17 @@
 
         public Cell next()
         {
-            shouldCallNext = false;
-            return cells[idx++];
+            try
+            {
+                shouldCallNext = false;
+                return cells[idx++];
+            }
+            catch (ArrayIndexOutOfBoundsException e)
+            {
+                NoSuchElementException ne = new NoSuchElementException(e.getMessage());
+                ne.initCause(e);
+                throw ne;
+            }
         }
 
         public void remove()
diff --git a/src/java/org/apache/cassandra/db/AtomDeserializer.java b/src/java/org/apache/cassandra/db/AtomDeserializer.java
index a103647..74f1946 100644
--- a/src/java/org/apache/cassandra/db/AtomDeserializer.java
+++ b/src/java/org/apache/cassandra/db/AtomDeserializer.java
@@ -23,7 +23,7 @@
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.composites.CellNameType;
 import org.apache.cassandra.db.composites.Composite;
-import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.format.Version;
 
 /**
  * Helper class to deserialize OnDiskAtom efficiently.
@@ -40,13 +40,13 @@
     private final DataInput in;
     private final ColumnSerializer.Flag flag;
     private final int expireBefore;
-    private final Descriptor.Version version;
+    private final Version version;
 
     // The "flag" for the next name (which correspond to the "masks" in ColumnSerializer) if it has been
     // read already, Integer.MIN_VALUE otherwise;
     private int nextFlags = Integer.MIN_VALUE;
 
-    public AtomDeserializer(CellNameType type, DataInput in, ColumnSerializer.Flag flag, int expireBefore, Descriptor.Version version)
+    public AtomDeserializer(CellNameType type, DataInput in, ColumnSerializer.Flag flag, int expireBefore, Version version)
     {
         this.type = type;
         this.nameDeserializer = type.newDeserializer(in);
diff --git a/src/java/org/apache/cassandra/db/AtomicBTreeColumns.java b/src/java/org/apache/cassandra/db/AtomicBTreeColumns.java
index d9eb29c..f5b7712 100644
--- a/src/java/org/apache/cassandra/db/AtomicBTreeColumns.java
+++ b/src/java/org/apache/cassandra/db/AtomicBTreeColumns.java
@@ -35,8 +35,11 @@
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.composites.Composite;
 import org.apache.cassandra.db.filter.ColumnSlice;
+import org.apache.cassandra.db.marshal.BytesType;
 import org.apache.cassandra.utils.*;
+import org.apache.cassandra.utils.SearchIterator;
 import org.apache.cassandra.utils.btree.BTree;
+import org.apache.cassandra.utils.btree.BTreeSearchIterator;
 import org.apache.cassandra.utils.btree.UpdateFunction;
 import org.apache.cassandra.utils.concurrent.Locks;
 import org.apache.cassandra.utils.concurrent.OpOrder;
@@ -52,12 +55,13 @@
  * isolated (in the sense of ACID). Typically a addAll is guaranteed that no
  * other thread can see the state where only parts but not all columns have
  * been added.
- * <p/>
+ * <p>
  * WARNING: removing element through getSortedColumns().iterator() is *not* supported
+ * </p>
  */
 public class AtomicBTreeColumns extends ColumnFamily
 {
-    static final long EMPTY_SIZE = ObjectSizes.measure(new AtomicBTreeColumns(CFMetaData.IndexCf, null))
+    static final long EMPTY_SIZE = ObjectSizes.measure(new AtomicBTreeColumns(CFMetaData.denseCFMetaData("keyspace", "table", BytesType.instance), null))
             + ObjectSizes.measure(new Holder(null, null));
 
     // Reserved values for wasteTracker field. These values must not be consecutive (see avoidReservedValues)
@@ -148,6 +152,11 @@
         delete(new DeletionInfo(tombstone, getComparator()));
     }
 
+    public SearchIterator<CellName, Cell> searchIterator()
+    {
+        return new BTreeSearchIterator<>(ref.tree, asymmetricComparator());
+    }
+
     public void delete(DeletionInfo info)
     {
         if (info.isLive())
@@ -189,7 +198,7 @@
      *
      * @return the difference in size seen after merging the given columns
      */
-    public Pair<Long, Long> addAllWithSizeDelta(final ColumnFamily cm, MemtableAllocator allocator, OpOrder.Group writeOp, Updater indexer)
+    public ColumnUpdater addAllWithSizeDelta(final ColumnFamily cm, MemtableAllocator allocator, OpOrder.Group writeOp, Updater indexer)
     {
         ColumnUpdater updater = new ColumnUpdater(this, cm.metadata, allocator, writeOp, indexer);
         DeletionInfo inputDeletionInfoCopy = null;
@@ -228,7 +237,7 @@
                 {
                     indexer.updateRowLevelIndexes();
                     updater.finish();
-                    return Pair.create(updater.dataSize, updater.colUpdateTimeDelta);
+                    return updater;
                 }
                 else if (!monitorOwned)
                 {
@@ -310,6 +319,11 @@
         throw new UnsupportedOperationException();
     }
 
+    public void appendColumn(Cell cell)
+    {
+        throw new UnsupportedOperationException();
+    }
+
     public void addAll(ColumnFamily cf)
     {
         throw new UnsupportedOperationException();
@@ -415,7 +429,7 @@
     }
 
     // the function we provide to the btree utilities to perform any column replacements
-    private static final class ColumnUpdater implements UpdateFunction<Cell>
+    static final class ColumnUpdater implements UpdateFunction<Cell>
     {
         final AtomicBTreeColumns updating;
         final CFMetaData metadata;
@@ -428,6 +442,7 @@
         long colUpdateTimeDelta = Long.MAX_VALUE;
         final MemtableAllocator.DataReclaimer reclaimer;
         List<Cell> inserted; // TODO: replace with walk of aborted BTree
+        long minTimestamp = Long.MAX_VALUE;
 
         private ColumnUpdater(AtomicBTreeColumns updating, CFMetaData metadata, MemtableAllocator allocator, OpOrder.Group writeOp, Updater indexer)
         {
@@ -448,6 +463,7 @@
             if (inserted == null)
                 inserted = new ArrayList<>();
             inserted.add(insert);
+            minTimestamp = Math.min(minTimestamp, insert.timestamp());
             return insert;
         }
 
@@ -455,6 +471,11 @@
         {
             Cell reconciled = existing.reconcile(update);
             indexer.update(existing, reconciled);
+            // pick the smallest timestamp because we want to be consistent with the logic applied when inserting
+            // a cell in apply(Cell insert) above. For example given 3 timestamps where T3 < T2 < T1 then we want
+            // [apply(T1) -> apply(T2) -> apply(T3)] and [apply(T3) -> apply(T2) -> apply(T1)] to both return the
+            // smallest value T3, see CompactionControllerTest.testMaxPurgeableTimestamp()
+            minTimestamp = Math.min(minTimestamp, update.timestamp());
             if (existing != reconciled)
             {
                 reconciled = reconciled.localCopy(metadata, allocator, writeOp);
@@ -481,6 +502,7 @@
                 inserted.clear();
             }
             reclaimer.cancel();
+            minTimestamp = Long.MAX_VALUE;
         }
 
         protected void abort(Cell abort)
diff --git a/src/java/org/apache/cassandra/db/BatchlogManager.java b/src/java/org/apache/cassandra/db/BatchlogManager.java
index 4588156..ba56f0d 100644
--- a/src/java/org/apache/cassandra/db/BatchlogManager.java
+++ b/src/java/org/apache/cassandra/db/BatchlogManager.java
@@ -31,20 +31,21 @@
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.*;
 import com.google.common.util.concurrent.RateLimiter;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.marshal.UUIDType;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.exceptions.WriteFailureException;
 import org.apache.cassandra.exceptions.WriteTimeoutException;
 import org.apache.cassandra.gms.FailureDetector;
 import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessageOut;
@@ -55,7 +56,6 @@
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.WrappedRunnable;
-
 import static org.apache.cassandra.cql3.QueryProcessor.executeInternal;
 
 public class BatchlogManager implements BatchlogManagerMBean
@@ -103,7 +103,7 @@
 
     public int countAllBatches()
     {
-        String query = String.format("SELECT count(*) FROM %s.%s", Keyspace.SYSTEM_KS, SystemKeyspace.BATCHLOG_CF);
+        String query = String.format("SELECT count(*) FROM %s.%s", SystemKeyspace.NAME, SystemKeyspace.BATCHLOG);
         return (int) executeInternal(query).one().getLong("count");
     }
 
@@ -138,35 +138,32 @@
     @VisibleForTesting
     static Mutation getBatchlogMutationFor(Collection<Mutation> mutations, UUID uuid, int version, long now)
     {
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(CFMetaData.BatchlogCf);
-        CFRowAdder adder = new CFRowAdder(cf, CFMetaData.BatchlogCf.comparator.builder().build(), now);
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(SystemKeyspace.Batchlog);
+        CFRowAdder adder = new CFRowAdder(cf, SystemKeyspace.Batchlog.comparator.builder().build(), now);
         adder.add("data", serializeMutations(mutations, version))
              .add("written_at", new Date(now / 1000))
              .add("version", version);
-        return new Mutation(Keyspace.SYSTEM_KS, UUIDType.instance.decompose(uuid), cf);
+        return new Mutation(SystemKeyspace.NAME, UUIDType.instance.decompose(uuid), cf);
     }
 
     private static ByteBuffer serializeMutations(Collection<Mutation> mutations, int version)
     {
-        DataOutputBuffer buf = new DataOutputBuffer();
-
-        try
+        try (DataOutputBuffer buf = new DataOutputBuffer())
         {
             buf.writeInt(mutations.size());
             for (Mutation mutation : mutations)
                 Mutation.serializer.serialize(mutation, buf, version);
+            return buf.buffer();
         }
         catch (IOException e)
         {
             throw new AssertionError(); // cannot happen.
         }
-
-        return buf.asByteBuffer();
     }
 
     private void replayAllFailedBatches() throws ExecutionException, InterruptedException
     {
-        logger.debug("Started replayAllFailedBatches");
+        logger.trace("Started replayAllFailedBatches");
 
         // rate limit is in bytes per second. Uses Double.MAX_VALUE if disabled (set to 0 in cassandra.yaml).
         // max rate is scaled by the number of nodes in the cluster (same as for HHOM - see CASSANDRA-5272).
@@ -174,8 +171,8 @@
         RateLimiter rateLimiter = RateLimiter.create(throttleInKB == 0 ? Double.MAX_VALUE : throttleInKB * 1024);
 
         UntypedResultSet page = executeInternal(String.format("SELECT id, data, written_at, version FROM %s.%s LIMIT %d",
-                                                              Keyspace.SYSTEM_KS,
-                                                              SystemKeyspace.BATCHLOG_CF,
+                                                              SystemKeyspace.NAME,
+                                                              SystemKeyspace.BATCHLOG,
                                                               PAGE_SIZE));
 
         while (!page.isEmpty())
@@ -186,21 +183,21 @@
                 break; // we've exhausted the batchlog, next query would be empty.
 
             page = executeInternal(String.format("SELECT id, data, written_at, version FROM %s.%s WHERE token(id) > token(?) LIMIT %d",
-                                                 Keyspace.SYSTEM_KS,
-                                                 SystemKeyspace.BATCHLOG_CF,
+                                                 SystemKeyspace.NAME,
+                                                 SystemKeyspace.BATCHLOG,
                                                  PAGE_SIZE),
                                    id);
         }
 
         cleanup();
 
-        logger.debug("Finished replayAllFailedBatches");
+        logger.trace("Finished replayAllFailedBatches");
     }
 
     private void deleteBatch(UUID id)
     {
-        Mutation mutation = new Mutation(Keyspace.SYSTEM_KS, UUIDType.instance.decompose(id));
-        mutation.delete(SystemKeyspace.BATCHLOG_CF, FBUtilities.timestampMicros());
+        Mutation mutation = new Mutation(SystemKeyspace.NAME, UUIDType.instance.decompose(id));
+        mutation.delete(SystemKeyspace.BATCHLOG, FBUtilities.timestampMicros());
         mutation.apply();
     }
 
@@ -265,7 +262,7 @@
         private final ByteBuffer data;
         private final int version;
 
-        private List<ReplayWriteResponseHandler> replayHandlers;
+        private List<ReplayWriteResponseHandler<Mutation>> replayHandlers;
 
         public Batch(UUID id, long writtenAt, ByteBuffer data, int version)
         {
@@ -277,7 +274,7 @@
 
         public int replay(RateLimiter rateLimiter) throws IOException
         {
-            logger.debug("Replaying batch {}", id);
+            logger.trace("Replaying batch {}", id);
 
             List<Mutation> mutations = replayingMutations();
 
@@ -299,14 +296,15 @@
         {
             for (int i = 0; i < replayHandlers.size(); i++)
             {
-                ReplayWriteResponseHandler handler = replayHandlers.get(i);
+                ReplayWriteResponseHandler<Mutation> handler = replayHandlers.get(i);
                 try
                 {
                     handler.get();
                 }
-                catch (WriteTimeoutException e)
+                catch (WriteTimeoutException|WriteFailureException e)
                 {
-                    logger.debug("Timed out replaying a batched mutation to a node, will write a hint");
+                    logger.trace("Failed replaying a batched mutation to a node, will write a hint");
+                    logger.trace("Failure was : {}", e.getMessage());
                     // writing hints for the rest to hints, starting from i
                     writeHintsForUndeliveredEndpoints(i);
                     return;
@@ -349,7 +347,7 @@
                 {
                     Mutation undeliveredMutation = replayingMutations.get(i);
                     int ttl = calculateHintTTL(replayingMutations);
-                    ReplayWriteResponseHandler handler = replayHandlers.get(i);
+                    ReplayWriteResponseHandler<Mutation> handler = replayHandlers.get(i);
 
                     if (ttl > 0 && handler != null)
                         for (InetAddress endpoint : handler.undelivered)
@@ -362,12 +360,12 @@
             }
         }
 
-        private List<ReplayWriteResponseHandler> sendReplays(List<Mutation> mutations, long writtenAt, int ttl)
+        private List<ReplayWriteResponseHandler<Mutation>> sendReplays(List<Mutation> mutations, long writtenAt, int ttl)
         {
-            List<ReplayWriteResponseHandler> handlers = new ArrayList<>(mutations.size());
+            List<ReplayWriteResponseHandler<Mutation>> handlers = new ArrayList<>(mutations.size());
             for (Mutation mutation : mutations)
             {
-                ReplayWriteResponseHandler handler = sendSingleReplayMutation(mutation, writtenAt, ttl);
+                ReplayWriteResponseHandler<Mutation> handler = sendSingleReplayMutation(mutation, writtenAt, ttl);
                 if (handler != null)
                     handlers.add(handler);
             }
@@ -380,7 +378,7 @@
          *
          * @return direct delivery handler to wait on or null, if no live nodes found
          */
-        private ReplayWriteResponseHandler sendSingleReplayMutation(final Mutation mutation, long writtenAt, int ttl)
+        private ReplayWriteResponseHandler<Mutation> sendSingleReplayMutation(final Mutation mutation, long writtenAt, int ttl)
         {
             Set<InetAddress> liveEndpoints = new HashSet<>();
             String ks = mutation.getKeyspaceName();
@@ -400,7 +398,7 @@
             if (liveEndpoints.isEmpty())
                 return null;
 
-            ReplayWriteResponseHandler handler = new ReplayWriteResponseHandler(liveEndpoints);
+            ReplayWriteResponseHandler<Mutation> handler = new ReplayWriteResponseHandler<>(liveEndpoints);
             MessageOut<Mutation> message = mutation.createMessage();
             for (InetAddress endpoint : liveEndpoints)
                 MessagingService.instance().sendRR(message, endpoint, handler, false);
@@ -419,7 +417,11 @@
             return unadjustedTTL - (int) TimeUnit.MILLISECONDS.toSeconds(System.currentTimeMillis() - writtenAt);
         }
 
-        private static class ReplayWriteResponseHandler extends WriteResponseHandler
+        /**
+         * A wrapper of WriteResponseHandler that stores the addresses of the endpoints from
+         * which we did not receive a successful reply.
+         */
+        private static class ReplayWriteResponseHandler<T> extends WriteResponseHandler<T>
         {
             private final Set<InetAddress> undelivered = Collections.newSetFromMap(new ConcurrentHashMap<InetAddress, Boolean>());
 
@@ -436,9 +438,9 @@
             }
 
             @Override
-            public void response(MessageIn m)
+            public void response(MessageIn<T> m)
             {
-                boolean removed = undelivered.remove(m.from);
+                boolean removed = undelivered.remove(m == null ? FBUtilities.getBroadcastAddress() : m.from);
                 assert removed;
                 super.response(m);
             }
@@ -448,7 +450,7 @@
     // force flush + compaction to reclaim space from the replayed batches
     private void cleanup() throws ExecutionException, InterruptedException
     {
-        ColumnFamilyStore cfs = Keyspace.open(Keyspace.SYSTEM_KS).getColumnFamilyStore(SystemKeyspace.BATCHLOG_CF);
+        ColumnFamilyStore cfs = Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStore(SystemKeyspace.BATCHLOG);
         cfs.forceBlockingFlush();
         Collection<Descriptor> descriptors = new ArrayList<>();
         for (SSTableReader sstr : cfs.getSSTables())
diff --git a/src/java/org/apache/cassandra/db/BufferDecoratedKey.java b/src/java/org/apache/cassandra/db/BufferDecoratedKey.java
index 8a1ad59..d375162 100644
--- a/src/java/org/apache/cassandra/db/BufferDecoratedKey.java
+++ b/src/java/org/apache/cassandra/db/BufferDecoratedKey.java
@@ -20,8 +20,6 @@
 import java.nio.ByteBuffer;
 
 import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.utils.FastByteOperations;
-import org.apache.cassandra.utils.memory.MemoryUtil;
 
 public class BufferDecoratedKey extends DecoratedKey
 {
diff --git a/src/java/org/apache/cassandra/db/BufferExpiringCell.java b/src/java/org/apache/cassandra/db/BufferExpiringCell.java
index 25172c8..efb56d5 100644
--- a/src/java/org/apache/cassandra/db/BufferExpiringCell.java
+++ b/src/java/org/apache/cassandra/db/BufferExpiringCell.java
@@ -137,9 +137,9 @@
         super.validateFields(metadata);
 
         if (timeToLive <= 0)
-            throw new MarshalException("A column TTL should be > 0");
+            throw new MarshalException("A column TTL should be > 0, but was " + timeToLive);
         if (localExpirationTime < 0)
-            throw new MarshalException("The local expiration time should not be negative");
+            throw new MarshalException("The local expiration time should not be negative but was " + localExpirationTime);
     }
 
     @Override
diff --git a/src/java/org/apache/cassandra/db/CFRowAdder.java b/src/java/org/apache/cassandra/db/CFRowAdder.java
index 3ff9171..6fab8d5 100644
--- a/src/java/org/apache/cassandra/db/CFRowAdder.java
+++ b/src/java/org/apache/cassandra/db/CFRowAdder.java
@@ -41,13 +41,20 @@
     public final ColumnFamily cf;
     public final Composite prefix;
     public final long timestamp;
+    public final int ttl;
     private final int ldt;
 
     public CFRowAdder(ColumnFamily cf, Composite prefix, long timestamp)
     {
+        this(cf, prefix, timestamp, 0);
+    }
+
+    public CFRowAdder(ColumnFamily cf, Composite prefix, long timestamp, int ttl)
+    {
         this.cf = cf;
         this.prefix = prefix;
         this.timestamp = timestamp;
+        this.ttl = ttl;
         this.ldt = (int) (System.currentTimeMillis() / 1000);
 
         // If a CQL3 table, add the row marker
@@ -103,7 +110,11 @@
             AbstractType valueType = def.type.isCollection()
                                    ? ((CollectionType) def.type).valueComparator()
                                    : def.type;
-            cf.addColumn(new BufferCell(name, value instanceof ByteBuffer ? (ByteBuffer)value : valueType.decompose(value), timestamp));
+            ByteBuffer valueBytes = value instanceof ByteBuffer ? (ByteBuffer)value : valueType.decompose(value);
+            if (ttl == 0)
+                cf.addColumn(new BufferCell(name, valueBytes, timestamp));
+            else
+                cf.addColumn(new BufferExpiringCell(name, valueBytes, timestamp, ttl));
         }
         return this;
     }
diff --git a/src/java/org/apache/cassandra/db/CachedHashDecoratedKey.java b/src/java/org/apache/cassandra/db/CachedHashDecoratedKey.java
new file mode 100644
index 0000000..5b81e73
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/CachedHashDecoratedKey.java
@@ -0,0 +1,52 @@
+/*
+ * 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.db;
+
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.dht.Token;
+
+public class CachedHashDecoratedKey extends BufferDecoratedKey
+{
+    long hash0;
+    long hash1;
+    volatile boolean hashCached;
+
+    public CachedHashDecoratedKey(Token token, ByteBuffer key)
+    {
+        super(token, key);
+        hashCached = false;
+    }
+
+    @Override
+    public void filterHash(long[] dest)
+    {
+        if (hashCached)
+        {
+            dest[0] = hash0;
+            dest[1] = hash1;
+        }
+        else
+        {
+            super.filterHash(dest);
+            hash0 = dest[0];
+            hash1 = dest[1];
+            hashCached = true;
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/CollationController.java b/src/java/org/apache/cassandra/db/CollationController.java
index 0f2674e..5be3bd2 100644
--- a/src/java/org/apache/cassandra/db/CollationController.java
+++ b/src/java/org/apache/cassandra/db/CollationController.java
@@ -28,18 +28,17 @@
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Iterators;
 
-import net.nicoulaj.compilecommand.annotations.Inline;
 import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.concurrent.StageManager;
 import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
-import org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.filter.NamesQueryFilter;
 import org.apache.cassandra.db.filter.QueryFilter;
 import org.apache.cassandra.db.marshal.CounterColumnType;
-import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.tracing.Tracing;
+import org.apache.cassandra.utils.SearchIterator;
 import org.apache.cassandra.utils.memory.HeapAllocator;
 
 public class CollationController
@@ -186,10 +185,11 @@
         if (container == null)
             return;
 
-        for (Iterator<CellName> iterator = ((NamesQueryFilter) filter.filter).columns.iterator(); iterator.hasNext(); )
+        SearchIterator<CellName, Cell> searchIter = container.searchIterator();
+        for (Iterator<CellName> iterator = ((NamesQueryFilter) filter.filter).columns.iterator(); iterator.hasNext() && searchIter.hasNext(); )
         {
             CellName filterColumn = iterator.next();
-            Cell cell = container.getColumn(filterColumn);
+            Cell cell = searchIter.next(filterColumn);
             if (cell != null && cell.timestamp() > sstableTimestamp)
                 iterator.remove();
         }
@@ -301,8 +301,11 @@
                     }
                 }
             }
+
             if (Tracing.isTracing())
-                Tracing.trace("Skipped {}/{} non-slice-intersecting sstables, included {} due to tombstones", new Object[] {nonIntersectingSSTables, view.sstables.size(), includedDueToTombstones});
+                Tracing.trace("Skipped {}/{} non-slice-intersecting sstables, included {} due to tombstones",
+                              nonIntersectingSSTables, view.sstables.size(), includedDueToTombstones);
+
             // we need to distinguish between "there is no data at all for this row" (BF will let us rebuild that efficiently)
             // and "there used to be data, but it's gone now" (we should cache the empty CF so we don't need to rebuild that slower)
             if (iterators.isEmpty())
diff --git a/src/java/org/apache/cassandra/db/ColumnFamily.java b/src/java/org/apache/cassandra/db/ColumnFamily.java
index c9a008f..a7243a2 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamily.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamily.java
@@ -183,6 +183,8 @@
     public abstract void delete(DeletionTime deletionTime);
     protected abstract void delete(RangeTombstone tombstone);
 
+    public abstract SearchIterator<CellName, Cell> searchIterator();
+
     /**
      * Purges top-level and range tombstones whose localDeletionTime is older than gcBefore.
      * @param gcBefore a timestamp (in seconds) before which tombstones should be purged
@@ -203,6 +205,11 @@
     public abstract void maybeAppendColumn(Cell cell, DeletionInfo.InOrderTester tester, int gcBefore);
 
     /**
+     * Appends a cell. Requires that the cell to add is sorted strictly after the last cell in the container.
+     */
+    public abstract void appendColumn(Cell cell);
+
+    /**
      * Adds all the columns of a given column map to this column map.
      * This is equivalent to:
      *   <code>
@@ -388,8 +395,8 @@
     {
         for (Cell cell : this)
             cell.updateDigest(digest);
-        if (MessagingService.instance().areAllNodesAtLeast21())
-            deletionInfo().updateDigest(digest);
+
+        deletionInfo().updateDigest(digest);
     }
 
     public static ColumnFamily diff(ColumnFamily cf1, ColumnFamily cf2)
@@ -514,9 +521,11 @@
 
     public ByteBuffer toBytes()
     {
-        DataOutputBuffer out = new DataOutputBuffer();
-        serializer.serialize(this, out, MessagingService.current_version);
-        return ByteBuffer.wrap(out.getData(), 0, out.getLength());
+        try (DataOutputBuffer out = new DataOutputBuffer())
+        {
+            serializer.serialize(this, out, MessagingService.current_version);
+            return ByteBuffer.wrap(out.getData(), 0, out.getLength());
+        }
     }
 
 
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilySerializer.java b/src/java/org/apache/cassandra/db/ColumnFamilySerializer.java
index f139369..928c21f 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilySerializer.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilySerializer.java
@@ -18,14 +18,13 @@
 package org.apache.cassandra.db;
 
 import java.io.DataInput;
-import java.io.DataOutput;
 import java.io.IOException;
 import java.util.UUID;
 
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.io.ISSTableSerializer;
 import org.apache.cassandra.io.IVersionedSerializer;
-import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.format.Version;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.utils.UUIDSerializer;
@@ -71,7 +70,7 @@
                 columnSerializer.serialize(cell, out);
                 written++;
             }
-            assert count == written: "Column family had " + count + " columns, but " + written + " written";
+            assert count == written: "Table had " + count + " columns, but " + written + " written";
         }
         catch (IOException e)
         {
@@ -147,7 +146,7 @@
         throw new UnsupportedOperationException();
     }
 
-    public ColumnFamily deserializeFromSSTable(DataInput in, Descriptor.Version version)
+    public ColumnFamily deserializeFromSSTable(DataInput in, Version version)
     {
         throw new UnsupportedOperationException();
     }
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 6e82745..2e52eb2 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -26,6 +26,7 @@
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.regex.Pattern;
+
 import javax.management.*;
 import javax.management.openmbean.*;
 
@@ -35,11 +36,14 @@
 import com.google.common.collect.*;
 import com.google.common.util.concurrent.*;
 
+import org.apache.cassandra.db.lifecycle.SSTableIntervalTree;
+import org.apache.cassandra.db.lifecycle.View;
+import org.apache.cassandra.db.lifecycle.Tracker;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.io.FSWriteError;
 import org.json.simple.*;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
 import org.apache.cassandra.cache.*;
 import org.apache.cassandra.concurrent.*;
 import org.apache.cassandra.config.*;
@@ -62,8 +66,9 @@
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.FSReadError;
 import org.apache.cassandra.io.compress.CompressionParameters;
-import org.apache.cassandra.io.sstable.*;
 import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.*;
+import org.apache.cassandra.io.sstable.format.*;
 import org.apache.cassandra.io.sstable.metadata.CompactionMetadata;
 import org.apache.cassandra.io.sstable.metadata.MetadataType;
 import org.apache.cassandra.io.util.FileUtils;
@@ -80,6 +85,8 @@
 
 import com.clearspring.analytics.stream.Counter;
 
+import static org.apache.cassandra.utils.Throwables.maybeFail;
+
 public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 {
     private static final Logger logger = LoggerFactory.getLogger(ColumnFamilyStore.class);
@@ -143,6 +150,9 @@
         }
     }
 
+    @VisibleForTesting
+    public static volatile ColumnFamilyStore discardFlushResults;
+
     public final Keyspace keyspace;
     public final String name;
     public final CFMetaData metadata;
@@ -153,12 +163,12 @@
     /**
      * Memtables and SSTables on disk for this column family.
      *
-     * We synchronize on the DataTracker to ensure isolation when we want to make sure
+     * We synchronize on the Tracker to ensure isolation when we want to make sure
      * that the memtable we're acting on doesn't change out from under us.  I.e., flush
      * syncronizes on it to make sure it can submit on both executors atomically,
      * so anyone else who wants to make sure flush doesn't interfere should as well.
      */
-    private final DataTracker data;
+    private final Tracker data;
 
     /* The read order, used to track accesses to off-heap memtable storage */
     public final OpOrder readOrdering = new OpOrder();
@@ -214,7 +224,7 @@
         int period = metadata.getMemtableFlushPeriod();
         if (period > 0)
         {
-            logger.debug("scheduling flush in {} ms", period);
+            logger.trace("scheduling flush in {} ms", period);
             WrappedRunnable runnable = new WrappedRunnable()
             {
                 protected void runMayThrow() throws Exception
@@ -340,13 +350,27 @@
         }
     }
 
-    private ColumnFamilyStore(Keyspace keyspace,
+    public ColumnFamilyStore(Keyspace keyspace,
+                             String columnFamilyName,
+                             IPartitioner partitioner,
+                             int generation,
+                             CFMetaData metadata,
+                             Directories directories,
+                             boolean loadSSTables)
+    {
+        this(keyspace, columnFamilyName, partitioner, generation, metadata, directories, loadSSTables, true);
+    }
+
+
+    @VisibleForTesting
+    public ColumnFamilyStore(Keyspace keyspace,
                               String columnFamilyName,
                               IPartitioner partitioner,
                               int generation,
                               CFMetaData metadata,
                               Directories directories,
-                              boolean loadSSTables)
+                              boolean loadSSTables,
+                              boolean registerBookkeeping)
     {
         assert metadata != null : "null metadata for " + keyspace + ":" + columnFamilyName;
 
@@ -360,14 +384,18 @@
         this.indexManager = new SecondaryIndexManager(this);
         this.metric = new ColumnFamilyMetrics(this);
         fileIndexGenerator.set(generation);
-        sampleLatencyNanos = DatabaseDescriptor.getReadRpcTimeout() / 2;
+        sampleLatencyNanos = TimeUnit.MILLISECONDS.toNanos(DatabaseDescriptor.getReadRpcTimeout() / 2);
 
         logger.info("Initializing {}.{}", keyspace.getName(), name);
 
-        // scan for sstables corresponding to this cf and load them
-        data = new DataTracker(this);
+        // Create Memtable only on online
+        Memtable initialMemtable = null;
+        if (DatabaseDescriptor.isDaemonInitialized())
+            initialMemtable = new Memtable(new AtomicReference<>(CommitLog.instance.getContext()), this);
+        data = new Tracker(initialMemtable, loadSSTables);
 
-        if (loadSSTables)
+        // scan for sstables corresponding to this cf and load them
+        if (data.loadsstables)
         {
             Directories.SSTableLister sstableFiles = directories.sstableLister().skipTemporary(true);
             Collection<SSTableReader> sstables = SSTableReader.openAll(sstableFiles.list().entrySet(), metadata, this.partitioner);
@@ -390,47 +418,59 @@
                 indexManager.addIndexedColumn(info);
         }
 
-        // register the mbean
-        String type = this.partitioner instanceof LocalPartitioner ? "IndexColumnFamilies" : "ColumnFamilies";
-        mbeanName = "org.apache.cassandra.db:type=" + type + ",keyspace=" + this.keyspace.getName() + ",columnfamily=" + name;
-        try
+        if (registerBookkeeping)
         {
-            MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
-            ObjectName nameObj = new ObjectName(mbeanName);
-            mbs.registerMBean(this, nameObj);
-        }
-        catch (Exception e)
-        {
-            throw new RuntimeException(e);
-        }
-        logger.debug("retryPolicy for {} is {}", name, this.metadata.getSpeculativeRetry());
-        latencyCalculator = ScheduledExecutors.optionalTasks.scheduleWithFixedDelay(new Runnable()
-        {
-            public void run()
+            // register the mbean
+            String type = this.partitioner instanceof LocalPartitioner ? "IndexColumnFamilies" : "ColumnFamilies";
+            mbeanName = "org.apache.cassandra.db:type=" + type + ",keyspace=" + this.keyspace.getName() + ",columnfamily=" + name;
+            try
             {
-                SpeculativeRetry retryPolicy = ColumnFamilyStore.this.metadata.getSpeculativeRetry();
-                switch (retryPolicy.type)
-                {
-                    case PERCENTILE:
-                        // get percentile in nanos
-                        assert metric.coordinatorReadLatency.durationUnit() == TimeUnit.MICROSECONDS;
-                        sampleLatencyNanos = (long) (metric.coordinatorReadLatency.getSnapshot().getValue(retryPolicy.value) * 1000d);
-                        break;
-                    case CUSTOM:
-                        // convert to nanos, since configuration is in millisecond
-                        sampleLatencyNanos = (long) (retryPolicy.value * 1000d * 1000d);
-                        break;
-                    default:
-                        sampleLatencyNanos = Long.MAX_VALUE;
-                        break;
-                }
+                MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
+                ObjectName nameObj = new ObjectName(mbeanName);
+                mbs.registerMBean(this, nameObj);
             }
-        }, DatabaseDescriptor.getReadRpcTimeout(), DatabaseDescriptor.getReadRpcTimeout(), TimeUnit.MILLISECONDS);
+            catch (Exception e)
+            {
+                throw new RuntimeException(e);
+            }
+            logger.trace("retryPolicy for {} is {}", name, this.metadata.getSpeculativeRetry());
+            latencyCalculator = ScheduledExecutors.optionalTasks.scheduleWithFixedDelay(new Runnable()
+            {
+                public void run()
+                {
+                    SpeculativeRetry retryPolicy = ColumnFamilyStore.this.metadata.getSpeculativeRetry();
+                    switch (retryPolicy.type)
+                    {
+                        case PERCENTILE:
+                            // get percentile in nanos
+                            sampleLatencyNanos = (long) (metric.coordinatorReadLatency.getSnapshot().getValue(retryPolicy.value));
+                            break;
+                        case CUSTOM:
+                            // convert to nanos, since configuration is in millisecond
+                            sampleLatencyNanos = (long) (retryPolicy.value * 1000d * 1000d);
+                            break;
+                        default:
+                            sampleLatencyNanos = Long.MAX_VALUE;
+                            break;
+                    }
+                }
+            }, DatabaseDescriptor.getReadRpcTimeout(), DatabaseDescriptor.getReadRpcTimeout(), TimeUnit.MILLISECONDS);
+        }
+        else
+        {
+            latencyCalculator = ScheduledExecutors.optionalTasks.schedule(Runnables.doNothing(), 0, TimeUnit.NANOSECONDS);
+            mbeanName = null;
+        }
     }
 
     /** call when dropping or renaming a CF. Performs mbean housekeeping and invalidates CFS to other operations */
     public void invalidate()
     {
+        invalidate(true);
+    }
+
+    public void invalidate(boolean expectMBean)
+    {
         // disable and cancel in-progress compactions before invalidating
         valid = false;
 
@@ -440,21 +480,24 @@
         }
         catch (Exception e)
         {
-            JVMStabilityInspector.inspectThrowable(e);
-            // this shouldn't block anything.
-            logger.warn("Failed unregistering mbean: {}", mbeanName, e);
+            if (expectMBean)
+            {
+                JVMStabilityInspector.inspectThrowable(e);
+                // this shouldn't block anything.
+                logger.warn("Failed unregistering mbean: {}", mbeanName, e);
+            }
         }
 
         latencyCalculator.cancel(false);
         SystemKeyspace.removeTruncationRecord(metadata.cfId);
-        data.unreferenceSSTables();
+        data.dropSSTables();
         indexManager.invalidate();
 
         invalidateCaches();
     }
 
     /**
-     * Removes every SSTable in the directory from the DataTracker's view.
+     * Removes every SSTable in the directory from the Tracker's view.
      * @param directory the unreadable directory, possibly with SSTables in it, but not necessarily.
      */
     void maybeRemoveUnreadableSSTables(File directory)
@@ -473,37 +516,13 @@
         metric.release();
     }
 
-    public long getMinRowSize()
+
+    public static ColumnFamilyStore createColumnFamilyStore(Keyspace keyspace, CFMetaData metadata, boolean loadSSTables)
     {
-        return metric.minRowSize.value();
+        return createColumnFamilyStore(keyspace, metadata.cfName, StorageService.getPartitioner(), metadata, loadSSTables);
     }
 
-    public long getMaxRowSize()
-    {
-        return metric.maxRowSize.value();
-    }
-
-    public long getMeanRowSize()
-    {
-        return metric.meanRowSize.value();
-    }
-
-    public int getMeanColumns()
-    {
-        return data.getMeanColumns();
-    }
-
-    public static ColumnFamilyStore createColumnFamilyStore(Keyspace keyspace, String columnFamily, boolean loadSSTables)
-    {
-        return createColumnFamilyStore(keyspace, columnFamily, StorageService.getPartitioner(), Schema.instance.getCFMetaData(keyspace.getName(), columnFamily), loadSSTables);
-    }
-
-    public static ColumnFamilyStore createColumnFamilyStore(Keyspace keyspace, String columnFamily, IPartitioner partitioner, CFMetaData metadata)
-    {
-        return createColumnFamilyStore(keyspace, columnFamily, partitioner, metadata, true);
-    }
-
-    private static synchronized ColumnFamilyStore createColumnFamilyStore(Keyspace keyspace,
+    public static synchronized ColumnFamilyStore createColumnFamilyStore(Keyspace keyspace,
                                                                          String columnFamily,
                                                                          IPartitioner partitioner,
                                                                          CFMetaData metadata,
@@ -519,7 +538,7 @@
             generations.add(desc.generation);
             if (!desc.isCompatible())
                 throw new RuntimeException(String.format("Incompatible SSTable found. Current version %s is unable to read file: %s. Please run upgradesstables.",
-                                                          Descriptor.Version.CURRENT, desc));
+                        desc.getFormat().getLatestVersion(), desc));
         }
         Collections.sort(generations);
         int value = (generations.size() > 0) ? (generations.get(generations.size() - 1)) : 0;
@@ -543,7 +562,7 @@
         {
             public boolean accept(File pathname)
             {
-                return pathname.toString().endsWith(StreamLockfile.FILE_EXT);
+                return pathname.getPath().endsWith(StreamLockfile.FILE_EXT);
             }
         };
         for (File dir : directories.getCFDirectories())
@@ -562,7 +581,7 @@
             }
         }
 
-        logger.debug("Removing compacted SSTable files from {} (see http://wiki.apache.org/cassandra/MemtableSSTable)", metadata.cfName);
+        logger.trace("Removing compacted SSTable files from {} (see http://wiki.apache.org/cassandra/MemtableSSTable)", metadata.cfName);
 
         for (Map.Entry<Descriptor,Set<Component>> sstableFiles : directories.sstableLister().list().entrySet())
         {
@@ -617,7 +636,7 @@
     }
 
     /**
-     * Replacing compacted sstables is atomic as far as observers of DataTracker are concerned, but not on the
+     * Replacing compacted sstables is atomic as far as observers of Tracker are concerned, but not on the
      * filesystem: first the new sstables are renamed to "live" status (i.e., the tmp marker is removed), then
      * their ancestors are removed.
      *
@@ -641,7 +660,7 @@
         {
             HashSet<Integer> missingGenerations = new HashSet<>(unfinishedGenerations);
             missingGenerations.removeAll(allGenerations);
-            logger.debug("Unfinished compactions of {}.{} reference missing sstables of generations {}",
+            logger.trace("Unfinished compactions of {}.{} reference missing sstables of generations {}",
                          metadata.ksName, metadata.cfName, missingGenerations);
         }
 
@@ -679,7 +698,7 @@
                 // any of the ancestors would work, so we'll just lookup the compaction task ID with the first one
                 UUID compactionTaskID = unfinishedCompactions.get(ancestors.iterator().next());
                 assert compactionTaskID != null;
-                logger.debug("Going to delete unfinished compaction product {}", desc);
+                logger.trace("Going to delete unfinished compaction product {}", desc);
                 SSTable.delete(desc, sstableFiles.getValue());
                 SystemKeyspace.finishCompaction(compactionTaskID);
             }
@@ -696,7 +715,7 @@
             if (completedAncestors.contains(desc.generation))
             {
                 // if any of the ancestors were participating in a compaction, finish that compaction
-                logger.debug("Going to delete leftover compaction ancestor {}", desc);
+                logger.trace("Going to delete leftover compaction ancestor {}", desc);
                 SSTable.delete(desc, sstableFiles.getValue());
                 UUID compactionTaskID = unfinishedCompactions.get(desc.generation);
                 if (compactionTaskID != null)
@@ -728,7 +747,7 @@
         Set<Descriptor> currentDescriptors = new HashSet<Descriptor>();
         for (SSTableReader sstable : data.getView().sstables)
             currentDescriptors.add(sstable.descriptor);
-        Set<SSTableReader> newSSTables = new HashSet<SSTableReader>();
+        Set<SSTableReader> newSSTables = new HashSet<>();
 
         Directories.SSTableLister lister = directories.sstableLister().skipTemporary(true);
         for (Map.Entry<Descriptor, Set<Component>> entry : lister.list().entrySet())
@@ -742,8 +761,8 @@
 
             if (!descriptor.isCompatible())
                 throw new RuntimeException(String.format("Can't open incompatible SSTable! Current version %s, found file: %s",
-                                                         Descriptor.Version.CURRENT,
-                                                         descriptor));
+                        descriptor.getFormat().getLatestVersion(),
+                        descriptor));
 
             // force foreign sstables to level 0
             try
@@ -767,7 +786,8 @@
                                                descriptor.ksname,
                                                descriptor.cfname,
                                                fileIndexGenerator.incrementAndGet(),
-                                               Descriptor.Type.FINAL);
+                                               Descriptor.Type.FINAL,
+                                               descriptor.formatType);
             }
             while (new File(newDescriptor.filenameFor(Component.DATA)).exists());
 
@@ -833,17 +853,23 @@
 
     public String getTempSSTablePath(File directory)
     {
-        return getTempSSTablePath(directory, Descriptor.Version.CURRENT);
+        return getTempSSTablePath(directory, DatabaseDescriptor.getSSTableFormat().info.getLatestVersion(), DatabaseDescriptor.getSSTableFormat());
     }
 
-    private String getTempSSTablePath(File directory, Descriptor.Version version)
+    public String getTempSSTablePath(File directory, SSTableFormat.Type format)
+    {
+        return getTempSSTablePath(directory, format.info.getLatestVersion(), format);
+    }
+
+    private String getTempSSTablePath(File directory, Version version, SSTableFormat.Type format)
     {
         Descriptor desc = new Descriptor(version,
                                          directory,
                                          keyspace.getName(),
                                          name,
                                          fileIndexGenerator.incrementAndGet(),
-                                         Descriptor.Type.TEMP);
+                                         Descriptor.Type.TEMP,
+                                         format);
         return desc.filenameFor(Component.DATA);
     }
 
@@ -852,24 +878,24 @@
      *
      * @param memtable
      */
-    public Future<?> switchMemtableIfCurrent(Memtable memtable)
+    public ListenableFuture<ReplayPosition> switchMemtableIfCurrent(Memtable memtable)
     {
         synchronized (data)
         {
             if (data.getView().getCurrentMemtable() == memtable)
                 return switchMemtable();
         }
-        return Futures.immediateFuture(null);
+        return waitForFlushes();
     }
 
     /*
      * switchMemtable puts Memtable.getSortedContents on the writer executor.  When the write is complete,
      * we turn the writer into an SSTableReader and add it to ssTables where it is available for reads.
-     * This method does not block except for synchronizing on DataTracker, but the Future it returns will
+     * This method does not block except for synchronizing on Tracker, but the Future it returns will
      * not complete until the Memtable (and all prior Memtables) have been successfully flushed, and the CL
      * marked clean up to the position owned by the Memtable.
      */
-    public ListenableFuture<?> switchMemtable()
+    public ListenableFuture<ReplayPosition> switchMemtable()
     {
         synchronized (data)
         {
@@ -879,13 +905,26 @@
                                                 previousFlushFailure);
             logFlush();
             Flush flush = new Flush(false);
-            ListenableFutureTask<?> flushTask = ListenableFutureTask.create(flush, null);
+            ListenableFutureTask<Void> flushTask = ListenableFutureTask.create(flush, null);
             flushExecutor.submit(flushTask);
-            ListenableFutureTask<?> task = ListenableFutureTask.create(flush.postFlush, null);
+            ListenableFutureTask<ReplayPosition> task = ListenableFutureTask.create(flush.postFlush);
             postFlushExecutor.submit(task);
 
             @SuppressWarnings("unchecked")
-            ListenableFuture<?> future = Futures.allAsList(flushTask, task);
+            ListenableFuture<ReplayPosition> future = 
+                    // If either of the two tasks errors out, resulting future must also error out.
+                    // Combine the two futures and only return post-flush result after both have completed.
+                    // Note that flushTask will always yield null, but Futures.allAsList is
+                    // order preserving, which is why the transform function returns the result
+                    // from item 1 in it's input list (i.e. what was yielded by task).
+                    Futures.transform(Futures.allAsList(flushTask, task),
+                                      new Function<List<Object>, ReplayPosition>()
+                                      {
+                                          public ReplayPosition apply(List<Object> input)
+                                          {
+                                              return (ReplayPosition) input.get(1);
+                                          }
+                                      });
             return future;
         }
     }
@@ -896,7 +935,7 @@
         // reclaiming includes that which we are GC-ing;
         float onHeapRatio = 0, offHeapRatio = 0;
         long onHeapTotal = 0, offHeapTotal = 0;
-        Memtable memtable = getDataTracker().getView().getCurrentMemtable();
+        Memtable memtable = getTracker().getView().getCurrentMemtable();
         onHeapRatio +=  memtable.getAllocator().onHeap().ownershipRatio();
         offHeapRatio += memtable.getAllocator().offHeap().ownershipRatio();
         onHeapTotal += memtable.getAllocator().onHeap().owns();
@@ -906,7 +945,7 @@
         {
             if (index.getIndexCfs() != null)
             {
-                MemtableAllocator allocator = index.getIndexCfs().getDataTracker().getView().getCurrentMemtable().getAllocator();
+                MemtableAllocator allocator = index.getIndexCfs().getTracker().getView().getCurrentMemtable().getAllocator();
                 onHeapRatio += allocator.onHeap().ownershipRatio();
                 offHeapRatio += allocator.offHeap().ownershipRatio();
                 onHeapTotal += allocator.onHeap().owns();
@@ -914,81 +953,100 @@
             }
         }
 
-        logger.info("Enqueuing flush of {}: {}", name, String.format("%d (%.0f%%) on-heap, %d (%.0f%%) off-heap",
+        logger.debug("Enqueuing flush of {}: {}", name, String.format("%d (%.0f%%) on-heap, %d (%.0f%%) off-heap",
                                                                      onHeapTotal, onHeapRatio * 100, offHeapTotal, offHeapRatio * 100));
     }
 
 
-    public ListenableFuture<?> forceFlush()
+    /**
+     * Flush if there is unflushed data in the memtables
+     *
+     * @return a Future yielding the commit log position that can be guaranteed to have been successfully written
+     *         to sstables for this table once the future completes
+     */
+    public ListenableFuture<ReplayPosition> forceFlush()
     {
-        return forceFlush(null);
+        synchronized (data)
+        {
+            Memtable current = data.getView().getCurrentMemtable();
+            for (ColumnFamilyStore cfs : concatWithIndexes())
+                if (!cfs.data.getView().getCurrentMemtable().isClean())
+                    return switchMemtableIfCurrent(current);
+            return waitForFlushes();
+        }
     }
 
     /**
      * Flush if there is unflushed data that was written to the CommitLog before @param flushIfDirtyBefore
-     * (inclusive).  If @param flushIfDirtyBefore is null, flush if there is any unflushed data.
+     * (inclusive).
      *
-     * @return a Future such that when the future completes, all data inserted before forceFlush was called,
-     * will be flushed.
+     * @return a Future yielding the commit log position that can be guaranteed to have been successfully written
+     *         to sstables for this table once the future completes
      */
-    public ListenableFuture<?> forceFlush(ReplayPosition flushIfDirtyBefore)
+    public ListenableFuture<ReplayPosition> forceFlush(ReplayPosition flushIfDirtyBefore)
     {
-        // we synchronize on the data tracker to ensure we don't race against other calls to switchMemtable(),
-        // unnecessarily queueing memtables that are about to be made clean
-        synchronized (data)
-        {
-            // during index build, 2ary index memtables can be dirty even if parent is not.  if so,
-            // we want to flush the 2ary index ones too.
-            boolean clean = true;
-            for (ColumnFamilyStore cfs : concatWithIndexes())
-                clean &= cfs.data.getView().getCurrentMemtable().isCleanAfter(flushIfDirtyBefore);
-
-            if (clean)
-            {
-                // We could have a memtable for this column family that is being
-                // flushed. Make sure the future returned wait for that so callers can
-                // assume that any data inserted prior to the call are fully flushed
-                // when the future returns (see #5241).
-                ListenableFutureTask<?> task = ListenableFutureTask.create(new Runnable()
-                {
-                    public void run()
-                    {
-                        logger.debug("forceFlush requested but everything is clean in {}", name);
-                    }
-                }, null);
-                postFlushExecutor.execute(task);
-                return task;
-            }
-
-            return switchMemtable();
-        }
+        // we don't loop through the remaining memtables since here we only care about commit log dirtiness
+        // and this does not vary between a table and its table-backed indexes
+        Memtable current = data.getView().getCurrentMemtable();
+        if (current.mayContainDataBefore(flushIfDirtyBefore))
+            return switchMemtableIfCurrent(current);
+        return waitForFlushes();
     }
 
-    public void forceBlockingFlush()
+    /**
+     * @return a Future yielding the commit log position that can be guaranteed to have been successfully written
+     *         to sstables for this table once the future completes
+     */
+    private ListenableFuture<ReplayPosition> waitForFlushes()
     {
-        FBUtilities.waitOnFuture(forceFlush());
+        // we grab the current memtable; once any preceding memtables have flushed, we know its
+        // commitLogLowerBound has been set (as this it is set with the upper bound of the preceding memtable)
+        final Memtable current = data.getView().getCurrentMemtable();
+        ListenableFutureTask<ReplayPosition> task = ListenableFutureTask.create(new Callable<ReplayPosition>()
+        {
+            public ReplayPosition call()
+            {
+                logger.debug("forceFlush requested but everything is clean in {}", name);
+                return current.getCommitLogLowerBound();
+            }
+        });
+        postFlushExecutor.execute(task);
+        return task;
+    }
+
+    public ReplayPosition forceBlockingFlush()
+    {
+        return FBUtilities.waitOnFuture(forceFlush());
     }
 
     /**
      * Both synchronises custom secondary indexes and provides ordering guarantees for futures on switchMemtable/flush
      * etc, which expect to be able to wait until the flush (and all prior flushes) requested have completed.
      */
-    private final class PostFlush implements Runnable
+    private final class PostFlush implements Callable<ReplayPosition>
     {
         final boolean flushSecondaryIndexes;
         final OpOrder.Barrier writeBarrier;
         final CountDownLatch latch = new CountDownLatch(1);
-        final ReplayPosition lastReplayPosition;
+        final ReplayPosition commitLogUpperBound;
+        final List<Memtable> memtables;
+        final List<SSTableReader> readers;
 
-        private PostFlush(boolean flushSecondaryIndexes, OpOrder.Barrier writeBarrier, ReplayPosition lastReplayPosition)
+        private PostFlush(boolean flushSecondaryIndexes, OpOrder.Barrier writeBarrier, ReplayPosition commitLogUpperBound,
+                          List<Memtable> memtables, List<SSTableReader> readers)
         {
             this.writeBarrier = writeBarrier;
             this.flushSecondaryIndexes = flushSecondaryIndexes;
-            this.lastReplayPosition = lastReplayPosition;
+            this.commitLogUpperBound = commitLogUpperBound;
+            this.memtables = memtables;
+            this.readers = readers;
         }
 
-        public void run()
+        public ReplayPosition call()
         {
+            if (discardFlushResults == ColumnFamilyStore.this)
+                return commitLogUpperBound;
+
             writeBarrier.await();
 
             /**
@@ -1010,7 +1068,7 @@
 
             try
             {
-                // we wait on the latch for the lastReplayPosition to be set, and so that waiters
+                // we wait on the latch for the commitLogUpperBound to be set, and so that waiters
                 // on this task can rely on all prior flushes being complete
                 latch.await();
             }
@@ -1019,19 +1077,22 @@
                 throw new IllegalStateException();
             }
 
-            // must check lastReplayPosition != null because Flush may find that all memtables are clean
-            // and so not set a lastReplayPosition
-            if (lastReplayPosition != null)
+            CommitLog.instance.discardCompletedSegments(metadata.cfId, commitLogUpperBound);
+            for (int i = 0 ; i < memtables.size() ; i++)
             {
-                CommitLog.instance.discardCompletedSegments(metadata.cfId, lastReplayPosition);
+                Memtable memtable = memtables.get(i);
+                SSTableReader reader = readers.get(i);
+                memtable.cfs.data.permitCompactionOfFlushed(reader);
+                memtable.cfs.compactionStrategyWrapper.replaceFlushed(memtable, reader);
             }
-
             metric.pendingFlushes.dec();
+
+            return commitLogUpperBound;
         }
     }
 
     /**
-     * Should only be constructed/used from switchMemtable() or truncate(), with ownership of the DataTracker monitor.
+     * Should only be constructed/used from switchMemtable() or truncate(), with ownership of the Tracker monitor.
      * In the constructor the current memtable(s) are swapped, and a barrier on outstanding writes is issued;
      * when run by the flushWriter the barrier is waited on to ensure all outstanding writes have completed
      * before all memtables are immediately written, and the CL is either immediately marked clean or, if
@@ -1041,7 +1102,8 @@
     private final class Flush implements Runnable
     {
         final OpOrder.Barrier writeBarrier;
-        final List<Memtable> memtables;
+        final List<Memtable> memtables = new ArrayList<>();
+        final List<SSTableReader> readers = new ArrayList<>();
         final PostFlush postFlush;
         final boolean truncate;
 
@@ -1057,43 +1119,33 @@
              * that all write operations register themselves with, and assigning this barrier to the memtables,
              * after which we *.issue()* the barrier. This barrier is used to direct write operations started prior
              * to the barrier.issue() into the memtable we have switched out, and any started after to its replacement.
-             * In doing so it also tells the write operations to update the lastReplayPosition of the memtable, so
+             * In doing so it also tells the write operations to update the commitLogUpperBound of the memtable, so
              * that we know the CL position we are dirty to, which can be marked clean when we complete.
              */
             writeBarrier = keyspace.writeOrder.newBarrier();
-            memtables = new ArrayList<>();
 
             // submit flushes for the memtable for any indexed sub-cfses, and our own
-            AtomicReference<ReplayPosition> lastReplayPositionHolder = new AtomicReference<>();
+            AtomicReference<ReplayPosition> commitLogUpperBound = new AtomicReference<>();
             for (ColumnFamilyStore cfs : concatWithIndexes())
             {
                 // switch all memtables, regardless of their dirty status, setting the barrier
                 // so that we can reach a coordinated decision about cleanliness once they
                 // are no longer possible to be modified
-                Memtable mt = cfs.data.switchMemtable(truncate);
-                mt.setDiscarding(writeBarrier, lastReplayPositionHolder);
-                memtables.add(mt);
+                Memtable newMemtable = new Memtable(commitLogUpperBound, cfs);
+                Memtable oldMemtable = cfs.data.switchMemtable(truncate, newMemtable);
+                oldMemtable.setDiscarding(writeBarrier, commitLogUpperBound);
+                memtables.add(oldMemtable);
             }
 
-            // we now attempt to define the lastReplayPosition; we do this by grabbing the current limit from the CL
-            // and attempting to set the holder to this value. at the same time all writes to the memtables are
-            // also maintaining this value, so if somebody sneaks ahead of us somehow (should be rare) we simply retry,
-            // so that we know all operations prior to the position have not reached it yet
-            ReplayPosition lastReplayPosition;
-            while (true)
-            {
-                lastReplayPosition = new Memtable.LastReplayPosition(CommitLog.instance.getContext());
-                ReplayPosition currentLast = lastReplayPositionHolder.get();
-                if ((currentLast == null || currentLast.compareTo(lastReplayPosition) <= 0)
-                    && lastReplayPositionHolder.compareAndSet(currentLast, lastReplayPosition))
-                    break;
-            }
+            // we then ensure an atomic decision is made about the upper bound of the continuous range of commit log
+            // records owned by this memtable
+            setCommitLogUpperBound(commitLogUpperBound);
 
             // we then issue the barrier; this lets us wait for all operations started prior to the barrier to complete;
-            // since this happens after wiring up the lastReplayPosition, we also know all operations with earlier
+            // since this happens after wiring up the commitLogUpperBound, we also know all operations with earlier
             // replay positions have also completed, i.e. the memtables are done and ready to flush
             writeBarrier.issue();
-            postFlush = new PostFlush(!truncate, writeBarrier, lastReplayPosition);
+            postFlush = new PostFlush(!truncate, writeBarrier, commitLogUpperBound.get(), memtables, readers);
         }
 
         public void run()
@@ -1112,7 +1164,8 @@
                 memtable.cfs.data.markFlushing(memtable);
                 if (memtable.isClean() || truncate)
                 {
-                    memtable.cfs.replaceFlushed(memtable, null);
+                    memtable.cfs.data.replaceFlushed(memtable, null);
+                    memtable.cfs.compactionStrategyWrapper.replaceFlushed(memtable, null);
                     reclaim(memtable);
                     iter.remove();
                 }
@@ -1131,8 +1184,10 @@
                 for (Memtable memtable : memtables)
                 {
                     // flush the memtable
-                    MoreExecutors.sameThreadExecutor().execute(memtable.flushRunnable());
+                    SSTableReader reader = memtable.flush();
+                    memtable.cfs.data.replaceFlushed(memtable, reader);
                     reclaim(memtable);
+                    readers.add(reader);
                 }
 
                 // signal the post-flush we've done our work
@@ -1174,6 +1229,38 @@
         }
     }
 
+    // atomically set the upper bound for the commit log
+    private static void setCommitLogUpperBound(AtomicReference<ReplayPosition> commitLogUpperBound)
+    {
+        // we attempt to set the holder to the current commit log context. at the same time all writes to the memtables are
+        // also maintaining this value, so if somebody sneaks ahead of us somehow (should be rare) we simply retry,
+        // so that we know all operations prior to the position have not reached it yet
+        ReplayPosition lastReplayPosition;
+        while (true)
+        {
+            lastReplayPosition = new Memtable.LastReplayPosition(CommitLog.instance.getContext());
+            ReplayPosition currentLast = commitLogUpperBound.get();
+            if ((currentLast == null || currentLast.compareTo(lastReplayPosition) <= 0)
+                && commitLogUpperBound.compareAndSet(currentLast, lastReplayPosition))
+                break;
+        }
+    }
+
+    @VisibleForTesting
+    // this method should ONLY be used for testing commit log behaviour; it discards the current memtable
+    // contents without marking the commit log clean, and prevents any proceeding flushes from marking
+    // the commit log as done, however they *will* terminate (unlike under typical failures) to ensure progress is made
+    public void simulateFailedFlush()
+    {
+        discardFlushResults = this;
+        data.markFlushing(data.switchMemtable(false, new Memtable(new AtomicReference<>(CommitLog.instance.getContext()), this)));
+    }
+
+    public void resumeFlushing()
+    {
+        discardFlushResults = null;
+    }
+
     /**
      * Finds the largest memtable, as a percentage of *either* on- or off-heap memory limits, and immediately
      * queues it for flushing. If the memtable selected is flushed before this completes, no work is done.
@@ -1190,7 +1277,7 @@
                 // we take a reference to the current main memtable for the CF prior to snapping its ownership ratios
                 // to ensure we have some ordering guarantee for performing the switchMemtableIf(), i.e. we will only
                 // swap if the memtables we are measuring here haven't already been swapped by the time we try to swap them
-                Memtable current = cfs.getDataTracker().getView().getCurrentMemtable();
+                Memtable current = cfs.getTracker().getView().getCurrentMemtable();
 
                 // find the total ownership ratio for the memtable and all SecondaryIndexes owned by this CF,
                 // both on- and off-heap, and select the largest of the two ratios to weight this CF
@@ -1202,7 +1289,7 @@
                 {
                     if (index.getIndexCfs() != null)
                     {
-                        MemtableAllocator allocator = index.getIndexCfs().getDataTracker().getView().getCurrentMemtable().getAllocator();
+                        MemtableAllocator allocator = index.getIndexCfs().getTracker().getView().getCurrentMemtable().getAllocator();
                         onHeap += allocator.onHeap().ownershipRatio();
                         offHeap += allocator.offHeap().ownershipRatio();
                     }
@@ -1227,7 +1314,7 @@
                 float flushingOffHeap = Memtable.MEMORY_POOL.offHeap.reclaimingRatio();
                 float thisOnHeap = largest.getAllocator().onHeap().ownershipRatio();
                 float thisOffHeap = largest.getAllocator().onHeap().ownershipRatio();
-                logger.info("Flushing largest {} to free up room. Used total: {}, live: {}, flushing: {}, this: {}",
+                logger.debug("Flushing largest {} to free up room. Used total: {}, live: {}, flushing: {}, this: {}",
                             largest.cfs, ratio(usedOnHeap, usedOffHeap), ratio(liveOnHeap, liveOffHeap),
                             ratio(flushingOnHeap, flushingOffHeap), ratio(thisOnHeap, thisOffHeap));
                 largest.cfs.switchMemtableIfCurrent(largest);
@@ -1264,8 +1351,13 @@
         maybeUpdateRowCache(key);
         metric.samplers.get(Sampler.WRITES).addSample(key.getKey(), key.hashCode(), 1);
         metric.writeLatency.addNano(System.nanoTime() - start);
+        // CASSANDRA-11117 - certain resolution paths on memtable put can result in very
+        // large time deltas, either through a variety of sentinel timestamps (used for empty values, ensuring
+        // a minimal write, etc). This limits the time delta to the max value the histogram
+        // can bucket correctly. This also filters the Long.MAX_VALUE case where there was no previous value
+        // to update.
         if(timeDelta < Long.MAX_VALUE)
-            metric.colUpdateTimeDeltaHistogram.update(timeDelta);
+            metric.colUpdateTimeDeltaHistogram.update(Math.min(18165375903306L, timeDelta));
     }
 
     /**
@@ -1362,13 +1454,15 @@
      */
     public Collection<SSTableReader> getOverlappingSSTables(Iterable<SSTableReader> sstables)
     {
-        logger.debug("Checking for sstables overlapping {}", sstables);
+        logger.trace("Checking for sstables overlapping {}", sstables);
 
         // a normal compaction won't ever have an empty sstables list, but we create a skeleton
         // compaction controller for streaming, and that passes an empty list.
         if (!sstables.iterator().hasNext())
             return ImmutableSet.of();
 
+
+
         List<SSTableReader> sortedByFirst = Lists.newArrayList(sstables);
         Collections.sort(sortedByFirst, new Comparator<SSTableReader>()
         {
@@ -1412,7 +1506,7 @@
             }
         }
         intervals.add(Interval.<RowPosition, SSTableReader>create(first, last));
-        DataTracker.SSTableIntervalTree tree = data.getView().intervalTree;
+        SSTableIntervalTree tree = data.getView().intervalTree;
         Set<SSTableReader> results = new HashSet<>();
 
         for (Interval<RowPosition, SSTableReader> interval : intervals)
@@ -1486,7 +1580,7 @@
                 expectedFileSize += position.right - position.left;
         }
 
-        double compressionRatio = getCompressionRatio();
+        double compressionRatio = metric.compressionRatio.getValue();
         if (compressionRatio > 0d)
             expectedFileSize *= compressionRatio;
 
@@ -1518,10 +1612,62 @@
 
     public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs) throws ExecutionException, InterruptedException
     {
+        return scrub(disableSnapshot, skipCorrupted, false, checkData, jobs);
+    }
+
+    @VisibleForTesting
+    public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean alwaysFail, boolean checkData, int jobs) throws ExecutionException, InterruptedException
+    {
         // skip snapshot creation during scrub, SEE JIRA 5891
         if(!disableSnapshot)
             snapshotWithoutFlush("pre-scrub-" + System.currentTimeMillis());
-        return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, jobs);
+
+        try
+        {
+            return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, jobs);
+        }
+        catch(Throwable t)
+        {
+            if (!rebuildOnFailedScrub(t))
+                throw t;
+
+            return alwaysFail ? CompactionManager.AllSSTableOpStatus.ABORTED : CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
+        }
+    }
+
+    /**
+     * CASSANDRA-5174 : For an index cfs we may be able to discard everything and just rebuild
+     * the index when a scrub fails.
+     *
+     * @return true if we are an index cfs and we successfully rebuilt the index
+     */
+    public boolean rebuildOnFailedScrub(Throwable failure)
+    {
+        if (!isIndex())
+            return false;
+
+        SecondaryIndex index = null;
+        if (metadata.cfName.contains(Directories.SECONDARY_INDEX_NAME_SEPARATOR))
+        {
+            String[] parts = metadata.cfName.split("\\" + Directories.SECONDARY_INDEX_NAME_SEPARATOR, 2);
+            ColumnFamilyStore parentCfs = keyspace.getColumnFamilyStore(parts[0]);
+            index = parentCfs.indexManager.getIndexByName(metadata.cfName);
+            assert index != null;
+        }
+
+        if (index == null)
+            return false;
+
+        truncateBlocking();
+
+        logger.warn("Rebuilding index for {} because of <{}>", name, failure.getMessage());
+        index.getBaseCfs().rebuildSecondaryIndex(index.getIndexName());
+        return true;
+    }
+
+    public CompactionManager.AllSSTableOpStatus verify(boolean extendedVerify) throws ExecutionException, InterruptedException
+    {
+        return CompactionManager.instance.performVerify(ColumnFamilyStore.this, extendedVerify);
     }
 
     public CompactionManager.AllSSTableOpStatus sstablesRewrite(boolean excludeCurrentVersion, int jobs) throws ExecutionException, InterruptedException
@@ -1532,12 +1678,7 @@
     public void markObsolete(Collection<SSTableReader> sstables, OperationType compactionType)
     {
         assert !sstables.isEmpty();
-        data.markObsolete(sstables, compactionType);
-    }
-
-    void replaceFlushed(Memtable memtable, SSTableReader sstable)
-    {
-        compactionStrategyWrapper.replaceFlushed(memtable, sstable);
+        maybeFail(data.dropSSTables(Predicates.in(sstables), compactionType, null));
     }
 
     public boolean isValid()
@@ -1545,25 +1686,13 @@
         return valid;
     }
 
-    public long getMemtableColumnsCount()
-    {
-        return metric.memtableColumnsCount.value();
-    }
 
-    public long getMemtableDataSize()
-    {
-        return metric.memtableOnHeapSize.value();
-    }
 
-    public int getMemtableSwitchCount()
-    {
-        return (int) metric.memtableSwitchCount.count();
-    }
 
     /**
      * Package protected for access from the CompactionManager.
      */
-    public DataTracker getDataTracker()
+    public Tracker getTracker()
     {
         return data;
     }
@@ -1573,99 +1702,14 @@
         return data.getSSTables();
     }
 
+    public Iterable<SSTableReader> getPermittedToCompactSSTables()
+    {
+        return data.getPermittedToCompact();
+    }
+
     public Set<SSTableReader> getUncompactingSSTables()
     {
-        return data.getUncompactingSSTables();
-    }
-
-    public long[] getRecentSSTablesPerReadHistogram()
-    {
-        return metric.recentSSTablesPerRead.getBuckets(true);
-    }
-
-    public long[] getSSTablesPerReadHistogram()
-    {
-        return metric.sstablesPerRead.getBuckets(false);
-    }
-
-    public long getReadCount()
-    {
-        return metric.readLatency.latency.count();
-    }
-
-    public double getRecentReadLatencyMicros()
-    {
-        return metric.readLatency.getRecentLatency();
-    }
-
-    public long[] getLifetimeReadLatencyHistogramMicros()
-    {
-        return metric.readLatency.totalLatencyHistogram.getBuckets(false);
-    }
-
-    public long[] getRecentReadLatencyHistogramMicros()
-    {
-        return metric.readLatency.recentLatencyHistogram.getBuckets(true);
-    }
-
-    public long getTotalReadLatencyMicros()
-    {
-        return metric.readLatency.totalLatency.count();
-    }
-
-    public int getPendingTasks()
-    {
-        return (int) metric.pendingFlushes.count();
-    }
-
-    public long getWriteCount()
-    {
-        return metric.writeLatency.latency.count();
-    }
-
-    public long getTotalWriteLatencyMicros()
-    {
-        return metric.writeLatency.totalLatency.count();
-    }
-
-    public double getRecentWriteLatencyMicros()
-    {
-        return metric.writeLatency.getRecentLatency();
-    }
-
-    public long[] getLifetimeWriteLatencyHistogramMicros()
-    {
-        return metric.writeLatency.totalLatencyHistogram.getBuckets(false);
-    }
-
-    public long[] getRecentWriteLatencyHistogramMicros()
-    {
-        return metric.writeLatency.recentLatencyHistogram.getBuckets(true);
-    }
-
-    public long getRangeCount()
-    {
-        return metric.rangeLatency.latency.count();
-    }
-
-    public double getRecentRangeLatencyMicros()
-    {
-        return metric.rangeLatency.getRecentLatency();
-    }
-
-    public long[] getLifetimeRangeLatencyHistogramMicros()
-    {
-        return metric.rangeLatency.totalLatencyHistogram.getBuckets(false);
-    }
-
-    public long[] getRecentRangeLatencyHistogramMicros()
-    {
-        return metric.rangeLatency.recentLatencyHistogram.getBuckets(true);
-    }
-
-    public long getTotalRangeLatencyMicros()
-    {
-        return metric.rangeLatency.totalLatency.count();
+        return data.getUncompacting();
     }
 
     public ColumnFamily getColumnFamily(DecoratedKey key,
@@ -1693,7 +1737,7 @@
     private ColumnFamily getThroughCache(UUID cfId, QueryFilter filter)
     {
         assert isRowCacheEnabled()
-               : String.format("Row cache is not enabled on column family [" + name + "]");
+               : String.format("Row cache is not enabled on table [" + name + "]");
 
         RowCacheKey key = new RowCacheKey(metadata.ksAndCFName, filter.key);
 
@@ -1941,7 +1985,8 @@
         return repairedSSTables;
     }
 
-    public RefViewFragment selectAndReference(Function<DataTracker.View, List<SSTableReader>> filter)
+    @SuppressWarnings("resource")
+    public RefViewFragment selectAndReference(Function<View, List<SSTableReader>> filter)
     {
         long failingSince = -1L;
         while (true)
@@ -1967,9 +2012,9 @@
         }
     }
 
-    public ViewFragment select(Function<DataTracker.View, List<SSTableReader>> filter)
+    public ViewFragment select(Function<View, List<SSTableReader>> filter)
     {
-        DataTracker.View view = data.getView();
+        View view = data.getView();
         List<SSTableReader> sstables = view.intervalTree.isEmpty()
                                        ? Collections.<SSTableReader>emptyList()
                                        : filter.apply(view);
@@ -1981,12 +2026,12 @@
      * @return a ViewFragment containing the sstables and memtables that may need to be merged
      * for the given @param key, according to the interval tree
      */
-    public Function<DataTracker.View, List<SSTableReader>> viewFilter(final DecoratedKey key)
+    public Function<View, List<SSTableReader>> viewFilter(final DecoratedKey key)
     {
-        assert !key.isMinimum(partitioner);
-        return new Function<DataTracker.View, List<SSTableReader>>()
+        assert !key.isMinimum();
+        return new Function<View, List<SSTableReader>>()
         {
-            public List<SSTableReader> apply(DataTracker.View view)
+            public List<SSTableReader> apply(View view)
             {
                 return compactionStrategyWrapper.filterSSTablesForReads(view.intervalTree.search(key));
             }
@@ -1997,13 +2042,51 @@
      * @return a ViewFragment containing the sstables and memtables that may need to be merged
      * for rows within @param rowBounds, inclusive, according to the interval tree.
      */
-    public Function<DataTracker.View, List<SSTableReader>> viewFilter(final AbstractBounds<RowPosition> rowBounds)
+    public Function<View, List<SSTableReader>> viewFilter(final AbstractBounds<RowPosition> rowBounds)
     {
-        return new Function<DataTracker.View, List<SSTableReader>>()
+        assert !AbstractBounds.strictlyWrapsAround(rowBounds.left, rowBounds.right);
+        return new Function<View, List<SSTableReader>>()
         {
-            public List<SSTableReader> apply(DataTracker.View view)
+            public List<SSTableReader> apply(View view)
             {
-                return compactionStrategyWrapper.filterSSTablesForReads(view.sstablesInBounds(rowBounds));
+                // Note that View.sstablesInBounds always includes it's bound while rowBounds may not. This is ok however
+                // because the fact we restrict the sstables returned by this function is an optimization in the first
+                // place and the returned sstables will (almost) never cover *exactly* rowBounds anyway. It's also
+                // *very* unlikely that a sstable is included *just* because we consider one of the bound inclusively
+                // instead of exclusively, so the performance impact is negligible in practice.
+                return view.sstablesInBounds(rowBounds.left, rowBounds.right);
+            }
+        };
+    }
+
+    /**
+     * @return a ViewFragment containing the sstables and memtables that may need to be merged
+     * for rows for all of @param rowBoundsCollection, inclusive, according to the interval tree.
+     */
+    public Function<View, List<SSTableReader>> viewFilter(final Collection<AbstractBounds<RowPosition>> rowBoundsCollection, final boolean includeRepaired)
+    {
+        assert AbstractBounds.noneStrictlyWrapsAround(rowBoundsCollection);
+        return new Function<View, List<SSTableReader>>()
+        {
+            public List<SSTableReader> apply(View view)
+            {
+                Set<SSTableReader> sstables = Sets.newHashSet();
+                for (AbstractBounds<RowPosition> rowBounds : rowBoundsCollection)
+                {
+                    // Note that View.sstablesInBounds always includes it's bound while rowBounds may not. This is ok however
+                    // because the fact we restrict the sstables returned by this function is an optimization in the first
+                    // place and the returned sstables will (almost) never cover *exactly* rowBounds anyway. It's also
+                    // *very* unlikely that a sstable is included *just* because we consider one of the bound inclusively
+                    // instead of exclusively, so the performance impact is negligible in practice.
+                    for (SSTableReader sstable : view.sstablesInBounds(rowBounds.left, rowBounds.right))
+                    {
+                        if (includeRepaired || !sstable.isRepaired())
+                            sstables.add(sstable);
+                    }
+                }
+
+                logger.trace("ViewFilter for {}/{} sstables", sstables.size(), getSSTables().size());
+                return ImmutableList.copyOf(sstables);
             }
         };
     }
@@ -2066,8 +2149,10 @@
     {
         Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(keyspace.getName());
 
-        for (RowCacheKey key : CacheService.instance.rowCache.getKeySet())
+        for (Iterator<RowCacheKey> keyIter = CacheService.instance.rowCache.keyIterator();
+             keyIter.hasNext(); )
         {
+            RowCacheKey key = keyIter.next();
             DecoratedKey dk = partitioner.decorateKey(ByteBuffer.wrap(key.key));
             if (key.ksAndCFName.equals(metadata.ksAndCFName) && !Range.isInRanges(dk.getToken(), ranges))
                 invalidateCachedRow(dk);
@@ -2075,8 +2160,10 @@
 
         if (metadata.isCounter())
         {
-            for (CounterCacheKey key : CacheService.instance.counterCache.getKeySet())
+            for (Iterator<CounterCacheKey> keyIter = CacheService.instance.counterCache.keyIterator();
+                 keyIter.hasNext(); )
             {
+                CounterCacheKey key = keyIter.next();
                 DecoratedKey dk = partitioner.decorateKey(ByteBuffer.wrap(key.partitionKey));
                 if (key.ksAndCFName.equals(metadata.ksAndCFName) && !Range.isInRanges(dk.getToken(), ranges))
                     CacheService.instance.counterCache.remove(key);
@@ -2097,9 +2184,10 @@
       *
       * @param range The range of keys and columns within those keys to fetch
      */
+    @SuppressWarnings("resource")
     private AbstractScanIterator getSequentialIterator(final DataRange range, long now)
     {
-        assert !(range.keyRange() instanceof Range) || !((Range)range.keyRange()).isWrapAround() || range.keyRange().right.isMinimum(partitioner) : range.keyRange();
+        assert !(range.keyRange() instanceof Range) || !((Range<?>)range.keyRange()).isWrapAround() || range.keyRange().right.isMinimum() : range.keyRange();
 
         final ViewFragment view = select(viewFilter(range.keyRange()));
         Tracing.trace("Executing seq scan across {} sstables for {}", view.sstables.size(), range.keyRange().getString(metadata.getKeyValidator()));
@@ -2111,24 +2199,27 @@
         {
             protected Row computeNext()
             {
-                // pull a row out of the iterator
-                if (!iterator.hasNext())
-                    return endOfData();
+                while (true)
+                {
+                    // pull a row out of the iterator
+                    if (!iterator.hasNext())
+                        return endOfData();
 
-                Row current = iterator.next();
-                DecoratedKey key = current.key;
+                    Row current = iterator.next();
+                    DecoratedKey key = current.key;
 
-                if (!range.stopKey().isMinimum(partitioner) && range.stopKey().compareTo(key) < 0)
-                    return endOfData();
+                    if (!range.stopKey().isMinimum() && range.stopKey().compareTo(key) < 0)
+                        return endOfData();
 
-                // skipping outside of assigned range
-                if (!range.contains(key))
-                    return computeNext();
+                    // skipping outside of assigned range
+                    if (!range.contains(key))
+                        continue;
 
-                if (logger.isTraceEnabled())
-                    logger.trace("scanned {}", metadata.getKeyValidator().getString(key.getKey()));
+                    if (logger.isTraceEnabled())
+                        logger.trace("scanned {}", metadata.getKeyValidator().getString(key.getKey()));
 
-                return current;
+                    return current;
+                }
             }
 
             public void close() throws IOException
@@ -2333,8 +2424,9 @@
     /**
      * @param ephemeral If this flag is set to true, the snapshot will be cleaned during next startup
      */
-    public void snapshotWithoutFlush(String snapshotName, Predicate<SSTableReader> predicate, boolean ephemeral)
+    public Set<SSTableReader> snapshotWithoutFlush(String snapshotName, Predicate<SSTableReader> predicate, boolean ephemeral)
     {
+        Set<SSTableReader> snapshottedSSTables = new HashSet<>();
         for (ColumnFamilyStore cfs : concatWithIndexes())
         {
             final JSONArray filesJSONArr = new JSONArray();
@@ -2349,8 +2441,9 @@
                     ssTable.createLinks(snapshotDirectory.getPath()); // hard links
                     filesJSONArr.add(ssTable.descriptor.relativeFilenameFor(Component.DATA));
 
-                    if (logger.isDebugEnabled())
-                        logger.debug("Snapshot for {} keyspace data file {} created in {}", keyspace, ssTable.getFilename(), snapshotDirectory);
+                    if (logger.isTraceEnabled())
+                        logger.trace("Snapshot for {} keyspace data file {} created in {}", keyspace, ssTable.getFilename(), snapshotDirectory);
+                    snapshottedSSTables.add(ssTable);
                 }
 
                 writeSnapshotManifest(filesJSONArr, snapshotName);
@@ -2358,21 +2451,24 @@
         }
         if (ephemeral)
             createEphemeralSnapshotMarkerFile(snapshotName);
+        return snapshottedSSTables;
     }
 
     private void writeSnapshotManifest(final JSONArray filesJSONArr, final String snapshotName)
     {
         final File manifestFile = directories.getSnapshotManifestFile(snapshotName);
-        final JSONObject manifestJSON = new JSONObject();
-        manifestJSON.put("files", filesJSONArr);
 
         try
         {
             if (!manifestFile.getParentFile().exists())
                 manifestFile.getParentFile().mkdirs();
-            PrintStream out = new PrintStream(manifestFile);
-            out.println(manifestJSON.toJSONString());
-            out.close();
+
+            try (PrintStream out = new PrintStream(manifestFile))
+            {
+                final JSONObject manifestJSON = new JSONObject();
+                manifestJSON.put("files", filesJSONArr);
+                out.println(manifestJSON.toJSONString());
+            }
         }
         catch (IOException e)
         {
@@ -2390,7 +2486,7 @@
                 ephemeralSnapshotMarker.getParentFile().mkdirs();
 
             Files.createFile(ephemeralSnapshotMarker.toPath());
-            logger.debug("Created ephemeral snapshot marker file on {}.", ephemeralSnapshotMarker.getAbsolutePath());
+            logger.trace("Created ephemeral snapshot marker file on {}.", ephemeralSnapshotMarker.getAbsolutePath());
         }
         catch (IOException e)
         {
@@ -2405,7 +2501,7 @@
     {
         for (String ephemeralSnapshot : directories.listEphemeralSnapshots())
         {
-            logger.debug("Clearing ephemeral snapshot {} leftover from previous session.", ephemeralSnapshot);
+            logger.trace("Clearing ephemeral snapshot {} leftover from previous session.", ephemeralSnapshot);
             Directories.clearSnapshot(ephemeralSnapshot, directories.getCFDirectories());
         }
     }
@@ -2426,17 +2522,17 @@
                 SSTableReader sstable = active.get(entries.getKey().generation);
                 if (sstable == null || !refs.tryRef(sstable))
                 {
-                    if (logger.isDebugEnabled())
-                        logger.debug("using snapshot sstable {}", entries.getKey());
+                    if (logger.isTraceEnabled())
+                        logger.trace("using snapshot sstable {}", entries.getKey());
                     // open without tracking hotness
                     sstable = SSTableReader.open(entries.getKey(), entries.getValue(), metadata, partitioner, true, false);
                     refs.tryRef(sstable);
                     // release the self ref as we never add the snapshot sstable to DataTracker where it is otherwise released
                     sstable.selfRef().release();
                 }
-                else if (logger.isDebugEnabled())
+                else if (logger.isTraceEnabled())
                 {
-                    logger.debug("using active sstable {}", entries.getKey());
+                    logger.trace("using active sstable {}", entries.getKey());
                 }
             }
         }
@@ -2455,18 +2551,19 @@
      *
      * @param snapshotName the name of the associated with the snapshot
      */
-    public void snapshot(String snapshotName)
+    public Set<SSTableReader> snapshot(String snapshotName)
     {
-        snapshot(snapshotName, null, false);
+        return snapshot(snapshotName, null, false);
     }
 
+
     /**
      * @param ephemeral If this flag is set to true, the snapshot will be cleaned up during next startup
      */
-    public void snapshot(String snapshotName, Predicate<SSTableReader> predicate, boolean ephemeral)
+    public Set<SSTableReader> snapshot(String snapshotName, Predicate<SSTableReader> predicate, boolean ephemeral)
     {
         forceBlockingFlush();
-        snapshotWithoutFlush(snapshotName, predicate, ephemeral);
+        return snapshotWithoutFlush(snapshotName, predicate, ephemeral);
     }
 
     public boolean snapshotExists(String snapshotName)
@@ -2500,19 +2597,9 @@
         return directories.getSnapshotDetails();
     }
 
-    public long getTotalDiskSpaceUsed()
+    public boolean hasUnreclaimedSpace()
     {
-        return metric.totalDiskSpaceUsed.count();
-    }
-
-    public long getLiveDiskSpaceUsed()
-    {
-        return metric.liveDiskSpaceUsed.count();
-    }
-
-    public int getLiveSSTableCount()
-    {
-        return metric.liveSSTableCount.value();
+        return metric.liveDiskSpaceUsed.getCount() < metric.totalDiskSpaceUsed.getCount();
     }
 
     /**
@@ -2543,8 +2630,10 @@
     public int invalidateRowCache(Collection<Bounds<Token>> boundsToInvalidate)
     {
         int invalidatedKeys = 0;
-        for (RowCacheKey key : CacheService.instance.rowCache.getKeySet())
+        for (Iterator<RowCacheKey> keyIter = CacheService.instance.rowCache.keyIterator();
+             keyIter.hasNext(); )
         {
+            RowCacheKey key = keyIter.next();
             DecoratedKey dk = partitioner.decorateKey(ByteBuffer.wrap(key.key));
             if (key.ksAndCFName.equals(metadata.ksAndCFName) && Bounds.isInBounds(dk.getToken(), boundsToInvalidate))
             {
@@ -2559,8 +2648,10 @@
     public int invalidateCounterCache(Collection<Bounds<Token>> boundsToInvalidate)
     {
         int invalidatedKeys = 0;
-        for (CounterCacheKey key : CacheService.instance.counterCache.getKeySet())
+        for (Iterator<CounterCacheKey> keyIter = CacheService.instance.counterCache.keyIterator();
+             keyIter.hasNext(); )
         {
+            CounterCacheKey key = keyIter.next();
             DecoratedKey dk = partitioner.decorateKey(ByteBuffer.wrap(key.partitionKey));
             if (key.ksAndCFName.equals(metadata.ksAndCFName) && Bounds.isInBounds(dk.getToken(), boundsToInvalidate))
             {
@@ -2609,7 +2700,13 @@
 
     public void forceMajorCompaction() throws InterruptedException, ExecutionException
     {
-        CompactionManager.instance.performMaximal(this);
+        forceMajorCompaction(false);
+    }
+
+
+    public void forceMajorCompaction(boolean splitOutput) throws InterruptedException, ExecutionException
+    {
+        CompactionManager.instance.performMaximal(this, splitOutput);
     }
 
     public static Iterable<ColumnFamilyStore> all()
@@ -2652,6 +2749,7 @@
      * thread safety.  All we do is wipe the sstable containers clean, while leaving the actual
      * data files present on disk.  (This allows tests to easily call loadNewSSTables on them.)
      */
+    @VisibleForTesting
     public void clearUnsafe()
     {
         for (final ColumnFamilyStore cfs : concatWithIndexes())
@@ -2660,7 +2758,9 @@
             {
                 public Void call()
                 {
-                    cfs.data.init();
+                    cfs.data.reset(new Memtable(new AtomicReference<>(ReplayPosition.NONE), cfs));
+                    cfs.getCompactionStrategy().shutdown();
+                    cfs.getCompactionStrategy().startup();
                     return null;
                 }
             }, true);
@@ -2672,6 +2772,18 @@
      */
     public void truncateBlocking()
     {
+        truncateBlocking(DatabaseDescriptor.isAutoSnapshot());
+    }
+
+    /**
+     * Truncate deletes the column family's data with no expensive tombstone creation,
+     * optionally snapshotting the data.
+     *
+     * @param takeSnapshot whether or not to take a snapshot <code>true</code> if snapshot should be taken,
+     *                     <code>false</code> otherwise
+     */
+    public void truncateBlocking(final boolean takeSnapshot)
+    {
         // We have two goals here:
         // - truncate should delete everything written before truncate was invoked
         // - but not delete anything that isn't part of the snapshot we create.
@@ -2684,53 +2796,58 @@
         // beginning if we restart before they [the CL segments] are discarded for
         // normal reasons post-truncate.  To prevent this, we store truncation
         // position in the System keyspace.
-        logger.debug("truncating {}", name);
+        logger.trace("truncating {}", name);
 
-        if (keyspace.getMetadata().durableWrites || DatabaseDescriptor.isAutoSnapshot())
+        final long truncatedAt;
+        final ReplayPosition replayAfter;
+
+        if (keyspace.getMetadata().durableWrites || takeSnapshot)
         {
-            // flush the CF being truncated before forcing the new segment
-            forceBlockingFlush();
-
-            // sleep a little to make sure that our truncatedAt comes after any sstable
-            // that was part of the flushed we forced; otherwise on a tie, it won't get deleted.
-            Uninterruptibles.sleepUninterruptibly(1, TimeUnit.MILLISECONDS);
+            replayAfter = forceBlockingFlush();
         }
         else
         {
             // just nuke the memtable data w/o writing to disk first
+            Future<ReplayPosition> replayAfterFuture;
             synchronized (data)
             {
                 final Flush flush = new Flush(true);
                 flushExecutor.execute(flush);
-                postFlushExecutor.submit(flush.postFlush);
+                replayAfterFuture = postFlushExecutor.submit(flush.postFlush);
             }
+            replayAfter = FBUtilities.waitOnFuture(replayAfterFuture);
         }
 
+        long now = System.currentTimeMillis();
+        // make sure none of our sstables are somehow in the future (clock drift, perhaps)
+        for (ColumnFamilyStore cfs : concatWithIndexes())
+            for (SSTableReader sstable : cfs.data.getSSTables())
+                now = Math.max(now, sstable.maxDataAge);
+        truncatedAt = now;
+
         Runnable truncateRunnable = new Runnable()
         {
             public void run()
             {
                 logger.debug("Discarding sstable data for truncated CF + indexes");
-
-                final long truncatedAt = System.currentTimeMillis();
                 data.notifyTruncated(truncatedAt);
 
-                if (DatabaseDescriptor.isAutoSnapshot())
+                if (takeSnapshot)
                     snapshot(Keyspace.getTimestampedSnapshotName(name));
 
-                ReplayPosition replayAfter = discardSSTables(truncatedAt);
+                discardSSTables(truncatedAt);
 
                 for (SecondaryIndex index : indexManager.getIndexes())
                     index.truncateBlocking(truncatedAt);
 
                 SystemKeyspace.saveTruncationRecord(ColumnFamilyStore.this, truncatedAt, replayAfter);
-                logger.debug("cleaning out row cache");
+                logger.trace("cleaning out row cache");
                 invalidateCaches();
             }
         };
 
         runWithCompactionsDisabled(Executors.callable(truncateRunnable), true);
-        logger.debug("truncate complete");
+        logger.trace("truncate complete");
     }
 
     public <V> V runWithCompactionsDisabled(Callable<V> callable, boolean interruptValidation)
@@ -2739,7 +2856,7 @@
         // and so we only run one major compaction at a time
         synchronized (this)
         {
-            logger.debug("Cancelling in-progress compactions for {}", metadata.cfName);
+            logger.trace("Cancelling in-progress compactions for {}", metadata.cfName);
 
             Iterable<ColumnFamilyStore> selfWithIndexes = concatWithIndexes();
             for (ColumnFamilyStore cfs : selfWithIndexes)
@@ -2753,13 +2870,13 @@
                 // doublecheck that we finished, instead of timing out
                 for (ColumnFamilyStore cfs : selfWithIndexes)
                 {
-                    if (!cfs.getDataTracker().getCompacting().isEmpty())
+                    if (!cfs.getTracker().getCompacting().isEmpty())
                     {
                         logger.warn("Unable to cancel in-progress compactions for {}.  Perhaps there is an unusually large row in progress somewhere, or the system is simply overloaded.", metadata.cfName);
                         return null;
                     }
                 }
-                logger.debug("Compactions successfully cancelled");
+                logger.trace("Compactions successfully cancelled");
 
                 // run our task
                 try
@@ -2779,64 +2896,25 @@
         }
     }
 
-    public Iterable<SSTableReader> markAllCompacting()
+    public LifecycleTransaction markAllCompacting(final OperationType operationType)
     {
-        Callable<Iterable<SSTableReader>> callable = new Callable<Iterable<SSTableReader>>()
+        Callable<LifecycleTransaction> callable = new Callable<LifecycleTransaction>()
         {
-            public Iterable<SSTableReader> call() throws Exception
+            public LifecycleTransaction call() throws Exception
             {
                 assert data.getCompacting().isEmpty() : data.getCompacting();
-                Collection<SSTableReader> sstables = Lists.newArrayList(AbstractCompactionStrategy.filterSuspectSSTables(getSSTables()));
-                if (Iterables.isEmpty(sstables))
-                    return Collections.emptyList();
-                boolean success = data.markCompacting(sstables);
-                assert success : "something marked things compacting while compactions are disabled";
-                return sstables;
+                Iterable<SSTableReader> sstables = getPermittedToCompactSSTables();
+                sstables = AbstractCompactionStrategy.filterSuspectSSTables(sstables);
+                sstables = ImmutableList.copyOf(sstables);
+                LifecycleTransaction modifier = data.tryModify(sstables, operationType);
+                assert modifier != null: "something marked things compacting while compactions are disabled";
+                return modifier;
             }
         };
 
         return runWithCompactionsDisabled(callable, false);
     }
 
-    public long getBloomFilterFalsePositives()
-    {
-        return metric.bloomFilterFalsePositives.value();
-    }
-
-    public long getRecentBloomFilterFalsePositives()
-    {
-        return metric.recentBloomFilterFalsePositives.value();
-    }
-
-    public double getBloomFilterFalseRatio()
-    {
-        return metric.bloomFilterFalseRatio.value();
-    }
-
-    public double getRecentBloomFilterFalseRatio()
-    {
-        return metric.recentBloomFilterFalseRatio.value();
-    }
-
-    public long getBloomFilterDiskSpaceUsed()
-    {
-        return metric.bloomFilterDiskSpaceUsed.value();
-    }
-
-    public long getBloomFilterOffHeapMemoryUsed()
-    {
-        return metric.bloomFilterOffHeapMemoryUsed.value();
-    }
-
-    public long getIndexSummaryOffHeapMemoryUsed()
-    {
-        return metric.indexSummaryOffHeapMemoryUsed.value();
-    }
-
-    public long getCompressionMetadataOffHeapMemoryUsed()
-    {
-        return metric.compressionMetadataOffHeapMemoryUsed.value();
-    }
 
     @Override
     public String toString()
@@ -2933,36 +3011,27 @@
                     "is deprecated, set the compaction strategy option 'enabled' to 'false' instead or use the nodetool command 'disableautocompaction'.");
     }
 
-    public double getTombstonesPerSlice()
-    {
-        return metric.tombstoneScannedHistogram.cf.getSnapshot().getMedian();
-    }
-
-    public double getLiveCellsPerSlice()
-    {
-        return metric.liveScannedHistogram.cf.getSnapshot().getMedian();
-    }
-
     // End JMX get/set.
 
+    public int getMeanColumns()
+    {
+        long sum = 0;
+        long count = 0;
+        for (SSTableReader sstable : getSSTables())
+        {
+            long n = sstable.getEstimatedColumnCount().count();
+            sum += sstable.getEstimatedColumnCount().mean() * n;
+            count += n;
+        }
+        return count > 0 ? (int) (sum / count) : 0;
+    }
+
     public long estimateKeys()
     {
-        return data.estimatedKeys();
-    }
-
-    public long[] getEstimatedRowSizeHistogram()
-    {
-        return metric.estimatedRowSizeHistogram.value();
-    }
-
-    public long[] getEstimatedColumnCountHistogram()
-    {
-        return metric.estimatedColumnCountHistogram.value();
-    }
-
-    public double getCompressionRatio()
-    {
-        return metric.compressionRatio.value();
+        long n = 0;
+        for (SSTableReader sstable : getSSTables())
+            n += sstable.estimatedKeys();
+        return n;
     }
 
     /** true if this CFS contains secondary index data */
@@ -3026,18 +3095,10 @@
         }
     }
 
-    /**
-     * Returns the creation time of the oldest memtable not fully flushed yet.
-     */
-    public long oldestUnflushedMemtable()
-    {
-        return data.getView().getOldestMemtable().creationTime();
-    }
-
     public boolean isEmpty()
     {
-        DataTracker.View view = data.getView();
-        return view.sstables.isEmpty() && view.getCurrentMemtable().getOperations() == 0 && view.getCurrentMemtable() == view.getOldestMemtable();
+        View view = data.getView();
+        return view.sstables.isEmpty() && view.getCurrentMemtable().getOperations() == 0 && view.liveMemtables.size() <= 1 && view.flushingMemtables.size() == 0;
     }
 
     public boolean isRowCacheEnabled()
@@ -3062,14 +3123,12 @@
      *
      * @param truncatedAt The timestamp of the truncation
      *                    (all SSTables before that timestamp are going be marked as compacted)
-     *
-     * @return the most recent replay position of the truncated data
      */
-    public ReplayPosition discardSSTables(long truncatedAt)
+    public void discardSSTables(long truncatedAt)
     {
         assert data.getCompacting().isEmpty() : data.getCompacting();
 
-        List<SSTableReader> truncatedSSTables = new ArrayList<SSTableReader>();
+        List<SSTableReader> truncatedSSTables = new ArrayList<>();
 
         for (SSTableReader sstable : getSSTables())
         {
@@ -3077,16 +3136,22 @@
                 truncatedSSTables.add(sstable);
         }
 
-        if (truncatedSSTables.isEmpty())
-            return ReplayPosition.NONE;
-
-        markObsolete(truncatedSSTables, OperationType.UNKNOWN);
-        return ReplayPosition.getReplayPosition(truncatedSSTables);
+        if (!truncatedSSTables.isEmpty())
+            markObsolete(truncatedSSTables, OperationType.UNKNOWN);
     }
 
     public double getDroppableTombstoneRatio()
     {
-        return getDataTracker().getDroppableTombstoneRatio();
+        double allDroppable = 0;
+        long allColumns = 0;
+        int localTime = (int)(System.currentTimeMillis()/1000);
+
+        for (SSTableReader sstable : getSSTables())
+        {
+            allDroppable += sstable.getDroppableTombstonesBefore(localTime - sstable.metadata.getGcGraceSeconds());
+            allColumns += sstable.getEstimatedColumnCount().mean() * sstable.getEstimatedColumnCount().count();
+        }
+        return allColumns > 0 ? allDroppable / allColumns : 0;
     }
 
     public long trueSnapshotsSize()
@@ -3103,9 +3168,9 @@
     // returns the "canonical" version of any current sstable, i.e. if an sstable is being replaced and is only partially
     // visible to reads, this sstable will be returned as its original entirety, and its replacement will not be returned
     // (even if it completely replaces it)
-    public static final Function<DataTracker.View, List<SSTableReader>> CANONICAL_SSTABLES = new Function<DataTracker.View, List<SSTableReader>>()
+    public static final Function<View, List<SSTableReader>> CANONICAL_SSTABLES = new Function<View, List<SSTableReader>>()
     {
-        public List<SSTableReader> apply(DataTracker.View view)
+        public List<SSTableReader> apply(View view)
         {
             List<SSTableReader> sstables = new ArrayList<>();
             for (SSTableReader sstable : view.compacting)
@@ -3118,9 +3183,9 @@
         }
     };
 
-    public static final Function<DataTracker.View, List<SSTableReader>> UNREPAIRED_SSTABLES = new Function<DataTracker.View, List<SSTableReader>>()
+    public static final Function<View, List<SSTableReader>> UNREPAIRED_SSTABLES = new Function<View, List<SSTableReader>>()
     {
-        public List<SSTableReader> apply(DataTracker.View view)
+        public List<SSTableReader> apply(View view)
         {
             List<SSTableReader> sstables = new ArrayList<>();
             for (SSTableReader sstable : CANONICAL_SSTABLES.apply(view))
@@ -3131,4 +3196,41 @@
             return sstables;
         }
     };
+
+    /**
+     * Returns a ColumnFamilyStore by cfId if it exists, null otherwise
+     * Differently from others, this method does not throw exception if the table does not exist.
+     */
+    public static ColumnFamilyStore getIfExists(UUID cfId)
+    {
+        Pair<String, String> kscf = Schema.instance.getCF(cfId);
+        if (kscf == null)
+            return null;
+
+        Keyspace keyspace = Keyspace.open(kscf.left);
+        if (keyspace == null)
+            return null;
+
+        return keyspace.getColumnFamilyStore(cfId);
+    }
+
+    /**
+     * Returns a ColumnFamilyStore by ksname and cfname if it exists, null otherwise
+     * Differently from others, this method does not throw exception if the keyspace or table does not exist.
+     */
+    public static ColumnFamilyStore getIfExists(String ksName, String cfName)
+    {
+        if (ksName == null || cfName == null)
+            return null;
+
+        Keyspace keyspace = Keyspace.open(ksName);
+        if (keyspace == null)
+            return null;
+
+        UUID id = Schema.instance.getId(ksName, cfName);
+        if (id == null)
+            return null;
+
+        return keyspace.getColumnFamilyStore(id);
+    }
 }
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStoreMBean.java b/src/java/org/apache/cassandra/db/ColumnFamilyStoreMBean.java
index b2ab1e1..b4511d0 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStoreMBean.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStoreMBean.java
@@ -35,255 +35,11 @@
     public String getColumnFamilyName();
 
     /**
-     * Returns the total amount of data stored in the memtable, including
-     * column related overhead.
-     *
-     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#memtableOnHeapSize
-     * @return The size in bytes.
-     * @deprecated
-     */
-    @Deprecated
-    public long getMemtableDataSize();
-
-    /**
-     * Returns the total number of columns present in the memtable.
-     *
-     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#memtableColumnsCount
-     * @return The number of columns.
-     */
-    @Deprecated
-    public long getMemtableColumnsCount();
-
-    /**
-     * Returns the number of times that a flush has resulted in the
-     * memtable being switched out.
-     *
-     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#memtableSwitchCount
-     * @return the number of memtable switches
-     */
-    @Deprecated
-    public int getMemtableSwitchCount();
-
-    /**
-     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#recentSSTablesPerRead
-     * @return a histogram of the number of sstable data files accessed per read: reading this property resets it
-     */
-    @Deprecated
-    public long[] getRecentSSTablesPerReadHistogram();
-
-    /**
-     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#sstablesPerReadHistogram
-     * @return a histogram of the number of sstable data files accessed per read
-     */
-    @Deprecated
-    public long[] getSSTablesPerReadHistogram();
-
-    /**
-     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#readLatency
-     * @return the number of read operations on this column family
-     */
-    @Deprecated
-    public long getReadCount();
-
-    /**
-     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#readLatency
-     * @return total read latency (divide by getReadCount() for average)
-     */
-    @Deprecated
-    public long getTotalReadLatencyMicros();
-
-    /**
-     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#readLatency
-     * @return an array representing the latency histogram
-     */
-    @Deprecated
-    public long[] getLifetimeReadLatencyHistogramMicros();
-
-    /**
-     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#readLatency
-     * @return an array representing the latency histogram
-     */
-    @Deprecated
-    public long[] getRecentReadLatencyHistogramMicros();
-
-    /**
-     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#readLatency
-     * @return average latency per read operation since the last call
-     */
-    @Deprecated
-    public double getRecentReadLatencyMicros();
-
-    /**
-     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#writeLatency
-     * @return the number of write operations on this column family
-     */
-    @Deprecated
-    public long getWriteCount();
-
-    /**
-     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#writeLatency
-     * @return total write latency (divide by getReadCount() for average)
-     */
-    @Deprecated
-    public long getTotalWriteLatencyMicros();
-
-    /**
-     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#writeLatency
-     * @return an array representing the latency histogram
-     */
-    @Deprecated
-    public long[] getLifetimeWriteLatencyHistogramMicros();
-
-    /**
-     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#writeLatency
-     * @return an array representing the latency histogram
-     */
-    @Deprecated
-    public long[] getRecentWriteLatencyHistogramMicros();
-
-    /**
-     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#writeLatency
-     * @return average latency per write operation since the last call
-     */
-    @Deprecated
-    public double getRecentWriteLatencyMicros();
-
-    /**
-     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#rangeLatency
-     * @return the number of range slice operations on this column family
-     */
-    @Deprecated
-    public long getRangeCount();
-
-    /**
-     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#rangeLatency
-     * @return total range slice latency (divide by getRangeCount() for average)
-     */
-    @Deprecated
-    public long getTotalRangeLatencyMicros();
-
-    /**
-     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#rangeLatency
-     * @return an array representing the latency histogram
-     */
-    @Deprecated
-    public long[] getLifetimeRangeLatencyHistogramMicros();
-
-    /**
-     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#rangeLatency
-     * @return an array representing the latency histogram
-     */
-    @Deprecated
-    public long[] getRecentRangeLatencyHistogramMicros();
-
-    /**
-     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#rangeLatency
-     * @return average latency per range slice operation since the last call
-     */
-    @Deprecated
-    public double getRecentRangeLatencyMicros();
-
-    /**
-     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#pendingFlushes
-     * @return the estimated number of tasks pending for this column family
-     */
-    @Deprecated
-    public int getPendingTasks();
-
-    /**
-     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#liveSSTableCount
-     * @return the number of SSTables on disk for this CF
-     */
-    @Deprecated
-    public int getLiveSSTableCount();
-
-    /**
-     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#liveDiskSpaceUsed
-     * @return disk space used by SSTables belonging to this CF
-     */
-    @Deprecated
-    public long getLiveDiskSpaceUsed();
-
-    /**
-     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#totalDiskSpaceUsed
-     * @return total disk space used by SSTables belonging to this CF, including obsolete ones waiting to be GC'd
-     */
-    @Deprecated
-    public long getTotalDiskSpaceUsed();
-
-    /**
      * force a major compaction of this column family
+     *
+     * @param splitOutput true if the output of the major compaction should be split in several sstables
      */
-    public void forceMajorCompaction() throws ExecutionException, InterruptedException;
-
-    /**
-     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#minRowSize
-     * @return the size of the smallest compacted row
-     */
-    @Deprecated
-    public long getMinRowSize();
-
-    /**
-     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#maxRowSize
-     * @return the size of the largest compacted row
-     */
-    @Deprecated
-    public long getMaxRowSize();
-
-    /**
-     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#meanRowSize
-     * @return the average row size across all the sstables
-     */
-    @Deprecated
-    public long getMeanRowSize();
-
-    /**
-     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#bloomFilterFalsePositives
-     */
-    @Deprecated
-    public long getBloomFilterFalsePositives();
-
-    /**
-     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#recentBloomFilterFalsePositives
-     */
-    @Deprecated
-    public long getRecentBloomFilterFalsePositives();
-
-    /**
-     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#bloomFilterFalseRatio
-     */
-    @Deprecated
-    public double getBloomFilterFalseRatio();
-
-    /**
-     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#recentBloomFilterFalseRatio
-     */
-    @Deprecated
-    public double getRecentBloomFilterFalseRatio();
-
-    /**
-     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#bloomFilterDiskSpaceUsed
-     */
-    @Deprecated
-    public long getBloomFilterDiskSpaceUsed();
-
-    /**
-     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#bloomFilterOffHeapMemoryUsed
-     */
-    @Deprecated
-    public long getBloomFilterOffHeapMemoryUsed();
-
-    /**
-     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#indexSummaryOffHeapMemoryUsed
-     */
-    @Deprecated
-    public long getIndexSummaryOffHeapMemoryUsed();
-
-    /**
-     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#compressionMetadataOffHeapMemoryUsed
-     */
-    @Deprecated
-    public long getCompressionMetadataOffHeapMemoryUsed();
+    public void forceMajorCompaction(boolean splitOutput) throws ExecutionException, InterruptedException;
 
     /**
      * Gets the minimum number of sstables in queue before compaction kicks off
@@ -360,31 +116,8 @@
 
     public boolean isAutoCompactionDisabled();
 
-    /** Number of tombstoned cells retreived during the last slicequery */
-    @Deprecated
-    public double getTombstonesPerSlice();
-
-    /** Number of live cells retreived during the last slicequery */
-    @Deprecated
-    public double getLiveCellsPerSlice();
-
     public long estimateKeys();
 
-    /**
-     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#estimatedRowSizeHistogram
-     */
-    @Deprecated
-    public long[] getEstimatedRowSizeHistogram();
-    /**
-     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#estimatedColumnCountHistogram
-     */
-    @Deprecated
-    public long[] getEstimatedColumnCountHistogram();
-    /**
-     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#compressionRatio
-     */
-    @Deprecated
-    public double getCompressionRatio();
 
     /**
      * Returns a list of the names of the built column indexes for current store
diff --git a/src/java/org/apache/cassandra/db/ConsistencyLevel.java b/src/java/org/apache/cassandra/db/ConsistencyLevel.java
index e322968..85ec0f3 100644
--- a/src/java/org/apache/cassandra/db/ConsistencyLevel.java
+++ b/src/java/org/apache/cassandra/db/ConsistencyLevel.java
@@ -25,7 +25,6 @@
 import java.util.Map;
 
 import com.google.common.collect.Iterables;
-import net.nicoulaj.compilecommand.annotations.Inline;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -261,7 +260,7 @@
                 int localLive = countLocalEndpoints(liveEndpoints);
                 if (localLive < blockFor)
                 {
-                    if (logger.isDebugEnabled())
+                    if (logger.isTraceEnabled())
                     {
                         StringBuilder builder = new StringBuilder("Local replicas [");
                         for (InetAddress endpoint : liveEndpoints)
@@ -270,7 +269,7 @@
                                 builder.append(endpoint).append(",");
                         }
                         builder.append("] are insufficient to satisfy LOCAL_QUORUM requirement of ").append(blockFor).append(" live nodes in '").append(DatabaseDescriptor.getLocalDataCenter()).append("'");
-                        logger.debug(builder.toString());
+                        logger.trace(builder.toString());
                     }
                     throw new UnavailableException(this, blockFor, localLive);
                 }
@@ -292,7 +291,7 @@
                 int live = Iterables.size(liveEndpoints);
                 if (live < blockFor)
                 {
-                    logger.debug("Live nodes {} do not satisfy ConsistencyLevel ({} required)", Iterables.toString(liveEndpoints), blockFor);
+                    logger.trace("Live nodes {} do not satisfy ConsistencyLevel ({} required)", Iterables.toString(liveEndpoints), blockFor);
                     throw new UnavailableException(this, blockFor, live);
                 }
                 break;
@@ -348,7 +347,7 @@
     public void validateCounterForWrite(CFMetaData metadata) throws InvalidRequestException
     {
         if (this == ConsistencyLevel.ANY)
-            throw new InvalidRequestException("Consistency level ANY is not yet supported for counter columnfamily " + metadata.cfName);
+            throw new InvalidRequestException("Consistency level ANY is not yet supported for counter table " + metadata.cfName);
 
         if (isSerialConsistency())
             throw new InvalidRequestException("Counter operations are inherently non-serializable");
diff --git a/src/java/org/apache/cassandra/db/CounterMutationVerbHandler.java b/src/java/org/apache/cassandra/db/CounterMutationVerbHandler.java
index d65fbd7..4dd8ac3 100644
--- a/src/java/org/apache/cassandra/db/CounterMutationVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/CounterMutationVerbHandler.java
@@ -34,31 +34,23 @@
 
     public void doVerb(final MessageIn<CounterMutation> message, final int id)
     {
-        try
-        {
-            final CounterMutation cm = message.payload;
-            logger.debug("Applying forwarded {}", cm);
+        final CounterMutation cm = message.payload;
+        logger.trace("Applying forwarded {}", cm);
 
-            String localDataCenter = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddress());
-            // We should not wait for the result of the write in this thread,
-            // otherwise we could have a distributed deadlock between replicas
-            // running this VerbHandler (see #4578).
-            // Instead, we use a callback to send the response. Note that the callback
-            // will not be called if the request timeout, but this is ok
-            // because the coordinator of the counter mutation will timeout on
-            // it's own in that case.
-            StorageProxy.applyCounterMutationOnLeader(cm, localDataCenter, new Runnable()
-            {
-                public void run()
-                {
-                    MessagingService.instance().sendReply(new WriteResponse().createMessage(), id, message.from);
-                }
-            });
-        }
-        catch (RequestExecutionException e)
+        String localDataCenter = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddress());
+        // We should not wait for the result of the write in this thread,
+        // otherwise we could have a distributed deadlock between replicas
+        // running this VerbHandler (see #4578).
+        // Instead, we use a callback to send the response. Note that the callback
+        // will not be called if the request timeout, but this is ok
+        // because the coordinator of the counter mutation will timeout on
+        // it's own in that case.
+        StorageProxy.applyCounterMutationOnLeader(cm, localDataCenter, new Runnable()
         {
-            // The coordinator will timeout on it's own so ignore
-            logger.debug("counter error", e);
-        }
+            public void run()
+            {
+                MessagingService.instance().sendReply(new WriteResponse().createMessage(), id, message.from);
+            }
+        });
     }
 }
diff --git a/src/java/org/apache/cassandra/db/DataRange.java b/src/java/org/apache/cassandra/db/DataRange.java
index 08fdd14..1e6f8c8 100644
--- a/src/java/org/apache/cassandra/db/DataRange.java
+++ b/src/java/org/apache/cassandra/db/DataRange.java
@@ -67,12 +67,17 @@
 
     public static DataRange allData(IPartitioner partitioner)
     {
-        return forKeyRange(new Range<Token>(partitioner.getMinimumToken(), partitioner.getMinimumToken()));
+        return forTokenRange(new Range<Token>(partitioner.getMinimumToken(), partitioner.getMinimumToken()));
     }
 
-    public static DataRange forKeyRange(Range<Token> keyRange)
+    public static DataRange forTokenRange(Range<Token> keyRange)
     {
-        return new DataRange(keyRange.toRowBounds(), new IdentityQueryFilter());
+        return forKeyRange(Range.makeRowRange(keyRange));
+    }
+
+    public static DataRange forKeyRange(Range<RowPosition> keyRange)
+    {
+        return new DataRange(keyRange, new IdentityQueryFilter());
     }
 
     public AbstractBounds<RowPosition> keyRange()
@@ -106,7 +111,7 @@
     public boolean isWrapAround()
     {
         // On range can ever wrap
-        return keyRange instanceof Range && ((Range)keyRange).isWrapAround();
+        return keyRange instanceof Range && ((Range<?>)keyRange).isWrapAround();
     }
 
     public boolean contains(RowPosition pos)
@@ -169,7 +174,7 @@
 
             // When using a paging range, we don't allow wrapped ranges, as it's unclear how to handle them properly.
             // This is ok for now since we only need this in range slice queries, and the range are "unwrapped" in that case.
-            assert !(range instanceof Range) || !((Range)range).isWrapAround() || range.right.isMinimum() : range;
+            assert !(range instanceof Range) || !((Range<?>)range).isWrapAround() || range.right.isMinimum() : range;
 
             this.sliceFilter = filter;
             this.cfm = cfm;
diff --git a/src/java/org/apache/cassandra/db/DataTracker.java b/src/java/org/apache/cassandra/db/DataTracker.java
deleted file mode 100644
index 927e717..0000000
--- a/src/java/org/apache/cassandra/db/DataTracker.java
+++ /dev/null
@@ -1,825 +0,0 @@
-/*
- * 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.db;
-
-import java.io.File;
-import java.util.*;
-import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.atomic.AtomicReference;
-
-import com.google.common.base.Predicate;
-import com.google.common.base.Predicates;
-import com.google.common.collect.*;
-import org.apache.cassandra.db.commitlog.ReplayPosition;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.compaction.OperationType;
-import org.apache.cassandra.dht.AbstractBounds;
-import org.apache.cassandra.dht.IPartitioner;
-import org.apache.cassandra.io.sstable.IndexSummary;
-import org.apache.cassandra.io.sstable.SSTableReader;
-import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.io.util.SegmentedFile;
-import org.apache.cassandra.metrics.StorageMetrics;
-import org.apache.cassandra.notifications.*;
-import org.apache.cassandra.utils.IFilter;
-import org.apache.cassandra.utils.Interval;
-import org.apache.cassandra.utils.IntervalTree;
-import org.apache.cassandra.utils.concurrent.OpOrder;
-import org.apache.cassandra.utils.concurrent.Refs;
-
-public class DataTracker
-{
-    private static final Logger logger = LoggerFactory.getLogger(DataTracker.class);
-
-    public final Collection<INotificationConsumer> subscribers = new CopyOnWriteArrayList<>();
-    public final ColumnFamilyStore cfstore;
-    private final AtomicReference<View> view;
-
-    public DataTracker(ColumnFamilyStore cfstore)
-    {
-        this.cfstore = cfstore;
-        this.view = new AtomicReference<>();
-        this.init();
-    }
-
-    // get the Memtable that the ordered writeOp should be directed to
-    public Memtable getMemtableFor(OpOrder.Group opGroup, ReplayPosition replayPosition)
-    {
-        // since any new memtables appended to the list after we fetch it will be for operations started
-        // after us, we can safely assume that we will always find the memtable that 'accepts' us;
-        // if the barrier for any memtable is set whilst we are reading the list, it must accept us.
-
-        // there may be multiple memtables in the list that would 'accept' us, however we only ever choose
-        // the oldest such memtable, as accepts() only prevents us falling behind (i.e. ensures we don't
-        // assign operations to a memtable that was retired/queued before we started)
-        for (Memtable memtable : view.get().liveMemtables)
-        {
-            if (memtable.accepts(opGroup, replayPosition))
-                return memtable;
-        }
-        throw new AssertionError(view.get().liveMemtables.toString());
-    }
-
-    public Set<SSTableReader> getSSTables()
-    {
-        return view.get().sstables;
-    }
-
-    public Set<SSTableReader> getUncompactingSSTables()
-    {
-        return view.get().nonCompactingSStables();
-    }
-
-    public Iterable<SSTableReader> getUncompactingSSTables(Iterable<SSTableReader> candidates)
-    {
-        final View v = view.get();
-        return Iterables.filter(candidates, new Predicate<SSTableReader>()
-        {
-            public boolean apply(SSTableReader sstable)
-            {
-                return !v.compacting.contains(sstable);
-            }
-        });
-    }
-
-    public View getView()
-    {
-        return view.get();
-    }
-
-    /**
-     * Switch the current memtable. This atomically appends a new memtable to the end of the list of active memtables,
-     * returning the previously last memtable. It leaves the previous Memtable in the list of live memtables until
-     * discarding(memtable) is called. These two methods must be synchronized/paired, i.e. m = switchMemtable
-     * must be followed by discarding(m), they cannot be interleaved.
-     *
-     * @return the previously active memtable
-     */
-    public Memtable switchMemtable(boolean truncating)
-    {
-        Memtable newMemtable = new Memtable(cfstore);
-        Memtable toFlushMemtable;
-        View currentView, newView;
-        do
-        {
-            currentView = view.get();
-            toFlushMemtable = currentView.getCurrentMemtable();
-            newView = currentView.switchMemtable(newMemtable);
-        }
-        while (!view.compareAndSet(currentView, newView));
-
-        if (truncating)
-            notifyRenewed(newMemtable);
-
-        return toFlushMemtable;
-    }
-
-    public void markFlushing(Memtable memtable)
-    {
-        View currentView, newView;
-        do
-        {
-            currentView = view.get();
-            newView = currentView.markFlushing(memtable);
-        }
-        while (!view.compareAndSet(currentView, newView));
-    }
-
-    public void replaceFlushed(Memtable memtable, SSTableReader sstable)
-    {
-        // sstable may be null if we flushed batchlog and nothing needed to be retained
-
-        if (!cfstore.isValid())
-        {
-            View currentView, newView;
-            do
-            {
-                currentView = view.get();
-                newView = currentView.replaceFlushed(memtable, sstable);
-                if (sstable != null)
-                    newView = newView.replace(Arrays.asList(sstable), Collections.<SSTableReader>emptyList());
-            }
-            while (!view.compareAndSet(currentView, newView));
-            return;
-        }
-
-        // back up before creating a new View (which makes the new one eligible for compaction)
-        if (sstable != null)
-            maybeIncrementallyBackup(sstable);
-
-        View currentView, newView;
-        do
-        {
-            currentView = view.get();
-            newView = currentView.replaceFlushed(memtable, sstable);
-        }
-        while (!view.compareAndSet(currentView, newView));
-
-        if (sstable != null)
-        {
-            addNewSSTablesSize(Arrays.asList(sstable));
-            notifyAdded(sstable);
-        }
-    }
-
-    public void maybeIncrementallyBackup(final SSTableReader sstable)
-    {
-        if (!DatabaseDescriptor.isIncrementalBackupsEnabled())
-            return;
-
-        File backupsDir = Directories.getBackupsDirectory(sstable.descriptor);
-        sstable.createLinks(FileUtils.getCanonicalPath(backupsDir));
-    }
-
-    /**
-     * @return true if we are able to mark the given @param sstables as compacted, before anyone else
-     *
-     * Note that we could acquire references on the marked sstables and release them in
-     * unmarkCompacting, but since we will never call markObsolete on a sstable marked
-     * as compacting (unless there is a serious bug), we can skip this.
-     */
-    public boolean markCompacting(Collection<SSTableReader> sstables)
-    {
-        return markCompacting(sstables, false, false);
-    }
-    public boolean markCompacting(Collection<SSTableReader> sstables, boolean newTables, boolean offline)
-    {
-        assert sstables != null && !Iterables.isEmpty(sstables);
-        while (true)
-        {
-            final View currentView = view.get();
-            if (Iterables.any(sstables, Predicates.in(currentView.compacting)))
-                return false;
-
-            Predicate live = new Predicate<SSTableReader>()
-            {
-                public boolean apply(SSTableReader sstable)
-                {
-                    return currentView.sstablesMap.get(sstable) == sstable && !sstable.isMarkedCompacted();
-                }
-            };
-            if (newTables)
-                assert !Iterables.any(sstables, Predicates.in(currentView.sstables));
-            else if (!offline && !Iterables.all(sstables, live))
-                return false;
-
-            View newView = currentView.markCompacting(sstables);
-            if (view.compareAndSet(currentView, newView))
-            {
-                notifyCompacting(sstables, true);
-                return true;
-            }
-        }
-    }
-
-    /**
-     * Removes files from compacting status: this is different from 'markObsolete'
-     * because it should be run regardless of whether a compaction succeeded.
-     */
-    public void unmarkCompacting(Iterable<SSTableReader> unmark)
-    {
-        View currentView, newView;
-        do
-        {
-            currentView = view.get();
-            newView = currentView.unmarkCompacting(unmark);
-        }
-        while (!view.compareAndSet(currentView, newView));
-
-        if (!cfstore.isValid())
-        {
-            // when the CFS is invalidated, it will call unreferenceSSTables().  However, unreferenceSSTables only deals
-            // with sstables that aren't currently being compacted.  If there are ongoing compactions that finish or are
-            // interrupted after the CFS is invalidated, those sstables need to be unreferenced as well, so we do that here.
-            unreferenceSSTables();
-        }
-
-        notifyCompacting(unmark, false);
-    }
-
-    public void markObsolete(Collection<SSTableReader> sstables, OperationType compactionType)
-    {
-        removeSSTablesFromTracker(sstables);
-        releaseReferences(sstables, false);
-        notifySSTablesChanged(sstables, Collections.<SSTableReader>emptyList(), compactionType);
-    }
-
-    /**
-     *
-     * @param oldSSTables
-     * @param allReplacements
-     * @param compactionType
-     */
-    // note that this DOES NOT insert the replacement sstables, it only removes the old sstables and notifies any listeners
-    // that they have been replaced by the provided sstables, which must have been performed by an earlier replaceReaders() call
-    public void markCompactedSSTablesReplaced(Collection<SSTableReader> oldSSTables, Collection<SSTableReader> allReplacements, OperationType compactionType)
-    {
-        removeSSTablesFromTracker(oldSSTables);
-        releaseReferences(oldSSTables, false);
-        notifySSTablesChanged(oldSSTables, allReplacements, compactionType);
-        addNewSSTablesSize(allReplacements);
-    }
-
-    public void addInitialSSTables(Collection<SSTableReader> sstables)
-    {
-        addSSTablesToTracker(sstables);
-        // no notifications or backup necessary
-    }
-
-    public void addSSTables(Collection<SSTableReader> sstables)
-    {
-        addSSTablesToTracker(sstables);
-        for (SSTableReader sstable : sstables)
-        {
-            maybeIncrementallyBackup(sstable);
-            notifyAdded(sstable);
-        }
-    }
-
-    /**
-     * Replaces existing sstables with new instances, makes sure compaction strategies have the correct instance
-     *
-     * @param toReplace
-     * @param replaceWith
-     */
-    public void replaceWithNewInstances(Collection<SSTableReader> toReplace, Collection<SSTableReader> replaceWith)
-    {
-        replaceReaders(toReplace, replaceWith, true);
-    }
-
-    /**
-     * Adds the early opened files to the data tracker, but does not tell compaction strategies about it
-     *
-     * note that we dont track the live size of these sstables
-     * @param toReplace
-     * @param replaceWith
-     */
-    public void replaceEarlyOpenedFiles(Collection<SSTableReader> toReplace, Collection<SSTableReader> replaceWith)
-    {
-        for (SSTableReader s : toReplace)
-            assert s.openReason.equals(SSTableReader.OpenReason.EARLY);
-        // note that we can replace an early opened file with a real one
-        replaceReaders(toReplace, replaceWith, false);
-    }
-
-    /**
-     * removes all sstables that are not busy compacting.
-     */
-    public void unreferenceSSTables()
-    {
-        Set<SSTableReader> notCompacting;
-
-        View currentView, newView;
-        do
-        {
-            currentView = view.get();
-            if (!currentView.compacting.isEmpty())
-                logger.error("Set of compacting sstables is non-empty when invalidating sstables {}", currentView.compacting);
-            notCompacting = currentView.nonCompactingSStables();
-            newView = currentView.replace(notCompacting, Collections.<SSTableReader>emptySet());
-        }
-        while (!view.compareAndSet(currentView, newView));
-
-        if (notCompacting.isEmpty())
-        {
-            // notifySSTablesChanged -> LeveledManifest.promote doesn't like a no-op "promotion"
-            return;
-        }
-        notifySSTablesChanged(notCompacting, Collections.<SSTableReader>emptySet(), OperationType.UNKNOWN);
-        removeOldSSTablesSize(notCompacting);
-        releaseReferences(notCompacting, true);
-    }
-
-    /**
-     * Removes every SSTable in the directory from the DataTracker's view.
-     * @param directory the unreadable directory, possibly with SSTables in it, but not necessarily.
-     */
-    void removeUnreadableSSTables(File directory)
-    {
-        View currentView, newView;
-        Set<SSTableReader> remaining = new HashSet<>();
-        do
-        {
-            currentView = view.get();
-            for (SSTableReader r : currentView.nonCompactingSStables())
-                if (!r.descriptor.directory.equals(directory))
-                    remaining.add(r);
-
-            if (remaining.size() == currentView.nonCompactingSStables().size())
-                return;
-
-            newView = currentView.replace(currentView.sstables, remaining);
-        }
-        while (!view.compareAndSet(currentView, newView));
-        for (SSTableReader sstable : currentView.sstables)
-            if (!remaining.contains(sstable))
-                sstable.selfRef().release();
-        notifySSTablesChanged(remaining, Collections.<SSTableReader>emptySet(), OperationType.UNKNOWN);
-    }
-
-    /** (Re)initializes the tracker, purging all references. */
-    void init()
-    {
-        view.set(new View(
-                         ImmutableList.of(new Memtable(cfstore)),
-                         ImmutableList.<Memtable>of(),
-                         Collections.<SSTableReader, SSTableReader>emptyMap(),
-                         Collections.<SSTableReader>emptySet(),
-                         Collections.<SSTableReader>emptySet(),
-                         SSTableIntervalTree.empty()));
-    }
-
-    /**
-     * A special kind of replacement for SSTableReaders that were cloned with a new index summary sampling level (see
-     * SSTableReader.cloneWithNewSummarySamplingLevel and CASSANDRA-5519).  This does not mark the old reader
-     * as compacted.
-     * @param oldSSTables replaced readers
-     * @param newSSTables replacement readers
-     */
-    private void replaceReaders(Collection<SSTableReader> oldSSTables, Collection<SSTableReader> newSSTables, boolean notify)
-    {
-        View currentView, newView;
-        do
-        {
-            currentView = view.get();
-            newView = currentView.replace(oldSSTables, newSSTables);
-        }
-        while (!view.compareAndSet(currentView, newView));
-
-        if (!oldSSTables.isEmpty() && notify)
-            notifySSTablesChanged(oldSSTables, newSSTables, OperationType.UNKNOWN);
-
-        for (SSTableReader sstable : newSSTables)
-            sstable.setupKeyCache();
-
-        Refs.release(Refs.selfRefs(oldSSTables));
-    }
-
-    private void removeSSTablesFromTracker(Collection<SSTableReader> oldSSTables)
-    {
-        View currentView, newView;
-        do
-        {
-            currentView = view.get();
-            newView = currentView.replace(oldSSTables, Collections.<SSTableReader>emptyList());
-        }
-        while (!view.compareAndSet(currentView, newView));
-        removeOldSSTablesSize(oldSSTables);
-    }
-
-    private void addSSTablesToTracker(Collection<SSTableReader> sstables)
-    {
-        View currentView, newView;
-        do
-        {
-            currentView = view.get();
-            newView = currentView.replace(Collections.<SSTableReader>emptyList(), sstables);
-        }
-        while (!view.compareAndSet(currentView, newView));
-        addNewSSTablesSize(sstables);
-    }
-
-    private void addNewSSTablesSize(Iterable<SSTableReader> newSSTables)
-    {
-        for (SSTableReader sstable : newSSTables)
-        {
-            if (logger.isDebugEnabled())
-                logger.debug(String.format("adding %s to list of files tracked for %s.%s",
-                            sstable.descriptor, cfstore.keyspace.getName(), cfstore.name));
-            long size = sstable.bytesOnDisk();
-            StorageMetrics.load.inc(size);
-            cfstore.metric.liveDiskSpaceUsed.inc(size);
-            cfstore.metric.totalDiskSpaceUsed.inc(size);
-            sstable.setupKeyCache();
-        }
-    }
-
-    private void removeOldSSTablesSize(Iterable<SSTableReader> oldSSTables)
-    {
-        for (SSTableReader sstable : oldSSTables)
-        {
-            if (logger.isDebugEnabled())
-                logger.debug(String.format("removing %s from list of files tracked for %s.%s",
-                            sstable.descriptor, cfstore.keyspace.getName(), cfstore.name));
-            long size = sstable.bytesOnDisk();
-            StorageMetrics.load.dec(size);
-            cfstore.metric.liveDiskSpaceUsed.dec(size);
-        }
-    }
-
-    private void releaseReferences(Iterable<SSTableReader> oldSSTables, boolean tolerateCompacted)
-    {
-        for (SSTableReader sstable : oldSSTables)
-        {
-            boolean firstToCompact = sstable.markObsolete(this);
-            assert tolerateCompacted || firstToCompact : sstable + " was already marked compacted";
-            sstable.selfRef().release();
-        }
-    }
-
-    public long estimatedKeys()
-    {
-        long n = 0;
-        for (SSTableReader sstable : getSSTables())
-            n += sstable.estimatedKeys();
-        return n;
-    }
-
-    public int getMeanColumns()
-    {
-        long sum = 0;
-        long count = 0;
-        for (SSTableReader sstable : getSSTables())
-        {
-            long n = sstable.getEstimatedColumnCount().count();
-            sum += sstable.getEstimatedColumnCount().mean() * n;
-            count += n;
-        }
-        return count > 0 ? (int) (sum / count) : 0;
-    }
-
-    public double getDroppableTombstoneRatio()
-    {
-        double allDroppable = 0;
-        long allColumns = 0;
-        int localTime = (int)(System.currentTimeMillis()/1000);
-
-        for (SSTableReader sstable : getSSTables())
-        {
-            allDroppable += sstable.getDroppableTombstonesBefore(localTime - sstable.metadata.getGcGraceSeconds());
-            allColumns += sstable.getEstimatedColumnCount().mean() * sstable.getEstimatedColumnCount().count();
-        }
-        return allColumns > 0 ? allDroppable / allColumns : 0;
-    }
-
-    public void notifySSTablesChanged(Collection<SSTableReader> removed, Collection<SSTableReader> added, OperationType compactionType)
-    {
-        INotification notification = new SSTableListChangedNotification(added, removed, compactionType);
-        for (INotificationConsumer subscriber : subscribers)
-            subscriber.handleNotification(notification, this);
-    }
-
-    public void notifyCompacting(Iterable<SSTableReader> reader, boolean compacting)
-    {
-        INotification notification = new SSTableCompactingNotification(reader, compacting);
-        for (INotificationConsumer subscriber : subscribers)
-            subscriber.handleNotification(notification, this);
-    }
-
-    public void notifyAdded(SSTableReader added)
-    {
-        INotification notification = new SSTableAddedNotification(added);
-        for (INotificationConsumer subscriber : subscribers)
-            subscriber.handleNotification(notification, this);
-    }
-
-    public void notifySSTableRepairedStatusChanged(Collection<SSTableReader> repairStatusesChanged)
-    {
-        INotification notification = new SSTableRepairStatusChanged(repairStatusesChanged);
-        for (INotificationConsumer subscriber : subscribers)
-            subscriber.handleNotification(notification, this);
-
-    }
-
-    public void notifyDeleting(SSTableReader deleting)
-    {
-        INotification notification = new SSTableDeletingNotification(deleting);
-        for (INotificationConsumer subscriber : subscribers)
-            subscriber.handleNotification(notification, this);
-    }
-
-    public void notifyRenewed(Memtable renewed)
-    {
-        INotification notification = new MemtableRenewedNotification(renewed);
-        for (INotificationConsumer subscriber : subscribers)
-            subscriber.handleNotification(notification, this);
-    }
-
-    public void notifyTruncated(long truncatedAt)
-    {
-        INotification notification = new TruncationNotification(truncatedAt);
-        for (INotificationConsumer subscriber : subscribers)
-            subscriber.handleNotification(notification, this);
-    }
-
-    public void subscribe(INotificationConsumer consumer)
-    {
-        subscribers.add(consumer);
-    }
-
-    public void unsubscribe(INotificationConsumer consumer)
-    {
-        subscribers.remove(consumer);
-    }
-
-    public static SSTableIntervalTree buildIntervalTree(Iterable<SSTableReader> sstables)
-    {
-        List<Interval<RowPosition, SSTableReader>> intervals = new ArrayList<>(Iterables.size(sstables));
-        for (SSTableReader sstable : sstables)
-            intervals.add(Interval.<RowPosition, SSTableReader>create(sstable.first, sstable.last, sstable));
-        return new SSTableIntervalTree(intervals);
-    }
-
-    public Set<SSTableReader> getCompacting()
-    {
-        return getView().compacting;
-    }
-
-    public SSTableReader getCurrentVersion(SSTableReader sstable)
-    {
-        if (!sstable.isReplaced())
-            return sstable;
-        SSTableReader current = view.get().sstablesMap.get(sstable);
-        if (current == null)
-            current = Iterables.find(view.get().shadowed, Predicates.equalTo(sstable), null);
-        assert current != null : sstable + " not in live set";
-        return current;
-    }
-
-    public static class SSTableIntervalTree extends IntervalTree<RowPosition, SSTableReader, Interval<RowPosition, SSTableReader>>
-    {
-        private static final SSTableIntervalTree EMPTY = new SSTableIntervalTree(null);
-
-        private SSTableIntervalTree(Collection<Interval<RowPosition, SSTableReader>> intervals)
-        {
-            super(intervals, null);
-        }
-
-        public static SSTableIntervalTree empty()
-        {
-            return EMPTY;
-        }
-    }
-
-    /**
-     * An immutable structure holding the current memtable, the memtables pending
-     * flush, the sstables for a column family, and the sstables that are active
-     * in compaction (a subset of the sstables).
-     */
-    public static class View
-    {
-        /**
-         * ordinarily a list of size 1, but when preparing to flush will contain both the memtable we will flush
-         * and the new replacement memtable, until all outstanding write operations on the old table complete.
-         * The last item in the list is always the "current" memtable.
-         */
-        private final List<Memtable> liveMemtables;
-        /**
-         * contains all memtables that are no longer referenced for writing and are queued for / in the process of being
-         * flushed. In chronologically ascending order.
-         */
-        private final List<Memtable> flushingMemtables;
-        public final Set<SSTableReader> compacting;
-        public final Set<SSTableReader> sstables;
-        // we use a Map here so that we can easily perform identity checks as well as equality checks.
-        // When marking compacting, we now  indicate if we expect the sstables to be present (by default we do),
-        // and we then check that not only are they all present in the live set, but that the exact instance present is
-        // the one we made our decision to compact against.
-        public final Map<SSTableReader, SSTableReader> sstablesMap;
-
-        // all sstables that are still in the live set, but have been completely shadowed by a replacement sstable
-        public final Set<SSTableReader> shadowed;
-        public final SSTableIntervalTree intervalTree;
-
-        View(List<Memtable> liveMemtables, List<Memtable> flushingMemtables, Map<SSTableReader, SSTableReader> sstables, Set<SSTableReader> compacting, Set<SSTableReader> shadowed, SSTableIntervalTree intervalTree)
-        {
-            this.shadowed = shadowed;
-            assert liveMemtables != null;
-            assert flushingMemtables != null;
-            assert sstables != null;
-            assert compacting != null;
-            assert intervalTree != null;
-
-            this.liveMemtables = liveMemtables;
-            this.flushingMemtables = flushingMemtables;
-
-            this.sstablesMap = sstables;
-            this.sstables = sstablesMap.keySet();
-            this.compacting = compacting;
-            this.intervalTree = intervalTree;
-        }
-
-        public Memtable getOldestMemtable()
-        {
-            if (!flushingMemtables.isEmpty())
-                return flushingMemtables.get(0);
-            return liveMemtables.get(0);
-        }
-
-        public Memtable getCurrentMemtable()
-        {
-            return liveMemtables.get(liveMemtables.size() - 1);
-        }
-
-        public Iterable<Memtable> getMemtablesPendingFlush()
-        {
-            if (liveMemtables.size() == 1)
-                return flushingMemtables;
-            return Iterables.concat(liveMemtables.subList(0, 1), flushingMemtables);
-        }
-
-        /**
-         * @return the active memtable and all the memtables that are pending flush.
-         */
-        public Iterable<Memtable> getAllMemtables()
-        {
-            return Iterables.concat(flushingMemtables, liveMemtables);
-        }
-
-        public Sets.SetView<SSTableReader> nonCompactingSStables()
-        {
-            return Sets.difference(ImmutableSet.copyOf(sstables), compacting);
-        }
-
-        View switchMemtable(Memtable newMemtable)
-        {
-            List<Memtable> newLiveMemtables = ImmutableList.<Memtable>builder().addAll(liveMemtables).add(newMemtable).build();
-            return new View(newLiveMemtables, flushingMemtables, sstablesMap, compacting, shadowed, intervalTree);
-        }
-
-        View markFlushing(Memtable toFlushMemtable)
-        {
-            List<Memtable> live = liveMemtables, flushing = flushingMemtables;
-
-            // since we can have multiple flushes queued, we may occasionally race and start a flush out of order,
-            // so must locate it in the list to remove, rather than just removing from the beginning
-            int i = live.indexOf(toFlushMemtable);
-            assert i < live.size() - 1;
-            List<Memtable> newLive = ImmutableList.<Memtable>builder()
-                                                  .addAll(live.subList(0, i))
-                                                  .addAll(live.subList(i + 1, live.size()))
-                                                  .build();
-
-            // similarly, if we out-of-order markFlushing once, we may afterwards need to insert a memtable into the
-            // flushing list in a position other than the end, though this will be rare
-            i = flushing.size();
-            while (i > 0 && flushing.get(i - 1).creationTime() > toFlushMemtable.creationTime())
-                i--;
-            List<Memtable> newFlushing = ImmutableList.<Memtable>builder()
-                                                      .addAll(flushing.subList(0, i))
-                                                      .add(toFlushMemtable)
-                                                      .addAll(flushing.subList(i, flushing.size()))
-                                                      .build();
-
-            return new View(newLive, newFlushing, sstablesMap, compacting, shadowed, intervalTree);
-        }
-
-        View replaceFlushed(Memtable flushedMemtable, SSTableReader newSSTable)
-        {
-            int index = flushingMemtables.indexOf(flushedMemtable);
-            List<Memtable> newQueuedMemtables = ImmutableList.<Memtable>builder()
-                                                             .addAll(flushingMemtables.subList(0, index))
-                                                             .addAll(flushingMemtables.subList(index + 1, flushingMemtables.size()))
-                                                             .build();
-            Map<SSTableReader, SSTableReader> newSSTables = sstablesMap;
-            SSTableIntervalTree intervalTree = this.intervalTree;
-            if (newSSTable != null)
-            {
-                assert !sstables.contains(newSSTable);
-                assert !shadowed.contains(newSSTable);
-                newSSTables = ImmutableMap.<SSTableReader, SSTableReader>builder()
-                                          .putAll(sstablesMap).put(newSSTable, newSSTable).build();
-                intervalTree = buildIntervalTree(newSSTables.keySet());
-            }
-            return new View(liveMemtables, newQueuedMemtables, newSSTables, compacting, shadowed, intervalTree);
-        }
-
-        View replace(Collection<SSTableReader> oldSSTables, Iterable<SSTableReader> replacements)
-        {
-            ImmutableSet<SSTableReader> oldSet = ImmutableSet.copyOf(oldSSTables);
-            int newSSTablesSize = shadowed.size() + sstables.size() - oldSSTables.size() + Iterables.size(replacements);
-            assert newSSTablesSize >= Iterables.size(replacements) : String.format("Incoherent new size %d replacing %s by %s in %s", newSSTablesSize, oldSSTables, replacements, this);
-            Map<SSTableReader, SSTableReader> newSSTables = new HashMap<>(newSSTablesSize);
-            Set<SSTableReader> newShadowed = new HashSet<>(shadowed.size());
-
-            for (SSTableReader sstable : sstables)
-                if (!oldSet.contains(sstable))
-                    newSSTables.put(sstable, sstable);
-
-            for (SSTableReader sstable : shadowed)
-                if (!oldSet.contains(sstable))
-                    newShadowed.add(sstable);
-
-            for (SSTableReader replacement : replacements)
-            {
-                if (replacement.openReason == SSTableReader.OpenReason.SHADOWED)
-                    newShadowed.add(replacement);
-                else
-                    newSSTables.put(replacement, replacement);
-            }
-
-            assert newSSTables.size() + newShadowed.size() == newSSTablesSize :
-                String.format("Expecting new size of %d, got %d while replacing %s by %s in %s",
-                          newSSTablesSize, newSSTables.size() + newShadowed.size(), oldSSTables, replacements, this);
-            newShadowed = ImmutableSet.copyOf(newShadowed);
-            newSSTables = ImmutableMap.copyOf(newSSTables);
-            SSTableIntervalTree intervalTree = buildIntervalTree(newSSTables.keySet());
-            return new View(liveMemtables, flushingMemtables, newSSTables, compacting, newShadowed, intervalTree);
-        }
-
-        View markCompacting(Collection<SSTableReader> tomark)
-        {
-            Set<SSTableReader> compactingNew = ImmutableSet.<SSTableReader>builder().addAll(compacting).addAll(tomark).build();
-            return new View(liveMemtables, flushingMemtables, sstablesMap, compactingNew, shadowed, intervalTree);
-        }
-
-        View unmarkCompacting(Iterable<SSTableReader> tounmark)
-        {
-            Set<SSTableReader> compactingNew = ImmutableSet.copyOf(Sets.difference(compacting, ImmutableSet.copyOf(tounmark)));
-            return new View(liveMemtables, flushingMemtables, sstablesMap, compactingNew, shadowed, intervalTree);
-        }
-
-        private Set<SSTableReader> newSSTables(Collection<SSTableReader> oldSSTables, Iterable<SSTableReader> replacements)
-        {
-            ImmutableSet<SSTableReader> oldSet = ImmutableSet.copyOf(oldSSTables);
-            int newSSTablesSize = sstables.size() - oldSSTables.size() + Iterables.size(replacements);
-            assert newSSTablesSize >= Iterables.size(replacements) : String.format("Incoherent new size %d replacing %s by %s in %s", newSSTablesSize, oldSSTables, replacements, this);
-            Set<SSTableReader> newSSTables = new HashSet<>(newSSTablesSize);
-
-            for (SSTableReader sstable : sstables)
-                if (!oldSet.contains(sstable))
-                    newSSTables.add(sstable);
-
-            Iterables.addAll(newSSTables, replacements);
-            assert newSSTables.size() == newSSTablesSize : String.format("Expecting new size of %d, got %d while replacing %s by %s in %s", newSSTablesSize, newSSTables.size(), oldSSTables, replacements, this);
-            return ImmutableSet.copyOf(newSSTables);
-        }
-
-        @Override
-        public String toString()
-        {
-            return String.format("View(pending_count=%d, sstables=%s, compacting=%s)", liveMemtables.size() + flushingMemtables.size() - 1, sstables, compacting);
-        }
-
-        public List<SSTableReader> sstablesInBounds(AbstractBounds<RowPosition> rowBounds)
-        {
-            return sstablesInBounds(rowBounds, intervalTree, liveMemtables.get(0).cfs.partitioner);
-        }
-
-        public static List<SSTableReader> sstablesInBounds(AbstractBounds<RowPosition> rowBounds, SSTableIntervalTree intervalTree, IPartitioner partitioner)
-        {
-            if (intervalTree.isEmpty())
-                return Collections.emptyList();
-            RowPosition stopInTree = rowBounds.right.isMinimum(partitioner) ? intervalTree.max() : rowBounds.right;
-            return intervalTree.search(Interval.<RowPosition, SSTableReader>create(rowBounds.left, stopInTree));
-        }
-    }
-}
diff --git a/src/java/org/apache/cassandra/db/DecoratedKey.java b/src/java/org/apache/cassandra/db/DecoratedKey.java
index 604cbb7..cc62a15 100644
--- a/src/java/org/apache/cassandra/db/DecoratedKey.java
+++ b/src/java/org/apache/cassandra/db/DecoratedKey.java
@@ -20,13 +20,12 @@
 import java.nio.ByteBuffer;
 import java.util.Comparator;
 
-import net.nicoulaj.compilecommand.annotations.Inline;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.dht.Token.KeyBound;
 import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.FastByteOperations;
-import org.apache.cassandra.utils.memory.MemoryUtil;
+import org.apache.cassandra.utils.MurmurHash;
+import org.apache.cassandra.utils.IFilter.FilterKey;
 
 /**
  * Represents a decorated key, handy for certain operations
@@ -37,7 +36,7 @@
  * if this matters, you can subclass RP to use a stronger hash, or use a non-lossy tokenization scheme (as in the
  * OrderPreservingPartitioner classes).
  */
-public abstract class DecoratedKey implements RowPosition
+public abstract class DecoratedKey implements RowPosition, FilterKey
 {
     public static final Comparator<DecoratedKey> comparator = new Comparator<DecoratedKey>()
     {
@@ -98,15 +97,20 @@
         return cmp == 0 ? ByteBufferUtil.compareUnsigned(key, otherKey.getKey()) : cmp;
     }
 
-    public boolean isMinimum(IPartitioner partitioner)
+    public IPartitioner getPartitioner()
     {
-        // A DecoratedKey can never be the minimum position on the ring
-        return false;
+        return getToken().getPartitioner();
+    }
+
+    public KeyBound minValue()
+    {
+        return getPartitioner().getMinimumToken().minKeyBound();
     }
 
     public boolean isMinimum()
     {
-        return isMinimum(StorageService.getPartitioner());
+        // A DecoratedKey can never be the minimum position on the ring
+        return false;
     }
 
     public RowPosition.Kind kind()
@@ -127,4 +131,10 @@
     }
 
     public abstract ByteBuffer getKey();
+
+    public void filterHash(long[] dest)
+    {
+        ByteBuffer key = getKey();
+        MurmurHash.hash3_x64_128(key, key.position(), key.remaining(), 0, dest);
+    }
 }
diff --git a/src/java/org/apache/cassandra/db/DefinitionsUpdateVerbHandler.java b/src/java/org/apache/cassandra/db/DefinitionsUpdateVerbHandler.java
index 5cb62ed..51d15b4 100644
--- a/src/java/org/apache/cassandra/db/DefinitionsUpdateVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/DefinitionsUpdateVerbHandler.java
@@ -26,6 +26,7 @@
 import org.apache.cassandra.concurrent.StageManager;
 import org.apache.cassandra.net.IVerbHandler;
 import org.apache.cassandra.net.MessageIn;
+import org.apache.cassandra.schema.LegacySchemaTables;
 import org.apache.cassandra.utils.WrappedRunnable;
 
 /**
@@ -40,13 +41,13 @@
 
     public void doVerb(final MessageIn<Collection<Mutation>> message, int id)
     {
-        logger.debug("Received schema mutation push from {}", message.from);
+        logger.trace("Received schema mutation push from {}", message.from);
 
         StageManager.getStage(Stage.MIGRATION).submit(new WrappedRunnable()
         {
             public void runMayThrow() throws Exception
             {
-                DefsTables.mergeSchema(message.payload);
+                LegacySchemaTables.mergeSchema(message.payload);
             }
         });
     }
diff --git a/src/java/org/apache/cassandra/db/DefsTables.java b/src/java/org/apache/cassandra/db/DefsTables.java
deleted file mode 100644
index 6bda277..0000000
--- a/src/java/org/apache/cassandra/db/DefsTables.java
+++ /dev/null
@@ -1,539 +0,0 @@
-/*
- * 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.db;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.*;
-
-import com.google.common.base.Function;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.MapDifference;
-import com.google.common.collect.Maps;
-import org.apache.cassandra.db.commitlog.CommitLog;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.KSMetaData;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.config.UTMetaData;
-import org.apache.cassandra.db.compaction.CompactionManager;
-import org.apache.cassandra.db.filter.QueryFilter;
-import org.apache.cassandra.db.marshal.AsciiType;
-import org.apache.cassandra.db.marshal.UserType;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.service.MigrationManager;
-import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-/**
- * SCHEMA_{KEYSPACES, COLUMNFAMILIES, COLUMNS}_CF are used to store Keyspace/ColumnFamily attributes to make schema
- * load/distribution easy, it replaces old mechanism when local migrations where serialized, stored in system.Migrations
- * and used for schema distribution.
- *
- * SCHEMA_KEYSPACES_CF layout:
- *
- * <key (AsciiType)>
- *   ascii => json_serialized_value
- *   ...
- * </key>
- *
- * Where <key> is a name of keyspace e.g. "ks".
- *
- * SCHEMA_COLUMNFAMILIES_CF layout:
- *
- * <key (AsciiType)>
- *     composite(ascii, ascii) => json_serialized_value
- * </key>
- *
- * Where <key> is a name of keyspace e.g. "ks"., first component of the column name is name of the ColumnFamily, last
- * component is the name of the ColumnFamily attribute.
- *
- * SCHEMA_COLUMNS_CF layout:
- *
- * <key (AsciiType)>
- *     composite(ascii, ascii, ascii) => json_serialized value
- * </key>
- *
- * Where <key> is a name of keyspace e.g. "ks".
- *
- * Cell names where made composite to support 3-level nesting which represents following structure:
- * "ColumnFamily name":"column name":"column attribute" => "value"
- *
- * Example of schema (using CLI):
- *
- * schema_keyspaces
- * ----------------
- * RowKey: ks
- *  => (column=durable_writes, value=true, timestamp=1327061028312185000)
- *  => (column=name, value="ks", timestamp=1327061028312185000)
- *  => (column=replication_factor, value=0, timestamp=1327061028312185000)
- *  => (column=strategy_class, value="org.apache.cassandra.locator.NetworkTopologyStrategy", timestamp=1327061028312185000)
- *  => (column=strategy_options, value={"datacenter1":"1"}, timestamp=1327061028312185000)
- *
- * schema_columnfamilies
- * ---------------------
- * RowKey: ks
- *  => (column=cf:bloom_filter_fp_chance, value=0.0, timestamp=1327061105833119000)
- *  => (column=cf:caching, value="NONE", timestamp=1327061105833119000)
- *  => (column=cf:column_type, value="Standard", timestamp=1327061105833119000)
- *  => (column=cf:comment, value="ColumnFamily", timestamp=1327061105833119000)
- *  => (column=cf:default_validation_class, value="org.apache.cassandra.db.marshal.BytesType", timestamp=1327061105833119000)
- *  => (column=cf:gc_grace_seconds, value=864000, timestamp=1327061105833119000)
- *  => (column=cf:id, value=1000, timestamp=1327061105833119000)
- *  => (column=cf:key_alias, value="S0VZ", timestamp=1327061105833119000)
- *  ... part of the output omitted.
- *
- * schema_columns
- * --------------
- * RowKey: ks
- *  => (column=cf:c:index_name, value=null, timestamp=1327061105833119000)
- *  => (column=cf:c:index_options, value=null, timestamp=1327061105833119000)
- *  => (column=cf:c:index_type, value=null, timestamp=1327061105833119000)
- *  => (column=cf:c:name, value="aGVsbG8=", timestamp=1327061105833119000)
- *  => (column=cf:c:validation_class, value="org.apache.cassandra.db.marshal.AsciiType", timestamp=1327061105833119000)
- */
-public class DefsTables
-{
-    private static final Logger logger = LoggerFactory.getLogger(DefsTables.class);
-
-    /**
-     * Load keyspace definitions for the system keyspace (system.SCHEMA_KEYSPACES_CF)
-     *
-     * @return Collection of found keyspace definitions
-     */
-    public static Collection<KSMetaData> loadFromKeyspace()
-    {
-        List<Row> serializedSchema = SystemKeyspace.serializedSchema(SystemKeyspace.SCHEMA_KEYSPACES_CF);
-
-        List<KSMetaData> keyspaces = new ArrayList<>(serializedSchema.size());
-
-        for (Row row : serializedSchema)
-        {
-            if (Schema.invalidSchemaRow(row) || Schema.ignoredSchemaRow(row))
-                continue;
-
-            keyspaces.add(KSMetaData.fromSchema(row, serializedColumnFamilies(row.key), serializedUserTypes(row.key)));
-        }
-
-        return keyspaces;
-    }
-
-    private static Row serializedColumnFamilies(DecoratedKey ksNameKey)
-    {
-        ColumnFamilyStore cfsStore = SystemKeyspace.schemaCFS(SystemKeyspace.SCHEMA_COLUMNFAMILIES_CF);
-        return new Row(ksNameKey, cfsStore.getColumnFamily(QueryFilter.getIdentityFilter(ksNameKey,
-                                                                                         SystemKeyspace.SCHEMA_COLUMNFAMILIES_CF,
-                                                                                         System.currentTimeMillis())));
-    }
-
-    private static Row serializedUserTypes(DecoratedKey ksNameKey)
-    {
-        ColumnFamilyStore cfsStore = SystemKeyspace.schemaCFS(SystemKeyspace.SCHEMA_USER_TYPES_CF);
-        return new Row(ksNameKey, cfsStore.getColumnFamily(QueryFilter.getIdentityFilter(ksNameKey,
-                                                                                         SystemKeyspace.SCHEMA_USER_TYPES_CF,
-                                                                                         System.currentTimeMillis())));
-    }
-
-    /**
-     * Merge remote schema in form of mutations with local and mutate ks/cf metadata objects
-     * (which also involves fs operations on add/drop ks/cf)
-     *
-     * @param mutations the schema changes to apply
-     *
-     * @throws ConfigurationException If one of metadata attributes has invalid value
-     * @throws IOException If data was corrupted during transportation or failed to apply fs operations
-     */
-    public static synchronized void mergeSchema(Collection<Mutation> mutations) throws ConfigurationException, IOException
-    {
-        mergeSchemaInternal(mutations, true);
-        Schema.instance.updateVersionAndAnnounce();
-    }
-
-    public static synchronized void mergeSchemaInternal(Collection<Mutation> mutations, boolean doFlush) throws ConfigurationException, IOException
-    {
-        // compare before/after schemas of the affected keyspaces only
-        Set<String> keyspaces = new HashSet<>(mutations.size());
-        for (Mutation mutation : mutations)
-            keyspaces.add(ByteBufferUtil.string(mutation.key()));
-
-        // current state of the schema
-        Map<DecoratedKey, ColumnFamily> oldKeyspaces = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_KEYSPACES_CF, keyspaces);
-        Map<DecoratedKey, ColumnFamily> oldColumnFamilies = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_COLUMNFAMILIES_CF, keyspaces);
-        Map<DecoratedKey, ColumnFamily> oldTypes = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_USER_TYPES_CF, keyspaces);
-
-        for (Mutation mutation : mutations)
-            mutation.apply();
-
-        if (doFlush && !StorageService.instance.isClientMode())
-            flushSchemaCFs();
-
-        // with new data applied
-        Map<DecoratedKey, ColumnFamily> newKeyspaces = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_KEYSPACES_CF, keyspaces);
-        Map<DecoratedKey, ColumnFamily> newColumnFamilies = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_COLUMNFAMILIES_CF, keyspaces);
-        Map<DecoratedKey, ColumnFamily> newTypes = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_USER_TYPES_CF, keyspaces);
-
-        Set<String> keyspacesToDrop = mergeKeyspaces(oldKeyspaces, newKeyspaces);
-        mergeColumnFamilies(oldColumnFamilies, newColumnFamilies);
-        mergeTypes(oldTypes, newTypes);
-
-        // it is safe to drop a keyspace only when all nested ColumnFamilies where deleted
-        for (String keyspaceToDrop : keyspacesToDrop)
-            dropKeyspace(keyspaceToDrop);
-    }
-
-    private static Set<String> mergeKeyspaces(Map<DecoratedKey, ColumnFamily> before, Map<DecoratedKey, ColumnFamily> after)
-    {
-        List<Row> created = new ArrayList<>();
-        List<String> altered = new ArrayList<>();
-        Set<String> dropped = new HashSet<>();
-
-        /*
-         * - we don't care about entriesOnlyOnLeft() or entriesInCommon(), because only the changes are of interest to us
-         * - of all entriesOnlyOnRight(), we only care about ones that have live columns; it's possible to have a ColumnFamily
-         *   there that only has the top-level deletion, if:
-         *      a) a pushed DROP KEYSPACE change for a keyspace hadn't ever made it to this node in the first place
-         *      b) a pulled dropped keyspace that got dropped before it could find a way to this node
-         * - of entriesDiffering(), we don't care about the scenario where both pre and post-values have zero live columns:
-         *   that means that a keyspace had been recreated and dropped, and the recreated keyspace had never found a way
-         *   to this node
-         */
-        MapDifference<DecoratedKey, ColumnFamily> diff = Maps.difference(before, after);
-
-        for (Map.Entry<DecoratedKey, ColumnFamily> entry : diff.entriesOnlyOnRight().entrySet())
-            if (entry.getValue().hasColumns())
-                created.add(new Row(entry.getKey(), entry.getValue()));
-
-        for (Map.Entry<DecoratedKey, MapDifference.ValueDifference<ColumnFamily>> entry : diff.entriesDiffering().entrySet())
-        {
-            String keyspaceName = AsciiType.instance.compose(entry.getKey().getKey());
-
-            ColumnFamily pre  = entry.getValue().leftValue();
-            ColumnFamily post = entry.getValue().rightValue();
-
-            if (pre.hasColumns() && post.hasColumns())
-                altered.add(keyspaceName);
-            else if (pre.hasColumns())
-                dropped.add(keyspaceName);
-            else if (post.hasColumns()) // a (re)created keyspace
-                created.add(new Row(entry.getKey(), post));
-        }
-
-        for (Row row : created)
-            addKeyspace(KSMetaData.fromSchema(row, Collections.<CFMetaData>emptyList(), new UTMetaData()));
-        for (String name : altered)
-            updateKeyspace(name);
-        return dropped;
-    }
-
-    // see the comments for mergeKeyspaces()
-    private static void mergeColumnFamilies(Map<DecoratedKey, ColumnFamily> before, Map<DecoratedKey, ColumnFamily> after)
-    {
-        List<CFMetaData> created = new ArrayList<>();
-        List<CFMetaData> altered = new ArrayList<>();
-        List<CFMetaData> dropped = new ArrayList<>();
-
-        MapDifference<DecoratedKey, ColumnFamily> diff = Maps.difference(before, after);
-
-        for (Map.Entry<DecoratedKey, ColumnFamily> entry : diff.entriesOnlyOnRight().entrySet())
-            if (entry.getValue().hasColumns())
-                created.addAll(KSMetaData.deserializeColumnFamilies(new Row(entry.getKey(), entry.getValue())).values());
-
-        for (Map.Entry<DecoratedKey, MapDifference.ValueDifference<ColumnFamily>> entry : diff.entriesDiffering().entrySet())
-        {
-            String keyspaceName = AsciiType.instance.compose(entry.getKey().getKey());
-
-            ColumnFamily pre  = entry.getValue().leftValue();
-            ColumnFamily post = entry.getValue().rightValue();
-
-            if (pre.hasColumns() && post.hasColumns())
-            {
-                MapDifference<String, CFMetaData> delta =
-                        Maps.difference(Schema.instance.getKSMetaData(keyspaceName).cfMetaData(),
-                                        KSMetaData.deserializeColumnFamilies(new Row(entry.getKey(), post)));
-
-                dropped.addAll(delta.entriesOnlyOnLeft().values());
-                created.addAll(delta.entriesOnlyOnRight().values());
-                Iterables.addAll(altered, Iterables.transform(delta.entriesDiffering().values(), new Function<MapDifference.ValueDifference<CFMetaData>, CFMetaData>()
-                {
-                    public CFMetaData apply(MapDifference.ValueDifference<CFMetaData> pair)
-                    {
-                        return pair.rightValue();
-                    }
-                }));
-            }
-            else if (pre.hasColumns())
-            {
-                dropped.addAll(Schema.instance.getKSMetaData(keyspaceName).cfMetaData().values());
-            }
-            else if (post.hasColumns())
-            {
-                created.addAll(KSMetaData.deserializeColumnFamilies(new Row(entry.getKey(), post)).values());
-            }
-        }
-
-        for (CFMetaData cfm : created)
-            addColumnFamily(cfm);
-        for (CFMetaData cfm : altered)
-            updateColumnFamily(cfm.ksName, cfm.cfName);
-        for (CFMetaData cfm : dropped)
-            dropColumnFamily(cfm.ksName, cfm.cfName);
-    }
-
-    // see the comments for mergeKeyspaces()
-    private static void mergeTypes(Map<DecoratedKey, ColumnFamily> before, Map<DecoratedKey, ColumnFamily> after)
-    {
-        List<UserType> created = new ArrayList<>();
-        List<UserType> altered = new ArrayList<>();
-        List<UserType> dropped = new ArrayList<>();
-
-        MapDifference<DecoratedKey, ColumnFamily> diff = Maps.difference(before, after);
-
-        // New keyspace with types
-        for (Map.Entry<DecoratedKey, ColumnFamily> entry : diff.entriesOnlyOnRight().entrySet())
-            if (entry.getValue().hasColumns())
-                created.addAll(UTMetaData.fromSchema(new Row(entry.getKey(), entry.getValue())).values());
-
-        for (Map.Entry<DecoratedKey, MapDifference.ValueDifference<ColumnFamily>> entry : diff.entriesDiffering().entrySet())
-        {
-            String keyspaceName = AsciiType.instance.compose(entry.getKey().getKey());
-
-            ColumnFamily pre  = entry.getValue().leftValue();
-            ColumnFamily post = entry.getValue().rightValue();
-
-            if (pre.hasColumns() && post.hasColumns())
-            {
-                MapDifference<ByteBuffer, UserType> delta =
-                        Maps.difference(Schema.instance.getKSMetaData(keyspaceName).userTypes.getAllTypes(),
-                                        UTMetaData.fromSchema(new Row(entry.getKey(), post)));
-
-                dropped.addAll(delta.entriesOnlyOnLeft().values());
-                created.addAll(delta.entriesOnlyOnRight().values());
-                Iterables.addAll(altered, Iterables.transform(delta.entriesDiffering().values(), new Function<MapDifference.ValueDifference<UserType>, UserType>()
-                {
-                    public UserType apply(MapDifference.ValueDifference<UserType> pair)
-                    {
-                        return pair.rightValue();
-                    }
-                }));
-            }
-            else if (pre.hasColumns())
-            {
-                dropped.addAll(Schema.instance.getKSMetaData(keyspaceName).userTypes.getAllTypes().values());
-            }
-            else if (post.hasColumns())
-            {
-                created.addAll(UTMetaData.fromSchema(new Row(entry.getKey(), post)).values());
-            }
-        }
-
-        for (UserType type : created)
-            addType(type);
-        for (UserType type : altered)
-            updateType(type);
-        for (UserType type : dropped)
-            dropType(type);
-    }
-
-    private static void addKeyspace(KSMetaData ksm)
-    {
-        assert Schema.instance.getKSMetaData(ksm.name) == null;
-        Schema.instance.load(ksm);
-
-        if (!StorageService.instance.isClientMode())
-        {
-            Keyspace.open(ksm.name);
-            MigrationManager.instance.notifyCreateKeyspace(ksm);
-        }
-    }
-
-    private static void addColumnFamily(CFMetaData cfm)
-    {
-        assert Schema.instance.getCFMetaData(cfm.ksName, cfm.cfName) == null;
-        KSMetaData ksm = Schema.instance.getKSMetaData(cfm.ksName);
-        ksm = KSMetaData.cloneWith(ksm, Iterables.concat(ksm.cfMetaData().values(), Collections.singleton(cfm)));
-
-        logger.info("Loading {}", cfm);
-
-        Schema.instance.load(cfm);
-
-        // make sure it's init-ed w/ the old definitions first,
-        // since we're going to call initCf on the new one manually
-        Keyspace.open(cfm.ksName);
-
-        Schema.instance.setKeyspaceDefinition(ksm);
-
-        if (!StorageService.instance.isClientMode())
-        {
-            Keyspace.open(ksm.name).initCf(cfm.cfId, cfm.cfName, true);
-            MigrationManager.instance.notifyCreateColumnFamily(cfm);
-        }
-    }
-
-    private static void addType(UserType ut)
-    {
-        KSMetaData ksm = Schema.instance.getKSMetaData(ut.keyspace);
-        assert ksm != null;
-
-        logger.info("Loading {}", ut);
-
-        ksm.userTypes.addType(ut);
-
-        if (!StorageService.instance.isClientMode())
-            MigrationManager.instance.notifyCreateUserType(ut);
-    }
-
-    private static void updateKeyspace(String ksName)
-    {
-        KSMetaData oldKsm = Schema.instance.getKSMetaData(ksName);
-        assert oldKsm != null;
-        KSMetaData newKsm = KSMetaData.cloneWith(oldKsm.reloadAttributes(), oldKsm.cfMetaData().values());
-
-        Schema.instance.setKeyspaceDefinition(newKsm);
-
-        if (!StorageService.instance.isClientMode())
-        {
-            Keyspace.open(ksName).setMetadata(newKsm);
-            MigrationManager.instance.notifyUpdateKeyspace(newKsm);
-        }
-    }
-
-    private static void updateColumnFamily(String ksName, String cfName)
-    {
-        CFMetaData cfm = Schema.instance.getCFMetaData(ksName, cfName);
-        assert cfm != null;
-        boolean columnsDidChange = cfm.reload();
-
-        if (!StorageService.instance.isClientMode())
-        {
-            Keyspace keyspace = Keyspace.open(cfm.ksName);
-            keyspace.getColumnFamilyStore(cfm.cfName).reload();
-            MigrationManager.instance.notifyUpdateColumnFamily(cfm, columnsDidChange);
-        }
-    }
-
-    private static void updateType(UserType ut)
-    {
-        KSMetaData ksm = Schema.instance.getKSMetaData(ut.keyspace);
-        assert ksm != null;
-
-        logger.info("Updating {}", ut);
-
-        ksm.userTypes.addType(ut);
-
-        if (!StorageService.instance.isClientMode())
-            MigrationManager.instance.notifyUpdateUserType(ut);
-    }
-
-    private static void dropKeyspace(String ksName)
-    {
-        KSMetaData ksm = Schema.instance.getKSMetaData(ksName);
-        String snapshotName = Keyspace.getTimestampedSnapshotName(ksName);
-
-        CompactionManager.instance.interruptCompactionFor(ksm.cfMetaData().values(), true);
-
-        Keyspace keyspace = Keyspace.open(ksm.name);
-
-        // remove all cfs from the keyspace instance.
-        List<UUID> droppedCfs = new ArrayList<>();
-        for (CFMetaData cfm : ksm.cfMetaData().values())
-        {
-            ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfm.cfName);
-
-            Schema.instance.purge(cfm);
-
-            if (!StorageService.instance.isClientMode())
-            {
-                if (DatabaseDescriptor.isAutoSnapshot())
-                    cfs.snapshot(snapshotName);
-                Keyspace.open(ksm.name).dropCf(cfm.cfId);
-            }
-
-            droppedCfs.add(cfm.cfId);
-        }
-
-        // remove the keyspace from the static instances.
-        Keyspace.clear(ksm.name);
-        Schema.instance.clearKeyspaceDefinition(ksm);
-
-        keyspace.writeOrder.awaitNewBarrier();
-
-        // force a new segment in the CL
-        CommitLog.instance.forceRecycleAllSegments(droppedCfs);
-
-        if (!StorageService.instance.isClientMode())
-        {
-            MigrationManager.instance.notifyDropKeyspace(ksm);
-        }
-    }
-
-    private static void dropColumnFamily(String ksName, String cfName)
-    {
-        KSMetaData ksm = Schema.instance.getKSMetaData(ksName);
-        assert ksm != null;
-        ColumnFamilyStore cfs = Keyspace.open(ksName).getColumnFamilyStore(cfName);
-        assert cfs != null;
-
-        // reinitialize the keyspace.
-        CFMetaData cfm = ksm.cfMetaData().get(cfName);
-
-        Schema.instance.purge(cfm);
-        Schema.instance.setKeyspaceDefinition(makeNewKeyspaceDefinition(ksm, cfm));
-
-        CompactionManager.instance.interruptCompactionFor(Arrays.asList(cfm), true);
-
-        if (!StorageService.instance.isClientMode())
-        {
-            if (DatabaseDescriptor.isAutoSnapshot())
-                cfs.snapshot(Keyspace.getTimestampedSnapshotName(cfs.name));
-            Keyspace.open(ksm.name).dropCf(cfm.cfId);
-            MigrationManager.instance.notifyDropColumnFamily(cfm);
-
-            CommitLog.instance.forceRecycleAllSegments(Collections.singleton(cfm.cfId));
-        }
-    }
-
-    private static void dropType(UserType ut)
-    {
-        KSMetaData ksm = Schema.instance.getKSMetaData(ut.keyspace);
-        assert ksm != null;
-
-        ksm.userTypes.removeType(ut);
-
-        if (!StorageService.instance.isClientMode())
-            MigrationManager.instance.notifyDropUserType(ut);
-    }
-
-    private static KSMetaData makeNewKeyspaceDefinition(KSMetaData ksm, CFMetaData toExclude)
-    {
-        // clone ksm but do not include the new def
-        List<CFMetaData> newCfs = new ArrayList<>(ksm.cfMetaData().values());
-        newCfs.remove(toExclude);
-        assert newCfs.size() == ksm.cfMetaData().size() - 1;
-        return KSMetaData.cloneWith(ksm, newCfs);
-    }
-
-    private static void flushSchemaCFs()
-    {
-        for (String cf : SystemKeyspace.allSchemaCfs)
-            SystemKeyspace.forceBlockingFlush(cf);
-    }
-}
-
diff --git a/src/java/org/apache/cassandra/db/DeletionInfo.java b/src/java/org/apache/cassandra/db/DeletionInfo.java
index 193f8b1..048324a 100644
--- a/src/java/org/apache/cassandra/db/DeletionInfo.java
+++ b/src/java/org/apache/cassandra/db/DeletionInfo.java
@@ -34,7 +34,6 @@
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.ObjectSizes;
 import org.apache.cassandra.utils.memory.AbstractAllocator;
-import org.apache.cassandra.utils.memory.MemtableAllocator;
 
 /**
  * A combination of a top-level (or row) tombstone and range tombstones describing the deletions
@@ -148,7 +147,7 @@
     /**
      * Returns a new {@link InOrderTester} in forward order.
      */
-    InOrderTester inOrderTester()
+    public InOrderTester inOrderTester()
     {
         return inOrderTester(false);
     }
diff --git a/src/java/org/apache/cassandra/db/Directories.java b/src/java/org/apache/cassandra/db/Directories.java
index 35aa447..2b3662f 100644
--- a/src/java/org/apache/cassandra/db/Directories.java
+++ b/src/java/org/apache/cassandra/db/Directories.java
@@ -48,31 +48,40 @@
 import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.io.sstable.*;
-import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Pair;
 
 /**
  * Encapsulate handling of paths to the data files.
  *
- * Since v2.1, the directory layout is the following:
- *   /<path_to_data_dir>/ks/cf1-cfId/ks-cf1-ka-1-Data.db
- *                         /cf2-cfId/ks-cf2-ka-1-Data.db
+ * <pre> {@code
+ *   /<path_to_data_dir>/ks/<cf dir>/ks-cf1-jb-1-Data.db
+ *                         /<cf dir>/la-2-Data.db
+ *                         /<cf dir>/.<index name>/ks-cf1.idx-jb-1-Data.db
+ *                         /<cf dir>/.<index name>/la-1-Data.db
  *                         ...
+ * } </pre>
  *
- * cfId is an hex encoded CFID.
+ * Until v2.0, {@code <cf dir>} is just column family name.
+ * Since v2.1, {@code <cf dir>} has column family ID(cfId) added to its end.
  *
- * For backward compatibility, Directories uses older directory layout if exists.
+ * SSTables from secondary indexes were put in the same directory as their parent.
+ * Since v2.2, they have their own directory under the parent directory whose name is index name.
+ * Upon startup, those secondary index files are moved to new directory when upgrading.
+ *
+ * For backward compatibility, Directories can use directory without cfId if exists.
  *
  * In addition, more that one 'root' data directory can be specified so that
- * <path_to_data_dir> potentially represents multiple locations.
+ * {@code <path_to_data_dir>} potentially represents multiple locations.
  * Note that in the case of multiple locations, the manifest for the leveled
  * compaction is only in one of the location.
  *
- * Snapshots (resp. backups) are always created along the sstables thare are
- * snapshoted (resp. backuped) but inside a subdirectory named 'snapshots'
- * (resp. backups) (and snapshots are furter inside a subdirectory of the name
- * of the snapshot).
+ * Snapshots (resp. backups) are always created along the sstables there are
+ * snapshotted (resp. backuped) but inside a subdirectory named 'snapshots'
+ * (resp. backups) (and snapshots are further inside a subdirectory of the name
+ * of the snapshot). For secondary indexes, snapshots (backups) are not created in
+ * their own directory, but are in their parent's snapshot (backup) directory.
  *
  * This class abstracts all those details from the rest of the code.
  */
@@ -175,28 +184,22 @@
      *
      * @param metadata metadata of ColumnFamily
      */
-    public Directories(CFMetaData metadata)
+    public Directories(final CFMetaData metadata)
     {
         this.metadata = metadata;
-        if (StorageService.instance.isClientMode())
-        {
-            dataPaths = null;
-            return;
-        }
 
         String cfId = ByteBufferUtil.bytesToHex(ByteBufferUtil.bytes(metadata.cfId));
         int idx = metadata.cfName.indexOf(SECONDARY_INDEX_NAME_SEPARATOR);
-        // secondary indicies go in the same directory as the base cf
-        String directoryName = idx > 0 ? metadata.cfName.substring(0, idx) + "-" + cfId : metadata.cfName + "-" + cfId;
+        String cfName = idx >= 0 ? metadata.cfName.substring(0, idx) : metadata.cfName;
+        String indexNameWithDot = idx >= 0 ? metadata.cfName.substring(idx) : null;
 
         this.dataPaths = new File[dataDirectories.length];
         // If upgraded from version less than 2.1, use existing directories
+        String oldSSTableRelativePath = join(metadata.ksName, cfName);
         for (int i = 0; i < dataDirectories.length; ++i)
         {
             // check if old SSTable directory exists
-            dataPaths[i] = new File(dataDirectories[i].location,
-                                    join(metadata.ksName,
-                                         idx > 0 ? metadata.cfName.substring(0, idx) : metadata.cfName));
+            dataPaths[i] = new File(dataDirectories[i].location, oldSSTableRelativePath);
         }
         boolean olderDirectoryExists = Iterables.any(Arrays.asList(dataPaths), new Predicate<File>()
         {
@@ -207,9 +210,16 @@
         });
         if (!olderDirectoryExists)
         {
-            // use 2.1-style path names
+            // use 2.1+ style
+            String newSSTableRelativePath = join(metadata.ksName, cfName + '-' + cfId);
             for (int i = 0; i < dataDirectories.length; ++i)
-                dataPaths[i] = new File(dataDirectories[i].location, join(metadata.ksName, directoryName));
+                dataPaths[i] = new File(dataDirectories[i].location, newSSTableRelativePath);
+        }
+        // if index, then move to its own directory
+        if (indexNameWithDot != null)
+        {
+            for (int i = 0; i < dataDirectories.length; ++i)
+                dataPaths[i] = new File(dataPaths[i], indexNameWithDot);
         }
 
         for (File dir : dataPaths)
@@ -225,6 +235,34 @@
                 FileUtils.handleFSError(e);
             }
         }
+
+        // if index, move existing older versioned SSTable files to new directory
+        if (indexNameWithDot != null)
+        {
+            for (File dataPath : dataPaths)
+            {
+                File[] indexFiles = dataPath.getParentFile().listFiles(new FileFilter()
+                {
+                    @Override
+                    public boolean accept(File file)
+                    {
+                        if (file.isDirectory())
+                            return false;
+
+                        Pair<Descriptor, Component> pair = SSTable.tryComponentFromFilename(file.getParentFile(),
+                                                                                            file.getName());
+                        return pair != null && pair.left.ksname.equals(metadata.ksName) && pair.left.cfname.equals(metadata.cfName);
+
+                    }
+                });
+                for (File indexFile : indexFiles)
+                {
+                    File destFile = new File(dataPath, indexFile.getName());
+                    logger.trace("Moving index file {} to {}", indexFile, destFile);
+                    FileUtils.renameWithConfirm(indexFile, destFile);
+                }
+            }
+        }
     }
 
     /**
@@ -291,14 +329,14 @@
         {
             if (BlacklistedDirectories.isUnwritable(getLocationForDisk(dataDir)))
             {
-                logger.debug("removing blacklisted candidate {}", dataDir.location);
+                logger.trace("removing blacklisted candidate {}", dataDir.location);
                 continue;
             }
             DataDirectoryCandidate candidate = new DataDirectoryCandidate(dataDir);
             // exclude directory if its total writeSize does not fit to data directory
             if (candidate.availableSpace < writeSize)
             {
-                logger.debug("removing candidate {}, usable={}, requested={}", candidate.dataDirectory.location, candidate.availableSpace, writeSize);
+                logger.trace("removing candidate {}, usable={}, requested={}", candidate.dataDirectory.location, candidate.availableSpace, writeSize);
                 tooBig = true;
                 continue;
             }
@@ -308,7 +346,7 @@
 
         if (candidates.isEmpty())
             if (tooBig)
-                return null;
+                throw new RuntimeException("Insufficient disk space to write " + writeSize + " bytes");
             else
                 throw new FSWriteError(new IOException("All configured data directories have been blacklisted as unwritable for erroring out"), "");
 
@@ -368,12 +406,36 @@
 
     public static File getSnapshotDirectory(Descriptor desc, String snapshotName)
     {
-        return getOrCreate(desc.directory, SNAPSHOT_SUBDIR, snapshotName);
+        return getSnapshotDirectory(desc.directory, snapshotName);
+    }
+
+    /**
+     * Returns directory to write snapshot. If directory does not exist, then one is created.
+     *
+     * If given {@code location} indicates secondary index, this will return
+     * {@code <cf dir>/snapshots/<snapshot name>/.<index name>}.
+     * Otherwise, this will return {@code <cf dir>/snapshots/<snapshot name>}.
+     *
+     * @param location base directory
+     * @param snapshotName snapshot name
+     * @return directory to write snapshot
+     */
+    public static File getSnapshotDirectory(File location, String snapshotName)
+    {
+        if (location.getName().startsWith(SECONDARY_INDEX_NAME_SEPARATOR))
+        {
+            return getOrCreate(location.getParentFile(), SNAPSHOT_SUBDIR, snapshotName, location.getName());
+        }
+        else
+        {
+            return getOrCreate(location, SNAPSHOT_SUBDIR, snapshotName);
+        }
     }
 
     public File getSnapshotManifestFile(String snapshotName)
     {
-         return new File(getDirectoryForNewSSTables(), join(SNAPSHOT_SUBDIR, snapshotName, "manifest.json"));
+        File snapshotDir = getSnapshotDirectory(getDirectoryForNewSSTables(), snapshotName);
+        return new File(snapshotDir, "manifest.json");
     }
 
     public File getNewEphemeralSnapshotMarkerFile(String snapshotName)
@@ -389,7 +451,19 @@
 
     public static File getBackupsDirectory(Descriptor desc)
     {
-        return getOrCreate(desc.directory, BACKUPS_SUBDIR);
+        return getBackupsDirectory(desc.directory);
+    }
+
+    public static File getBackupsDirectory(File location)
+    {
+        if (location.getName().startsWith(SECONDARY_INDEX_NAME_SEPARATOR))
+        {
+            return getOrCreate(location.getParentFile(), BACKUPS_SUBDIR, location.getName());
+        }
+        else
+        {
+            return getOrCreate(location, BACKUPS_SUBDIR);
+        }
     }
 
     public SSTableLister sstableLister()
@@ -519,7 +593,7 @@
 
                 if (snapshotName != null)
                 {
-                    new File(location, join(SNAPSHOT_SUBDIR, snapshotName)).listFiles(getFilter());
+                    getSnapshotDirectory(location, snapshotName).listFiles(getFilter());
                     continue;
                 }
 
@@ -527,28 +601,29 @@
                     location.listFiles(getFilter());
 
                 if (includeBackups)
-                    new File(location, BACKUPS_SUBDIR).listFiles(getFilter());
+                    getBackupsDirectory(location).listFiles(getFilter());
             }
             filtered = true;
         }
 
         private FileFilter getFilter()
         {
-            // Note: the prefix needs to include cfname + separator to distinguish between a cfs and it's secondary indexes
-            final String sstablePrefix = getSSTablePrefix();
             return new FileFilter()
             {
                 // This function always return false since accepts adds to the components map
                 public boolean accept(File file)
                 {
-                    // we are only interested in the SSTable files that belong to the specific ColumnFamily
-                    if (file.isDirectory() || !file.getName().startsWith(sstablePrefix))
+                    if (file.isDirectory())
                         return false;
 
                     Pair<Descriptor, Component> pair = SSTable.tryComponentFromFilename(file.getParentFile(), file.getName());
                     if (pair == null)
                         return false;
 
+                    // we are only interested in the SSTable files that belong to the specific ColumnFamily
+                    if (!pair.left.ksname.equals(metadata.ksName) || !pair.left.cfname.equals(metadata.cfName))
+                        return false;
+
                     if (skipTemporary && pair.left.type.isTemporary)
                         return false;
 
@@ -605,7 +680,9 @@
         final List<File> snapshots = new LinkedList<>();
         for (final File dir : dataPaths)
         {
-            final File snapshotDir = new File(dir,SNAPSHOT_SUBDIR);
+            File snapshotDir = dir.getName().startsWith(SECONDARY_INDEX_NAME_SEPARATOR) ?
+                                       new File(dir.getParent(), SNAPSHOT_SUBDIR) :
+                                       new File(dir, SNAPSHOT_SUBDIR);
             if (snapshotDir.exists() && snapshotDir.isDirectory())
             {
                 final File[] snapshotDirs  = snapshotDir.listFiles();
@@ -627,7 +704,15 @@
     {
         for (File dir : dataPaths)
         {
-            File snapshotDir = new File(dir, join(SNAPSHOT_SUBDIR, snapshotName));
+            File snapshotDir;
+            if (dir.getName().startsWith(SECONDARY_INDEX_NAME_SEPARATOR))
+            {
+                snapshotDir = new File(dir.getParentFile(), join(SNAPSHOT_SUBDIR, snapshotName, dir.getName()));
+            }
+            else
+            {
+                snapshotDir = new File(dir, join(SNAPSHOT_SUBDIR, snapshotName));
+            }
             if (snapshotDir.exists())
                 return true;
         }
@@ -643,8 +728,18 @@
             File snapshotDir = new File(dir, join(SNAPSHOT_SUBDIR, tag));
             if (snapshotDir.exists())
             {
-                logger.debug("Removing snapshot directory {}", snapshotDir);
-                FileUtils.deleteRecursive(snapshotDir);
+                logger.trace("Removing snapshot directory {}", snapshotDir);
+                try
+                {
+                    FileUtils.deleteRecursive(snapshotDir);
+                }
+                catch (FSWriteError e)
+                {
+                    if (FBUtilities.isWindows())
+                        SnapshotDeletingTask.addFailedSnapshot(snapshotDir);
+                    else
+                        throw e;
+                }
             }
         }
     }
@@ -654,31 +749,34 @@
     {
         for (File dir : dataPaths)
         {
-            File snapshotDir = new File(dir, join(SNAPSHOT_SUBDIR, snapshotName));
+            File snapshotDir = getSnapshotDirectory(dir, snapshotName);
             if (snapshotDir.exists())
                 return snapshotDir.lastModified();
         }
         throw new RuntimeException("Snapshot " + snapshotName + " doesn't exist");
     }
-    
+
+    /**
+     * @return total snapshot size in byte for all snapshots.
+     */
     public long trueSnapshotsSize()
     {
         long result = 0L;
         for (File dir : dataPaths)
-            result += getTrueAllocatedSizeIn(new File(dir, join(SNAPSHOT_SUBDIR)));
+        {
+            File snapshotDir = dir.getName().startsWith(SECONDARY_INDEX_NAME_SEPARATOR) ?
+                                       new File(dir.getParent(), SNAPSHOT_SUBDIR) :
+                                       new File(dir, SNAPSHOT_SUBDIR);
+            result += getTrueAllocatedSizeIn(snapshotDir);
+        }
         return result;
     }
 
-    private String getSSTablePrefix()
-    {
-        return metadata.ksName + Component.separator + metadata.cfName + Component.separator;
-    }
-
     public long getTrueAllocatedSizeIn(File input)
     {
         if (!input.isDirectory())
             return 0;
-        
+
         TrueFilesSizeVisitor visitor = new TrueFilesSizeVisitor();
         try
         {
@@ -688,7 +786,7 @@
         {
             logger.error("Could not calculate the size of {}. {}", input, e);
         }
-    
+
         return visitor.getAllocatedSize();
     }
 
@@ -762,21 +860,23 @@
         private final AtomicLong size = new AtomicLong(0);
         private final Set<String> visited = newHashSet(); //count each file only once
         private final Set<String> alive;
-        private final String prefix = getSSTablePrefix();
 
-        public TrueFilesSizeVisitor()
+        TrueFilesSizeVisitor()
         {
             super();
             Builder<String> builder = ImmutableSet.builder();
-            for (File file: sstableLister().listFiles())
+            for (File file : sstableLister().listFiles())
                 builder.add(file.getName());
             alive = builder.build();
         }
 
         private boolean isAcceptable(Path file)
         {
-            String fileName = file.toFile().getName(); 
-            return fileName.startsWith(prefix)
+            String fileName = file.toFile().getName();
+            Pair<Descriptor, Component> pair = SSTable.tryComponentFromFilename(file.getParent().toFile(), fileName);
+            return pair != null
+                    && pair.left.ksname.equals(metadata.ksName)
+                    && pair.left.cfname.equals(metadata.cfName)
                     && !visited.contains(fileName)
                     && !alive.contains(fileName);
         }
diff --git a/src/java/org/apache/cassandra/db/HintedHandOffManager.java b/src/java/org/apache/cassandra/db/HintedHandOffManager.java
index 0d3ef39..74f0a72 100644
--- a/src/java/org/apache/cassandra/db/HintedHandOffManager.java
+++ b/src/java/org/apache/cassandra/db/HintedHandOffManager.java
@@ -40,7 +40,6 @@
 import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
 import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.compaction.CompactionManager;
@@ -53,6 +52,7 @@
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.exceptions.WriteFailureException;
 import org.apache.cassandra.exceptions.WriteTimeoutException;
 import org.apache.cassandra.gms.ApplicationState;
 import org.apache.cassandra.gms.FailureDetector;
@@ -124,7 +124,7 @@
             new NamedThreadFactory("HintedHandoff", Thread.MIN_PRIORITY),
             "internal");
 
-    private final ColumnFamilyStore hintStore = Keyspace.open(Keyspace.SYSTEM_KS).getColumnFamilyStore(SystemKeyspace.HINTS_CF);
+    private final ColumnFamilyStore hintStore = Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStore(SystemKeyspace.HINTS);
 
     /**
      * Returns a mutation representing a Hint to be sent to <code>targetId</code>
@@ -141,11 +141,11 @@
 
         UUID hintId = UUIDGen.getTimeUUID();
         // serialize the hint with id and version as a composite column name
-        CellName name = CFMetaData.HintsCf.comparator.makeCellName(hintId, MessagingService.current_version);
+        CellName name = SystemKeyspace.Hints.comparator.makeCellName(hintId, MessagingService.current_version);
         ByteBuffer value = ByteBuffer.wrap(FBUtilities.serialize(mutation, Mutation.serializer, MessagingService.current_version));
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(Schema.instance.getCFMetaData(Keyspace.SYSTEM_KS, SystemKeyspace.HINTS_CF));
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(Schema.instance.getCFMetaData(SystemKeyspace.NAME, SystemKeyspace.HINTS));
         cf.addColumn(name, value, now, ttl);
-        return new Mutation(Keyspace.SYSTEM_KS, UUIDType.instance.decompose(targetId), cf);
+        return new Mutation(SystemKeyspace.NAME, UUIDType.instance.decompose(targetId), cf);
     }
 
     /*
@@ -173,7 +173,7 @@
         {
             throw new RuntimeException(e);
         }
-        logger.debug("Created HHOM instance, registered MBean.");
+        logger.trace("Created HHOM instance, registered MBean.");
 
         Runnable runnable = new Runnable()
         {
@@ -188,8 +188,8 @@
 
     private static void deleteHint(ByteBuffer tokenBytes, CellName columnName, long timestamp)
     {
-        Mutation mutation = new Mutation(Keyspace.SYSTEM_KS, tokenBytes);
-        mutation.delete(SystemKeyspace.HINTS_CF, columnName, timestamp);
+        Mutation mutation = new Mutation(SystemKeyspace.NAME, tokenBytes);
+        mutation.delete(SystemKeyspace.HINTS, columnName, timestamp);
         mutation.applyUnsafe(); // don't bother with commitlog since we're going to flush as soon as we're done with delivery
     }
 
@@ -217,8 +217,8 @@
             return;
 
         ByteBuffer hostIdBytes = ByteBuffer.wrap(UUIDGen.decompose(hostId));
-        final Mutation mutation = new Mutation(Keyspace.SYSTEM_KS, hostIdBytes);
-        mutation.delete(SystemKeyspace.HINTS_CF, System.currentTimeMillis());
+        final Mutation mutation = new Mutation(SystemKeyspace.NAME, hostIdBytes);
+        mutation.delete(SystemKeyspace.HINTS, System.currentTimeMillis());
 
         // execute asynchronously to avoid blocking caller (which may be processing gossip)
         Runnable runnable = new Runnable()
@@ -252,7 +252,7 @@
                 try
                 {
                     logger.info("Truncating all stored hints.");
-                    Keyspace.open(Keyspace.SYSTEM_KS).getColumnFamilyStore(SystemKeyspace.HINTS_CF).truncateBlocking();
+                    Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStore(SystemKeyspace.HINTS).truncateBlocking();
                 }
                 catch (Exception e)
                 {
@@ -267,7 +267,7 @@
     protected synchronized void compact()
     {
         ArrayList<Descriptor> descriptors = new ArrayList<>();
-        for (SSTable sstable : hintStore.getDataTracker().getUncompactingSSTables())
+        for (SSTable sstable : hintStore.getTracker().getUncompacting())
             descriptors.add(sstable.descriptor);
 
         if (descriptors.isEmpty())
@@ -319,7 +319,7 @@
         }
         if (gossiper.getEndpointStateForEndpoint(endpoint) == null)
             throw new TimeoutException("Node " + endpoint + " vanished while waiting for agreement");
-        logger.debug("schema for {} matches local schema", endpoint);
+        logger.trace("schema for {} matches local schema", endpoint);
         return waited;
     }
 
@@ -331,11 +331,11 @@
         // check if hints delivery has been paused
         if (hintedHandOffPaused)
         {
-            logger.debug("Hints delivery process is paused, aborting");
+            logger.trace("Hints delivery process is paused, aborting");
             return;
         }
 
-        logger.debug("Checking remote({}) schema before delivering hints", endpoint);
+        logger.trace("Checking remote({}) schema before delivering hints", endpoint);
         try
         {
             waitForSchemaAgreement(endpoint);
@@ -347,7 +347,7 @@
 
         if (!FailureDetector.instance.isAlive(endpoint))
         {
-            logger.debug("Endpoint {} died before hint delivery, aborting", endpoint);
+            logger.trace("Endpoint {} died before hint delivery, aborting", endpoint);
             return;
         }
 
@@ -372,7 +372,7 @@
         Composite startColumn = Composites.EMPTY;
 
         int pageSize = calculatePageSize();
-        logger.debug("Using pageSize of {}", pageSize);
+        logger.trace("Using pageSize of {}", pageSize);
 
         // rate limit is in bytes per second. Uses Double.MAX_VALUE if disabled (set to 0 in cassandra.yaml).
         // max rate is scaled by the number of nodes in the cluster (CASSANDRA-5272).
@@ -385,7 +385,7 @@
         {
             long now = System.currentTimeMillis();
             QueryFilter filter = QueryFilter.getSliceFilter(epkey,
-                                                            SystemKeyspace.HINTS_CF,
+                                                            SystemKeyspace.HINTS,
                                                             startColumn,
                                                             Composites.EMPTY,
                                                             false,
@@ -407,13 +407,13 @@
                 break;
             }
 
-            List<WriteResponseHandler> responseHandlers = Lists.newArrayList();
+            List<WriteResponseHandler<Mutation>> responseHandlers = Lists.newArrayList();
             for (final Cell hint : hintsPage)
             {
                 // check if hints delivery has been paused during the process
                 if (hintedHandOffPaused)
                 {
-                    logger.debug("Hints delivery process is paused, aborting");
+                    logger.trace("Hints delivery process is paused, aborting");
                     break delivery;
                 }
 
@@ -436,7 +436,7 @@
                 }
                 catch (UnknownColumnFamilyException e)
                 {
-                    logger.debug("Skipping delivery of hint for deleted columnfamily", e);
+                    logger.trace("Skipping delivery of hint for deleted table", e);
                     deleteHint(hostIdBytes, hint.name(), hint.timestamp());
                     continue;
                 }
@@ -449,7 +449,7 @@
                 {
                     if (hint.timestamp() <= SystemKeyspace.getTruncatedAt(cfId))
                     {
-                        logger.debug("Skipping delivery of hint for truncated columnfamily {}", cfId);
+                        logger.trace("Skipping delivery of hint for truncated table {}", cfId);
                         mutation = mutation.without(cfId);
                     }
                 }
@@ -470,20 +470,21 @@
                         deleteHint(hostIdBytes, hint.name(), hint.timestamp());
                     }
                 };
-                WriteResponseHandler responseHandler = new WriteResponseHandler(endpoint, WriteType.SIMPLE, callback);
+                WriteResponseHandler<Mutation> responseHandler = new WriteResponseHandler<>(endpoint, WriteType.SIMPLE, callback);
                 MessagingService.instance().sendRR(message, endpoint, responseHandler, false);
                 responseHandlers.add(responseHandler);
             }
 
-            for (WriteResponseHandler handler : responseHandlers)
+            for (WriteResponseHandler<Mutation> handler : responseHandlers)
             {
                 try
                 {
                     handler.get();
                 }
-                catch (WriteTimeoutException e)
+                catch (WriteTimeoutException|WriteFailureException e)
                 {
-                    logger.info("Timed out replaying hints to {}; aborting ({} delivered)", endpoint, rowsReplayed);
+                    logger.info("Failed replaying hints to {}; aborting ({} delivered), error : {}",
+                        endpoint, rowsReplayed, e.getMessage());
                     break delivery;
                 }
             }
@@ -500,7 +501,7 @@
         if (meanColumnCount <= 0)
             return PAGE_SIZE;
 
-        int averageColumnSize = (int) (hintStore.getMeanRowSize() / meanColumnCount);
+        int averageColumnSize = (int) (hintStore.metric.meanRowSize.getValue() / meanColumnCount);
         if (averageColumnSize <= 0)
             return PAGE_SIZE;
 
@@ -514,7 +515,7 @@
      */
     private void scheduleAllDeliveries()
     {
-        logger.debug("Started scheduleAllDeliveries");
+        logger.trace("Started scheduleAllDeliveries");
 
         // Force a major compaction to get rid of the tombstones and expired hints. Do it once, before we schedule any
         // individual replay, to avoid N - 1 redundant individual compactions (when N is the number of nodes with hints
@@ -523,7 +524,7 @@
 
         IPartitioner p = StorageService.getPartitioner();
         RowPosition minPos = p.getMinimumToken().minKeyBound();
-        Range<RowPosition> range = new Range<>(minPos, minPos, p);
+        Range<RowPosition> range = new Range<>(minPos, minPos);
         IDiskAtomFilter filter = new NamesQueryFilter(ImmutableSortedSet.<CellName>of());
         List<Row> rows = hintStore.getRangeSlice(range, null, filter, Integer.MAX_VALUE, System.currentTimeMillis());
         for (Row row : rows)
@@ -535,7 +536,7 @@
                 scheduleHintDelivery(target, false);
         }
 
-        logger.debug("Finished scheduleAllDeliveries");
+        logger.trace("Finished scheduleAllDeliveries");
     }
 
     /*
@@ -549,7 +550,7 @@
         if (!queuedDeliveries.add(to))
             return;
 
-        logger.debug("Scheduling delivery of Hints to {}", to);
+        logger.trace("Scheduling delivery of Hints to {}", to);
 
         hintDeliveryExecutor.execute(new Runnable()
         {
@@ -610,8 +611,8 @@
 
         try
         {
-            RangeSliceCommand cmd = new RangeSliceCommand(Keyspace.SYSTEM_KS,
-                                                          SystemKeyspace.HINTS_CF,
+            RangeSliceCommand cmd = new RangeSliceCommand(SystemKeyspace.NAME,
+                                                          SystemKeyspace.HINTS,
                                                           System.currentTimeMillis(),
                                                           predicate,
                                                           range,
diff --git a/src/java/org/apache/cassandra/db/Keyspace.java b/src/java/org/apache/cassandra/db/Keyspace.java
index 03c3d2b..c126f5c 100644
--- a/src/java/org/apache/cassandra/db/Keyspace.java
+++ b/src/java/org/apache/cassandra/db/Keyspace.java
@@ -19,19 +19,12 @@
 
 import java.io.File;
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
-import java.util.UUID;
+import java.util.*;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.Future;
 
 import com.google.common.base.Function;
-import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -43,7 +36,7 @@
 import org.apache.cassandra.db.filter.QueryFilter;
 import org.apache.cassandra.db.index.SecondaryIndex;
 import org.apache.cassandra.db.index.SecondaryIndexManager;
-import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.locator.AbstractReplicationStrategy;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.service.pager.QueryPagers;
@@ -56,25 +49,27 @@
  */
 public class Keyspace
 {
-    public static final String SYSTEM_KS = "system";
     private static final int DEFAULT_PAGE_SIZE = 10000;
 
     private static final Logger logger = LoggerFactory.getLogger(Keyspace.class);
 
+    private static final String TEST_FAIL_WRITES_KS = System.getProperty("cassandra.test.fail_writes_ks", "");
+    private static final boolean TEST_FAIL_WRITES = !TEST_FAIL_WRITES_KS.isEmpty();
+
     public final KeyspaceMetrics metric;
 
     // It is possible to call Keyspace.open without a running daemon, so it makes sense to ensure
     // proper directories here as well as in CassandraDaemon.
     static
     {
-        if (!(Config.isClientMode() || StorageService.instance.isClientMode()))
+        if (!Config.isClientMode())
             DatabaseDescriptor.createAllDirectories();
     }
 
     public final OpOrder writeOrder = new OpOrder();
 
     /* ColumnFamilyStore per column family */
-    private final ConcurrentMap<UUID, ColumnFamilyStore> columnFamilyStores = new ConcurrentHashMap<UUID, ColumnFamilyStore>();
+    private final ConcurrentMap<UUID, ColumnFamilyStore> columnFamilyStores = new ConcurrentHashMap<>();
     private volatile KSMetaData metadata;
     private volatile AbstractReplicationStrategy replicationStrategy;
 
@@ -94,7 +89,7 @@
 
     public static Keyspace open(String keyspaceName)
     {
-        assert initialized || keyspaceName.equals(SYSTEM_KS);
+        assert initialized || keyspaceName.equals(SystemKeyspace.NAME);
         return open(keyspaceName, Schema.instance, true);
     }
 
@@ -147,7 +142,7 @@
     }
 
     /**
-     * Removes every SSTable in the directory from the appropriate DataTracker's view.
+     * Removes every SSTable in the directory from the appropriate Tracker's view.
      * @param directory the unreadable directory, possibly with SSTables in it, but not necessarily.
      */
     public static void removeUnreadableSSTables(File directory)
@@ -205,7 +200,7 @@
         }
 
         if ((columnFamilyName != null) && !tookSnapShot)
-            throw new IOException("Failed taking snapshot. Column family " + columnFamilyName + " does not exist.");
+            throw new IOException("Failed taking snapshot. Table " + columnFamilyName + " does not exist.");
     }
 
     /**
@@ -256,7 +251,7 @@
      */
     public List<SSTableReader> getAllSSTables()
     {
-        List<SSTableReader> list = new ArrayList<SSTableReader>(columnFamilyStores.size());
+        List<SSTableReader> list = new ArrayList<>(columnFamilyStores.size());
         for (ColumnFamilyStore cfStore : columnFamilyStores.values())
             list.addAll(cfStore.getSSTables());
         return list;
@@ -269,13 +264,25 @@
         createReplicationStrategy(metadata);
 
         this.metric = new KeyspaceMetrics(this);
-        for (CFMetaData cfm : new ArrayList<CFMetaData>(metadata.cfMetaData().values()))
+        for (CFMetaData cfm : new ArrayList<>(metadata.cfMetaData().values()))
         {
-            logger.debug("Initializing {}.{}", getName(), cfm.cfName);
-            initCf(cfm.cfId, cfm.cfName, loadSSTables);
+            logger.trace("Initializing {}.{}", getName(), cfm.cfName);
+            initCf(cfm, loadSSTables);
         }
     }
 
+    private Keyspace(KSMetaData metadata)
+    {
+        this.metadata = metadata;
+        createReplicationStrategy(metadata);
+        this.metric = new KeyspaceMetrics(this);
+    }
+
+    public static Keyspace mockKS(KSMetaData metadata)
+    {
+        return new Keyspace(metadata);
+    }
+
     private void createReplicationStrategy(KSMetaData ksm)
     {
         replicationStrategy = AbstractReplicationStrategy.createReplicationStrategy(ksm.name,
@@ -323,25 +330,25 @@
     /**
      * adds a cf to internal structures, ends up creating disk files).
      */
-    public void initCf(UUID cfId, String cfName, boolean loadSSTables)
+    public void initCf(CFMetaData metadata, boolean loadSSTables)
     {
-        ColumnFamilyStore cfs = columnFamilyStores.get(cfId);
+        ColumnFamilyStore cfs = columnFamilyStores.get(metadata.cfId);
 
         if (cfs == null)
         {
             // CFS being created for the first time, either on server startup or new CF being added.
             // We don't worry about races here; startup is safe, and adding multiple idential CFs
             // simultaneously is a "don't do that" scenario.
-            ColumnFamilyStore oldCfs = columnFamilyStores.putIfAbsent(cfId, ColumnFamilyStore.createColumnFamilyStore(this, cfName, loadSSTables));
+            ColumnFamilyStore oldCfs = columnFamilyStores.putIfAbsent(metadata.cfId, ColumnFamilyStore.createColumnFamilyStore(this, metadata, loadSSTables));
             // CFS mbean instantiation will error out before we hit this, but in case that changes...
             if (oldCfs != null)
-                throw new IllegalStateException("added multiple mappings for cf id " + cfId);
+                throw new IllegalStateException("added multiple mappings for cf id " + metadata.cfId);
         }
         else
         {
             // re-initializing an existing CF.  This will happen if you cleared the schema
             // on this node and it's getting repopulated from the rest of the cluster.
-            assert cfs.name.equals(cfName);
+            assert cfs.name.equals(metadata.cfName);
             cfs.metadata.reload();
             cfs.reload();
         }
@@ -369,6 +376,9 @@
      */
     public void apply(Mutation mutation, boolean writeCommitLog, boolean updateIndexes)
     {
+        if (TEST_FAIL_WRITES && metadata.name.equals(TEST_FAIL_WRITES_KS))
+            throw new RuntimeException("Testing write failures");
+
         try (OpOrder.Group opGroup = writeOrder.start())
         {
             // write the mutation to the commitlog and memtables
@@ -385,7 +395,7 @@
                 ColumnFamilyStore cfs = columnFamilyStores.get(cf.id());
                 if (cfs == null)
                 {
-                    logger.error("Attempting to mutate non-existant column family {}", cf.id());
+                    logger.error("Attempting to mutate non-existant table {}", cf.id());
                     continue;
                 }
 
@@ -410,16 +420,15 @@
      */
     public static void indexRow(DecoratedKey key, ColumnFamilyStore cfs, Set<String> idxNames)
     {
-        if (logger.isDebugEnabled())
-            logger.debug("Indexing row {} ", cfs.metadata.getKeyValidator().getString(key.getKey()));
+        if (logger.isTraceEnabled())
+            logger.trace("Indexing row {} ", cfs.metadata.getKeyValidator().getString(key.getKey()));
 
-        try (OpOrder.Group opGroup = cfs.keyspace.writeOrder.start())
+        Set<SecondaryIndex> indexes = cfs.indexManager.getIndexesByNames(idxNames);
+
+        Iterator<ColumnFamily> pager = QueryPagers.pageRowLocally(cfs, key.getKey(), DEFAULT_PAGE_SIZE);
+        while (pager.hasNext())
         {
-            Set<SecondaryIndex> indexes = cfs.indexManager.getIndexesByNames(idxNames);
-
-            Iterator<ColumnFamily> pager = QueryPagers.pageRowLocally(cfs, key.getKey(), DEFAULT_PAGE_SIZE);
-            while (pager.hasNext())
-            {
+            try (OpOrder.Group opGroup = cfs.keyspace.writeOrder.start()) {
                 ColumnFamily cf = pager.next();
                 ColumnFamily cf2 = cf.cloneMeShallow();
                 for (Cell cell : cf)
@@ -434,12 +443,82 @@
 
     public List<Future<?>> flush()
     {
-        List<Future<?>> futures = new ArrayList<Future<?>>(columnFamilyStores.size());
-        for (UUID cfId : columnFamilyStores.keySet())
-            futures.add(columnFamilyStores.get(cfId).forceFlush());
+        List<Future<?>> futures = new ArrayList<>(columnFamilyStores.size());
+        for (ColumnFamilyStore cfs : columnFamilyStores.values())
+            futures.add(cfs.forceFlush());
         return futures;
     }
 
+    public Iterable<ColumnFamilyStore> getValidColumnFamilies(boolean allowIndexes, boolean autoAddIndexes, String... cfNames) throws IOException
+    {
+        Set<ColumnFamilyStore> valid = new HashSet<>();
+
+        if (cfNames.length == 0)
+        {
+            // all stores are interesting
+            for (ColumnFamilyStore cfStore : getColumnFamilyStores())
+            {
+                valid.add(cfStore);
+                if (autoAddIndexes)
+                {
+                    for (SecondaryIndex si : cfStore.indexManager.getIndexes())
+                    {
+                        if (si.getIndexCfs() != null) {
+                            logger.info("adding secondary index {} to operation", si.getIndexName());
+                            valid.add(si.getIndexCfs());
+                        }
+                    }
+
+                }
+            }
+            return valid;
+        }
+        // filter out interesting stores
+        for (String cfName : cfNames)
+        {
+            //if the CF name is an index, just flush the CF that owns the index
+            String baseCfName = cfName;
+            String idxName = null;
+            if (cfName.contains(".")) // secondary index
+            {
+                if(!allowIndexes)
+                {
+                    logger.warn("Operation not allowed on secondary Index table ({})", cfName);
+                    continue;
+                }
+
+                String[] parts = cfName.split("\\.", 2);
+                baseCfName = parts[0];
+                idxName = parts[1];
+            }
+
+            ColumnFamilyStore cfStore = getColumnFamilyStore(baseCfName);
+            if (idxName != null)
+            {
+                Collection< SecondaryIndex > indexes = cfStore.indexManager.getIndexesByNames(new HashSet<>(Arrays.asList(cfName)));
+                if (indexes.isEmpty())
+                    throw new IllegalArgumentException(String.format("Invalid index specified: %s/%s.", baseCfName, idxName));
+                else
+                    valid.add(Iterables.get(indexes, 0).getIndexCfs());
+            }
+            else
+            {
+                valid.add(cfStore);
+                if(autoAddIndexes)
+                {
+                    for(SecondaryIndex si : cfStore.indexManager.getIndexes())
+                    {
+                        if (si.getIndexCfs() != null) {
+                            logger.info("adding secondary index {} to operation", si.getIndexName());
+                            valid.add(si.getIndexCfs());
+                        }
+                    }
+                }
+            }
+        }
+        return valid;
+    }
+
     public static Iterable<Keyspace> all()
     {
         return Iterables.transform(Schema.instance.getKeyspaces(), keyspaceTransformer);
@@ -452,7 +531,7 @@
 
     public static Iterable<Keyspace> system()
     {
-        return Iterables.transform(Schema.systemKeyspaceNames, keyspaceTransformer);
+        return Iterables.transform(Collections.singleton(SystemKeyspace.NAME), keyspaceTransformer);
     }
 
     @Override
diff --git a/src/java/org/apache/cassandra/db/Memtable.java b/src/java/org/apache/cassandra/db/Memtable.java
index 375195f..b4ada09 100644
--- a/src/java/org/apache/cassandra/db/Memtable.java
+++ b/src/java/org/apache/cassandra/db/Memtable.java
@@ -23,36 +23,37 @@
 import java.util.Map;
 import java.util.concurrent.ConcurrentNavigableMap;
 import java.util.concurrent.ConcurrentSkipListMap;
-import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
 
-import com.google.common.base.Throwables;
-import org.apache.cassandra.utils.*;
+import com.google.common.annotations.VisibleForTesting;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.commitlog.CommitLog;
 import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.apache.cassandra.db.composites.CellNameType;
 import org.apache.cassandra.db.index.SecondaryIndexManager;
-import org.apache.cassandra.dht.LongToken;
-import org.apache.cassandra.io.sstable.SSTableReader;
-import org.apache.cassandra.io.sstable.SSTableWriter;
+import org.apache.cassandra.dht.Murmur3Partitioner.LongToken;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
-import org.apache.cassandra.io.util.DiskAwareRunnable;
 import org.apache.cassandra.service.ActiveRepairService;
+import org.apache.cassandra.utils.*;
 import org.apache.cassandra.utils.concurrent.OpOrder;
 import org.apache.cassandra.utils.memory.*;
 
-public class Memtable
+public class Memtable implements Comparable<Memtable>
 {
     private static final Logger logger = LoggerFactory.getLogger(Memtable.class);
 
     static final MemtablePool MEMORY_POOL = DatabaseDescriptor.getMemtableAllocatorPool();
-    private static final int ROW_OVERHEAD_HEAP_SIZE = estimateRowOverhead(Integer.valueOf(System.getProperty("cassandra.memtable_row_overhead_computation_step", "100000")));
+    private static final int ROW_OVERHEAD_HEAP_SIZE = estimateRowOverhead(Integer.parseInt(System.getProperty("cassandra.memtable_row_overhead_computation_step", "100000")));
 
     private final MemtableAllocator allocator;
     private final AtomicLong liveDataSize = new AtomicLong(0);
@@ -60,10 +61,18 @@
 
     // the write barrier for directing writes to this memtable during a switch
     private volatile OpOrder.Barrier writeBarrier;
-    // the last ReplayPosition owned by this Memtable; all ReplayPositions lower are owned by this or an earlier Memtable
-    private volatile AtomicReference<ReplayPosition> lastReplayPosition;
-    // the "first" ReplayPosition owned by this Memtable; this is inaccurate, and only used as a convenience to prevent CLSM flushing wantonly
-    private final ReplayPosition minReplayPosition = CommitLog.instance.getContext();
+    // the precise upper bound of ReplayPosition owned by this memtable
+    private volatile AtomicReference<ReplayPosition> commitLogUpperBound;
+    // the precise lower bound of ReplayPosition owned by this memtable; equal to its predecessor's commitLogUpperBound
+    private AtomicReference<ReplayPosition> commitLogLowerBound;
+    // the approximate lower bound by this memtable; must be <= commitLogLowerBound once our predecessor
+    // has been finalised, and this is enforced in the ColumnFamilyStore.setCommitLogUpperBound
+    private final ReplayPosition approximateCommitLogLowerBound = CommitLog.instance.getContext();
+
+    public int compareTo(Memtable that)
+    {
+        return this.approximateCommitLogLowerBound.compareTo(that.approximateCommitLogLowerBound);
+    }
 
     public static final class LastReplayPosition extends ReplayPosition
     {
@@ -77,22 +86,34 @@
     // actually only store DecoratedKey.
     private final ConcurrentNavigableMap<RowPosition, AtomicBTreeColumns> rows = new ConcurrentSkipListMap<>();
     public final ColumnFamilyStore cfs;
-    private final long creationTime = System.currentTimeMillis();
     private final long creationNano = System.nanoTime();
 
+    // The smallest timestamp for all partitions stored in this memtable
+    private long minTimestamp = Long.MAX_VALUE;
+
     // Record the comparator of the CFS at the creation of the memtable. This
     // is only used when a user update the CF comparator, to know if the
     // memtable was created with the new or old comparator.
     public final CellNameType initialComparator;
 
-    public Memtable(ColumnFamilyStore cfs)
+    public Memtable(AtomicReference<ReplayPosition> commitLogLowerBound, ColumnFamilyStore cfs)
     {
         this.cfs = cfs;
+        this.commitLogLowerBound = commitLogLowerBound;
         this.allocator = MEMORY_POOL.newAllocator();
         this.initialComparator = cfs.metadata.comparator;
         this.cfs.scheduleFlush();
     }
 
+    // ONLY to be used for testing, to create a mock Memtable
+    @VisibleForTesting
+    public Memtable(CFMetaData metadata)
+    {
+        this.initialComparator = metadata.comparator;
+        this.cfs = null;
+        this.allocator = null;
+    }
+
     public MemtableAllocator getAllocator()
     {
         return allocator;
@@ -108,10 +129,11 @@
         return currentOperations.get();
     }
 
-    void setDiscarding(OpOrder.Barrier writeBarrier, AtomicReference<ReplayPosition> lastReplayPosition)
+    @VisibleForTesting
+    public void setDiscarding(OpOrder.Barrier writeBarrier, AtomicReference<ReplayPosition> lastReplayPosition)
     {
         assert this.writeBarrier == null;
-        this.lastReplayPosition = lastReplayPosition;
+        this.commitLogUpperBound = lastReplayPosition;
         this.writeBarrier = writeBarrier;
         allocator.setDiscarding();
     }
@@ -141,16 +163,21 @@
             // its current value and ours; if it HAS been finalised, we simply accept its judgement
             // this permits us to coordinate a safe boundary, as the boundary choice is made
             // atomically wrt our max() maintenance, so an operation cannot sneak into the past
-            ReplayPosition currentLast = lastReplayPosition.get();
+            ReplayPosition currentLast = commitLogUpperBound.get();
             if (currentLast instanceof LastReplayPosition)
                 return currentLast.compareTo(replayPosition) >= 0;
             if (currentLast != null && currentLast.compareTo(replayPosition) >= 0)
                 return true;
-            if (lastReplayPosition.compareAndSet(currentLast, replayPosition))
+            if (commitLogUpperBound.compareAndSet(currentLast, replayPosition))
                 return true;
         }
     }
 
+    public ReplayPosition getCommitLogLowerBound()
+    {
+        return commitLogLowerBound.get();
+    }
+
     public boolean isLive()
     {
         return allocator.isLive();
@@ -161,9 +188,9 @@
         return rows.isEmpty();
     }
 
-    public boolean isCleanAfter(ReplayPosition position)
+    public boolean mayContainDataBefore(ReplayPosition position)
     {
-        return isClean() || (position != null && minReplayPosition.compareTo(position) >= 0);
+        return approximateCommitLogLowerBound.compareTo(position) < 0;
     }
 
     /**
@@ -197,7 +224,7 @@
                 previous = empty;
                 // allocate the row overhead after the fact; this saves over allocating and having to free after, but
                 // means we can overshoot our declared limit.
-                int overhead = (int) (cfs.partitioner.getHeapSizeOf(key.getToken()) + ROW_OVERHEAD_HEAP_SIZE);
+                int overhead = (int) (key.getToken().getHeapSize() + ROW_OVERHEAD_HEAP_SIZE);
                 allocator.onHeap().allocate(overhead, opGroup);
                 initialSize = 8;
             }
@@ -207,10 +234,11 @@
             }
         }
 
-        final Pair<Long, Long> pair = previous.addAllWithSizeDelta(cf, allocator, opGroup, indexer);
-        liveDataSize.addAndGet(initialSize + pair.left);
+        final AtomicBTreeColumns.ColumnUpdater updater = previous.addAllWithSizeDelta(cf, allocator, opGroup, indexer);
+        minTimestamp = Math.min(minTimestamp, updater.minTimestamp);
+        liveDataSize.addAndGet(initialSize + updater.dataSize);
         currentOperations.addAndGet(cf.getColumnCount() + (cf.isMarkedForDelete() ? 1 : 0) + cf.deletionInfo().rangeCount());
-        return pair.right;
+        return updater.colUpdateTimeDelta;
     }
 
     // for debugging
@@ -231,11 +259,6 @@
         return rows.size();
     }
 
-    public FlushRunnable flushRunnable()
-    {
-        return new FlushRunnable(lastReplayPosition.get());
-    }
-
     public String toString()
     {
         return String.format("Memtable-%s@%s(%s serialized bytes, %s ops, %.0f%%/%.0f%% of on/off-heap limit)",
@@ -251,7 +274,7 @@
     {
         return new Iterator<Map.Entry<DecoratedKey, ColumnFamily>>()
         {
-            private Iterator<? extends Map.Entry<? extends RowPosition, AtomicBTreeColumns>> iter = stopAt.isMinimum(cfs.partitioner)
+            private Iterator<? extends Map.Entry<? extends RowPosition, AtomicBTreeColumns>> iter = stopAt.isMinimum()
                     ? rows.tailMap(startWith).entrySet().iterator()
                     : rows.subMap(startWith, true, stopAt, true).entrySet().iterator();
 
@@ -264,20 +287,21 @@
 
             public Map.Entry<DecoratedKey, ColumnFamily> next()
             {
-                Map.Entry<? extends RowPosition, ? extends ColumnFamily> entry = iter.next();
+                Map.Entry<? extends RowPosition, ? extends ColumnFamily> entryRowPosition = iter.next();
                 // Actual stored key should be true DecoratedKey
-                assert entry.getKey() instanceof DecoratedKey;
+                assert entryRowPosition.getKey() instanceof DecoratedKey;
+                @SuppressWarnings("unchecked") // Object cast is required since otherwise we can't turn RowPosition into DecoratedKey
+                Map.Entry<DecoratedKey, ColumnFamily> entry = (Map.Entry<DecoratedKey, ColumnFamily>) entryRowPosition;
                 if (MEMORY_POOL.needToCopyOnHeap())
                 {
-                    DecoratedKey key = (DecoratedKey) entry.getKey();
+                    DecoratedKey key = entry.getKey();
                     key = new BufferDecoratedKey(key.getToken(), HeapAllocator.instance.clone(key.getKey()));
                     ColumnFamily cells = ArrayBackedSortedColumns.localCopy(entry.getValue(), HeapAllocator.instance);
                     entry = new AbstractMap.SimpleImmutableEntry<>(key, cells);
                 }
                 // Store the reference to the current entry so that remove() can update the current size.
                 currentEntry = entry;
-                // Object cast is required since otherwise we can't turn RowPosition into DecoratedKey
-                return (Map.Entry<DecoratedKey, ColumnFamily>) entry;
+                return entry;
             }
 
             public void remove()
@@ -294,148 +318,120 @@
         return rows.get(key);
     }
 
-    public long creationTime()
+    public SSTableReader flush()
     {
-        return creationTime;
+        long estimatedSize = estimatedSize();
+        Directories.DataDirectory dataDirectory = cfs.directories.getWriteableLocation(estimatedSize);
+        File sstableDirectory = cfs.directories.getLocationForDisk(dataDirectory);
+        assert sstableDirectory != null : "Flush task is not bound to any disk";
+        return writeSortedContents(sstableDirectory);
     }
 
-    class FlushRunnable extends DiskAwareRunnable
+    public long getMinTimestamp()
     {
-        private final ReplayPosition context;
-        private final long estimatedSize;
+        return minTimestamp;
+    }
 
-        FlushRunnable(ReplayPosition context)
+    private long estimatedSize()
+    {
+        long keySize = 0;
+        for (RowPosition key : rows.keySet())
         {
-            this.context = context;
+            //  make sure we don't write non-sensical keys
+            assert key instanceof DecoratedKey;
+            keySize += ((DecoratedKey)key).getKey().remaining();
+        }
+        return (long) ((keySize // index entries
+                        + keySize // keys in data file
+                        + liveDataSize.get()) // data
+                       * 1.2); // bloom filter and row index overhead
+    }
 
-            long keySize = 0;
-            for (RowPosition key : rows.keySet())
+    private SSTableReader writeSortedContents(File sstableDirectory)
+    {
+        logger.info("Writing {}", Memtable.this.toString());
+
+        SSTableReader ssTable;
+        // errors when creating the writer that may leave empty temp files.
+        try (SSTableWriter writer = createFlushWriter(cfs.getTempSSTablePath(sstableDirectory)))
+        {
+            boolean trackContention = logger.isTraceEnabled();
+            int heavilyContendedRowCount = 0;
+            // (we can't clear out the map as-we-go to free up memory,
+            //  since the memtable is being used for queries in the "pending flush" category)
+            for (Map.Entry<RowPosition, AtomicBTreeColumns> entry : rows.entrySet())
             {
-                //  make sure we don't write non-sensical keys
-                assert key instanceof DecoratedKey;
-                keySize += ((DecoratedKey)key).getKey().remaining();
+                AtomicBTreeColumns cf = entry.getValue();
+
+                if (cf.isMarkedForDelete() && cf.hasColumns())
+                {
+                    // When every node is up, there's no reason to write batchlog data out to sstables
+                    // (which in turn incurs cost like compaction) since the BL write + delete cancel each other out,
+                    // and BL data is strictly local, so we don't need to preserve tombstones for repair.
+                    // If we have a data row + row level tombstone, then writing it is effectively an expensive no-op so we skip it.
+                    // See CASSANDRA-4667.
+                    if (cfs.name.equals(SystemKeyspace.BATCHLOG) && cfs.keyspace.getName().equals(SystemKeyspace.NAME))
+                        continue;
+                }
+
+                if (trackContention && cf.usePessimisticLocking())
+                    heavilyContendedRowCount++;
+
+                if (!cf.isEmpty())
+                    writer.append((DecoratedKey)entry.getKey(), cf);
             }
-            estimatedSize = (long) ((keySize // index entries
-                                    + keySize // keys in data file
-                                    + liveDataSize.get()) // data
-                                    * 1.2); // bloom filter and row index overhead
-        }
 
-        public long getExpectedWriteSize()
-        {
-            return estimatedSize;
-        }
-
-        protected void runMayThrow() throws Exception
-        {
-            long writeSize = getExpectedWriteSize();
-            Directories.DataDirectory dataDirectory = getWriteDirectory(writeSize);
-            File sstableDirectory = cfs.directories.getLocationForDisk(dataDirectory);
-            assert sstableDirectory != null : "Flush task is not bound to any disk";
-            SSTableReader sstable = writeSortedContents(context, sstableDirectory);
-            cfs.replaceFlushed(Memtable.this, sstable);
-        }
-
-        protected Directories getDirectories()
-        {
-            return cfs.directories;
-        }
-
-        private SSTableReader writeSortedContents(ReplayPosition context, File sstableDirectory)
-        throws ExecutionException, InterruptedException
-        {
-            logger.info("Writing {}", Memtable.this.toString());
-
-            SSTableReader ssTable;
-            // errors when creating the writer that may leave empty temp files.
-            SSTableWriter writer = createFlushWriter(cfs.getTempSSTablePath(sstableDirectory));
-            try
+            if (writer.getFilePointer() > 0)
             {
-                boolean trackContention = logger.isDebugEnabled();
-                int heavilyContendedRowCount = 0;
-                // (we can't clear out the map as-we-go to free up memory,
-                //  since the memtable is being used for queries in the "pending flush" category)
-                for (Map.Entry<RowPosition, AtomicBTreeColumns> entry : rows.entrySet())
-                {
-                    AtomicBTreeColumns cf = entry.getValue();
+                logger.debug(String.format("Completed flushing %s (%s) for commitlog position %s",
+                                           writer.getFilename(),
+                                           FBUtilities.prettyPrintMemory(writer.getOnDiskFilePointer()),
+                                           commitLogUpperBound));
 
-                    if (cf.isMarkedForDelete() && cf.hasColumns())
-                    {
-                        // When every node is up, there's no reason to write batchlog data out to sstables
-                        // (which in turn incurs cost like compaction) since the BL write + delete cancel each other out,
-                        // and BL data is strictly local, so we don't need to preserve tombstones for repair.
-                        // If we have a data row + row level tombstone, then writing it is effectively an expensive no-op so we skip it.
-                        // See CASSANDRA-4667.
-                        if (cfs.name.equals(SystemKeyspace.BATCHLOG_CF) && cfs.keyspace.getName().equals(Keyspace.SYSTEM_KS))
-                            continue;
-                    }
-
-                    if (trackContention && cf.usePessimisticLocking())
-                        heavilyContendedRowCount++;
-
-                    if (!cf.isEmpty())
-                        writer.append((DecoratedKey)entry.getKey(), cf);
-                }
-
-                if (writer.getFilePointer() > 0)
-                {
-                    logger.info(String.format("Completed flushing %s (%s) for commitlog position %s",
-                                              writer.getFilename(),
-                                              FBUtilities.prettyPrintMemory(writer.getOnDiskFilePointer()),
-                                              context));
-
-                    writer.isolateReferences();
-                    // temp sstables should contain non-repaired data.
-                    ssTable = writer.closeAndOpenReader();
-                }
-                else
-                {
-                    logger.info("Completed flushing {}; nothing needed to be retained.  Commitlog position was {}",
-                                writer.getFilename(), context);
-                    writer.abort();
-                    ssTable = null;
-                }
-
-                if (heavilyContendedRowCount > 0)
-                    logger.debug(String.format("High update contention in %d/%d partitions of %s ", heavilyContendedRowCount, rows.size(), Memtable.this.toString()));
-
-                return ssTable;
+                // temp sstables should contain non-repaired data.
+                ssTable = writer.finish(true);
             }
-            catch (Throwable e)
+            else
             {
+                logger.debug("Completed flushing {}; nothing needed to be retained.  Commitlog position was {}",
+                             writer.getFilename(), commitLogUpperBound);
                 writer.abort();
-                throw Throwables.propagate(e);
+                ssTable = null;
             }
-        }
 
-        public SSTableWriter createFlushWriter(String filename) throws ExecutionException, InterruptedException
-        {
-            MetadataCollector sstableMetadataCollector = new MetadataCollector(cfs.metadata.comparator).replayPosition(context);
-            return new SSTableWriter(filename,
-                                     rows.size(),
-                                     ActiveRepairService.UNREPAIRED_SSTABLE,
-                                     cfs.metadata,
-                                     cfs.partitioner,
-                                     sstableMetadataCollector);
+            if (heavilyContendedRowCount > 0)
+                logger.trace(String.format("High update contention in %d/%d partitions of %s ", heavilyContendedRowCount, rows.size(), Memtable.this.toString()));
+
+            return ssTable;
         }
     }
 
+    private SSTableWriter createFlushWriter(String filename)
+    {
+        MetadataCollector sstableMetadataCollector = new MetadataCollector(cfs.metadata.comparator)
+                                                     .commitLogLowerBound(commitLogLowerBound.get())
+                                                     .commitLogUpperBound(commitLogUpperBound.get());
+        return SSTableWriter.create(Descriptor.fromFilename(filename), (long) rows.size(), ActiveRepairService.UNREPAIRED_SSTABLE, cfs.metadata, cfs.partitioner, sstableMetadataCollector);
+    }
+
     private static int estimateRowOverhead(final int count)
     {
         // calculate row overhead
-        final OpOrder.Group group = new OpOrder().start();
-        int rowOverhead;
-        MemtableAllocator allocator = MEMORY_POOL.newAllocator();
-        ConcurrentNavigableMap<RowPosition, Object> rows = new ConcurrentSkipListMap<>();
-        final Object val = new Object();
-        for (int i = 0 ; i < count ; i++)
-            rows.put(allocator.clone(new BufferDecoratedKey(new LongToken((long) i), ByteBufferUtil.EMPTY_BYTE_BUFFER), group), val);
-        double avgSize = ObjectSizes.measureDeep(rows) / (double) count;
-        rowOverhead = (int) ((avgSize - Math.floor(avgSize)) < 0.05 ? Math.floor(avgSize) : Math.ceil(avgSize));
-        rowOverhead -= ObjectSizes.measureDeep(new LongToken((long) 0));
-        rowOverhead += AtomicBTreeColumns.EMPTY_SIZE;
-        allocator.setDiscarding();
-        allocator.setDiscarded();
-        return rowOverhead;
+        try (final OpOrder.Group group = new OpOrder().start())
+        {
+            int rowOverhead;
+            MemtableAllocator allocator = MEMORY_POOL.newAllocator();
+            ConcurrentNavigableMap<RowPosition, Object> rows = new ConcurrentSkipListMap<>();
+            final Object val = new Object();
+            for (int i = 0; i < count; i++)
+                rows.put(allocator.clone(new BufferDecoratedKey(new LongToken(i), ByteBufferUtil.EMPTY_BYTE_BUFFER), group), val);
+            double avgSize = ObjectSizes.measureDeep(rows) / (double) count;
+            rowOverhead = (int) ((avgSize - Math.floor(avgSize)) < 0.05 ? Math.floor(avgSize) : Math.ceil(avgSize));
+            rowOverhead -= ObjectSizes.measureDeep(new LongToken(0));
+            rowOverhead += AtomicBTreeColumns.EMPTY_SIZE;
+            allocator.setDiscarding();
+            allocator.setDiscarded();
+            return rowOverhead;
+        }
     }
 }
diff --git a/src/java/org/apache/cassandra/db/MigrationRequestVerbHandler.java b/src/java/org/apache/cassandra/db/MigrationRequestVerbHandler.java
index d4503ba..ab934c6 100644
--- a/src/java/org/apache/cassandra/db/MigrationRequestVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/MigrationRequestVerbHandler.java
@@ -26,6 +26,7 @@
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.LegacySchemaTables;
 import org.apache.cassandra.service.MigrationManager;
 
 /**
@@ -38,9 +39,9 @@
 
     public void doVerb(MessageIn message, int id)
     {
-        logger.debug("Received migration request from {}.", message.from);
+        logger.trace("Received migration request from {}.", message.from);
         MessageOut<Collection<Mutation>> response = new MessageOut<>(MessagingService.Verb.INTERNAL_RESPONSE,
-                                                                     SystemKeyspace.serializeSchema(),
+                                                                     LegacySchemaTables.convertSchemaToMutations(),
                                                                      MigrationManager.MigrationsSerializer.instance);
         MessagingService.instance().sendReply(response, id, message.from);
     }
diff --git a/src/java/org/apache/cassandra/db/Mutation.java b/src/java/org/apache/cassandra/db/Mutation.java
index 0424f5a..8079ef8 100644
--- a/src/java/org/apache/cassandra/db/Mutation.java
+++ b/src/java/org/apache/cassandra/db/Mutation.java
@@ -125,7 +125,7 @@
         ColumnFamily prev = modifications.put(columnFamily.id(), columnFamily);
         if (prev != null)
             // developer error
-            throw new IllegalArgumentException("ColumnFamily " + columnFamily + " already has modifications in this mutation: " + prev);
+            throw new IllegalArgumentException("Table " + columnFamily + " already has modifications in this mutation: " + prev);
     }
 
     /**
@@ -305,7 +305,7 @@
             }
             else
             {
-                modifications = new HashMap<UUID, ColumnFamily>();
+                modifications = new HashMap<UUID, ColumnFamily>(size);
                 for (int i = 0; i < size; ++i)
                 {
                     ColumnFamily cf = deserializeOneCf(in, version, flag);
diff --git a/src/java/org/apache/cassandra/db/MutationVerbHandler.java b/src/java/org/apache/cassandra/db/MutationVerbHandler.java
index 43ffeae..3baa93e 100644
--- a/src/java/org/apache/cassandra/db/MutationVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/MutationVerbHandler.java
@@ -18,24 +18,20 @@
 package org.apache.cassandra.db;
 
 import java.io.DataInputStream;
+import java.io.IOError;
 import java.io.IOException;
 import java.net.InetAddress;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import org.apache.cassandra.io.util.FastByteArrayInputStream;
 import org.apache.cassandra.net.*;
 import org.apache.cassandra.tracing.Tracing;
 
 public class MutationVerbHandler implements IVerbHandler<Mutation>
 {
-    private static final Logger logger = LoggerFactory.getLogger(MutationVerbHandler.class);
+    private static final boolean TEST_FAIL_WRITES = System.getProperty("cassandra.test.fail_writes", "false").equalsIgnoreCase("true");
 
-    public void doVerb(MessageIn<Mutation> message, int id)
+    public void doVerb(MessageIn<Mutation> message, int id)  throws IOException
     {
-        try
-        {
             // Check if there were any forwarding headers in this message
             byte[] from = message.parameters.get(Mutation.FORWARD_FROM);
             InetAddress replyTo;
@@ -55,11 +51,6 @@
             WriteResponse response = new WriteResponse();
             Tracing.trace("Enqueuing response to {}", replyTo);
             MessagingService.instance().sendReply(response.createMessage(), id, replyTo);
-        }
-        catch (IOException e)
-        {
-            logger.error("Error in mutation", e);
-        }
     }
 
     /**
@@ -68,18 +59,20 @@
      */
     private void forwardToLocalNodes(Mutation mutation, MessagingService.Verb verb, byte[] forwardBytes, InetAddress from) throws IOException
     {
-        DataInputStream in = new DataInputStream(new FastByteArrayInputStream(forwardBytes));
-        int size = in.readInt();
-
-        // tell the recipients who to send their ack to
-        MessageOut<Mutation> message = new MessageOut<>(verb, mutation, Mutation.serializer).withParameter(Mutation.FORWARD_FROM, from.getAddress());
-        // Send a message to each of the addresses on our Forward List
-        for (int i = 0; i < size; i++)
+        try (DataInputStream in = new DataInputStream(new FastByteArrayInputStream(forwardBytes)))
         {
-            InetAddress address = CompactEndpointSerializationHelper.deserialize(in);
-            int id = in.readInt();
-            Tracing.trace("Enqueuing forwarded write to {}", address);
-            MessagingService.instance().sendOneWay(message, id, address);
+            int size = in.readInt();
+
+            // tell the recipients who to send their ack to
+            MessageOut<Mutation> message = new MessageOut<>(verb, mutation, Mutation.serializer).withParameter(Mutation.FORWARD_FROM, from.getAddress());
+            // Send a message to each of the addresses on our Forward List
+            for (int i = 0; i < size; i++)
+            {
+                InetAddress address = CompactEndpointSerializationHelper.deserialize(in);
+                int id = in.readInt();
+                Tracing.trace("Enqueuing forwarded write to {}", address);
+                MessagingService.instance().sendOneWay(message, id, address);
+            }
         }
     }
 }
diff --git a/src/java/org/apache/cassandra/db/NativeDecoratedKey.java b/src/java/org/apache/cassandra/db/NativeDecoratedKey.java
index 79dc53f..ca874c3 100644
--- a/src/java/org/apache/cassandra/db/NativeDecoratedKey.java
+++ b/src/java/org/apache/cassandra/db/NativeDecoratedKey.java
@@ -20,7 +20,6 @@
 import java.nio.ByteBuffer;
 
 import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.utils.FastByteOperations;
 import org.apache.cassandra.utils.concurrent.OpOrder;
 import org.apache.cassandra.utils.memory.MemoryUtil;
 import org.apache.cassandra.utils.memory.NativeAllocator;
diff --git a/src/java/org/apache/cassandra/db/OnDiskAtom.java b/src/java/org/apache/cassandra/db/OnDiskAtom.java
index 3e768ea..2a9c39f 100644
--- a/src/java/org/apache/cassandra/db/OnDiskAtom.java
+++ b/src/java/org/apache/cassandra/db/OnDiskAtom.java
@@ -25,7 +25,7 @@
 import org.apache.cassandra.db.composites.CellNameType;
 import org.apache.cassandra.db.composites.Composite;
 import org.apache.cassandra.io.ISSTableSerializer;
-import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.format.Version;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.serializers.MarshalException;
 
@@ -71,12 +71,12 @@
             }
         }
 
-        public OnDiskAtom deserializeFromSSTable(DataInput in, Descriptor.Version version) throws IOException
+        public OnDiskAtom deserializeFromSSTable(DataInput in, Version version) throws IOException
         {
             return deserializeFromSSTable(in, ColumnSerializer.Flag.LOCAL, Integer.MIN_VALUE, version);
         }
 
-        public OnDiskAtom deserializeFromSSTable(DataInput in, ColumnSerializer.Flag flag, int expireBefore, Descriptor.Version version) throws IOException
+        public OnDiskAtom deserializeFromSSTable(DataInput in, ColumnSerializer.Flag flag, int expireBefore, Version version) throws IOException
         {
             Composite name = type.serializer().deserialize(in);
             if (name.isEmpty())
diff --git a/src/java/org/apache/cassandra/db/PagedRangeCommand.java b/src/java/org/apache/cassandra/db/PagedRangeCommand.java
index 3188dba..40ef88e 100644
--- a/src/java/org/apache/cassandra/db/PagedRangeCommand.java
+++ b/src/java/org/apache/cassandra/db/PagedRangeCommand.java
@@ -129,7 +129,8 @@
             out.writeUTF(cmd.columnFamily);
             out.writeLong(cmd.timestamp);
 
-            AbstractBounds.serializer.serialize(cmd.keyRange, out, version);
+            MessagingService.validatePartitioner(cmd.keyRange);
+            AbstractBounds.rowPositionSerializer.serialize(cmd.keyRange, out, version);
 
             CFMetaData metadata = Schema.instance.getCFMetaData(cmd.keyspace, cmd.columnFamily);
 
@@ -158,7 +159,8 @@
             String columnFamily = in.readUTF();
             long timestamp = in.readLong();
 
-            AbstractBounds<RowPosition> keyRange = AbstractBounds.serializer.deserialize(in, version).toRowBounds();
+            AbstractBounds<RowPosition> keyRange =
+                    AbstractBounds.rowPositionSerializer.deserialize(in, MessagingService.globalPartitioner(), version);
 
             CFMetaData metadata = Schema.instance.getCFMetaData(keyspace, columnFamily);
             if (metadata == null)
@@ -196,7 +198,7 @@
             size += TypeSizes.NATIVE.sizeof(cmd.columnFamily);
             size += TypeSizes.NATIVE.sizeof(cmd.timestamp);
 
-            size += AbstractBounds.serializer.serializedSize(cmd.keyRange, version);
+            size += AbstractBounds.rowPositionSerializer.serializedSize(cmd.keyRange, version);
 
             CFMetaData metadata = Schema.instance.getCFMetaData(cmd.keyspace, cmd.columnFamily);
 
diff --git a/src/java/org/apache/cassandra/db/PreHashedDecoratedKey.java b/src/java/org/apache/cassandra/db/PreHashedDecoratedKey.java
new file mode 100644
index 0000000..472cc19
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/PreHashedDecoratedKey.java
@@ -0,0 +1,42 @@
+/*
+ * 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.db;
+
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.dht.Token;
+
+public class PreHashedDecoratedKey extends BufferDecoratedKey
+{
+    final long hash0;
+    final long hash1;
+
+    public PreHashedDecoratedKey(Token token, ByteBuffer key, long hash0, long hash1)
+    {
+        super(token, key);
+        this.hash0 = hash0;
+        this.hash1 = hash1;
+    }
+
+    @Override
+    public void filterHash(long[] dest)
+    {
+        dest[0] = hash0;
+        dest[1] = hash1;
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/RangeSliceCommand.java b/src/java/org/apache/cassandra/db/RangeSliceCommand.java
index dda26d2..664eeee 100644
--- a/src/java/org/apache/cassandra/db/RangeSliceCommand.java
+++ b/src/java/org/apache/cassandra/db/RangeSliceCommand.java
@@ -172,7 +172,8 @@
                 expr.writeTo(out);
             }
         }
-        AbstractBounds.serializer.serialize(sliceCommand.keyRange, out, version);
+        MessagingService.validatePartitioner(sliceCommand.keyRange);
+        AbstractBounds.rowPositionSerializer.serialize(sliceCommand.keyRange, out, version);
         out.writeInt(sliceCommand.maxResults);
         out.writeBoolean(sliceCommand.countCQL3Rows);
         out.writeBoolean(sliceCommand.isPaging);
@@ -202,7 +203,7 @@
         {
             rowFilter.add(IndexExpression.readFrom(in));
         }
-        AbstractBounds<RowPosition> range = AbstractBounds.serializer.deserialize(in, version).toRowBounds();
+        AbstractBounds<RowPosition> range = AbstractBounds.rowPositionSerializer.deserialize(in, MessagingService.globalPartitioner(), version);
 
         int maxResults = in.readInt();
         boolean countCQL3Rows = in.readBoolean();
@@ -236,7 +237,7 @@
                 size += TypeSizes.NATIVE.sizeofWithShortLength(expr.value);
             }
         }
-        size += AbstractBounds.serializer.serializedSize(rsc.keyRange, version);
+        size += AbstractBounds.rowPositionSerializer.serializedSize(rsc.keyRange, version);
         size += TypeSizes.NATIVE.sizeof(rsc.maxResults);
         size += TypeSizes.NATIVE.sizeof(rsc.countCQL3Rows);
         size += TypeSizes.NATIVE.sizeof(rsc.isPaging);
diff --git a/src/java/org/apache/cassandra/db/RangeSliceReply.java b/src/java/org/apache/cassandra/db/RangeSliceReply.java
index 5964ea8..ed1f523 100644
--- a/src/java/org/apache/cassandra/db/RangeSliceReply.java
+++ b/src/java/org/apache/cassandra/db/RangeSliceReply.java
@@ -57,7 +57,10 @@
 
     public static RangeSliceReply read(byte[] body, int version) throws IOException
     {
-        return serializer.deserialize(new DataInputStream(new FastByteArrayInputStream(body)), version);
+        try (DataInputStream dis = new DataInputStream(new FastByteArrayInputStream(body)))
+        {
+            return serializer.deserialize(dis, version);
+        }
     }
 
     private static class RangeSliceReplySerializer implements IVersionedSerializer<RangeSliceReply>
diff --git a/src/java/org/apache/cassandra/db/RangeTombstone.java b/src/java/org/apache/cassandra/db/RangeTombstone.java
index 9dc2723..eecf801 100644
--- a/src/java/org/apache/cassandra/db/RangeTombstone.java
+++ b/src/java/org/apache/cassandra/db/RangeTombstone.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.db;
 
 import java.io.DataInput;
-import java.io.DataOutput;
 import java.io.IOException;
 import java.security.MessageDigest;
 import java.util.*;
@@ -28,7 +27,7 @@
 import org.apache.cassandra.db.composites.CType;
 import org.apache.cassandra.db.composites.Composite;
 import org.apache.cassandra.io.ISSTableSerializer;
-import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.format.Version;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.serializers.MarshalException;
@@ -71,16 +70,16 @@
     {
         digest.update(min.toByteBuffer().duplicate());
         digest.update(max.toByteBuffer().duplicate());
-        DataOutputBuffer buffer = new DataOutputBuffer();
-        try
+
+        try (DataOutputBuffer buffer = new DataOutputBuffer())
         {
             buffer.writeLong(data.markedForDeleteAt);
+            digest.update(buffer.getData(), 0, buffer.getLength());
         }
         catch (IOException e)
         {
             throw new RuntimeException(e);
         }
-        digest.update(buffer.getData(), 0, buffer.getLength());
     }
 
     /**
@@ -360,7 +359,7 @@
             DeletionTime.serializer.serialize(t.data, out);
         }
 
-        public RangeTombstone deserializeFromSSTable(DataInput in, Descriptor.Version version) throws IOException
+        public RangeTombstone deserializeFromSSTable(DataInput in, Version version) throws IOException
         {
             Composite min = type.serializer().deserialize(in);
 
@@ -369,7 +368,7 @@
             return deserializeBody(in, min, version);
         }
 
-        public RangeTombstone deserializeBody(DataInput in, Composite min, Descriptor.Version version) throws IOException
+        public RangeTombstone deserializeBody(DataInput in, Composite min, Version version) throws IOException
         {
             Composite max = type.serializer().deserialize(in);
             DeletionTime dt = DeletionTime.serializer.deserialize(in);
@@ -380,7 +379,7 @@
             return new RangeTombstone(min, max, dt);
         }
 
-        public void skipBody(DataInput in, Descriptor.Version version) throws IOException
+        public void skipBody(DataInput in, Version version) throws IOException
         {
             type.serializer().skip(in);
             DeletionTime.serializer.skip(in);
diff --git a/src/java/org/apache/cassandra/db/RangeTombstoneList.java b/src/java/org/apache/cassandra/db/RangeTombstoneList.java
index bd6e669..37f1ef4 100644
--- a/src/java/org/apache/cassandra/db/RangeTombstoneList.java
+++ b/src/java/org/apache/cassandra/db/RangeTombstoneList.java
@@ -39,7 +39,6 @@
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.utils.ObjectSizes;
 import org.apache.cassandra.utils.memory.AbstractAllocator;
-import org.apache.cassandra.utils.memory.HeapPool;
 
 /**
  * Data structure holding the range tombstones of a ColumnFamily.
diff --git a/src/java/org/apache/cassandra/db/ReadVerbHandler.java b/src/java/org/apache/cassandra/db/ReadVerbHandler.java
index 35082e6..8c167ed 100644
--- a/src/java/org/apache/cassandra/db/ReadVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/ReadVerbHandler.java
@@ -17,10 +17,6 @@
  */
 package org.apache.cassandra.db;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.db.filter.TombstoneOverwhelmingException;
 import org.apache.cassandra.net.IVerbHandler;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessageOut;
@@ -30,8 +26,6 @@
 
 public class ReadVerbHandler implements IVerbHandler<ReadCommand>
 {
-    private static final Logger logger = LoggerFactory.getLogger( ReadVerbHandler.class );
-
     public void doVerb(MessageIn<ReadCommand> message, int id)
     {
         if (StorageService.instance.isBootstrapMode())
@@ -41,16 +35,7 @@
 
         ReadCommand command = message.payload;
         Keyspace keyspace = Keyspace.open(command.ksName);
-        Row row;
-        try
-        {
-            row = command.getRow(keyspace);
-        }
-        catch (TombstoneOverwhelmingException e)
-        {
-            // error already logged.  Drop the request
-            return;
-        }
+        Row row = command.getRow(keyspace);
 
         MessageOut<ReadResponse> reply = new MessageOut<ReadResponse>(MessagingService.Verb.REQUEST_RESPONSE,
                                                                       getResponse(command, row),
diff --git a/src/java/org/apache/cassandra/db/RowIndexEntry.java b/src/java/org/apache/cassandra/db/RowIndexEntry.java
index 77b745c..f9d8c6d 100644
--- a/src/java/org/apache/cassandra/db/RowIndexEntry.java
+++ b/src/java/org/apache/cassandra/db/RowIndexEntry.java
@@ -27,15 +27,14 @@
 import com.google.common.primitives.Ints;
 
 import org.apache.cassandra.cache.IMeasurableMemory;
-import org.apache.cassandra.db.composites.CType;
 import org.apache.cassandra.io.ISerializer;
-import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.sstable.IndexHelper;
+import org.apache.cassandra.io.sstable.format.Version;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.utils.ObjectSizes;
 
-public class RowIndexEntry implements IMeasurableMemory
+public class RowIndexEntry<T> implements IMeasurableMemory
 {
     private static final long EMPTY_SIZE = ObjectSizes.measure(new RowIndexEntry(0));
 
@@ -46,12 +45,12 @@
         this.position = position;
     }
 
-    protected int promotedSize(CType type)
+    public int promotedSize(ISerializer<T> idxSerializer)
     {
         return 0;
     }
 
-    public static RowIndexEntry create(long position, DeletionTime deletionTime, ColumnIndex index)
+    public static RowIndexEntry<IndexHelper.IndexInfo> create(long position, DeletionTime deletionTime, ColumnIndex index)
     {
         assert index != null;
         assert deletionTime != null;
@@ -62,7 +61,7 @@
         if (index.columnsIndex.size() > 1)
             return new IndexedEntry(position, deletionTime, index.columnsIndex);
         else
-            return new RowIndexEntry(position);
+            return new RowIndexEntry<>(position);
     }
 
     /**
@@ -79,7 +78,16 @@
         throw new UnsupportedOperationException();
     }
 
-    public List<IndexHelper.IndexInfo> columnsIndex()
+    /**
+     * @return the offset to the start of the header information for this row.
+     * For some formats this may not be the start of the row.
+     */
+    public long headerOffset()
+    {
+        return 0;
+    }
+
+    public List<T> columnsIndex()
     {
         return Collections.emptyList();
     }
@@ -89,31 +97,37 @@
         return EMPTY_SIZE;
     }
 
-    public static class Serializer
+    public static interface IndexSerializer<T>
     {
-        private final CType type;
+        void serialize(RowIndexEntry<T> rie, DataOutputPlus out) throws IOException;
+        RowIndexEntry<T> deserialize(DataInput in, Version version) throws IOException;
+        public int serializedSize(RowIndexEntry<T> rie);
+    }
 
-        public Serializer(CType type)
+    public static class Serializer implements IndexSerializer<IndexHelper.IndexInfo>
+    {
+        private final ISerializer<IndexHelper.IndexInfo> idxSerializer;
+
+        public Serializer(ISerializer<IndexHelper.IndexInfo> idxSerializer)
         {
-            this.type = type;
+            this.idxSerializer = idxSerializer;
         }
 
-        public void serialize(RowIndexEntry rie, DataOutputPlus out) throws IOException
+        public void serialize(RowIndexEntry<IndexHelper.IndexInfo> rie, DataOutputPlus out) throws IOException
         {
             out.writeLong(rie.position);
-            out.writeInt(rie.promotedSize(type));
+            out.writeInt(rie.promotedSize(idxSerializer));
 
             if (rie.isIndexed())
             {
                 DeletionTime.serializer.serialize(rie.deletionTime(), out);
                 out.writeInt(rie.columnsIndex().size());
-                ISerializer<IndexHelper.IndexInfo> idxSerializer = type.indexSerializer();
                 for (IndexHelper.IndexInfo info : rie.columnsIndex())
                     idxSerializer.serialize(info, out);
             }
         }
 
-        public RowIndexEntry deserialize(DataInput in, Descriptor.Version version) throws IOException
+        public RowIndexEntry<IndexHelper.IndexInfo> deserialize(DataInput in, Version version) throws IOException
         {
             long position = in.readLong();
 
@@ -123,8 +137,7 @@
                 DeletionTime deletionTime = DeletionTime.serializer.deserialize(in);
 
                 int entries = in.readInt();
-                ISerializer<IndexHelper.IndexInfo> idxSerializer = type.indexSerializer();
-                List<IndexHelper.IndexInfo> columnsIndex = new ArrayList<IndexHelper.IndexInfo>(entries);
+                List<IndexHelper.IndexInfo> columnsIndex = new ArrayList<>(entries);
                 for (int i = 0; i < entries; i++)
                     columnsIndex.add(idxSerializer.deserialize(in));
 
@@ -132,7 +145,7 @@
             }
             else
             {
-                return new RowIndexEntry(position);
+                return new RowIndexEntry<>(position);
             }
         }
 
@@ -151,9 +164,9 @@
             FileUtils.skipBytesFully(in, size);
         }
 
-        public int serializedSize(RowIndexEntry rie)
+        public int serializedSize(RowIndexEntry<IndexHelper.IndexInfo> rie)
         {
-            int size = TypeSizes.NATIVE.sizeof(rie.position) + TypeSizes.NATIVE.sizeof(rie.promotedSize(type));
+            int size = TypeSizes.NATIVE.sizeof(rie.position) + TypeSizes.NATIVE.sizeof(rie.promotedSize(idxSerializer));
 
             if (rie.isIndexed())
             {
@@ -162,11 +175,11 @@
                 size += DeletionTime.serializer.serializedSize(rie.deletionTime(), TypeSizes.NATIVE);
                 size += TypeSizes.NATIVE.sizeof(index.size());
 
-                ISerializer<IndexHelper.IndexInfo> idxSerializer = type.indexSerializer();
                 for (IndexHelper.IndexInfo info : index)
                     size += idxSerializer.serializedSize(info, TypeSizes.NATIVE);
             }
 
+
             return size;
         }
     }
@@ -174,7 +187,7 @@
     /**
      * An entry in the row index for a row whose columns are indexed.
      */
-    private static class IndexedEntry extends RowIndexEntry
+    private static class IndexedEntry extends RowIndexEntry<IndexHelper.IndexInfo>
     {
         private final DeletionTime deletionTime;
         private final List<IndexHelper.IndexInfo> columnsIndex;
@@ -204,12 +217,11 @@
         }
 
         @Override
-        public int promotedSize(CType type)
+        public int promotedSize(ISerializer<IndexHelper.IndexInfo> idxSerializer)
         {
             TypeSizes typeSizes = TypeSizes.NATIVE;
             long size = DeletionTime.serializer.serializedSize(deletionTime, typeSizes);
             size += typeSizes.sizeof(columnsIndex.size()); // number of entries
-            ISerializer<IndexHelper.IndexInfo> idxSerializer = type.indexSerializer();
             for (IndexHelper.IndexInfo info : columnsIndex)
                 size += idxSerializer.serializedSize(info, typeSizes);
 
diff --git a/src/java/org/apache/cassandra/db/RowIteratorFactory.java b/src/java/org/apache/cassandra/db/RowIteratorFactory.java
index d6c8905..f4619f2 100644
--- a/src/java/org/apache/cassandra/db/RowIteratorFactory.java
+++ b/src/java/org/apache/cassandra/db/RowIteratorFactory.java
@@ -26,7 +26,8 @@
 import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
 import org.apache.cassandra.db.filter.QueryFilter;
 import org.apache.cassandra.db.filter.IDiskAtomFilter;
-import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.db.filter.TombstoneOverwhelmingException;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.utils.CloseableIterator;
 import org.apache.cassandra.utils.MergeIterator;
 
diff --git a/src/java/org/apache/cassandra/db/RowPosition.java b/src/java/org/apache/cassandra/db/RowPosition.java
index 3bcd627..3fa0465 100644
--- a/src/java/org/apache/cassandra/db/RowPosition.java
+++ b/src/java/org/apache/cassandra/db/RowPosition.java
@@ -56,7 +56,7 @@
     public Kind kind();
     public boolean isMinimum();
 
-    public static class RowPositionSerializer implements ISerializer<RowPosition>
+    public static class RowPositionSerializer implements IPartitionerDependentSerializer<RowPosition>
     {
         /*
          * We need to be able to serialize both Token.KeyBound and
@@ -69,17 +69,17 @@
          * token is recreated on the other side). In the other cases, we then
          * serialize the token.
          */
-        public void serialize(RowPosition pos, DataOutputPlus out) throws IOException
+        public void serialize(RowPosition pos, DataOutputPlus out, int version) throws IOException
         {
             Kind kind = pos.kind();
             out.writeByte(kind.ordinal());
             if (kind == Kind.ROW_KEY)
                 ByteBufferUtil.writeWithShortLength(((DecoratedKey)pos).getKey(), out);
             else
-                Token.serializer.serialize(pos.getToken(), out);
+                Token.serializer.serialize(pos.getToken(), out, version);
         }
 
-        public RowPosition deserialize(DataInput in) throws IOException
+        public RowPosition deserialize(DataInput in, IPartitioner p, int version) throws IOException
         {
             Kind kind = Kind.fromOrdinal(in.readByte());
             if (kind == Kind.ROW_KEY)
@@ -89,23 +89,23 @@
             }
             else
             {
-                Token t = Token.serializer.deserialize(in);
+                Token t = Token.serializer.deserialize(in, p, version);
                 return kind == Kind.MIN_BOUND ? t.minKeyBound() : t.maxKeyBound();
             }
         }
 
-        public long serializedSize(RowPosition pos, TypeSizes typeSizes)
+        public long serializedSize(RowPosition pos, int version)
         {
             Kind kind = pos.kind();
             int size = 1; // 1 byte for enum
             if (kind == Kind.ROW_KEY)
             {
                 int keySize = ((DecoratedKey)pos).getKey().remaining();
-                size += typeSizes.sizeof((short) keySize) + keySize;
+                size += TypeSizes.NATIVE.sizeof((short) keySize) + keySize;
             }
             else
             {
-                size += Token.serializer.serializedSize(pos.getToken(), typeSizes);
+                size += Token.serializer.serializedSize(pos.getToken(), version);
             }
             return size;
         }
diff --git a/src/java/org/apache/cassandra/db/SchemaCheckVerbHandler.java b/src/java/org/apache/cassandra/db/SchemaCheckVerbHandler.java
index 1a1f7a9..4270a24 100644
--- a/src/java/org/apache/cassandra/db/SchemaCheckVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/SchemaCheckVerbHandler.java
@@ -35,7 +35,7 @@
 
     public void doVerb(MessageIn message, int id)
     {
-        logger.debug("Received schema check request.");
+        logger.trace("Received schema check request.");
         MessageOut<UUID> response = new MessageOut<UUID>(MessagingService.Verb.INTERNAL_RESPONSE, Schema.instance.getVersion(), UUIDSerializer.serializer);
         MessagingService.instance().sendReply(response, id, message.from);
     }
diff --git a/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java b/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java
index fe4ebd3..2f14fb1 100644
--- a/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java
+++ b/src/java/org/apache/cassandra/db/SizeEstimatesRecorder.java
@@ -25,7 +25,8 @@
 
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.locator.TokenMetadata;
 import org.apache.cassandra.service.MigrationListener;
 import org.apache.cassandra.service.MigrationManager;
 import org.apache.cassandra.service.StorageService;
@@ -56,17 +57,18 @@
 
     public void run()
     {
-        if (!StorageService.instance.getTokenMetadata().isMember(FBUtilities.getBroadcastAddress()))
+        TokenMetadata metadata = StorageService.instance.getTokenMetadata().cloneOnlyTokenMap();
+        if (!metadata.isMember(FBUtilities.getBroadcastAddress()))
         {
             logger.debug("Node is not part of the ring; not recording size estimates");
             return;
         }
 
-        logger.debug("Recording size estimates");
+        logger.trace("Recording size estimates");
 
         // find primary token ranges for the local node.
         Collection<Token> localTokens = StorageService.instance.getLocalTokens();
-        Collection<Range<Token>> localRanges = StorageService.instance.getTokenMetadata().getPrimaryRangesFor(localTokens);
+        Collection<Range<Token>> localRanges = metadata.getPrimaryRangesFor(localTokens);
 
         for (Keyspace keyspace : Keyspace.nonSystem())
         {
@@ -75,7 +77,7 @@
                 long start = System.nanoTime();
                 recordSizeEstimates(table, localRanges);
                 long passed = System.nanoTime() - start;
-                logger.debug("Spent {} milliseconds on estimating {}.{} size",
+                logger.trace("Spent {} milliseconds on estimating {}.{} size",
                              TimeUnit.NANOSECONDS.toMillis(passed),
                              table.metadata.ksName,
                              table.metadata.cfName);
@@ -83,30 +85,34 @@
         }
     }
 
+    @SuppressWarnings("resource")
     private void recordSizeEstimates(ColumnFamilyStore table, Collection<Range<Token>> localRanges)
     {
+        List<Range<Token>> unwrappedRanges = Range.normalize(localRanges);
         // for each local primary range, estimate (crudely) mean partition size and partitions count.
         Map<Range<Token>, Pair<Long, Long>> estimates = new HashMap<>(localRanges.size());
-        for (Range<Token> range : localRanges)
+        for (Range<Token> range : unwrappedRanges)
         {
             // filter sstables that have partitions in this range.
             Refs<SSTableReader> refs = null;
-            while (refs == null)
-            {
-                ColumnFamilyStore.ViewFragment view = table.select(table.viewFilter(range.toRowBounds()));
-                refs = Refs.tryRef(view.sstables);
-            }
-
             long partitionsCount, meanPartitionSize;
+
             try
             {
+                while (refs == null)
+                {
+                    ColumnFamilyStore.ViewFragment view = table.select(table.viewFilter(Range.makeRowRange(range)));
+                    refs = Refs.tryRef(view.sstables);
+                }
+
                 // calculate the estimates.
                 partitionsCount = estimatePartitionsCount(refs, range);
                 meanPartitionSize = estimateMeanPartitionSize(refs);
             }
             finally
             {
-                refs.release();
+                if (refs != null)
+                    refs.release();
             }
 
             estimates.put(range, Pair.create(partitionsCount, meanPartitionSize));
diff --git a/src/java/org/apache/cassandra/db/SliceFromReadCommand.java b/src/java/org/apache/cassandra/db/SliceFromReadCommand.java
index 461a3a1..edace9d 100644
--- a/src/java/org/apache/cassandra/db/SliceFromReadCommand.java
+++ b/src/java/org/apache/cassandra/db/SliceFromReadCommand.java
@@ -65,7 +65,7 @@
         // reads in order to guarantee that the static columns are fetched.  See CASSANDRA-8502 for more details.
         if (filter.reversed && filter.hasStaticSlice(cfm))
         {
-            logger.debug("Splitting reversed slice with static columns into two reads");
+            logger.trace("Splitting reversed slice with static columns into two reads");
             Pair<SliceQueryFilter, SliceQueryFilter> newFilters = filter.splitOutStaticSlice(cfm);
 
             Row normalResults =  keyspace.getRow(new QueryFilter(dk, cfName, newFilters.right, timestamp));
diff --git a/src/java/org/apache/cassandra/db/SuperColumns.java b/src/java/org/apache/cassandra/db/SuperColumns.java
index 2006cbd..65e153f 100644
--- a/src/java/org/apache/cassandra/db/SuperColumns.java
+++ b/src/java/org/apache/cassandra/db/SuperColumns.java
@@ -186,7 +186,7 @@
             {
                 // Note that, because the filter in argument is the one from thrift, 'name' are SimpleDenseCellName.
                 // So calling name.slice() would be incorrect, as simple cell names don't handle the EOC properly.
-                // This is why we call toByteBuffer() and rebuild a  Composite of the right type before call slice().
+                // This is why we call buffer() and rebuild a  Composite of the right type before call slice().
                 slices[i++] = type.make(name.toByteBuffer()).slice();
             }
             return new SliceQueryFilter(slices, false, slices.length, 1);
diff --git a/src/java/org/apache/cassandra/db/SystemKeyspace.java b/src/java/org/apache/cassandra/db/SystemKeyspace.java
index f8cf1ab..e0d5f66 100644
--- a/src/java/org/apache/cassandra/db/SystemKeyspace.java
+++ b/src/java/org/apache/cassandra/db/SystemKeyspace.java
@@ -17,41 +17,42 @@
  */
 package org.apache.cassandra.db;
 
-import java.io.DataInputStream;
-import java.io.IOException;
+import java.io.*;
 import java.net.InetAddress;
 import java.nio.ByteBuffer;
 import java.util.*;
-import javax.management.openmbean.*;
+import java.util.concurrent.TimeUnit;
+import javax.management.openmbean.OpenDataException;
+import javax.management.openmbean.TabularData;
 
 import com.google.common.base.Function;
-import com.google.common.collect.HashMultimap;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.SetMultimap;
-import com.google.common.collect.Sets;
+import com.google.common.collect.*;
+import com.google.common.io.ByteStreams;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.cache.CachingOptions;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.KSMetaData;
-import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
-import org.apache.cassandra.db.columniterator.IdentityQueryFilter;
-import org.apache.cassandra.db.compaction.CompactionHistoryTabularData;
 import org.apache.cassandra.db.commitlog.ReplayPosition;
+import org.apache.cassandra.db.compaction.CompactionHistoryTabularData;
+import org.apache.cassandra.db.compaction.LeveledCompactionStrategy;
 import org.apache.cassandra.db.composites.Composite;
 import org.apache.cassandra.db.filter.QueryFilter;
 import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.locator.IEndpointSnitch;
+import org.apache.cassandra.locator.LocalStrategy;
 import org.apache.cassandra.metrics.RestorableMeter;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.LegacySchemaTables;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.service.paxos.Commit;
 import org.apache.cassandra.service.paxos.PaxosState;
@@ -62,37 +63,220 @@
 import static org.apache.cassandra.cql3.QueryProcessor.executeInternal;
 import static org.apache.cassandra.cql3.QueryProcessor.executeOnceInternal;
 
-public class SystemKeyspace
+public final class SystemKeyspace
 {
     private static final Logger logger = LoggerFactory.getLogger(SystemKeyspace.class);
 
-    // see CFMetaData for schema definitions
-    public static final String PEERS_CF = "peers";
-    public static final String PEER_EVENTS_CF = "peer_events";
-    public static final String LOCAL_CF = "local";
-    public static final String INDEX_CF = "IndexInfo";
-    public static final String HINTS_CF = "hints";
-    public static final String RANGE_XFERS_CF = "range_xfers";
-    public static final String BATCHLOG_CF = "batchlog";
-    // see layout description in the DefsTables class header
-    public static final String SCHEMA_KEYSPACES_CF = "schema_keyspaces";
-    public static final String SCHEMA_COLUMNFAMILIES_CF = "schema_columnfamilies";
-    public static final String SCHEMA_COLUMNS_CF = "schema_columns";
-    public static final String SCHEMA_TRIGGERS_CF = "schema_triggers";
-    public static final String SCHEMA_USER_TYPES_CF = "schema_usertypes";
-    public static final String COMPACTION_LOG = "compactions_in_progress";
-    public static final String PAXOS_CF = "paxos";
-    public static final String SSTABLE_ACTIVITY_CF = "sstable_activity";
-    public static final String COMPACTION_HISTORY_CF = "compaction_history";
-    public static final String SIZE_ESTIMATES_CF = "size_estimates";
+    // Used to indicate that there was a previous version written to the legacy (pre 1.2)
+    // system.Versions table, but that we cannot read it. Suffice to say, any upgrade should
+    // proceed through 1.2.x before upgrading to the current version.
+    public static final CassandraVersion UNREADABLE_VERSION = new CassandraVersion("0.0.0-unknown");
 
-    private static final String LOCAL_KEY = "local";
+    // Used to indicate that no previous version information was found. When encountered, we assume that
+    // Cassandra was not previously installed and we're in the process of starting a fresh node.
+    public static final CassandraVersion NULL_VERSION = new CassandraVersion("0.0.0-absent");
 
-    public static final List<String> allSchemaCfs = Arrays.asList(SCHEMA_KEYSPACES_CF,
-                                                                  SCHEMA_COLUMNFAMILIES_CF,
-                                                                  SCHEMA_COLUMNS_CF,
-                                                                  SCHEMA_TRIGGERS_CF,
-                                                                  SCHEMA_USER_TYPES_CF);
+    public static final String NAME = "system";
+
+    public static final String HINTS = "hints";
+    public static final String BATCHLOG = "batchlog";
+    public static final String PAXOS = "paxos";
+    public static final String BUILT_INDEXES = "IndexInfo";
+    public static final String LOCAL = "local";
+    public static final String PEERS = "peers";
+    public static final String PEER_EVENTS = "peer_events";
+    public static final String RANGE_XFERS = "range_xfers";
+    public static final String COMPACTIONS_IN_PROGRESS = "compactions_in_progress";
+    public static final String COMPACTION_HISTORY = "compaction_history";
+    public static final String SSTABLE_ACTIVITY = "sstable_activity";
+    public static final String SIZE_ESTIMATES = "size_estimates";
+    public static final String AVAILABLE_RANGES = "available_ranges";
+
+    public static final CFMetaData Hints =
+        compile(HINTS,
+                "hints awaiting delivery",
+                "CREATE TABLE %s ("
+                + "target_id uuid,"
+                + "hint_id timeuuid,"
+                + "message_version int,"
+                + "mutation blob,"
+                + "PRIMARY KEY ((target_id), hint_id, message_version)) "
+                + "WITH COMPACT STORAGE")
+                .compactionStrategyOptions(Collections.singletonMap("enabled", "false"))
+                .gcGraceSeconds(0);
+
+    public static final CFMetaData Batchlog =
+        compile(BATCHLOG,
+                "batches awaiting replay",
+                "CREATE TABLE %s ("
+                + "id uuid,"
+                + "data blob,"
+                + "version int,"
+                + "written_at timestamp,"
+                + "PRIMARY KEY ((id)))")
+                .compactionStrategyOptions(Collections.singletonMap("min_threshold", "2"))
+                .gcGraceSeconds(0);
+
+    private static final CFMetaData Paxos =
+        compile(PAXOS,
+                "in-progress paxos proposals",
+                "CREATE TABLE %s ("
+                + "row_key blob,"
+                + "cf_id UUID,"
+                + "in_progress_ballot timeuuid,"
+                + "most_recent_commit blob,"
+                + "most_recent_commit_at timeuuid,"
+                + "proposal blob,"
+                + "proposal_ballot timeuuid,"
+                + "PRIMARY KEY ((row_key), cf_id))")
+                .compactionStrategyClass(LeveledCompactionStrategy.class);
+
+    // TODO: make private
+    public static final CFMetaData BuiltIndexes =
+        compile(BUILT_INDEXES,
+                "built column indexes",
+                "CREATE TABLE \"%s\" ("
+                + "table_name text,"
+                + "index_name text,"
+                + "PRIMARY KEY ((table_name), index_name)) "
+                + "WITH COMPACT STORAGE");
+
+    private static final CFMetaData Local =
+        compile(LOCAL,
+                "information about the local node",
+                "CREATE TABLE %s ("
+                + "key text,"
+                + "bootstrapped text,"
+                + "broadcast_address inet,"
+                + "cluster_name text,"
+                + "cql_version text,"
+                + "data_center text,"
+                + "gossip_generation int,"
+                + "host_id uuid,"
+                + "listen_address inet,"
+                + "native_protocol_version text,"
+                + "partitioner text,"
+                + "rack text,"
+                + "release_version text,"
+                + "rpc_address inet,"
+                + "schema_version uuid,"
+                + "thrift_version text,"
+                + "tokens set<varchar>,"
+                + "truncated_at map<uuid, blob>,"
+                + "PRIMARY KEY ((key)))");
+
+    private static final CFMetaData Peers =
+        compile(PEERS,
+                "information about known peers in the cluster",
+                "CREATE TABLE %s ("
+                + "peer inet,"
+                + "data_center text,"
+                + "host_id uuid,"
+                + "preferred_ip inet,"
+                + "rack text,"
+                + "release_version text,"
+                + "rpc_address inet,"
+                + "schema_version uuid,"
+                + "tokens set<varchar>,"
+                + "PRIMARY KEY ((peer)))");
+
+    private static final CFMetaData PeerEvents =
+        compile(PEER_EVENTS,
+                "events related to peers",
+                "CREATE TABLE %s ("
+                + "peer inet,"
+                + "hints_dropped map<uuid, int>,"
+                + "PRIMARY KEY ((peer)))");
+
+    private static final CFMetaData RangeXfers =
+        compile(RANGE_XFERS,
+                "ranges requested for transfer",
+                "CREATE TABLE %s ("
+                + "token_bytes blob,"
+                + "requested_at timestamp,"
+                + "PRIMARY KEY ((token_bytes)))");
+
+    private static final CFMetaData CompactionsInProgress =
+        compile(COMPACTIONS_IN_PROGRESS,
+                "unfinished compactions",
+                "CREATE TABLE %s ("
+                + "id uuid,"
+                + "columnfamily_name text,"
+                + "inputs set<int>,"
+                + "keyspace_name text,"
+                + "PRIMARY KEY ((id)))");
+
+    private static final CFMetaData CompactionHistory =
+        compile(COMPACTION_HISTORY,
+                "week-long compaction history",
+                "CREATE TABLE %s ("
+                + "id uuid,"
+                + "bytes_in bigint,"
+                + "bytes_out bigint,"
+                + "columnfamily_name text,"
+                + "compacted_at timestamp,"
+                + "keyspace_name text,"
+                + "rows_merged map<int, bigint>,"
+                + "PRIMARY KEY ((id)))")
+                .defaultTimeToLive((int) TimeUnit.DAYS.toSeconds(7));
+
+    private static final CFMetaData SSTableActivity =
+        compile(SSTABLE_ACTIVITY,
+                "historic sstable read rates",
+                "CREATE TABLE %s ("
+                + "keyspace_name text,"
+                + "columnfamily_name text,"
+                + "generation int,"
+                + "rate_120m double,"
+                + "rate_15m double,"
+                + "PRIMARY KEY ((keyspace_name, columnfamily_name, generation)))");
+
+    private static final CFMetaData SizeEstimates =
+        compile(SIZE_ESTIMATES,
+                "per-table primary range size estimates",
+                "CREATE TABLE %s ("
+                + "keyspace_name text,"
+                + "table_name text,"
+                + "range_start text,"
+                + "range_end text,"
+                + "mean_partition_size bigint,"
+                + "partitions_count bigint,"
+                + "PRIMARY KEY ((keyspace_name), table_name, range_start, range_end))")
+                .gcGraceSeconds(0);
+
+    private static final CFMetaData AvailableRanges =
+        compile(AVAILABLE_RANGES,
+                "Available keyspace/ranges during bootstrap/replace that are ready to be served",
+                "CREATE TABLE %s ("
+                        + "keyspace_name text PRIMARY KEY,"
+                        + "ranges set<blob>"
+                        + ")");
+
+    private static CFMetaData compile(String name, String description, String schema)
+    {
+        return CFMetaData.compile(String.format(schema, name), NAME)
+                         .comment(description);
+    }
+
+    public static KSMetaData definition()
+    {
+        Iterable<CFMetaData> tables =
+            Iterables.concat(LegacySchemaTables.All,
+                             Arrays.asList(BuiltIndexes,
+                                           Hints,
+                                           Batchlog,
+                                           Paxos,
+                                           Local,
+                                           Peers,
+                                           PeerEvents,
+                                           RangeXfers,
+                                           CompactionsInProgress,
+                                           CompactionHistory,
+                                           SSTableActivity,
+                                           SizeEstimates,
+                                           AvailableRanges));
+        return new KSMetaData(NAME, LocalStrategy.class, Collections.<String, String>emptyMap(), true, tables);
+    }
 
     private static volatile Map<UUID, Pair<ReplayPosition, Long>> truncationRecords;
 
@@ -110,37 +294,29 @@
 
     public static void finishStartup()
     {
-        setupVersion();
-
-        migrateIndexInterval();
-        migrateCachingOption();
-        // add entries to system schema columnfamilies for the hardcoded system definitions
-        KSMetaData ksmd = Schema.instance.getKSMetaData(Keyspace.SYSTEM_KS);
-
-        long timestamp = FBUtilities.timestampMicros();
-
-        // delete old, possibly obsolete entries in schema columnfamilies
-        for (String cfname : Arrays.asList(SystemKeyspace.SCHEMA_KEYSPACES_CF,
-                                           SystemKeyspace.SCHEMA_COLUMNFAMILIES_CF,
-                                           SystemKeyspace.SCHEMA_COLUMNS_CF,
-                                           SystemKeyspace.SCHEMA_TRIGGERS_CF,
-                                           SystemKeyspace.SCHEMA_USER_TYPES_CF))
-        {
-            executeOnceInternal(String.format("DELETE FROM system.%s USING TIMESTAMP ? WHERE keyspace_name = ?", cfname),
-                                timestamp,
-                                ksmd.name);
-        }
-
-        // (+1 to timestamp to make sure we don't get shadowed by the tombstones we just added)
-        ksmd.toSchema(timestamp + 1).apply();
+        LegacySchemaTables.saveSystemKeyspaceSchema();
     }
 
-    private static void setupVersion()
+    public static void persistLocalMetadata()
     {
-        String req = "INSERT INTO system.%s (key, release_version, cql_version, thrift_version, native_protocol_version, data_center, rack, partitioner, rpc_address, broadcast_address, listen_address) VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
+        String req = "INSERT INTO system.%s (" +
+                     "key," +
+                     "cluster_name," +
+                     "release_version," +
+                     "cql_version," +
+                     "thrift_version," +
+                     "native_protocol_version," +
+                     "data_center," +
+                     "rack," +
+                     "partitioner," +
+                     "rpc_address," +
+                     "broadcast_address," +
+                     "listen_address" +
+                     ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
         IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
-        executeOnceInternal(String.format(req, LOCAL_CF),
-                            LOCAL_KEY,
+        executeOnceInternal(String.format(req, LOCAL),
+                            LOCAL,
+                            DatabaseDescriptor.getClusterName(),
                             FBUtilities.getReleaseVersionString(),
                             QueryProcessor.CQL_VERSION.toString(),
                             cassandraConstants.VERSION,
@@ -153,66 +329,16 @@
                             FBUtilities.getLocalAddress());
     }
 
-    // TODO: In 3.0, remove this and the index_interval column from system.schema_columnfamilies
-    /** Migrates index_interval values to min_index_interval and sets index_interval to null */
-    private static void migrateIndexInterval()
-    {
-        for (UntypedResultSet.Row row : executeOnceInternal(String.format("SELECT * FROM system.%s", SCHEMA_COLUMNFAMILIES_CF)))
-        {
-            if (!row.has("index_interval"))
-                continue;
-
-            logger.debug("Migrating index_interval to min_index_interval");
-
-            CFMetaData table = CFMetaData.fromSchema(row);
-            String query = String.format("SELECT writetime(type) FROM system.%s WHERE keyspace_name = ? AND columnfamily_name = ?", SCHEMA_COLUMNFAMILIES_CF);
-            long timestamp = executeOnceInternal(query, table.ksName, table.cfName).one().getLong("writetime(type)");
-            try
-            {
-                table.toSchema(timestamp).apply();
-            }
-            catch (ConfigurationException e)
-            {
-                // shouldn't happen
-            }
-        }
-    }
-
-    private static void migrateCachingOption()
-    {
-        for (UntypedResultSet.Row row : executeOnceInternal(String.format("SELECT * FROM system.%s", SCHEMA_COLUMNFAMILIES_CF)))
-        {
-            if (!row.has("caching"))
-                continue;
-
-            if (!CachingOptions.isLegacy(row.getString("caching")))
-                continue;
-            try
-            {
-                CachingOptions caching = CachingOptions.fromString(row.getString("caching"));
-                CFMetaData table = CFMetaData.fromSchema(row);
-                logger.info("Migrating caching option {} to {} for {}.{}", row.getString("caching"), caching.toString(), table.ksName, table.cfName);
-                String query = String.format("SELECT writetime(type) FROM system.%s WHERE keyspace_name = ? AND columnfamily_name = ?", SCHEMA_COLUMNFAMILIES_CF);
-                long timestamp = executeOnceInternal(query, table.ksName, table.cfName).one().getLong("writetime(type)");
-                table.toSchema(timestamp).apply();
-            }
-            catch (ConfigurationException e)
-            {
-                // shouldn't happen
-            }
-        }
-    }
-
     /**
      * Write compaction log, except columfamilies under system keyspace.
      *
-     * @param cfs
+     * @param cfs cfs to compact
      * @param toCompact sstables to compact
      * @return compaction task id or null if cfs is under system keyspace
      */
     public static UUID startCompaction(ColumnFamilyStore cfs, Iterable<SSTableReader> toCompact)
     {
-        if (Keyspace.SYSTEM_KS.equals(cfs.keyspace.getName()))
+        if (NAME.equals(cfs.keyspace.getName()))
             return null;
 
         UUID compactionId = UUIDGen.getTimeUUID();
@@ -224,8 +350,8 @@
             }
         });
         String req = "INSERT INTO system.%s (id, keyspace_name, columnfamily_name, inputs) VALUES (?, ?, ?, ?)";
-        executeInternal(String.format(req, COMPACTION_LOG), compactionId, cfs.keyspace.getName(), cfs.name, Sets.newHashSet(generations));
-        forceBlockingFlush(COMPACTION_LOG);
+        executeInternal(String.format(req, COMPACTIONS_IN_PROGRESS), compactionId, cfs.keyspace.getName(), cfs.name, Sets.newHashSet(generations));
+        forceBlockingFlush(COMPACTIONS_IN_PROGRESS);
         return compactionId;
     }
 
@@ -238,8 +364,8 @@
     {
         assert taskId != null;
 
-        executeInternal(String.format("DELETE FROM system.%s WHERE id = ?", COMPACTION_LOG), taskId);
-        forceBlockingFlush(COMPACTION_LOG);
+        executeInternal(String.format("DELETE FROM system.%s WHERE id = ?", COMPACTIONS_IN_PROGRESS), taskId);
+        forceBlockingFlush(COMPACTIONS_IN_PROGRESS);
     }
 
     /**
@@ -249,7 +375,7 @@
     public static Map<Pair<String, String>, Map<Integer, UUID>> getUnfinishedCompactions()
     {
         String req = "SELECT * FROM system.%s";
-        UntypedResultSet resultSet = executeInternal(String.format(req, COMPACTION_LOG));
+        UntypedResultSet resultSet = executeInternal(String.format(req, COMPACTIONS_IN_PROGRESS));
 
         Map<Pair<String, String>, Map<Integer, UUID>> unfinishedCompactions = new HashMap<>();
         for (UntypedResultSet.Row row : resultSet)
@@ -274,8 +400,8 @@
 
     public static void discardCompactionsInProgress()
     {
-        ColumnFamilyStore compactionLog = Keyspace.open(Keyspace.SYSTEM_KS).getColumnFamilyStore(COMPACTION_LOG);
-        compactionLog.truncateBlocking();
+        ColumnFamilyStore compactionLog = Keyspace.open(NAME).getColumnFamilyStore(COMPACTIONS_IN_PROGRESS);
+        compactionLog.truncateBlocking(false);
     }
 
     public static void updateCompactionHistory(String ksname,
@@ -286,24 +412,24 @@
                                                Map<Integer, Long> rowsMerged)
     {
         // don't write anything when the history table itself is compacted, since that would in turn cause new compactions
-        if (ksname.equals("system") && cfname.equals(COMPACTION_HISTORY_CF))
+        if (ksname.equals("system") && cfname.equals(COMPACTION_HISTORY))
             return;
         String req = "INSERT INTO system.%s (id, keyspace_name, columnfamily_name, compacted_at, bytes_in, bytes_out, rows_merged) VALUES (?, ?, ?, ?, ?, ?, ?)";
-        executeInternal(String.format(req, COMPACTION_HISTORY_CF), UUIDGen.getTimeUUID(), ksname, cfname, ByteBufferUtil.bytes(compactedAt), bytesIn, bytesOut, rowsMerged);
+        executeInternal(String.format(req, COMPACTION_HISTORY), UUIDGen.getTimeUUID(), ksname, cfname, ByteBufferUtil.bytes(compactedAt), bytesIn, bytesOut, rowsMerged);
     }
 
     public static TabularData getCompactionHistory() throws OpenDataException
     {
-        UntypedResultSet queryResultSet = executeInternal(String.format("SELECT * from system.%s", COMPACTION_HISTORY_CF));
+        UntypedResultSet queryResultSet = executeInternal(String.format("SELECT * from system.%s", COMPACTION_HISTORY));
         return CompactionHistoryTabularData.from(queryResultSet);
     }
 
     public static synchronized void saveTruncationRecord(ColumnFamilyStore cfs, long truncatedAt, ReplayPosition position)
     {
         String req = "UPDATE system.%s SET truncated_at = truncated_at + ? WHERE key = '%s'";
-        executeInternal(String.format(req, LOCAL_CF, LOCAL_KEY), truncationAsMapEntry(cfs, truncatedAt, position));
+        executeInternal(String.format(req, LOCAL, LOCAL), truncationAsMapEntry(cfs, truncatedAt, position));
         truncationRecords = null;
-        forceBlockingFlush(LOCAL_CF);
+        forceBlockingFlush(LOCAL);
     }
 
     /**
@@ -312,24 +438,23 @@
     public static synchronized void removeTruncationRecord(UUID cfId)
     {
         String req = "DELETE truncated_at[?] from system.%s WHERE key = '%s'";
-        executeInternal(String.format(req, LOCAL_CF, LOCAL_KEY), cfId);
+        executeInternal(String.format(req, LOCAL, LOCAL), cfId);
         truncationRecords = null;
-        forceBlockingFlush(LOCAL_CF);
+        forceBlockingFlush(LOCAL);
     }
 
     private static Map<UUID, ByteBuffer> truncationAsMapEntry(ColumnFamilyStore cfs, long truncatedAt, ReplayPosition position)
     {
-        DataOutputBuffer out = new DataOutputBuffer();
-        try
+        try (DataOutputBuffer out = new DataOutputBuffer())
         {
             ReplayPosition.serializer.serialize(position, out);
             out.writeLong(truncatedAt);
+            return Collections.singletonMap(cfs.metadata.cfId, ByteBuffer.wrap(out.getData(), 0, out.getLength()));
         }
         catch (IOException e)
         {
             throw new RuntimeException(e);
         }
-        return Collections.<UUID, ByteBuffer>singletonMap(cfs.metadata.cfId, ByteBuffer.wrap(out.getData(), 0, out.getLength()));
     }
 
     public static ReplayPosition getTruncatedPosition(UUID cfId)
@@ -353,7 +478,7 @@
 
     private static Map<UUID, Pair<ReplayPosition, Long>> readTruncationRecords()
     {
-        UntypedResultSet rows = executeInternal(String.format("SELECT truncated_at FROM system.%s WHERE key = '%s'", LOCAL_CF, LOCAL_KEY));
+        UntypedResultSet rows = executeInternal(String.format("SELECT truncated_at FROM system.%s WHERE key = '%s'", LOCAL, LOCAL));
 
         Map<UUID, Pair<ReplayPosition, Long>> records = new HashMap<>();
 
@@ -392,14 +517,14 @@
         }
 
         String req = "INSERT INTO system.%s (peer, tokens) VALUES (?, ?)";
-        executeInternal(String.format(req, PEERS_CF), ep, tokensAsSet(tokens));
+        executeInternal(String.format(req, PEERS), ep, tokensAsSet(tokens));
     }
 
     public static synchronized void updatePreferredIP(InetAddress ep, InetAddress preferred_ip)
     {
         String req = "INSERT INTO system.%s (peer, preferred_ip) VALUES (?, ?)";
-        executeInternal(String.format(req, PEERS_CF), ep, preferred_ip);
-        forceBlockingFlush(PEERS_CF);
+        executeInternal(String.format(req, PEERS), ep, preferred_ip);
+        forceBlockingFlush(PEERS);
     }
 
     public static synchronized void updatePeerInfo(InetAddress ep, String columnName, Object value)
@@ -408,20 +533,20 @@
             return;
 
         String req = "INSERT INTO system.%s (peer, %s) VALUES (?, ?)";
-        executeInternal(String.format(req, PEERS_CF, columnName), ep, value);
+        executeInternal(String.format(req, PEERS, columnName), ep, value);
     }
 
     public static synchronized void updateHintsDropped(InetAddress ep, UUID timePeriod, int value)
     {
         // with 30 day TTL
         String req = "UPDATE system.%s USING TTL 2592000 SET hints_dropped[ ? ] = ? WHERE peer = ?";
-        executeInternal(String.format(req, PEER_EVENTS_CF), timePeriod, value, ep);
+        executeInternal(String.format(req, PEER_EVENTS), timePeriod, value, ep);
     }
 
     public static synchronized void updateSchemaVersion(UUID version)
     {
         String req = "INSERT INTO system.%s (key, schema_version) VALUES ('%s', ?)";
-        executeInternal(String.format(req, LOCAL_CF, LOCAL_KEY), version);
+        executeInternal(String.format(req, LOCAL, LOCAL), version);
     }
 
     private static Set<String> tokensAsSet(Collection<Token> tokens)
@@ -436,7 +561,7 @@
     private static Collection<Token> deserializeTokens(Collection<String> tokensStrings)
     {
         Token.TokenFactory factory = StorageService.getPartitioner().getTokenFactory();
-        List<Token> tokens = new ArrayList<Token>(tokensStrings.size());
+        List<Token> tokens = new ArrayList<>(tokensStrings.size());
         for (String tk : tokensStrings)
             tokens.add(factory.fromString(tk));
         return tokens;
@@ -448,7 +573,7 @@
     public static synchronized void removeEndpoint(InetAddress ep)
     {
         String req = "DELETE FROM system.%s WHERE peer = ?";
-        executeInternal(String.format(req, PEERS_CF), ep);
+        executeInternal(String.format(req, PEERS), ep);
     }
 
     /**
@@ -458,14 +583,14 @@
     {
         assert !tokens.isEmpty() : "removeEndpoint should be used instead";
         String req = "INSERT INTO system.%s (key, tokens) VALUES ('%s', ?)";
-        executeInternal(String.format(req, LOCAL_CF, LOCAL_KEY), tokensAsSet(tokens));
-        forceBlockingFlush(LOCAL_CF);
+        executeInternal(String.format(req, LOCAL, LOCAL), tokensAsSet(tokens));
+        forceBlockingFlush(LOCAL);
     }
 
     public static void forceBlockingFlush(String cfname)
     {
         if (!Boolean.getBoolean("cassandra.unsafesystem"))
-            FBUtilities.waitOnFuture(Keyspace.open(Keyspace.SYSTEM_KS).getColumnFamilyStore(cfname).forceFlush());
+            FBUtilities.waitOnFuture(Keyspace.open(NAME).getColumnFamilyStore(cfname).forceFlush());
     }
 
     /**
@@ -475,7 +600,7 @@
     public static SetMultimap<InetAddress, Token> loadTokens()
     {
         SetMultimap<InetAddress, Token> tokenMap = HashMultimap.create();
-        for (UntypedResultSet.Row row : executeInternal("SELECT peer, tokens FROM system." + PEERS_CF))
+        for (UntypedResultSet.Row row : executeInternal("SELECT peer, tokens FROM system." + PEERS))
         {
             InetAddress peer = row.getInetAddress("peer");
             if (row.has("tokens"))
@@ -491,8 +616,8 @@
      */
     public static Map<InetAddress, UUID> loadHostIds()
     {
-        Map<InetAddress, UUID> hostIdMap = new HashMap<InetAddress, UUID>();
-        for (UntypedResultSet.Row row : executeInternal("SELECT peer, host_id FROM system." + PEERS_CF))
+        Map<InetAddress, UUID> hostIdMap = new HashMap<>();
+        for (UntypedResultSet.Row row : executeInternal("SELECT peer, host_id FROM system." + PEERS))
         {
             InetAddress peer = row.getInetAddress("peer");
             if (row.has("host_id"))
@@ -512,7 +637,7 @@
     public static InetAddress getPreferredIP(InetAddress ep)
     {
         String req = "SELECT preferred_ip FROM system.%s WHERE peer=?";
-        UntypedResultSet result = executeInternal(String.format(req, PEERS_CF), ep);
+        UntypedResultSet result = executeInternal(String.format(req, PEERS), ep);
         if (!result.isEmpty() && result.one().has("preferred_ip"))
             return result.one().getInetAddress("preferred_ip");
         return ep;
@@ -523,13 +648,13 @@
      */
     public static Map<InetAddress, Map<String,String>> loadDcRackInfo()
     {
-        Map<InetAddress, Map<String, String>> result = new HashMap<InetAddress, Map<String, String>>();
-        for (UntypedResultSet.Row row : executeInternal("SELECT peer, data_center, rack from system." + PEERS_CF))
+        Map<InetAddress, Map<String, String>> result = new HashMap<>();
+        for (UntypedResultSet.Row row : executeInternal("SELECT peer, data_center, rack from system." + PEERS))
         {
             InetAddress peer = row.getInetAddress("peer");
             if (row.has("data_center") && row.has("rack"))
             {
-                Map<String, String> dcRack = new HashMap<String, String>();
+                Map<String, String> dcRack = new HashMap<>();
                 dcRack.put("data_center", row.getString("data_center"));
                 dcRack.put("rack", row.getString("rack"));
                 result.put(peer, dcRack);
@@ -545,19 +670,19 @@
      * @param ep endpoint address to check
      * @return Release version or null if version is unknown.
      */
-    public static SemanticVersion getReleaseVersion(InetAddress ep)
+    public static CassandraVersion getReleaseVersion(InetAddress ep)
     {
         try
         {
             if (FBUtilities.getBroadcastAddress().equals(ep))
             {
-                return new SemanticVersion(FBUtilities.getReleaseVersionString());
+                return new CassandraVersion(FBUtilities.getReleaseVersionString());
             }
             String req = "SELECT release_version FROM system.%s WHERE peer=?";
-            UntypedResultSet result = executeInternal(String.format(req, PEERS_CF), ep);
+            UntypedResultSet result = executeInternal(String.format(req, PEERS), ep);
             if (result != null && result.one().has("release_version"))
             {
-                return new SemanticVersion(result.one().getString("release_version"));
+                return new CassandraVersion(result.one().getString("release_version"));
             }
             // version is unknown
             return null;
@@ -581,7 +706,7 @@
         Keyspace keyspace;
         try
         {
-            keyspace = Keyspace.open(Keyspace.SYSTEM_KS);
+            keyspace = Keyspace.open(NAME);
         }
         catch (AssertionError err)
         {
@@ -590,77 +715,39 @@
             ex.initCause(err);
             throw ex;
         }
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(LOCAL_CF);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(LOCAL);
 
+        String req = "SELECT cluster_name FROM system.%s WHERE key='%s'";
+        UntypedResultSet result = executeInternal(String.format(req, LOCAL, LOCAL));
+
+        if (result.isEmpty() || !result.one().has("cluster_name"))
         {
-            String req = "SELECT cluster_name FROM system.%s WHERE key='%s'";
-            UntypedResultSet result = executeInternal(String.format(req, LOCAL_CF, LOCAL_KEY));
+            // this is a brand new node
+            if (!cfs.getSSTables().isEmpty())
+                throw new ConfigurationException("Found system keyspace files, but they couldn't be loaded!");
 
-            if (result.isEmpty() || !result.one().has("cluster_name"))
-            {
-                // this is a brand new node
-                if (!cfs.getSSTables().isEmpty())
-                    throw new ConfigurationException("Found system keyspace files, but they couldn't be loaded!");
-
-                // no system files.  this is a new node.
-                req = "INSERT INTO system.%s (key, cluster_name) VALUES ('%s', ?)";
-                executeInternal(String.format(req, LOCAL_CF, LOCAL_KEY), DatabaseDescriptor.getClusterName());
-            }
-            else
-            {
-                String savedClusterName = result.one().getString("cluster_name");
-                if (!DatabaseDescriptor.getClusterName().equals(savedClusterName))
-                    throw new ConfigurationException("Saved cluster name " + savedClusterName + " != configured name " + DatabaseDescriptor.getClusterName());
-            }
+            // no system files.  this is a new node.
+            return;
         }
 
-        String req = "SELECT rack, data_center FROM system.%s WHERE key='%s'";
-        UntypedResultSet result = executeInternal(String.format(req, LOCAL_CF, LOCAL_KEY));
-
-        if (!Boolean.getBoolean("cassandra.ignore_dc"))
-        {
-            // Look up the dc (return it if found)
-            if (!result.isEmpty() && result.one().has("data_center"))
-            {
-                String storedDc = result.one().getString("data_center");
-                String currentDc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddress());
-                if (!storedDc.equals(currentDc))
-                {
-                    throw new ConfigurationException("Cannot start node if snitch's data center (" + currentDc + ") differs from previous data center (" + storedDc + "). " +
-                                                     "Please fix the snitch configuration, decommission and rebootstrap this node or use the flag -Dcassandra.ignore_dc=true.");
-                }
-            }
-        }
-
-        if (!Boolean.getBoolean("cassandra.ignore_rack"))
-        {
-            // Look up the Rack (return it if found)
-            if (!result.isEmpty() && result.one().has("rack"))
-            {
-                String storedRack = result.one().getString("rack");
-                String currentRack = DatabaseDescriptor.getEndpointSnitch().getRack(FBUtilities.getBroadcastAddress());
-                if (!storedRack.equals(currentRack))
-                {
-                    throw new ConfigurationException("Cannot start node if snitch's rack (" + currentRack + ") differs from previous rack (" + storedRack + "). " +
-                                                     "Please fix the snitch or decommission and rebootstrap this node.");
-                }
-            }
-        }
+        String savedClusterName = result.one().getString("cluster_name");
+        if (!DatabaseDescriptor.getClusterName().equals(savedClusterName))
+            throw new ConfigurationException("Saved cluster name " + savedClusterName + " != configured name " + DatabaseDescriptor.getClusterName());
     }
 
     public static Collection<Token> getSavedTokens()
     {
         String req = "SELECT tokens FROM system.%s WHERE key='%s'";
-        UntypedResultSet result = executeInternal(String.format(req, LOCAL_CF, LOCAL_KEY));
+        UntypedResultSet result = executeInternal(String.format(req, LOCAL, LOCAL));
         return result.isEmpty() || !result.one().has("tokens")
              ? Collections.<Token>emptyList()
-             : deserializeTokens(result.one().<String>getSet("tokens", UTF8Type.instance));
+             : deserializeTokens(result.one().getSet("tokens", UTF8Type.instance));
     }
 
     public static int incrementAndGetGeneration()
     {
         String req = "SELECT gossip_generation FROM system.%s WHERE key='%s'";
-        UntypedResultSet result = executeInternal(String.format(req, LOCAL_CF, LOCAL_KEY));
+        UntypedResultSet result = executeInternal(String.format(req, LOCAL, LOCAL));
 
         int generation;
         if (result.isEmpty() || !result.one().has("gossip_generation"))
@@ -688,8 +775,8 @@
         }
 
         req = "INSERT INTO system.%s (key, gossip_generation) VALUES ('%s', ?)";
-        executeInternal(String.format(req, LOCAL_CF, LOCAL_KEY), generation);
-        forceBlockingFlush(LOCAL_CF);
+        executeInternal(String.format(req, LOCAL, LOCAL), generation);
+        forceBlockingFlush(LOCAL);
 
         return generation;
     }
@@ -697,7 +784,7 @@
     public static BootstrapState getBootstrapState()
     {
         String req = "SELECT bootstrapped FROM system.%s WHERE key='%s'";
-        UntypedResultSet result = executeInternal(String.format(req, LOCAL_CF, LOCAL_KEY));
+        UntypedResultSet result = executeInternal(String.format(req, LOCAL, LOCAL));
 
         if (result.isEmpty() || !result.one().has("bootstrapped"))
             return BootstrapState.NEEDS_BOOTSTRAP;
@@ -718,15 +805,15 @@
     public static void setBootstrapState(BootstrapState state)
     {
         String req = "INSERT INTO system.%s (key, bootstrapped) VALUES ('%s', ?)";
-        executeInternal(String.format(req, LOCAL_CF, LOCAL_KEY), state.name());
-        forceBlockingFlush(LOCAL_CF);
+        executeInternal(String.format(req, LOCAL, LOCAL), state.name());
+        forceBlockingFlush(LOCAL);
     }
 
     public static boolean isIndexBuilt(String keyspaceName, String indexName)
     {
-        ColumnFamilyStore cfs = Keyspace.open(Keyspace.SYSTEM_KS).getColumnFamilyStore(INDEX_CF);
+        ColumnFamilyStore cfs = Keyspace.open(NAME).getColumnFamilyStore(BUILT_INDEXES);
         QueryFilter filter = QueryFilter.getNamesFilter(decorate(ByteBufferUtil.bytes(keyspaceName)),
-                                                        INDEX_CF,
+                                                        BUILT_INDEXES,
                                                         FBUtilities.singleton(cfs.getComparator().makeCellName(indexName), cfs.getComparator()),
                                                         System.currentTimeMillis());
         return ColumnFamilyStore.removeDeleted(cfs.getColumnFamily(filter), Integer.MAX_VALUE) != null;
@@ -734,17 +821,18 @@
 
     public static void setIndexBuilt(String keyspaceName, String indexName)
     {
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(Keyspace.SYSTEM_KS, INDEX_CF);
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(NAME, BUILT_INDEXES);
         cf.addColumn(new BufferCell(cf.getComparator().makeCellName(indexName), ByteBufferUtil.EMPTY_BYTE_BUFFER, FBUtilities.timestampMicros()));
-        new Mutation(Keyspace.SYSTEM_KS, ByteBufferUtil.bytes(keyspaceName), cf).apply();
-        forceBlockingFlush(INDEX_CF);
+        new Mutation(NAME, ByteBufferUtil.bytes(keyspaceName), cf).apply();
+        forceBlockingFlush(BUILT_INDEXES);
     }
 
     public static void setIndexRemoved(String keyspaceName, String indexName)
     {
-        Mutation mutation = new Mutation(Keyspace.SYSTEM_KS, ByteBufferUtil.bytes(keyspaceName));
-        mutation.delete(INDEX_CF, CFMetaData.IndexCf.comparator.makeCellName(indexName), FBUtilities.timestampMicros());
+        Mutation mutation = new Mutation(NAME, ByteBufferUtil.bytes(keyspaceName));
+        mutation.delete(BUILT_INDEXES, BuiltIndexes.comparator.makeCellName(indexName), FBUtilities.timestampMicros());
         mutation.apply();
+        forceBlockingFlush(BUILT_INDEXES);
     }
 
     /**
@@ -754,7 +842,7 @@
     public static UUID getLocalHostId()
     {
         String req = "SELECT host_id FROM system.%s WHERE key='%s'";
-        UntypedResultSet result = executeInternal(String.format(req, LOCAL_CF, LOCAL_KEY));
+        UntypedResultSet result = executeInternal(String.format(req, LOCAL, LOCAL));
 
         // Look up the Host UUID (return it if found)
         if (!result.isEmpty() && result.one().has("host_id"))
@@ -772,134 +860,44 @@
     public static UUID setLocalHostId(UUID hostId)
     {
         String req = "INSERT INTO system.%s (key, host_id) VALUES ('%s', ?)";
-        executeInternal(String.format(req, LOCAL_CF, LOCAL_KEY), hostId);
+        executeInternal(String.format(req, LOCAL, LOCAL), hostId);
         return hostId;
     }
 
     /**
-     * @param cfName The name of the ColumnFamily responsible for part of the schema (keyspace, ColumnFamily, columns)
-     * @return CFS responsible to hold low-level serialized schema
+     * Gets the stored rack for the local node, or null if none have been set yet.
      */
-    public static ColumnFamilyStore schemaCFS(String cfName)
+    public static String getRack()
     {
-        return Keyspace.open(Keyspace.SYSTEM_KS).getColumnFamilyStore(cfName);
-    }
+        String req = "SELECT rack FROM system.%s WHERE key='%s'";
+        UntypedResultSet result = executeInternal(String.format(req, LOCAL, LOCAL));
 
-    public static List<Row> serializedSchema()
-    {
-        List<Row> schema = new ArrayList<>();
+        // Look up the Rack (return it if found)
+        if (!result.isEmpty() && result.one().has("rack"))
+            return result.one().getString("rack");
 
-        for (String cf : allSchemaCfs)
-            schema.addAll(serializedSchema(cf));
-
-        return schema;
+        return null;
     }
 
     /**
-     * @param schemaCfName The name of the ColumnFamily responsible for part of the schema (keyspace, ColumnFamily, columns)
-     * @return low-level schema representation (each row represents individual Keyspace or ColumnFamily)
+     * Gets the stored data center for the local node, or null if none have been set yet.
      */
-    public static List<Row> serializedSchema(String schemaCfName)
+    public static String getDatacenter()
     {
-        Token minToken = StorageService.getPartitioner().getMinimumToken();
+        String req = "SELECT data_center FROM system.%s WHERE key='%s'";
+        UntypedResultSet result = executeInternal(String.format(req, LOCAL, LOCAL));
 
-        return schemaCFS(schemaCfName).getRangeSlice(new Range<RowPosition>(minToken.minKeyBound(), minToken.maxKeyBound()),
-                                                     null,
-                                                     new IdentityQueryFilter(),
-                                                     Integer.MAX_VALUE,
-                                                     System.currentTimeMillis());
-    }
+        // Look up the Data center (return it if found)
+        if (!result.isEmpty() && result.one().has("data_center"))
+            return result.one().getString("data_center");
 
-    public static Collection<Mutation> serializeSchema()
-    {
-        Map<DecoratedKey, Mutation> mutationMap = new HashMap<>();
-
-        for (String cf : allSchemaCfs)
-            serializeSchema(mutationMap, cf);
-
-        return mutationMap.values();
-    }
-
-    private static void serializeSchema(Map<DecoratedKey, Mutation> mutationMap, String schemaCfName)
-    {
-        for (Row schemaRow : serializedSchema(schemaCfName))
-        {
-            if (Schema.ignoredSchemaRow(schemaRow))
-                continue;
-
-            Mutation mutation = mutationMap.get(schemaRow.key);
-            if (mutation == null)
-            {
-                mutation = new Mutation(Keyspace.SYSTEM_KS, schemaRow.key.getKey());
-                mutationMap.put(schemaRow.key, mutation);
-            }
-
-            mutation.add(schemaRow.cf);
-        }
-    }
-
-    public static Map<DecoratedKey, ColumnFamily> getSchema(String schemaCfName, Set<String> keyspaces)
-    {
-        Map<DecoratedKey, ColumnFamily> schema = new HashMap<>();
-
-        for (String keyspace : keyspaces)
-        {
-            Row schemaEntity = readSchemaRow(schemaCfName, keyspace);
-            if (schemaEntity.cf != null)
-                schema.put(schemaEntity.key, schemaEntity.cf);
-        }
-
-        return schema;
-    }
-
-    public static ByteBuffer getSchemaKSKey(String ksName)
-    {
-        return AsciiType.instance.fromString(ksName);
-    }
-
-    /**
-     * Fetches a subset of schema (table data, columns metadata or triggers) for the keyspace.
-     *
-     * @param schemaCfName the schema table to get the data from (schema_keyspaces, schema_columnfamilies, schema_columns or schema_triggers)
-     * @param ksName the keyspace of the tables we are interested in
-     * @return a Row containing the schema data of a particular type for the keyspace
-     */
-    public static Row readSchemaRow(String schemaCfName, String ksName)
-    {
-        DecoratedKey key = StorageService.getPartitioner().decorateKey(getSchemaKSKey(ksName));
-
-        ColumnFamilyStore schemaCFS = SystemKeyspace.schemaCFS(schemaCfName);
-        ColumnFamily result = schemaCFS.getColumnFamily(QueryFilter.getIdentityFilter(key, schemaCfName, System.currentTimeMillis()));
-
-        return new Row(key, result);
-    }
-
-    /**
-     * Fetches a subset of schema (table data, columns metadata or triggers) for the keyspace+table pair.
-     *
-     * @param schemaCfName the schema table to get the data from (schema_columnfamilies, schema_columns or schema_triggers)
-     * @param ksName the keyspace of the table we are interested in
-     * @param cfName the table we are interested in
-     * @return a Row containing the schema data of a particular type for the table
-     */
-    public static Row readSchemaRow(String schemaCfName, String ksName, String cfName)
-    {
-        DecoratedKey key = StorageService.getPartitioner().decorateKey(getSchemaKSKey(ksName));
-        ColumnFamilyStore schemaCFS = SystemKeyspace.schemaCFS(schemaCfName);
-        Composite prefix = schemaCFS.getComparator().make(cfName);
-        ColumnFamily cf = schemaCFS.getColumnFamily(key,
-                                                    prefix,
-                                                    prefix.end(),
-                                                    false,
-                                                    Integer.MAX_VALUE,
-                                                    System.currentTimeMillis());
-        return new Row(key, cf);
+        return null;
     }
 
     public static PaxosState loadPaxosState(ByteBuffer key, CFMetaData metadata, long now)
     {
         String req = "SELECT * FROM system.%s WHERE row_key = ? AND cf_id = ?";
-        UntypedResultSet results = QueryProcessor.executeInternalWithNow(now, String.format(req, PAXOS_CF), key, metadata.cfId);
+        UntypedResultSet results = QueryProcessor.executeInternalWithNow(now, String.format(req, PAXOS), key, metadata.cfId);
         if (results.isEmpty())
             return new PaxosState(key, metadata);
         UntypedResultSet.Row row = results.one();
@@ -920,7 +918,7 @@
     public static void savePaxosPromise(Commit promise)
     {
         String req = "UPDATE system.%s USING TIMESTAMP ? AND TTL ? SET in_progress_ballot = ? WHERE row_key = ? AND cf_id = ?";
-        executeInternal(String.format(req, PAXOS_CF),
+        executeInternal(String.format(req, PAXOS),
                         UUIDGen.microsTimestamp(promise.ballot),
                         paxosTtl(promise.update.metadata),
                         promise.ballot,
@@ -930,7 +928,7 @@
 
     public static void savePaxosProposal(Commit proposal)
     {
-        executeInternal(String.format("UPDATE system.%s USING TIMESTAMP ? AND TTL ? SET proposal_ballot = ?, proposal = ? WHERE row_key = ? AND cf_id = ?", PAXOS_CF),
+        executeInternal(String.format("UPDATE system.%s USING TIMESTAMP ? AND TTL ? SET proposal_ballot = ?, proposal = ? WHERE row_key = ? AND cf_id = ?", PAXOS),
                         UUIDGen.microsTimestamp(proposal.ballot),
                         paxosTtl(proposal.update.metadata),
                         proposal.ballot,
@@ -950,7 +948,7 @@
         // We always erase the last proposal (with the commit timestamp to no erase more recent proposal in case the commit is old)
         // even though that's really just an optimization  since SP.beginAndRepairPaxos will exclude accepted proposal older than the mrc.
         String cql = "UPDATE system.%s USING TIMESTAMP ? AND TTL ? SET proposal_ballot = null, proposal = null, most_recent_commit_at = ?, most_recent_commit = ? WHERE row_key = ? AND cf_id = ?";
-        executeInternal(String.format(cql, PAXOS_CF),
+        executeInternal(String.format(cql, PAXOS),
                         UUIDGen.microsTimestamp(commit.ballot),
                         paxosTtl(commit.update.metadata),
                         commit.ballot,
@@ -969,7 +967,7 @@
     public static RestorableMeter getSSTableReadMeter(String keyspace, String table, int generation)
     {
         String cql = "SELECT * FROM system.%s WHERE keyspace_name=? and columnfamily_name=? and generation=?";
-        UntypedResultSet results = executeInternal(String.format(cql, SSTABLE_ACTIVITY_CF), keyspace, table, generation);
+        UntypedResultSet results = executeInternal(String.format(cql, SSTABLE_ACTIVITY), keyspace, table, generation);
 
         if (results.isEmpty())
             return new RestorableMeter();
@@ -987,7 +985,7 @@
     {
         // Store values with a one-day TTL to handle corner cases where cleanup might not occur
         String cql = "INSERT INTO system.%s (keyspace_name, columnfamily_name, generation, rate_15m, rate_120m) VALUES (?, ?, ?, ?, ?) USING TTL 864000";
-        executeInternal(String.format(cql, SSTABLE_ACTIVITY_CF),
+        executeInternal(String.format(cql, SSTABLE_ACTIVITY),
                         keyspace,
                         table,
                         generation,
@@ -1001,7 +999,7 @@
     public static void clearSSTableReadMeter(String keyspace, String table, int generation)
     {
         String cql = "DELETE FROM system.%s WHERE keyspace_name=? AND columnfamily_name=? and generation=?";
-        executeInternal(String.format(cql, SSTABLE_ACTIVITY_CF), keyspace, table, generation);
+        executeInternal(String.format(cql, SSTABLE_ACTIVITY), keyspace, table, generation);
     }
 
     /**
@@ -1010,22 +1008,21 @@
     public static void updateSizeEstimates(String keyspace, String table, Map<Range<Token>, Pair<Long, Long>> estimates)
     {
         long timestamp = FBUtilities.timestampMicros();
-        CFMetaData estimatesTable = CFMetaData.SizeEstimatesCf;
-        Mutation mutation = new Mutation(Keyspace.SYSTEM_KS, UTF8Type.instance.decompose(keyspace));
+        Mutation mutation = new Mutation(NAME, UTF8Type.instance.decompose(keyspace));
 
         // delete all previous values with a single range tombstone.
-        mutation.deleteRange(SIZE_ESTIMATES_CF,
-                             estimatesTable.comparator.make(table).start(),
-                             estimatesTable.comparator.make(table).end(),
+        mutation.deleteRange(SIZE_ESTIMATES,
+                             SizeEstimates.comparator.make(table).start(),
+                             SizeEstimates.comparator.make(table).end(),
                              timestamp - 1);
 
         // add a CQL row for each primary token range.
-        ColumnFamily cells = mutation.addOrGet(estimatesTable);
+        ColumnFamily cells = mutation.addOrGet(SizeEstimates);
         for (Map.Entry<Range<Token>, Pair<Long, Long>> entry : estimates.entrySet())
         {
             Range<Token> range = entry.getKey();
             Pair<Long, Long> values = entry.getValue();
-            Composite prefix = estimatesTable.comparator.make(table, range.left.toString(), range.right.toString());
+            Composite prefix = SizeEstimates.comparator.make(table, range.left.toString(), range.right.toString());
             CFRowAdder adder = new CFRowAdder(cells, prefix, timestamp);
             adder.add("partitions_count", values.left)
                  .add("mean_partition_size", values.right);
@@ -1039,9 +1036,131 @@
      */
     public static void clearSizeEstimates(String keyspace, String table)
     {
-        String cql = String.format("DELETE FROM %s.%s WHERE keyspace_name = ? AND table_name = ?",
-                                   Keyspace.SYSTEM_KS,
-                                   SIZE_ESTIMATES_CF);
+        String cql = String.format("DELETE FROM %s.%s WHERE keyspace_name = ? AND table_name = ?", NAME, SIZE_ESTIMATES);
         executeInternal(cql, keyspace, table);
     }
+
+    public static synchronized void updateAvailableRanges(String keyspace, Collection<Range<Token>> completedRanges)
+    {
+        String cql = "UPDATE system.%s SET ranges = ranges + ? WHERE keyspace_name = ?";
+        Set<ByteBuffer> rangesToUpdate = new HashSet<>(completedRanges.size());
+        for (Range<Token> range : completedRanges)
+        {
+            rangesToUpdate.add(rangeToBytes(range));
+        }
+        executeInternal(String.format(cql, AVAILABLE_RANGES), rangesToUpdate, keyspace);
+    }
+
+    public static synchronized Set<Range<Token>> getAvailableRanges(String keyspace, IPartitioner partitioner)
+    {
+        Set<Range<Token>> result = new HashSet<>();
+        String query = "SELECT * FROM system.%s WHERE keyspace_name=?";
+        UntypedResultSet rs = executeInternal(String.format(query, AVAILABLE_RANGES), keyspace);
+        for (UntypedResultSet.Row row : rs)
+        {
+            Set<ByteBuffer> rawRanges = row.getSet("ranges", BytesType.instance);
+            for (ByteBuffer rawRange : rawRanges)
+            {
+                result.add(byteBufferToRange(rawRange, partitioner));
+            }
+        }
+        return ImmutableSet.copyOf(result);
+    }
+
+    public static void resetAvailableRanges()
+    {
+        ColumnFamilyStore availableRanges = Keyspace.open(NAME).getColumnFamilyStore(AVAILABLE_RANGES);
+        availableRanges.truncateBlocking();
+    }
+
+    /**
+     * Compare the release version in the system.local table with the one included in the distro.
+     * If they don't match, snapshot all tables in the system keyspace. This is intended to be
+     * called at startup to create a backup of the system tables during an upgrade
+     *
+     * @throws IOException
+     */
+    public static void snapshotOnVersionChange() throws IOException
+    {
+        String previous = getPreviousVersionString();
+        String next = FBUtilities.getReleaseVersionString();
+
+        // if we're restarting after an upgrade, snapshot the system keyspace
+        if (!previous.equals(NULL_VERSION.toString()) && !previous.equals(next))
+
+        {
+            logger.info("Detected version upgrade from {} to {}, snapshotting system keyspace", previous, next);
+            String snapshotName = Keyspace.getTimestampedSnapshotName(String.format("upgrade-%s-%s",
+                                                                                    previous,
+                                                                                    next));
+            Keyspace systemKs = Keyspace.open(SystemKeyspace.NAME);
+            systemKs.snapshot(snapshotName, null);
+        }
+    }
+
+    /**
+     * Try to determine what the previous version, if any, was installed on this node.
+     * Primary source of truth is the release version in system.local. If the previous
+     * version cannot be determined by looking there then either:
+     * * the node never had a C* install before
+     * * the was a very old version (pre 1.2) installed, which did not include system.local
+     *
+     * @return either a version read from the system.local table or one of two special values
+     * indicating either no previous version (SystemUpgrade.NULL_VERSION) or an unreadable,
+     * legacy version (SystemUpgrade.UNREADABLE_VERSION).
+     */
+    private static String getPreviousVersionString()
+    {
+        String req = "SELECT release_version FROM system.%s WHERE key='%s'";
+        UntypedResultSet result = executeInternal(String.format(req, SystemKeyspace.LOCAL, SystemKeyspace.LOCAL));
+        if (result.isEmpty() || !result.one().has("release_version"))
+        {
+            // it isn't inconceivable that one might try to upgrade a node straight from <= 1.1 to whatever
+            // the current version is. If we couldn't read a previous version from system.local we check for
+            // the existence of the legacy system.Versions table. We don't actually attempt to read a version
+            // from there, but it informs us that this isn't a completely new node.
+            for (File dataDirectory : Directories.getKSChildDirectories(SystemKeyspace.NAME))
+            {
+                if (dataDirectory.getName().equals("Versions") && dataDirectory.listFiles().length > 0)
+                {
+                    logger.trace("Found unreadable versions info in pre 1.2 system.Versions table");
+                    return UNREADABLE_VERSION.toString();
+                }
+            }
+
+            // no previous version information found, we can assume that this is a new node
+            return NULL_VERSION.toString();
+        }
+        // report back whatever we found in the system table
+        return result.one().getString("release_version");
+    }
+
+    private static ByteBuffer rangeToBytes(Range<Token> range)
+    {
+        try (DataOutputBuffer out = new DataOutputBuffer())
+        {
+            Range.tokenSerializer.serialize(range, out, MessagingService.VERSION_22);
+            return out.buffer();
+        }
+        catch (IOException e)
+        {
+            throw new IOError(e);
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    private static Range<Token> byteBufferToRange(ByteBuffer rawRange, IPartitioner partitioner)
+    {
+        try
+        {
+            return (Range<Token>) Range.tokenSerializer.deserialize(ByteStreams.newDataInput(ByteBufferUtil.getArray(rawRange)),
+                                                                    partitioner,
+                                                                    MessagingService.VERSION_22);
+        }
+        catch (IOException e)
+        {
+            throw new IOError(e);
+        }
+    }
+
 }
diff --git a/src/java/org/apache/cassandra/db/WindowsFailedSnapshotTracker.java b/src/java/org/apache/cassandra/db/WindowsFailedSnapshotTracker.java
new file mode 100644
index 0000000..7cc7893
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/WindowsFailedSnapshotTracker.java
@@ -0,0 +1,116 @@
+/*
+ * 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.db;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.io.util.FileUtils;
+
+
+public class WindowsFailedSnapshotTracker
+{
+    private static final Logger logger = LoggerFactory.getLogger(WindowsFailedSnapshotTracker.class);
+    private static PrintWriter _failedSnapshotFile;
+
+    @VisibleForTesting
+    // Need to handle null for unit tests
+    public static final String TODELETEFILE = System.getenv("CASSANDRA_HOME") == null
+                 ? ".toDelete"
+                 : System.getenv("CASSANDRA_HOME") + File.separator + ".toDelete";
+
+    public static void deleteOldSnapshots()
+    {
+        if (new File(TODELETEFILE).exists())
+        {
+            try
+            {
+                try (BufferedReader reader = new BufferedReader(new FileReader(TODELETEFILE)))
+                {
+                    String snapshotDirectory;
+                    while ((snapshotDirectory = reader.readLine()) != null)
+                    {
+                        File f = new File(snapshotDirectory);
+
+                        // Skip folders that aren't a subset of temp or a data folder. We don't want people to accidentally
+                        // delete something important by virtue of adding something invalid to the .toDelete file.
+                        boolean validFolder = FileUtils.isSubDirectory(new File(System.getenv("TEMP")), f);
+                        for (String s : DatabaseDescriptor.getAllDataFileLocations())
+                            validFolder |= FileUtils.isSubDirectory(new File(s), f);
+
+                        if (!validFolder)
+                        {
+                            logger.warn("Skipping invalid directory found in .toDelete: {}. Only %TEMP% or data file subdirectories are valid.", f);
+                            continue;
+                        }
+
+                        // Could be a non-existent directory if deletion worked on previous JVM shutdown.
+                        if (f.exists())
+                        {
+                            logger.warn("Discovered obsolete snapshot. Deleting directory [{}]", snapshotDirectory);
+                            FileUtils.deleteRecursive(new File(snapshotDirectory));
+                        }
+                    }
+                }
+
+                // Only delete the old .toDelete file if we succeed in deleting all our known bad snapshots.
+                Files.delete(Paths.get(TODELETEFILE));
+            }
+            catch (IOException e)
+            {
+                logger.warn("Failed to open {}. Obsolete snapshots from previous runs will not be deleted.", TODELETEFILE);
+                logger.warn("Exception: " + e);
+            }
+        }
+
+        try
+        {
+            _failedSnapshotFile = new PrintWriter(new FileWriter(TODELETEFILE, true));
+        }
+        catch (IOException e)
+        {
+            throw new RuntimeException(String.format("Failed to create failed snapshot tracking file [%s]. Aborting", TODELETEFILE));
+        }
+    }
+
+    public static synchronized void handleFailedSnapshot(File dir)
+    {
+        assert _failedSnapshotFile != null : "_failedSnapshotFile not initialized within WindowsFailedSnapshotTracker";
+        FileUtils.deleteRecursiveOnExit(dir);
+        _failedSnapshotFile.println(dir.toString());
+        _failedSnapshotFile.flush();
+    }
+
+    @VisibleForTesting
+    public static void resetForTests()
+    {
+        _failedSnapshotFile.close();
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/columniterator/IndexedSliceReader.java b/src/java/org/apache/cassandra/db/columniterator/IndexedSliceReader.java
deleted file mode 100644
index b74a686..0000000
--- a/src/java/org/apache/cassandra/db/columniterator/IndexedSliceReader.java
+++ /dev/null
@@ -1,541 +0,0 @@
-/*
- * 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.db.columniterator;
-
-import java.io.IOException;
-import java.util.ArrayDeque;
-import java.util.Deque;
-import java.util.List;
-
-import com.google.common.collect.AbstractIterator;
-
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.composites.CellNameType;
-import org.apache.cassandra.db.composites.Composite;
-import org.apache.cassandra.db.filter.ColumnSlice;
-import org.apache.cassandra.io.sstable.CorruptSSTableException;
-import org.apache.cassandra.io.sstable.IndexHelper;
-import org.apache.cassandra.io.sstable.IndexHelper.IndexInfo;
-import org.apache.cassandra.io.sstable.SSTableReader;
-import org.apache.cassandra.io.util.FileDataInput;
-import org.apache.cassandra.io.util.FileMark;
-import org.apache.cassandra.tracing.Tracing;
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-/**
- * This is a reader that finds the block for a starting column and returns blocks before/after it for each next call.
- * This function assumes that the CF is sorted by name and exploits the name index.
- */
-class IndexedSliceReader extends AbstractIterator<OnDiskAtom> implements OnDiskAtomIterator
-{
-    private final ColumnFamily emptyColumnFamily;
-
-    private final SSTableReader sstable;
-    private final List<IndexHelper.IndexInfo> indexes;
-    private final FileDataInput originalInput;
-    private FileDataInput file;
-    private final boolean reversed;
-    private final ColumnSlice[] slices;
-    private final BlockFetcher fetcher;
-    private final Deque<OnDiskAtom> blockColumns = new ArrayDeque<OnDiskAtom>();
-    private final CellNameType comparator;
-
-    // Holds range tombstone in reverse queries. See addColumn()
-    private final Deque<OnDiskAtom> rangeTombstonesReversed;
-
-    /**
-     * This slice reader assumes that slices are sorted correctly, e.g. that for forward lookup slices are in
-     * lexicographic order of start elements and that for reverse lookup they are in reverse lexicographic order of
-     * finish (reverse start) elements. i.e. forward: [a,b],[d,e],[g,h] reverse: [h,g],[e,d],[b,a]. This reader also
-     * assumes that validation has been performed in terms of intervals (no overlapping intervals).
-     */
-    public IndexedSliceReader(SSTableReader sstable, RowIndexEntry indexEntry, FileDataInput input, ColumnSlice[] slices, boolean reversed)
-    {
-        Tracing.trace("Seeking to partition indexed section in data file");
-        this.sstable = sstable;
-        this.originalInput = input;
-        this.reversed = reversed;
-        this.slices = slices;
-        this.comparator = sstable.metadata.comparator;
-        this.rangeTombstonesReversed = reversed ? new ArrayDeque<OnDiskAtom>() : null;
-
-        try
-        {
-            this.indexes = indexEntry.columnsIndex();
-            emptyColumnFamily = ArrayBackedSortedColumns.factory.create(sstable.metadata);
-            if (indexes.isEmpty())
-            {
-                setToRowStart(indexEntry, input);
-                emptyColumnFamily.delete(DeletionTime.serializer.deserialize(file));
-                fetcher = new SimpleBlockFetcher();
-            }
-            else
-            {
-                emptyColumnFamily.delete(indexEntry.deletionTime());
-                fetcher = new IndexedBlockFetcher(indexEntry.position);
-            }
-        }
-        catch (IOException e)
-        {
-            sstable.markSuspect();
-            throw new CorruptSSTableException(e, file.getPath());
-        }
-    }
-
-    /**
-     * Sets the seek position to the start of the row for column scanning.
-     */
-    private void setToRowStart(RowIndexEntry rowEntry, FileDataInput in) throws IOException
-    {
-        if (in == null)
-        {
-            this.file = sstable.getFileDataInput(rowEntry.position);
-        }
-        else
-        {
-            this.file = in;
-            in.seek(rowEntry.position);
-        }
-        sstable.partitioner.decorateKey(ByteBufferUtil.readWithShortLength(file));
-    }
-
-    public ColumnFamily getColumnFamily()
-    {
-        return emptyColumnFamily;
-    }
-
-    public DecoratedKey getKey()
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    protected OnDiskAtom computeNext()
-    {
-        while (true)
-        {
-            if (reversed)
-            {
-                // Return all tombstone for the block first (see addColumn() below)
-                OnDiskAtom column = rangeTombstonesReversed.poll();
-                if (column != null)
-                    return column;
-            }
-
-            OnDiskAtom column = blockColumns.poll();
-            if (column == null)
-            {
-                if (!fetcher.fetchMoreData())
-                    return endOfData();
-            }
-            else
-            {
-                return column;
-            }
-        }
-    }
-
-    public void close() throws IOException
-    {
-        if (originalInput == null && file != null)
-            file.close();
-    }
-
-    protected void addColumn(OnDiskAtom col)
-    {
-        if (reversed)
-        {
-            /*
-             * We put range tomstone markers at the beginning of the range they delete. But for reversed queries,
-             * the caller still need to know about a RangeTombstone before it sees any column that it covers.
-             * To make that simple, we keep said tombstones separate and return them all before any column for
-             * a given block.
-             */
-            if (col instanceof RangeTombstone)
-                rangeTombstonesReversed.addFirst(col);
-            else
-                blockColumns.addFirst(col);
-        }
-        else
-        {
-            blockColumns.addLast(col);
-        }
-    }
-
-    private abstract class BlockFetcher
-    {
-        protected int currentSliceIdx;
-
-        protected BlockFetcher(int sliceIdx)
-        {
-            this.currentSliceIdx = sliceIdx;
-        }
-
-        /*
-         * Return the smallest key selected by the current ColumnSlice.
-         */
-        protected Composite currentStart()
-        {
-            return reversed ? slices[currentSliceIdx].finish : slices[currentSliceIdx].start;
-        }
-
-        /*
-         * Return the biggest key selected by the current ColumnSlice.
-         */
-        protected Composite currentFinish()
-        {
-            return reversed ? slices[currentSliceIdx].start : slices[currentSliceIdx].finish;
-        }
-
-        protected abstract boolean setNextSlice();
-
-        protected abstract boolean fetchMoreData();
-
-        protected boolean isColumnBeforeSliceStart(OnDiskAtom column)
-        {
-            return isBeforeSliceStart(column.name());
-        }
-
-        protected boolean isBeforeSliceStart(Composite name)
-        {
-            Composite start = currentStart();
-            return !start.isEmpty() && comparator.compare(name, start) < 0;
-        }
-
-        protected boolean isColumnBeforeSliceFinish(OnDiskAtom column)
-        {
-            Composite finish = currentFinish();
-            return finish.isEmpty() || comparator.compare(column.name(), finish) <= 0;
-        }
-
-        protected boolean isAfterSliceFinish(Composite name)
-        {
-            Composite finish = currentFinish();
-            return !finish.isEmpty() && comparator.compare(name, finish) > 0;
-        }
-    }
-
-    private class IndexedBlockFetcher extends BlockFetcher
-    {
-        // where this row starts
-        private final long columnsStart;
-
-        // the index entry for the next block to deserialize
-        private int nextIndexIdx = -1;
-
-        // index of the last block we've read from disk;
-        private int lastDeserializedBlock = -1;
-
-        // For reversed, keep columns at the beginning of the last deserialized block that
-        // may still match a slice
-        private final Deque<OnDiskAtom> prefetched;
-
-        public IndexedBlockFetcher(long columnsStart)
-        {
-            super(-1);
-            this.columnsStart = columnsStart;
-            this.prefetched = reversed ? new ArrayDeque<OnDiskAtom>() : null;
-            setNextSlice();
-        }
-
-        protected boolean setNextSlice()
-        {
-            while (++currentSliceIdx < slices.length)
-            {
-                nextIndexIdx = IndexHelper.indexFor(slices[currentSliceIdx].start, indexes, comparator, reversed, nextIndexIdx);
-                if (nextIndexIdx < 0 || nextIndexIdx >= indexes.size())
-                    // no index block for that slice
-                    continue;
-
-                // Check if we can exclude this slice entirely from the index
-                IndexInfo info = indexes.get(nextIndexIdx);
-                if (reversed)
-                {
-                    if (!isBeforeSliceStart(info.lastName))
-                        return true;
-                }
-                else
-                {
-                    if (!isAfterSliceFinish(info.firstName))
-                        return true;
-                }
-            }
-            nextIndexIdx = -1;
-            return false;
-        }
-
-        protected boolean hasMoreSlice()
-        {
-            return currentSliceIdx < slices.length;
-        }
-
-        protected boolean fetchMoreData()
-        {
-            if (!hasMoreSlice())
-                return false;
-
-            // If we read blocks in reversed disk order, we may have columns from the previous block to handle.
-            // Note that prefetched keeps columns in reversed disk order.
-            // Also note that Range Tombstone handling is a bit tricky, because we may run into range tombstones
-            // that cover a slice *after* we've move to the previous slice. To keep it simple, we simply include
-            // every RT in prefetched: it's only slightly inefficient to do so and there is only so much RT that
-            // can be mistakenly added this way.
-            if (reversed && !prefetched.isEmpty())
-            {
-                // Avoids some comparison when we know it's not useful
-                boolean inSlice = false;
-
-                OnDiskAtom prefetchedCol;
-                while ((prefetchedCol = prefetched.peek()) != null)
-                {
-                    // col is before slice, we update the slice
-                    if (isColumnBeforeSliceStart(prefetchedCol))
-                    {
-                        inSlice = false;
-
-                        // As explained above, we add RT unconditionally
-                        if (prefetchedCol instanceof RangeTombstone)
-                        {
-                            blockColumns.addLast(prefetched.poll());
-                            continue;
-                        }
-
-                        // Otherwise, we either move to the next slice. If we have no more slice, then
-                        // simply unwind prefetched entirely and add all RT.
-                        if (!setNextSlice())
-                        {
-                            while ((prefetchedCol = prefetched.poll()) != null)
-                                if (prefetchedCol instanceof RangeTombstone)
-                                    blockColumns.addLast(prefetchedCol);
-                            break;
-                        }
-
-                    }
-                    // col is within slice, all columns
-                    // (we go in reverse, so as soon as we are in a slice, no need to check
-                    // we're after the slice until we change slice)
-                    else if (inSlice || isColumnBeforeSliceFinish(prefetchedCol))
-                    {
-                        blockColumns.addLast(prefetched.poll());
-                        inSlice = true;
-                    }
-                    // if col is after slice, ignore
-                    else
-                    {
-                        prefetched.poll();
-                    }
-                }
-
-                if (!blockColumns.isEmpty())
-                    return true;
-                else if (!hasMoreSlice())
-                    return false;
-            }
-            try
-            {
-                return getNextBlock();
-            }
-            catch (IOException e)
-            {
-                throw new CorruptSSTableException(e, file.getPath());
-            }
-        }
-
-        private boolean getNextBlock() throws IOException
-        {
-            if (lastDeserializedBlock == nextIndexIdx)
-            {
-                if (reversed)
-                    nextIndexIdx--;
-                else
-                    nextIndexIdx++;
-            }
-            lastDeserializedBlock = nextIndexIdx;
-
-            // Are we done?
-            if (lastDeserializedBlock < 0 || lastDeserializedBlock >= indexes.size())
-                return false;
-
-            IndexInfo currentIndex = indexes.get(lastDeserializedBlock);
-
-            /* seek to the correct offset to the data, and calculate the data size */
-            long positionToSeek = columnsStart + currentIndex.offset;
-
-            // With new promoted indexes, our first seek in the data file will happen at that point.
-            if (file == null)
-                file = originalInput == null ? sstable.getFileDataInput(positionToSeek) : originalInput;
-
-            AtomDeserializer deserializer = emptyColumnFamily.metadata().getOnDiskDeserializer(file, sstable.descriptor.version);
-
-            file.seek(positionToSeek);
-            FileMark mark = file.mark();
-
-            // We remenber when we are whithin a slice to avoid some comparison
-            boolean inSlice = false;
-
-            // scan from index start
-            while (file.bytesPastMark(mark) < currentIndex.width || deserializer.hasUnprocessed())
-            {
-                // col is before slice
-                // (If in slice, don't bother checking that until we change slice)
-                Composite start = currentStart();
-                if (!inSlice && !start.isEmpty() && deserializer.compareNextTo(start) < 0)
-                {
-                    // If it's a rangeTombstone, then we need to read it and include it unless it's end
-                    // stops before our slice start.
-                    if (deserializer.nextIsRangeTombstone())
-                    {
-                        RangeTombstone rt = (RangeTombstone)deserializer.readNext();
-                        if (comparator.compare(rt.max, start) >= 0)
-                            addColumn(rt);
-                        continue;
-                    }
-
-                    if (reversed)
-                    {
-                        // the next slice select columns that are before the current one, so it may
-                        // match this column, so keep it around.
-                        prefetched.addFirst(deserializer.readNext());
-                    }
-                    else
-                    {
-                        deserializer.skipNext();
-                    }
-                }
-                // col is within slice
-                else
-                {
-                    Composite finish = currentFinish();
-                    if (finish.isEmpty() || deserializer.compareNextTo(finish) <= 0)
-                    {
-                        inSlice = true;
-                        addColumn(deserializer.readNext());
-                    }
-                    // col is after slice.
-                    else
-                    {
-                        // When reading forward, if we hit a column that sorts after the current slice, it means we're done with this slice.
-                        // For reversed, this may either mean that we're done with the current slice, or that we need to read the previous
-                        // index block. However, we can be sure that we are in the first case though (the current slice is done) if the first
-                        // columns of the block were not part of the current slice, i.e. if we have columns in prefetched.
-                        if (reversed && prefetched.isEmpty())
-                            break;
-
-                        if (!setNextSlice())
-                            break;
-
-                        inSlice = false;
-
-                        // The next index block now corresponds to the first block that may have columns for the newly set slice.
-                        // So if it's different from the current block, we're done with this block. And in that case, we know
-                        // that our prefetched columns won't match.
-                        if (nextIndexIdx != lastDeserializedBlock)
-                        {
-                            if (reversed)
-                                prefetched.clear();
-                            break;
-                        }
-
-                        // Even if the next slice may have column in this blocks, if we're reversed, those columns have been
-                        // prefetched and we're done with that block
-                        if (reversed)
-                            break;
-
-                        // otherwise, we will deal with that column at the next iteration
-                    }
-                }
-            }
-            return true;
-        }
-    }
-
-    private class SimpleBlockFetcher extends BlockFetcher
-    {
-        public SimpleBlockFetcher() throws IOException
-        {
-            // Since we have to deserialize in order and will read all slices might as well reverse the slices and
-            // behave as if it was not reversed
-            super(reversed ? slices.length - 1 : 0);
-
-            // We remenber when we are whithin a slice to avoid some comparison
-            boolean inSlice = false;
-
-            AtomDeserializer deserializer = emptyColumnFamily.metadata().getOnDiskDeserializer(file, sstable.descriptor.version);
-            while (deserializer.hasNext())
-            {
-                // col is before slice
-                // (If in slice, don't bother checking that until we change slice)
-                Composite start = currentStart();
-                if (!inSlice && !start.isEmpty() && deserializer.compareNextTo(start) < 0)
-                {
-                    // If it's a rangeTombstone, then we need to read it and include it unless it's end
-                    // stops before our slice start. Otherwise, we can skip it.
-                    if (deserializer.nextIsRangeTombstone())
-                    {
-                        RangeTombstone rt = (RangeTombstone)deserializer.readNext();
-                        if (comparator.compare(rt.max, start) >= 0)
-                            addColumn(rt);
-                    }
-                    else
-                    {
-                        deserializer.skipNext();
-                    }
-                    continue;
-                }
-
-                // col is within slice
-                Composite finish = currentFinish();
-                if (finish.isEmpty() || deserializer.compareNextTo(finish) <= 0)
-                {
-                    inSlice = true;
-                    addColumn(deserializer.readNext());
-                }
-                // col is after slice. more slices?
-                else
-                {
-                    inSlice = false;
-                    if (!setNextSlice())
-                        break;
-                }
-            }
-        }
-
-        protected boolean setNextSlice()
-        {
-            if (reversed)
-            {
-                if (currentSliceIdx <= 0)
-                    return false;
-
-                currentSliceIdx--;
-            }
-            else
-            {
-                if (currentSliceIdx >= slices.length - 1)
-                    return false;
-
-                currentSliceIdx++;
-            }
-            return true;
-        }
-
-        protected boolean fetchMoreData()
-        {
-            return false;
-        }
-    }
-}
diff --git a/src/java/org/apache/cassandra/db/columniterator/SSTableNamesIterator.java b/src/java/org/apache/cassandra/db/columniterator/SSTableNamesIterator.java
deleted file mode 100644
index eb5eed9..0000000
--- a/src/java/org/apache/cassandra/db/columniterator/SSTableNamesIterator.java
+++ /dev/null
@@ -1,261 +0,0 @@
-/*
- * 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.db.columniterator;
-
-import java.io.IOException;
-import java.util.*;
-
-import com.google.common.collect.AbstractIterator;
-
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.composites.CellName;
-import org.apache.cassandra.db.composites.CellNameType;
-import org.apache.cassandra.io.sstable.CorruptSSTableException;
-import org.apache.cassandra.io.sstable.IndexHelper;
-import org.apache.cassandra.io.sstable.SSTableReader;
-import org.apache.cassandra.io.util.FileDataInput;
-import org.apache.cassandra.io.util.FileMark;
-import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-public class SSTableNamesIterator extends AbstractIterator<OnDiskAtom> implements OnDiskAtomIterator
-{
-    private ColumnFamily cf;
-    private final SSTableReader sstable;
-    private FileDataInput fileToClose;
-    private Iterator<OnDiskAtom> iter;
-    public final SortedSet<CellName> columns;
-    public final DecoratedKey key;
-
-    public SSTableNamesIterator(SSTableReader sstable, DecoratedKey key, SortedSet<CellName> columns)
-    {
-        assert columns != null;
-        this.sstable = sstable;
-        this.columns = columns;
-        this.key = key;
-
-        RowIndexEntry indexEntry = sstable.getPosition(key, SSTableReader.Operator.EQ);
-        if (indexEntry == null)
-            return;
-
-        try
-        {
-            read(sstable, null, indexEntry);
-        }
-        catch (IOException e)
-        {
-            sstable.markSuspect();
-            throw new CorruptSSTableException(e, sstable.getFilename());
-        }
-        finally
-        {
-            if (fileToClose != null)
-                FileUtils.closeQuietly(fileToClose);
-        }
-    }
-
-    public SSTableNamesIterator(SSTableReader sstable, FileDataInput file, DecoratedKey key, SortedSet<CellName> columns, RowIndexEntry indexEntry)
-    {
-        assert columns != null;
-        this.sstable = sstable;
-        this.columns = columns;
-        this.key = key;
-
-        try
-        {
-            read(sstable, file, indexEntry);
-        }
-        catch (IOException e)
-        {
-            sstable.markSuspect();
-            throw new CorruptSSTableException(e, sstable.getFilename());
-        }
-    }
-
-    private FileDataInput createFileDataInput(long position)
-    {
-        fileToClose = sstable.getFileDataInput(position);
-        return fileToClose;
-    }
-
-    private void read(SSTableReader sstable, FileDataInput file, RowIndexEntry indexEntry)
-    throws IOException
-    {
-        List<IndexHelper.IndexInfo> indexList;
-
-        // If the entry is not indexed or the index is not promoted, read from the row start
-        if (!indexEntry.isIndexed())
-        {
-            if (file == null)
-                file = createFileDataInput(indexEntry.position);
-            else
-                file.seek(indexEntry.position);
-
-            DecoratedKey keyInDisk = sstable.partitioner.decorateKey(ByteBufferUtil.readWithShortLength(file));
-            assert keyInDisk.equals(key) : String.format("%s != %s in %s", keyInDisk, key, file.getPath());
-        }
-
-        indexList = indexEntry.columnsIndex();
-
-        if (!indexEntry.isIndexed())
-        {
-            ColumnFamilySerializer serializer = ColumnFamily.serializer;
-            try
-            {
-                cf = ArrayBackedSortedColumns.factory.create(sstable.metadata);
-                cf.delete(DeletionTime.serializer.deserialize(file));
-            }
-            catch (Exception e)
-            {
-                throw new IOException(serializer + " failed to deserialize " + sstable.getColumnFamilyName() + " with " + sstable.metadata + " from " + file, e);
-            }
-        }
-        else
-        {
-            cf = ArrayBackedSortedColumns.factory.create(sstable.metadata);
-            cf.delete(indexEntry.deletionTime());
-        }
-
-        List<OnDiskAtom> result = new ArrayList<OnDiskAtom>();
-        if (indexList.isEmpty())
-        {
-            readSimpleColumns(file, columns, result);
-        }
-        else
-        {
-            readIndexedColumns(sstable.metadata, file, columns, indexList, indexEntry.position, result);
-        }
-
-        // create an iterator view of the columns we read
-        iter = result.iterator();
-    }
-
-    private void readSimpleColumns(FileDataInput file, SortedSet<CellName> columnNames, List<OnDiskAtom> result)
-    {
-        Iterator<OnDiskAtom> atomIterator = cf.metadata().getOnDiskIterator(file, sstable.descriptor.version);
-        int n = 0;
-        while (atomIterator.hasNext())
-        {
-            OnDiskAtom column = atomIterator.next();
-            if (column instanceof Cell)
-            {
-                if (columnNames.contains(column.name()))
-                {
-                    result.add(column);
-                    if (++n >= columns.size())
-                        break;
-                }
-            }
-            else
-            {
-                result.add(column);
-            }
-        }
-    }
-
-    private void readIndexedColumns(CFMetaData metadata,
-                                    FileDataInput file,
-                                    SortedSet<CellName> columnNames,
-                                    List<IndexHelper.IndexInfo> indexList,
-                                    long basePosition,
-                                    List<OnDiskAtom> result)
-    throws IOException
-    {
-        /* get the various column ranges we have to read */
-        CellNameType comparator = metadata.comparator;
-        List<IndexHelper.IndexInfo> ranges = new ArrayList<IndexHelper.IndexInfo>();
-        int lastIndexIdx = -1;
-        for (CellName name : columnNames)
-        {
-            int index = IndexHelper.indexFor(name, indexList, comparator, false, lastIndexIdx);
-            if (index < 0 || index == indexList.size())
-                continue;
-            IndexHelper.IndexInfo indexInfo = indexList.get(index);
-            // Check the index block does contain the column names and that we haven't inserted this block yet.
-            if (comparator.compare(name, indexInfo.firstName) < 0 || index == lastIndexIdx)
-                continue;
-
-            ranges.add(indexInfo);
-            lastIndexIdx = index;
-        }
-
-        if (ranges.isEmpty())
-            return;
-
-        Iterator<CellName> toFetch = columnNames.iterator();
-        CellName nextToFetch = toFetch.next();
-        for (IndexHelper.IndexInfo indexInfo : ranges)
-        {
-            long positionToSeek = basePosition + indexInfo.offset;
-
-            // With new promoted indexes, our first seek in the data file will happen at that point.
-            if (file == null)
-                file = createFileDataInput(positionToSeek);
-
-            AtomDeserializer deserializer = cf.metadata().getOnDiskDeserializer(file, sstable.descriptor.version);
-            file.seek(positionToSeek);
-            FileMark mark = file.mark();
-            while (file.bytesPastMark(mark) < indexInfo.width && nextToFetch != null)
-            {
-                int cmp = deserializer.compareNextTo(nextToFetch);
-                if (cmp < 0)
-                {
-                    // If it's a rangeTombstone, then we need to read it and include
-                    // it if it includes our target. Otherwise, we can skip it.
-                    if (deserializer.nextIsRangeTombstone())
-                    {
-                        RangeTombstone rt = (RangeTombstone)deserializer.readNext();
-                        if (comparator.compare(rt.max, nextToFetch) >= 0)
-                            result.add(rt);
-                    }
-                    else
-                    {
-                        deserializer.skipNext();
-                    }
-                }
-                else if (cmp == 0)
-                {
-                    nextToFetch = toFetch.hasNext() ? toFetch.next() : null;
-                    result.add(deserializer.readNext());
-                }
-                else
-                    nextToFetch = toFetch.hasNext() ? toFetch.next() : null;
-            }
-        }
-    }
-
-    public DecoratedKey getKey()
-    {
-        return key;
-    }
-
-    public ColumnFamily getColumnFamily()
-    {
-        return cf;
-    }
-
-    protected OnDiskAtom computeNext()
-    {
-        if (iter == null || !iter.hasNext())
-            return endOfData();
-        return iter.next();
-    }
-
-    public void close() throws IOException { }
-}
diff --git a/src/java/org/apache/cassandra/db/columniterator/SSTableSliceIterator.java b/src/java/org/apache/cassandra/db/columniterator/SSTableSliceIterator.java
deleted file mode 100644
index 0057d52..0000000
--- a/src/java/org/apache/cassandra/db/columniterator/SSTableSliceIterator.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * 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.db.columniterator;
-
-import java.io.IOException;
-
-import org.apache.cassandra.db.ColumnFamily;
-import org.apache.cassandra.db.DecoratedKey;
-import org.apache.cassandra.db.OnDiskAtom;
-import org.apache.cassandra.db.RowIndexEntry;
-import org.apache.cassandra.db.filter.ColumnSlice;
-import org.apache.cassandra.io.sstable.SSTableReader;
-import org.apache.cassandra.io.util.FileDataInput;
-
-/**
- *  A Cell Iterator over SSTable
- */
-public class SSTableSliceIterator implements OnDiskAtomIterator
-{
-    private final OnDiskAtomIterator reader;
-    private final DecoratedKey key;
-
-    public SSTableSliceIterator(SSTableReader sstable, DecoratedKey key, ColumnSlice[] slices, boolean reversed)
-    {
-        this.key = key;
-        RowIndexEntry indexEntry = sstable.getPosition(key, SSTableReader.Operator.EQ);
-        this.reader = indexEntry == null ? null : createReader(sstable, indexEntry, null, slices, reversed);
-    }
-
-    /**
-     * An iterator for a slice within an SSTable
-     * @param sstable Keyspace for the CFS we are reading from
-     * @param file Optional parameter that input is read from.  If null is passed, this class creates an appropriate one automatically.
-     * If this class creates, it will close the underlying file when #close() is called.
-     * If a caller passes a non-null argument, this class will NOT close the underlying file when the iterator is closed (i.e. the caller is responsible for closing the file)
-     * In all cases the caller should explicitly #close() this iterator.
-     * @param key The key the requested slice resides under
-     * @param slices the column slices
-     * @param reversed Results are returned in reverse order iff reversed is true.
-     * @param indexEntry position of the row
-     */
-    public SSTableSliceIterator(SSTableReader sstable, FileDataInput file, DecoratedKey key, ColumnSlice[] slices, boolean reversed, RowIndexEntry indexEntry)
-    {
-        this.key = key;
-        reader = createReader(sstable, indexEntry, file, slices, reversed);
-    }
-
-    private static OnDiskAtomIterator createReader(SSTableReader sstable, RowIndexEntry indexEntry, FileDataInput file, ColumnSlice[] slices, boolean reversed)
-    {
-        return slices.length == 1 && slices[0].start.isEmpty() && !reversed
-             ? new SimpleSliceReader(sstable, indexEntry, file, slices[0].finish)
-             : new IndexedSliceReader(sstable, indexEntry, file, slices, reversed);
-    }
-
-    public DecoratedKey getKey()
-    {
-        return key;
-    }
-
-    public ColumnFamily getColumnFamily()
-    {
-        return reader == null ? null : reader.getColumnFamily();
-    }
-
-    public boolean hasNext()
-    {
-        return reader != null && reader.hasNext();
-    }
-
-    public OnDiskAtom next()
-    {
-        return reader.next();
-    }
-
-    public void remove()
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    public void close() throws IOException
-    {
-        if (reader != null)
-            reader.close();
-    }
-
-}
diff --git a/src/java/org/apache/cassandra/db/columniterator/SimpleSliceReader.java b/src/java/org/apache/cassandra/db/columniterator/SimpleSliceReader.java
deleted file mode 100644
index 702bddc..0000000
--- a/src/java/org/apache/cassandra/db/columniterator/SimpleSliceReader.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/*
- * 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.db.columniterator;
-
-import java.io.IOException;
-import java.util.Iterator;
-
-import com.google.common.collect.AbstractIterator;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.composites.CellNameType;
-import org.apache.cassandra.db.composites.Composite;
-import org.apache.cassandra.io.sstable.CorruptSSTableException;
-import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.sstable.SSTableReader;
-import org.apache.cassandra.io.util.FileDataInput;
-import org.apache.cassandra.tracing.Tracing;
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-class SimpleSliceReader extends AbstractIterator<OnDiskAtom> implements OnDiskAtomIterator
-{
-    private static final Logger logger = LoggerFactory.getLogger(SimpleSliceReader.class);
-
-    private final FileDataInput file;
-    private final boolean needsClosing;
-    private final Composite finishColumn;
-    private final CellNameType comparator;
-    private final ColumnFamily emptyColumnFamily;
-    private final Iterator<OnDiskAtom> atomIterator;
-
-    public SimpleSliceReader(SSTableReader sstable, RowIndexEntry indexEntry, FileDataInput input, Composite finishColumn)
-    {
-        Tracing.trace("Seeking to partition beginning in data file");
-        this.finishColumn = finishColumn;
-        this.comparator = sstable.metadata.comparator;
-        try
-        {
-            if (input == null)
-            {
-                this.file = sstable.getFileDataInput(indexEntry.position);
-                this.needsClosing = true;
-            }
-            else
-            {
-                this.file = input;
-                input.seek(indexEntry.position);
-                this.needsClosing = false;
-            }
-
-            // Skip key and data size
-            ByteBufferUtil.skipShortLength(file);
-
-            emptyColumnFamily = ArrayBackedSortedColumns.factory.create(sstable.metadata);
-            emptyColumnFamily.delete(DeletionTime.serializer.deserialize(file));
-            atomIterator = emptyColumnFamily.metadata().getOnDiskIterator(file, sstable.descriptor.version);
-        }
-        catch (IOException e)
-        {
-            sstable.markSuspect();
-            throw new CorruptSSTableException(e, sstable.getFilename());
-        }
-    }
-
-    protected OnDiskAtom computeNext()
-    {
-        if (!atomIterator.hasNext())
-            return endOfData();
-
-        OnDiskAtom column = atomIterator.next();
-        if (!finishColumn.isEmpty() && comparator.compare(column.name(), finishColumn) > 0)
-            return endOfData();
-
-        return column;
-    }
-
-    public ColumnFamily getColumnFamily()
-    {
-        return emptyColumnFamily;
-    }
-
-    public void close() throws IOException
-    {
-        if (needsClosing)
-            file.close();
-    }
-
-    public DecoratedKey getKey()
-    {
-        throw new UnsupportedOperationException();
-    }
-}
diff --git a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java b/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java
index 9f3b410..702ace5 100644
--- a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java
+++ b/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java
@@ -31,7 +31,7 @@
     // how often should we log syngs that lag behind our desired period
     private static final long LAG_REPORT_INTERVAL = TimeUnit.MINUTES.toMillis(5);
 
-    private final Thread thread;
+    private Thread thread;
     private volatile boolean shutdown = false;
 
     // all Allocations written before this time will be synced
@@ -45,6 +45,10 @@
     protected final WaitQueue syncComplete = new WaitQueue();
     protected final Semaphore haveWork = new Semaphore(1);
 
+    final CommitLog commitLog;
+    private final String name;
+    private final long pollIntervalMillis;
+
     private static final Logger logger = LoggerFactory.getLogger(AbstractCommitLogService.class);
 
     /**
@@ -55,6 +59,14 @@
      */
     AbstractCommitLogService(final CommitLog commitLog, final String name, final long pollIntervalMillis)
     {
+        this.commitLog = commitLog;
+        this.name = name;
+        this.pollIntervalMillis = pollIntervalMillis;
+    }
+
+    // Separated into individual method to ensure relevant objects are constructed before this is started.
+    void start()
+    {
         if (pollIntervalMillis < 1)
             throw new IllegalArgumentException(String.format("Commit log flush interval must be positive: %dms", pollIntervalMillis));
 
@@ -171,6 +183,29 @@
         haveWork.release(1);
     }
 
+    /**
+     * FOR TESTING ONLY
+     */
+    public void restartUnsafe()
+    {
+        while (haveWork.availablePermits() < 1)
+            haveWork.release();
+
+        while (haveWork.availablePermits() > 1)
+        {
+            try
+            {
+                haveWork.acquire();
+            }
+            catch (InterruptedException e)
+            {
+                throw new RuntimeException(e);
+            }
+        }
+        shutdown = false;
+        start();
+    }
+
     public void awaitTermination() throws InterruptedException
     {
         thread.join();
@@ -185,4 +220,4 @@
     {
         return pending.get();
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/db/commitlog/BatchCommitLogService.java b/src/java/org/apache/cassandra/db/commitlog/BatchCommitLogService.java
index dd55d24..ceb5d64 100644
--- a/src/java/org/apache/cassandra/db/commitlog/BatchCommitLogService.java
+++ b/src/java/org/apache/cassandra/db/commitlog/BatchCommitLogService.java
@@ -31,7 +31,7 @@
         // wait until record has been safely persisted to disk
         pending.incrementAndGet();
         haveWork.release();
-        alloc.awaitDiskSync();
+        alloc.awaitDiskSync(commitLog.metrics.waitingOnCommit);
         pending.decrementAndGet();
     }
 }
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
index 2484e9b..460ecfe 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
@@ -26,22 +26,28 @@
 import javax.management.ObjectName;
 
 import com.google.common.annotations.VisibleForTesting;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.commons.lang3.StringUtils;
 
+import com.github.tjake.ICRC32;
+
 import org.apache.cassandra.config.Config;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.ParameterizedClass;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.io.FSWriteError;
-import org.apache.cassandra.io.sstable.SSTableDeletingTask;
-import org.apache.cassandra.io.util.DataOutputByteBuffer;
+import org.apache.cassandra.io.compress.CompressionParameters;
+import org.apache.cassandra.io.compress.ICompressor;
+import org.apache.cassandra.io.util.BufferedDataOutputStreamPlus;
+import org.apache.cassandra.io.util.DataOutputBufferFixed;
 import org.apache.cassandra.metrics.CommitLogMetrics;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.CRC32Factory;
 import org.apache.cassandra.utils.JVMStabilityInspector;
-import org.apache.cassandra.utils.PureJavaCrc32;
 
 import static org.apache.cassandra.db.commitlog.CommitLogSegment.*;
 
@@ -53,39 +59,61 @@
 {
     private static final Logger logger = LoggerFactory.getLogger(CommitLog.class);
 
-    public static final CommitLog instance = new CommitLog();
+    public static final CommitLog instance = CommitLog.construct();
 
     // we only permit records HALF the size of a commit log, to ensure we don't spin allocating many mostly
     // empty segments when writing large records
-    private static final long MAX_MUTATION_SIZE = DatabaseDescriptor.getCommitLogSegmentSize() >> 1;
+    private final long MAX_MUTATION_SIZE = DatabaseDescriptor.getCommitLogSegmentSize() >> 1;
 
     public final CommitLogSegmentManager allocator;
-    public final CommitLogArchiver archiver = new CommitLogArchiver();
+    public final CommitLogArchiver archiver;
     final CommitLogMetrics metrics;
     final AbstractCommitLogService executor;
 
-    private CommitLog()
+    volatile Configuration configuration;
+    final public String location;
+
+    static private CommitLog construct()
     {
-        DatabaseDescriptor.createAllDirectories();
-
-        allocator = new CommitLogSegmentManager();
-
-        executor = DatabaseDescriptor.getCommitLogSync() == Config.CommitLogSync.batch
-                 ? new BatchCommitLogService(this)
-                 : new PeriodicCommitLogService(this);
+        CommitLog log = new CommitLog(DatabaseDescriptor.getCommitLogLocation(), CommitLogArchiver.construct());
 
         MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
         try
         {
-            mbs.registerMBean(this, new ObjectName("org.apache.cassandra.db:type=Commitlog"));
+            mbs.registerMBean(log, new ObjectName("org.apache.cassandra.db:type=Commitlog"));
         }
         catch (Exception e)
         {
             throw new RuntimeException(e);
         }
+        return log.start();
+    }
+
+    @VisibleForTesting
+    CommitLog(String location, CommitLogArchiver archiver)
+    {
+        this.location = location;
+        this.configuration = new Configuration(DatabaseDescriptor.getCommitLogCompression());
+        DatabaseDescriptor.createAllDirectories();
+
+        this.archiver = archiver;
+        metrics = new CommitLogMetrics();
+
+        executor = DatabaseDescriptor.getCommitLogSync() == Config.CommitLogSync.batch
+                ? new BatchCommitLogService(this)
+                : new PeriodicCommitLogService(this);
+
+        allocator = new CommitLogSegmentManager(this);
 
         // register metrics
-        metrics = new CommitLogMetrics(executor, allocator);
+        metrics.attach(executor, allocator);
+    }
+
+    CommitLog start()
+    {
+        executor.start();
+        allocator.start();
+        return this;
     }
 
     /**
@@ -95,6 +123,10 @@
      */
     public int recover() throws IOException
     {
+        // If createReserveSegments is already flipped, the CLSM is running and recovery has already taken place.
+        if (allocator.createReserveSegments)
+            return 0;
+
         FilenameFilter unmanagedFilesFilter = new FilenameFilter()
         {
             public boolean accept(File dir, String name)
@@ -130,7 +162,7 @@
             logger.info("Log replay complete, {} replayed mutations", replayed);
 
             for (File f : files)
-                CommitLog.instance.allocator.recycleSegment(f);
+                allocator.recycleSegment(f);
         }
 
         allocator.enableReserveSegmentCreation();
@@ -145,7 +177,7 @@
      */
     public int recover(File... clogs) throws IOException
     {
-        CommitLogReplayer recovery = new CommitLogReplayer();
+        CommitLogReplayer recovery = CommitLogReplayer.construct(this);
         recovery.recover(clogs);
         return recovery.blockForWrites();
     }
@@ -155,7 +187,9 @@
      */
     public void recover(String path) throws IOException
     {
-        recover(new File(path));
+        CommitLogReplayer recovery = CommitLogReplayer.construct(this);
+        recovery.recover(new File(path), false);
+        recovery.blockForWrites();
     }
 
     /**
@@ -224,12 +258,10 @@
         }
 
         Allocation alloc = allocator.allocate(mutation, (int) totalSize);
-        try
+        ICRC32 checksum = CRC32Factory.instance.create();
+        final ByteBuffer buffer = alloc.getBuffer();
+        try (BufferedDataOutputStreamPlus dos = new DataOutputBufferFixed(buffer))
         {
-            PureJavaCrc32 checksum = new PureJavaCrc32();
-            final ByteBuffer buffer = alloc.getBuffer();
-            DataOutputByteBuffer dos = new DataOutputByteBuffer(buffer);
-
             // checksummed length
             dos.writeInt((int) size);
             checksum.update(buffer, buffer.position() - 4, 4);
@@ -263,7 +295,7 @@
      */
     public void discardCompletedSegments(final UUID cfId, final ReplayPosition context)
     {
-        logger.debug("discard completed log segments for {}, column family {}", context, cfId);
+        logger.trace("discard completed log segments for {}, table {}", context, cfId);
 
         // Go thru the active segment files, which are ordered oldest to newest, marking the
         // flushed CF as clean, until we reach the segment file containing the ReplayPosition passed
@@ -276,12 +308,12 @@
 
             if (segment.isUnused())
             {
-                logger.debug("Commit log segment {} is unused", segment);
+                logger.trace("Commit log segment {} is unused", segment);
                 allocator.recycleSegment(segment);
             }
             else
             {
-                logger.debug("Not safe to delete{} commit log segment {}; dirty is {}",
+                logger.trace("Not safe to delete{} commit log segment {}; dirty is {}",
                         (iter.hasNext() ? "" : " active"), segment, segment.dirtyString());
             }
 
@@ -293,26 +325,6 @@
     }
 
     @Override
-    public long getCompletedTasks()
-    {
-        return metrics.completedTasks.value();
-    }
-
-    @Override
-    public long getPendingTasks()
-    {
-        return metrics.pendingTasks.value();
-    }
-
-    /**
-     * @return the total size occupied by commitlog segments expressed in bytes. (used by MBean)
-     */
-    public long getTotalCommitlogSize()
-    {
-        return metrics.totalCommitLogSize.value();
-    }
-
-    @Override
     public String getArchiveCommand()
     {
         return archiver.archiveCommand;
@@ -355,6 +367,30 @@
         return new ArrayList<>(archiver.archivePending.keySet());
     }
 
+    @Override
+    public long getActiveContentSize()
+    {
+        long size = 0;
+        for (CommitLogSegment segment : allocator.getActiveSegments())
+            size += segment.contentSize();
+        return size;
+    }
+
+    @Override
+    public long getActiveOnDiskSize()
+    {
+        return allocator.onDiskSize();
+    }
+
+    @Override
+    public Map<String, Double> getActiveSegmentCompressionRatios()
+    {
+        Map<String, Double> segmentRatios = new TreeMap<>();
+        for (CommitLogSegment segment : allocator.getActiveSegments())
+            segmentRatios.put(segment.getName(), 1.0 * segment.onDiskSize() / segment.contentSize());
+        return segmentRatios;
+    }
+
     /**
      * Shuts down the threads used by the commit log, blocking until completion.
      */
@@ -368,24 +404,66 @@
 
     /**
      * FOR TESTING PURPOSES. See CommitLogAllocator.
-     *
-     * There is a race at the moment, even if this method
-     * is synchronized we can still create an allocation
-     * on a segment that will be closed in this method,
-     * therefore causing {@link Allocation#awaitDiskSync()} to hang
-     * forever. This typically happens because of the mutations created
-     * by {@link org.apache.cassandra.io.sstable.SSTableReader.GlobalTidy},
-     * that's why we wait for all deletions to complete firtst.
+     * @return the number of files recovered
      */
-    public synchronized void resetUnsafe()
+    public int resetUnsafe(boolean deleteSegments) throws IOException
     {
-        SSTableDeletingTask.waitForDeletions();
-        sync(true);
-        allocator.resetUnsafe();
+        stopUnsafe(deleteSegments);
+        resetConfiguration();
+        return restartUnsafe();
+    }
+
+    /**
+     * FOR TESTING PURPOSES. See CommitLogAllocator.
+     */
+    public void stopUnsafe(boolean deleteSegments)
+    {
+        executor.shutdown();
+        try
+        {
+            executor.awaitTermination();
+        }
+        catch (InterruptedException e)
+        {
+            throw new RuntimeException(e);
+        }
+        allocator.stopUnsafe(deleteSegments);
         CommitLogSegment.resetReplayLimit();
     }
 
     /**
+     * FOR TESTING PURPOSES.
+     */
+    public void resetConfiguration()
+    {
+        this.configuration = new Configuration(DatabaseDescriptor.getCommitLogCompression());
+    }
+
+    /**
+     * FOR TESTING PURPOSES.  See CommitLogAllocator
+     */
+    public int restartUnsafe() throws IOException
+    {
+        allocator.start();
+        executor.restartUnsafe();
+        try
+        {
+            return recover();
+        }
+        catch (FSWriteError e)
+        {
+            // Workaround for a class of races that keeps showing up on Windows tests.
+            // stop/start/reset path on Windows with segment deletion is very touchy/brittle
+            // and the timing keeps getting screwed up. Rather than chasing our tail further
+            // or rewriting the CLSM, just report that we didn't recover anything back up
+            // the chain. This will silence most intermittent test failures on Windows
+            // and appropriately fail tests that expected segments to be recovered that
+            // were not.
+            return 0;
+        }
+    }
+
+    /**
      * Used by tests.
      *
      * @return the number of active segments (segments with unflushed data in them)
@@ -405,6 +483,7 @@
             case die:
             case stop:
                 StorageService.instance.stopTransports();
+                //$FALL-THROUGH$
             case stop_commit:
                 logger.error(String.format("%s. Commit disk failure policy is %s; terminating thread", message, DatabaseDescriptor.getCommitFailurePolicy()), t);
                 return false;
@@ -416,4 +495,58 @@
         }
     }
 
+    public static final class Configuration
+    {
+        /**
+         * The compressor class.
+         */
+        private final ParameterizedClass compressorClass;
+
+        /**
+         * The compressor used to compress the segments.
+         */
+        private final ICompressor compressor;
+
+        public Configuration(ParameterizedClass compressorClass)
+        {
+            this.compressorClass = compressorClass;
+            this.compressor = compressorClass != null ? CompressionParameters.createCompressor(compressorClass) : null;
+        }
+
+        /**
+         * Checks if the segments must be compressed.
+         * @return <code>true</code> if the segments must be compressed, <code>false</code> otherwise.
+         */
+        public boolean useCompression()
+        {
+            return compressor != null;
+        }
+
+        /**
+         * Returns the compressor used to compress the segments.
+         * @return the compressor used to compress the segments
+         */
+        public ICompressor getCompressor()
+        {
+            return compressor;
+        }
+
+        /**
+         * Returns the compressor class.
+         * @return the compressor class
+         */
+        public ParameterizedClass getCompressorClass()
+        {
+            return compressorClass;
+        }
+
+        /**
+         * Returns the compressor name.
+         * @return the compressor name.
+         */
+        public String getCompressorName()
+        {
+            return useCompression() ? compressor.getClass().getSimpleName() : "none";
+        }
+    }
 }
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogArchiver.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogArchiver.java
index 8d9a6b3..d9a511e 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogArchiver.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogArchiver.java
@@ -1,5 +1,5 @@
 /*
- *
+ * 
  * 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
@@ -7,16 +7,16 @@
  * 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.db.commitlog;
 
@@ -32,10 +32,11 @@
 
 import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.compress.CompressionParameters;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.WrappedRunnable;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -52,36 +53,45 @@
     }
 
     public final Map<String, Future<?>> archivePending = new ConcurrentHashMap<String, Future<?>>();
-    private final ExecutorService executor = new JMXEnabledThreadPoolExecutor("CommitLogArchiver");
+    private final ExecutorService executor;
     final String archiveCommand;
     final String restoreCommand;
     final String restoreDirectories;
     public long restorePointInTime;
     public final TimeUnit precision;
 
-    public CommitLogArchiver()
+    public CommitLogArchiver(String archiveCommand, String restoreCommand, String restoreDirectories,
+            long restorePointInTime, TimeUnit precision)
+    {
+        this.archiveCommand = archiveCommand;
+        this.restoreCommand = restoreCommand;
+        this.restoreDirectories = restoreDirectories;
+        this.restorePointInTime = restorePointInTime;
+        this.precision = precision;
+        executor = !Strings.isNullOrEmpty(archiveCommand) ? new JMXEnabledThreadPoolExecutor("CommitLogArchiver") : null;
+    }
+
+    public static CommitLogArchiver disabled()
+    {
+        return new CommitLogArchiver(null, null, null, Long.MAX_VALUE, TimeUnit.MICROSECONDS);
+    }
+
+    public static CommitLogArchiver construct()
     {
         Properties commitlog_commands = new Properties();
-        InputStream stream = null;
-        try
+        try (InputStream stream = CommitLogArchiver.class.getClassLoader().getResourceAsStream("commitlog_archiving.properties"))
         {
-            stream = getClass().getClassLoader().getResourceAsStream("commitlog_archiving.properties");
-
             if (stream == null)
             {
-                logger.debug("No commitlog_archiving properties found; archive + pitr will be disabled");
-                archiveCommand = null;
-                restoreCommand = null;
-                restoreDirectories = null;
-                restorePointInTime = Long.MAX_VALUE;
-                precision = TimeUnit.MICROSECONDS;
+                logger.trace("No commitlog_archiving properties found; archive + pitr will be disabled");
+                return disabled();
             }
             else
             {
                 commitlog_commands.load(stream);
-                archiveCommand = commitlog_commands.getProperty("archive_command");
-                restoreCommand = commitlog_commands.getProperty("restore_command");
-                restoreDirectories = commitlog_commands.getProperty("restore_directories");
+                String archiveCommand = commitlog_commands.getProperty("archive_command");
+                String restoreCommand = commitlog_commands.getProperty("restore_command");
+                String restoreDirectories = commitlog_commands.getProperty("restore_directories");
                 if (restoreDirectories != null && !restoreDirectories.isEmpty())
                 {
                     for (String dir : restoreDirectories.split(DELIMITER))
@@ -97,7 +107,8 @@
                     }
                 }
                 String targetTime = commitlog_commands.getProperty("restore_point_in_time");
-                precision = TimeUnit.valueOf(commitlog_commands.getProperty("precision", "MICROSECONDS"));
+                TimeUnit precision = TimeUnit.valueOf(commitlog_commands.getProperty("precision", "MICROSECONDS"));
+                long restorePointInTime;
                 try
                 {
                     restorePointInTime = Strings.isNullOrEmpty(targetTime) ? Long.MAX_VALUE : format.parse(targetTime).getTime();
@@ -106,16 +117,14 @@
                 {
                     throw new RuntimeException("Unable to parse restore target time", e);
                 }
+                return new CommitLogArchiver(archiveCommand, restoreCommand, restoreDirectories, restorePointInTime, precision);
             }
         }
         catch (IOException e)
         {
             throw new RuntimeException("Unable to load commitlog_archiving.properties", e);
         }
-        finally
-        {
-            FileUtils.closeQuietly(stream);
-        }
+
     }
 
     public void maybeArchive(final CommitLogSegment segment)
@@ -206,7 +215,7 @@
                 CommitLogDescriptor descriptor;
                 if (fromHeader == null && fromName == null)
                     throw new IllegalStateException("Cannot safely construct descriptor for segment, either from its name or its header: " + fromFile.getPath());
-                else if (fromHeader != null && fromName != null && !fromHeader.equals(fromName))
+                else if (fromHeader != null && fromName != null && !fromHeader.equalsIgnoringCompression(fromName))
                     throw new IllegalStateException(String.format("Cannot safely construct descriptor for segment, as name and header descriptors do not match (%s vs %s): %s", fromHeader, fromName, fromFile.getPath()));
                 else if (fromName != null && fromHeader == null && fromName.version >= CommitLogDescriptor.VERSION_21)
                     throw new IllegalStateException("Cannot safely construct descriptor for segment, as name descriptor implies a version that should contain a header descriptor, but that descriptor could not be read: " + fromFile.getPath());
@@ -214,13 +223,24 @@
                     descriptor = fromHeader;
                 else descriptor = fromName;
 
-                if (descriptor.version > CommitLogDescriptor.VERSION_21)
+                if (descriptor.version > CommitLogDescriptor.VERSION_22)
                     throw new IllegalStateException("Unsupported commit log version: " + descriptor.version);
 
+                if (descriptor.compression != null) {
+                    try
+                    {
+                        CompressionParameters.createCompressor(descriptor.compression);
+                    }
+                    catch (ConfigurationException e)
+                    {
+                        throw new IllegalStateException("Unknown compression", e);
+                    }
+                }
+
                 File toFile = new File(DatabaseDescriptor.getCommitLogLocation(), descriptor.fileName());
                 if (toFile.exists())
                 {
-                    logger.debug("Skipping restore of archive {} as the segment already exists in the restore location {}",
+                    logger.trace("Skipping restore of archive {} as the segment already exists in the restore location {}",
                                  fromFile.getPath(), toFile.getPath());
                     continue;
                 }
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogDescriptor.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogDescriptor.java
index 91c81e1..c4728fd 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogDescriptor.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogDescriptor.java
@@ -20,18 +20,28 @@
  */
 package org.apache.cassandra.db.commitlog;
 
+import java.io.DataInput;
 import java.io.EOFException;
 import java.io.File;
 import java.io.IOException;
 import java.io.RandomAccessFile;
 import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Map;
+import java.util.TreeMap;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.github.tjake.ICRC32;
+
+import org.apache.cassandra.config.ParameterizedClass;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.FSReadError;
 import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.PureJavaCrc32;
+import org.apache.cassandra.utils.CRC32Factory;
+import org.json.simple.JSONValue;
 
 public class CommitLogDescriptor
 {
@@ -40,42 +50,69 @@
     private static final String FILENAME_EXTENSION = ".log";
     // match both legacy and new version of commitlogs Ex: CommitLog-12345.log and CommitLog-4-12345.log.
     private static final Pattern COMMIT_LOG_FILE_PATTERN = Pattern.compile(FILENAME_PREFIX + "((\\d+)(" + SEPARATOR + "\\d+)?)" + FILENAME_EXTENSION);
+    private static final String COMPRESSION_PARAMETERS_KEY = "compressionParameters";
+    private static final String COMPRESSION_CLASS_KEY = "compressionClass";
 
     public static final int VERSION_12 = 2;
     public static final int VERSION_20 = 3;
     public static final int VERSION_21 = 4;
+    public static final int VERSION_22 = 5;
     /**
      * Increment this number if there is a changes in the commit log disc layout or MessagingVersion changes.
      * Note: make sure to handle {@link #getMessagingVersion()}
      */
-    public static final int current_version = VERSION_21;
-
-    // [version, id, checksum]
-    static final int HEADER_SIZE = 4 + 8 + 4;
+    @VisibleForTesting
+    public static final int current_version = VERSION_22;
 
     final int version;
     public final long id;
+    public final ParameterizedClass compression;
 
-    public CommitLogDescriptor(int version, long id)
+    public CommitLogDescriptor(int version, long id, ParameterizedClass compression)
     {
         this.version = version;
         this.id = id;
+        this.compression = compression;
     }
 
-    public CommitLogDescriptor(long id)
+    public CommitLogDescriptor(long id, ParameterizedClass compression)
     {
-        this(current_version, id);
+        this(current_version, id, compression);
     }
 
-    static void writeHeader(ByteBuffer out, CommitLogDescriptor descriptor)
+    public static void writeHeader(ByteBuffer out, CommitLogDescriptor descriptor)
     {
-        out.putInt(0, descriptor.version);
-        out.putLong(4, descriptor.id);
-        PureJavaCrc32 crc = new PureJavaCrc32();
+        ICRC32 crc = CRC32Factory.instance.create();
+        out.putInt(descriptor.version);
         crc.updateInt(descriptor.version);
+        out.putLong(descriptor.id);
         crc.updateInt((int) (descriptor.id & 0xFFFFFFFFL));
         crc.updateInt((int) (descriptor.id >>> 32));
-        out.putInt(12, crc.getCrc());
+        if (descriptor.version >= VERSION_22) {
+            String parametersString = constructParametersString(descriptor);
+            byte[] parametersBytes = parametersString.getBytes(StandardCharsets.UTF_8);
+            if (parametersBytes.length != (((short) parametersBytes.length) & 0xFFFF))
+                throw new ConfigurationException(String.format("Compression parameters too long, length %d cannot be above 65535.",
+                                                               parametersBytes.length));
+            out.putShort((short) parametersBytes.length);
+            crc.updateInt(parametersBytes.length);
+            out.put(parametersBytes);
+            crc.update(parametersBytes, 0, parametersBytes.length);
+        } else
+            assert descriptor.compression == null;
+        out.putInt(crc.getCrc());
+    }
+
+    private static String constructParametersString(CommitLogDescriptor descriptor)
+    {
+        Map<String, Object> params = new TreeMap<String, Object>();
+        ParameterizedClass compression = descriptor.compression;
+        if (compression != null)
+        {
+            params.put(COMPRESSION_PARAMETERS_KEY, compression.parameters);
+            params.put(COMPRESSION_CLASS_KEY, compression.class_name);
+        }
+        return JSONValue.toJSONString(params);
     }
 
     public static CommitLogDescriptor fromHeader(File file)
@@ -83,16 +120,7 @@
         try (RandomAccessFile raf = new RandomAccessFile(file, "r"))
         {
             assert raf.getFilePointer() == 0;
-            int version = raf.readInt();
-            long id = raf.readLong();
-            int crc = raf.readInt();
-            PureJavaCrc32 checkcrc = new PureJavaCrc32();
-            checkcrc.updateInt(version);
-            checkcrc.updateInt((int) (id & 0xFFFFFFFFL));
-            checkcrc.updateInt((int) (id >>> 32));
-            if (crc == checkcrc.getCrc())
-                return new CommitLogDescriptor(version, id);
-            return null;
+            return readHeader(raf);
         }
         catch (EOFException e)
         {
@@ -104,6 +132,44 @@
         }
     }
 
+    public static CommitLogDescriptor readHeader(DataInput input) throws IOException
+    {
+        ICRC32 checkcrc = CRC32Factory.instance.create();
+        int version = input.readInt();
+        checkcrc.updateInt(version);
+        long id = input.readLong();
+        checkcrc.updateInt((int) (id & 0xFFFFFFFFL));
+        checkcrc.updateInt((int) (id >>> 32));
+        int parametersLength = 0;
+        if (version >= VERSION_22) {
+            parametersLength = input.readShort() & 0xFFFF;
+            checkcrc.updateInt(parametersLength);
+        }
+        // This should always succeed as parametersLength cannot be too long even for a
+        // corrupt segment file.
+        byte[] parametersBytes = new byte[parametersLength];
+        input.readFully(parametersBytes);
+        checkcrc.update(parametersBytes, 0, parametersBytes.length);
+        int crc = input.readInt();
+        if (crc == checkcrc.getCrc())
+            return new CommitLogDescriptor(version, id,
+                    parseCompression((Map<?, ?>) JSONValue.parse(new String(parametersBytes, StandardCharsets.UTF_8))));
+        return null;
+    }
+
+    @SuppressWarnings("unchecked")
+    private static ParameterizedClass parseCompression(Map<?, ?> params)
+    {
+        if (params == null)
+            return null;
+        String className = (String) params.get(COMPRESSION_CLASS_KEY);
+        if (className == null)
+            return null;
+
+        Map<String, String> cparams = (Map<String, String>) params.get(COMPRESSION_PARAMETERS_KEY);
+        return new ParameterizedClass(className, cparams);
+    }
+
     public static CommitLogDescriptor fromFileName(String name)
     {
         Matcher matcher;
@@ -114,7 +180,7 @@
             throw new UnsupportedOperationException("Commitlog segment is too old to open; upgrade to 1.2.5+ first");
 
         long id = Long.parseLong(matcher.group(3).split(SEPARATOR)[1]);
-        return new CommitLogDescriptor(Integer.parseInt(matcher.group(2)), id);
+        return new CommitLogDescriptor(Integer.parseInt(matcher.group(2)), id, null);
     }
 
     public int getMessagingVersion()
@@ -127,6 +193,8 @@
                 return MessagingService.VERSION_20;
             case VERSION_21:
                 return MessagingService.VERSION_21;
+            case VERSION_22:
+                return MessagingService.VERSION_22;
             default:
                 throw new IllegalStateException("Unknown commitlog version " + version);
         }
@@ -148,7 +216,7 @@
 
     public String toString()
     {
-        return "(" + version + "," + id + ")";
+        return "(" + version + "," + id + (compression != null ? "," + compression : "") + ")";
     }
 
     public boolean equals(Object that)
@@ -156,9 +224,14 @@
         return that instanceof CommitLogDescriptor && equals((CommitLogDescriptor) that);
     }
 
-    public boolean equals(CommitLogDescriptor that)
+    public boolean equalsIgnoringCompression(CommitLogDescriptor that)
     {
         return this.version == that.version && this.id == that.id;
     }
 
+    public boolean equals(CommitLogDescriptor that)
+    {
+        return equalsIgnoringCompression(that) && Objects.equal(this.compression, that.compression);
+    }
+
 }
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogMBean.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogMBean.java
index 9f3eb9b..3b20bbc 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogMBean.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogMBean.java
@@ -20,31 +20,11 @@
 
 import java.io.IOException;
 import java.util.List;
+import java.util.Map;
 
 public interface CommitLogMBean
 {
     /**
-     * Get the number of completed tasks
-     * @see org.apache.cassandra.metrics.CommitLogMetrics#completedTasks
-     */
-    @Deprecated
-    public long getCompletedTasks();
-
-    /**
-     * Get the number of tasks waiting to be executed
-     * @see org.apache.cassandra.metrics.CommitLogMetrics#pendingTasks
-     */
-    @Deprecated
-    public long getPendingTasks();
-
-    /**
-     * Get the current size used by all the commitlog segments.
-     * @see org.apache.cassandra.metrics.CommitLogMetrics#totalCommitLogSize
-     */
-    @Deprecated
-    public long getTotalCommitlogSize();
-
-    /**
      *  Command to execute to archive a commitlog segment.  Blank to disabled.
      */
     public String getArchiveCommand();
@@ -84,9 +64,24 @@
      * @return file names (not full paths) of active commit log segments (segments containing unflushed data)
      */
     public List<String> getActiveSegmentNames();
-    
+
     /**
      * @return Files which are pending for archival attempt.  Does NOT include failed archive attempts.
      */
     public List<String> getArchivingSegmentNames();
+
+    /**
+     * @return The size of the mutations in all active commit log segments (uncompressed).
+     */
+    public long getActiveContentSize();
+
+    /**
+     * @return The space taken on disk by the commit log (compressed).
+     */
+    public long getActiveOnDiskSize();
+
+    /**
+     * @return A map between active log segments and the compression ratio achieved for each.
+     */
+    public Map<String, Double> getActiveSegmentCompressionRatios();
 }
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
index 57f4b90..3cf4d0f 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
@@ -18,7 +18,13 @@
  */
 package org.apache.cassandra.db.commitlog;
 
-import java.io.*;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.EOFException;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.concurrent.Future;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -27,24 +33,34 @@
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Multimap;
-import com.google.common.collect.Ordering;
 import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import com.github.tjake.ICRC32;
 
 import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.concurrent.StageManager;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.*;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.compress.CompressionParameters;
+import org.apache.cassandra.io.compress.ICompressor;
+import org.apache.cassandra.io.util.ByteBufferDataInput;
 import org.apache.cassandra.io.util.FastByteArrayInputStream;
+import org.apache.cassandra.io.util.FileDataInput;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.io.util.RandomAccessReader;
-import org.apache.cassandra.utils.*;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.CRC32Factory;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.WrappedRunnable;
 import org.cliffc.high_scale_lib.NonBlockingHashSet;
 
 public class CommitLogReplayer
 {
+    static final String IGNORE_REPLAY_ERRORS_PROPERTY = "cassandra.commitlog.ignorereplayerrors";
     private static final Logger logger = LoggerFactory.getLogger(CommitLogReplayer.class);
     private static final int MAX_OUTSTANDING_REPLAY_COUNT = Integer.getInteger("cassandra.commitlog_max_outstanding_replay_count", 1024);
     private static final int LEGACY_END_OF_SEGMENT_MARKER = 0;
@@ -53,35 +69,39 @@
     private final List<Future<?>> futures;
     private final Map<UUID, AtomicInteger> invalidMutations;
     private final AtomicInteger replayedCount;
-    private final Map<UUID, ReplayPosition> cfPositions;
+    private final Map<UUID, ReplayPosition.ReplayFilter> cfPersisted;
     private final ReplayPosition globalPosition;
-    private final PureJavaCrc32 checksum;
+    private final ICRC32 checksum;
     private byte[] buffer;
+    private byte[] uncompressedBuffer;
 
     private final ReplayFilter replayFilter;
+    private final CommitLogArchiver archiver;
 
-    public CommitLogReplayer()
+    CommitLogReplayer(CommitLog commitLog, ReplayPosition globalPosition, Map<UUID, ReplayPosition.ReplayFilter> cfPersisted, ReplayFilter replayFilter)
     {
         this.keyspacesRecovered = new NonBlockingHashSet<Keyspace>();
         this.futures = new ArrayList<Future<?>>();
         this.buffer = new byte[4096];
+        this.uncompressedBuffer = new byte[4096];
         this.invalidMutations = new HashMap<UUID, AtomicInteger>();
         // count the number of replayed mutation. We don't really care about atomicity, but we need it to be a reference.
         this.replayedCount = new AtomicInteger();
-        this.checksum = new PureJavaCrc32();
+        this.checksum = CRC32Factory.instance.create();
+        this.cfPersisted = cfPersisted;
+        this.globalPosition = globalPosition;
+        this.replayFilter = replayFilter;
+        this.archiver = commitLog.archiver;
+    }
 
-        replayFilter = ReplayFilter.create();
-
+    public static CommitLogReplayer construct(CommitLog commitLog)
+    {
         // compute per-CF and global replay positions
-        cfPositions = new HashMap<UUID, ReplayPosition>();
-        Ordering<ReplayPosition> replayPositionOrdering = Ordering.from(ReplayPosition.comparator);
+        Map<UUID, ReplayPosition.ReplayFilter> cfPersisted = new HashMap<>();
+        ReplayFilter replayFilter = ReplayFilter.create();
+        ReplayPosition globalPosition = null;
         for (ColumnFamilyStore cfs : ColumnFamilyStore.all())
         {
-            // it's important to call RP.gRP per-cf, before aggregating all the positions w/ the Ordering.min call
-            // below: gRP will return NONE if there are no flushed sstables, which is important to have in the
-            // list (otherwise we'll just start replay from the first flush position that we do have, which is not correct).
-            ReplayPosition rp = ReplayPosition.getReplayPosition(cfs.getSSTables());
-
             // but, if we've truncted the cf in question, then we need to need to start replay after the truncation
             ReplayPosition truncatedAt = SystemKeyspace.getTruncatedPosition(cfs.metadata.cfId);
             if (truncatedAt != null)
@@ -89,7 +109,7 @@
                 // Point in time restore is taken to mean that the tables need to be recovered even if they were
                 // deleted at a later point in time. Any truncation record after that point must thus be cleared prior
                 // to recovery (CASSANDRA-9195).
-                long restoreTime = CommitLog.instance.archiver.restorePointInTime;
+                long restoreTime = commitLog.archiver.restorePointInTime;
                 long truncatedTime = SystemKeyspace.getTruncatedAt(cfs.metadata.cfId);
                 if (truncatedTime > restoreTime)
                 {
@@ -99,34 +119,38 @@
                                     cfs.metadata.ksName,
                                     cfs.metadata.cfName);
                         SystemKeyspace.removeTruncationRecord(cfs.metadata.cfId);
+                        truncatedAt = null;
                     }
                 }
-                else
-                {
-                    rp = replayPositionOrdering.max(Arrays.asList(rp, truncatedAt));
-                }
             }
 
-            cfPositions.put(cfs.metadata.cfId, rp);
+            ReplayPosition.ReplayFilter filter = new ReplayPosition.ReplayFilter(cfs.getSSTables(), truncatedAt);
+            if (!filter.isEmpty())
+                cfPersisted.put(cfs.metadata.cfId, filter);
+            else
+                globalPosition = ReplayPosition.NONE; // if we have no ranges for this CF, we must replay everything and filter
         }
-        globalPosition = replayPositionOrdering.min(cfPositions.values());
-        logger.debug("Global replay position is {} from columnfamilies {}", globalPosition, FBUtilities.toString(cfPositions));
+        if (globalPosition == null)
+            globalPosition = ReplayPosition.firstNotCovered(cfPersisted.values());
+        logger.debug("Global replay position is {} from columnfamilies {}", globalPosition, FBUtilities.toString(cfPersisted));
+        return new CommitLogReplayer(commitLog, globalPosition, cfPersisted, replayFilter);
     }
 
     public void recover(File[] clogs) throws IOException
     {
-        for (final File file : clogs)
-            recover(file);
+        int i;
+        for (i = 0; i < clogs.length; ++i)
+            recover(clogs[i], i + 1 == clogs.length);
     }
 
     public int blockForWrites()
     {
         for (Map.Entry<UUID, AtomicInteger> entry : invalidMutations.entrySet())
-            logger.info(String.format("Skipped %d mutations from unknown (probably removed) CF with id %s", entry.getValue().intValue(), entry.getKey()));
+            logger.warn(String.format("Skipped %d mutations from unknown (probably removed) CF with id %s", entry.getValue().intValue(), entry.getKey()));
 
         // wait for all the writes to finish on the mutation stage
         FBUtilities.waitOnFutures(futures);
-        logger.debug("Finished waiting on mutations from recovery");
+        logger.trace("Finished waiting on mutations from recovery");
 
         // flush replayed keyspaces
         futures.clear();
@@ -136,58 +160,41 @@
         return replayedCount.get();
     }
 
-    private int readSyncMarker(CommitLogDescriptor descriptor, int offset, RandomAccessReader reader) throws IOException
+    private int readSyncMarker(CommitLogDescriptor descriptor, int offset, RandomAccessReader reader, boolean tolerateTruncation) throws IOException
     {
         if (offset > reader.length() - CommitLogSegment.SYNC_MARKER_SIZE)
         {
-            if (offset != reader.length() && offset != Integer.MAX_VALUE)
-                logger.warn("Encountered bad header at position {} of Commit log {}; not enough room for a header", offset, reader.getPath());
-            // cannot possibly be a header here. if we're == length(), assume it's a correctly written final segment
+            // There was no room in the segment to write a final header. No data could be present here.
             return -1;
         }
         reader.seek(offset);
-        PureJavaCrc32 crc = new PureJavaCrc32();
+        ICRC32 crc = CRC32Factory.instance.create();
         crc.updateInt((int) (descriptor.id & 0xFFFFFFFFL));
         crc.updateInt((int) (descriptor.id >>> 32));
         crc.updateInt((int) reader.getPosition());
         int end = reader.readInt();
-        long filecrc;
-        if (descriptor.version < CommitLogDescriptor.VERSION_21)
-            filecrc = reader.readLong();
-        else
-            filecrc = reader.readInt() & 0xffffffffL;
+        long filecrc = reader.readInt() & 0xffffffffL;
         if (crc.getValue() != filecrc)
         {
             if (end != 0 || filecrc != 0)
             {
-                logger.warn("Encountered bad header at position {} of commit log {}, with invalid CRC. The end of segment marker should be zero.", offset, reader.getPath());
+                handleReplayError(false,
+                                  "Encountered bad header at position %d of commit log %s, with invalid CRC. " +
+                                  "The end of segment marker should be zero.",
+                                  offset, reader.getPath());
             }
             return -1;
         }
         else if (end < offset || end > reader.length())
         {
-            logger.warn("Encountered bad header at position {} of commit log {}, with bad position but valid CRC", offset, reader.getPath());
+            handleReplayError(tolerateTruncation, "Encountered bad header at position %d of commit log %s, with bad position but valid CRC",
+                              offset, reader.getPath());
             return -1;
         }
         return end;
     }
 
-    private int getStartOffset(long segmentId, int version)
-    {
-        if (globalPosition.segment < segmentId)
-        {
-            if (version >= CommitLogDescriptor.VERSION_21)
-                return CommitLogDescriptor.HEADER_SIZE + CommitLogSegment.SYNC_MARKER_SIZE;
-            else
-                return 0;
-        }
-        else if (globalPosition.segment == segmentId)
-            return globalPosition.position;
-        else
-            return -1;
-    }
-
-    private abstract static class ReplayFilter
+    abstract static class ReplayFilter
     {
         public abstract Iterable<ColumnFamily> filter(Mutation mutation);
 
@@ -262,232 +269,397 @@
         }
     }
 
-    public void recover(File file) throws IOException
+    /**
+     * consult the known-persisted ranges for our sstables;
+     * if the position is covered by one of them it does not need to be replayed
+     *
+     * @return true iff replay is necessary
+     */
+    private boolean shouldReplay(UUID cfId, ReplayPosition position)
     {
-        logger.info("Replaying {}", file.getPath());
-        CommitLogDescriptor desc = CommitLogDescriptor.fromFileName(file.getName());
-        final long segmentId = desc.id;
-        logger.info("Replaying {} (CL version {}, messaging version {})",
-                    file.getPath(),
-                    desc.version,
-                    desc.getMessagingVersion());
-        RandomAccessReader reader = RandomAccessReader.open(new File(file.getAbsolutePath()));
+        ReplayPosition.ReplayFilter filter = cfPersisted.get(cfId);
+        return filter == null || filter.shouldReplay(position);
+    }
 
+    @SuppressWarnings("resource")
+    public void recover(File file, boolean tolerateTruncation) throws IOException
+    {
+        CommitLogDescriptor desc = CommitLogDescriptor.fromFileName(file.getName());
+        RandomAccessReader reader = RandomAccessReader.open(new File(file.getAbsolutePath()));
         try
         {
-            assert reader.length() <= Integer.MAX_VALUE;
-            int offset = getStartOffset(segmentId, desc.version);
-            if (offset < 0)
+            if (desc.version < CommitLogDescriptor.VERSION_21)
             {
-                logger.debug("skipping replay of fully-flushed {}", file);
+                if (logAndCheckIfShouldSkip(file, desc))
+                    return;
+                if (globalPosition.segment == desc.id)
+                    reader.seek(globalPosition.position);
+                replaySyncSection(reader, (int) reader.getPositionLimit(), desc, desc.fileName(), tolerateTruncation);
                 return;
             }
 
-            int prevEnd = CommitLogDescriptor.HEADER_SIZE;
-            main: while (true)
+            final long segmentId = desc.id;
+            try
             {
+                desc = CommitLogDescriptor.readHeader(reader);
+            }
+            catch (IOException e)
+            {
+                desc = null;
+            }
+            if (desc == null) {
+                handleReplayError(false, "Could not read commit log descriptor in file %s", file);
+                return;
+            }
+            if (segmentId != desc.id)
+            {
+                handleReplayError(false, "Segment id mismatch (filename %d, descriptor %d) in file %s", segmentId, desc.id, file);
+                // continue processing if ignored.
+            }
 
-                int end = prevEnd;
-                if (desc.version < CommitLogDescriptor.VERSION_21)
-                    end = Integer.MAX_VALUE;
-                else
+            if (logAndCheckIfShouldSkip(file, desc))
+                return;
+
+            ICompressor compressor = null;
+            if (desc.compression != null)
+            {
+                try
                 {
-                    do { end = readSyncMarker(desc, end, reader); }
-                    while (end < offset && end > prevEnd);
+                    compressor = CompressionParameters.createCompressor(desc.compression);
+                }
+                catch (ConfigurationException e)
+                {
+                    handleReplayError(false, "Unknown compression: %s", e.getMessage());
+                    return;
+                }
+            }
+
+            assert reader.length() <= Integer.MAX_VALUE;
+            int end = (int) reader.getFilePointer();
+            int replayEnd = end;
+
+            while ((end = readSyncMarker(desc, end, reader, tolerateTruncation)) >= 0)
+            {
+                int replayPos = replayEnd + CommitLogSegment.SYNC_MARKER_SIZE;
+
+                if (logger.isTraceEnabled())
+                    logger.trace("Replaying {} between {} and {}", file, reader.getFilePointer(), end);
+                if (compressor != null)
+                {
+                    int uncompressedLength = reader.readInt();
+                    replayEnd = replayPos + uncompressedLength;
+                } else
+                {
+                    replayEnd = end;
                 }
 
-                if (end < prevEnd)
-                    break;
+                if (segmentId == globalPosition.segment && replayEnd < globalPosition.position)
+                    // Skip over flushed section.
+                    continue;
 
-                if (logger.isDebugEnabled())
-                    logger.debug("Replaying {} between {} and {}", file, offset, end);
-
-                reader.seek(offset);
-
-                 /* read the logs populate Mutation and apply */
-                while (reader.getPosition() < end && !reader.isEOF())
+                FileDataInput sectionReader = reader;
+                String errorContext = desc.fileName();
+                // In the uncompressed case the last non-fully-flushed section can be anywhere in the file.
+                boolean tolerateErrorsInSection = tolerateTruncation;
+                if (compressor != null)
                 {
-                    if (logger.isDebugEnabled())
-                        logger.debug("Reading mutation at {}", reader.getFilePointer());
+                    // In the compressed case we know if this is the last section.
+                    tolerateErrorsInSection &= end == reader.length() || end < 0;
 
-                    long claimedCRC32;
-                    int serializedSize;
+                    int start = (int) reader.getFilePointer();
                     try
                     {
-                        // any of the reads may hit EOF
-                        serializedSize = reader.readInt();
-                        if (serializedSize == LEGACY_END_OF_SEGMENT_MARKER)
-                        {
-                            logger.debug("Encountered end of segment marker at {}", reader.getFilePointer());
-                            break main;
-                        }
-
-                        // Mutation must be at LEAST 10 bytes:
-                        // 3 each for a non-empty Keyspace and Key (including the
-                        // 2-byte length from writeUTF/writeWithShortLength) and 4 bytes for column count.
-                        // This prevents CRC by being fooled by special-case garbage in the file; see CASSANDRA-2128
-                        if (serializedSize < 10)
-                            break main;
-
-                        long claimedSizeChecksum;
-                        if (desc.version < CommitLogDescriptor.VERSION_21)
-                            claimedSizeChecksum = reader.readLong();
-                        else
-                            claimedSizeChecksum = reader.readInt() & 0xffffffffL;
-                        checksum.reset();
-                        if (desc.version < CommitLogDescriptor.VERSION_20)
-                            checksum.update(serializedSize);
-                        else
-                            checksum.updateInt(serializedSize);
-
-                        if (checksum.getValue() != claimedSizeChecksum)
-                            break main; // entry wasn't synced correctly/fully. that's
-                        // ok.
-
-                        if (serializedSize > buffer.length)
-                            buffer = new byte[(int) (1.2 * serializedSize)];
-                        reader.readFully(buffer, 0, serializedSize);
-                        if (desc.version < CommitLogDescriptor.VERSION_21)
-                            claimedCRC32 = reader.readLong();
-                        else
-                            claimedCRC32 = reader.readInt() & 0xffffffffL;
+                        int compressedLength = end - start;
+                        if (logger.isTraceEnabled())
+                            logger.trace("Decompressing {} between replay positions {} and {}",
+                                         file,
+                                         replayPos,
+                                         replayEnd);
+                        if (compressedLength > buffer.length)
+                            buffer = new byte[(int) (1.2 * compressedLength)];
+                        reader.readFully(buffer, 0, compressedLength);
+                        int uncompressedLength = replayEnd - replayPos;
+                        if (uncompressedLength > uncompressedBuffer.length)
+                            uncompressedBuffer = new byte[(int) (1.2 * uncompressedLength)];
+                        compressedLength = compressor.uncompress(buffer, 0, compressedLength, uncompressedBuffer, 0);
+                        sectionReader = new ByteBufferDataInput(ByteBuffer.wrap(uncompressedBuffer), reader.getPath(), replayPos, 0);
+                        errorContext = "compressed section at " + start + " in " + errorContext;
                     }
-                    catch (EOFException eof)
+                    catch (IOException | ArrayIndexOutOfBoundsException e)
                     {
-                        break main; // last CL entry didn't get completely written. that's ok.
-                    }
-
-                    checksum.update(buffer, 0, serializedSize);
-                    if (claimedCRC32 != checksum.getValue())
-                    {
-                        // this entry must not have been fsynced. probably the rest is bad too,
-                        // but just in case there is no harm in trying them (since we still read on an entry boundary)
+                        handleReplayError(tolerateErrorsInSection,
+                                          "Unexpected exception decompressing section at %d: %s",
+                                          start, e);
                         continue;
                     }
-
-                    /* deserialize the commit log entry */
-                    FastByteArrayInputStream bufIn = new FastByteArrayInputStream(buffer, 0, serializedSize);
-                    final Mutation mutation;
-                    try
-                    {
-                        mutation = Mutation.serializer.deserialize(new DataInputStream(bufIn),
-                                                                   desc.getMessagingVersion(),
-                                                                   ColumnSerializer.Flag.LOCAL);
-                        // doublecheck that what we read is [still] valid for the current schema
-                        for (ColumnFamily cf : mutation.getColumnFamilies())
-                            for (Cell cell : cf)
-                                cf.getComparator().validate(cell.name());
-                    }
-                    catch (UnknownColumnFamilyException ex)
-                    {
-                        if (ex.cfId == null)
-                            continue;
-                        AtomicInteger i = invalidMutations.get(ex.cfId);
-                        if (i == null)
-                        {
-                            i = new AtomicInteger(1);
-                            invalidMutations.put(ex.cfId, i);
-                        }
-                        else
-                            i.incrementAndGet();
-                        continue;
-                    }
-                    catch (Throwable t)
-                    {
-                        JVMStabilityInspector.inspectThrowable(t);
-                        File f = File.createTempFile("mutation", "dat");
-                        DataOutputStream out = new DataOutputStream(new FileOutputStream(f));
-                        try
-                        {
-                            out.write(buffer, 0, serializedSize);
-                        }
-                        finally
-                        {
-                            out.close();
-                        }
-                        String st = String.format("Unexpected error deserializing mutation; saved to %s and ignored.  This may be caused by replaying a mutation against a table with the same name but incompatible schema.  Exception follows: ",
-                                                  f.getAbsolutePath());
-                        logger.error(st, t);
-                        continue;
-                    }
-
-                    if (logger.isDebugEnabled())
-                        logger.debug("replaying mutation for {}.{}: {}", mutation.getKeyspaceName(), ByteBufferUtil.bytesToHex(mutation.key()), "{" + StringUtils.join(mutation.getColumnFamilies().iterator(), ", ") + "}");
-
-                    final long entryLocation = reader.getFilePointer();
-                    Runnable runnable = new WrappedRunnable()
-                    {
-                        public void runMayThrow() throws IOException
-                        {
-                            if (Schema.instance.getKSMetaData(mutation.getKeyspaceName()) == null)
-                                return;
-                            if (pointInTimeExceeded(mutation))
-                                return;
-
-                            final Keyspace keyspace = Keyspace.open(mutation.getKeyspaceName());
-
-                            // Rebuild the mutation, omitting column families that
-                            //    a) the user has requested that we ignore,
-                            //    b) have already been flushed,
-                            // or c) are part of a cf that was dropped.
-                            // Keep in mind that the cf.name() is suspect. do every thing based on the cfid instead.
-                            Mutation newMutation = null;
-                            for (ColumnFamily columnFamily : replayFilter.filter(mutation))
-                            {
-                                if (Schema.instance.getCF(columnFamily.id()) == null)
-                                    continue; // dropped
-
-                                ReplayPosition rp = cfPositions.get(columnFamily.id());
-
-                                // replay if current segment is newer than last flushed one or,
-                                // if it is the last known segment, if we are after the replay position
-                                if (segmentId > rp.segment || (segmentId == rp.segment && entryLocation > rp.position))
-                                {
-                                    if (newMutation == null)
-                                        newMutation = new Mutation(mutation.getKeyspaceName(), mutation.key());
-                                    newMutation.add(columnFamily);
-                                    replayedCount.incrementAndGet();
-                                }
-                            }
-                            if (newMutation != null)
-                            {
-                                assert !newMutation.isEmpty();
-                                Keyspace.open(newMutation.getKeyspaceName()).apply(newMutation, false);
-                                keyspacesRecovered.add(keyspace);
-                            }
-                        }
-                    };
-                    futures.add(StageManager.getStage(Stage.MUTATION).submit(runnable));
-                    if (futures.size() > MAX_OUTSTANDING_REPLAY_COUNT)
-                    {
-                        FBUtilities.waitOnFutures(futures);
-                        futures.clear();
-                    }
                 }
 
-                if (desc.version < CommitLogDescriptor.VERSION_21)
+                if (!replaySyncSection(sectionReader, replayEnd, desc, errorContext, tolerateErrorsInSection))
                     break;
-
-                offset = end + CommitLogSegment.SYNC_MARKER_SIZE;
-                prevEnd = end;
             }
         }
         finally
         {
             FileUtils.closeQuietly(reader);
-            logger.info("Finished reading {}", file);
+            logger.debug("Finished reading {}", file);
+        }
+    }
+
+    public boolean logAndCheckIfShouldSkip(File file, CommitLogDescriptor desc)
+    {
+        logger.debug("Replaying {} (CL version {}, messaging version {}, compression {})",
+                    file.getPath(),
+                    desc.version,
+                    desc.getMessagingVersion(),
+                    desc.compression);
+
+        if (globalPosition.segment > desc.id)
+        {
+            logger.trace("skipping replay of fully-flushed {}", file);
+            return true;
+        }
+        return false;
+    }
+
+    /**
+     * Replays a sync section containing a list of mutations.
+     *
+     * @return Whether replay should continue with the next section.
+     */
+    private boolean replaySyncSection(FileDataInput reader, int end, CommitLogDescriptor desc, String errorContext, boolean tolerateErrors) throws IOException
+    {
+         /* read the logs populate Mutation and apply */
+        while (reader.getFilePointer() < end && !reader.isEOF())
+        {
+            long mutationStart = reader.getFilePointer();
+            if (logger.isTraceEnabled())
+                logger.trace("Reading mutation at {}", mutationStart);
+
+            long claimedCRC32;
+            int serializedSize;
+            try
+            {
+                // We rely on reading serialized size == 0 (LEGACY_END_OF_SEGMENT_MARKER) to identify the end
+                // of a segment, which happens naturally due to the 0 padding of the empty segment on creation.
+                // However, with 2.1 era commitlogs it's possible that the last mutation ended less than 4 bytes 
+                // from the end of the file, which means that we'll be unable to read an a full int and instead 
+                // read an EOF here
+                if(end - reader.getFilePointer() < 4)
+                {
+                    logger.trace("Not enough bytes left for another mutation in this CommitLog segment, continuing");
+                    return false;
+                }
+
+                // any of the reads may hit EOF
+                serializedSize = reader.readInt();
+                if (serializedSize == LEGACY_END_OF_SEGMENT_MARKER)
+                {
+                    logger.trace("Encountered end of segment marker at {}", reader.getFilePointer());
+                    return false;
+                }
+
+                // Mutation must be at LEAST 10 bytes:
+                // 3 each for a non-empty Keyspace and Key (including the
+                // 2-byte length from writeUTF/writeWithShortLength) and 4 bytes for column count.
+                // This prevents CRC by being fooled by special-case garbage in the file; see CASSANDRA-2128
+                if (serializedSize < 10)
+                {
+                    handleReplayError(tolerateErrors,
+                                      "Invalid mutation size %d at %d in %s",
+                                      serializedSize, mutationStart, errorContext);
+                    return false;
+                }
+
+                long claimedSizeChecksum;
+                if (desc.version < CommitLogDescriptor.VERSION_21)
+                    claimedSizeChecksum = reader.readLong();
+                else
+                    claimedSizeChecksum = reader.readInt() & 0xffffffffL;
+                checksum.reset();
+                if (desc.version < CommitLogDescriptor.VERSION_20)
+                    checksum.update(serializedSize);
+                else
+                    checksum.updateInt(serializedSize);
+
+                if (checksum.getValue() != claimedSizeChecksum)
+                {
+                    handleReplayError(tolerateErrors,
+                                      "Mutation size checksum failure at %d in %s",
+                                      mutationStart, errorContext);
+                    return false;
+                }
+                // ok.
+
+                if (serializedSize > buffer.length)
+                    buffer = new byte[(int) (1.2 * serializedSize)];
+                reader.readFully(buffer, 0, serializedSize);
+                if (desc.version < CommitLogDescriptor.VERSION_21)
+                    claimedCRC32 = reader.readLong();
+                else
+                    claimedCRC32 = reader.readInt() & 0xffffffffL;
+            }
+            catch (EOFException eof)
+            {
+                handleReplayError(tolerateErrors,
+                                  "Unexpected end of segment",
+                                  mutationStart, errorContext);
+                return false; // last CL entry didn't get completely written. that's ok.
+            }
+
+            checksum.update(buffer, 0, serializedSize);
+            if (claimedCRC32 != checksum.getValue())
+            {
+                handleReplayError(tolerateErrors,
+                                  "Mutation checksum failure at %d in %s",
+                                  mutationStart, errorContext);
+                continue;
+            }
+            replayMutation(buffer, serializedSize, (int) reader.getFilePointer(), desc);
+        }
+        return true;
+    }
+
+    /**
+     * Deserializes and replays a commit log entry.
+     */
+    void replayMutation(byte[] inputBuffer, int size,
+            final int entryLocation, final CommitLogDescriptor desc) throws IOException
+    {
+
+        final Mutation mutation;
+        try (FastByteArrayInputStream bufIn = new FastByteArrayInputStream(inputBuffer, 0, size))
+        {
+            mutation = Mutation.serializer.deserialize(new DataInputStream(bufIn),
+                                                       desc.getMessagingVersion(),
+                                                       ColumnSerializer.Flag.LOCAL);
+            // doublecheck that what we read is [still] valid for the current schema
+            for (ColumnFamily cf : mutation.getColumnFamilies())
+                for (Cell cell : cf)
+                    cf.getComparator().validate(cell.name());
+        }
+        catch (UnknownColumnFamilyException ex)
+        {
+            if (ex.cfId == null)
+                return;
+            AtomicInteger i = invalidMutations.get(ex.cfId);
+            if (i == null)
+            {
+                i = new AtomicInteger(1);
+                invalidMutations.put(ex.cfId, i);
+            }
+            else
+                i.incrementAndGet();
+            return;
+        }
+        catch (Throwable t)
+        {
+            JVMStabilityInspector.inspectThrowable(t);
+            File f = File.createTempFile("mutation", "dat");
+
+            try (DataOutputStream out = new DataOutputStream(new FileOutputStream(f)))
+            {
+                out.write(inputBuffer, 0, size);
+            }
+
+            // Checksum passed so this error can't be permissible.
+            handleReplayError(false,
+                              "Unexpected error deserializing mutation; saved to %s.  " +
+                              "This may be caused by replaying a mutation against a table with the same name but incompatible schema.  " +
+                              "Exception follows: %s",
+                              f.getAbsolutePath(),
+                              t);
+            return;
+        }
+
+        if (logger.isTraceEnabled())
+            logger.trace("replaying mutation for {}.{}: {}", mutation.getKeyspaceName(), ByteBufferUtil.bytesToHex(mutation.key()), "{" + StringUtils.join(mutation.getColumnFamilies().iterator(), ", ") + "}");
+
+        Runnable runnable = new WrappedRunnable()
+        {
+            public void runMayThrow() throws IOException
+            {
+                if (Schema.instance.getKSMetaData(mutation.getKeyspaceName()) == null)
+                    return;
+                if (pointInTimeExceeded(mutation))
+                    return;
+
+                final Keyspace keyspace = Keyspace.open(mutation.getKeyspaceName());
+
+                // Rebuild the mutation, omitting column families that
+                //    a) the user has requested that we ignore,
+                //    b) have already been flushed,
+                // or c) are part of a cf that was dropped.
+                // Keep in mind that the cf.name() is suspect. do every thing based on the cfid instead.
+                Mutation newMutation = null;
+                for (ColumnFamily columnFamily : replayFilter.filter(mutation))
+                {
+                    if (Schema.instance.getCF(columnFamily.id()) == null)
+                        continue; // dropped
+
+                    if (shouldReplay(columnFamily.id(), new ReplayPosition(desc.id, entryLocation)))
+                    {
+                        if (newMutation == null)
+                            newMutation = new Mutation(mutation.getKeyspaceName(), mutation.key());
+                        newMutation.add(columnFamily);
+                        replayedCount.incrementAndGet();
+                    }
+                }
+                if (newMutation != null)
+                {
+                    assert !newMutation.isEmpty();
+                    Keyspace.open(newMutation.getKeyspaceName()).apply(newMutation, false);
+                    keyspacesRecovered.add(keyspace);
+                }
+            }
+        };
+        futures.add(StageManager.getStage(Stage.MUTATION).submit(runnable));
+        if (futures.size() > MAX_OUTSTANDING_REPLAY_COUNT)
+        {
+            FBUtilities.waitOnFutures(futures);
+            futures.clear();
         }
     }
 
     protected boolean pointInTimeExceeded(Mutation fm)
     {
-        long restoreTarget = CommitLog.instance.archiver.restorePointInTime;
+        long restoreTarget = archiver.restorePointInTime;
 
         for (ColumnFamily families : fm.getColumnFamilies())
         {
-            if (CommitLog.instance.archiver.precision.toMillis(families.maxTimestamp()) > restoreTarget)
+            if (archiver.precision.toMillis(families.maxTimestamp()) > restoreTarget)
                 return true;
         }
         return false;
     }
+
+    static void handleReplayError(boolean permissible, String message, Object... messageArgs) throws IOException
+    {
+        String msg = String.format(message, messageArgs);
+        IOException e = new CommitLogReplayException(msg);
+        if (permissible)
+            logger.error("Ignoring commit log replay error likely due to incomplete flush to disk", e);
+        else if (Boolean.getBoolean(IGNORE_REPLAY_ERRORS_PROPERTY))
+            logger.error("Ignoring commit log replay error", e);
+        else if (!CommitLog.handleCommitError("Failed commit log replay", e))
+        {
+            logger.error("Replay stopped. If you wish to override this error and continue starting the node ignoring " +
+                         "commit log replay problems, specify -D" + IGNORE_REPLAY_ERRORS_PROPERTY + "=true " +
+                         "on the command line");
+            throw e;
+        }
+    }
+
+    @SuppressWarnings("serial")
+    public static class CommitLogReplayException extends IOException
+    {
+        public CommitLogReplayException(String message, Throwable cause)
+        {
+            super(message, cause);
+        }
+
+        public CommitLogReplayException(String message)
+        {
+            super(message);
+        }
+    }
 }
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
index d68b3c7..ba28f3e 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
@@ -19,10 +19,9 @@
 
 import java.io.File;
 import java.io.IOException;
-import java.io.RandomAccessFile;
 import java.nio.ByteBuffer;
-import java.nio.MappedByteBuffer;
 import java.nio.channels.FileChannel;
+import java.nio.file.StandardOpenOption;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Comparator;
@@ -34,7 +33,12 @@
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import com.codahale.metrics.Timer;
+import com.github.tjake.ICRC32;
+
+import org.apache.cassandra.utils.CRC32Factory;
 import org.cliffc.high_scale_lib.NonBlockingHashMap;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -46,7 +50,6 @@
 import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.utils.CLibrary;
-import org.apache.cassandra.utils.PureJavaCrc32;
 import org.apache.cassandra.utils.concurrent.OpOrder;
 import org.apache.cassandra.utils.concurrent.WaitQueue;
 
@@ -55,7 +58,7 @@
  * as well as tracking the last mutation position of any "dirty" CFs covered by the segment file. Segment
  * files are initially allocated to a fixed size and can grow to accomidate a larger value if necessary.
  */
-public class CommitLogSegment
+public abstract class CommitLogSegment
 {
     private static final Logger logger = LoggerFactory.getLogger(CommitLogSegment.class);
 
@@ -86,12 +89,13 @@
 
     // Everything before this offset has been synced and written.  The SYNC_MARKER_SIZE bytes after
     // each sync are reserved, and point forwards to the next such offset.  The final
-    // sync marker in a segment will be zeroed out, or point to EOF.
+    // sync marker in a segment will be zeroed out, or point to a position too close to the EOF to fit a marker.
     private volatile int lastSyncedOffset;
 
-    // the amount of the tail of the file we have allocated but not used - this is used when we discard a log segment
-    // to ensure nobody writes to it after we've decided we're done with it
-    private int discardedTailFrom;
+    // The end position of the buffer. Initially set to its capacity and updated to point to the last written position
+    // as the segment is being closed.
+    // No need to be volatile as writes are protected by appendOrder barrier.
+    private int endOfBuffer;
 
     // a signal for writers to wait on to confirm the log message they provided has been written to disk
     private final WaitQueue syncComplete = new WaitQueue();
@@ -104,20 +108,19 @@
 
     public final long id;
 
-    private final File logFile;
-    private final RandomAccessFile logFileAccessor;
-    private final int fd;
+    final File logFile;
+    final FileChannel channel;
+    final int fd;
 
-    private final MappedByteBuffer buffer;
+    ByteBuffer buffer;
 
+    final CommitLog commitLog;
     public final CommitLogDescriptor descriptor;
 
-    /**
-     * @return a newly minted segment file
-     */
-    static CommitLogSegment freshSegment()
+    static CommitLogSegment createSegment(CommitLog commitLog)
     {
-        return new CommitLogSegment(null);
+        return commitLog.configuration.useCompression() ? new CompressedSegment(commitLog)
+                                                        : new MemoryMappedSegment(commitLog);
     }
 
     static long getNextId()
@@ -130,59 +133,38 @@
      *
      * @param filePath  if not null, recycles the existing file by renaming it and truncating it to CommitLog.SEGMENT_SIZE.
      */
-    CommitLogSegment(String filePath)
+    CommitLogSegment(CommitLog commitLog)
     {
+        this.commitLog = commitLog;
         id = getNextId();
-        descriptor = new CommitLogDescriptor(id);
-        logFile = new File(DatabaseDescriptor.getCommitLogLocation(), descriptor.fileName());
-        boolean isCreating = true;
+        descriptor = new CommitLogDescriptor(id, commitLog.configuration.getCompressorClass());
+        logFile = new File(commitLog.location, descriptor.fileName());
 
         try
         {
-            if (filePath != null)
-            {
-                File oldFile = new File(filePath);
-
-                if (oldFile.exists())
-                {
-                    logger.debug("Re-using discarded CommitLog segment for {} from {}", id, filePath);
-                    if (!oldFile.renameTo(logFile))
-                        throw new IOException("Rename from " + filePath + " to " + id + " failed");
-                    isCreating = false;
-                }
-            }
-
-            // Open the initial the segment file
-            logFileAccessor = new RandomAccessFile(logFile, "rw");
-
-            if (isCreating)
-                logger.debug("Creating new commit log segment {}", logFile.getPath());
-
-            // Map the segment, extending or truncating it to the standard segment size.
-            // (We may have restarted after a segment size configuration change, leaving "incorrectly"
-            // sized segments on disk.)
-            logFileAccessor.setLength(DatabaseDescriptor.getCommitLogSegmentSize());
-            fd = CLibrary.getfd(logFileAccessor.getFD());
-
-            buffer = logFileAccessor.getChannel().map(FileChannel.MapMode.READ_WRITE, 0, DatabaseDescriptor.getCommitLogSegmentSize());
-            // write the header
-            CommitLogDescriptor.writeHeader(buffer, descriptor);
-            // mark the initial sync marker as uninitialised
-            buffer.putInt(CommitLogDescriptor.HEADER_SIZE, 0);
-            buffer.putLong(CommitLogDescriptor.HEADER_SIZE + 4, 0);
-            allocatePosition.set(CommitLogDescriptor.HEADER_SIZE + SYNC_MARKER_SIZE);
-            lastSyncedOffset = CommitLogDescriptor.HEADER_SIZE;
+            channel = FileChannel.open(logFile.toPath(), StandardOpenOption.WRITE, StandardOpenOption.READ, StandardOpenOption.CREATE);
+            fd = CLibrary.getfd(channel);
         }
         catch (IOException e)
         {
             throw new FSWriteError(e, logFile);
         }
+        
+        buffer = createBuffer(commitLog);
+        // write the header
+        CommitLogDescriptor.writeHeader(buffer, descriptor);
+        endOfBuffer = buffer.capacity();
+        lastSyncedOffset = buffer.position();
+        allocatePosition.set(lastSyncedOffset + SYNC_MARKER_SIZE);
     }
 
+    abstract ByteBuffer createBuffer(CommitLog commitLog);
+
     /**
      * Allocate space in this buffer for the provided mutation, and return the allocated Allocation object.
      * Returns null if there is not enough space in this segment, and a new segment is needed.
      */
+    @SuppressWarnings("resource") //we pass the op order around
     Allocation allocate(Mutation mutation, int size)
     {
         final OpOrder.Group opGroup = appendOrder.start();
@@ -224,32 +206,42 @@
         {
             int prev = allocatePosition.get();
             int next = prev + size;
-            if (next >= buffer.capacity())
+            if (next >= endOfBuffer)
                 return -1;
             if (allocatePosition.compareAndSet(prev, next))
+            {
+                assert buffer != null;
                 return prev;
+            }
         }
     }
 
     // ensures no more of this segment is writeable, by allocating any unused section at the end and marking it discarded
     void discardUnusedTail()
     {
-        // we guard this with the OpOrdering instead of synchronised due to potential dead-lock with CLSM.advanceAllocatingFrom()
-        // this actually isn't strictly necessary, as currently all calls to discardUnusedTail occur within a block
-        // already protected by this OpOrdering, but to prevent future potential mistakes, we duplicate the protection here
-        // so that the contract between discardUnusedTail() and sync() is more explicit.
+        // We guard this with the OpOrdering instead of synchronised due to potential dead-lock with CLSM.advanceAllocatingFrom()
+        // Ensures endOfBuffer update is reflected in the buffer end position picked up by sync().
+        // This actually isn't strictly necessary, as currently all calls to discardUnusedTail are executed either by the thread
+        // running sync or within a mutation already protected by this OpOrdering, but to prevent future potential mistakes,
+        // we duplicate the protection here so that the contract between discardUnusedTail() and sync() is more explicit.
         try (OpOrder.Group group = appendOrder.start())
         {
             while (true)
             {
                 int prev = allocatePosition.get();
-                // we set allocatePosition past buffer.capacity() to make sure we always set discardedTailFrom
-                int next = buffer.capacity() + 1;
-                if (prev == next)
+
+                int next = endOfBuffer + 1;
+                if (prev >= next)
+                {
+                    // Already stopped allocating, might also be closed.
+                    assert buffer == null || prev == buffer.capacity() + 1;
                     return;
+                }
                 if (allocatePosition.compareAndSet(prev, next))
                 {
-                    discardedTailFrom = prev;
+                    // Stopped allocating now. Can only succeed once, no further allocation or discardUnusedTail can succeed.
+                    endOfBuffer = prev;
+                    assert buffer != null && next == buffer.capacity() + 1;
                     return;
                 }
             }
@@ -270,113 +262,72 @@
      */
     synchronized void sync()
     {
-        try
+        boolean close = false;
+        // check we have more work to do
+        if (allocatePosition.get() <= lastSyncedOffset + SYNC_MARKER_SIZE)
+            return;
+        // Note: Even if the very first allocation of this sync section failed, we still want to enter this
+        // to ensure the segment is closed. As allocatePosition is set to 1 beyond the capacity of the buffer,
+        // this will always be entered when a mutation allocation has been attempted after the marker allocation
+        // succeeded in the previous sync. 
+        assert buffer != null;  // Only close once.
+
+        int startMarker = lastSyncedOffset;
+        // Allocate a new sync marker; this is both necessary in itself, but also serves to demarcate
+        // the point at which we can safely consider records to have been completely written to.
+        int nextMarker = allocate(SYNC_MARKER_SIZE);
+        if (nextMarker < 0)
         {
-            // check we have more work to do
-            if (allocatePosition.get() <= lastSyncedOffset + SYNC_MARKER_SIZE)
-                return;
+            // Ensure no more of this CLS is writeable, and mark ourselves for closing.
+            discardUnusedTail();
+            close = true;
 
-            // allocate a new sync marker; this is both necessary in itself, but also serves to demarcate
-            // the point at which we can safely consider records to have been completely written to
-            int nextMarker;
-            nextMarker = allocate(SYNC_MARKER_SIZE);
-            boolean close = false;
-            if (nextMarker < 0)
-            {
-                // ensure no more of this CLS is writeable, and mark ourselves for closing
-                discardUnusedTail();
-                close = true;
-
-                // wait for modifications guards both discardedTailFrom, and any outstanding appends
-                waitForModifications();
-
-                if (discardedTailFrom < buffer.capacity() - SYNC_MARKER_SIZE)
-                {
-                    // if there's room in the discard section to write an empty header, use that as the nextMarker
-                    nextMarker = discardedTailFrom;
-                }
-                else
-                {
-                    // not enough space left in the buffer, so mark the next sync marker as the EOF position
-                    nextMarker = buffer.capacity();
-                }
-            }
-            else
-            {
-                waitForModifications();
-            }
-
-            assert nextMarker > lastSyncedOffset;
-
-            // write previous sync marker to point to next sync marker
-            // we don't chain the crcs here to ensure this method is idempotent if it fails
-            int offset = lastSyncedOffset;
-            final PureJavaCrc32 crc = new PureJavaCrc32();
-            crc.updateInt((int) (id & 0xFFFFFFFFL));
-            crc.updateInt((int) (id >>> 32));
-            crc.updateInt(offset);
-            buffer.putInt(offset, nextMarker);
-            buffer.putInt(offset + 4, crc.getCrc());
-
-            // zero out the next sync marker so replayer can cleanly exit
-            if (nextMarker < buffer.capacity())
-            {
-                buffer.putInt(nextMarker, 0);
-                buffer.putInt(nextMarker + 4, 0);
-            }
-
-            // actually perform the sync and signal those waiting for it
-            buffer.force();
-
-            if (close)
-                nextMarker = buffer.capacity();
-
-            lastSyncedOffset = nextMarker;
-            syncComplete.signalAll();
-
-            CLibrary.trySkipCache(fd, offset, nextMarker);
-            if (close)
-                internalClose();
+            // We use the buffer size as the synced position after a close instead of the end of the actual data
+            // to make sure we only close the buffer once.
+            // The endOfBuffer position may be incorrect at this point (to be written by another stalled thread).
+            nextMarker = buffer.capacity();
         }
-        catch (Exception e) // MappedByteBuffer.force() does not declare IOException but can actually throw it
-        {
-            throw new FSWriteError(e, getPath());
-        }
+
+        // Wait for mutations to complete as well as endOfBuffer to have been written.
+        waitForModifications();
+        int sectionEnd = close ? endOfBuffer : nextMarker;
+
+        // Perform compression, writing to file and flush.
+        write(startMarker, sectionEnd);
+
+        // Signal the sync as complete.
+        lastSyncedOffset = nextMarker;
+        if (close)
+            internalClose();
+        syncComplete.signalAll();
     }
 
+    protected void writeSyncMarker(ByteBuffer buffer, int offset, int filePos, int nextMarker)
+    {
+        ICRC32 crc = CRC32Factory.instance.create();
+        crc.updateInt((int) (id & 0xFFFFFFFFL));
+        crc.updateInt((int) (id >>> 32));
+        crc.updateInt(filePos);
+        buffer.putInt(offset, nextMarker);
+        buffer.putInt(offset + 4, crc.getCrc());
+    }
+
+    abstract void write(int lastSyncedOffset, int nextMarker);
+
     public boolean isStillAllocating()
     {
-        return allocatePosition.get() < buffer.capacity();
+        return allocatePosition.get() < endOfBuffer;
     }
 
     /**
      * Completely discards a segment file by deleting it. (Potentially blocking operation)
      */
-    void delete()
+    void discard(boolean deleteFile)
     {
-       FileUtils.deleteWithConfirm(logFile);
-    }
-
-    /**
-     * Recycle processes an unneeded segment file for reuse.
-     *
-     * @return a new CommitLogSegment representing the newly reusable segment.
-     */
-    CommitLogSegment recycle()
-    {
-        try
-        {
-            sync();
-        }
-        catch (FSWriteError e)
-        {
-            logger.error("I/O error flushing {} {}", this, e.getMessage());
-            throw e;
-        }
-
         close();
-
-        return new CommitLogSegment(getPath());
+        if (deleteFile)
+            FileUtils.deleteWithConfirm(logFile);
+        commitLog.allocator.addSize(-onDiskSize());
     }
 
     /**
@@ -408,7 +359,7 @@
         while (true)
         {
             WaitQueue.Signal signal = syncComplete.register();
-            if (lastSyncedOffset < buffer.capacity())
+            if (lastSyncedOffset < endOfBuffer)
             {
                 signal.awaitUninterruptibly();
             }
@@ -420,25 +371,39 @@
         }
     }
 
+    void waitForSync(int position, Timer waitingOnCommit)
+    {
+        while (lastSyncedOffset < position)
+        {
+            WaitQueue.Signal signal = waitingOnCommit != null ?
+                                      syncComplete.register(waitingOnCommit.time()) :
+                                      syncComplete.register();
+            if (lastSyncedOffset < position)
+                signal.awaitUninterruptibly();
+            else
+                signal.cancel();
+        }
+    }
+
     /**
-     * Close the segment file.
+     * Stop writing to this file, sync and close it. Does nothing if the file is already closed.
      */
     synchronized void close()
     {
         discardUnusedTail();
-        waitForModifications();
-        lastSyncedOffset = buffer.capacity();
-        syncComplete.signalAll();
-        internalClose();
+        sync();
+        assert buffer == null;
     }
 
-    void internalClose()
+    /**
+     * Close the segment file. Do not call from outside this class, use syncAndClose() instead.
+     */
+    protected void internalClose()
     {
         try
         {
-            if (FileUtils.isCleanerAvailable())
-                FileUtils.clean(buffer);
-            logFileAccessor.close();
+            channel.close();
+            buffer = null;
         }
         catch (IOException e)
         {
@@ -453,7 +418,7 @@
             // check for deleted CFS
             CFMetaData cfm = columnFamily.metadata();
             if (cfm.isPurged())
-                logger.error("Attempted to write commit log entry for unrecognized column family: {}", columnFamily.id());
+                logger.error("Attempted to write commit log entry for unrecognized table: {}", columnFamily.id());
             else
                 ensureAtleast(cfDirty, cfm.cfId, allocatedPosition);
         }
@@ -580,6 +545,13 @@
         return sb.toString();
     }
 
+    abstract public long onDiskSize();
+
+    public long contentSize()
+    {
+        return lastSyncedOffset;
+    }
+
     @Override
     public String toString()
     {
@@ -634,16 +606,9 @@
             appendOp.close();
         }
 
-        void awaitDiskSync()
+        void awaitDiskSync(Timer waitingOnCommit)
         {
-            while (segment.lastSyncedOffset < position)
-            {
-                WaitQueue.Signal signal = segment.syncComplete.register(CommitLog.instance.metrics.waitingOnCommit.time());
-                if (segment.lastSyncedOffset < position)
-                    signal.awaitUninterruptibly();
-                else
-                    signal.cancel();
-            }
+            segment.waitForSync(position, waitingOnCommit);
         }
 
         public ReplayPosition getReplayPosition()
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
index 6838de6..8670fd7 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
@@ -28,7 +28,6 @@
 import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.Future;
 import java.util.concurrent.LinkedBlockingQueue;
@@ -38,6 +37,7 @@
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Iterables;
 import com.google.common.util.concurrent.*;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -47,7 +47,6 @@
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.concurrent.WaitQueue;
 import org.apache.cassandra.utils.JVMStabilityInspector;
@@ -64,10 +63,9 @@
     static final Logger logger = LoggerFactory.getLogger(CommitLogSegmentManager.class);
 
     /**
-     * Queue of work to be done by the manager thread.  This is usually a recycle operation, which returns
-     * a CommitLogSegment, or a delete operation, which returns null.
+     * Queue of work to be done by the manager thread, also used to wake the thread to perform segment allocation.
      */
-    private final BlockingQueue<Callable<CommitLogSegment>> segmentManagementTasks = new LinkedBlockingQueue<>();
+    private final BlockingQueue<Runnable> segmentManagementTasks = new LinkedBlockingQueue<>();
 
     /** Segments that are ready to be used. Head of the queue is the one we allocate writes to */
     private final ConcurrentLinkedQueue<CommitLogSegment> availableSegments = new ConcurrentLinkedQueue<>();
@@ -92,12 +90,18 @@
      * New segment creation is initially disabled because we'll typically get some "free" segments
      * recycled after log replay.
      */
-    private volatile boolean createReserveSegments = false;
+    volatile boolean createReserveSegments = false;
 
-    private final Thread managerThread;
+    private Thread managerThread;
     private volatile boolean run = true;
+    private final CommitLog commitLog;
 
-    public CommitLogSegmentManager()
+    CommitLogSegmentManager(final CommitLog commitLog)
+    {
+        this.commitLog = commitLog;
+    }
+
+    void start()
     {
         // The run loop for the manager thread
         Runnable runnable = new WrappedRunnable()
@@ -108,16 +112,15 @@
                 {
                     try
                     {
-                        Callable<CommitLogSegment> task = segmentManagementTasks.poll();
+                        Runnable task = segmentManagementTasks.poll();
                         if (task == null)
                         {
                             // if we have no more work to do, check if we should create a new segment
                             if (availableSegments.isEmpty() && (activeSegments.isEmpty() || createReserveSegments))
                             {
-                                logger.debug("No segments in reserve; creating a fresh one");
-                                size.addAndGet(DatabaseDescriptor.getCommitLogSegmentSize());
+                                logger.trace("No segments in reserve; creating a fresh one");
                                 // TODO : some error handling in case we fail to create a new segment
-                                availableSegments.add(CommitLogSegment.freshSegment());
+                                availableSegments.add(CommitLogSegment.createSegment(commitLog));
                                 hasAvailableSegments.signalAll();
                             }
 
@@ -150,13 +153,7 @@
                             }
                         }
 
-                        CommitLogSegment recycled = task.call();
-                        if (recycled != null)
-                        {
-                            // if the work resulted in a segment to recycle, publish it
-                            availableSegments.add(recycled);
-                            hasAvailableSegments.signalAll();
-                        }
+                        task.run();
                     }
                     catch (Throwable t)
                     {
@@ -170,6 +167,8 @@
             }
         };
 
+        run = true;
+
         managerThread = new Thread(runnable, "COMMIT-LOG-ALLOCATOR");
         managerThread.start();
     }
@@ -235,19 +234,19 @@
                 {
                     // Now we can run the user defined command just after switching to the new commit log.
                     // (Do this here instead of in the recycle call so we can get a head start on the archive.)
-                    CommitLog.instance.archiver.maybeArchive(old);
+                    commitLog.archiver.maybeArchive(old);
 
                     // ensure we don't continue to use the old file; not strictly necessary, but cleaner to enforce it
                     old.discardUnusedTail();
                 }
 
                 // request that the CL be synced out-of-band, as we've finished a segment
-                CommitLog.instance.requestExtraSync();
+                commitLog.requestExtraSync();
                 return;
             }
 
             // no more segments, so register to receive a signal when not empty
-            WaitQueue.Signal signal = hasAvailableSegments.register(CommitLog.instance.metrics.waitingOnSegmentAllocation.time());
+            WaitQueue.Signal signal = hasAvailableSegments.register(commitLog.metrics.waitingOnSegmentAllocation.time());
 
             // trigger the management thread; this must occur after registering
             // the signal to ensure we are woken by any new segment creation
@@ -276,13 +275,7 @@
     private void wakeManager()
     {
         // put a NO-OP on the queue, to trigger management thread (and create a new segment if necessary)
-        segmentManagementTasks.add(new Callable<CommitLogSegment>()
-        {
-            public CommitLogSegment call()
-            {
-                return null;
-            }
-        });
+        segmentManagementTasks.add(Runnables.doNothing());
     }
 
     /**
@@ -347,32 +340,16 @@
      */
     void recycleSegment(final CommitLogSegment segment)
     {
-        boolean archiveSuccess = CommitLog.instance.archiver.maybeWaitForArchiving(segment.getName());
-        if (!activeSegments.remove(segment))
-        {
-            logger.warn("segment {} not found in activeSegments queue", segment);
-            return;
-        }
-        if (!archiveSuccess)
+        boolean archiveSuccess = commitLog.archiver.maybeWaitForArchiving(segment.getName());
+        if (activeSegments.remove(segment))
         {
             // if archiving (command) was not successful then leave the file alone. don't delete or recycle.
-            discardSegment(segment, false);
-            return;
+            discardSegment(segment, archiveSuccess);
         }
-        if (isCapExceeded() || !DatabaseDescriptor.getCommitLogSegmentRecyclingEnabled())
+        else
         {
-            discardSegment(segment, true);
-            return;
+            logger.warn("segment {} not found in activeSegments queue", segment);
         }
-
-        logger.debug("Recycling {}", segment);
-        segmentManagementTasks.add(new Callable<CommitLogSegment>()
-        {
-            public CommitLogSegment call()
-            {
-                return segment.recycle();
-            }
-        });
     }
 
     /**
@@ -383,26 +360,9 @@
      */
     void recycleSegment(final File file)
     {
-        if (isCapExceeded()
-            || CommitLogDescriptor.fromFileName(file.getName()).getMessagingVersion() != MessagingService.current_version
-            || !DatabaseDescriptor.getCommitLogSegmentRecyclingEnabled())
-        {
-            // (don't decrease managed size, since this was never a "live" segment)
-            logger.debug("(Unopened) segment {} is no longer needed and will be deleted now", file);
-            FileUtils.deleteWithConfirm(file);
-            return;
-        }
-
-        logger.debug("Recycling {}", file);
-        // this wasn't previously a live segment, so add it to the managed size when we make it live
-        size.addAndGet(DatabaseDescriptor.getCommitLogSegmentSize());
-        segmentManagementTasks.add(new Callable<CommitLogSegment>()
-        {
-            public CommitLogSegment call()
-            {
-                return new CommitLogSegment(file.getPath());
-            }
-        });
+        // (don't decrease managed size, since this was never a "live" segment)
+        logger.trace("(Unopened) segment {} is no longer needed and will be deleted now", file);
+        FileUtils.deleteWithConfirm(file);
     }
 
     /**
@@ -412,29 +372,42 @@
      */
     private void discardSegment(final CommitLogSegment segment, final boolean deleteFile)
     {
-        logger.debug("Segment {} is no longer active and will be deleted {}", segment, deleteFile ? "now" : "by the archive script");
-        size.addAndGet(-DatabaseDescriptor.getCommitLogSegmentSize());
+        logger.trace("Segment {} is no longer active and will be deleted {}", segment, deleteFile ? "now" : "by the archive script");
 
-        segmentManagementTasks.add(new Callable<CommitLogSegment>()
+        segmentManagementTasks.add(new Runnable()
         {
-            public CommitLogSegment call()
+            public void run()
             {
-                segment.close();
-                if (deleteFile)
-                    segment.delete();
-                return null;
+                segment.discard(deleteFile);
             }
         });
     }
 
     /**
+     * Adjust the tracked on-disk size. Called by individual segments to reflect writes, allocations and discards.
+     * @param addedSize
+     */
+    void addSize(long addedSize)
+    {
+        size.addAndGet(addedSize);
+    }
+
+    /**
      * @return the space (in bytes) used by all segment files.
      */
-    public long bytesUsed()
+    public long onDiskSize()
     {
         return size.get();
     }
 
+    private long unusedCapacity()
+    {
+        long total = DatabaseDescriptor.getTotalCommitlogSpaceInMB() * 1024 * 1024;
+        long currentSize = size.get();
+        logger.trace("Total active commitlog segment space used is {} out of {}", currentSize, total);
+        return total - currentSize;
+    }
+
     /**
      * @param name the filename to check
      * @return true if file is managed by this manager.
@@ -448,27 +421,10 @@
     }
 
     /**
-     * Check to see if the speculative current size exceeds the cap.
-     *
-     * @return true if cap is exceeded
-     */
-    private boolean isCapExceeded()
-    {
-        return unusedCapacity() < 0;
-    }
-
-    private long unusedCapacity()
-    {
-        long currentSize = size.get();
-        logger.debug("Total active commitlog segment space used is {}", currentSize);
-        return DatabaseDescriptor.getTotalCommitlogSpaceInMB() * 1024 * 1024 - currentSize;
-    }
-
-    /**
      * Throws a flag that enables the behavior of keeping at least one spare segment
      * available at all times.
      */
-    public void enableReserveSegmentCreation()
+    void enableReserveSegmentCreation()
     {
         createReserveSegments = true;
         wakeManager();
@@ -497,7 +453,7 @@
                 {
                     // even though we remove the schema entry before a final flush when dropping a CF,
                     // it's still possible for a writer to race and finish his append after the flush.
-                    logger.debug("Marking clean CF {} that doesn't exist anymore", dirtyCFId);
+                    logger.trace("Marking clean CF {} that doesn't exist anymore", dirtyCFId);
                     segment.markClean(dirtyCFId, segment.getContext());
                 }
                 else if (!flushes.containsKey(dirtyCFId))
@@ -515,24 +471,67 @@
     }
 
     /**
-     * Resets all the segments, for testing purposes. DO NOT USE THIS OUTSIDE OF TESTS.
+     * Stops CL, for testing purposes. DO NOT USE THIS OUTSIDE OF TESTS.
+     * Only call this after the AbstractCommitLogService is shut down.
      */
-    public void resetUnsafe()
+    public void stopUnsafe(boolean deleteSegments)
     {
-        logger.debug("Closing and clearing existing commit log segments...");
+        logger.trace("CLSM closing and clearing existing commit log segments...");
+        createReserveSegments = false;
 
-        while (!segmentManagementTasks.isEmpty())
-            Thread.yield();
+        awaitManagementTasksCompletion();
 
-        for (CommitLogSegment segment : activeSegments)
-            segment.close();
-        activeSegments.clear();
+        shutdown();
+        try
+        {
+            awaitTermination();
+        }
+        catch (InterruptedException e)
+        {
+            throw new RuntimeException(e);
+        }
 
-        for (CommitLogSegment segment : availableSegments)
-            segment.close();
-        availableSegments.clear();
+        synchronized (this)
+        {
+            for (CommitLogSegment segment : activeSegments)
+                closeAndDeleteSegmentUnsafe(segment, deleteSegments);
+            activeSegments.clear();
+
+            for (CommitLogSegment segment : availableSegments)
+                closeAndDeleteSegmentUnsafe(segment, deleteSegments);
+            availableSegments.clear();
+        }
 
         allocatingFrom = null;
+
+        segmentManagementTasks.clear();
+
+        size.set(0L);
+
+        logger.trace("CLSM done with closing and clearing existing commit log segments.");
+    }
+
+    // Used by tests only.
+    void awaitManagementTasksCompletion()
+    {
+        while (!segmentManagementTasks.isEmpty())
+            Thread.yield();
+        // The last management task is not yet complete. Wait a while for it.
+        Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
+        // TODO: If this functionality is required by anything other than tests, signalling must be used to ensure
+        // waiting completes correctly.
+    }
+
+    private static void closeAndDeleteSegmentUnsafe(CommitLogSegment segment, boolean delete)
+    {
+        try
+        {
+            segment.discard(delete);
+        }
+        catch (AssertionError ignored)
+        {
+            // segment file does not exist
+        }
     }
 
     /**
@@ -541,7 +540,7 @@
     public void shutdown()
     {
         run = false;
-        segmentManagementTasks.add(Callables.<CommitLogSegment>returning(null));
+        wakeManager();
     }
 
     /**
@@ -550,6 +549,14 @@
     public void awaitTermination() throws InterruptedException
     {
         managerThread.join();
+
+        for (CommitLogSegment segment : activeSegments)
+            segment.close();
+
+        for (CommitLogSegment segment : availableSegments)
+            segment.close();
+
+        CompressedSegment.shutdown();
     }
 
     /**
diff --git a/src/java/org/apache/cassandra/db/commitlog/CompressedSegment.java b/src/java/org/apache/cassandra/db/commitlog/CompressedSegment.java
new file mode 100644
index 0000000..219709b
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/commitlog/CompressedSegment.java
@@ -0,0 +1,161 @@
+/*
+ * 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.db.commitlog;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Queue;
+import java.util.concurrent.ConcurrentLinkedQueue;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.io.compress.BufferType;
+import org.apache.cassandra.io.compress.ICompressor;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.SyncUtil;
+
+/*
+ * Compressed commit log segment. Provides an in-memory buffer for the mutation threads. On sync compresses the written
+ * section of the buffer and writes it to the destination channel.
+ */
+public class CompressedSegment extends CommitLogSegment
+{
+    static private final ThreadLocal<ByteBuffer> compressedBufferHolder = new ThreadLocal<ByteBuffer>() {
+        protected ByteBuffer initialValue()
+        {
+            return ByteBuffer.allocate(0);
+        }
+    };
+    static Queue<ByteBuffer> bufferPool = new ConcurrentLinkedQueue<>();
+
+    /**
+     * Maximum number of buffers in the compression pool. The default value is 3, it should not be set lower than that
+     * (one segment in compression, one written to, one in reserve); delays in compression may cause the log to use
+     * more, depending on how soon the sync policy stops all writing threads.
+     */
+    static final int MAX_BUFFERPOOL_SIZE = DatabaseDescriptor.getCommitLogMaxCompressionBuffersInPool();
+
+    static final int COMPRESSED_MARKER_SIZE = SYNC_MARKER_SIZE + 4;
+    final ICompressor compressor;
+
+    volatile long lastWrittenPos = 0;
+
+    /**
+     * Constructs a new segment file.
+     */
+    CompressedSegment(CommitLog commitLog)
+    {
+        super(commitLog);
+        this.compressor = commitLog.configuration.getCompressor();
+        try
+        {
+            channel.write((ByteBuffer) buffer.duplicate().flip());
+            commitLog.allocator.addSize(lastWrittenPos = buffer.position());
+        }
+        catch (IOException e)
+        {
+            throw new FSWriteError(e, getPath());
+        }
+    }
+
+    ByteBuffer allocate(int size)
+    {
+        return compressor.preferredBufferType().allocate(size);
+    }
+
+    ByteBuffer createBuffer(CommitLog commitLog)
+    {
+        ByteBuffer buf = bufferPool.poll();
+        if (buf == null)
+        {
+            // this.compressor is not yet set, so we must use the commitLog's one.
+            buf = commitLog.configuration.getCompressor()
+                                         .preferredBufferType()
+                                         .allocate(DatabaseDescriptor.getCommitLogSegmentSize());
+        } else
+            buf.clear();
+        return buf;
+    }
+
+    static long startMillis = System.currentTimeMillis();
+
+    @Override
+    void write(int startMarker, int nextMarker)
+    {
+        int contentStart = startMarker + SYNC_MARKER_SIZE;
+        int length = nextMarker - contentStart;
+        // The length may be 0 when the segment is being closed.
+        assert length > 0 || length == 0 && !isStillAllocating();
+
+        try
+        {
+            int neededBufferSize = compressor.initialCompressedBufferLength(length) + COMPRESSED_MARKER_SIZE;
+            ByteBuffer compressedBuffer = compressedBufferHolder.get();
+            if (compressor.preferredBufferType() != BufferType.typeOf(compressedBuffer) ||
+                compressedBuffer.capacity() < neededBufferSize)
+            {
+                FileUtils.clean(compressedBuffer);
+                compressedBuffer = allocate(neededBufferSize);
+                compressedBufferHolder.set(compressedBuffer);
+            }
+
+            ByteBuffer inputBuffer = buffer.duplicate();
+            inputBuffer.limit(contentStart + length).position(contentStart);
+            compressedBuffer.limit(compressedBuffer.capacity()).position(COMPRESSED_MARKER_SIZE);
+            compressor.compress(inputBuffer, compressedBuffer);
+
+            compressedBuffer.flip();
+            compressedBuffer.putInt(SYNC_MARKER_SIZE, length);
+
+            // Only one thread can be here at a given time.
+            // Protected by synchronization on CommitLogSegment.sync().
+            writeSyncMarker(compressedBuffer, 0, (int) channel.position(), (int) channel.position() + compressedBuffer.remaining());
+            commitLog.allocator.addSize(compressedBuffer.limit());
+            channel.write(compressedBuffer);
+            assert channel.position() - lastWrittenPos == compressedBuffer.limit();
+            lastWrittenPos = channel.position();
+            SyncUtil.force(channel, true);
+        }
+        catch (Exception e)
+        {
+            throw new FSWriteError(e, getPath());
+        }
+    }
+
+    @Override
+    protected void internalClose()
+    {
+        if (bufferPool.size() < MAX_BUFFERPOOL_SIZE)
+            bufferPool.add(buffer);
+        else
+            FileUtils.clean(buffer);
+
+        super.internalClose();
+    }
+
+    static void shutdown()
+    {
+        bufferPool.clear();
+    }
+
+    @Override
+    public long onDiskSize()
+    {
+        return lastWrittenPos;
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/commitlog/MemoryMappedSegment.java b/src/java/org/apache/cassandra/db/commitlog/MemoryMappedSegment.java
new file mode 100644
index 0000000..fa9ef37
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/commitlog/MemoryMappedSegment.java
@@ -0,0 +1,105 @@
+/*
+ * 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.db.commitlog;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.MappedByteBuffer;
+import java.nio.channels.FileChannel;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.CLibrary;
+import org.apache.cassandra.utils.SyncUtil;
+
+/*
+ * Memory-mapped segment. Maps the destination channel into an appropriately-sized memory-mapped buffer in which the
+ * mutation threads write. On sync forces the buffer to disk.
+ * If possible, recycles used segment files to avoid reallocating large chunks of disk.
+ */
+public class MemoryMappedSegment extends CommitLogSegment
+{
+    /**
+     * Constructs a new segment file.
+     *
+     * @param filePath  if not null, recycles the existing file by renaming it and truncating it to CommitLog.SEGMENT_SIZE.
+     * @param commitLog the commit log it will be used with.
+     */
+    MemoryMappedSegment(CommitLog commitLog)
+    {
+        super(commitLog);
+        // mark the initial sync marker as uninitialised
+        int firstSync = buffer.position();
+        buffer.putInt(firstSync + 0, 0);
+        buffer.putInt(firstSync + 4, 0);
+    }
+
+    ByteBuffer createBuffer(CommitLog commitLog)
+    {
+        try
+        {
+            MappedByteBuffer mappedFile = channel.map(FileChannel.MapMode.READ_WRITE, 0, DatabaseDescriptor.getCommitLogSegmentSize());
+            commitLog.allocator.addSize(DatabaseDescriptor.getCommitLogSegmentSize());
+            return mappedFile;
+        }
+        catch (IOException e)
+        {
+            throw new FSWriteError(e, logFile);
+        }
+    }
+
+    @Override
+    void write(int startMarker, int nextMarker)
+    {
+        // if there's room in the discard section to write an empty header,
+        // zero out the next sync marker so replayer can cleanly exit
+        if (nextMarker <= buffer.capacity() - SYNC_MARKER_SIZE)
+        {
+            buffer.putInt(nextMarker, 0);
+            buffer.putInt(nextMarker + 4, 0);
+        }
+
+        // write previous sync marker to point to next sync marker
+        // we don't chain the crcs here to ensure this method is idempotent if it fails
+        writeSyncMarker(buffer, startMarker, startMarker, nextMarker);
+
+        try {
+            SyncUtil.force((MappedByteBuffer) buffer);
+        }
+        catch (Exception e) // MappedByteBuffer.force() does not declare IOException but can actually throw it
+        {
+            throw new FSWriteError(e, getPath());
+        }
+        CLibrary.trySkipCache(fd, startMarker, nextMarker);
+    }
+
+    @Override
+    public long onDiskSize()
+    {
+        return DatabaseDescriptor.getCommitLogSegmentSize();
+    }
+
+    @Override
+    protected void internalClose()
+    {
+        if (FileUtils.isCleanerAvailable())
+            FileUtils.clean(buffer);
+        super.internalClose();
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/commitlog/PeriodicCommitLogService.java b/src/java/org/apache/cassandra/db/commitlog/PeriodicCommitLogService.java
index 14bb367..86a248b 100644
--- a/src/java/org/apache/cassandra/db/commitlog/PeriodicCommitLogService.java
+++ b/src/java/org/apache/cassandra/db/commitlog/PeriodicCommitLogService.java
@@ -22,7 +22,6 @@
 
 class PeriodicCommitLogService extends AbstractCommitLogService
 {
-
     private static final int blockWhenSyncLagsMillis = (int) (DatabaseDescriptor.getCommitLogSyncPeriod() * 1.5);
 
     public PeriodicCommitLogService(final CommitLog commitLog)
@@ -39,7 +38,7 @@
             pending.incrementAndGet();
             while (waitForSyncToCatchUp(started))
             {
-                WaitQueue.Signal signal = syncComplete.register(CommitLog.instance.metrics.waitingOnCommit.time());
+                WaitQueue.Signal signal = syncComplete.register(commitLog.metrics.waitingOnCommit.time());
                 if (waitForSyncToCatchUp(started))
                     signal.awaitUninterruptibly();
                 else
diff --git a/src/java/org/apache/cassandra/db/commitlog/ReplayPosition.java b/src/java/org/apache/cassandra/db/commitlog/ReplayPosition.java
index 31fc28e..17802ad 100644
--- a/src/java/org/apache/cassandra/db/commitlog/ReplayPosition.java
+++ b/src/java/org/apache/cassandra/db/commitlog/ReplayPosition.java
@@ -19,15 +19,15 @@
 
 import java.io.DataInput;
 import java.io.IOException;
-import java.util.Comparator;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.TreeMap;
 
-import com.google.common.base.Function;
-import com.google.common.collect.Iterables;
 import com.google.common.collect.Ordering;
 
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.ISerializer;
-import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.util.DataOutputPlus;
 
 public class ReplayPosition implements Comparable<ReplayPosition>
@@ -35,46 +35,78 @@
     public static final ReplayPositionSerializer serializer = new ReplayPositionSerializer();
 
     // NONE is used for SSTables that are streamed from other nodes and thus have no relationship
-    // with our local commitlog. The values satisfy the critera that
+    // with our local commitlog. The values satisfy the criteria that
     //  - no real commitlog segment will have the given id
     //  - it will sort before any real replayposition, so it will be effectively ignored by getReplayPosition
     public static final ReplayPosition NONE = new ReplayPosition(-1, 0);
 
-    /**
-     * Convenience method to compute the replay position for a group of SSTables.
-     * @param sstables
-     * @return the most recent (highest) replay position
-     */
-    public static ReplayPosition getReplayPosition(Iterable<? extends SSTableReader> sstables)
-    {
-        if (Iterables.isEmpty(sstables))
-            return NONE;
-
-        Function<SSTableReader, ReplayPosition> f = new Function<SSTableReader, ReplayPosition>()
-        {
-            public ReplayPosition apply(SSTableReader sstable)
-            {
-                return sstable.getReplayPosition();
-            }
-        };
-        Ordering<ReplayPosition> ordering = Ordering.from(ReplayPosition.comparator);
-        return ordering.max(Iterables.transform(sstables, f));
-    }
-
-
     public final long segment;
     public final int position;
 
-    public static final Comparator<ReplayPosition> comparator = new Comparator<ReplayPosition>()
+    /**
+     * A filter of known safe-to-discard commit log replay positions, based on
+     * the range covered by on disk sstables and those prior to the most recent truncation record
+     */
+    public static class ReplayFilter
     {
-        public int compare(ReplayPosition o1, ReplayPosition o2)
+        final NavigableMap<ReplayPosition, ReplayPosition> persisted = new TreeMap<>();
+        public ReplayFilter(Iterable<SSTableReader> onDisk, ReplayPosition truncatedAt)
         {
-            if (o1.segment != o2.segment)
-                return Long.valueOf(o1.segment).compareTo(o2.segment);
-
-            return Integer.valueOf(o1.position).compareTo(o2.position);
+            for (SSTableReader reader : onDisk)
+            {
+                ReplayPosition start = reader.getSSTableMetadata().commitLogLowerBound;
+                ReplayPosition end = reader.getSSTableMetadata().commitLogUpperBound;
+                add(persisted, start, end);
+            }
+            if (truncatedAt != null)
+                add(persisted, ReplayPosition.NONE, truncatedAt);
         }
-    };
+
+        private static void add(NavigableMap<ReplayPosition, ReplayPosition> ranges, ReplayPosition start, ReplayPosition end)
+        {
+            // extend ourselves to cover any ranges we overlap
+            // record directly preceding our end may extend past us, so take the max of our end and its
+            Map.Entry<ReplayPosition, ReplayPosition> extend = ranges.floorEntry(end);
+            if (extend != null && extend.getValue().compareTo(end) > 0)
+                end = extend.getValue();
+
+            // record directly preceding our start may extend into us; if it does, we take it as our start
+            extend = ranges.lowerEntry(start);
+            if (extend != null && extend.getValue().compareTo(start) >= 0)
+                start = extend.getKey();
+
+            ranges.subMap(start, end).clear();
+            ranges.put(start, end);
+        }
+
+        public boolean shouldReplay(ReplayPosition position)
+        {
+            // replay ranges are start exclusive, end inclusive
+            Map.Entry<ReplayPosition, ReplayPosition> range = persisted.lowerEntry(position);
+            return range == null || position.compareTo(range.getValue()) > 0;
+        }
+
+        public boolean isEmpty()
+        {
+            return persisted.isEmpty();
+        }
+    }
+
+    public static ReplayPosition firstNotCovered(Iterable<ReplayFilter> ranges)
+    {
+        ReplayPosition min = null;
+        for (ReplayFilter map : ranges)
+        {
+            ReplayPosition first = map.persisted.firstEntry().getValue();
+            if (min == null)
+                min = first;
+            else
+                min = Ordering.natural().min(min, first);
+        }
+        if (min == null)
+            return NONE;
+        return min;
+    }
 
     public ReplayPosition(long segment, int position)
     {
@@ -83,9 +115,12 @@
         this.position = position;
     }
 
-    public int compareTo(ReplayPosition other)
+    public int compareTo(ReplayPosition that)
     {
-        return comparator.compare(this, other);
+        if (this.segment != that.segment)
+            return Long.compare(this.segment, that.segment);
+
+        return Integer.compare(this.position, that.position);
     }
 
     @Override
diff --git a/src/java/org/apache/cassandra/db/compaction/AbstractCompactedRow.java b/src/java/org/apache/cassandra/db/compaction/AbstractCompactedRow.java
index c49bee5..16b5fac 100644
--- a/src/java/org/apache/cassandra/db/compaction/AbstractCompactedRow.java
+++ b/src/java/org/apache/cassandra/db/compaction/AbstractCompactedRow.java
@@ -18,14 +18,13 @@
 package org.apache.cassandra.db.compaction;
 
 import java.io.Closeable;
-import java.io.DataOutput;
 import java.io.IOException;
 import java.security.MessageDigest;
 
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.RowIndexEntry;
 import org.apache.cassandra.io.sstable.ColumnStats;
-import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.SequentialWriter;
 
 /**
  * a CompactedRow is an object that takes a bunch of rows (keys + columnfamilies)
@@ -48,7 +47,7 @@
      *
      * @return index information for the written row, or null if the compaction resulted in only expired tombstones.
      */
-    public abstract RowIndexEntry write(long currentPosition, DataOutputPlus out) throws IOException;
+    public abstract RowIndexEntry write(long currentPosition, SequentialWriter out) throws IOException;
 
     /**
      * update @param digest with the data bytes of the row (not including row key or row size).
diff --git a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionIterable.java b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionIterable.java
index 5ac2c8b..9fe8fd9 100644
--- a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionIterable.java
+++ b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionIterable.java
@@ -18,6 +18,7 @@
 package org.apache.cassandra.db.compaction;
 
 import java.util.List;
+import java.util.UUID;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.cassandra.io.sstable.ISSTableScanner;
@@ -30,6 +31,7 @@
     protected final long totalBytes;
     protected volatile long bytesRead;
     protected final List<ISSTableScanner> scanners;
+    protected final UUID compactionId;
     /*
      * counters for merged rows.
      * array index represents (number of merged rows - 1), so index 0 is counter for no merge (1 row),
@@ -37,12 +39,13 @@
      */
     protected final AtomicLong[] mergeCounters;
 
-    public AbstractCompactionIterable(CompactionController controller, OperationType type, List<ISSTableScanner> scanners)
+    public AbstractCompactionIterable(CompactionController controller, OperationType type, List<ISSTableScanner> scanners, UUID compactionId)
     {
         this.controller = controller;
         this.type = type;
         this.scanners = scanners;
         this.bytesRead = 0;
+        this.compactionId = compactionId;
 
         long bytes = 0;
         for (ISSTableScanner scanner : scanners)
@@ -58,7 +61,8 @@
         return new CompactionInfo(controller.cfs.metadata,
                                   type,
                                   bytesRead,
-                                  totalBytes);
+                                  totalBytes,
+                                  compactionId);
     }
 
     protected void updateCounterFor(int rows)
diff --git a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
index bcd37ac..5170375 100644
--- a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
+++ b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
@@ -24,17 +24,18 @@
 import com.google.common.base.Predicate;
 import com.google.common.collect.Iterables;
 import com.google.common.util.concurrent.RateLimiter;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Memtable;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.ISSTableScanner;
-import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 
 /**
@@ -162,7 +163,7 @@
      *
      * Is responsible for marking its sstables as compaction-pending.
      */
-    public abstract Collection<AbstractCompactionTask> getMaximalTask(final int gcBefore);
+    public abstract Collection<AbstractCompactionTask> getMaximalTask(final int gcBefore, boolean splitOutput);
 
     /**
      * @param sstables SSTables to compact. Must be marked as compacting.
@@ -175,9 +176,9 @@
      */
     public abstract AbstractCompactionTask getUserDefinedTask(Collection<SSTableReader> sstables, final int gcBefore);
 
-    public AbstractCompactionTask getCompactionTask(Collection<SSTableReader> sstables, final int gcBefore, long maxSSTableBytes)
+    public AbstractCompactionTask getCompactionTask(LifecycleTransaction txn, final int gcBefore, long maxSSTableBytes)
     {
-        return new CompactionTask(cfs, sstables, gcBefore, false);
+        return new CompactionTask(cfs, txn, gcBefore, false);
     }
 
     /**
@@ -231,9 +232,6 @@
      */
     public void replaceFlushed(Memtable memtable, SSTableReader sstable)
     {
-        cfs.getDataTracker().replaceFlushed(memtable, sstable);
-        if (sstable != null)
-            CompactionManager.instance.submitBackground(cfs);
     }
 
     /**
@@ -267,6 +265,7 @@
      * allow for a more memory efficient solution if we know the sstable don't overlap (see
      * LeveledCompactionStrategy for instance).
      */
+    @SuppressWarnings("resource")
     public ScannerList getScanners(Collection<SSTableReader> sstables, Range<Token> range)
     {
         RateLimiter limiter = CompactionManager.instance.getRateLimiter();
@@ -397,7 +396,7 @@
             long keys = sstable.estimatedKeys();
             Set<Range<Token>> ranges = new HashSet<Range<Token>>(overlaps.size());
             for (SSTableReader overlap : overlaps)
-                ranges.add(new Range<Token>(overlap.first.getToken(), overlap.last.getToken(), overlap.partitioner));
+                ranges.add(new Range<>(overlap.first.getToken(), overlap.last.getToken()));
             long remainingKeys = keys - sstable.estimatedKeysForRanges(ranges);
             // next, calculate what percentage of columns we have within those keys
             long columns = sstable.getEstimatedColumnCount().mean() * remainingKeys;
@@ -473,4 +472,35 @@
 
         return optionValue == null || Boolean.parseBoolean(optionValue);
     }
+
+
+    /**
+     * Method for grouping similar SSTables together, This will be used by
+     * anti-compaction to determine which SSTables should be anitcompacted
+     * as a group. If a given compaction strategy creates sstables which
+     * cannot be merged due to some constraint it must override this method.
+     */
+    public Collection<Collection<SSTableReader>> groupSSTablesForAntiCompaction(Collection<SSTableReader> sstablesToGroup)
+    {
+        int groupSize = 2;
+        List<SSTableReader> sortedSSTablesToGroup = new ArrayList<>(sstablesToGroup);
+        Collections.sort(sortedSSTablesToGroup, SSTableReader.sstableComparator);
+
+        Collection<Collection<SSTableReader>> groupedSSTables = new ArrayList<>();
+        Collection<SSTableReader> currGroup = new ArrayList<>();
+
+        for (SSTableReader sstable : sortedSSTablesToGroup)
+        {
+            currGroup.add(sstable);
+            if (currGroup.size() == groupSize)
+            {
+                groupedSSTables.add(currGroup);
+                currGroup = new ArrayList<>();
+            }
+        }
+
+        if (currGroup.size() != 0)
+            groupedSSTables.add(currGroup);
+        return groupedSSTables;
+    }
 }
diff --git a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionTask.java b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionTask.java
index 59338f4..3bf224e 100644
--- a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionTask.java
+++ b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionTask.java
@@ -20,32 +20,32 @@
 import java.util.Set;
 
 import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.compaction.CompactionManager.CompactionExecutorStatsCollector;
-import org.apache.cassandra.io.sstable.SSTableReader;
-import org.apache.cassandra.io.util.DiskAwareRunnable;
+import org.apache.cassandra.db.compaction.writers.CompactionAwareWriter;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.utils.WrappedRunnable;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 
-public abstract class AbstractCompactionTask extends DiskAwareRunnable
+public abstract class AbstractCompactionTask extends WrappedRunnable
 {
     protected final ColumnFamilyStore cfs;
-    protected Set<SSTableReader> sstables;
+    protected LifecycleTransaction transaction;
     protected boolean isUserDefined;
     protected OperationType compactionType;
 
     /**
      * @param cfs
-     * @param sstables must be marked compacting
+     * @param transaction the modifying managing the status of the sstables we're replacing
      */
-    public AbstractCompactionTask(ColumnFamilyStore cfs, Set<SSTableReader> sstables)
+    public AbstractCompactionTask(ColumnFamilyStore cfs, LifecycleTransaction transaction)
     {
         this.cfs = cfs;
-        this.sstables = sstables;
+        this.transaction = transaction;
         this.isUserDefined = false;
         this.compactionType = OperationType.COMPACTION;
-
         // enforce contract that caller should mark sstables compacting
-        Set<SSTableReader> compacting = cfs.getDataTracker().getCompacting();
-        for (SSTableReader sstable : sstables)
+        Set<SSTableReader> compacting = transaction.tracker.getCompacting();
+        for (SSTableReader sstable : transaction.originals())
             assert compacting.contains(sstable) : sstable.getFilename() + " is not correctly marked compacting";
     }
 
@@ -60,17 +60,13 @@
         }
         finally
         {
-            cfs.getDataTracker().unmarkCompacting(sstables);
+            transaction.close();
         }
     }
+    public abstract CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables);
 
     protected abstract int executeInternal(CompactionExecutorStatsCollector collector);
 
-    protected Directories getDirectories()
-    {
-        return cfs.directories;
-    }
-
     public AbstractCompactionTask setUserDefined(boolean isUserDefined)
     {
         this.isUserDefined = isUserDefined;
@@ -85,6 +81,6 @@
 
     public String toString()
     {
-        return "CompactionTask(" + sstables + ")";
+        return "CompactionTask(" + transaction + ")";
     }
 }
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionController.java b/src/java/org/apache/cassandra/db/compaction/CompactionController.java
index 1139ee4..e895573 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionController.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionController.java
@@ -17,24 +17,27 @@
  */
 package org.apache.cassandra.db.compaction;
 
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
+import java.util.*;
 
+import com.google.common.base.Predicate;
+import com.google.common.base.Predicates;
+
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.db.ColumnFamily;
 import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.DataTracker;
 import org.apache.cassandra.db.DecoratedKey;
-import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.db.Memtable;
+import org.apache.cassandra.db.RowPosition;
 import org.apache.cassandra.utils.AlwaysPresentFilter;
 
+import org.apache.cassandra.utils.OverlapIterator;
 import org.apache.cassandra.utils.concurrent.Refs;
 
+import static org.apache.cassandra.db.lifecycle.SSTableIntervalTree.buildIntervals;
+
 /**
  * Manage compaction options.
  */
@@ -44,12 +47,11 @@
     static final boolean NEVER_PURGE_TOMBSTONES = Boolean.getBoolean("cassandra.never_purge_tombstones");
 
     public final ColumnFamilyStore cfs;
-
-    // note that overlappingTree and overlappingSSTables will be null if NEVER_PURGE_TOMBSTONES is set - this is a
+    // note that overlapIterator and overlappingSSTables will be null if NEVER_PURGE_TOMBSTONES is set - this is a
     // good thing so that noone starts using them and thinks that if overlappingSSTables is empty, there
     // is no overlap.
-    private DataTracker.SSTableIntervalTree overlappingTree;
     private Refs<SSTableReader> overlappingSSTables;
+    private OverlapIterator<RowPosition, SSTableReader> overlapIterator;
     private final Iterable<SSTableReader> compacting;
 
     public final int gcBefore;
@@ -100,7 +102,7 @@
             overlappingSSTables = Refs.tryRef(Collections.<SSTableReader>emptyList());
         else
             overlappingSSTables = cfs.getAndReferenceOverlappingSSTables(compacting);
-        this.overlappingTree = DataTracker.buildIntervalTree(overlappingSSTables);
+        this.overlapIterator = new OverlapIterator<>(buildIntervals(overlappingSSTables));
     }
 
     public Set<SSTableReader> getFullyExpiredSSTables()
@@ -112,7 +114,7 @@
      * Finds expired sstables
      *
      * works something like this;
-     * 1. find "global" minTimestamp of overlapping sstables and compacting sstables containing any non-expired data
+     * 1. find "global" minTimestamp of overlapping sstables, compacting sstables and memtables containing any non-expired data
      * 2. build a list of fully expired candidates
      * 3. check if the candidates to be dropped actually can be dropped (maxTimestamp < global minTimestamp)
      *    - if not droppable, remove from candidates
@@ -126,12 +128,12 @@
      */
     public static Set<SSTableReader> getFullyExpiredSSTables(ColumnFamilyStore cfStore, Iterable<SSTableReader> compacting, Iterable<SSTableReader> overlapping, int gcBefore)
     {
-        logger.debug("Checking droppable sstables in {}", cfStore);
+        logger.trace("Checking droppable sstables in {}", cfStore);
 
         if (compacting == null || NEVER_PURGE_TOMBSTONES)
             return Collections.<SSTableReader>emptySet();
 
-        List<SSTableReader> candidates = new ArrayList<SSTableReader>();
+        List<SSTableReader> candidates = new ArrayList<>();
 
         long minTimestamp = Long.MAX_VALUE;
 
@@ -151,8 +153,11 @@
                 minTimestamp = Math.min(minTimestamp, candidate.getMinTimestamp());
         }
 
+        for (Memtable memtable : cfStore.getTracker().getView().getAllMemtables())
+            minTimestamp = Math.min(minTimestamp, memtable.getMinTimestamp());
+
         // At this point, minTimestamp denotes the lowest timestamp of any relevant
-        // SSTable that contains a constructive value. candidates contains all the
+        // SSTable or Memtable that contains a constructive value. candidates contains all the
         // candidates with no constructive values. The ones out of these that have
         // (getMaxTimestamp() < minTimestamp) serve no purpose anymore.
 
@@ -166,7 +171,7 @@
             }
             else
             {
-               logger.debug("Dropping expired SSTable {} (maxLocalDeletionTime={}, gcBefore={})",
+               logger.trace("Dropping expired SSTable {} (maxLocalDeletionTime={}, gcBefore={})",
                         candidate, candidate.getSSTableMetadata().maxLocalDeletionTime, gcBefore);
             }
         }
@@ -184,28 +189,59 @@
     }
 
     /**
-     * @return the largest timestamp before which it's okay to drop tombstones for the given partition;
-     * i.e., after the maxPurgeableTimestamp there may exist newer data that still needs to be suppressed
-     * in other sstables.  This returns the minimum timestamp for any SSTable that contains this partition and is not
-     * participating in this compaction, or LONG.MAX_VALUE if no such SSTable exists.
+     * @param key
+     * @return a predicate for whether tombstones marked for deletion at the given time for the given partition are
+     * purgeable; we calculate this by checking whether the deletion time is less than the min timestamp of all SSTables
+     * containing his partition and not participating in the compaction. This means there isn't any data in those
+     * sstables that might still need to be suppressed by a tombstone at this timestamp.
      */
-    public long maxPurgeableTimestamp(DecoratedKey key)
+    public Predicate<Long> getPurgeEvaluator(DecoratedKey key)
     {
         if (NEVER_PURGE_TOMBSTONES)
-            return Long.MIN_VALUE;
+            return Predicates.alwaysFalse();
 
-        List<SSTableReader> filteredSSTables = overlappingTree.search(key);
-        long min = Long.MAX_VALUE;
-        for (SSTableReader sstable : filteredSSTables)
+        overlapIterator.update(key);
+        Set<SSTableReader> filteredSSTables = overlapIterator.overlaps();
+        Iterable<Memtable> memtables = cfs.getTracker().getView().getAllMemtables();
+        long minTimestampSeen = Long.MAX_VALUE;
+        boolean hasTimestamp = false;
+
+        for (SSTableReader sstable: filteredSSTables)
         {
             // if we don't have bloom filter(bf_fp_chance=1.0 or filter file is missing),
             // we check index file instead.
-            if (sstable.getBloomFilter() instanceof AlwaysPresentFilter && sstable.getPosition(key, SSTableReader.Operator.EQ, false) != null)
-                min = Math.min(min, sstable.getMinTimestamp());
-            else if (sstable.getBloomFilter().isPresent(key.getKey()))
-                min = Math.min(min, sstable.getMinTimestamp());
+            if (sstable.getBloomFilter() instanceof AlwaysPresentFilter && sstable.getPosition(key, SSTableReader.Operator.EQ, false) != null
+                || sstable.getBloomFilter().isPresent(key))
+            {
+                minTimestampSeen = Math.min(minTimestampSeen, sstable.getMinTimestamp());
+                hasTimestamp = true;
+            }
+
         }
-        return min;
+
+        for (Memtable memtable : memtables)
+        {
+            ColumnFamily cf = memtable.getColumnFamily(key);
+            if (cf != null)
+            {
+                minTimestampSeen = Math.min(minTimestampSeen, memtable.getMinTimestamp());
+                hasTimestamp = true;
+            }
+        }
+
+        if (!hasTimestamp)
+            return Predicates.alwaysTrue();
+        else
+        {
+            final long finalTimestamp = minTimestampSeen;
+            return new Predicate<Long>()
+            {
+                public boolean apply(Long time)
+                {
+                    return time < finalTimestamp;
+                }
+            };
+        }
     }
 
     public void close()
@@ -213,4 +249,5 @@
         if (overlappingSSTables != null)
             overlappingSSTables.release();
     }
+
 }
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionInfo.java b/src/java/org/apache/cassandra/db/compaction/CompactionInfo.java
index e88143e..3cd8737 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionInfo.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionInfo.java
@@ -23,6 +23,7 @@
 import java.util.UUID;
 
 import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.metrics.StorageMetrics;
 import org.apache.cassandra.service.StorageService;
 
 /** Implements serializable to allow structured info to be returned via JMX. */
@@ -34,45 +35,47 @@
     private final long completed;
     private final long total;
     private final String unit;
+    private final UUID compactionId;
 
-    public CompactionInfo(CFMetaData cfm, OperationType tasktype, long bytesComplete, long totalBytes)
+    public CompactionInfo(CFMetaData cfm, OperationType tasktype, long bytesComplete, long totalBytes, UUID compactionId)
     {
-        this(cfm, tasktype, bytesComplete, totalBytes, "bytes");
+        this(cfm, tasktype, bytesComplete, totalBytes, "bytes", compactionId);
     }
 
-    public CompactionInfo(OperationType tasktype, long completed, long total, String unit)
+    public CompactionInfo(OperationType tasktype, long completed, long total, String unit, UUID compactionId)
     {
-        this(null, tasktype, completed, total, unit);
+        this(null, tasktype, completed, total, unit, compactionId);
     }
 
-    public CompactionInfo(CFMetaData cfm, OperationType tasktype, long completed, long total, String unit)
+    public CompactionInfo(CFMetaData cfm, OperationType tasktype, long completed, long total, String unit, UUID compactionId)
     {
         this.tasktype = tasktype;
         this.completed = completed;
         this.total = total;
         this.cfm = cfm;
         this.unit = unit;
+        this.compactionId = compactionId;
     }
 
     /** @return A copy of this CompactionInfo with updated progress. */
     public CompactionInfo forProgress(long complete, long total)
     {
-        return new CompactionInfo(cfm, tasktype, complete, total, unit);
+        return new CompactionInfo(cfm, tasktype, complete, total, unit, compactionId);
     }
 
     public UUID getId()
     {
-        return cfm.cfId;
+        return cfm != null ? cfm.cfId : null;
     }
 
     public String getKeyspace()
     {
-        return cfm.ksName;
+        return cfm != null ? cfm.ksName : null;
     }
 
     public String getColumnFamily()
     {
-        return cfm.cfName;
+        return cfm != null ? cfm.cfName : null;
     }
 
     public CFMetaData getCFMetaData()
@@ -95,6 +98,11 @@
         return tasktype;
     }
 
+    public UUID compactionId()
+    {
+        return compactionId;
+    }
+
     public String toString()
     {
         StringBuilder buff = new StringBuilder();
@@ -122,6 +130,7 @@
         ret.put("total", Long.toString(total));
         ret.put("taskType", tasktype.toString());
         ret.put("unit", unit);
+        ret.put("compactionId", compactionId == null ? "" : compactionId.toString());
         return ret;
     }
 
@@ -129,7 +138,7 @@
     {
         private volatile boolean stopRequested = false;
         public abstract CompactionInfo getCompactionInfo();
-        double load = StorageService.instance.getLoad();
+        double load = StorageMetrics.load.getCount();
         double reportedSeverity = 0d;
 
         public void stop()
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionIterable.java b/src/java/org/apache/cassandra/db/compaction/CompactionIterable.java
index fdcec6e..23d8a4a 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionIterable.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionIterable.java
@@ -20,16 +20,20 @@
 import java.util.ArrayList;
 import java.util.Comparator;
 import java.util.List;
+import java.util.UUID;
 
 import com.google.common.collect.ImmutableList;
 
 import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
 import org.apache.cassandra.io.sstable.ISSTableScanner;
 import org.apache.cassandra.utils.CloseableIterator;
 import org.apache.cassandra.utils.MergeIterator;
 
 public class CompactionIterable extends AbstractCompactionIterable
 {
+    final SSTableFormat format;
+
     private static final Comparator<OnDiskAtomIterator> comparator = new Comparator<OnDiskAtomIterator>()
     {
         public int compare(OnDiskAtomIterator i1, OnDiskAtomIterator i2)
@@ -38,9 +42,14 @@
         }
     };
 
-    public CompactionIterable(OperationType type, List<ISSTableScanner> scanners, CompactionController controller)
+    public CompactionIterable(OperationType type,
+                              List<ISSTableScanner> scanners,
+                              CompactionController controller,
+                              SSTableFormat.Type formatType,
+                              UUID compactionId)
     {
-        super(controller, type, scanners);
+        super(controller, type, scanners, compactionId);
+        this.format = formatType.info;
     }
 
     public CloseableIterator<AbstractCompactedRow> iterator()
@@ -72,7 +81,7 @@
                 // create a new container for rows, since we're going to clear ours for the next one,
                 // and the AbstractCompactionRow code should be able to assume that the collection it receives
                 // won't be pulled out from under it.
-                return new LazilyCompactedRow(controller, ImmutableList.copyOf(rows));
+                return format.getCompactedRowWriter(controller, ImmutableList.copyOf(rows));
             }
             finally
             {
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index 87819ba..d21f1e8 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@ -20,32 +20,17 @@
 import java.io.File;
 import java.io.IOException;
 import java.lang.management.ManagementFactory;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
+import java.util.*;
 import java.util.concurrent.*;
 import javax.management.MBeanServer;
 import javax.management.ObjectName;
 import javax.management.openmbean.OpenDataException;
 import javax.management.openmbean.TabularData;
 
-import com.google.common.base.Predicate;
-import com.google.common.base.Throwables;
-import com.google.common.collect.ArrayListMultimap;
-import com.google.common.collect.ConcurrentHashMultiset;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Multimap;
-import com.google.common.collect.Multiset;
-import com.google.common.collect.Sets;
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Predicate;
+import com.google.common.base.Predicates;
+import com.google.common.collect.*;
 import com.google.common.util.concurrent.*;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -57,34 +42,34 @@
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.db.Cell;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.DecoratedKey;
-import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.OnDiskAtom;
-import org.apache.cassandra.db.SystemKeyspace;
+import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.compaction.CompactionInfo.Holder;
 import org.apache.cassandra.db.index.SecondaryIndexBuilder;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.dht.Bounds;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.sstable.*;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.metrics.CompactionMetrics;
 import org.apache.cassandra.repair.Validator;
 import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.utils.concurrent.OpOrder;
 import org.apache.cassandra.utils.*;
-
+import org.apache.cassandra.utils.concurrent.OpOrder;
 import org.apache.cassandra.utils.concurrent.Refs;
 
+import static java.util.Collections.singleton;
+
 /**
+ * <p>
  * A singleton which manages a private executor of ongoing compactions.
- * <p/>
+ * </p>
  * Scheduling for compaction is accomplished by swapping sstables to be compacted into
- * a set via DataTracker. New scheduling attempts will ignore currently compacting
+ * a set via Tracker. New scheduling attempts will ignore currently compacting
  * sstables.
  */
 public class CompactionManager implements CompactionManagerMBean
@@ -131,24 +116,33 @@
     private final RateLimiter compactionRateLimiter = RateLimiter.create(Double.MAX_VALUE);
 
     /**
-     * Gets compaction rate limiter. When compaction_throughput_mb_per_sec is 0 or node is bootstrapping,
-     * this returns rate limiter with the rate of Double.MAX_VALUE bytes per second.
+     * Gets compaction rate limiter.
      * Rate unit is bytes per sec.
      *
      * @return RateLimiter with rate limit set
      */
     public RateLimiter getRateLimiter()
     {
-        double currentThroughput = DatabaseDescriptor.getCompactionThroughputMbPerSec() * 1024.0 * 1024.0;
-        // if throughput is set to 0, throttling is disabled
-        if (currentThroughput == 0 || StorageService.instance.isBootstrapMode())
-            currentThroughput = Double.MAX_VALUE;
-        if (compactionRateLimiter.getRate() != currentThroughput)
-            compactionRateLimiter.setRate(currentThroughput);
+        setRate(DatabaseDescriptor.getCompactionThroughputMbPerSec());
         return compactionRateLimiter;
     }
 
     /**
+     * Sets the rate for the rate limiter. When compaction_throughput_mb_per_sec is 0 or node is bootstrapping,
+     * this sets the rate to Double.MAX_VALUE bytes per second.
+     * @param throughPutMbPerSec throughput to set in mb per second
+     */
+    public void setRate(final double throughPutMbPerSec)
+    {
+        double throughput = throughPutMbPerSec * 1024.0 * 1024.0;
+        // if throughput is set to 0, throttling is disabled
+        if (throughput == 0 || StorageService.instance.isBootstrapMode())
+            throughput = Double.MAX_VALUE;
+        if (compactionRateLimiter.getRate() != throughput)
+            compactionRateLimiter.setRate(throughput);
+    }
+
+    /**
      * Call this whenever a compaction might be needed on the given columnfamily.
      * It's okay to over-call (within reason) if a call is unnecessary, it will
      * turn into a no-op in the bucketing/candidate-scan phase.
@@ -157,39 +151,37 @@
     {
         if (cfs.isAutoCompactionDisabled())
         {
-            logger.debug("Autocompaction is disabled");
+            logger.trace("Autocompaction is disabled");
             return Collections.emptyList();
         }
 
         int count = compactingCF.count(cfs);
         if (count > 0 && executor.getActiveCount() >= executor.getMaximumPoolSize())
         {
-            logger.debug("Background compaction is still running for {}.{} ({} remaining). Skipping",
+            logger.trace("Background compaction is still running for {}.{} ({} remaining). Skipping",
                          cfs.keyspace.getName(), cfs.name, count);
             return Collections.emptyList();
         }
 
-        logger.debug("Scheduling a background task check for {}.{} with {}",
+        logger.trace("Scheduling a background task check for {}.{} with {}",
                      cfs.keyspace.getName(),
                      cfs.name,
                      cfs.getCompactionStrategy().getName());
-        List<Future<?>> futures = new ArrayList<>();
-        // we must schedule it at least once, otherwise compaction will stop for a CF until next flush
-        if (executor.isShutdown())
-        {
-            logger.info("Executor has shut down, not submitting background task");
-            return Collections.emptyList();
-        }
-        compactingCF.add(cfs);
-        futures.add(executor.submit(new BackgroundCompactionCandidate(cfs)));
 
+        List<Future<?>> futures = new ArrayList<>(1);
+        Future<?> fut = executor.submitIfRunning(new BackgroundCompactionCandidate(cfs), "background task");
+        if (!fut.isCancelled())
+        {
+            compactingCF.add(cfs);
+            futures.add(fut);
+        }
         return futures;
     }
 
     public boolean isCompacting(Iterable<ColumnFamilyStore> cfses)
     {
         for (ColumnFamilyStore cfs : cfses)
-            if (!cfs.getDataTracker().getCompacting().isEmpty())
+            if (!cfs.getTracker().getCompacting().isEmpty())
                 return true;
         return false;
     }
@@ -217,7 +209,8 @@
         {
             try
             {
-                exec.awaitTermination(1, TimeUnit.MINUTES);
+                if (!exec.awaitTermination(1, TimeUnit.MINUTES))
+                    logger.warn("Failed to wait for compaction executors shutdown");
             }
             catch (InterruptedException e)
             {
@@ -247,10 +240,10 @@
         {
             try
             {
-                logger.debug("Checking {}.{}", cfs.keyspace.getName(), cfs.name);
+                logger.trace("Checking {}.{}", cfs.keyspace.getName(), cfs.name);
                 if (!cfs.isValid())
                 {
-                    logger.debug("Aborting compaction for dropped CF");
+                    logger.trace("Aborting compaction for dropped CF");
                     return;
                 }
 
@@ -258,7 +251,7 @@
                 AbstractCompactionTask task = strategy.getNextBackgroundTask(getDefaultGcBefore(cfs));
                 if (task == null)
                 {
-                    logger.debug("No tasks available");
+                    logger.trace("No tasks available");
                     return;
                 }
                 task.execute(metrics);
@@ -281,53 +274,40 @@
      * @throws ExecutionException
      * @throws InterruptedException
      */
-    private AllSSTableOpStatus parallelAllSSTableOperation(final ColumnFamilyStore cfs, final OneSSTableOperation operation, int jobs) throws ExecutionException, InterruptedException
+    @SuppressWarnings("resource")
+    private AllSSTableOpStatus parallelAllSSTableOperation(final ColumnFamilyStore cfs, final OneSSTableOperation operation, int jobs, OperationType operationType) throws ExecutionException, InterruptedException
     {
-        Iterable<SSTableReader> compactingSSTables = cfs.markAllCompacting();
-        if (compactingSSTables == null)
+        List<LifecycleTransaction> transactions = new ArrayList<>();
+        try (LifecycleTransaction compacting = cfs.markAllCompacting(operationType))
         {
-            logger.info("Aborting operation on {}.{} after failing to interrupt other compaction operations", cfs.keyspace.getName(), cfs.name);
-            return AllSSTableOpStatus.ABORTED;
-        }
-        if (Iterables.isEmpty(compactingSSTables))
-        {
-            logger.info("No sstables for {}.{}", cfs.keyspace.getName(), cfs.name);
-            return AllSSTableOpStatus.SUCCESSFUL;
-        }
-        Set<SSTableReader> sstables = Sets.newHashSet(operation.filterSSTables(compactingSSTables));
-        Set<SSTableReader> filteredAway = Sets.difference(Sets.newHashSet(compactingSSTables), sstables);
-        cfs.getDataTracker().unmarkCompacting(filteredAway);
-        final Set<SSTableReader> finished = Sets.newConcurrentHashSet();
+            Iterable<SSTableReader> sstables = compacting != null ? Lists.newArrayList(operation.filterSSTables(compacting)) : Collections.<SSTableReader>emptyList();
+            if (Iterables.isEmpty(sstables))
+            {
+                logger.info("No sstables for {}.{}", cfs.keyspace.getName(), cfs.name);
+                return AllSSTableOpStatus.SUCCESSFUL;
+            }
 
-        List<Future<Object>> futures = new ArrayList<>();
-        try
-        {
+            List<Future<?>> futures = new ArrayList<>();
+
             for (final SSTableReader sstable : sstables)
             {
-                if (executor.isShutdown())
-                {
-                    logger.info("Executor has shut down, not submitting task");
-                    return AllSSTableOpStatus.ABORTED;
-                }
-
+                final LifecycleTransaction txn = compacting.split(singleton(sstable));
+                transactions.add(txn);
                 Callable<Object> callable = new Callable<Object>()
                 {
                     @Override
                     public Object call() throws Exception
                     {
-                        try
-                        {
-                            operation.execute(sstable);
-                        }
-                        finally
-                        {
-                            cfs.getDataTracker().unmarkCompacting(Collections.singleton(sstable));
-                            finished.add(sstable);
-                        }
+                        operation.execute(txn);
                         return this;
                     }
                 };
-                futures.add(executor.submit(callable));
+                Future<?> fut = executor.submitIfRunning(callable, "paralell sstable operation");
+                if (!fut.isCancelled())
+                    futures.add(fut);
+                else
+                    return AllSSTableOpStatus.ABORTED;
+
                 if (jobs > 0 && futures.size() == jobs)
                 {
                     FBUtilities.waitOnFutures(futures);
@@ -335,18 +315,21 @@
                 }
             }
             FBUtilities.waitOnFutures(futures);
+            assert compacting.originals().isEmpty();
+            return AllSSTableOpStatus.SUCCESSFUL;
         }
         finally
         {
-            cfs.getDataTracker().unmarkCompacting(Sets.difference(sstables, finished));
+            Throwable fail = Throwables.close(null, transactions);
+            if (fail != null)
+                logger.error("Failed to cleanup lifecycle transactions {}", fail);
         }
-        return AllSSTableOpStatus.SUCCESSFUL;
     }
 
     private static interface OneSSTableOperation
     {
-        Iterable<SSTableReader> filterSSTables(Iterable<SSTableReader> input);
-        void execute(SSTableReader input) throws IOException;
+        Iterable<SSTableReader> filterSSTables(LifecycleTransaction transaction);
+        void execute(LifecycleTransaction input) throws IOException;
     }
 
     public enum AllSSTableOpStatus { ABORTED(1), SUCCESSFUL(0);
@@ -358,23 +341,42 @@
         }
     }
 
-    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData, int jobs) throws InterruptedException, ExecutionException
+    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData, int jobs)
+    throws InterruptedException, ExecutionException
+    {
+        return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
+        {
+            @Override
+            public Iterable<SSTableReader> filterSSTables(LifecycleTransaction input)
+            {
+                return input.originals();
+            }
+
+            @Override
+            public void execute(LifecycleTransaction input) throws IOException
+            {
+                scrubOne(cfs, input, skipCorrupted, checkData);
+            }
+        }, jobs, OperationType.SCRUB);
+    }
+
+    public AllSSTableOpStatus performVerify(final ColumnFamilyStore cfs, final boolean extendedVerify) throws InterruptedException, ExecutionException
     {
         assert !cfs.isIndex();
         return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
         {
             @Override
-            public Iterable<SSTableReader> filterSSTables(Iterable<SSTableReader> input)
+            public Iterable<SSTableReader> filterSSTables(LifecycleTransaction input)
             {
-                return input;
+                return input.originals();
             }
 
             @Override
-            public void execute(SSTableReader input) throws IOException
+            public void execute(LifecycleTransaction input) throws IOException
             {
-                scrubOne(cfs, input, skipCorrupted, checkData);
+                verifyOne(cfs, input.onlyOne(), extendedVerify);
             }
-        }, jobs);
+        }, 0, OperationType.VERIFY);
     }
 
     public AllSSTableOpStatus performSSTableRewrite(final ColumnFamilyStore cfs, final boolean excludeCurrentVersion, int jobs) throws InterruptedException, ExecutionException
@@ -382,58 +384,67 @@
         return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
         {
             @Override
-            public Iterable<SSTableReader> filterSSTables(Iterable<SSTableReader> input)
+            public Iterable<SSTableReader> filterSSTables(LifecycleTransaction transaction)
             {
-                return Iterables.filter(input, new Predicate<SSTableReader>()
+                Iterable<SSTableReader> sstables = new ArrayList<>(transaction.originals());
+                Iterator<SSTableReader> iter = sstables.iterator();
+                while (iter.hasNext())
                 {
-                    @Override
-                    public boolean apply(SSTableReader sstable)
+                    SSTableReader sstable = iter.next();
+                    if (excludeCurrentVersion && sstable.descriptor.version.equals(sstable.descriptor.getFormat().getLatestVersion()))
                     {
-                        return !(excludeCurrentVersion && sstable.descriptor.version.equals(Descriptor.Version.CURRENT));
+                        transaction.cancel(sstable);
+                        iter.remove();
                     }
-                });
+                }
+                return sstables;
             }
 
             @Override
-            public void execute(SSTableReader input) throws IOException
+            public void execute(LifecycleTransaction txn) throws IOException
             {
-                AbstractCompactionTask task = cfs.getCompactionStrategy().getCompactionTask(Collections.singleton(input), NO_GC, Long.MAX_VALUE);
+                AbstractCompactionTask task = cfs.getCompactionStrategy().getCompactionTask(txn, NO_GC, Long.MAX_VALUE);
                 task.setUserDefined(true);
                 task.setCompactionType(OperationType.UPGRADE_SSTABLES);
                 task.execute(metrics);
             }
-        }, jobs);
+        }, jobs, OperationType.UPGRADE_SSTABLES);
     }
 
     public AllSSTableOpStatus performCleanup(final ColumnFamilyStore cfStore, int jobs) throws InterruptedException, ExecutionException
     {
         assert !cfStore.isIndex();
         Keyspace keyspace = cfStore.keyspace;
-        final Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(keyspace.getName());
-        if (ranges.isEmpty())
+        if (!StorageService.instance.isJoined())
         {
             logger.info("Cleanup cannot run before a node has joined the ring");
             return AllSSTableOpStatus.ABORTED;
         }
+        final Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(keyspace.getName());
+        if (ranges.isEmpty())
+        {
+            logger.info("Node owns no data for keyspace {}", keyspace.getName());
+            return AllSSTableOpStatus.SUCCESSFUL;
+        }
         final boolean hasIndexes = cfStore.indexManager.hasIndexes();
 
         return parallelAllSSTableOperation(cfStore, new OneSSTableOperation()
         {
             @Override
-            public Iterable<SSTableReader> filterSSTables(Iterable<SSTableReader> input)
+            public Iterable<SSTableReader> filterSSTables(LifecycleTransaction transaction)
             {
-                List<SSTableReader> sortedSSTables = Lists.newArrayList(input);
+                List<SSTableReader> sortedSSTables = Lists.newArrayList(transaction.originals());
                 Collections.sort(sortedSSTables, new SSTableReader.SizeComparator());
                 return sortedSSTables;
             }
 
             @Override
-            public void execute(SSTableReader input) throws IOException
+            public void execute(LifecycleTransaction txn) throws IOException
             {
                 CleanupStrategy cleanupStrategy = CleanupStrategy.get(cfStore, ranges);
-                doCleanupOne(cfStore, input, cleanupStrategy, ranges, hasIndexes);
+                doCleanupOne(cfStore, txn, cleanupStrategy, ranges, hasIndexes);
             }
-        }, jobs);
+        }, jobs, OperationType.CLEANUP);
     }
 
     public ListenableFuture<?> submitAntiCompaction(final ColumnFamilyStore cfs,
@@ -443,51 +454,57 @@
     {
         Runnable runnable = new WrappedRunnable() {
             @Override
+            @SuppressWarnings("resource")
             public void runMayThrow() throws Exception
             {
-                boolean success = false;
-                while (!success)
+                LifecycleTransaction modifier = null;
+                while (modifier == null)
                 {
-                    for (SSTableReader compactingSSTable : cfs.getDataTracker().getCompacting())
+                    for (SSTableReader compactingSSTable : cfs.getTracker().getCompacting())
                         sstables.releaseIfHolds(compactingSSTable);
                     Set<SSTableReader> compactedSSTables = new HashSet<>();
                     for (SSTableReader sstable : sstables)
                         if (sstable.isMarkedCompacted())
                             compactedSSTables.add(sstable);
                     sstables.release(compactedSSTables);
-                    success = sstables.isEmpty() || cfs.getDataTracker().markCompacting(sstables);
+                    modifier = cfs.getTracker().tryModify(sstables, OperationType.ANTICOMPACTION);
                 }
-                performAnticompaction(cfs, ranges, sstables, repairedAt);
+                performAnticompaction(cfs, ranges, sstables, modifier, repairedAt);
             }
         };
-        if (executor.isShutdown())
-        {
-            logger.info("Compaction executor has shut down, not submitting anticompaction");
-            return Futures.immediateCancelledFuture();
-        }
 
-        ListenableFutureTask<?> task = ListenableFutureTask.create(runnable, null);
-        executor.submit(task);
-        return task;
+        ListenableFuture<?> ret = null;
+        try
+        {
+            ret = executor.submitIfRunning(runnable, "anticompaction");
+            return ret;
+        }
+        finally
+        {
+            if (ret == null || ret.isCancelled())
+                sstables.release();
+        }
     }
 
     /**
      * Make sure the {validatedForRepair} are marked for compaction before calling this.
      *
-     * Caller must reference the validatedForRepair sstables (via ParentRepairSession.getActiveRepairedSSTableRefsForAntiCompaction(..)).
+     * Caller must reference the validatedForRepair sstables (via ParentRepairSession.getActiveRepairedSSTableRefs(..)).
      *
      * @param cfs
      * @param ranges Ranges that the repair was carried out on
      * @param validatedForRepair SSTables containing the repaired ranges. Should be referenced before passing them.
-     * @throws InterruptedException, ExecutionException, IOException
+     * @throws InterruptedException
+     * @throws IOException
      */
     public void performAnticompaction(ColumnFamilyStore cfs,
                                       Collection<Range<Token>> ranges,
                                       Refs<SSTableReader> validatedForRepair,
-                                      long repairedAt) throws InterruptedException, ExecutionException, IOException
+                                      LifecycleTransaction txn,
+                                      long repairedAt) throws InterruptedException, IOException
     {
         logger.info("Starting anticompaction for {}.{} on {}/{} sstables", cfs.keyspace.getName(), cfs.getColumnFamilyName(), validatedForRepair.size(), cfs.getSSTables().size());
-        logger.debug("Starting anticompaction for ranges {}", ranges);
+        logger.trace("Starting anticompaction for ranges {}", ranges);
         Set<SSTableReader> sstables = new HashSet<>(validatedForRepair);
         Set<SSTableReader> mutatedRepairStatuses = new HashSet<>();
         // we should only notify that repair status changed if it actually did:
@@ -507,7 +524,7 @@
             {
                 SSTableReader sstable = sstableIterator.next();
 
-                Range<Token> sstableRange = new Range<>(sstable.first.getToken(), sstable.last.getToken(), sstable.partitioner);
+                Range<Token> sstableRange = new Range<>(sstable.first.getToken(), sstable.last.getToken());
 
                 boolean shouldAnticompact = false;
 
@@ -539,32 +556,34 @@
                     sstableIterator.remove();
                 }
             }
+            cfs.getTracker().notifySSTableRepairedStatusChanged(mutatedRepairStatusToNotify);
+            txn.cancel(Sets.union(nonAnticompacting, mutatedRepairStatuses));
             validatedForRepair.release(Sets.union(nonAnticompacting, mutatedRepairStatuses));
-            cfs.getDataTracker().notifySSTableRepairedStatusChanged(mutatedRepairStatusToNotify);
-            cfs.getDataTracker().unmarkCompacting(Sets.union(nonAnticompacting, mutatedRepairStatuses));
+            assert txn.originals().equals(sstables);
             if (!sstables.isEmpty())
-                doAntiCompaction(cfs, ranges, sstables, repairedAt);
+                doAntiCompaction(cfs, ranges, txn, repairedAt);
+            txn.finish();
         }
         finally
         {
             validatedForRepair.release();
-            cfs.getDataTracker().unmarkCompacting(sstables);
+            txn.close();
         }
 
         logger.info("Completed anticompaction successfully");
     }
 
-    public void performMaximal(final ColumnFamilyStore cfStore) throws InterruptedException, ExecutionException
+    public void performMaximal(final ColumnFamilyStore cfStore, boolean splitOutput)
     {
-        FBUtilities.waitOnFutures(submitMaximal(cfStore, getDefaultGcBefore(cfStore)));
+        FBUtilities.waitOnFutures(submitMaximal(cfStore, getDefaultGcBefore(cfStore), splitOutput));
     }
 
-    public List<Future<?>> submitMaximal(final ColumnFamilyStore cfStore, final int gcBefore)
+    public List<Future<?>> submitMaximal(final ColumnFamilyStore cfStore, final int gcBefore, boolean splitOutput)
     {
         // here we compute the task off the compaction executor, so having that present doesn't
         // confuse runWithCompactionsDisabled -- i.e., we don't want to deadlock ourselves, waiting
         // for ourselves to finish/acknowledge cancellation before continuing.
-        final Collection<AbstractCompactionTask> tasks = cfStore.getCompactionStrategy().getMaximalTask(gcBefore);
+        final Collection<AbstractCompactionTask> tasks = cfStore.getCompactionStrategy().getMaximalTask(gcBefore, splitOutput);
 
         if (tasks == null)
             return Collections.emptyList();
@@ -573,7 +592,7 @@
         int nonEmptyTasks = 0;
         for (final AbstractCompactionTask task : tasks)
         {
-            if (task.sstables.size() > 0)
+            if (task.transaction.originals().size() > 0)
                 nonEmptyTasks++;
             Runnable runnable = new WrappedRunnable()
             {
@@ -582,12 +601,10 @@
                     task.execute(metrics);
                 }
             };
-            if (executor.isShutdown())
-            {
-                logger.info("Compaction executor has shut down, not submitting task");
-                return Collections.emptyList();
-            }
-            futures.add(executor.submit(runnable));
+
+            Future<?> fut = executor.submitIfRunning(runnable, "maximal task");
+            if (!fut.isCancelled())
+                futures.add(fut);
         }
         if (nonEmptyTasks > 1)
             logger.info("Cannot perform a full major compaction as repaired and unrepaired sstables cannot be compacted together. These two set of sstables will be compacted separately.");
@@ -610,7 +627,7 @@
             }
             // group by keyspace/columnfamily
             ColumnFamilyStore cfs = Keyspace.open(desc.ksname).getColumnFamilyStore(desc.cfname);
-            descriptors.put(cfs, cfs.directories.find(filename.trim()));
+            descriptors.put(cfs, cfs.directories.find(new File(filename.trim()).getName()));
         }
 
         List<Future<?>> futures = new ArrayList<>();
@@ -627,7 +644,7 @@
             {
                 // look up the sstables now that we're on the compaction executor, so we don't try to re-compact
                 // something that was already being compacted earlier.
-                Collection<SSTableReader> sstables = new ArrayList<SSTableReader>(dataFiles.size());
+                Collection<SSTableReader> sstables = new ArrayList<>(dataFiles.size());
                 for (Descriptor desc : dataFiles)
                 {
                     // inefficient but not in a performance sensitive path
@@ -654,13 +671,8 @@
                 }
             }
         };
-        if (executor.isShutdown())
-        {
-            logger.info("Compaction executor has shut down, not submitting task");
-            return Futures.immediateCancelledFuture();
-        }
 
-        return executor.submit(runnable);
+        return executor.submitIfRunning(runnable, "user defined task");
     }
 
     // This acquire a reference on the sstable
@@ -678,7 +690,7 @@
     /**
      * Does not mutate data, so is not scheduled.
      */
-    public Future<Object> submitValidation(final ColumnFamilyStore cfStore, final Validator validator)
+    public Future<?> submitValidation(final ColumnFamilyStore cfStore, final Validator validator)
     {
         Callable<Object> callable = new Callable<Object>()
         {
@@ -697,7 +709,8 @@
                 return this;
             }
         };
-        return validationExecutor.submit(callable);
+
+        return validationExecutor.submitIfRunning(callable, "validation");
     }
 
     /* Used in tests. */
@@ -710,20 +723,37 @@
         }
     }
 
-    private void scrubOne(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, boolean checkData) throws IOException
+    private void scrubOne(ColumnFamilyStore cfs, LifecycleTransaction modifier, boolean skipCorrupted, boolean checkData) throws IOException
     {
-        Scrubber scrubber = new Scrubber(cfs, sstable, skipCorrupted, false, checkData);
+        CompactionInfo.Holder scrubInfo = null;
 
-        CompactionInfo.Holder scrubInfo = scrubber.getScrubInfo();
-        metrics.beginCompaction(scrubInfo);
-        try
+        try (Scrubber scrubber = new Scrubber(cfs, modifier, skipCorrupted, checkData))
         {
+            scrubInfo = scrubber.getScrubInfo();
+            metrics.beginCompaction(scrubInfo);
             scrubber.scrub();
         }
         finally
         {
-            scrubber.close();
-            metrics.finishCompaction(scrubInfo);
+            if (scrubInfo != null)
+                metrics.finishCompaction(scrubInfo);
+        }
+    }
+
+    private void verifyOne(ColumnFamilyStore cfs, SSTableReader sstable, boolean extendedVerify) throws IOException
+    {
+        CompactionInfo.Holder verifyInfo = null;
+
+        try (Verifier verifier = new Verifier(cfs, sstable, false))
+        {
+            verifyInfo = verifier.getVerifyInfo();
+            metrics.beginCompaction(verifyInfo);
+            verifier.verify(extendedVerify);
+        }
+        finally
+        {
+            if (verifyInfo != null)
+                metrics.finishCompaction(verifyInfo);
         }
     }
 
@@ -787,20 +817,21 @@
      *
      * @throws IOException
      */
-    private void doCleanupOne(final ColumnFamilyStore cfs, SSTableReader sstable, CleanupStrategy cleanupStrategy, Collection<Range<Token>> ranges, boolean hasIndexes) throws IOException
+    private void doCleanupOne(final ColumnFamilyStore cfs, LifecycleTransaction txn, CleanupStrategy cleanupStrategy, Collection<Range<Token>> ranges, boolean hasIndexes) throws IOException
     {
         assert !cfs.isIndex();
 
-        Set<SSTableReader> sstableSet = Collections.singleton(sstable);
+        SSTableReader sstable = txn.onlyOne();
 
         if (!hasIndexes && !new Bounds<>(sstable.first.getToken(), sstable.last.getToken()).intersects(ranges))
         {
-            cfs.getDataTracker().markCompactedSSTablesReplaced(sstableSet, Collections.<SSTableReader>emptyList(), OperationType.CLEANUP);
+            txn.obsoleteOriginals();
+            txn.finish();
             return;
         }
         if (!needsCleanup(sstable, ranges))
         {
-            logger.debug("Skipping {} for cleanup; all rows should be kept", sstable);
+            logger.trace("Skipping {} for cleanup; all rows should be kept", sstable);
             return;
         }
 
@@ -809,13 +840,13 @@
         long totalkeysWritten = 0;
 
         long expectedBloomFilterSize = Math.max(cfs.metadata.getMinIndexInterval(),
-                                                SSTableReader.getApproximateKeyCount(sstableSet));
-        if (logger.isDebugEnabled())
-            logger.debug("Expected bloom filter size : {}", expectedBloomFilterSize);
+                                               SSTableReader.getApproximateKeyCount(txn.originals()));
+        if (logger.isTraceEnabled())
+            logger.trace("Expected bloom filter size : {}", expectedBloomFilterSize);
 
         logger.info("Cleaning up {}", sstable);
 
-        File compactionFileLocation = cfs.directories.getWriteableLocationAsFile(cfs.getExpectedCompactedFileSize(sstableSet, OperationType.CLEANUP));
+        File compactionFileLocation = cfs.directories.getWriteableLocationAsFile(cfs.getExpectedCompactedFileSize(txn.originals(), OperationType.CLEANUP));
         if (compactionFileLocation == null)
             throw new IOException("disk full");
 
@@ -823,10 +854,9 @@
         CleanupInfo ci = new CleanupInfo(sstable, scanner);
 
         metrics.beginCompaction(ci);
-        Set<SSTableReader> oldSSTable = Sets.newHashSet(sstable);
-        SSTableRewriter writer = new SSTableRewriter(cfs, oldSSTable, sstable.maxDataAge, false);
         List<SSTableReader> finished;
-        try (CompactionController controller = new CompactionController(cfs, sstableSet, getDefaultGcBefore(cfs)))
+        try (SSTableRewriter writer = new SSTableRewriter(cfs, txn, sstable.maxDataAge, false);
+             CompactionController controller = new CompactionController(cfs, txn.originals(), getDefaultGcBefore(cfs)))
         {
             writer.switchWriter(createWriter(cfs, compactionFileLocation, expectedBloomFilterSize, sstable.getSSTableMetadata().repairedAt, sstable));
 
@@ -835,10 +865,11 @@
                 if (ci.isStopRequested())
                     throw new CompactionInterruptedException(ci.getCompactionInfo());
 
-                SSTableIdentityIterator row = (SSTableIdentityIterator) scanner.next();
-                row = cleanupStrategy.cleanup(row);
+                @SuppressWarnings("resource")
+                SSTableIdentityIterator row = cleanupStrategy.cleanup((SSTableIdentityIterator) scanner.next());
                 if (row == null)
                     continue;
+                @SuppressWarnings("resource")
                 AbstractCompactedRow compactedRow = new LazilyCompactedRow(controller, Collections.singletonList(row));
                 if (writer.append(compactedRow) != null)
                     totalkeysWritten++;
@@ -848,12 +879,6 @@
             cfs.indexManager.flushIndexesBlocking();
 
             finished = writer.finish();
-            cfs.getDataTracker().markCompactedSSTablesReplaced(oldSSTable, finished, OperationType.CLEANUP);
-        }
-        catch (Throwable e)
-        {
-            writer.abort();
-            throw Throwables.propagate(e);
         }
         finally
         {
@@ -980,18 +1005,51 @@
                                              SSTableReader sstable)
     {
         FileUtils.createDirectory(compactionFileLocation);
-        return new SSTableWriter(cfs.getTempSSTablePath(compactionFileLocation),
-                                 expectedBloomFilterSize,
-                                 repairedAt,
-                                 cfs.metadata,
-                                 cfs.partitioner,
-                                 new MetadataCollector(Collections.singleton(sstable), cfs.metadata.comparator, sstable.getSSTableLevel()));
+
+        return SSTableWriter.create(cfs.metadata,
+                                    Descriptor.fromFilename(cfs.getTempSSTablePath(compactionFileLocation)),
+                                    expectedBloomFilterSize,
+                                    repairedAt,
+                                    sstable.getSSTableLevel(),
+                                    cfs.partitioner);
     }
 
+    public static SSTableWriter createWriterForAntiCompaction(ColumnFamilyStore cfs,
+                                             File compactionFileLocation,
+                                             int expectedBloomFilterSize,
+                                             long repairedAt,
+                                             Collection<SSTableReader> sstables)
+    {
+        FileUtils.createDirectory(compactionFileLocation);
+        int minLevel = Integer.MAX_VALUE;
+        // if all sstables have the same level, we can compact them together without creating overlap during anticompaction
+        // note that we only anticompact from unrepaired sstables, which is not leveled, but we still keep original level
+        // after first migration to be able to drop the sstables back in their original place in the repaired sstable manifest
+        for (SSTableReader sstable : sstables)
+        {
+            if (minLevel == Integer.MAX_VALUE)
+                minLevel = sstable.getSSTableLevel();
+
+            if (minLevel != sstable.getSSTableLevel())
+            {
+                minLevel = 0;
+                break;
+            }
+        }
+        return SSTableWriter.create(Descriptor.fromFilename(cfs.getTempSSTablePath(compactionFileLocation)),
+                                    (long) expectedBloomFilterSize,
+                                    repairedAt,
+                                    cfs.metadata,
+                                    cfs.partitioner,
+                                    new MetadataCollector(sstables, cfs.metadata.comparator, minLevel));
+    }
+
+
     /**
      * Performs a readonly "compaction" of all sstables in order to validate complete rows,
      * but without writing the merge result
      */
+    @SuppressWarnings("resource")
     private void doValidationCompaction(ColumnFamilyStore cfs, Validator validator) throws IOException
     {
         // this isn't meant to be race-proof, because it's not -- it won't cause bugs for a CFS to be dropped
@@ -1006,14 +1064,24 @@
         try
         {
 
-            String snapshotName = validator.desc.sessionId.toString();
             int gcBefore;
+            UUID parentRepairSessionId = validator.desc.parentSessionId;
+            String snapshotName;
+            boolean isGlobalSnapshotValidation = cfs.snapshotExists(parentRepairSessionId.toString());
+            if (isGlobalSnapshotValidation)
+                snapshotName = parentRepairSessionId.toString();
+            else
+                snapshotName = validator.desc.sessionId.toString();
             boolean isSnapshotValidation = cfs.snapshotExists(snapshotName);
+
             if (isSnapshotValidation)
             {
                 // If there is a snapshot created for the session then read from there.
+                // note that we populate the parent repair session when creating the snapshot, meaning the sstables in the snapshot are the ones we
+                // are supposed to validate.
                 sstables = cfs.getSnapshotSSTableReader(snapshotName);
 
+
                 // Computing gcbefore based on the current time wouldn't be very good because we know each replica will execute
                 // this at a different time (that's the whole purpose of repair with snaphsot). So instead we take the creation
                 // time of the snapshot, which should give us roughtly the same time on each replica (roughtly being in that case
@@ -1024,17 +1092,9 @@
             {
                 // flush first so everyone is validating data that is as similar as possible
                 StorageService.instance.forceKeyspaceFlush(cfs.keyspace.getName(), cfs.name);
-                // we don't mark validating sstables as compacting in DataTracker, so we have to mark them referenced
-                // instead so they won't be cleaned up if they do get compacted during the validation
-                if (validator.desc.parentSessionId == null || ActiveRepairService.instance.getParentRepairSession(validator.desc.parentSessionId) == null)
-                    sstables = cfs.selectAndReference(ColumnFamilyStore.CANONICAL_SSTABLES).refs;
-                else
-                {
-                    ActiveRepairService.ParentRepairSession prs = ActiveRepairService.instance.getParentRepairSession(validator.desc.parentSessionId);
-                    prs.markSSTablesRepairing(cfs.metadata.cfId, validator.desc.parentSessionId);
-                    sstables = cfs.selectAndReference(ColumnFamilyStore.UNREPAIRED_SSTABLES).refs;
-                }
-
+                sstables = getSSTablesToValidate(cfs, validator);
+                if (sstables == null)
+                    return; // this means the parent repair session was removed - the repair session failed on another node and we removed it
                 if (validator.gcBefore > 0)
                     gcBefore = validator.gcBefore;
                 else
@@ -1046,10 +1106,10 @@
             long numPartitions = 0;
             for (SSTableReader sstable : sstables)
             {
-                numPartitions += sstable.estimatedKeysForRanges(Collections.singleton(validator.desc.range));
+                numPartitions += sstable.estimatedKeysForRanges(singleton(validator.desc.range));
             }
-            // determine tree depth from number of partitions, but cap at 20 to prevent large tree.
-            int depth = numPartitions > 0 ? (int) Math.min(Math.floor(Math.log(numPartitions)), 20) : 0;
+            // determine tree depth from number of partitions, but cap at 20 to prevent large tree (CASSANDRA-5263)
+            int depth = numPartitions > 0 ? (int) Math.min(Math.ceil(Math.log(numPartitions) / Math.log(2)), 20) : 0;
             MerkleTree tree = new MerkleTree(cfs.partitioner, validator.desc.range, MerkleTree.RECOMMENDED_DEPTH, (int) Math.pow(2, depth));
 
             long start = System.nanoTime();
@@ -1073,7 +1133,9 @@
                 }
                 finally
                 {
-                    if (isSnapshotValidation)
+                    // we can only clear the snapshot if we are not doing a global snapshot validation (we then clear it once anticompaction
+                    // is done).
+                    if (isSnapshotValidation && !isGlobalSnapshotValidation)
                     {
                         cfs.clearSnapshot(snapshotName);
                     }
@@ -1082,11 +1144,11 @@
                 }
             }
 
-            if (logger.isDebugEnabled())
+            if (logger.isTraceEnabled())
             {
                 // MT serialize may take time
                 long duration = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
-                logger.debug("Validation finished in {} msec, depth {} for {} keys, serialized size {} bytes for {}",
+                logger.trace("Validation finished in {} msec, depth {} for {} keys, serialized size {} bytes for {}",
                              duration,
                              depth,
                              numPartitions,
@@ -1101,95 +1163,175 @@
         }
     }
 
+    private synchronized Refs<SSTableReader> getSSTablesToValidate(ColumnFamilyStore cfs, Validator validator)
+    {
+        Refs<SSTableReader> sstables;
+
+        ActiveRepairService.ParentRepairSession prs = ActiveRepairService.instance.getParentRepairSession(validator.desc.parentSessionId);
+        if (prs == null)
+            return null;
+        Set<SSTableReader> sstablesToValidate = new HashSet<>();
+
+        if (prs.isGlobal)
+            prs.markSSTablesRepairing(cfs.metadata.cfId, validator.desc.parentSessionId);
+
+        // note that we always grab all existing sstables for this - if we were to just grab the ones that
+        // were marked as repairing, we would miss any ranges that were compacted away and this would cause us to overstream
+        try (ColumnFamilyStore.RefViewFragment sstableCandidates = cfs.selectAndReference(prs.isIncremental ? ColumnFamilyStore.UNREPAIRED_SSTABLES : ColumnFamilyStore.CANONICAL_SSTABLES))
+        {
+            for (SSTableReader sstable : sstableCandidates.sstables)
+            {
+                if (new Bounds<>(sstable.first.getToken(), sstable.last.getToken()).intersects(Collections.singletonList(validator.desc.range)))
+                {
+                    sstablesToValidate.add(sstable);
+                }
+            }
+
+            sstables = Refs.tryRef(sstablesToValidate);
+            if (sstables == null)
+            {
+                logger.error("Could not reference sstables");
+                throw new RuntimeException("Could not reference sstables");
+            }
+        }
+
+        return sstables;
+    }
+
     /**
      * Splits up an sstable into two new sstables. The first of the new tables will store repaired ranges, the second
      * will store the non-repaired ranges. Once anticompation is completed, the original sstable is marked as compacted
      * and subsequently deleted.
      * @param cfs
-     * @param repairedSSTables
+     * @param repaired a transaction over the repaired sstables to anticompacy
      * @param ranges Repaired ranges to be placed into one of the new sstables. The repaired table will be tracked via
      * the {@link org.apache.cassandra.io.sstable.metadata.StatsMetadata#repairedAt} field.
      */
-    private Collection<SSTableReader> doAntiCompaction(ColumnFamilyStore cfs, Collection<Range<Token>> ranges, Collection<SSTableReader> repairedSSTables, long repairedAt)
+    private void doAntiCompaction(ColumnFamilyStore cfs, Collection<Range<Token>> ranges, LifecycleTransaction repaired, long repairedAt)
     {
-        List<SSTableReader> anticompactedSSTables = new ArrayList<>();
-        int repairedKeyCount = 0;
-        int unrepairedKeyCount = 0;
-        logger.info("Performing anticompaction on {} sstables", repairedSSTables.size());
-        // iterate over sstables to check if the repaired / unrepaired ranges intersect them.
-        for (SSTableReader sstable : repairedSSTables)
+        logger.info("Performing anticompaction on {} sstables", repaired.originals().size());
+
+        //Group SSTables
+        Set<SSTableReader> sstables = repaired.originals();
+
+        // Repairs can take place on both unrepaired (incremental + full) and repaired (full) data.
+        // Although anti-compaction could work on repaired sstables as well and would result in having more accurate
+        // repairedAt values for these, we still avoid anti-compacting already repaired sstables, as we currently don't
+        // make use of any actual repairedAt value and splitting up sstables just for that is not worth it at this point.
+        Set<SSTableReader> unrepairedSSTables = ImmutableSet.copyOf(Iterables.filter(sstables, new Predicate<SSTableReader>()
         {
-            // check that compaction hasn't stolen any sstables used in previous repair sessions
-            // if we need to skip the anticompaction, it will be carried out by the next repair
-            if (!new File(sstable.getFilename()).exists())
+            public boolean apply(SSTableReader input)
             {
-                logger.info("Skipping anticompaction for {}, required sstable was compacted and is no longer available.", sstable);
-                continue;
+                return !input.isRepaired();
             }
+        }));
 
-            logger.info("Anticompacting {}", sstable);
-            Set<SSTableReader> sstableAsSet = Sets.newHashSet(sstable);
-
-            File destination = cfs.directories.getWriteableLocationAsFile(cfs.getExpectedCompactedFileSize(sstableAsSet, OperationType.ANTICOMPACTION));
-            SSTableRewriter repairedSSTableWriter = new SSTableRewriter(cfs, sstableAsSet, sstable.maxDataAge, false, false);
-            SSTableRewriter unRepairedSSTableWriter = new SSTableRewriter(cfs, sstableAsSet, sstable.maxDataAge, false, false);
-
-            try (AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategy().getScanners(new HashSet<>(Collections.singleton(sstable)));
-                 CompactionController controller = new CompactionController(cfs, sstableAsSet, getDefaultGcBefore(cfs)))
+        Collection<Collection<SSTableReader>> groupedSSTables = cfs.getCompactionStrategy().groupSSTablesForAntiCompaction(unrepairedSSTables);
+        // iterate over sstables to check if the repaired / unrepaired ranges intersect them.
+        int antiCompactedSSTableCount = 0;
+        for (Collection<SSTableReader> sstableGroup : groupedSSTables)
+        {
+            try (LifecycleTransaction txn = repaired.split(sstableGroup))
             {
-                int expectedBloomFilterSize = Math.max(cfs.metadata.getMinIndexInterval(), (int)sstable.estimatedKeys());
-                repairedSSTableWriter.switchWriter(CompactionManager.createWriter(cfs, destination, expectedBloomFilterSize, repairedAt, sstable));
-                unRepairedSSTableWriter.switchWriter(CompactionManager.createWriter(cfs, destination, expectedBloomFilterSize, ActiveRepairService.UNREPAIRED_SSTABLE, sstable));
-
-                CompactionIterable ci = new CompactionIterable(OperationType.ANTICOMPACTION, scanners.scanners, controller);
-                Iterator<AbstractCompactedRow> iter = ci.iterator();
-                metrics.beginCompaction(ci);
-                try
-                {
-                    Range.OrderedRangeContainmentChecker containmentChecker = new Range.OrderedRangeContainmentChecker(ranges);
-                    while (iter.hasNext())
-                    {
-                        AbstractCompactedRow row = iter.next();
-                        if (containmentChecker.contains(row.key.getToken()))
-                        {
-                            repairedSSTableWriter.append(row);
-                            repairedKeyCount++;
-                        }
-                        // otherwise save into the new 'non-repaired' table
-                        else
-                        {
-                            unRepairedSSTableWriter.append(row);
-                            unrepairedKeyCount++;
-                        }
-                    }
-                }
-                finally
-                {
-                    metrics.finishCompaction(ci);
-                }
-
-                List<SSTableReader> anticompacted = new ArrayList<>();
-                anticompacted.addAll(repairedSSTableWriter.finish(repairedAt));
-                anticompacted.addAll(unRepairedSSTableWriter.finish(ActiveRepairService.UNREPAIRED_SSTABLE));
-                anticompactedSSTables.addAll(anticompacted);
-
-                cfs.getDataTracker().markCompactedSSTablesReplaced(sstableAsSet, anticompacted, OperationType.ANTICOMPACTION);
-                cfs.getDataTracker().unmarkCompacting(sstableAsSet);
-            }
-            catch (Throwable e)
-            {
-                JVMStabilityInspector.inspectThrowable(e);
-                logger.error("Error anticompacting " + sstable, e);
-                repairedSSTableWriter.abort();
-                unRepairedSSTableWriter.abort();
+                int antiCompacted = antiCompactGroup(cfs, ranges, txn, repairedAt);
+                antiCompactedSSTableCount += antiCompacted;
             }
         }
-        String format = "Repaired {} keys of {} for {}/{}";
-        logger.debug(format, repairedKeyCount, (repairedKeyCount + unrepairedKeyCount), cfs.keyspace, cfs.getColumnFamilyName());
-        String format2 = "Anticompaction completed successfully, anticompacted from {} to {} sstable(s).";
-        logger.info(format2, repairedSSTables.size(), anticompactedSSTables.size());
 
-        return anticompactedSSTables;
+        String format = "Anticompaction completed successfully, anticompacted from {} to {} sstable(s).";
+        logger.info(format, repaired.originals().size(), antiCompactedSSTableCount);
+    }
+
+    private int antiCompactGroup(ColumnFamilyStore cfs, Collection<Range<Token>> ranges,
+                             LifecycleTransaction anticompactionGroup, long repairedAt)
+    {
+        long groupMaxDataAge = -1;
+
+        for (Iterator<SSTableReader> i = anticompactionGroup.originals().iterator(); i.hasNext();)
+        {
+            SSTableReader sstable = i.next();
+            if (groupMaxDataAge < sstable.maxDataAge)
+                groupMaxDataAge = sstable.maxDataAge;
+        }
+
+        if (anticompactionGroup.originals().size() == 0)
+        {
+            logger.info("No valid anticompactions for this group, All sstables were compacted and are no longer available");
+            return 0;
+        }
+
+        logger.info("Anticompacting {}", anticompactionGroup);
+        Set<SSTableReader> sstableAsSet = anticompactionGroup.originals();
+
+        File destination = cfs.directories.getWriteableLocationAsFile(cfs.getExpectedCompactedFileSize(sstableAsSet, OperationType.ANTICOMPACTION));
+        long repairedKeyCount = 0;
+        long unrepairedKeyCount = 0;
+        AbstractCompactionStrategy strategy = cfs.getCompactionStrategy();
+        try (SSTableRewriter repairedSSTableWriter = new SSTableRewriter(cfs, anticompactionGroup, groupMaxDataAge, false, false);
+             SSTableRewriter unRepairedSSTableWriter = new SSTableRewriter(cfs, anticompactionGroup, groupMaxDataAge, false, false);
+             AbstractCompactionStrategy.ScannerList scanners = strategy.getScanners(anticompactionGroup.originals());
+             CompactionController controller = new CompactionController(cfs, sstableAsSet, getDefaultGcBefore(cfs)))
+        {
+            int expectedBloomFilterSize = Math.max(cfs.metadata.getMinIndexInterval(), (int)(SSTableReader.getApproximateKeyCount(sstableAsSet)));
+
+            repairedSSTableWriter.switchWriter(CompactionManager.createWriterForAntiCompaction(cfs, destination, expectedBloomFilterSize, repairedAt, sstableAsSet));
+            unRepairedSSTableWriter.switchWriter(CompactionManager.createWriterForAntiCompaction(cfs, destination, expectedBloomFilterSize, ActiveRepairService.UNREPAIRED_SSTABLE, sstableAsSet));
+
+            CompactionIterable ci = new CompactionIterable(OperationType.ANTICOMPACTION, scanners.scanners, controller, DatabaseDescriptor.getSSTableFormat(), UUIDGen.getTimeUUID());
+            metrics.beginCompaction(ci);
+            try
+            {
+                @SuppressWarnings("resource")
+                CloseableIterator<AbstractCompactedRow> iter = ci.iterator();
+                Range.OrderedRangeContainmentChecker containmentChecker = new Range.OrderedRangeContainmentChecker(ranges);
+                while (iter.hasNext())
+                {
+                    @SuppressWarnings("resource")
+                    AbstractCompactedRow row = iter.next();
+                    // if current range from sstable is repaired, save it into the new repaired sstable
+                    if (containmentChecker.contains(row.key.getToken()))
+                    {
+                        repairedSSTableWriter.append(row);
+                        repairedKeyCount++;
+                    }
+                    // otherwise save into the new 'non-repaired' table
+                    else
+                    {
+                        unRepairedSSTableWriter.append(row);
+                        unrepairedKeyCount++;
+                    }
+                }
+            }
+            finally
+            {
+                metrics.finishCompaction(ci);
+            }
+
+            List<SSTableReader> anticompactedSSTables = new ArrayList<>();
+            // since both writers are operating over the same Transaction, we cannot use the convenience Transactional.finish() method,
+            // as on the second finish() we would prepareToCommit() on a Transaction that has already been committed, which is forbidden by the API
+            // (since it indicates misuse). We call permitRedundantTransitions so that calls that transition to a state already occupied are permitted.
+            anticompactionGroup.permitRedundantTransitions();
+            repairedSSTableWriter.setRepairedAt(repairedAt).prepareToCommit();
+            unRepairedSSTableWriter.prepareToCommit();
+            anticompactedSSTables.addAll(repairedSSTableWriter.finished());
+            anticompactedSSTables.addAll(unRepairedSSTableWriter.finished());
+            repairedSSTableWriter.commit();
+            unRepairedSSTableWriter.commit();
+
+            logger.trace("Repaired {} keys out of {} for {}/{} in {}", repairedKeyCount,
+                                                                       repairedKeyCount + unrepairedKeyCount,
+                                                                       cfs.keyspace.getName(),
+                                                                       cfs.getColumnFamilyName(),
+                                                                       anticompactionGroup);
+            return anticompactedSSTables.size();
+        }
+        catch (Throwable e)
+        {
+            JVMStabilityInspector.inspectThrowable(e);
+            logger.error("Error anticompacting " + anticompactionGroup, e);
+        }
+        return 0;
     }
 
     /**
@@ -1212,13 +1354,8 @@
                 }
             }
         };
-        if (executor.isShutdown())
-        {
-            logger.info("Compaction executor has shut down, not submitting index build");
-            return null;
-        }
 
-        return executor.submit(runnable);
+        return executor.submitIfRunning(runnable, "index build");
     }
 
     public Future<?> submitCacheWrite(final AutoSavingCache.Writer writer)
@@ -1229,7 +1366,7 @@
             {
                 if (!AutoSavingCache.flushInProgress.add(writer.cacheType()))
                 {
-                    logger.debug("Cache flushing was already in progress: skipping {}", writer.getCompactionInfo());
+                    logger.trace("Cache flushing was already in progress: skipping {}", writer.getCompactionInfo());
                     return;
                 }
                 try
@@ -1250,12 +1387,8 @@
                 }
             }
         };
-        if (executor.isShutdown())
-        {
-            logger.info("Executor has shut down, not submitting background task");
-            Futures.immediateCancelledFuture();
-        }
-        return executor.submit(runnable);
+
+        return executor.submitIfRunning(runnable, "cache write");
     }
 
     public List<SSTableReader> runIndexSummaryRedistribution(IndexSummaryRedistribution redistribution) throws IOException
@@ -1272,7 +1405,7 @@
         }
     }
 
-    static int getDefaultGcBefore(ColumnFamilyStore cfs)
+    public static int getDefaultGcBefore(ColumnFamilyStore cfs)
     {
         // 2ndary indexes have ExpiringColumns too, so we need to purge tombstones deleted before now. We do not need to
         // add any GcGrace however since 2ndary indexes are local to a node.
@@ -1281,9 +1414,10 @@
 
     private static class ValidationCompactionIterable extends CompactionIterable
     {
+        @SuppressWarnings("resource")
         public ValidationCompactionIterable(ColumnFamilyStore cfs, List<ISSTableScanner> scanners, int gcBefore)
         {
-            super(OperationType.VALIDATION, scanners, new ValidationCompactionController(cfs, gcBefore));
+            super(OperationType.VALIDATION, scanners, new ValidationCompactionController(cfs, gcBefore), DatabaseDescriptor.getSSTableFormat(), UUIDGen.getTimeUUID());
         }
     }
 
@@ -1301,7 +1435,7 @@
         }
 
         @Override
-        public long maxPurgeableTimestamp(DecoratedKey key)
+        public Predicate<Long> getPurgeEvaluator(DecoratedKey key)
         {
             /*
              * The main reason we always purge is that including gcable tombstone would mean that the
@@ -1314,7 +1448,7 @@
              * a tombstone that could shadow a column in another sstable, but this is doubly not a concern
              * since validation compaction is read-only.
              */
-            return Long.MAX_VALUE;
+            return Predicates.alwaysTrue();
         }
     }
 
@@ -1364,13 +1498,57 @@
                     if (t.getSuppressed() != null && t.getSuppressed().length > 0)
                         logger.warn("Interruption of compaction encountered exceptions:", t);
                     else
-                        logger.debug("Full interruption stack trace:", t);
+                        logger.trace("Full interruption stack trace:", t);
                 }
                 else
                 {
                     DebuggableThreadPoolExecutor.handleOrLog(t);
                 }
             }
+
+            // Snapshots cannot be deleted on Windows while segments of the root element are mapped in NTFS. Compactions
+            // unmap those segments which could free up a snapshot for successful deletion.
+            SnapshotDeletingTask.rescheduleFailedTasks();
+        }
+
+        public ListenableFuture<?> submitIfRunning(Runnable task, String name)
+        {
+            return submitIfRunning(Executors.callable(task, null), name);
+        }
+
+        /**
+         * Submit the task but only if the executor has not been shutdown.If the executor has
+         * been shutdown, or in case of a rejected execution exception return a cancelled future.
+         *
+         * @param task - the task to submit
+         * @param name - the task name to use in log messages
+         *
+         * @return the future that will deliver the task result, or a future that has already been
+         *         cancelled if the task could not be submitted.
+         */
+        public ListenableFuture<?> submitIfRunning(Callable<?> task, String name)
+        {
+            if (isShutdown())
+            {
+                logger.info("Executor has been shut down, not submitting {}", name);
+                return Futures.immediateCancelledFuture();
+            }
+
+            try
+            {
+                ListenableFutureTask ret = ListenableFutureTask.create(task);
+                submit(ret);
+                return ret;
+            }
+            catch (RejectedExecutionException ex)
+            {
+                if (isShutdown())
+                    logger.info("Executor has shut down, could not submit {}", name);
+                else
+                    logger.error("Failed to submit {}", name, ex);
+
+                return Futures.immediateCancelledFuture();
+            }
         }
     }
 
@@ -1429,33 +1607,35 @@
 
     public long getTotalBytesCompacted()
     {
-        return metrics.bytesCompacted.count();
+        return metrics.bytesCompacted.getCount();
     }
 
     public long getTotalCompactionsCompleted()
     {
-        return metrics.totalCompactionsCompleted.count();
+        return metrics.totalCompactionsCompleted.getCount();
     }
 
     public int getPendingTasks()
     {
-        return metrics.pendingTasks.value();
+        return metrics.pendingTasks.getValue();
     }
 
     public long getCompletedTasks()
     {
-        return metrics.completedTasks.value();
+        return metrics.completedTasks.getValue();
     }
 
     private static class CleanupInfo extends CompactionInfo.Holder
     {
         private final SSTableReader sstable;
         private final ISSTableScanner scanner;
+        private final UUID cleanupCompactionId;
 
         public CleanupInfo(SSTableReader sstable, ISSTableScanner scanner)
         {
             this.sstable = sstable;
             this.scanner = scanner;
+            cleanupCompactionId = UUIDGen.getTimeUUID();
         }
 
         public CompactionInfo getCompactionInfo()
@@ -1465,7 +1645,8 @@
                 return new CompactionInfo(sstable.metadata,
                                           OperationType.CLEANUP,
                                           scanner.getCurrentPosition(),
-                                          scanner.getLengthInBytes());
+                                          scanner.getLengthInBytes(),
+                                          cleanupCompactionId);
             }
             catch (Exception e)
             {
@@ -1484,6 +1665,16 @@
         }
     }
 
+    public void stopCompactionById(String compactionId)
+    {
+        for (Holder holder : CompactionMetrics.getCompactions())
+        {
+            UUID holderId = holder.getCompactionInfo().compactionId();
+            if (holderId != null && holderId.equals(UUID.fromString(compactionId)))
+                holder.stop();
+        }
+    }
+
     public int getCoreCompactorThreads()
     {
         return executor.getCorePoolSize();
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionManagerMBean.java b/src/java/org/apache/cassandra/db/compaction/CompactionManagerMBean.java
index acf1e52..8e200a1 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManagerMBean.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManagerMBean.java
@@ -19,6 +19,7 @@
 
 import java.util.List;
 import java.util.Map;
+import java.util.UUID;
 import javax.management.openmbean.TabularData;
 
 public interface CompactionManagerMBean
@@ -33,40 +34,13 @@
     public TabularData getCompactionHistory();
 
     /**
-     * @see org.apache.cassandra.metrics.CompactionMetrics#pendingTasks
-     * @return estimated number of compactions remaining to perform
-     */
-    @Deprecated
-    public int getPendingTasks();
-
-    /**
-     * @see org.apache.cassandra.metrics.CompactionMetrics#completedTasks
-     * @return number of completed compactions since server [re]start
-     */
-    @Deprecated
-    public long getCompletedTasks();
-
-    /**
-     * @see org.apache.cassandra.metrics.CompactionMetrics#bytesCompacted
-     * @return total number of bytes compacted since server [re]start
-     */
-    @Deprecated
-    public long getTotalBytesCompacted();
-
-    /**
-     * @see org.apache.cassandra.metrics.CompactionMetrics#totalCompactionsCompleted
-     * @return total number of compactions since server [re]start
-     */
-    @Deprecated
-    public long getTotalCompactionsCompleted();
-
-    /**
      * Triggers the compaction of user specified sstables.
      * You can specify files from various keyspaces and columnfamilies.
      * If you do so, user defined compaction is performed several times to the groups of files
      * in the same keyspace/columnfamily.
      *
-     * @param dataFiles a comma separated list of sstable filename to compact
+     * @param dataFiles a comma separated list of sstable file to compact.
+     *                  must contain keyspace and columnfamily name in path(for 2.1+) or file name itself.
      */
     public void forceUserDefinedCompaction(String dataFiles);
 
@@ -82,6 +56,13 @@
     public void stopCompaction(String type);
 
     /**
+     * Stop an individual running compaction using the compactionId.
+     * @param compactionId Compaction ID of compaction to stop. Such IDs can be found in
+     *                     the compactions_in_progress table of the system keyspace.
+     */
+    public void stopCompactionById(String compactionId);
+
+    /**
      * Returns core size of compaction thread pool
      */
     public int getCoreCompactorThreads();
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
index c6e3d2f..006c8ff 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
@@ -17,11 +17,9 @@
  */
 package org.apache.cassandra.db.compaction;
 
-import java.io.File;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashMap;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -31,6 +29,11 @@
 import com.google.common.base.Predicate;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Sets;
+
+import org.apache.cassandra.db.compaction.writers.CompactionAwareWriter;
+import org.apache.cassandra.db.compaction.writers.DefaultCompactionWriter;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -39,12 +42,10 @@
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.db.compaction.CompactionManager.CompactionExecutorStatsCollector;
-import org.apache.cassandra.io.sstable.SSTableReader;
-import org.apache.cassandra.io.sstable.SSTableRewriter;
-import org.apache.cassandra.io.sstable.SSTableWriter;
-import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.utils.CloseableIterator;
+import org.apache.cassandra.utils.UUIDGen;
 import org.apache.cassandra.utils.concurrent.Refs;
 
 public class CompactionTask extends AbstractCompactionTask
@@ -55,9 +56,9 @@
     protected static long totalBytesCompacted = 0;
     private CompactionExecutorStatsCollector collector;
 
-    public CompactionTask(ColumnFamilyStore cfs, Iterable<SSTableReader> sstables, int gcBefore, boolean offline)
+    public CompactionTask(ColumnFamilyStore cfs, LifecycleTransaction txn, int gcBefore, boolean offline)
     {
-        super(cfs, Sets.newHashSet(sstables));
+        super(cfs, txn);
         this.gcBefore = gcBefore;
         this.offline = offline;
     }
@@ -71,28 +72,22 @@
     {
         this.collector = collector;
         run();
-        return sstables.size();
+        return transaction.originals().size();
     }
 
-    public long getExpectedWriteSize()
+    public boolean reduceScopeForLimitedSpace(long expectedSize)
     {
-        return cfs.getExpectedCompactedFileSize(sstables, compactionType);
-    }
-
-    public boolean reduceScopeForLimitedSpace()
-    {
-        if (partialCompactionsAcceptable() && sstables.size() > 1)
+        if (partialCompactionsAcceptable() && transaction.originals().size() > 1)
         {
             // Try again w/o the largest one.
-            logger.warn("insufficient space to compact all requested files {}", StringUtils.join(sstables, ", "));
+            logger.warn("insufficient space to compact all requested files. {}MB required, {}",
+                        (float) expectedSize / 1024 / 1024,
+                        StringUtils.join(transaction.originals(), ", "));
             // Note that we have removed files that are still marked as compacting.
             // This suboptimal but ok since the caller will unmark all the sstables at the end.
-            SSTableReader removedSSTable = cfs.getMaxSizeFile(sstables);
-            if (sstables.remove(removedSSTable))
-            {
-                cfs.getDataTracker().unmarkCompacting(Arrays.asList(removedSSTable));
-                return true;
-            }
+            SSTableReader removedSSTable = cfs.getMaxSizeFile(transaction.originals());
+            transaction.cancel(removedSSTable);
+            return true;
         }
         return false;
     }
@@ -106,9 +101,9 @@
     {
         // The collection of sstables passed may be empty (but not null); even if
         // it is not empty, it may compact down to nothing if all rows are deleted.
-        assert sstables != null;
+        assert transaction != null;
 
-        if (sstables.size() == 0)
+        if (transaction.originals().isEmpty())
             return;
 
         // Note that the current compaction strategy, is not necessarily the one this task was created under.
@@ -120,11 +115,11 @@
 
         // note that we need to do a rough estimate early if we can fit the compaction on disk - this is pessimistic, but
         // since we might remove sstables from the compaction in checkAvailableDiskSpace it needs to be done here
-        long earlySSTableEstimate = Math.max(1, cfs.getExpectedCompactedFileSize(sstables, compactionType) / strategy.getMaxSSTableBytes());
-        checkAvailableDiskSpace(earlySSTableEstimate);
+
+        checkAvailableDiskSpace();
 
         // sanity check: all sstables must belong to the same cfs
-        assert !Iterables.any(sstables, new Predicate<SSTableReader>()
+        assert !Iterables.any(transaction.originals(), new Predicate<SSTableReader>()
         {
             @Override
             public boolean apply(SSTableReader sstable)
@@ -133,26 +128,30 @@
             }
         });
 
-        UUID taskId = SystemKeyspace.startCompaction(cfs, sstables);
+        UUID taskId = offline ? null : SystemKeyspace.startCompaction(cfs, transaction.originals());
 
         // new sstables from flush can be added during a compaction, but only the compaction can remove them,
         // so in our single-threaded compaction world this is a valid way of determining if we're compacting
         // all the sstables (that existed when we started)
-        logger.info("Compacting {}", sstables);
+        StringBuilder ssTableLoggerMsg = new StringBuilder("[");
+        for (SSTableReader sstr : transaction.originals())
+        {
+            ssTableLoggerMsg.append(String.format("%s:level=%d, ", sstr.getFilename(), sstr.getSSTableLevel()));
+        }
+        ssTableLoggerMsg.append("]");
+        String taskIdLoggerMsg = taskId == null ? UUIDGen.getTimeUUID().toString() : taskId.toString();
+        logger.debug("Compacting ({}) {}", taskIdLoggerMsg, ssTableLoggerMsg);
 
         long start = System.nanoTime();
 
         long totalKeysWritten = 0;
 
-        try (CompactionController controller = getCompactionController(sstables);)
+        long estimatedKeys = 0;
+        try (CompactionController controller = getCompactionController(transaction.originals()))
         {
-            Set<SSTableReader> actuallyCompact = Sets.difference(sstables, controller.getFullyExpiredSSTables());
+            Set<SSTableReader> actuallyCompact = Sets.difference(transaction.originals(), controller.getFullyExpiredSSTables());
 
-            long estimatedTotalKeys = Math.max(cfs.metadata.getMinIndexInterval(), SSTableReader.getApproximateKeyCount(actuallyCompact));
-            long estimatedSSTables = Math.max(1, cfs.getExpectedCompactedFileSize(actuallyCompact, compactionType) / strategy.getMaxSSTableBytes());
-            long keysPerSSTable = (long) Math.ceil((double) estimatedTotalKeys / estimatedSSTables);
-            long expectedSSTableSize = Math.min(getExpectedWriteSize(), strategy.getMaxSSTableBytes());
-            logger.debug("Expected bloom filter size : {}", keysPerSSTable);
+            SSTableFormat.Type sstableFormat = getFormatType(transaction.originals());
 
             List<SSTableReader> newSStables;
             AbstractCompactionIterable ci;
@@ -163,89 +162,57 @@
             try (Refs<SSTableReader> refs = Refs.ref(actuallyCompact);
                  AbstractCompactionStrategy.ScannerList scanners = strategy.getScanners(actuallyCompact))
             {
-                ci = new CompactionIterable(compactionType, scanners.scanners, controller);
-                Iterator<AbstractCompactedRow> iter = ci.iterator();
-                // we can't preheat until the tracker has been set. This doesn't happen until we tell the cfs to
-                // replace the old entries.  Track entries to preheat here until then.
-                long minRepairedAt = getMinRepairedAt(actuallyCompact);
-                // we only need the age of the data that we're actually retaining
-                long maxAge = getMaxDataAge(actuallyCompact);
-                if (collector != null)
-                    collector.beginCompaction(ci);
-                long lastCheckObsoletion = start;
-                SSTableRewriter writer = new SSTableRewriter(cfs, sstables, maxAge, offline);
-                try
+                ci = new CompactionIterable(compactionType, scanners.scanners, controller, sstableFormat, taskId);
+                try (CloseableIterator<AbstractCompactedRow> iter = ci.iterator())
                 {
+                    long lastCheckObsoletion = start;
+
                     if (!controller.cfs.getCompactionStrategy().isActive)
-                       throw new CompactionInterruptedException(ci.getCompactionInfo());
-                    if (!iter.hasNext())
-                    {
-                        // don't mark compacted in the finally block, since if there _is_ nondeleted data,
-                        // we need to sync it (via closeAndOpen) first, so there is no period during which
-                        // a crash could cause data loss.
-                        cfs.markObsolete(sstables, compactionType);
-                        return;
-                    }
+                        throw new CompactionInterruptedException(ci.getCompactionInfo());
 
-                    writer.switchWriter(createCompactionWriter(cfs.directories.getLocationForDisk(getWriteDirectory(expectedSSTableSize)), keysPerSSTable, minRepairedAt));
-                    while (iter.hasNext())
-                    {
-                        if (ci.isStopRequested())
-                            throw new CompactionInterruptedException(ci.getCompactionInfo());
+                    if (collector != null)
+                        collector.beginCompaction(ci);
 
-                        AbstractCompactedRow row = iter.next();
-                        if (writer.append(row) != null)
+                    try (CompactionAwareWriter writer = getCompactionAwareWriter(cfs, transaction, actuallyCompact))
+                    {
+                        estimatedKeys = writer.estimatedKeys();
+                        while (iter.hasNext())
                         {
-                            totalKeysWritten++;
-                            if (newSSTableSegmentThresholdReached(writer.currentWriter()))
+                            if (ci.isStopRequested())
+                                throw new CompactionInterruptedException(ci.getCompactionInfo());
+
+                            try (AbstractCompactedRow row = iter.next())
                             {
-                                writer.switchWriter(createCompactionWriter(cfs.directories.getLocationForDisk(getWriteDirectory(expectedSSTableSize)), keysPerSSTable, minRepairedAt));
+                                if (writer.append(row))
+                                    totalKeysWritten++;
+
+                                if (System.nanoTime() - lastCheckObsoletion > TimeUnit.MINUTES.toNanos(1L))
+                                {
+                                    controller.maybeRefreshOverlaps();
+                                    lastCheckObsoletion = System.nanoTime();
+                                }
                             }
                         }
 
-                        if (System.nanoTime() - lastCheckObsoletion > TimeUnit.MINUTES.toNanos(1L))
-                        {
-                            controller.maybeRefreshOverlaps();
-                            lastCheckObsoletion = System.nanoTime();
-                        }
+                        // don't replace old sstables yet, as we need to mark the compaction finished in the system table
+                        newSStables = writer.finish();
                     }
-
-                    // don't replace old sstables yet, as we need to mark the compaction finished in the system table
-                    newSStables = writer.finish();
-                }
-                catch (Throwable t)
-                {
-                    try
+                    finally
                     {
-                        writer.abort();
-                    }
-                    catch (Throwable t2)
-                    {
-                        t.addSuppressed(t2);
-                    }
-                    throw t;
-                }
-                finally
-                {
-                    // point of no return -- the new sstables are live on disk; next we'll start deleting the old ones
-                    // (in replaceCompactedSSTables)
-                    if (taskId != null)
-                        SystemKeyspace.finishCompaction(taskId);
+                        // point of no return -- the new sstables are live on disk; next we'll start deleting the old ones
+                        // (in replaceCompactedSSTables)
+                        if (taskId != null)
+                            SystemKeyspace.finishCompaction(taskId);
 
-                    if (collector != null)
-                        collector.finishCompaction(ci);
+                        if (collector != null)
+                            collector.finishCompaction(ci);
+                    }
                 }
             }
 
-            Collection<SSTableReader> oldSStables = this.sstables;
-            if (!offline)
-                cfs.getDataTracker().markCompactedSSTablesReplaced(oldSStables, newSStables, compactionType);
-            else
-                Refs.release(Refs.selfRefs(newSStables));
-
             // log a bunch of statistics about the result and save to system table compaction_history
             long dTime = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
-            long startsize = SSTableReader.getTotalBytes(oldSStables);
+            long startsize = SSTableReader.getTotalBytes(transaction.originals());
             long endsize = SSTableReader.getTotalBytes(newSStables);
             double ratio = (double) endsize / (double) startsize;
 
@@ -253,32 +220,50 @@
             for (SSTableReader reader : newSStables)
                 newSSTableNames.append(reader.descriptor.baseFilename()).append(",");
 
-            double mbps = dTime > 0 ? (double) endsize / (1024 * 1024) / ((double) dTime / 1000) : 0;
-            long totalSourceRows = 0;
-            long[] counts = ci.getMergedRowCounts();
-            StringBuilder mergeSummary = new StringBuilder(counts.length * 10);
-            Map<Integer, Long> mergedRows = new HashMap<>();
-            for (int i = 0; i < counts.length; i++)
+            if (offline)
             {
-                long count = counts[i];
-                if (count == 0)
-                    continue;
-
-                int rows = i + 1;
-                totalSourceRows += rows * count;
-                mergeSummary.append(String.format("%d:%d, ", rows, count));
-                mergedRows.put(rows, count);
+                Refs.release(Refs.selfRefs(newSStables));
             }
-
-            SystemKeyspace.updateCompactionHistory(cfs.keyspace.getName(), cfs.name, System.currentTimeMillis(), startsize, endsize, mergedRows);
-            logger.info(String.format("Compacted %d sstables to [%s].  %,d bytes to %,d (~%d%% of original) in %,dms = %fMB/s.  %,d total partitions merged to %,d.  Partition merge counts were {%s}",
-                                      oldSStables.size(), newSSTableNames.toString(), startsize, endsize, (int) (ratio * 100), dTime, mbps, totalSourceRows, totalKeysWritten, mergeSummary.toString()));
-            logger.debug(String.format("CF Total Bytes Compacted: %,d", CompactionTask.addToTotalBytesCompacted(endsize)));
-            logger.debug("Actual #keys: {}, Estimated #keys:{}, Err%: {}", totalKeysWritten, estimatedTotalKeys, ((double)(totalKeysWritten - estimatedTotalKeys)/totalKeysWritten));
+            else
+            {
+                double mbps = dTime > 0 ? (double) endsize / (1024 * 1024) / ((double) dTime / 1000) : 0;
+                long totalSourceRows = 0;
+                String mergeSummary = updateCompactionHistory(cfs.keyspace.getName(), cfs.getColumnFamilyName(), ci, startsize, endsize);
+                logger.debug(String.format("Compacted (%s) %d sstables to [%s] to level=%d.  %,d bytes to %,d (~%d%% of original) in %,dms = %fMB/s.  %,d total partitions merged to %,d.  Partition merge counts were {%s}",
+                                           taskIdLoggerMsg, transaction.originals().size(), newSSTableNames.toString(), getLevel(), startsize, endsize, (int) (ratio * 100), dTime, mbps, totalSourceRows, totalKeysWritten, mergeSummary));
+                logger.trace(String.format("CF Total Bytes Compacted: %,d", CompactionTask.addToTotalBytesCompacted(endsize)));
+                logger.trace("Actual #keys: {}, Estimated #keys:{}, Err%: {}", totalKeysWritten, estimatedKeys, ((double) (totalKeysWritten - estimatedKeys) / totalKeysWritten));
+            }
         }
     }
 
-    private long getMinRepairedAt(Set<SSTableReader> actuallyCompact)
+    @Override
+    public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, LifecycleTransaction transaction, Set<SSTableReader> nonExpiredSSTables)
+    {
+        return new DefaultCompactionWriter(cfs, transaction, nonExpiredSSTables, offline, compactionType);
+
+    }
+
+    public static String updateCompactionHistory(String keyspaceName, String columnFamilyName, AbstractCompactionIterable ci, long startSize, long endSize)
+    {
+        long[] counts = ci.getMergedRowCounts();
+        StringBuilder mergeSummary = new StringBuilder(counts.length * 10);
+        Map<Integer, Long> mergedRows = new HashMap<>();
+        for (int i = 0; i < counts.length; i++)
+        {
+            long count = counts[i];
+            if (count == 0)
+                continue;
+
+            int rows = i + 1;
+            mergeSummary.append(String.format("%d:%d, ", rows, count));
+            mergedRows.put(rows, count);
+        }
+        SystemKeyspace.updateCompactionHistory(keyspaceName, columnFamilyName, System.currentTimeMillis(), startSize, endSize, mergedRows);
+        return mergeSummary.toString();
+    }
+
+    public static long getMinRepairedAt(Set<SSTableReader> actuallyCompact)
     {
         long minRepairedAt= Long.MAX_VALUE;
         for (SSTableReader sstable : actuallyCompact)
@@ -288,24 +273,34 @@
         return minRepairedAt;
     }
 
-    protected void checkAvailableDiskSpace(long estimatedSSTables)
+    /*
+    Checks if we have enough disk space to execute the compaction.  Drops the largest sstable out of the Task until
+    there's enough space (in theory) to handle the compaction.  Does not take into account space that will be taken by
+    other compactions.
+     */
+    protected void checkAvailableDiskSpace()
     {
-        while (!getDirectories().hasAvailableDiskSpace(estimatedSSTables, getExpectedWriteSize()))
-        {
-            if (!reduceScopeForLimitedSpace())
-                throw new RuntimeException(String.format("Not enough space for compaction, estimated sstables = %d, expected write size = %d", estimatedSSTables, getExpectedWriteSize()));
-        }
-    }
+        AbstractCompactionStrategy strategy = cfs.getCompactionStrategy();
 
-    private SSTableWriter createCompactionWriter(File sstableDirectory, long keysPerSSTable, long repairedAt)
-    {
-        assert sstableDirectory != null;
-        return new SSTableWriter(cfs.getTempSSTablePath(sstableDirectory),
-                                 keysPerSSTable,
-                                 repairedAt,
-                                 cfs.metadata,
-                                 cfs.partitioner,
-                                 new MetadataCollector(sstables, cfs.metadata.comparator, getLevel()));
+        while(true)
+        {
+            long expectedWriteSize = cfs.getExpectedCompactedFileSize(transaction.originals(), compactionType);
+            long estimatedSSTables = Math.max(1, expectedWriteSize / strategy.getMaxSSTableBytes());
+
+            if(cfs.directories.hasAvailableDiskSpace(estimatedSSTables, expectedWriteSize))
+                break;
+
+            if (!reduceScopeForLimitedSpace(expectedWriteSize))
+            {
+                // we end up here if we can't take any more sstables out of the compaction.
+                // usually means we've run out of disk space
+                String msg = String.format("Not enough space for compaction, estimated sstables = %d, expected write size = %d", estimatedSSTables, expectedWriteSize);
+                logger.warn(msg);
+                throw new RuntimeException(msg);
+            }
+            logger.warn("Not enough space for compaction, {}MB estimated.  Reducing scope.",
+                            (float) expectedWriteSize / 1024 / 1024);
+        }
     }
 
     protected int getLevel()
@@ -323,12 +318,6 @@
         return !isUserDefined;
     }
 
-    // extensibility point for other strategies that may want to limit the upper bounds of the sstable segment size
-    protected boolean newSSTableSegmentThresholdReached(SSTableWriter writer)
-    {
-        return false;
-    }
-
     public static long getMaxDataAge(Collection<SSTableReader> sstables)
     {
         long max = 0;
@@ -339,4 +328,13 @@
         }
         return max;
     }
+
+    public static SSTableFormat.Type getFormatType(Collection<SSTableReader> sstables)
+    {
+        if (sstables.isEmpty() || !SSTableFormat.enableSSTableDevelopmentTestMode)
+            return DatabaseDescriptor.getSSTableFormat();
+
+        //Allows us to test compaction of non-default formats
+        return sstables.iterator().next().descriptor.formatType;
+    }
 }
diff --git a/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategy.java
index 64a599d..8c59e1a 100644
--- a/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategy.java
+++ b/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategy.java
@@ -22,13 +22,14 @@
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Predicate;
 import com.google.common.collect.*;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.cql3.statements.CFPropDefs;
 import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.utils.Pair;
 
 public class DateTieredCompactionStrategy extends AbstractCompactionStrategy
@@ -49,15 +50,16 @@
         if (!options.containsKey(AbstractCompactionStrategy.TOMBSTONE_COMPACTION_INTERVAL_OPTION) && !options.containsKey(AbstractCompactionStrategy.TOMBSTONE_THRESHOLD_OPTION))
         {
             disableTombstoneCompactions = true;
-            logger.debug("Disabling tombstone compactions for DTCS");
+            logger.trace("Disabling tombstone compactions for DTCS");
         }
         else
-            logger.debug("Enabling tombstone compactions for DTCS");
+            logger.trace("Enabling tombstone compactions for DTCS");
 
         this.stcsOptions = new SizeTieredCompactionStrategyOptions(options);
     }
 
     @Override
+    @SuppressWarnings("resource")
     public synchronized AbstractCompactionTask getNextBackgroundTask(int gcBefore)
     {
         while (true)
@@ -67,8 +69,9 @@
             if (latestBucket.isEmpty())
                 return null;
 
-            if (cfs.getDataTracker().markCompacting(latestBucket))
-                return new CompactionTask(cfs, latestBucket, gcBefore, false);
+            LifecycleTransaction modifier = cfs.getTracker().tryModify(latestBucket, OperationType.COMPACTION);
+            if (modifier != null)
+                return new CompactionTask(cfs, modifier, gcBefore, false);
         }
     }
 
@@ -97,7 +100,7 @@
         List<SSTableReader> compactionCandidates = new ArrayList<>(getNextNonExpiredSSTables(Sets.difference(candidates, expired), gcBefore));
         if (!expired.isEmpty())
         {
-            logger.debug("Including expired sstables: {}", expired);
+            logger.trace("Including expired sstables: {}", expired);
             compactionCandidates.addAll(expired);
         }
         return compactionCandidates;
@@ -383,28 +386,32 @@
     }
 
     @Override
-    public synchronized Collection<AbstractCompactionTask> getMaximalTask(int gcBefore)
+    @SuppressWarnings("resource")
+    public synchronized Collection<AbstractCompactionTask> getMaximalTask(int gcBefore, boolean splitOutput)
     {
         Iterable<SSTableReader> filteredSSTables = filterSuspectSSTables(sstables);
         if (Iterables.isEmpty(filteredSSTables))
             return null;
-        if (!cfs.getDataTracker().markCompacting(ImmutableList.copyOf(filteredSSTables)))
+        LifecycleTransaction txn = cfs.getTracker().tryModify(filteredSSTables, OperationType.COMPACTION);
+        if (txn == null)
             return null;
-        return Collections.<AbstractCompactionTask>singleton(new CompactionTask(cfs, filteredSSTables, gcBefore, false));
+        return Collections.<AbstractCompactionTask>singleton(new CompactionTask(cfs, txn, gcBefore, false));
     }
 
     @Override
+    @SuppressWarnings("resource")
     public synchronized AbstractCompactionTask getUserDefinedTask(Collection<SSTableReader> sstables, int gcBefore)
     {
         assert !sstables.isEmpty(); // checked for by CM.submitUserDefined
 
-        if (!cfs.getDataTracker().markCompacting(sstables))
+        LifecycleTransaction modifier = cfs.getTracker().tryModify(sstables, OperationType.COMPACTION);
+        if (modifier == null)
         {
-            logger.debug("Unable to mark {} for compaction; probably a background compaction got to it first.  You can disable background compactions temporarily if this is a problem", sstables);
+            logger.trace("Unable to mark {} for compaction; probably a background compaction got to it first.  You can disable background compactions temporarily if this is a problem", sstables);
             return null;
         }
 
-        return new CompactionTask(cfs, sstables, gcBefore, false).setUserDefined(true);
+        return new CompactionTask(cfs, modifier, gcBefore, false).setUserDefined(true);
     }
 
     public int getEstimatedRemainingTasks()
@@ -417,6 +424,20 @@
         return Long.MAX_VALUE;
     }
 
+    /**
+     * DTCS should not group sstables for anticompaction - this can mix new and old data
+     */
+    @Override
+    public Collection<Collection<SSTableReader>> groupSSTablesForAntiCompaction(Collection<SSTableReader> sstablesToGroup)
+    {
+        Collection<Collection<SSTableReader>> groups = new ArrayList<>();
+        for (SSTableReader sstable : sstablesToGroup)
+        {
+            groups.add(Collections.singleton(sstable));
+        }
+        return groups;
+    }
+
     public static Map<String, String> validateOptions(Map<String, String> options) throws ConfigurationException
     {
         Map<String, String> uncheckedOptions = AbstractCompactionStrategy.validateOptions(options);
diff --git a/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java b/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java
index f912da2..eaceead 100644
--- a/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java
+++ b/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java
@@ -24,20 +24,23 @@
 import java.util.Iterator;
 import java.util.List;
 
+import com.google.common.base.Predicate;
 import com.google.common.base.Predicates;
 import com.google.common.collect.Iterators;
 
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
 import org.apache.cassandra.db.index.SecondaryIndexManager;
+import org.apache.cassandra.io.sstable.format.big.BigTableWriter;
 import org.apache.cassandra.io.sstable.ColumnNameHelper;
 import org.apache.cassandra.io.sstable.ColumnStats;
 import org.apache.cassandra.io.sstable.SSTable;
-import org.apache.cassandra.io.sstable.SSTableWriter;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.SequentialWriter;
 import org.apache.cassandra.utils.MergeIterator;
 import org.apache.cassandra.utils.StreamingHistogram;
+import org.apache.cassandra.utils.Throwables;
 
 /**
  * LazilyCompactedRow only computes the row bloom filter and column index in memory
@@ -48,18 +51,17 @@
  */
 public class LazilyCompactedRow extends AbstractCompactedRow
 {
-    private final List<? extends OnDiskAtomIterator> rows;
-    private final CompactionController controller;
-    private boolean hasCalculatedMaxPurgeableTimestamp = false;
-    private long maxPurgeableTimestamp;
-    private final ColumnFamily emptyColumnFamily;
-    private ColumnStats columnStats;
-    private boolean closed;
-    private ColumnIndex.Builder indexBuilder;
-    private final SecondaryIndexManager.Updater indexer;
-    private final Reducer reducer;
-    private final Iterator<OnDiskAtom> merger;
-    private DeletionTime maxRowTombstone;
+    protected final List<? extends OnDiskAtomIterator> rows;
+    protected final CompactionController controller;
+    protected Predicate<Long> purgeEvaluator;
+    protected final ColumnFamily emptyColumnFamily;
+    protected ColumnStats columnStats;
+    protected boolean closed;
+    protected ColumnIndex.Builder indexBuilder;
+    protected final SecondaryIndexManager.Updater indexer;
+    protected final Reducer reducer;
+    protected final Iterator<OnDiskAtom> merger;
+    protected DeletionTime maxRowTombstone;
 
     public LazilyCompactedRow(CompactionController controller, List<? extends OnDiskAtomIterator> rows)
     {
@@ -80,25 +82,21 @@
 
         emptyColumnFamily = ArrayBackedSortedColumns.factory.create(controller.cfs.metadata);
         emptyColumnFamily.delete(maxRowTombstone);
-        if (!maxRowTombstone.isLive() && maxRowTombstone.markedForDeleteAt < getMaxPurgeableTimestamp())
+        if (!maxRowTombstone.isLive() && getPurgeEvaluator().apply(maxRowTombstone.markedForDeleteAt))
             emptyColumnFamily.purgeTombstones(controller.gcBefore);
 
         reducer = new Reducer();
         merger = Iterators.filter(MergeIterator.get(rows, emptyColumnFamily.getComparator().onDiskAtomComparator(), reducer), Predicates.notNull());
     }
 
-    /**
-     * tombstones with a localDeletionTime before this can be purged.  This is the minimum timestamp for any sstable
-     * containing `key` outside of the set of sstables involved in this compaction.
-     */
-    private long getMaxPurgeableTimestamp()
+    private Predicate<Long> getPurgeEvaluator()
     {
-        if (!hasCalculatedMaxPurgeableTimestamp)
+        if (purgeEvaluator == null)
         {
-            hasCalculatedMaxPurgeableTimestamp = true;
-            maxPurgeableTimestamp = controller.maxPurgeableTimestamp(key);
+            purgeEvaluator = controller.getPurgeEvaluator(key);
         }
-        return maxPurgeableTimestamp;
+
+        return purgeEvaluator;
     }
 
     private static void removeDeleted(ColumnFamily cf, boolean shouldPurge, DecoratedKey key, CompactionController controller)
@@ -110,10 +108,12 @@
         ColumnFamilyStore.removeDeletedColumnsOnly(cf, overriddenGCBefore, controller.cfs.indexManager.gcUpdaterFor(key));
     }
 
-    public RowIndexEntry write(long currentPosition, DataOutputPlus out) throws IOException
+    public RowIndexEntry write(long currentPosition, SequentialWriter dataFile) throws IOException
     {
         assert !closed;
 
+        DataOutputPlus out = dataFile.stream;
+
         ColumnIndex columnsIndex;
         try
         {
@@ -141,7 +141,7 @@
         // in case no columns were ever written, we may still need to write an empty header with a top-level tombstone
         indexBuilder.maybeWriteEmptyRowHeader();
 
-        out.writeShort(SSTableWriter.END_OF_ROW);
+        out.writeShort(BigTableWriter.END_OF_ROW);
 
         close();
 
@@ -154,27 +154,26 @@
 
         // no special-case for rows.size == 1, we're actually skipping some bytes here so just
         // blindly updating everything wouldn't be correct
-        DataOutputBuffer out = new DataOutputBuffer();
-        OnDiskAtom.SerializerForWriting serializer = new OnDiskAtom.SerializerForWriting()
+        try (DataOutputBuffer out = new DataOutputBuffer())
         {
-            @Override
-            public void serializeForSSTable(OnDiskAtom atom, DataOutputPlus out) throws IOException
+            OnDiskAtom.SerializerForWriting serializer = new OnDiskAtom.SerializerForWriting()
             {
-                atom.updateDigest(digest);
-            }
+                @Override
+                public void serializeForSSTable(OnDiskAtom atom, DataOutputPlus out) throws IOException
+                {
+                    atom.updateDigest(digest);
+                }
 
-            @Override
-            public long serializedSizeForSSTable(OnDiskAtom atom)
-            {
-                return 0;
-            }
-        };
+                @Override
+                public long serializedSizeForSSTable(OnDiskAtom atom)
+                {
+                    return 0;
+                }
+            };
 
-        // initialize indexBuilder for the benefit of its tombstoneTracker, used by our reducing iterator
-        indexBuilder = new ColumnIndex.Builder(emptyColumnFamily, key.getKey(), out, serializer);
+            // initialize indexBuilder for the benefit of its tombstoneTracker, used by our reducing iterator
+            indexBuilder = new ColumnIndex.Builder(emptyColumnFamily, key.getKey(), out, serializer);
 
-        try
-        {
             DeletionTime.serializer.serialize(emptyColumnFamily.deletionInfo().getTopLevelDeletion(), out);
 
             // do not update digest in case of missing or purged row level tombstones, see CASSANDRA-8979
@@ -202,6 +201,7 @@
 
     public void close()
     {
+        Throwable accumulate = null;
         for (OnDiskAtomIterator row : rows)
         {
             try
@@ -210,13 +210,14 @@
             }
             catch (IOException e)
             {
-                throw new RuntimeException(e);
+                accumulate = Throwables.merge(accumulate, e);
             }
         }
         closed = true;
+        Throwables.maybeFail(accumulate);
     }
 
-    private class Reducer extends MergeIterator.Reducer<OnDiskAtom, OnDiskAtom>
+    protected class Reducer extends MergeIterator.Reducer<OnDiskAtom, OnDiskAtom>
     {
         // all columns reduced together will have the same name, so there will only be one column
         // in the container; we just want to leverage the conflict resolution code from CF.
@@ -225,9 +226,9 @@
 
         // tombstone reference; will be reconciled w/ column during getReduced.  Note that the top-level (row) tombstone
         // is held by LCR.deletionInfo.
-        RangeTombstone tombstone;
+        public RangeTombstone tombstone;
 
-        int columns = 0;
+        public int columns = 0;
         // if the row tombstone is 'live' we need to set timestamp to MAX_VALUE to be able to overwrite it later
         // markedForDeleteAt is MIN_VALUE for 'live' row tombstones (which we use to default maxTimestampSeen)
 
@@ -237,10 +238,10 @@
         // we are bound to have either a RangeTombstone or standard cells will set this properly:
         ColumnStats.MaxIntTracker maxDeletionTimeTracker = new ColumnStats.MaxIntTracker(Integer.MAX_VALUE);
 
-        StreamingHistogram tombstones = new StreamingHistogram(SSTable.TOMBSTONE_HISTOGRAM_BIN_SIZE);
-        List<ByteBuffer> minColumnNameSeen = Collections.emptyList();
-        List<ByteBuffer> maxColumnNameSeen = Collections.emptyList();
-        boolean hasLegacyCounterShards = false;
+        public StreamingHistogram tombstones = new StreamingHistogram(SSTable.TOMBSTONE_HISTOGRAM_BIN_SIZE);
+        public List<ByteBuffer> minColumnNameSeen = Collections.emptyList();
+        public List<ByteBuffer> maxColumnNameSeen = Collections.emptyList();
+        public boolean hasLegacyCounterShards = false;
 
         public Reducer()
         {
@@ -286,7 +287,8 @@
                 RangeTombstone t = tombstone;
                 tombstone = null;
 
-                if (t.data.isGcAble(controller.gcBefore) && t.timestamp() < getMaxPurgeableTimestamp())
+                if (t.data.isGcAble(controller.gcBefore) && getPurgeEvaluator().apply(t.timestamp()) ||
+                    maxRowTombstone.markedForDeleteAt >= t.timestamp())
                 {
                     indexBuilder.tombstoneTracker().update(t, true);
                     return null;
@@ -308,7 +310,7 @@
                 container.delete(maxRowTombstone);
                 Iterator<Cell> iter = container.iterator();
                 Cell c = iter.next();
-                boolean shouldPurge = c.getLocalDeletionTime() < Integer.MAX_VALUE && c.timestamp() < getMaxPurgeableTimestamp();
+                boolean shouldPurge = c.getLocalDeletionTime() < Integer.MAX_VALUE && getPurgeEvaluator().apply(c.timestamp());
                 removeDeleted(container, shouldPurge, key, controller);
                 iter = container.iterator();
                 if (!iter.hasNext())
diff --git a/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java
index ad39e04..bdc9141 100644
--- a/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java
+++ b/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java
@@ -20,20 +20,23 @@
 import java.io.IOException;
 import java.util.*;
 
+
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Joiner;
 import com.google.common.collect.*;
 import com.google.common.primitives.Doubles;
+
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.io.sstable.ISSTableScanner;
-import org.apache.cassandra.io.sstable.SSTableReader;
 
 public class LeveledCompactionStrategy extends AbstractCompactionStrategy
 {
@@ -68,7 +71,7 @@
         maxSSTableSizeInMB = configuredMaxSSTableSize;
 
         manifest = new LeveledManifest(cfs, this.maxSSTableSizeInMB, localOptions);
-        logger.debug("Created {}", manifest);
+        logger.trace("Created {}", manifest);
     }
 
     public int getLevelSize(int i)
@@ -85,16 +88,9 @@
      * the only difference between background and maximal in LCS is that maximal is still allowed
      * (by explicit user request) even when compaction is disabled.
      */
+    @SuppressWarnings("resource")
     public synchronized AbstractCompactionTask getNextBackgroundTask(int gcBefore)
     {
-        Collection<AbstractCompactionTask> tasks = getMaximalTask(gcBefore);
-        if (tasks == null || tasks.size() == 0)
-            return null;
-        return tasks.iterator().next();
-    }
-
-    public Collection<AbstractCompactionTask> getMaximalTask(int gcBefore)
-    {
         while (true)
         {
             OperationType op;
@@ -105,7 +101,7 @@
                 SSTableReader sstable = findDroppableSSTable(gcBefore);
                 if (sstable == null)
                 {
-                    logger.debug("No compaction necessary for {}", this);
+                    logger.trace("No compaction necessary for {}", this);
                     return null;
                 }
                 candidate = new LeveledManifest.CompactionCandidate(Collections.singleton(sstable),
@@ -118,34 +114,95 @@
                 op = OperationType.COMPACTION;
             }
 
-            if (cfs.getDataTracker().markCompacting(candidate.sstables))
+            LifecycleTransaction txn = cfs.getTracker().tryModify(candidate.sstables, OperationType.COMPACTION);
+            if (txn != null)
             {
-                LeveledCompactionTask newTask = new LeveledCompactionTask(cfs, candidate.sstables, candidate.level, gcBefore, candidate.maxSSTableBytes);
+                LeveledCompactionTask newTask = new LeveledCompactionTask(cfs, txn, candidate.level, gcBefore, candidate.maxSSTableBytes, false);
                 newTask.setCompactionType(op);
-                return Arrays.<AbstractCompactionTask>asList(newTask);
+                return newTask;
             }
         }
     }
 
+    @SuppressWarnings("resource")
+    public synchronized Collection<AbstractCompactionTask> getMaximalTask(int gcBefore, boolean splitOutput)
+    {
+        Iterable<SSTableReader> sstables = manifest.getAllSSTables();
+
+        Iterable<SSTableReader> filteredSSTables = filterSuspectSSTables(sstables);
+        if (Iterables.isEmpty(sstables))
+            return null;
+        LifecycleTransaction txn = cfs.getTracker().tryModify(filteredSSTables, OperationType.COMPACTION);
+        if (txn == null)
+            return null;
+        return Arrays.<AbstractCompactionTask>asList(new LeveledCompactionTask(cfs, txn, 0, gcBefore, getMaxSSTableBytes(), true));
+
+    }
+
+    @Override
     public AbstractCompactionTask getUserDefinedTask(Collection<SSTableReader> sstables, int gcBefore)
     {
         throw new UnsupportedOperationException("LevelDB compaction strategy does not allow user-specified compactions");
     }
 
     @Override
-    public AbstractCompactionTask getCompactionTask(Collection<SSTableReader> sstables, int gcBefore, long maxSSTableBytes)
+    public AbstractCompactionTask getCompactionTask(LifecycleTransaction txn, int gcBefore, long maxSSTableBytes)
     {
-        assert sstables.size() > 0;
+        assert txn.originals().size() > 0;
         int level = -1;
         // if all sstables are in the same level, we can set that level:
-        for (SSTableReader sstable : sstables)
+        for (SSTableReader sstable : txn.originals())
         {
             if (level == -1)
                 level = sstable.getSSTableLevel();
             if (level != sstable.getSSTableLevel())
                 level = 0;
         }
-        return new LeveledCompactionTask(cfs, sstables, level, gcBefore, maxSSTableBytes);
+        return new LeveledCompactionTask(cfs, txn, level, gcBefore, maxSSTableBytes, false);
+    }
+
+    /**
+     * Leveled compaction strategy has guarantees on the data contained within each level so we
+     * have to make sure we only create groups of SSTables with members from the same level.
+     * This way we won't end up creating invalid sstables during anti-compaction.
+     * @param ssTablesToGroup
+     * @return Groups of sstables from the same level
+     */
+    @Override
+    public Collection<Collection<SSTableReader>> groupSSTablesForAntiCompaction(Collection<SSTableReader> ssTablesToGroup)
+    {
+        int groupSize = 2;
+        Map<Integer, Collection<SSTableReader>> sstablesByLevel = new HashMap<>();
+        for (SSTableReader sstable : ssTablesToGroup)
+        {
+            Integer level = sstable.getSSTableLevel();
+            if (!sstablesByLevel.containsKey(level))
+            {
+                sstablesByLevel.put(level, new ArrayList<SSTableReader>());
+            }
+            sstablesByLevel.get(level).add(sstable);
+        }
+
+        Collection<Collection<SSTableReader>> groupedSSTables = new ArrayList<>();
+
+        for (Collection<SSTableReader> levelOfSSTables : sstablesByLevel.values())
+        {
+            Collection<SSTableReader> currGroup = new ArrayList<>();
+            for (SSTableReader sstable : levelOfSSTables)
+            {
+                currGroup.add(sstable);
+                if (currGroup.size() == groupSize)
+                {
+                    groupedSSTables.add(currGroup);
+                    currGroup = new ArrayList<>();
+                }
+            }
+
+            if (currGroup.size() != 0)
+                groupedSSTables.add(currGroup);
+        }
+        return groupedSSTables;
+
     }
 
     public int getEstimatedRemainingTasks()
@@ -184,7 +241,7 @@
         {
             for (Integer level : byLevel.keySet())
             {
-                // level can be -1 when sstables are added to DataTracker but not to LeveledManifest
+                // level can be -1 when sstables are added to Tracker but not to LeveledManifest
                 // since we don't know which level those sstable belong yet, we simply do the same as L0 sstables.
                 if (level <= 0)
                 {
@@ -252,7 +309,7 @@
             this.range = range;
 
             // add only sstables that intersect our range, and estimate how much data that involves
-            this.sstables = new ArrayList<SSTableReader>(sstables.size());
+            this.sstables = new ArrayList<>(sstables.size());
             long length = 0;
             for (SSTableReader sstable : sstables)
             {
@@ -275,10 +332,10 @@
 
         public static List<SSTableReader> intersecting(Collection<SSTableReader> sstables, Range<Token> range)
         {
-            ArrayList<SSTableReader> filtered = new ArrayList<SSTableReader>();
+            ArrayList<SSTableReader> filtered = new ArrayList<>();
             for (SSTableReader sstable : sstables)
             {
-                Range<Token> sstableRange = new Range<Token>(sstable.first.getToken(), sstable.last.getToken(), sstable.partitioner);
+                Range<Token> sstableRange = new Range<>(sstable.first.getToken(), sstable.last.getToken());
                 if (range == null || sstableRange.intersects(range))
                     filtered.add(sstable);
             }
@@ -360,7 +417,7 @@
             if (sstables.isEmpty())
                 continue;
 
-            Set<SSTableReader> compacting = cfs.getDataTracker().getCompacting();
+            Set<SSTableReader> compacting = cfs.getTracker().getCompacting();
             for (SSTableReader sstable : sstables)
             {
                 if (sstable.getEstimatedDroppableTombstoneRatio(gcBefore) <= tombstoneThreshold)
diff --git a/src/java/org/apache/cassandra/db/compaction/LeveledCompactionTask.java b/src/java/org/apache/cassandra/db/compaction/LeveledCompactionTask.java
index 902be75..4980ec3 100644
--- a/src/java/org/apache/cassandra/db/compaction/LeveledCompactionTask.java
+++ b/src/java/org/apache/cassandra/db/compaction/LeveledCompactionTask.java
@@ -17,28 +17,35 @@
  */
 package org.apache.cassandra.db.compaction;
 
-import java.util.Collection;
+import java.util.Set;
 
 import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.io.sstable.SSTableReader;
-import org.apache.cassandra.io.sstable.SSTableWriter;
+import org.apache.cassandra.db.compaction.writers.CompactionAwareWriter;
+import org.apache.cassandra.db.compaction.writers.MajorLeveledCompactionWriter;
+import org.apache.cassandra.db.compaction.writers.MaxSSTableSizeWriter;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 
 public class LeveledCompactionTask extends CompactionTask
 {
     private final int level;
     private final long maxSSTableBytes;
+    private final boolean majorCompaction;
 
-    public LeveledCompactionTask(ColumnFamilyStore cfs, Collection<SSTableReader> sstables, int level, final int gcBefore, long maxSSTableBytes)
+    public LeveledCompactionTask(ColumnFamilyStore cfs, LifecycleTransaction txn, int level, int gcBefore, long maxSSTableBytes, boolean majorCompaction)
     {
-        super(cfs, sstables, gcBefore, false);
+        super(cfs, txn, gcBefore, false);
         this.level = level;
         this.maxSSTableBytes = maxSSTableBytes;
+        this.majorCompaction = majorCompaction;
     }
 
     @Override
-    protected boolean newSSTableSegmentThresholdReached(SSTableWriter writer)
+    public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables)
     {
-        return writer.getOnDiskFilePointer() > maxSSTableBytes;
+        if (majorCompaction)
+            return new MajorLeveledCompactionWriter(cfs, txn, nonExpiredSSTables, maxSSTableBytes, false, compactionType);
+        return new MaxSSTableSizeWriter(cfs, txn, nonExpiredSSTables, maxSSTableBytes, getLevel(), false, compactionType);
     }
 
     @Override
diff --git a/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java b/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
index 11af6c4..3b207da 100644
--- a/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
+++ b/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
@@ -23,24 +23,22 @@
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Predicate;
 import com.google.common.base.Predicates;
-import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.ImmutableSortedSet;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 import com.google.common.primitives.Ints;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.RowPosition;
 import org.apache.cassandra.dht.Bounds;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.io.sstable.*;
+import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.Pair;
 
 public class LeveledManifest
@@ -118,7 +116,7 @@
         if (canAddSSTable(reader))
         {
             // adding the sstable does not cause overlap in the level
-            logger.debug("Adding {} to L{}", reader, level);
+            logger.trace("Adding {} to L{}", reader, level);
             generations[level].add(reader);
         }
         else
@@ -147,8 +145,8 @@
     {
         assert !removed.isEmpty(); // use add() instead of promote when adding new sstables
         logDistribution();
-        if (logger.isDebugEnabled())
-            logger.debug("Replacing [{}]", toString(removed));
+        if (logger.isTraceEnabled())
+            logger.trace("Replacing [{}]", toString(removed));
 
         // the level for the added sstables is the max of the removed ones,
         // plus one if the removed were all on the same level
@@ -164,8 +162,8 @@
         if (added.isEmpty())
             return;
 
-        if (logger.isDebugEnabled())
-            logger.debug("Adding [{}]", toString(added));
+        if (logger.isTraceEnabled())
+            logger.trace("Adding [{}]", toString(added));
 
         for (SSTableReader ssTableReader : added)
             add(ssTableReader);
@@ -254,8 +252,7 @@
         return builder.toString();
     }
 
-    @VisibleForTesting
-    long maxBytesForLevel(int level)
+    public static long maxBytesForLevel(int level, long maxSSTableSizeInBytes)
     {
         if (level == 0)
             return 4L * maxSSTableSizeInBytes;
@@ -271,6 +268,18 @@
      */
     public synchronized CompactionCandidate getCompactionCandidates()
     {
+        // during bootstrap we only do size tiering in L0 to make sure
+        // the streamed files can be placed in their original levels
+        if (StorageService.instance.isBootstrapMode())
+        {
+            List<SSTableReader> mostInteresting = getSSTablesForSTCS(getLevel(0));
+            if (!mostInteresting.isEmpty())
+            {
+                logger.info("Bootstrapping - doing STCS in L0");
+                return new CompactionCandidate(mostInteresting, 0, Long.MAX_VALUE);
+            }
+            return null;
+        }
         // LevelDB gives each level a score of how much data it contains vs its ideal amount, and
         // compacts the level with the highest score. But this falls apart spectacularly once you
         // get behind.  Consider this set of levels:
@@ -305,9 +314,9 @@
                 continue; // mostly this just avoids polluting the debug log with zero scores
             // we want to calculate score excluding compacting ones
             Set<SSTableReader> sstablesInLevel = Sets.newHashSet(sstables);
-            Set<SSTableReader> remaining = Sets.difference(sstablesInLevel, cfs.getDataTracker().getCompacting());
-            double score = (double) SSTableReader.getTotalBytes(remaining) / (double)maxBytesForLevel(i);
-            logger.debug("Compaction score for level {} is {}", i, score);
+            Set<SSTableReader> remaining = Sets.difference(sstablesInLevel, cfs.getTracker().getCompacting());
+            double score = (double) SSTableReader.getTotalBytes(remaining) / (double)maxBytesForLevel(i, maxSSTableSizeInBytes);
+            logger.trace("Compaction score for level {} is {}", i, score);
 
             if (score > 1.001)
             {
@@ -322,13 +331,13 @@
                 {
                     int nextLevel = getNextLevel(candidates);
                     candidates = getOverlappingStarvedSSTables(nextLevel, candidates);
-                    if (logger.isDebugEnabled())
-                        logger.debug("Compaction candidates for L{} are {}", i, toString(candidates));
+                    if (logger.isTraceEnabled())
+                        logger.trace("Compaction candidates for L{} are {}", i, toString(candidates));
                     return new CompactionCandidate(candidates, nextLevel, cfs.getCompactionStrategy().getMaxSSTableBytes());
                 }
                 else
                 {
-                    logger.debug("No compaction candidates for L{}", i);
+                    logger.trace("No compaction candidates for L{}", i);
                 }
             }
         }
@@ -364,7 +373,7 @@
 
     private List<SSTableReader> getSSTablesForSTCS(Collection<SSTableReader> sstables)
     {
-        Iterable<SSTableReader> candidates = cfs.getDataTracker().getUncompactingSSTables(sstables);
+        Iterable<SSTableReader> candidates = cfs.getTracker().getUncompacting(sstables);
         List<Pair<SSTableReader,Long>> pairs = SizeTieredCompactionStrategy.createSSTableAndLengthPairs(AbstractCompactionStrategy.filterSuspectSSTables(candidates));
         List<List<SSTableReader>> buckets = SizeTieredCompactionStrategy.getBuckets(pairs,
                                                                                     options.bucketHigh,
@@ -391,10 +400,10 @@
         for (int i = generations.length - 1; i > 0; i--)
             compactionCounter[i]++;
         compactionCounter[targetLevel] = 0;
-        if (logger.isDebugEnabled())
+        if (logger.isTraceEnabled())
         {
             for (int j = 0; j < compactionCounter.length; j++)
-                logger.debug("CompactionCounter: {}: {}", j, compactionCounter[j]);
+                logger.trace("CompactionCounter: {}: {}", j, compactionCounter[j]);
         }
 
         for (int i = generations.length - 1; i > 0; i--)
@@ -418,7 +427,7 @@
                     }
                     if (min == null || max == null || min.equals(max)) // single partition sstables - we cannot include a high level sstable.
                         return candidates;
-                    Set<SSTableReader> compacting = cfs.getDataTracker().getCompacting();
+                    Set<SSTableReader> compacting = cfs.getTracker().getCompacting();
                     Range<RowPosition> boundaries = new Range<>(min, max);
                     for (SSTableReader sstable : getLevel(i))
                     {
@@ -455,13 +464,13 @@
 
     private void logDistribution()
     {
-        if (logger.isDebugEnabled())
+        if (logger.isTraceEnabled())
         {
             for (int i = 0; i < generations.length; i++)
             {
                 if (!getLevel(i).isEmpty())
                 {
-                    logger.debug("L{} contains {} SSTables ({} bytes) in {}",
+                    logger.trace("L{} contains {} SSTables ({} bytes) in {}",
                                  i, getLevel(i).size(), SSTableReader.getTotalBytes(getLevel(i)), this);
                 }
             }
@@ -516,7 +525,7 @@
     private static Set<SSTableReader> overlapping(Token start, Token end, Iterable<SSTableReader> sstables)
     {
         assert start.compareTo(end) <= 0;
-        Set<SSTableReader> overlapped = new HashSet<SSTableReader>();
+        Set<SSTableReader> overlapped = new HashSet<>();
         Bounds<Token> promotedBounds = new Bounds<Token>(start, end);
         for (SSTableReader candidate : sstables)
         {
@@ -543,9 +552,9 @@
     private Collection<SSTableReader> getCandidatesFor(int level)
     {
         assert !getLevel(level).isEmpty();
-        logger.debug("Choosing candidates for L{}", level);
+        logger.trace("Choosing candidates for L{}", level);
 
-        final Set<SSTableReader> compacting = cfs.getDataTracker().getCompacting();
+        final Set<SSTableReader> compacting = cfs.getTracker().getCompacting();
 
         if (level == 0)
         {
@@ -574,8 +583,8 @@
             // Note that we ignore suspect-ness of L1 sstables here, since if an L1 sstable is suspect we're
             // basically screwed, since we expect all or most L0 sstables to overlap with each L1 sstable.
             // So if an L1 sstable is suspect we can't do much besides try anyway and hope for the best.
-            Set<SSTableReader> candidates = new HashSet<SSTableReader>();
-            Set<SSTableReader> remaining = new HashSet<SSTableReader>();
+            Set<SSTableReader> candidates = new HashSet<>();
+            Set<SSTableReader> remaining = new HashSet<>();
             Iterables.addAll(remaining, Iterables.filter(getLevel(0), Predicates.not(suspectP)));
             for (SSTableReader sstable : ageSortedSSTables(remaining))
             {
@@ -653,7 +662,7 @@
     {
         Set<SSTableReader> sstables = new HashSet<>();
         Set<SSTableReader> levelSSTables = new HashSet<>(getLevel(level));
-        for (SSTableReader sstable : cfs.getDataTracker().getCompacting())
+        for (SSTableReader sstable : cfs.getTracker().getCompacting())
         {
             if (levelSSTables.contains(sstable))
                 sstables.add(sstable);
@@ -663,7 +672,7 @@
 
     private List<SSTableReader> ageSortedSSTables(Collection<SSTableReader> candidates)
     {
-        List<SSTableReader> ageSortedCandidates = new ArrayList<SSTableReader>(candidates);
+        List<SSTableReader> ageSortedCandidates = new ArrayList<>(candidates);
         Collections.sort(ageSortedCandidates, SSTableReader.maxTimestampComparator);
         return ageSortedCandidates;
     }
@@ -713,11 +722,11 @@
         {
             List<SSTableReader> sstables = getLevel(i);
             // If there is 1 byte over TBL - (MBL * 1.001), there is still a task left, so we need to round up.
-            estimated[i] = (long)Math.ceil((double)Math.max(0L, SSTableReader.getTotalBytes(sstables) - (long)(maxBytesForLevel(i) * 1.001)) / (double)maxSSTableSizeInBytes);
+            estimated[i] = (long)Math.ceil((double)Math.max(0L, SSTableReader.getTotalBytes(sstables) - (long)(maxBytesForLevel(i, maxSSTableSizeInBytes) * 1.001)) / (double)maxSSTableSizeInBytes);
             tasks += estimated[i];
         }
 
-        logger.debug("Estimating {} compactions to do for {}.{}",
+        logger.trace("Estimating {} compactions to do for {}.{}",
                      Arrays.toString(estimated), cfs.keyspace.getName(), cfs.name);
         return Ints.checkedCast(tasks);
     }
@@ -746,6 +755,16 @@
 
     }
 
+    public Iterable<SSTableReader> getAllSSTables()
+    {
+        Set<SSTableReader> sstables = new HashSet<>();
+        for (List<SSTableReader> generation : generations)
+        {
+            sstables.addAll(generation);
+        }
+        return sstables;
+    }
+
     public static class CompactionCandidate
     {
         public final Collection<SSTableReader> sstables;
diff --git a/src/java/org/apache/cassandra/db/compaction/OperationType.java b/src/java/org/apache/cassandra/db/compaction/OperationType.java
index 475b591..6b66ded 100644
--- a/src/java/org/apache/cassandra/db/compaction/OperationType.java
+++ b/src/java/org/apache/cassandra/db/compaction/OperationType.java
@@ -32,6 +32,7 @@
     TOMBSTONE_COMPACTION("Tombstone Compaction"),
     UNKNOWN("Unknown compaction type"),
     ANTICOMPACTION("Anticompaction after repair"),
+    VERIFY("Verify"),
     INDEX_SUMMARY("Index summary redistribution");
 
     private final String type;
diff --git a/src/java/org/apache/cassandra/db/compaction/SSTableSplitter.java b/src/java/org/apache/cassandra/db/compaction/SSTableSplitter.java
index 6b9f161..6b302d2 100644
--- a/src/java/org/apache/cassandra/db/compaction/SSTableSplitter.java
+++ b/src/java/org/apache/cassandra/db/compaction/SSTableSplitter.java
@@ -19,8 +19,14 @@
 
 import java.util.*;
 
+import com.google.common.base.Predicate;
+import com.google.common.base.Predicates;
+
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.io.sstable.*;
+import org.apache.cassandra.db.compaction.writers.CompactionAwareWriter;
+import org.apache.cassandra.db.compaction.writers.MaxSSTableSizeWriter;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 
 public class SSTableSplitter {
 
@@ -28,9 +34,9 @@
 
     private CompactionInfo.Holder info;
 
-    public SSTableSplitter(ColumnFamilyStore cfs, SSTableReader sstable, int sstableSizeInMB)
+    public SSTableSplitter(ColumnFamilyStore cfs, LifecycleTransaction transaction, int sstableSizeInMB)
     {
-        this.task = new SplittingCompactionTask(cfs, sstable, sstableSizeInMB);
+        this.task = new SplittingCompactionTask(cfs, transaction, sstableSizeInMB);
     }
 
     public void split()
@@ -55,9 +61,9 @@
     {
         private final int sstableSizeInMB;
 
-        public SplittingCompactionTask(ColumnFamilyStore cfs, SSTableReader sstable, int sstableSizeInMB)
+        public SplittingCompactionTask(ColumnFamilyStore cfs, LifecycleTransaction transaction, int sstableSizeInMB)
         {
-            super(cfs, Collections.singletonList(sstable), CompactionManager.NO_GC, true);
+            super(cfs, transaction, CompactionManager.NO_GC, true);
             this.sstableSizeInMB = sstableSizeInMB;
 
             if (sstableSizeInMB <= 0)
@@ -71,9 +77,9 @@
         }
 
         @Override
-        protected boolean newSSTableSegmentThresholdReached(SSTableWriter writer)
+        public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables)
         {
-            return writer.getOnDiskFilePointer() > sstableSizeInMB * 1024L * 1024L;
+            return new MaxSSTableSizeWriter(cfs, txn, nonExpiredSSTables, sstableSizeInMB * 1024L * 1024L, 0, true, compactionType);
         }
 
         @Override
@@ -91,9 +97,9 @@
         }
 
         @Override
-        public long maxPurgeableTimestamp(DecoratedKey key)
+        public Predicate<Long> getPurgeEvaluator(DecoratedKey key)
         {
-            return Long.MIN_VALUE;
+            return Predicates.alwaysFalse();
         }
     }
 }
diff --git a/src/java/org/apache/cassandra/db/compaction/Scrubber.java b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
index 2df3665..aaed234 100644
--- a/src/java/org/apache/cassandra/db/compaction/Scrubber.java
+++ b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
@@ -22,37 +22,43 @@
 import java.util.*;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Predicate;
+import com.google.common.base.Predicates;
 import com.google.common.base.Throwables;
 import com.google.common.collect.AbstractIterator;
-import com.google.common.collect.Sets;
 
 import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
 import org.apache.cassandra.io.sstable.*;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.io.util.RandomAccessReader;
 import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.UUIDGen;
 
 public class Scrubber implements Closeable
 {
     private final ColumnFamilyStore cfs;
     private final SSTableReader sstable;
+    private final LifecycleTransaction transaction;
     private final File destination;
     private final boolean skipCorrupted;
-    public final boolean validateColumns;
 
     private final CompactionController controller;
     private final boolean isCommutative;
+    private final boolean isIndex;
+    private final boolean checkData;
     private final long expectedBloomFilterSize;
 
     private final RandomAccessReader dataFile;
     private final RandomAccessReader indexFile;
     private final ScrubInfo scrubInfo;
-
-    private final boolean isOffline;
+    private final RowIndexEntry.IndexSerializer rowIndexEntrySerializer;
 
     private SSTableReader newSstable;
     private SSTableReader newInOrderSstable;
@@ -77,40 +83,40 @@
     };
     private final SortedSet<Row> outOfOrderRows = new TreeSet<>(rowComparator);
 
-    public Scrubber(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, boolean isOffline, boolean checkData) throws IOException
+    public Scrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, boolean skipCorrupted, boolean checkData) throws IOException
     {
-        this(cfs, sstable, skipCorrupted, new OutputHandler.LogOutput(), isOffline, checkData);
+        this(cfs, transaction, skipCorrupted, new OutputHandler.LogOutput(), checkData);
     }
 
-    public Scrubber(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, OutputHandler outputHandler, boolean isOffline, boolean checkData) throws IOException
+    @SuppressWarnings("resource")
+    public Scrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, boolean skipCorrupted, OutputHandler outputHandler, boolean checkData) throws IOException
     {
         this.cfs = cfs;
-        this.sstable = sstable;
+        this.transaction = transaction;
+        this.sstable = transaction.onlyOne();
         this.outputHandler = outputHandler;
         this.skipCorrupted = skipCorrupted;
-        this.isOffline = isOffline;
-        this.validateColumns = checkData;
+        this.rowIndexEntrySerializer = sstable.descriptor.version.getSSTableFormat().getIndexSerializer(sstable.metadata);
 
         List<SSTableReader> toScrub = Collections.singletonList(sstable);
 
         // Calculate the expected compacted filesize
         this.destination = cfs.directories.getWriteableLocationAsFile(cfs.getExpectedCompactedFileSize(toScrub, OperationType.SCRUB));
-        if (destination == null)
-            throw new IOException("disk full");
 
         // If we run scrub offline, we should never purge tombstone, as we cannot know if other sstable have data that the tombstone deletes.
-        this.controller = isOffline
+        this.controller = transaction.isOffline()
                         ? new ScrubController(cfs)
                         : new CompactionController(cfs, Collections.singleton(sstable), CompactionManager.getDefaultGcBefore(cfs));
         this.isCommutative = cfs.metadata.isCounter();
 
         boolean hasIndexFile = (new File(sstable.descriptor.filenameFor(Component.PRIMARY_INDEX))).exists();
+        this.isIndex = cfs.isIndex();
         if (!hasIndexFile)
         {
             // if there's any corruption in the -Data.db then rows can't be skipped over. but it's worth a shot.
             outputHandler.warn("Missing component: " + sstable.descriptor.filenameFor(Component.PRIMARY_INDEX));
         }
-
+        this.checkData = checkData && !this.isIndex; //LocalByPartitionerType does not support validation
         this.expectedBloomFilterSize = Math.max(
             cfs.metadata.getMinIndexInterval(),
             hasIndexFile ? SSTableReader.getApproximateKeyCount(toScrub) : 0);
@@ -119,7 +125,7 @@
         // we'll also loop through the index at the same time, using the position from the index to recover if the
         // row header (key or data size) is corrupt. (This means our position in the index file will be one row
         // "ahead" of the data file.)
-        this.dataFile = isOffline
+        this.dataFile = transaction.isOffline()
                         ? sstable.openDataReader()
                         : sstable.openDataReader(CompactionManager.instance.getRateLimiter());
 
@@ -136,15 +142,13 @@
     public void scrub()
     {
         outputHandler.output(String.format("Scrubbing %s (%s bytes)", sstable, dataFile.length()));
-        Set<SSTableReader> oldSSTable = Sets.newHashSet(sstable);
-        SSTableRewriter writer = new SSTableRewriter(cfs, oldSSTable, sstable.maxDataAge, isOffline);
-        try
+        try (SSTableRewriter writer = new SSTableRewriter(cfs, transaction, sstable.maxDataAge, transaction.isOffline()))
         {
             nextIndexKey = indexAvailable() ? ByteBufferUtil.readWithShortLength(indexFile) : null;
             if (indexAvailable())
             {
                 // throw away variable so we don't have a side effect in the assert
-                long firstRowPositionFromIndex = sstable.metadata.comparator.rowIndexEntrySerializer().deserialize(indexFile, sstable.descriptor.version).position;
+                long firstRowPositionFromIndex = rowIndexEntrySerializer.deserialize(indexFile, sstable.descriptor.version).position;
                 assert firstRowPositionFromIndex == 0 : firstRowPositionFromIndex;
             }
 
@@ -166,7 +170,7 @@
                     {
                         // Encountered CASSANDRA-10791. Place post-END_OF_ROW data in the out-of-order table.
                         saveOutOfOrderRow(prevKey,
-                                          SSTableIdentityIterator.createFragmentIterator(sstable, dataFile, prevKey, nextRowStart - dataFile.getFilePointer(), validateColumns),
+                                          SSTableIdentityIterator.createFragmentIterator(sstable, dataFile, prevKey, nextRowStart - dataFile.getFilePointer(), checkData),
                                           String.format("Row fragment detected after END_OF_ROW at key %s", prevKey));
                         if (dataFile.isEOF())
                             break;
@@ -177,7 +181,6 @@
                 outputHandler.debug("Reading row at " + rowStart);
 
                 DecoratedKey key = null;
-                long dataSize = -1;
                 try
                 {
                     key = sstable.partitioner.decorateKey(ByteBufferUtil.readWithShortLength(dataFile));
@@ -198,10 +201,9 @@
                     dataSizeFromIndex = nextRowPositionFromIndex - dataStartFromIndex;
                 }
 
-                dataSize = dataSizeFromIndex;
                 // avoid an NPE if key is null
                 String keyName = key == null ? "(unreadable key)" : ByteBufferUtil.bytesToHex(key.getKey());
-                outputHandler.debug(String.format("row %s is %s bytes", keyName, dataSize));
+                outputHandler.debug(String.format("row %s is %s bytes", keyName, dataSizeFromIndex));
 
                 assert currentIndexKey != null || !indexAvailable();
 
@@ -216,16 +218,13 @@
                                 ByteBufferUtil.bytesToHex(key.getKey()), ByteBufferUtil.bytesToHex(currentIndexKey))));
                     }
 
-                    if (dataSize > dataFile.length())
-                        throw new IOError(new IOException("Impossible row size (greater than file length): " + dataSize));
+                    if (indexFile != null && dataSizeFromIndex > dataFile.length())
+                        throw new IOError(new IOException("Impossible row size (greater than file length): " + dataSizeFromIndex));
 
                     if (indexFile != null && dataStart != dataStartFromIndex)
                         outputHandler.warn(String.format("Data file row position %d differs from index file row position %d", dataStart, dataStartFromIndex));
 
-                    if (indexFile != null && dataSize != dataSizeFromIndex)
-                        outputHandler.warn(String.format("Data file row size %d different from index file row size %d", dataSize, dataSizeFromIndex));
-
-                    if (tryAppend(prevKey, key, dataSize, writer))
+                    if (tryAppend(prevKey, key, writer))
                         prevKey = key;
                 }
                 catch (Throwable th)
@@ -234,7 +233,7 @@
                     outputHandler.warn("Error reading row (stacktrace follows):", th);
 
                     if (currentIndexKey != null
-                        && (key == null || !key.getKey().equals(currentIndexKey) || dataStart != dataStartFromIndex || dataSize != dataSizeFromIndex))
+                        && (key == null || !key.getKey().equals(currentIndexKey) || dataStart != dataStartFromIndex))
                     {
                         outputHandler.output(String.format("Retrying from row index; data is %s bytes starting at %s",
                                                   dataSizeFromIndex, dataStartFromIndex));
@@ -242,13 +241,14 @@
                         try
                         {
                             dataFile.seek(dataStartFromIndex);
-                            if (tryAppend(prevKey, key, dataSize, writer))
+
+                            if (tryAppend(prevKey, key, writer))
                                 prevKey = key;
                         }
                         catch (Throwable th2)
                         {
                             throwIfFatal(th2);
-                            throwIfCommutative(key, th2);
+                            throwIfCannotContinue(key, th2);
 
                             outputHandler.warn("Retry failed too. Skipping to next row (retry's stacktrace follows)", th2);
                             badRows++;
@@ -257,7 +257,7 @@
                     }
                     else
                     {
-                        throwIfCommutative(key, th);
+                        throwIfCannotContinue(key, th);
 
                         outputHandler.warn("Row starting at position " + dataStart + " is unreadable; skipping to next");
                         badRows++;
@@ -271,33 +271,31 @@
             {
                 // out of order rows, but no bad rows found - we can keep our repairedAt time
                 long repairedAt = badRows > 0 ? ActiveRepairService.UNREPAIRED_SSTABLE : sstable.getSSTableMetadata().repairedAt;
-                SSTableWriter inOrderWriter = CompactionManager.createWriter(cfs, destination, expectedBloomFilterSize, repairedAt, sstable);
-                for (Row row : outOfOrderRows)
-                    inOrderWriter.append(row.key, row.cf);
-                newInOrderSstable = inOrderWriter.closeAndOpenReader(sstable.maxDataAge);
-                if (!isOffline)
-                    cfs.getDataTracker().addSSTables(Collections.singleton(newInOrderSstable));
-                else if (newInOrderSstable != null)
+                try (SSTableWriter inOrderWriter = CompactionManager.createWriter(cfs, destination, expectedBloomFilterSize, repairedAt, sstable);)
+                {
+                    for (Row row : outOfOrderRows)
+                        inOrderWriter.append(row.key, row.cf);
+                    newInOrderSstable = inOrderWriter.finish(-1, sstable.maxDataAge, true);
+                }
+                transaction.update(newInOrderSstable, false);
+                if (transaction.isOffline() && newInOrderSstable != null)
                     newInOrderSstable.selfRef().release();
                 outputHandler.warn(String.format("%d out of order rows found while scrubbing %s; Those have been written (in order) to a new sstable (%s)", outOfOrderRows.size(), sstable, newInOrderSstable));
             }
 
             // finish obsoletes the old sstable
-            List<SSTableReader> finished = writer.finish(badRows > 0 ? ActiveRepairService.UNREPAIRED_SSTABLE : sstable.getSSTableMetadata().repairedAt);
+            List<SSTableReader> finished = writer.setRepairedAt(badRows > 0 ? ActiveRepairService.UNREPAIRED_SSTABLE : sstable.getSSTableMetadata().repairedAt).finish();
             if (!finished.isEmpty())
                 newSstable = finished.get(0);
-            if (!isOffline)
-                cfs.getDataTracker().markCompactedSSTablesReplaced(oldSSTable, finished, OperationType.SCRUB);
         }
-        catch (Throwable t)
+        catch (IOException e)
         {
-            writer.abort();
-            throw Throwables.propagate(t);
+            throw Throwables.propagate(e);
         }
         finally
         {
             controller.close();
-            if (isOffline && newSstable != null)
+            if (transaction.isOffline() && newSstable != null)
                 newSstable.selfRef().release();
         }
 
@@ -317,12 +315,12 @@
     }
 
     @SuppressWarnings("resource")
-    private boolean tryAppend(DecoratedKey prevKey, DecoratedKey key, long dataSize, SSTableRewriter writer)
+    private boolean tryAppend(DecoratedKey prevKey, DecoratedKey key, SSTableRewriter writer)
     {
         // OrderCheckerIterator will check, at iteration time, that the cells are in the proper order. If it detects
         // that one cell is out of order, it will stop returning them. The remaining cells will be sorted and added
         // to the outOfOrderRows that will be later written to a new SSTable.
-        OrderCheckerIterator atoms = new OrderCheckerIterator(new SSTableIdentityIterator(sstable, dataFile, key, dataSize, validateColumns),
+        OrderCheckerIterator atoms = new OrderCheckerIterator(new SSTableIdentityIterator(sstable, dataFile, key, checkData),
                                                               cfs.metadata.comparator.onDiskAtomComparator());
         if (prevKey != null && prevKey.compareTo(key) > 0)
         {
@@ -352,7 +350,7 @@
 
             nextRowPositionFromIndex = !indexAvailable()
                     ? dataFile.length()
-                    : sstable.metadata.comparator.rowIndexEntrySerializer().deserialize(indexFile, sstable.descriptor.version).position;
+                    : rowIndexEntrySerializer.deserialize(indexFile, sstable.descriptor.version).position;
         }
         catch (Throwable th)
         {
@@ -430,8 +428,15 @@
             throw (Error) th;
     }
 
-    private void throwIfCommutative(DecoratedKey key, Throwable th)
+    private void throwIfCannotContinue(DecoratedKey key, Throwable th)
     {
+        if (isIndex)
+        {
+            outputHandler.warn(String.format("An error occurred while scrubbing the row with key '%s' for an index table. " +
+                                             "Scrubbing will abort for this table and the index will be rebuilt.", key));
+            throw new IOError(th);
+        }
+
         if (isCommutative && !skipCorrupted)
         {
             outputHandler.warn(String.format("An error occurred while scrubbing the row with key '%s'.  Skipping corrupt " +
@@ -458,11 +463,13 @@
     {
         private final RandomAccessReader dataFile;
         private final SSTableReader sstable;
+        private final UUID scrubCompactionId;
 
         public ScrubInfo(RandomAccessReader dataFile, SSTableReader sstable)
         {
             this.dataFile = dataFile;
             this.sstable = sstable;
+            scrubCompactionId = UUIDGen.getTimeUUID();
         }
 
         public CompactionInfo getCompactionInfo()
@@ -472,7 +479,8 @@
                 return new CompactionInfo(sstable.metadata,
                                           OperationType.SCRUB,
                                           dataFile.getFilePointer(),
-                                          dataFile.length());
+                                          dataFile.length(),
+                                          scrubCompactionId);
             }
             catch (Exception e)
             {
@@ -489,9 +497,9 @@
         }
 
         @Override
-        public long maxPurgeableTimestamp(DecoratedKey key)
+        public Predicate<Long> getPurgeEvaluator(DecoratedKey key)
         {
-            return Long.MIN_VALUE;
+            return Predicates.alwaysFalse();
         }
     }
 
diff --git a/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java
index 1f4acdb..b4125bb 100644
--- a/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java
+++ b/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java
@@ -21,19 +21,19 @@
 import java.util.Map.Entry;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
-import com.google.common.primitives.Longs;
+
+import org.apache.cassandra.db.compaction.writers.CompactionAwareWriter;
+import org.apache.cassandra.db.compaction.writers.SplittingSizeTieredCompactionWriter;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.cql3.statements.CFPropDefs;
 import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.io.sstable.ColumnNameHelper;
-import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.utils.Pair;
 
 public class SizeTieredCompactionStrategy extends AbstractCompactionStrategy
@@ -62,7 +62,7 @@
         }
     };
 
-    protected SizeTieredCompactionStrategyOptions options;
+    protected SizeTieredCompactionStrategyOptions sizeTieredOptions;
     protected volatile int estimatedRemainingTasks;
     private final Set<SSTableReader> sstables = new HashSet<>();
 
@@ -70,7 +70,7 @@
     {
         super(cfs, options);
         this.estimatedRemainingTasks = 0;
-        this.options = new SizeTieredCompactionStrategyOptions(options);
+        this.sizeTieredOptions = new SizeTieredCompactionStrategyOptions(options);
     }
 
     private List<SSTableReader> getNextBackgroundSSTables(final int gcBefore)
@@ -81,8 +81,8 @@
 
         Iterable<SSTableReader> candidates = filterSuspectSSTables(Sets.intersection(cfs.getUncompactingSSTables(), sstables));
 
-        List<List<SSTableReader>> buckets = getBuckets(createSSTableAndLengthPairs(candidates), options.bucketHigh, options.bucketLow, options.minSSTableSize);
-        logger.debug("Compaction buckets are {}", buckets);
+        List<List<SSTableReader>> buckets = getBuckets(createSSTableAndLengthPairs(candidates), sizeTieredOptions.bucketHigh, sizeTieredOptions.bucketLow, sizeTieredOptions.minSSTableSize);
+        logger.trace("Compaction buckets are {}", buckets);
         updateEstimatedCompactionsByTasks(buckets);
         List<SSTableReader> mostInteresting = mostInterestingBucket(buckets, minThreshold, maxThreshold);
         if (!mostInteresting.isEmpty())
@@ -157,7 +157,7 @@
 
     private static Map<SSTableReader, Double> getHotnessMap(Collection<SSTableReader> sstables)
     {
-        Map<SSTableReader, Double> hotness = new HashMap<>();
+        Map<SSTableReader, Double> hotness = new HashMap<>(sstables.size());
         for (SSTableReader sstable : sstables)
             hotness.put(sstable, hotness(sstable));
         return hotness;
@@ -172,6 +172,7 @@
         return sstr.getReadMeter() == null ? 0.0 : sstr.getReadMeter().twoHourRate() / sstr.estimatedKeys();
     }
 
+    @SuppressWarnings("resource")
     public synchronized AbstractCompactionTask getNextBackgroundTask(int gcBefore)
     {
         while (true)
@@ -181,32 +182,39 @@
             if (hottestBucket.isEmpty())
                 return null;
 
-            if (cfs.getDataTracker().markCompacting(hottestBucket))
-                return new CompactionTask(cfs, hottestBucket, gcBefore, false);
+            LifecycleTransaction transaction = cfs.getTracker().tryModify(hottestBucket, OperationType.COMPACTION);
+            if (transaction != null)
+                return new CompactionTask(cfs, transaction, gcBefore, false);
         }
     }
 
-    public Collection<AbstractCompactionTask> getMaximalTask(final int gcBefore)
+    @SuppressWarnings("resource")
+    public Collection<AbstractCompactionTask> getMaximalTask(final int gcBefore, boolean splitOutput)
     {
         Iterable<SSTableReader> filteredSSTables = filterSuspectSSTables(sstables);
         if (Iterables.isEmpty(filteredSSTables))
             return null;
-        if (!cfs.getDataTracker().markCompacting(ImmutableList.copyOf(filteredSSTables)))
+        LifecycleTransaction txn = cfs.getTracker().tryModify(filteredSSTables, OperationType.COMPACTION);
+        if (txn == null)
             return null;
-        return Arrays.<AbstractCompactionTask>asList(new CompactionTask(cfs, filteredSSTables, gcBefore, false));
+        if (splitOutput)
+            return Arrays.<AbstractCompactionTask>asList(new SplittingCompactionTask(cfs, txn, gcBefore, false));
+        return Arrays.<AbstractCompactionTask>asList(new CompactionTask(cfs, txn, gcBefore, false));
     }
 
+    @SuppressWarnings("resource")
     public AbstractCompactionTask getUserDefinedTask(Collection<SSTableReader> sstables, final int gcBefore)
     {
         assert !sstables.isEmpty(); // checked for by CM.submitUserDefined
 
-        if (!cfs.getDataTracker().markCompacting(sstables))
+        LifecycleTransaction transaction = cfs.getTracker().tryModify(sstables, OperationType.COMPACTION);
+        if (transaction == null)
         {
-            logger.debug("Unable to mark {} for compaction; probably a background compaction got to it first.  You can disable background compactions temporarily if this is a problem", sstables);
+            logger.trace("Unable to mark {} for compaction; probably a background compaction got to it first.  You can disable background compactions temporarily if this is a problem", sstables);
             return null;
         }
 
-        return new CompactionTask(cfs, sstables, gcBefore, false).setUserDefined(true);
+        return new CompactionTask(cfs, transaction, gcBefore, false).setUserDefined(true);
     }
 
     public int getEstimatedRemainingTasks()
@@ -216,7 +224,7 @@
 
     public static List<Pair<SSTableReader, Long>> createSSTableAndLengthPairs(Iterable<SSTableReader> sstables)
     {
-        List<Pair<SSTableReader, Long>> sstableLengthPairs = new ArrayList<Pair<SSTableReader, Long>>(Iterables.size(sstables));
+        List<Pair<SSTableReader, Long>> sstableLengthPairs = new ArrayList<>(Iterables.size(sstables));
         for(SSTableReader sstable : sstables)
             sstableLengthPairs.add(Pair.create(sstable, sstable.onDiskLength()));
         return sstableLengthPairs;
@@ -324,4 +332,18 @@
             cfs.getMinimumCompactionThreshold(),
             cfs.getMaximumCompactionThreshold());
     }
+
+    private static class SplittingCompactionTask extends CompactionTask
+    {
+        public SplittingCompactionTask(ColumnFamilyStore cfs, LifecycleTransaction txn, int gcBefore, boolean offline)
+        {
+            super(cfs, txn, gcBefore, offline);
+        }
+
+        @Override
+        public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables)
+        {
+            return new SplittingSizeTieredCompactionWriter(cfs, txn, nonExpiredSSTables, compactionType);
+        }
+    }
 }
diff --git a/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyOptions.java b/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyOptions.java
index 9a840e1..911bb9f 100644
--- a/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyOptions.java
+++ b/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyOptions.java
@@ -29,8 +29,6 @@
     protected static final String MIN_SSTABLE_SIZE_KEY = "min_sstable_size";
     protected static final String BUCKET_LOW_KEY = "bucket_low";
     protected static final String BUCKET_HIGH_KEY = "bucket_high";
-    @Deprecated
-    protected static final String COLD_READS_TO_OMIT_KEY = "cold_reads_to_omit";
 
     protected long minSSTableSize;
     protected double bucketLow;
@@ -93,7 +91,6 @@
         uncheckedOptions.remove(MIN_SSTABLE_SIZE_KEY);
         uncheckedOptions.remove(BUCKET_LOW_KEY);
         uncheckedOptions.remove(BUCKET_HIGH_KEY);
-        uncheckedOptions.remove(COLD_READS_TO_OMIT_KEY);
 
         return uncheckedOptions;
     }
diff --git a/src/java/org/apache/cassandra/db/compaction/Upgrader.java b/src/java/org/apache/cassandra/db/compaction/Upgrader.java
index 5d245ac..d6ef60e 100644
--- a/src/java/org/apache/cassandra/db/compaction/Upgrader.java
+++ b/src/java/org/apache/cassandra/db/compaction/Upgrader.java
@@ -20,19 +20,27 @@
 import java.io.File;
 import java.util.*;
 
+import com.google.common.base.Predicate;
+import com.google.common.base.Predicates;
 import com.google.common.base.Throwables;
-import com.google.common.collect.Sets;
 
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.io.sstable.*;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
+import org.apache.cassandra.utils.CloseableIterator;
 import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.UUIDGen;
 
 public class Upgrader
 {
     private final ColumnFamilyStore cfs;
     private final SSTableReader sstable;
+    private final LifecycleTransaction transaction;
     private final File directory;
 
     private final OperationType compactionType = OperationType.UPGRADE_SSTABLES;
@@ -42,10 +50,11 @@
 
     private final OutputHandler outputHandler;
 
-    public Upgrader(ColumnFamilyStore cfs, SSTableReader sstable, OutputHandler outputHandler)
+    public Upgrader(ColumnFamilyStore cfs, LifecycleTransaction txn, OutputHandler outputHandler)
     {
         this.cfs = cfs;
-        this.sstable = sstable;
+        this.transaction = txn;
+        this.sstable = txn.onlyOne();
         this.outputHandler = outputHandler;
 
         this.directory = new File(sstable.getFilename()).getParentFile();
@@ -71,32 +80,31 @@
                 sstableMetadataCollector.addAncestor(i);
         }
         sstableMetadataCollector.sstableLevel(sstable.getSSTableLevel());
-        return new SSTableWriter(cfs.getTempSSTablePath(directory), estimatedRows, repairedAt, cfs.metadata, cfs.partitioner, sstableMetadataCollector);
+        return SSTableWriter.create(Descriptor.fromFilename(cfs.getTempSSTablePath(directory)), estimatedRows, repairedAt, cfs.metadata, cfs.partitioner, sstableMetadataCollector);
     }
 
     public void upgrade()
     {
         outputHandler.output("Upgrading " + sstable);
-        Set<SSTableReader> toUpgrade = Sets.newHashSet(sstable);
-        SSTableRewriter writer = new SSTableRewriter(cfs, toUpgrade, CompactionTask.getMaxDataAge(toUpgrade), true);
-        try (AbstractCompactionStrategy.ScannerList scanners = strategy.getScanners(toUpgrade))
+
+        try (SSTableRewriter writer = new SSTableRewriter(cfs, transaction, CompactionTask.getMaxDataAge(transaction.originals()), true);
+             AbstractCompactionStrategy.ScannerList scanners = strategy.getScanners(transaction.originals());
+             CloseableIterator<AbstractCompactedRow> iter = new CompactionIterable(compactionType, scanners.scanners, controller, DatabaseDescriptor.getSSTableFormat(), UUIDGen.getTimeUUID()).iterator())
         {
-            Iterator<AbstractCompactedRow> iter = new CompactionIterable(compactionType, scanners.scanners, controller).iterator();
             writer.switchWriter(createCompactionWriter(sstable.getSSTableMetadata().repairedAt));
             while (iter.hasNext())
             {
+                @SuppressWarnings("resource")
                 AbstractCompactedRow row = iter.next();
                 writer.append(row);
             }
 
             writer.finish();
             outputHandler.output("Upgrade of " + sstable + " complete.");
-
         }
-        catch (Throwable t)
+        catch (Exception e)
         {
-            writer.abort();
-            throw Throwables.propagate(t);
+            Throwables.propagate(e);
         }
         finally
         {
@@ -112,9 +120,9 @@
         }
 
         @Override
-        public long maxPurgeableTimestamp(DecoratedKey key)
+        public Predicate<Long> getPurgeEvaluator(DecoratedKey key)
         {
-            return Long.MIN_VALUE;
+            return Predicates.alwaysFalse();
         }
     }
 }
diff --git a/src/java/org/apache/cassandra/db/compaction/Verifier.java b/src/java/org/apache/cassandra/db/compaction/Verifier.java
new file mode 100644
index 0000000..42302fe
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/compaction/Verifier.java
@@ -0,0 +1,286 @@
+/*
+ * 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.db.compaction;
+
+import com.google.common.base.Predicate;
+import com.google.common.base.Predicates;
+import com.google.common.base.Throwables;
+import com.google.common.collect.Sets;
+import org.apache.cassandra.db.*;
+
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.SSTableIdentityIterator;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.DataIntegrityMetadata;
+import org.apache.cassandra.io.util.DataIntegrityMetadata.FileDigestValidator;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.service.ActiveRepairService;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.UUIDGen;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOError;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.*;
+
+public class Verifier implements Closeable
+{
+    private final ColumnFamilyStore cfs;
+    private final SSTableReader sstable;
+
+    private final CompactionController controller;
+
+
+    private final RandomAccessReader dataFile;
+    private final RandomAccessReader indexFile;
+    private final VerifyInfo verifyInfo;
+    private final RowIndexEntry.IndexSerializer rowIndexEntrySerializer;
+
+    private int goodRows;
+    private int badRows;
+
+    private final OutputHandler outputHandler;
+    private FileDigestValidator validator;
+
+    public Verifier(ColumnFamilyStore cfs, SSTableReader sstable, boolean isOffline) throws IOException
+    {
+        this(cfs, sstable, new OutputHandler.LogOutput(), isOffline);
+    }
+
+    public Verifier(ColumnFamilyStore cfs, SSTableReader sstable, OutputHandler outputHandler, boolean isOffline) throws IOException
+    {
+        this.cfs = cfs;
+        this.sstable = sstable;
+        this.outputHandler = outputHandler;
+        this.rowIndexEntrySerializer = sstable.descriptor.version.getSSTableFormat().getIndexSerializer(sstable.metadata);
+
+        this.controller = new VerifyController(cfs);
+
+        this.dataFile = isOffline
+                        ? sstable.openDataReader()
+                        : sstable.openDataReader(CompactionManager.instance.getRateLimiter());
+        this.indexFile = RandomAccessReader.open(new File(sstable.descriptor.filenameFor(Component.PRIMARY_INDEX)));
+        this.verifyInfo = new VerifyInfo(dataFile, sstable);
+    }
+
+    public void verify(boolean extended) throws IOException
+    {
+        long rowStart = 0;
+
+        outputHandler.output(String.format("Verifying %s (%s bytes)", sstable, dataFile.length()));
+        outputHandler.output(String.format("Checking computed hash of %s ", sstable));
+
+
+        // Verify will use the adler32 Digest files, which works for both compressed and uncompressed sstables
+        try
+        {
+            validator = null;
+
+            if (new File(sstable.descriptor.filenameFor(Component.DIGEST)).exists())
+            {
+                validator = DataIntegrityMetadata.fileDigestValidator(sstable.descriptor);
+                validator.validate();
+            }
+            else
+            {
+                outputHandler.output("Data digest missing, assuming extended verification of disk atoms");
+                extended = true;
+            }
+        }
+        catch (IOException e)
+        {
+            outputHandler.debug(e.getMessage());
+            markAndThrow();
+        }
+        finally
+        {
+            FileUtils.closeQuietly(validator);
+        }
+
+        if ( !extended )
+            return;
+
+        outputHandler.output("Extended Verify requested, proceeding to inspect atoms");
+
+
+        try
+        {
+            ByteBuffer nextIndexKey = ByteBufferUtil.readWithShortLength(indexFile);
+            {
+                long firstRowPositionFromIndex = rowIndexEntrySerializer.deserialize(indexFile, sstable.descriptor.version).position;
+                if (firstRowPositionFromIndex != 0)
+                    markAndThrow();
+            }
+
+            DecoratedKey prevKey = null;
+
+            while (!dataFile.isEOF())
+            {
+
+                if (verifyInfo.isStopRequested())
+                    throw new CompactionInterruptedException(verifyInfo.getCompactionInfo());
+
+                rowStart = dataFile.getFilePointer();
+                outputHandler.debug("Reading row at " + rowStart);
+
+                DecoratedKey key = null;
+                try
+                {
+                    key = sstable.partitioner.decorateKey(ByteBufferUtil.readWithShortLength(dataFile));
+                }
+                catch (Throwable th)
+                {
+                    throwIfFatal(th);
+                    // check for null key below
+                }
+
+                ByteBuffer currentIndexKey = nextIndexKey;
+                long nextRowPositionFromIndex = 0;
+                try
+                {
+                    nextIndexKey = indexFile.isEOF() ? null : ByteBufferUtil.readWithShortLength(indexFile);
+                    nextRowPositionFromIndex = indexFile.isEOF()
+                                             ? dataFile.length()
+                                             : rowIndexEntrySerializer.deserialize(indexFile, sstable.descriptor.version).position;
+                }
+                catch (Throwable th)
+                {
+                    markAndThrow();
+                }
+
+                long dataStart = dataFile.getFilePointer();
+                long dataStartFromIndex = currentIndexKey == null
+                                        ? -1
+                                        : rowStart + 2 + currentIndexKey.remaining();
+
+                long dataSize = nextRowPositionFromIndex - dataStartFromIndex;
+                // avoid an NPE if key is null
+                String keyName = key == null ? "(unreadable key)" : ByteBufferUtil.bytesToHex(key.getKey());
+                outputHandler.debug(String.format("row %s is %s bytes", keyName, dataSize));
+
+                assert currentIndexKey != null || indexFile.isEOF();
+
+                try
+                {
+                    if (key == null || dataSize > dataFile.length())
+                        markAndThrow();
+
+                    //mimic the scrub read path
+                    new SSTableIdentityIterator(sstable, dataFile, key, true);
+                    if ( (prevKey != null && prevKey.compareTo(key) > 0) || !key.getKey().equals(currentIndexKey) || dataStart != dataStartFromIndex )
+                        markAndThrow();
+                    
+                    goodRows++;
+                    prevKey = key;
+
+
+                    outputHandler.debug(String.format("Row %s at %s valid, moving to next row at %s ", goodRows, rowStart, nextRowPositionFromIndex));
+                    dataFile.seek(nextRowPositionFromIndex);
+                }
+                catch (Throwable th)
+                {
+                    badRows++;
+                    markAndThrow();
+                }
+            }
+        }
+        catch (Throwable t)
+        {
+            throw Throwables.propagate(t);
+        }
+        finally
+        {
+            controller.close();
+        }
+
+        outputHandler.output("Verify of " + sstable + " succeeded. All " + goodRows + " rows read successfully");
+    }
+
+    public void close()
+    {
+        FileUtils.closeQuietly(dataFile);
+        FileUtils.closeQuietly(indexFile);
+    }
+
+    private void throwIfFatal(Throwable th)
+    {
+        if (th instanceof Error && !(th instanceof AssertionError || th instanceof IOError))
+            throw (Error) th;
+    }
+
+    private void markAndThrow() throws IOException
+    {
+        sstable.descriptor.getMetadataSerializer().mutateRepairedAt(sstable.descriptor, ActiveRepairService.UNREPAIRED_SSTABLE);
+        throw new CorruptSSTableException(new Exception(String.format("Invalid SSTable %s, please force repair", sstable.getFilename())), sstable.getFilename());
+    }
+
+    public CompactionInfo.Holder getVerifyInfo()
+    {
+        return verifyInfo;
+    }
+
+    private static class VerifyInfo extends CompactionInfo.Holder
+    {
+        private final RandomAccessReader dataFile;
+        private final SSTableReader sstable;
+        private final UUID verificationCompactionId;
+
+        public VerifyInfo(RandomAccessReader dataFile, SSTableReader sstable)
+        {
+            this.dataFile = dataFile;
+            this.sstable = sstable;
+            verificationCompactionId = UUIDGen.getTimeUUID();
+        }
+
+        public CompactionInfo getCompactionInfo()
+        {
+            try
+            {
+                return new CompactionInfo(sstable.metadata,
+                                          OperationType.VERIFY,
+                                          dataFile.getFilePointer(),
+                                          dataFile.length(),
+                                          verificationCompactionId);
+            }
+            catch (Exception e)
+            {
+                throw new RuntimeException();
+            }
+        }
+    }
+
+    private static class VerifyController extends CompactionController
+    {
+        public VerifyController(ColumnFamilyStore cfs)
+        {
+            super(cfs, Integer.MAX_VALUE);
+        }
+
+        @Override
+        public Predicate<Long> getPurgeEvaluator(DecoratedKey key)
+        {
+            return Predicates.alwaysFalse();
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/compaction/WrappingCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/WrappingCompactionStrategy.java
index 71a6bc1..8555432 100644
--- a/src/java/org/apache/cassandra/db/compaction/WrappingCompactionStrategy.java
+++ b/src/java/org/apache/cassandra/db/compaction/WrappingCompactionStrategy.java
@@ -32,10 +32,11 @@
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.ISSTableScanner;
-import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.notifications.INotification;
 import org.apache.cassandra.notifications.INotificationConsumer;
 import org.apache.cassandra.notifications.SSTableAddedNotification;
@@ -62,8 +63,8 @@
     {
         super(cfs, cfs.metadata.compactionStrategyOptions);
         reloadCompactionStrategy(cfs.metadata);
-        cfs.getDataTracker().subscribe(this);
-        logger.debug("{} subscribed to the data tracker.", this);
+        cfs.getTracker().subscribe(this);
+        logger.trace("{} subscribed to the data tracker.", this);
     }
 
     @Override
@@ -90,7 +91,7 @@
     }
 
     @Override
-    public Collection<AbstractCompactionTask> getMaximalTask(final int gcBefore)
+    public Collection<AbstractCompactionTask> getMaximalTask(final int gcBefore, final boolean splitOutput)
     {
         // runWithCompactionsDisabled cancels active compactions and disables them, then we are able
         // to make the repaired/unrepaired strategies mark their own sstables as compacting. Once the
@@ -102,8 +103,8 @@
             {
                 synchronized (WrappingCompactionStrategy.this)
                 {
-                    Collection<AbstractCompactionTask> repairedTasks = repaired.getMaximalTask(gcBefore);
-                    Collection<AbstractCompactionTask> unrepairedTasks = unrepaired.getMaximalTask(gcBefore);
+                    Collection<AbstractCompactionTask> repairedTasks = repaired.getMaximalTask(gcBefore, splitOutput);
+                    Collection<AbstractCompactionTask> unrepairedTasks = unrepaired.getMaximalTask(gcBefore, splitOutput);
 
                     if (repairedTasks == null && unrepairedTasks == null)
                         return null;
@@ -123,18 +124,18 @@
     }
 
     @Override
-    public AbstractCompactionTask getCompactionTask(Collection<SSTableReader> sstables, final int gcBefore, long maxSSTableBytes)
+    public AbstractCompactionTask getCompactionTask(LifecycleTransaction txn, final int gcBefore, long maxSSTableBytes)
     {
-        assert sstables.size() > 0;
-        boolean repairedSSTables = sstables.iterator().next().isRepaired();
-        for (SSTableReader sstable : sstables)
+        assert txn.originals().size() > 0;
+        boolean repairedSSTables = txn.originals().iterator().next().isRepaired();
+        for (SSTableReader sstable : txn.originals())
             if (repairedSSTables != sstable.isRepaired())
                 throw new RuntimeException("Can't mix repaired and unrepaired sstables in a compaction");
 
         if (repairedSSTables)
-            return repaired.getCompactionTask(sstables, gcBefore, maxSSTableBytes);
+            return repaired.getCompactionTask(txn, gcBefore, maxSSTableBytes);
         else
-            return unrepaired.getCompactionTask(sstables, gcBefore, maxSSTableBytes);
+            return unrepaired.getCompactionTask(txn, gcBefore, maxSSTableBytes);
     }
 
     @Override
@@ -394,6 +395,7 @@
     }
 
     @Override
+    @SuppressWarnings("resource")
     public synchronized ScannerList getScanners(Collection<SSTableReader> sstables, Range<Token> range)
     {
         List<SSTableReader> repairedSSTables = new ArrayList<>();
@@ -411,6 +413,11 @@
         return new ScannerList(scanners);
     }
 
+    public Collection<Collection<SSTableReader>> groupSSTablesForAntiCompaction(Collection<SSTableReader> sstablesToGroup)
+    {
+        return unrepaired.groupSSTablesForAntiCompaction(sstablesToGroup);
+    }
+
     public List<AbstractCompactionStrategy> getWrappedStrategies()
     {
         return Arrays.asList(repaired, unrepaired);
diff --git a/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java
new file mode 100644
index 0000000..20c96d6
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java
@@ -0,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.db.compaction.writers;
+
+import java.util.List;
+import java.util.Set;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Directories;
+import org.apache.cassandra.db.compaction.AbstractCompactedRow;
+import org.apache.cassandra.db.compaction.CompactionTask;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.io.sstable.SSTableRewriter;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.utils.concurrent.Transactional;
+
+
+/**
+ * Class that abstracts away the actual writing of files to make it possible to use CompactionTask for more
+ * use cases.
+ */
+public abstract class CompactionAwareWriter extends Transactional.AbstractTransactional implements Transactional
+{
+    protected final ColumnFamilyStore cfs;
+    protected final Set<SSTableReader> nonExpiredSSTables;
+    protected final long estimatedTotalKeys;
+    protected final long maxAge;
+    protected final long minRepairedAt;
+    protected final SSTableRewriter sstableWriter;
+
+    public CompactionAwareWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables, boolean offline)
+    {
+        this.cfs = cfs;
+        this.nonExpiredSSTables = nonExpiredSSTables;
+        this.estimatedTotalKeys = SSTableReader.getApproximateKeyCount(nonExpiredSSTables);
+        this.maxAge = CompactionTask.getMaxDataAge(nonExpiredSSTables);
+        this.minRepairedAt = CompactionTask.getMinRepairedAt(nonExpiredSSTables);
+        this.sstableWriter = new SSTableRewriter(cfs, txn, maxAge, offline);
+    }
+
+    /**
+     * Writes a row in an implementation specific way
+     * @param row the row to append
+     * @return true if the row was written, false otherwise
+     */
+    public abstract boolean append(AbstractCompactedRow row);
+
+    @Override
+    protected Throwable doAbort(Throwable accumulate)
+    {
+        return sstableWriter.abort(accumulate);
+    }
+
+    @Override
+    protected Throwable doCommit(Throwable accumulate)
+    {
+        return sstableWriter.commit(accumulate);
+    }
+
+    @Override
+    protected void doPrepare()
+    {
+        sstableWriter.prepareToCommit();
+    }
+
+    /**
+     * we are done, return the finished sstables so that the caller can mark the old ones as compacted
+     * @return all the written sstables sstables
+     */
+    @Override
+    public List<SSTableReader> finish()
+    {
+        super.finish();
+        return sstableWriter.finished();
+    }
+
+    /**
+     * estimated number of keys we should write
+     */
+    public long estimatedKeys()
+    {
+        return estimatedTotalKeys;
+    }
+
+    /**
+     * The directories we can write to
+     */
+    public Directories getDirectories()
+    {
+        return cfs.directories;
+    }
+
+    /**
+     * Return a directory where we can expect expectedWriteSize to fit.
+     */
+    public Directories.DataDirectory getWriteDirectory(long expectedWriteSize)
+    {
+        Directories.DataDirectory directory = getDirectories().getWriteableLocation(expectedWriteSize);
+        if (directory == null)
+            throw new RuntimeException("Insufficient disk space to write " + expectedWriteSize + " bytes");
+
+        return directory;
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java
new file mode 100644
index 0000000..6611b3f
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java
@@ -0,0 +1,72 @@
+/*
+ * 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.db.compaction.writers;
+
+
+import java.io.File;
+import java.util.Set;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.compaction.AbstractCompactedRow;
+import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
+
+
+/**
+ * The default compaction writer - creates one output file in L0
+ */
+public class DefaultCompactionWriter extends CompactionAwareWriter
+{
+    protected static final Logger logger = LoggerFactory.getLogger(DefaultCompactionWriter.class);
+
+    @SuppressWarnings("resource")
+    public DefaultCompactionWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables, boolean offline, OperationType compactionType)
+    {
+        super(cfs, txn, nonExpiredSSTables, offline);
+        logger.trace("Expected bloom filter size : {}", estimatedTotalKeys);
+        long expectedWriteSize = cfs.getExpectedCompactedFileSize(nonExpiredSSTables, compactionType);
+        File sstableDirectory = cfs.directories.getLocationForDisk(getWriteDirectory(expectedWriteSize));
+        @SuppressWarnings("resource")
+        SSTableWriter writer = SSTableWriter.create(Descriptor.fromFilename(cfs.getTempSSTablePath(sstableDirectory)),
+                                                    estimatedTotalKeys,
+                                                    minRepairedAt,
+                                                    cfs.metadata,
+                                                    cfs.partitioner,
+                                                    new MetadataCollector(txn.originals(), cfs.metadata.comparator, 0));
+        sstableWriter.switchWriter(writer);
+    }
+
+    @Override
+    public boolean append(AbstractCompactedRow row)
+    {
+        return sstableWriter.append(row) != null;
+    }
+
+    @Override
+    public long estimatedKeys()
+    {
+        return estimatedTotalKeys;
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/db/compaction/writers/MajorLeveledCompactionWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/MajorLeveledCompactionWriter.java
new file mode 100644
index 0000000..95d7a0c
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/compaction/writers/MajorLeveledCompactionWriter.java
@@ -0,0 +1,106 @@
+/*
+ * 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.db.compaction.writers;
+
+import java.io.File;
+import java.util.Set;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.RowIndexEntry;
+import org.apache.cassandra.db.compaction.AbstractCompactedRow;
+import org.apache.cassandra.db.compaction.LeveledManifest;
+import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
+
+public class MajorLeveledCompactionWriter extends CompactionAwareWriter
+{
+    private static final Logger logger = LoggerFactory.getLogger(MajorLeveledCompactionWriter.class);
+    private final long maxSSTableSize;
+    private final long expectedWriteSize;
+    private final Set<SSTableReader> allSSTables;
+    private int currentLevel = 1;
+    private long averageEstimatedKeysPerSSTable;
+    private long partitionsWritten = 0;
+    private long totalWrittenInLevel = 0;
+    private int sstablesWritten = 0;
+    private final boolean skipAncestors;
+
+    @SuppressWarnings("resource")
+    public MajorLeveledCompactionWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables, long maxSSTableSize, boolean offline, OperationType compactionType)
+    {
+        super(cfs, txn, nonExpiredSSTables, offline);
+        this.maxSSTableSize = maxSSTableSize;
+        this.allSSTables = txn.originals();
+        expectedWriteSize = Math.min(maxSSTableSize, cfs.getExpectedCompactedFileSize(nonExpiredSSTables, compactionType));
+        long estimatedSSTables = Math.max(1, SSTableReader.getTotalBytes(nonExpiredSSTables) / maxSSTableSize);
+        long keysPerSSTable = estimatedTotalKeys / estimatedSSTables;
+        File sstableDirectory = cfs.directories.getLocationForDisk(getWriteDirectory(expectedWriteSize));
+        skipAncestors = estimatedSSTables * allSSTables.size() > 200000; // magic number, avoid storing too much ancestor information since allSSTables are ancestors to *all* resulting sstables
+
+        if (skipAncestors)
+            logger.warn("Many sstables involved in compaction, skipping storing ancestor information to avoid running out of memory");
+
+        @SuppressWarnings("resource")
+        SSTableWriter writer = SSTableWriter.create(Descriptor.fromFilename(cfs.getTempSSTablePath(sstableDirectory)),
+                                                    keysPerSSTable,
+                                                    minRepairedAt,
+                                                    cfs.metadata,
+                                                    cfs.partitioner,
+                                                    new MetadataCollector(allSSTables, cfs.metadata.comparator, currentLevel, skipAncestors));
+        sstableWriter.switchWriter(writer);
+    }
+
+    @Override
+    @SuppressWarnings("resource")
+    public boolean append(AbstractCompactedRow row)
+    {
+        long posBefore = sstableWriter.currentWriter().getOnDiskFilePointer();
+        RowIndexEntry rie = sstableWriter.append(row);
+        totalWrittenInLevel += sstableWriter.currentWriter().getOnDiskFilePointer() - posBefore;
+        partitionsWritten++;
+        if (sstableWriter.currentWriter().getOnDiskFilePointer() > maxSSTableSize)
+        {
+            if (totalWrittenInLevel > LeveledManifest.maxBytesForLevel(currentLevel, maxSSTableSize))
+            {
+                totalWrittenInLevel = 0;
+                currentLevel++;
+            }
+
+            averageEstimatedKeysPerSSTable = Math.round(((double) averageEstimatedKeysPerSSTable * sstablesWritten + partitionsWritten) / (sstablesWritten + 1));
+            File sstableDirectory = cfs.directories.getLocationForDisk(getWriteDirectory(expectedWriteSize));
+            SSTableWriter writer = SSTableWriter.create(Descriptor.fromFilename(cfs.getTempSSTablePath(sstableDirectory)),
+                                                        averageEstimatedKeysPerSSTable,
+                                                        minRepairedAt,
+                                                        cfs.metadata,
+                                                        cfs.partitioner,
+                                                        new MetadataCollector(allSSTables, cfs.metadata.comparator, currentLevel, skipAncestors));
+            sstableWriter.switchWriter(writer);
+            partitionsWritten = 0;
+            sstablesWritten++;
+        }
+        return rie != null;
+
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java
new file mode 100644
index 0000000..2bae3b8
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java
@@ -0,0 +1,94 @@
+/*
+ * 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.db.compaction.writers;
+
+import java.io.File;
+import java.util.Set;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.RowIndexEntry;
+import org.apache.cassandra.db.compaction.AbstractCompactedRow;
+import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
+
+public class MaxSSTableSizeWriter extends CompactionAwareWriter
+{
+    private final long expectedWriteSize;
+    private final long maxSSTableSize;
+    private final int level;
+    private final long estimatedSSTables;
+    private final Set<SSTableReader> allSSTables;
+
+    @SuppressWarnings("resource")
+    public MaxSSTableSizeWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables, long maxSSTableSize, int level, boolean offline, OperationType compactionType)
+    {
+        super(cfs, txn, nonExpiredSSTables, offline);
+        this.allSSTables = txn.originals();
+        this.level = level;
+        this.maxSSTableSize = maxSSTableSize;
+        long totalSize = getTotalWriteSize(nonExpiredSSTables, estimatedTotalKeys, cfs, compactionType);
+        expectedWriteSize = Math.min(maxSSTableSize, totalSize);
+        estimatedSSTables = Math.max(1, totalSize / maxSSTableSize);
+        File sstableDirectory = cfs.directories.getLocationForDisk(getWriteDirectory(expectedWriteSize));
+        @SuppressWarnings("resource")
+        SSTableWriter writer = SSTableWriter.create(Descriptor.fromFilename(cfs.getTempSSTablePath(sstableDirectory)),
+                                                    estimatedTotalKeys / estimatedSSTables,
+                                                    minRepairedAt,
+                                                    cfs.metadata,
+                                                    cfs.partitioner,
+                                                    new MetadataCollector(allSSTables, cfs.metadata.comparator, level));
+        sstableWriter.switchWriter(writer);
+    }
+
+    /**
+     * Gets the estimated total amount of data to write during compaction
+     */
+    private static long getTotalWriteSize(Iterable<SSTableReader> nonExpiredSSTables, long estimatedTotalKeys, ColumnFamilyStore cfs, OperationType compactionType)
+    {
+        long estimatedKeysBeforeCompaction = 0;
+        for (SSTableReader sstable : nonExpiredSSTables)
+            estimatedKeysBeforeCompaction += sstable.estimatedKeys();
+        estimatedKeysBeforeCompaction = Math.max(1, estimatedKeysBeforeCompaction);
+        double estimatedCompactionRatio = (double) estimatedTotalKeys / estimatedKeysBeforeCompaction;
+        return Math.round(estimatedCompactionRatio * cfs.getExpectedCompactedFileSize(nonExpiredSSTables, compactionType));
+    }
+
+    @Override
+    public boolean append(AbstractCompactedRow row)
+    {
+        RowIndexEntry rie = sstableWriter.append(row);
+        if (sstableWriter.currentWriter().getOnDiskFilePointer() > maxSSTableSize)
+        {
+            File sstableDirectory = cfs.directories.getLocationForDisk(getWriteDirectory(expectedWriteSize));
+            @SuppressWarnings("resource")
+            SSTableWriter writer = SSTableWriter.create(Descriptor.fromFilename(cfs.getTempSSTablePath(sstableDirectory)),
+                                                                estimatedTotalKeys / estimatedSSTables,
+                                                                minRepairedAt,
+                                                                cfs.metadata,
+                                                                cfs.partitioner,
+                                                                new MetadataCollector(allSSTables, cfs.metadata.comparator, level));
+
+            sstableWriter.switchWriter(writer);
+        }
+        return rie != null;
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java
new file mode 100644
index 0000000..ed07df9
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java
@@ -0,0 +1,121 @@
+/*
+ * 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.db.compaction.writers;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.Set;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.RowIndexEntry;
+import org.apache.cassandra.db.compaction.AbstractCompactedRow;
+import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
+
+/**
+ * CompactionAwareWriter that splits input in differently sized sstables
+ *
+ * Biggest sstable will be total_compaction_size / 2, second biggest total_compaction_size / 4 etc until
+ * the result would be sub 50MB, all those are put in the same
+ */
+public class SplittingSizeTieredCompactionWriter extends CompactionAwareWriter
+{
+    private static final Logger logger = LoggerFactory.getLogger(SplittingSizeTieredCompactionWriter.class);
+
+    public static final long DEFAULT_SMALLEST_SSTABLE_BYTES = 50_000_000;
+    private final double[] ratios;
+    private final long totalSize;
+    private final Set<SSTableReader> allSSTables;
+    private long currentBytesToWrite;
+    private int currentRatioIndex = 0;
+
+    public SplittingSizeTieredCompactionWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables, OperationType compactionType)
+    {
+        this(cfs, txn, nonExpiredSSTables, compactionType, DEFAULT_SMALLEST_SSTABLE_BYTES);
+    }
+
+    @SuppressWarnings("resource")
+    public SplittingSizeTieredCompactionWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables, OperationType compactionType, long smallestSSTable)
+    {
+        super(cfs, txn, nonExpiredSSTables, false);
+        this.allSSTables = txn.originals();
+        totalSize = cfs.getExpectedCompactedFileSize(nonExpiredSSTables, compactionType);
+        double[] potentialRatios = new double[20];
+        double currentRatio = 1;
+        for (int i = 0; i < potentialRatios.length; i++)
+        {
+            currentRatio /= 2;
+            potentialRatios[i] = currentRatio;
+        }
+
+        int noPointIndex = 0;
+        // find how many sstables we should create - 50MB min sstable size
+        for (double ratio : potentialRatios)
+        {
+            noPointIndex++;
+            if (ratio * totalSize < smallestSSTable)
+            {
+                break;
+            }
+        }
+        ratios = Arrays.copyOfRange(potentialRatios, 0, noPointIndex);
+        File sstableDirectory = cfs.directories.getLocationForDisk(getWriteDirectory(Math.round(totalSize * ratios[currentRatioIndex])));
+        long currentPartitionsToWrite = Math.round(estimatedTotalKeys * ratios[currentRatioIndex]);
+        currentBytesToWrite = Math.round(totalSize * ratios[currentRatioIndex]);
+        @SuppressWarnings("resource")
+        SSTableWriter writer = SSTableWriter.create(Descriptor.fromFilename(cfs.getTempSSTablePath(sstableDirectory)),
+                                                                            currentPartitionsToWrite,
+                                                                            minRepairedAt,
+                                                                            cfs.metadata,
+                                                                            cfs.partitioner,
+                                                                            new MetadataCollector(allSSTables, cfs.metadata.comparator, 0));
+
+        sstableWriter.switchWriter(writer);
+        logger.trace("Ratios={}, expectedKeys = {}, totalSize = {}, currentPartitionsToWrite = {}, currentBytesToWrite = {}", ratios, estimatedTotalKeys, totalSize, currentPartitionsToWrite, currentBytesToWrite);
+    }
+
+    @Override
+    public boolean append(AbstractCompactedRow row)
+    {
+        RowIndexEntry rie = sstableWriter.append(row);
+        if (sstableWriter.currentWriter().getOnDiskFilePointer() > currentBytesToWrite && currentRatioIndex < ratios.length - 1) // if we underestimate how many keys we have, the last sstable might get more than we expect
+        {
+            currentRatioIndex++;
+            currentBytesToWrite = Math.round(totalSize * ratios[currentRatioIndex]);
+            long currentPartitionsToWrite = Math.round(ratios[currentRatioIndex] * estimatedTotalKeys);
+            File sstableDirectory = cfs.directories.getLocationForDisk(getWriteDirectory(Math.round(totalSize * ratios[currentRatioIndex])));
+            @SuppressWarnings("resource")
+            SSTableWriter writer = SSTableWriter.create(Descriptor.fromFilename(cfs.getTempSSTablePath(sstableDirectory)),
+                                                                                currentPartitionsToWrite,
+                                                                                minRepairedAt,
+                                                                                cfs.metadata,
+                                                                                cfs.partitioner,
+                                                                                new MetadataCollector(allSSTables, cfs.metadata.comparator, 0));
+            sstableWriter.switchWriter(writer);
+            logger.trace("Switching writer, currentPartitionsToWrite = {}", currentPartitionsToWrite);
+        }
+        return rie != null;
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/db/composites/AbstractCType.java b/src/java/org/apache/cassandra/db/composites/AbstractCType.java
index fecc847..2190c69 100644
--- a/src/java/org/apache/cassandra/db/composites/AbstractCType.java
+++ b/src/java/org/apache/cassandra/db/composites/AbstractCType.java
@@ -26,13 +26,11 @@
 import org.apache.cassandra.db.DeletionInfo;
 import org.apache.cassandra.db.NativeCell;
 import org.apache.cassandra.db.RangeTombstone;
-import org.apache.cassandra.db.RowIndexEntry;
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.db.filter.ColumnSlice;
 import org.apache.cassandra.db.filter.SliceQueryFilter;
 import org.apache.cassandra.db.marshal.AbstractCompositeType;
 import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.io.ISerializer;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -80,12 +78,10 @@
 
     private final Serializer serializer;
 
-    private final ISerializer<IndexInfo> indexSerializer;
     private final IVersionedSerializer<ColumnSlice> sliceSerializer;
     private final IVersionedSerializer<SliceQueryFilter> sliceQueryFilterSerializer;
     private final DeletionInfo.Serializer deletionInfoSerializer;
     private final RangeTombstone.Serializer rangeTombstoneSerializer;
-    private final RowIndexEntry.Serializer rowIndexEntrySerializer;
 
     protected final boolean isByteOrderComparable;
 
@@ -115,12 +111,10 @@
 
         serializer = new Serializer(this);
 
-        indexSerializer = new IndexInfo.Serializer(this);
         sliceSerializer = new ColumnSlice.Serializer(this);
         sliceQueryFilterSerializer = new SliceQueryFilter.Serializer(this);
         deletionInfoSerializer = new DeletionInfo.Serializer(this);
         rangeTombstoneSerializer = new RangeTombstone.Serializer(this);
-        rowIndexEntrySerializer = new RowIndexEntry.Serializer(this);
         this.isByteOrderComparable = isByteOrderComparable;
     }
 
@@ -295,11 +289,6 @@
         return indexReverseComparator;
     }
 
-    public ISerializer<IndexInfo> indexSerializer()
-    {
-        return indexSerializer;
-    }
-
     public IVersionedSerializer<ColumnSlice> sliceSerializer()
     {
         return sliceSerializer;
@@ -320,11 +309,6 @@
         return rangeTombstoneSerializer;
     }
 
-    public RowIndexEntry.Serializer rowIndexEntrySerializer()
-    {
-        return rowIndexEntrySerializer;
-    }
-
     @Override
     public boolean equals(Object o)
     {
diff --git a/src/java/org/apache/cassandra/db/composites/AbstractCompoundCellNameType.java b/src/java/org/apache/cassandra/db/composites/AbstractCompoundCellNameType.java
index 40537ed..bf303a7 100644
--- a/src/java/org/apache/cassandra/db/composites/AbstractCompoundCellNameType.java
+++ b/src/java/org/apache/cassandra/db/composites/AbstractCompoundCellNameType.java
@@ -20,9 +20,7 @@
 import java.io.DataInput;
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.util.Comparator;
 
-import org.apache.cassandra.db.Cell;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.CompositeType;
 
diff --git a/src/java/org/apache/cassandra/db/composites/CType.java b/src/java/org/apache/cassandra/db/composites/CType.java
index 280f7af..7f70313 100644
--- a/src/java/org/apache/cassandra/db/composites/CType.java
+++ b/src/java/org/apache/cassandra/db/composites/CType.java
@@ -24,7 +24,6 @@
 
 import org.apache.cassandra.db.DeletionInfo;
 import org.apache.cassandra.db.RangeTombstone;
-import org.apache.cassandra.db.RowIndexEntry;
 import org.apache.cassandra.db.filter.ColumnSlice;
 import org.apache.cassandra.db.filter.SliceQueryFilter;
 import org.apache.cassandra.db.marshal.AbstractType;
@@ -99,19 +98,19 @@
     /**
      * Deserialize a Composite from a ByteBuffer.
      *
-     * This is meant for thrift/cql2 to convert the fully serialized buffer we
+     * This is meant for thrift to convert the fully serialized buffer we
      * get from the clients to composites.
      */
     public Composite fromByteBuffer(ByteBuffer bb);
 
     /**
-     * Returns a AbstractType corresponding to this CType for thrift/cql2 sake.
+     * Returns a AbstractType corresponding to this CType for thrift sake.
      *
      * If the CType is a "simple" one, this just return the wrapped type, otherwise
      * it returns the corresponding org.apache.cassandra.db.marshal.CompositeType.
      *
      * This is only meant to be use for backward compatibility (particularly for
-     * thrift/cql2) but it's not meant to be used internally.
+     * thrift) but it's not meant to be used internally.
      */
     public AbstractType<?> asAbstractType();
 
@@ -130,12 +129,10 @@
 
     public Serializer serializer();
 
-    public ISerializer<IndexInfo> indexSerializer();
     public IVersionedSerializer<ColumnSlice> sliceSerializer();
     public IVersionedSerializer<SliceQueryFilter> sliceQueryFilterSerializer();
     public DeletionInfo.Serializer deletionInfoSerializer();
     public RangeTombstone.Serializer rangeTombstoneSerializer();
-    public RowIndexEntry.Serializer rowIndexEntrySerializer();
 
     public interface Serializer extends ISerializer<Composite>
     {
diff --git a/src/java/org/apache/cassandra/db/composites/CellNameType.java b/src/java/org/apache/cassandra/db/composites/CellNameType.java
index 7b4fd36..6c89660 100644
--- a/src/java/org/apache/cassandra/db/composites/CellNameType.java
+++ b/src/java/org/apache/cassandra/db/composites/CellNameType.java
@@ -145,7 +145,7 @@
      * Deserialize a Composite from a ByteBuffer.
      *
      * This is equilvalent to CType#fromByteBuffer but assumes the buffer is a full cell
-     * name. This is meant for thrift/cql2 to convert the fully serialized buffer we
+     * name. This is meant for thrift to convert the fully serialized buffer we
      * get from the clients.
      */
     public CellName cellFromByteBuffer(ByteBuffer bb);
diff --git a/src/java/org/apache/cassandra/db/composites/Composites.java b/src/java/org/apache/cassandra/db/composites/Composites.java
index f6626e0..fa0df48 100644
--- a/src/java/org/apache/cassandra/db/composites/Composites.java
+++ b/src/java/org/apache/cassandra/db/composites/Composites.java
@@ -22,8 +22,11 @@
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.filter.ColumnSlice;
-import org.apache.cassandra.utils.memory.AbstractAllocator;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Lists;
 
 public abstract class Composites
 {
@@ -31,6 +34,23 @@
 
     public static final Composite EMPTY = new EmptyComposite();
 
+    /**
+     * Converts the specified <code>Composites</code> into <code>ByteBuffer</code>s.
+     *
+     * @param composites the composites to convert.
+     * @return the <code>ByteBuffer</code>s corresponding to the specified <code>Composites</code>.
+     */
+    public static List<ByteBuffer> toByteBuffers(List<Composite> composites)
+    {
+        return Lists.transform(composites, new Function<Composite, ByteBuffer>()
+        {
+            public ByteBuffer apply(Composite composite)
+            {
+                return composite.toByteBuffer();
+            }
+        });
+    }
+
     static final CBuilder EMPTY_BUILDER = new CBuilder()
     {
         public int remainingCount() { return 0; }
diff --git a/src/java/org/apache/cassandra/db/composites/CompositesBuilder.java b/src/java/org/apache/cassandra/db/composites/CompositesBuilder.java
new file mode 100644
index 0000000..9a4da9e
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/composites/CompositesBuilder.java
@@ -0,0 +1,299 @@
+/*
+ * 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.db.composites;
+
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import org.apache.cassandra.db.composites.Composite.EOC;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static java.util.Collections.singletonList;
+
+/**
+ * Builder that allow to build multiple composites at the same time.
+ */
+public final class CompositesBuilder
+{
+    /**
+     * The composite type.
+     */
+    private final CType ctype;
+
+    /**
+     * The elements of the composites
+     */
+    private final List<List<ByteBuffer>> elementsList = new ArrayList<>();
+
+    /**
+     * The number of elements that have been added.
+     */
+    private int size;
+
+    /**
+     * <code>true</code> if the composites have been build, <code>false</code> otherwise.
+     */
+    private boolean built;
+
+    /**
+     * <code>true</code> if the composites contains some <code>null</code> elements.
+     */
+    private boolean containsNull;
+
+    /**
+     * <code>true</code> if some empty collection have been added.
+     */
+    private boolean hasMissingElements;
+
+    /**
+     * <code>true</code> if the composites contains some <code>unset</code> elements.
+     */
+    private boolean containsUnset;
+
+    public CompositesBuilder(CType ctype)
+    {
+        this.ctype = ctype;
+    }
+
+    /**
+     * Adds the specified element to all the composites.
+     * <p>
+     * If this builder contains 2 composites: A-B and A-C a call to this method to add D will result in the composites:
+     * A-B-D and A-C-D.
+     * </p>
+     *
+     * @param value the value of the next element
+     * @return this <code>CompositeBuilder</code>
+     */
+    public CompositesBuilder addElementToAll(ByteBuffer value)
+    {
+        checkUpdateable();
+
+        if (isEmpty())
+            elementsList.add(new ArrayList<ByteBuffer>());
+
+        for (int i = 0, m = elementsList.size(); i < m; i++)
+        {
+            if (value == null)
+                containsNull = true;
+            if (value == ByteBufferUtil.UNSET_BYTE_BUFFER)
+                containsUnset = true;
+            elementsList.get(i).add(value);
+        }
+        size++;
+        return this;
+    }
+
+    /**
+     * Adds individually each of the specified elements to the end of all of the existing composites.
+     * <p>
+     * If this builder contains 2 composites: A-B and A-C a call to this method to add D and E will result in the 4
+     * composites: A-B-D, A-B-E, A-C-D and A-C-E.
+     * </p>
+     *
+     * @param values the elements to add
+     * @return this <code>CompositeBuilder</code>
+     */
+    public CompositesBuilder addEachElementToAll(List<ByteBuffer> values)
+    {
+        checkUpdateable();
+
+        if (isEmpty())
+            elementsList.add(new ArrayList<ByteBuffer>());
+
+        if (values.isEmpty())
+        {
+            hasMissingElements = true;
+        }
+        else
+        {
+            for (int i = 0, m = elementsList.size(); i < m; i++)
+            {
+                List<ByteBuffer> oldComposite = elementsList.remove(0);
+
+                for (int j = 0, n = values.size(); j < n; j++)
+                {
+                    List<ByteBuffer> newComposite = new ArrayList<>(oldComposite);
+                    elementsList.add(newComposite);
+
+                    ByteBuffer value = values.get(j);
+
+                    if (value == null)
+                        containsNull = true;
+                    if (value == ByteBufferUtil.UNSET_BYTE_BUFFER)
+                        containsUnset = true;
+
+                    newComposite.add(values.get(j));
+                }
+            }
+        }
+        size++;
+        return this;
+    }
+
+
+    /**
+     * Adds individually each of the specified list of elements to the end of all of the existing composites.
+     * <p>
+     * If this builder contains 2 composites: A-B and A-C a call to this method to add [[D, E], [F, G]] will result in the 4
+     * composites: A-B-D-E, A-B-F-G, A-C-D-E and A-C-F-G.
+     * </p>
+     *
+     * @param values the elements to add
+     * @return this <code>CompositeBuilder</code>
+     */
+    public CompositesBuilder addAllElementsToAll(List<List<ByteBuffer>> values)
+    {
+        checkUpdateable();
+
+        if (isEmpty())
+            elementsList.add(new ArrayList<ByteBuffer>());
+
+        if (values.isEmpty())
+        {
+            hasMissingElements = true;
+        }
+        else
+        {
+            for (int i = 0, m = elementsList.size(); i < m; i++)
+            {
+                List<ByteBuffer> oldComposite = elementsList.remove(0);
+
+                for (int j = 0, n = values.size(); j < n; j++)
+                {
+                    List<ByteBuffer> newComposite = new ArrayList<>(oldComposite);
+                    elementsList.add(newComposite);
+
+                    List<ByteBuffer> value = values.get(j);
+
+                    if (value.contains(null))
+                        containsNull = true;
+                    if (value.contains(ByteBufferUtil.UNSET_BYTE_BUFFER))
+                        containsUnset = true;
+
+                    newComposite.addAll(value);
+                }
+            }
+            size += values.get(0).size();
+        }
+        return this;
+    }
+
+    /**
+     * Returns the number of elements that can be added to the composites.
+     *
+     * @return the number of elements that can be added to the composites.
+     */
+    public int remainingCount()
+    {
+        return ctype.size() - size;
+    }
+
+    /**
+     * Checks if some elements can still be added to the composites.
+     *
+     * @return <code>true</code> if it is possible to add more elements to the composites, <code>false</code> otherwise.
+     */
+    public boolean hasRemaining()
+    {
+        return remainingCount() > 0;
+    }
+
+    /**
+     * Checks if this builder is empty.
+     *
+     * @return <code>true</code> if this builder is empty, <code>false</code> otherwise.
+     */
+    public boolean isEmpty()
+    {
+        return elementsList.isEmpty();
+    }
+
+    /**
+     * Checks if the composites contains null elements.
+     *
+     * @return <code>true</code> if the composites contains <code>null</code> elements, <code>false</code> otherwise.
+     */
+    public boolean containsNull()
+    {
+        return containsNull;
+    }
+
+    /**
+     * Checks if some empty list of values have been added
+     * @return <code>true</code> if the composites have some missing elements, <code>false</code> otherwise.
+     */
+    public boolean hasMissingElements()
+    {
+        return hasMissingElements;
+    }
+
+    /**
+     * Checks if the composites contains unset elements.
+     *
+     * @return <code>true</code> if the composites contains <code>unset</code> elements, <code>false</code> otherwise.
+     */
+    public boolean containsUnset()
+    {
+        return containsUnset;
+    }
+
+    /**
+     * Builds the <code>Composites</code>.
+     *
+     * @return the composites
+     */
+    public List<Composite> build()
+    {
+        return buildWithEOC(EOC.NONE);
+    }
+
+    /**
+     * Builds the <code>Composites</code> with the specified EOC.
+     *
+     * @return the composites
+     */
+    public List<Composite> buildWithEOC(EOC eoc)
+    {
+        built = true;
+
+        if (hasMissingElements)
+            return Collections.emptyList();
+
+        CBuilder builder = ctype.builder();
+
+        if (elementsList.isEmpty())
+            return singletonList(builder.build().withEOC(eoc));
+
+        List<Composite> list = new ArrayList<>();
+
+        for (int i = 0, m = elementsList.size(); i < m; i++)
+        {
+            List<ByteBuffer> elements = elementsList.get(i);
+            list.add(builder.buildWith(elements).withEOC(eoc));
+        }
+
+        return list;
+    }
+
+    private void checkUpdateable()
+    {
+        if (!hasRemaining() || built)
+            throw new IllegalStateException("this CompositesBuilder cannot be updated anymore");
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/composites/SimpleCType.java b/src/java/org/apache/cassandra/db/composites/SimpleCType.java
index c824179..7ee45ac 100644
--- a/src/java/org/apache/cassandra/db/composites/SimpleCType.java
+++ b/src/java/org/apache/cassandra/db/composites/SimpleCType.java
@@ -18,11 +18,9 @@
 package org.apache.cassandra.db.composites;
 
 import java.nio.ByteBuffer;
-import java.util.Comparator;
 import java.util.List;
 
 import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.utils.ByteBufferUtil;
 
 /**
  * A not truly-composite CType.
diff --git a/src/java/org/apache/cassandra/db/context/CounterContext.java b/src/java/org/apache/cassandra/db/context/CounterContext.java
index 455ffc7..ffffbb1 100644
--- a/src/java/org/apache/cassandra/db/context/CounterContext.java
+++ b/src/java/org/apache/cassandra/db/context/CounterContext.java
@@ -509,7 +509,7 @@
             if (state.getElementIndex() > 0)
                 sb.append(",");
             sb.append("{");
-            sb.append(state.getCounterId().toString()).append(", ");
+            sb.append(state.getCounterId()).append(", ");
             sb.append(state.getClock()).append(", ");
             sb.append(state.getCount());
             sb.append("}");
diff --git a/src/java/org/apache/cassandra/db/filter/ColumnCounter.java b/src/java/org/apache/cassandra/db/filter/ColumnCounter.java
index d7a8873..594fde8 100644
--- a/src/java/org/apache/cassandra/db/filter/ColumnCounter.java
+++ b/src/java/org/apache/cassandra/db/filter/ColumnCounter.java
@@ -20,6 +20,8 @@
  */
 package org.apache.cassandra.db.filter;
 
+import java.util.Iterator;
+
 import org.apache.cassandra.db.Cell;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.composites.CellNameType;
@@ -37,17 +39,23 @@
         this.timestamp = timestamp;
     }
 
-    public void count(Cell cell, DeletionInfo.InOrderTester tester)
+    /**
+     * @return true if the cell counted as a live cell or a valid tombstone; false if it got immediately discarded for
+     *         being shadowed by a range- or a partition tombstone
+     */
+    public boolean count(Cell cell, DeletionInfo.InOrderTester tester)
     {
         // The cell is shadowed by a higher-level deletion, and won't be retained.
         // For the purposes of this counter, we don't care if it's a tombstone or not.
         if (tester.isDeleted(cell))
-            return;
+            return false;
 
         if (cell.isLive(timestamp))
             live++;
         else
             tombstones++;
+
+        return true;
     }
 
     public int live()
@@ -66,11 +74,18 @@
             return this;
 
         DeletionInfo.InOrderTester tester = container.inOrderDeletionTester();
-        for (Cell c : container)
-            count(c, tester);
+        Iterator<Cell> cells = getCellIterator(container);
+        while (cells.hasNext())
+            count(cells.next(), tester);
         return this;
     }
 
+    protected Iterator<Cell> getCellIterator(ColumnFamily container)
+    {
+        // overridden by GroupByPrefixReversed to return a reverse iterator
+        return container.iterator();
+    }
+
     public static class GroupByPrefix extends ColumnCounter
     {
         protected final CellNameType type;
@@ -96,21 +111,22 @@
             assert toGroup == 0 || type != null;
         }
 
-        public void count(Cell cell, DeletionInfo.InOrderTester tester)
+        @Override
+        public boolean count(Cell cell, DeletionInfo.InOrderTester tester)
         {
             if (tester.isDeleted(cell))
-                return;
+                return false;
 
             if (!cell.isLive(timestamp))
             {
                 tombstones++;
-                return;
+                return true;
             }
 
             if (toGroup == 0)
             {
                 live = 1;
-                return;
+                return true;
             }
 
             CellName current = cell.name();
@@ -132,7 +148,7 @@
                 }
 
                 if (isSameGroup)
-                    return;
+                    return true;
 
                 // We want to count the static group as 1 (CQL) row only if it's the only
                 // group in the partition. So, since we have already counted it at this point,
@@ -140,12 +156,14 @@
                 if (previous.isStatic())
                 {
                     previous = current;
-                    return;
+                    return true;
                 }
             }
 
             live++;
             previous = current;
+
+            return true;
         }
     }
 
@@ -160,21 +178,27 @@
         }
 
         @Override
-        public void count(Cell cell, DeletionInfo.InOrderTester tester)
+        public Iterator<Cell> getCellIterator(ColumnFamily container)
+        {
+            return container.reverseIterator();
+        }
+
+        @Override
+        public boolean count(Cell cell, DeletionInfo.InOrderTester tester)
         {
             if (tester.isDeleted(cell))
-                return;
+                return false;
 
             if (!cell.isLive(timestamp))
             {
                 tombstones++;
-                return;
+                return true;
             }
 
             if (toGroup == 0)
             {
                 live = 1;
-                return;
+                return true;
             }
 
             CellName current = cell.name();
@@ -197,10 +221,12 @@
                         // it's a new group
                         live++;
                         previous = current;
-                        return;
+                        return true;
                     }
                 }
             }
+
+            return true;
         }
     }
 }
diff --git a/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java b/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java
index e46dca7..bd4718a 100644
--- a/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java
@@ -214,7 +214,7 @@
             {
                 // if we have a high chance of getting all the columns in a single index slice (and it's not too costly), do that.
                 // otherwise, the extraFilter (lazily created) will fetch by name the columns referenced by the additional expressions.
-                if (cfs.getMaxRowSize() < DatabaseDescriptor.getColumnIndexSize())
+                if (cfs.metric.maxRowSize.getValue() < DatabaseDescriptor.getColumnIndexSize())
                 {
                     logger.trace("Expanding slice filter to entire row to cover additional expressions");
                     return new SliceQueryFilter(ColumnSlice.ALL_COLUMNS_ARRAY, ((SliceQueryFilter)filter).reversed, Integer.MAX_VALUE);
@@ -261,7 +261,7 @@
             {
                 if (data.getColumn(data.getComparator().cellFromByteBuffer(expr.column)) == null)
                 {
-                    logger.debug("adding extraFilter to cover additional expressions");
+                    logger.trace("adding extraFilter to cover additional expressions");
                     return true;
                 }
             }
@@ -330,7 +330,7 @@
                 {
                     if (def.type.isCollection() && def.type.isMultiCell())
                     {
-                        if (!collectionSatisfies(def, data, prefix, expression, collectionElement))
+                        if (!collectionSatisfies(def, data, prefix, expression))
                             return false;
                         continue;
                     }
@@ -382,7 +382,7 @@
             return true;
         }
 
-        private static boolean collectionSatisfies(ColumnDefinition def, ColumnFamily data, Composite prefix, IndexExpression expr, ByteBuffer collectionElement)
+        private static boolean collectionSatisfies(ColumnDefinition def, ColumnFamily data, Composite prefix, IndexExpression expr)
         {
             assert def.type.isCollection() && def.type.isMultiCell();
             CollectionType type = (CollectionType)def.type;
@@ -409,23 +409,21 @@
                 return false;
             }
 
-            switch (type.kind)
-            {
-                case LIST:
-                    assert collectionElement != null;
-                    return type.valueComparator().compare(data.getColumn(data.getComparator().create(prefix, def, collectionElement)).value(), expr.value) == 0;
-                case SET:
-                    return data.getColumn(data.getComparator().create(prefix, def, expr.value)) != null;
-                case MAP:
-                    if (expr.isContainsKey())
-                    {
-                        return data.getColumn(data.getComparator().create(prefix, def, expr.value)) != null;
-                    }
+            assert type.kind == CollectionType.Kind.MAP;
+            if (expr.isContainsKey())
+                return data.getColumn(data.getComparator().create(prefix, def, expr.value)) != null;
 
-                    assert collectionElement != null;
-                    return type.valueComparator().compare(data.getColumn(data.getComparator().create(prefix, def, collectionElement)).value(), expr.value) == 0;
+            Iterator<Cell> iter = data.iterator(new ColumnSlice[]{ data.getComparator().create(prefix, def).slice() });
+            ByteBuffer key = CompositeType.extractComponent(expr.value, 0);
+            ByteBuffer value = CompositeType.extractComponent(expr.value, 1);
+            while (iter.hasNext())
+            {
+                Cell next = iter.next();
+                if (type.nameComparator().compare(next.name().collectionElement(), key) == 0 &&
+                    type.valueComparator().compare(next.value(), value) == 0)
+                    return true;
             }
-            throw new AssertionError();
+            return false;
         }
 
         private ByteBuffer extractDataValue(ColumnDefinition def, ByteBuffer rowKey, ColumnFamily data, Composite prefix)
diff --git a/src/java/org/apache/cassandra/db/filter/IDiskAtomFilter.java b/src/java/org/apache/cassandra/db/filter/IDiskAtomFilter.java
index da5663b..a541d5e 100644
--- a/src/java/org/apache/cassandra/db/filter/IDiskAtomFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/IDiskAtomFilter.java
@@ -28,7 +28,7 @@
 import org.apache.cassandra.db.composites.Composite;
 import org.apache.cassandra.db.composites.CType;
 import org.apache.cassandra.io.IVersionedSerializer;
-import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.io.util.FileDataInput;
 
diff --git a/src/java/org/apache/cassandra/db/filter/NamesQueryFilter.java b/src/java/org/apache/cassandra/db/filter/NamesQueryFilter.java
index f31d1a09..c8f63bb 100644
--- a/src/java/org/apache/cassandra/db/filter/NamesQueryFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/NamesQueryFilter.java
@@ -24,22 +24,22 @@
 import java.util.SortedSet;
 import java.util.TreeSet;
 
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.commons.lang3.StringUtils;
 import com.google.common.collect.AbstractIterator;
 import com.google.common.collect.Iterators;
 
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
-import org.apache.cassandra.db.columniterator.SSTableNamesIterator;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.composites.CellNameType;
 import org.apache.cassandra.db.composites.Composite;
 import org.apache.cassandra.db.composites.CType;
 import org.apache.cassandra.io.ISerializer;
 import org.apache.cassandra.io.IVersionedSerializer;
-import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.utils.SearchIterator;
 
 public class NamesQueryFilter implements IDiskAtomFilter
 {
@@ -86,12 +86,12 @@
 
     public OnDiskAtomIterator getSSTableColumnIterator(SSTableReader sstable, DecoratedKey key)
     {
-        return new SSTableNamesIterator(sstable, key, columns);
+        return sstable.iterator(key, columns);
     }
 
     public OnDiskAtomIterator getSSTableColumnIterator(SSTableReader sstable, FileDataInput file, DecoratedKey key, RowIndexEntry indexEntry)
     {
-        return new SSTableNamesIterator(sstable, file, key, columns, indexEntry);
+        return sstable.iterator(file, key, columns, indexEntry);
     }
 
     public void collectReducedColumns(ColumnFamily container, Iterator<Cell> reducedColumns, DecoratedKey key, int gcBefore, long now)
@@ -191,21 +191,23 @@
     {
         private final ColumnFamily cf;
         private final DecoratedKey key;
-        private final Iterator<CellName> iter;
+        private final Iterator<CellName> names;
+        private final SearchIterator<CellName, Cell> cells;
 
-        public ByNameColumnIterator(Iterator<CellName> iter, DecoratedKey key, ColumnFamily cf)
+        public ByNameColumnIterator(Iterator<CellName> names, DecoratedKey key, ColumnFamily cf)
         {
-            this.iter = iter;
+            this.names = names;
             this.cf = cf;
             this.key = key;
+            this.cells = cf.searchIterator();
         }
 
         protected OnDiskAtom computeNext()
         {
-            while (iter.hasNext())
+            while (names.hasNext() && cells.hasNext())
             {
-                CellName current = iter.next();
-                Cell cell = cf.getColumn(current);
+                CellName current = names.next();
+                Cell cell = cells.next(current);
                 if (cell != null)
                     return cell;
             }
@@ -248,7 +250,7 @@
         public NamesQueryFilter deserialize(DataInput in, int version) throws IOException
         {
             int size = in.readInt();
-            SortedSet<CellName> columns = new TreeSet<CellName>(type);
+            SortedSet<CellName> columns = new TreeSet<>(type);
             ISerializer<CellName> serializer = type.cellSerializer();
             for (int i = 0; i < size; ++i)
                 columns.add(serializer.deserialize(in));
@@ -271,7 +273,7 @@
     public Iterator<RangeTombstone> getRangeTombstoneIterator(final ColumnFamily source)
     {
         if (!source.deletionInfo().hasRanges())
-            return Iterators.<RangeTombstone>emptyIterator();
+            return Iterators.emptyIterator();
 
         return new AbstractIterator<RangeTombstone>()
         {
diff --git a/src/java/org/apache/cassandra/db/filter/QueryFilter.java b/src/java/org/apache/cassandra/db/filter/QueryFilter.java
index db531a5..15ee33d 100644
--- a/src/java/org/apache/cassandra/db/filter/QueryFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/QueryFilter.java
@@ -33,7 +33,7 @@
 import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.composites.Composite;
-import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.utils.MergeIterator;
 
 public class QueryFilter
@@ -105,6 +105,7 @@
         Iterator<Cell> reduced = toCollate.size() == 1
                                ? toCollate.get(0)
                                : MergeIterator.get(toCollate, comparator, getReducer(comparator));
+
         filter.collectReducedColumns(returnCF, reduced, key, gcBefore, timestamp);
     }
 
diff --git a/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java b/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
index 957d1ba..822d838 100644
--- a/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
@@ -24,21 +24,22 @@
 
 import com.google.common.collect.AbstractIterator;
 import com.google.common.collect.Iterators;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.composites.*;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
-import org.apache.cassandra.db.columniterator.SSTableSliceIterator;
-import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.io.IVersionedSerializer;
-import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.service.ClientWarn;
 import org.apache.cassandra.tracing.Tracing;
-import org.apache.cassandra.utils.Pair;
 
 public class SliceQueryFilter implements IDiskAtomFilter
 {
@@ -242,12 +243,12 @@
 
     public OnDiskAtomIterator getSSTableColumnIterator(SSTableReader sstable, DecoratedKey key)
     {
-        return new SSTableSliceIterator(sstable, key, slices, reversed);
+        return sstable.iterator(key, slices, reversed);
     }
 
     public OnDiskAtomIterator getSSTableColumnIterator(SSTableReader sstable, FileDataInput file, DecoratedKey key, RowIndexEntry indexEntry)
     {
-        return new SSTableSliceIterator(sstable, file, key, slices, reversed, indexEntry);
+        return sstable.iterator(file, key, slices, reversed, indexEntry);
     }
 
     public Comparator<Cell> getColumnComparator(CellNameType comparator)
@@ -268,47 +269,33 @@
                 logger.trace("collecting {} of {}: {}", columnCounter.live(), count, cell.getString(container.getComparator()));
 
             // An expired tombstone will be immediately discarded in memory, and needn't be counted.
-            if (cell.getLocalDeletionTime() < gcBefore)
+            // Neither should be any cell shadowed by a range- or a partition tombstone.
+            if (cell.getLocalDeletionTime() < gcBefore || !columnCounter.count(cell, tester))
                 continue;
 
-            columnCounter.count(cell, tester);
-
             if (columnCounter.live() > count)
                 break;
 
             if (respectTombstoneThresholds() && columnCounter.tombstones() > DatabaseDescriptor.getTombstoneFailureThreshold())
             {
-                Tracing.trace("Scanned over {} tombstones; query aborted (see tombstone_failure_threshold)",
-                              DatabaseDescriptor.getTombstoneFailureThreshold());
-                String msg = String.format("Scanned over %d tombstones in %s.%s for key: %1.512s; query aborted (see tombstone_failure_threshold).",
-                                           DatabaseDescriptor.getTombstoneFailureThreshold(),
-                                           container.metadata().ksName,
-                                           container.metadata().cfName,
-                                           container.metadata().getKeyValidator().getString(key.getKey()));
-                logger.error(msg);
-                throw new TombstoneOverwhelmingException();
+                Tracing.trace("Scanned over {} tombstones; query aborted (see tombstone_failure_threshold); slices={}",
+                              DatabaseDescriptor.getTombstoneFailureThreshold(), getSlicesInfo(container));
+
+                throw new TombstoneOverwhelmingException(columnCounter.tombstones(),
+                                                         count,
+                                                         container.metadata().ksName,
+                                                         container.metadata().cfName,
+                                                         container.getComparator().getString(cell.name()),
+                                                         getSlicesInfo(container),
+                                                         container.metadata().getKeyValidator().getString(key.getKey()));
             }
 
-            container.maybeAppendColumn(cell, tester, gcBefore);
+            container.appendColumn(cell);
         }
 
-        Tracing.trace("Read {} live and {} tombstone cells", columnCounter.live(), columnCounter.tombstones());
-        if (logger.isWarnEnabled() && respectTombstoneThresholds() && columnCounter.tombstones() > DatabaseDescriptor.getTombstoneWarnThreshold())
+        boolean warnTombstones = logger.isWarnEnabled() && respectTombstoneThresholds() && columnCounter.tombstones() > DatabaseDescriptor.getTombstoneWarnThreshold();
+        if (warnTombstones)
         {
-            StringBuilder sb = new StringBuilder();
-            CellNameType type = container.metadata().comparator;
-
-            for (ColumnSlice sl : slices)
-            {
-                assert sl != null;
-
-                sb.append('[');
-                sb.append(type.getString(sl.start));
-                sb.append('-');
-                sb.append(type.getString(sl.finish));
-                sb.append(']');
-            }
-
             String msg = String.format("Read %d live and %d tombstone cells in %s.%s for key: %1.512s (see tombstone_warn_threshold). %d columns were requested, slices=%1.512s",
                                        columnCounter.live(),
                                        columnCounter.tombstones(),
@@ -316,9 +303,31 @@
                                        container.metadata().cfName,
                                        container.metadata().getKeyValidator().getString(key.getKey()),
                                        count,
-                                       sb);
+                                       getSlicesInfo(container));
+            ClientWarn.instance.warn(msg);
             logger.warn(msg);
         }
+        Tracing.trace("Read {} live and {} tombstone cells{}",
+                      columnCounter.live(),
+                      columnCounter.tombstones(),
+                      warnTombstones ? " (see tombstone_warn_threshold)" : "");
+    }
+
+    private String getSlicesInfo(ColumnFamily container)
+    {
+        StringBuilder sb = new StringBuilder();
+        CellNameType type = container.metadata().comparator;
+        for (ColumnSlice sl : slices)
+        {
+            assert sl != null;
+
+            sb.append('[');
+            sb.append(type.getString(sl.start));
+            sb.append('-');
+            sb.append(type.getString(sl.finish));
+            sb.append(']');
+        }
+        return sb.toString();
     }
 
     protected boolean respectTombstoneThresholds()
diff --git a/src/java/org/apache/cassandra/db/filter/TombstoneOverwhelmingException.java b/src/java/org/apache/cassandra/db/filter/TombstoneOverwhelmingException.java
index 6a6b0f6..da4e30f 100644
--- a/src/java/org/apache/cassandra/db/filter/TombstoneOverwhelmingException.java
+++ b/src/java/org/apache/cassandra/db/filter/TombstoneOverwhelmingException.java
@@ -20,4 +20,41 @@
 
 public class TombstoneOverwhelmingException extends RuntimeException
 {
+    private final int numTombstones;
+    private final int numRequested;
+    private final String ksName;
+    private final String cfName;
+    private final String lastCellName;
+    private final String slicesInfo;
+    private final String partitionKey;
+
+    public TombstoneOverwhelmingException(int numTombstones,
+                                          int numRequested,
+                                          String ksName,
+                                          String cfName,
+                                          String lastCellName,
+                                          String slicesInfo,
+                                          String partitionKey)
+    {
+        this.numTombstones = numTombstones;
+        this.numRequested = numRequested;
+        this.ksName = ksName;
+        this.cfName = cfName;
+        this.lastCellName = lastCellName;
+        this.slicesInfo = slicesInfo;
+        this.partitionKey = partitionKey;
+    }
+
+    public String getLocalizedMessage()
+    {
+        return getMessage();
+    }
+
+    public String getMessage()
+    {
+        return String.format(
+                "Scanned over %d tombstones in %s.%s; %d columns were requested; query aborted " +
+                "(see tombstone_failure_threshold); partitionKey=%s; lastCell=%s; slices=%s",
+                numTombstones, ksName, cfName, numRequested, partitionKey, lastCellName, slicesInfo);
+    }
 }
diff --git a/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java b/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java
index d7a1d8a..4410acc 100644
--- a/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java
+++ b/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java
@@ -27,7 +27,6 @@
 import org.apache.cassandra.db.composites.CellNameType;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.dht.LocalPartitioner;
-import org.apache.cassandra.dht.LocalToken;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.concurrent.OpOrder;
@@ -57,7 +56,8 @@
         indexCfs = ColumnFamilyStore.createColumnFamilyStore(baseCfs.keyspace,
                                                              indexedCfMetadata.cfName,
                                                              new LocalPartitioner(getIndexKeyComparator()),
-                                                             indexedCfMetadata);
+                                                             indexedCfMetadata,
+                                                             baseCfs.getTracker().loadsstables);
     }
 
     protected AbstractType<?> getIndexKeyComparator()
@@ -66,12 +66,6 @@
     }
 
     @Override
-    public DecoratedKey getIndexKeyFor(ByteBuffer value)
-    {
-        return new BufferDecoratedKey(new LocalToken(getIndexKeyComparator(), value), value);
-    }
-
-    @Override
     String indexTypeForGrouping()
     {
         return "_internal_";
@@ -107,8 +101,8 @@
         ColumnFamily cfi = ArrayBackedSortedColumns.factory.create(indexCfs.metadata, false, 1);
         cfi.addTombstone(makeIndexColumnName(rowKey, cell), localDeletionTime, cell.timestamp());
         indexCfs.apply(valueKey, cfi, SecondaryIndexManager.nullUpdater, opGroup, null);
-        if (logger.isDebugEnabled())
-            logger.debug("removed index entry for cleaned-up value {}:{}", valueKey, cfi);
+        if (logger.isTraceEnabled())
+            logger.trace("removed index entry for cleaned-up value {}:{}", valueKey, cfi);
     }
 
     public void insert(ByteBuffer rowKey, Cell cell, OpOrder.Group opGroup)
@@ -125,8 +119,8 @@
         {
             cfi.addColumn(new BufferCell(name, ByteBufferUtil.EMPTY_BYTE_BUFFER, cell.timestamp()));
         }
-        if (logger.isDebugEnabled())
-            logger.debug("applying index row {} in {}", indexCfs.metadata.getKeyValidator().getString(valueKey.getKey()), cfi);
+        if (logger.isTraceEnabled())
+            logger.trace("applying index row {} in {}", indexCfs.metadata.getKeyValidator().getString(valueKey.getKey()), cfi);
 
         indexCfs.apply(valueKey, cfi, SecondaryIndexManager.nullUpdater, opGroup, null);
     }
@@ -149,7 +143,7 @@
     {
         Future<?> wait;
         // we synchronise on the baseCfs to make sure we are ordered correctly with other flushes to the base CFS
-        synchronized (baseCfs.getDataTracker())
+        synchronized (baseCfs.getTracker())
         {
             wait = indexCfs.forceFlush();
         }
diff --git a/src/java/org/apache/cassandra/db/index/IndexNotAvailableException.java b/src/java/org/apache/cassandra/db/index/IndexNotAvailableException.java
new file mode 100644
index 0000000..750e899
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/index/IndexNotAvailableException.java
@@ -0,0 +1,49 @@
+/*
+ * 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.db.index;
+
+/**
+ * Thrown if a secondary index is not currently available.
+ */
+public final class IndexNotAvailableException extends RuntimeException
+{
+    /**
+     * Creates a new <code>IndexNotAvailableException</code> for the specified index.
+     * @param name the index name
+     */
+    public IndexNotAvailableException(String name)
+    {
+        super(String.format("The secondary index '%s' is not yet available",
+                            removeTableNameIfNeeded(name)));
+    }
+
+    /**
+     * Extract the name of the index if necessary.
+     *
+     * @param name the index name prefixed by the tablename or not
+     * @return the index name
+     */
+    private static String removeTableNameIfNeeded(String name)
+    {
+        int index = name.indexOf('.');
+        if (index < 0)
+            return name;
+
+        return name.substring(index + 1);
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/index/SecondaryIndex.java b/src/java/org/apache/cassandra/db/index/SecondaryIndex.java
index 2dc0a22..cf2deeb 100644
--- a/src/java/org/apache/cassandra/db/index/SecondaryIndex.java
+++ b/src/java/org/apache/cassandra/db/index/SecondaryIndex.java
@@ -27,14 +27,16 @@
 
 import com.google.common.base.Objects;
 import org.apache.commons.lang3.StringUtils;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.Operator;
-import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.Cell;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.composites.CellNameType;
@@ -44,10 +46,9 @@
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.BytesType;
 import org.apache.cassandra.db.marshal.LocalByPartionerType;
-import org.apache.cassandra.dht.LocalToken;
 import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.ReducingKeyIterator;
-import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.FBUtilities;
 
@@ -74,6 +75,11 @@
      */
     public static final String INDEX_VALUES_OPTION_NAME = "index_values";
 
+    /**
+     * The name of the option used to specify that the index is on the collection (map) entries.
+     */
+    public static final String INDEX_ENTRIES_OPTION_NAME = "index_keys_and_values";
+
     public static final AbstractType<?> keyComparator = StorageService.getPartitioner().preservesOrder()
                                                       ? BytesType.instance
                                                       : new LocalByPartionerType(StorageService.getPartitioner());
@@ -83,6 +89,12 @@
      */
     protected ColumnFamilyStore baseCfs;
 
+    // We need to keep track if the index is queryable or not to be sure that we can safely use it. If the index
+    // is still being build, using it will return incomplete results.
+    /**
+     * Specify if the index is queryable or not.
+     */
+    private volatile boolean queryable;
 
     /**
      * The column definitions which this index is responsible for
@@ -144,8 +156,18 @@
         return SystemKeyspace.isIndexBuilt(baseCfs.keyspace.getName(), getNameForSystemKeyspace(columnName));
     }
 
+    /**
+     * Checks if the index is ready.
+     * @return <code>true</code> if the index is ready, <code>false</code> otherwise
+     */
+    public boolean isQueryable()
+    {
+        return queryable;
+    }
+
     public void setIndexBuilt()
     {
+        queryable = true;
         for (ColumnDefinition columnDef : columnDefs)
             SystemKeyspace.setIndexBuilt(baseCfs.keyspace.getName(), getNameForSystemKeyspace(columnDef.name.bytes));
     }
@@ -223,7 +245,7 @@
      *
      * @return A future object which the caller can block on (optional)
      */
-    public Future<?> buildIndexAsync()
+    public final Future<?> buildIndexAsync()
     {
         // if we're just linking in the index to indexedColumns on an already-built index post-restart, we're done
         boolean allAreBuilt = true;
@@ -237,7 +259,17 @@
         }
 
         if (allAreBuilt)
+        {
+            queryable = true;
             return null;
+        }
+
+        // If the base table is empty we can directly mark the index as built.
+        if (baseCfs.isEmpty())
+        {
+            setIndexBuilt();
+            return null;
+        }
 
         // build it asynchronously; addIndex gets called by CFS open and schema update, neither of which
         // we want to block for a long period.  (actual build is serialized on CompactionManager.)
@@ -292,15 +324,13 @@
     }
 
     /**
-     * Returns the decoratedKey for a column value
+     * Returns the decoratedKey for a column value. Assumes an index CFS is present.
      * @param value column value
      * @return decorated key
      */
     public DecoratedKey getIndexKeyFor(ByteBuffer value)
     {
-        // FIXME: this imply one column definition per index
-        ByteBuffer name = columnDefs.iterator().next().name.bytes;
-        return new BufferDecoratedKey(new LocalToken(baseCfs.metadata.getColumnDefinition(name).type, value), value);
+        return getIndexCfs().partitioner.decorateKey(value);
     }
 
     /**
diff --git a/src/java/org/apache/cassandra/db/index/SecondaryIndexBuilder.java b/src/java/org/apache/cassandra/db/index/SecondaryIndexBuilder.java
index eb09e43..916c286 100644
--- a/src/java/org/apache/cassandra/db/index/SecondaryIndexBuilder.java
+++ b/src/java/org/apache/cassandra/db/index/SecondaryIndexBuilder.java
@@ -19,6 +19,7 @@
 
 import java.io.IOException;
 import java.util.Set;
+import java.util.UUID;
 
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.DecoratedKey;
@@ -27,6 +28,7 @@
 import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.db.compaction.CompactionInterruptedException;
 import org.apache.cassandra.io.sstable.ReducingKeyIterator;
+import org.apache.cassandra.utils.UUIDGen;
 
 /**
  * Manages building an entire index from column family data. Runs on to compaction manager.
@@ -36,12 +38,14 @@
     private final ColumnFamilyStore cfs;
     private final Set<String> idxNames;
     private final ReducingKeyIterator iter;
+    private final UUID compactionId;
 
     public SecondaryIndexBuilder(ColumnFamilyStore cfs, Set<String> idxNames, ReducingKeyIterator iter)
     {
         this.cfs = cfs;
         this.idxNames = idxNames;
         this.iter = iter;
+        compactionId = UUIDGen.getTimeUUID();
     }
 
     public CompactionInfo getCompactionInfo()
@@ -49,7 +53,8 @@
         return new CompactionInfo(cfs.metadata,
                                   OperationType.INDEX_BUILD,
                                   iter.getBytesRead(),
-                                  iter.getTotalBytes());
+                                  iter.getTotalBytes(),
+                                  compactionId);
     }
 
     public void build()
diff --git a/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java b/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
index 179126b..26327d4 100644
--- a/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
+++ b/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
@@ -34,11 +34,12 @@
 import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.concurrent.Future;
 
-import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.base.Joiner;
+
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.config.IndexType;
 import org.apache.cassandra.db.Cell;
@@ -50,11 +51,12 @@
 import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.composites.CellName;
+import org.apache.cassandra.db.composites.CellNameType;
 import org.apache.cassandra.db.filter.ExtendedFilter;
-import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.io.sstable.ReducingKeyIterator;
-import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.concurrent.OpOrder;
 
@@ -277,15 +279,7 @@
 
         assert cdef.getIndexType() != null;
 
-        SecondaryIndex index;
-        try
-        {
-            index = SecondaryIndex.createInstance(baseCfs, cdef);
-        }
-        catch (ConfigurationException e)
-        {
-            throw new RuntimeException(e);
-        }
+        SecondaryIndex index = SecondaryIndex.createInstance(baseCfs, cdef);
 
         // Keep a single instance of the index per-cf for row level indexes
         // since we want all columns to be under the index
@@ -323,11 +317,7 @@
         // Add to all indexes set:
         indexesByName.put(index.getIndexName(), index);
 
-        // if we're just linking in the index to indexedColumns on an
-        // already-built index post-restart, we're done
-        if (index.isIndexBuilt(cdef.name.bytes))
-            return null;
-
+        // We do not need to check if the index is already build as buildIndexAsync will do it for us
         return index.buildIndexAsync();
     }
 
@@ -357,7 +347,7 @@
     {
         // despatch flushes for all CFS backed indexes
         List<Future<?>> wait = new ArrayList<>();
-        synchronized (baseCfs.getDataTracker())
+        synchronized (baseCfs.getTracker())
         {
             for (SecondaryIndex index : allIndexes)
                 if (index.getIndexCfs() != null)
@@ -521,7 +511,9 @@
      */
     public Updater gcUpdaterFor(DecoratedKey key)
     {
-        return new GCUpdater(key);
+        return (indexesByColumn.isEmpty() && rowLevelIndexMap.isEmpty())
+               ? nullUpdater
+               : new GCUpdater(key);
     }
 
     /**
@@ -608,8 +600,15 @@
             }
         }
 
-        if (!haveSupportedIndexLookup)
+        CellNameType comparator = baseCfs.metadata.comparator;
+        // For thrift static CFs we can use filtering if no indexes can be used
+        if (!haveSupportedIndexLookup && (comparator.isDense() ||  comparator.isCompound()))
         {
+            if (expressionsByIndexType.isEmpty())
+                throw new InvalidRequestException(
+                    String.format("Predicates on non-primary-key columns (%s) are not yet supported for non secondary index queries",
+                                  Joiner.on(", ").join(getColumnNames(clause))));
+
             // build the error message
             int i = 0;
             StringBuilder sb = new StringBuilder("No secondary indexes on the restricted columns support the provided operators: ");
@@ -620,15 +619,7 @@
                     if (i++ > 0)
                         sb.append(", ");
                     sb.append("'");
-                    String columnName;
-                    try
-                    {
-                        columnName = ByteBufferUtil.string(expression.column);
-                    }
-                    catch (CharacterCodingException ex)
-                    {
-                        columnName = "<unprintable>";
-                    }
+                    String columnName = getColumnName(expression);
                     sb.append(columnName).append(" ").append(expression.operator).append(" <value>").append("'");
                 }
             }
@@ -637,6 +628,27 @@
         }
     }
 
+    private static String getColumnName(IndexExpression expression)
+    {
+        try
+        {
+            return ByteBufferUtil.string(expression.column);
+        }
+        catch (CharacterCodingException ex)
+        {
+            return "<unprintable>";
+        }
+    }
+
+    private static Set<String> getColumnNames(List<IndexExpression> expressions)
+    {
+        Set<String> columnNames = new HashSet<>();
+        for (IndexExpression expression : expressions)
+            columnNames.add(getColumnName(expression));
+
+        return columnNames;
+    }
+
     /**
      * Performs a search across a number of column indexes
      *
diff --git a/src/java/org/apache/cassandra/db/index/SecondaryIndexSearcher.java b/src/java/org/apache/cassandra/db/index/SecondaryIndexSearcher.java
index ab2cd75..5812e9d 100644
--- a/src/java/org/apache/cassandra/db/index/SecondaryIndexSearcher.java
+++ b/src/java/org/apache/cassandra/db/index/SecondaryIndexSearcher.java
@@ -18,14 +18,12 @@
 package org.apache.cassandra.db.index;
 
 import java.nio.ByteBuffer;
-import java.nio.charset.CharacterCodingException;
 import java.util.*;
 
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.ExtendedFilter;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.tracing.Tracing;
-import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
 public abstract class SecondaryIndexSearcher
diff --git a/src/java/org/apache/cassandra/db/index/composites/CompositesIndex.java b/src/java/org/apache/cassandra/db/index/composites/CompositesIndex.java
index ec965fd..0be78cc 100644
--- a/src/java/org/apache/cassandra/db/index/composites/CompositesIndex.java
+++ b/src/java/org/apache/cassandra/db/index/composites/CompositesIndex.java
@@ -67,9 +67,12 @@
                 case SET:
                     return new CompositesIndexOnCollectionKey();
                 case MAP:
-                    return cfDef.hasIndexOption(SecondaryIndex.INDEX_KEYS_OPTION_NAME)
-                         ? new CompositesIndexOnCollectionKey()
-                         : new CompositesIndexOnCollectionValue();
+                    if (cfDef.hasIndexOption(SecondaryIndex.INDEX_KEYS_OPTION_NAME))
+                        return new CompositesIndexOnCollectionKey();
+                    else if (cfDef.hasIndexOption(SecondaryIndex.INDEX_ENTRIES_OPTION_NAME))
+                        return new CompositesIndexOnCollectionKeyAndValue();
+                    else
+                        return new CompositesIndexOnCollectionValue();
             }
         }
 
@@ -99,9 +102,12 @@
                 case SET:
                     return CompositesIndexOnCollectionKey.buildIndexComparator(baseMetadata, cfDef);
                 case MAP:
-                    return cfDef.hasIndexOption(SecondaryIndex.INDEX_KEYS_OPTION_NAME)
-                         ? CompositesIndexOnCollectionKey.buildIndexComparator(baseMetadata, cfDef)
-                         : CompositesIndexOnCollectionValue.buildIndexComparator(baseMetadata, cfDef);
+                    if (cfDef.hasIndexOption(SecondaryIndex.INDEX_KEYS_OPTION_NAME))
+                        return CompositesIndexOnCollectionKey.buildIndexComparator(baseMetadata, cfDef);
+                    else if (cfDef.hasIndexOption(SecondaryIndex.INDEX_ENTRIES_OPTION_NAME))
+                        return CompositesIndexOnCollectionKeyAndValue.buildIndexComparator(baseMetadata, cfDef);
+                    else
+                        return CompositesIndexOnCollectionValue.buildIndexComparator(baseMetadata, cfDef);
             }
         }
 
@@ -136,8 +142,8 @@
         ColumnFamily cfi = ArrayBackedSortedColumns.factory.create(indexCfs.metadata);
         cfi.addTombstone(entry.indexEntry, localDeletionTime, entry.timestamp);
         indexCfs.apply(entry.indexValue, cfi, SecondaryIndexManager.nullUpdater, opGroup, null);
-        if (logger.isDebugEnabled())
-            logger.debug("removed index entry for cleaned-up value {}:{}", entry.indexValue, cfi);
+        if (logger.isTraceEnabled())
+            logger.trace("removed index entry for cleaned-up value {}:{}", entry.indexValue, cfi);
     }
 
     protected AbstractType<?> getExpressionComparator()
@@ -162,6 +168,7 @@
         {
             options.remove(SecondaryIndex.INDEX_VALUES_OPTION_NAME);
             options.remove(SecondaryIndex.INDEX_KEYS_OPTION_NAME);
+            options.remove(SecondaryIndex.INDEX_ENTRIES_OPTION_NAME);
         }
 
         if (!options.isEmpty())
diff --git a/src/java/org/apache/cassandra/db/index/composites/CompositesIndexIncludingCollectionKey.java b/src/java/org/apache/cassandra/db/index/composites/CompositesIndexIncludingCollectionKey.java
new file mode 100644
index 0000000..402ea05
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/index/composites/CompositesIndexIncludingCollectionKey.java
@@ -0,0 +1,89 @@
+/*
+ * 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.db.index.composites;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.composites.CBuilder;
+import org.apache.cassandra.db.composites.CellName;
+import org.apache.cassandra.db.composites.CellNameType;
+import org.apache.cassandra.db.composites.Composite;
+import org.apache.cassandra.db.composites.CompoundDenseCellNameType;
+import org.apache.cassandra.db.index.SecondaryIndex;
+import org.apache.cassandra.db.marshal.*;
+
+/**
+ * Common superclass for indexes that capture collection keys, including
+ * indexes on such keys themselves.
+ *
+ * A cell indexed by this index will have the general form:
+ *   ck_0 ... ck_n c_name [col_elt] : v
+ * where ck_i are the cluster keys, c_name the CQL3 column name, col_elt the
+ * collection element that we want to index (which may or may not be there depending
+ * on whether c_name is the collection we're indexing), and v the cell value.
+ *
+ * Such a cell is indexed if c_name is the indexed collection (in which case we are guaranteed to have
+ * col_elt). The index entry can be viewed in the following way:
+ *   - the row key is determined by subclasses of this type.
+ *   - the cell name will be 'rk ck_0 ... ck_n' where rk is the row key of the initial cell.
+ */
+public abstract class CompositesIndexIncludingCollectionKey extends CompositesIndex
+{
+    public static CellNameType buildIndexComparator(CFMetaData baseMetadata, ColumnDefinition columnDef)
+    {
+        int count = 1 + baseMetadata.clusteringColumns().size(); // row key + clustering prefix
+        List<AbstractType<?>> types = new ArrayList<AbstractType<?>>(count);
+        types.add(SecondaryIndex.keyComparator);
+        for (int i = 0; i < count - 1; i++)
+            types.add(baseMetadata.comparator.subtype(i));
+        return new CompoundDenseCellNameType(types);
+    }
+
+    protected Composite makeIndexColumnPrefix(ByteBuffer rowKey, Composite cellName)
+    {
+        int count = 1 + baseCfs.metadata.clusteringColumns().size();
+        CBuilder builder = getIndexComparator().builder();
+        builder.add(rowKey);
+        for (int i = 0; i < Math.min(cellName.size(), count - 1); i++)
+            builder.add(cellName.get(i));
+        return builder.build();
+    }
+
+    public IndexedEntry decodeEntry(DecoratedKey indexedValue, Cell indexEntry)
+    {
+        int count = 1 + baseCfs.metadata.clusteringColumns().size();
+        CBuilder builder = baseCfs.getComparator().builder();
+        for (int i = 0; i < count - 1; i++)
+            builder.add(indexEntry.name().get(i + 1));
+        return new IndexedEntry(indexedValue, indexEntry.name(), indexEntry.timestamp(), indexEntry.name().get(0), builder.build());
+    }
+
+    @Override
+    public boolean indexes(CellName name)
+    {
+        // We index if the CQL3 column name is the one of the collection we index
+        AbstractType<?> comp = baseCfs.metadata.getColumnDefinitionComparator(columnDef);
+        return name.size() > columnDef.position()
+            && comp.compare(name.get(columnDef.position()), columnDef.name.bytes) == 0;
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionKey.java b/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionKey.java
index 81982bb..1e40710 100644
--- a/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionKey.java
+++ b/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionKey.java
@@ -18,47 +18,20 @@
 package org.apache.cassandra.db.index.composites;
 
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.Operator;
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.composites.CBuilder;
 import org.apache.cassandra.db.composites.CellName;
-import org.apache.cassandra.db.composites.CellNameType;
-import org.apache.cassandra.db.composites.Composite;
-import org.apache.cassandra.db.composites.CompoundDenseCellNameType;
-import org.apache.cassandra.db.index.SecondaryIndex;
 import org.apache.cassandra.db.marshal.*;
 
 /**
  * Index on the collection element of the cell name of a collection.
  *
- * A cell indexed by this index will have the general form:
- *   ck_0 ... ck_n c_name [col_elt] : v
- * where ck_i are the cluster keys, c_name the CQL3 column name, col_elt the
- * collection element that we want to index (which may or may not be there depending
- * on whether c_name is the collection we're indexing) and v the cell value.
- *
- * Such a cell is indexed if c_name is the indexed collection (in which case we are guaranteed to have
- * col_elt). The index entry will be:
- *   - row key will be col_elt value (getIndexedValue()).
- *   - cell name will be 'rk ck_0 ... ck_n' where rk is the row key of the initial cell.
+ * The row keys for this index are given by the collection element for
+ * indexed columns.
  */
-public class CompositesIndexOnCollectionKey extends CompositesIndex
+public class CompositesIndexOnCollectionKey extends CompositesIndexIncludingCollectionKey
 {
-    public static CellNameType buildIndexComparator(CFMetaData baseMetadata, ColumnDefinition columnDef)
-    {
-        int count = 1 + baseMetadata.clusteringColumns().size(); // row key + clustering prefix
-        List<AbstractType<?>> types = new ArrayList<AbstractType<?>>(count);
-        types.add(SecondaryIndex.keyComparator);
-        for (int i = 0; i < count - 1; i++)
-            types.add(baseMetadata.comparator.subtype(i));
-        return new CompoundDenseCellNameType(types);
-    }
-
     @Override
     protected AbstractType<?> getIndexKeyComparator()
     {
@@ -70,25 +43,6 @@
         return cell.name().get(columnDef.position() + 1);
     }
 
-    protected Composite makeIndexColumnPrefix(ByteBuffer rowKey, Composite cellName)
-    {
-        int count = 1 + baseCfs.metadata.clusteringColumns().size();
-        CBuilder builder = getIndexComparator().builder();
-        builder.add(rowKey);
-        for (int i = 0; i < Math.min(cellName.size(), count - 1); i++)
-            builder.add(cellName.get(i));
-        return builder.build();
-    }
-
-    public IndexedEntry decodeEntry(DecoratedKey indexedValue, Cell indexEntry)
-    {
-        int count = 1 + baseCfs.metadata.clusteringColumns().size();
-        CBuilder builder = baseCfs.getComparator().builder();
-        for (int i = 0; i < count - 1; i++)
-            builder.add(indexEntry.name().get(i + 1));
-        return new IndexedEntry(indexedValue, indexEntry.name(), indexEntry.timestamp(), indexEntry.name().get(0), builder.build());
-    }
-
     @Override
     public boolean supportsOperator(Operator operator)
     {
@@ -96,15 +50,6 @@
                 operator == Operator.CONTAINS && columnDef.type instanceof SetType;
     }
 
-    @Override
-    public boolean indexes(CellName name)
-    {
-        // We index if the CQL3 column name is the one of the collection we index
-        AbstractType<?> comp = baseCfs.metadata.getColumnDefinitionComparator(columnDef);
-        return name.size() > columnDef.position()
-            && comp.compare(name.get(columnDef.position()), columnDef.name.bytes) == 0;
-    }
-
     public boolean isStale(IndexedEntry entry, ColumnFamily data, long now)
     {
         CellName name = data.getComparator().create(entry.indexedEntryPrefix, columnDef, entry.indexValue.getKey());
diff --git a/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionKeyAndValue.java b/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionKeyAndValue.java
new file mode 100644
index 0000000..0b7f579
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionKeyAndValue.java
@@ -0,0 +1,87 @@
+/*
+ * 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.db.index.composites;
+
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.composites.CellName;
+import org.apache.cassandra.db.marshal.*;
+
+/**
+ * Index on the element and value of cells participating in a collection.
+ *
+ * The row keys for this index are a composite of the collection element
+ * and value of indexed columns.
+ */
+public class CompositesIndexOnCollectionKeyAndValue extends CompositesIndexIncludingCollectionKey
+{
+    @Override
+    protected AbstractType<?> getIndexKeyComparator()
+    {
+        CollectionType colType = (CollectionType)columnDef.type;
+        return CompositeType.getInstance(colType.nameComparator(), colType.valueComparator());
+    }
+
+    @Override
+    protected ByteBuffer getIndexedValue(ByteBuffer rowKey, Cell cell)
+    {
+        final ByteBuffer key = cell.name().get(columnDef.position() + 1);
+        final ByteBuffer value = cell.value();
+        return CompositeType.build(key, value);
+    }
+
+    @Override
+    public boolean isStale(IndexedEntry entry, ColumnFamily data, long now)
+    {
+        Cell cell = extractTargetCell(entry, data);
+        if (cellIsDead(cell, now))
+            return true;
+        ByteBuffer indexCollectionValue = extractCollectionValue(entry);
+        ByteBuffer targetCollectionValue = cell.value();
+        AbstractType<?> valueComparator = ((CollectionType)columnDef.type).valueComparator();
+        return valueComparator.compare(indexCollectionValue, targetCollectionValue) != 0;
+    }
+
+    private Cell extractTargetCell(IndexedEntry entry, ColumnFamily data)
+    {
+        ByteBuffer collectionKey = extractCollectionKey(entry);
+        CellName name = data.getComparator().create(entry.indexedEntryPrefix, columnDef, collectionKey);
+        return data.getColumn(name);
+    }
+
+    private ByteBuffer extractCollectionKey(IndexedEntry entry)
+    {
+        return extractIndexKeyComponent(entry, 0);
+    }
+
+    private ByteBuffer extractIndexKeyComponent(IndexedEntry entry, int component)
+    {
+        return CompositeType.extractComponent(entry.indexValue.getKey(), component);
+    }
+
+    private ByteBuffer extractCollectionValue(IndexedEntry entry)
+    {
+        return extractIndexKeyComponent(entry, 1);
+    }
+
+    private boolean cellIsDead(Cell cell, long now)
+    {
+        return cell == null || !cell.isLive(now);
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/index/composites/CompositesSearcher.java b/src/java/org/apache/cassandra/db/index/composites/CompositesSearcher.java
index a2d08e7..a67aa2b 100644
--- a/src/java/org/apache/cassandra/db/index/composites/CompositesSearcher.java
+++ b/src/java/org/apache/cassandra/db/index/composites/CompositesSearcher.java
@@ -40,6 +40,8 @@
 import org.apache.cassandra.db.filter.IDiskAtomFilter;
 import org.apache.cassandra.db.filter.QueryFilter;
 import org.apache.cassandra.db.filter.SliceQueryFilter;
+import org.apache.cassandra.db.index.IndexNotAvailableException;
+import org.apache.cassandra.db.index.SecondaryIndex;
 import org.apache.cassandra.db.index.SecondaryIndexManager;
 import org.apache.cassandra.db.index.SecondaryIndexSearcher;
 import org.apache.cassandra.dht.AbstractBounds;
@@ -60,12 +62,15 @@
     {
         assert filter.getClause() != null && !filter.getClause().isEmpty();
         final IndexExpression primary = highestSelectivityPredicate(filter.getClause(), true);
-        final CompositesIndex index = (CompositesIndex)indexManager.getIndexForColumn(primary.column);
+        final SecondaryIndex index = indexManager.getIndexForColumn(primary.column);
+        if (!index.isQueryable())
+            throw new IndexNotAvailableException(index.getIndexName());
+
         // TODO: this should perhaps not open and maintain a writeOp for the full duration, but instead only *try* to delete stale entries, without blocking if there's no room
         // as it stands, we open a writeOp and keep it open for the duration to ensure that should this CF get flushed to make room we don't block the reclamation of any room being made
         try (OpOrder.Group writeOp = baseCfs.keyspace.writeOrder.start(); OpOrder.Group baseOp = baseCfs.readOrdering.start(); OpOrder.Group indexOp = index.getIndexCfs().readOrdering.start())
         {
-            return baseCfs.filter(getIndexedIterator(writeOp, filter, primary, index), filter);
+            return baseCfs.filter(getIndexedIterator(writeOp, filter, primary, (CompositesIndex) index), filter);
         }
     }
 
@@ -98,8 +103,8 @@
         assert index.getIndexCfs() != null;
         final DecoratedKey indexKey = index.getIndexKeyFor(primary.value);
 
-        if (logger.isDebugEnabled())
-            logger.debug("Most-selective indexed predicate is {}", index.expressionString(primary));
+        if (logger.isTraceEnabled())
+            logger.trace("Most-selective indexed predicate is {}", index.expressionString(primary));
 
         /*
          * XXX: If the range requested is a token range, we'll have to start at the beginning (and stop at the end) of
@@ -233,14 +238,14 @@
                         {
                             // Either we're not yet in the range cause the range is start excluding, or we're
                             // past it.
-                            if (!range.right.isMinimum(baseCfs.partitioner) && range.right.compareTo(dk) < 0)
+                            if (!range.right.isMinimum() && range.right.compareTo(dk) < 0)
                             {
                                 logger.trace("Reached end of assigned scan range");
                                 return endOfData();
                             }
                             else
                             {
-                                logger.debug("Skipping entry {} before assigned scan range", dk.getToken());
+                                logger.trace("Skipping entry {} before assigned scan range", dk.getToken());
                                 continue;
                             }
                         }
diff --git a/src/java/org/apache/cassandra/db/index/keys/KeysSearcher.java b/src/java/org/apache/cassandra/db/index/keys/KeysSearcher.java
index 4ba636d..2b07c41 100644
--- a/src/java/org/apache/cassandra/db/index/keys/KeysSearcher.java
+++ b/src/java/org/apache/cassandra/db/index/keys/KeysSearcher.java
@@ -26,7 +26,6 @@
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.composites.CellNameType;
@@ -55,6 +54,9 @@
         assert filter.getClause() != null && !filter.getClause().isEmpty();
         final IndexExpression primary = highestSelectivityPredicate(filter.getClause(), true);
         final SecondaryIndex index = indexManager.getIndexForColumn(primary.column);
+        if (!index.isQueryable())
+            throw new IndexNotAvailableException(index.getIndexName());
+
         // TODO: this should perhaps not open and maintain a writeOp for the full duration, but instead only *try* to delete stale entries, without blocking if there's no room
         // as it stands, we open a writeOp and keep it open for the duration to ensure that should this CF get flushed to make room we don't block the reclamation of any room  being made
         try (OpOrder.Group writeOp = baseCfs.keyspace.writeOrder.start(); OpOrder.Group baseOp = baseCfs.readOrdering.start(); OpOrder.Group indexOp = index.getIndexCfs().readOrdering.start())
@@ -73,8 +75,8 @@
         assert index.getIndexCfs() != null;
         final DecoratedKey indexKey = index.getIndexKeyFor(primary.value);
 
-        if (logger.isDebugEnabled())
-            logger.debug("Most-selective indexed predicate is {}",
+        if (logger.isTraceEnabled())
+            logger.trace("Most-selective indexed predicate is {}",
                          ((AbstractSimplePerColumnSecondaryIndex) index).expressionString(primary));
 
         /*
@@ -160,7 +162,7 @@
                         }
 
                         DecoratedKey dk = baseCfs.partitioner.decorateKey(lastSeenKey.toByteBuffer());
-                        if (!range.right.isMinimum(baseCfs.partitioner) && range.right.compareTo(dk) < 0)
+                        if (!range.right.isMinimum() && range.right.compareTo(dk) < 0)
                         {
                             logger.trace("Reached end of assigned scan range");
                             return endOfData();
diff --git a/src/java/org/apache/cassandra/db/lifecycle/Helpers.java b/src/java/org/apache/cassandra/db/lifecycle/Helpers.java
new file mode 100644
index 0000000..536e13c
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/lifecycle/Helpers.java
@@ -0,0 +1,232 @@
+/*
+ * 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.db.lifecycle;
+
+import java.util.*;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.*;
+
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+
+import static com.google.common.base.Predicates.*;
+import static com.google.common.collect.Iterables.any;
+import static com.google.common.collect.Iterables.concat;
+import static com.google.common.collect.Iterables.filter;
+import static com.google.common.collect.Iterables.getFirst;
+import static org.apache.cassandra.utils.Throwables.merge;
+
+class Helpers
+{
+    /**
+     * update the contents of a set with the provided sets, ensuring that the items to remove are
+     * really present, and that the items to add are not (unless we're also removing them)
+     * @return a new set with the contents of the provided one modified
+     */
+    static <T> Set<T> replace(Set<T> original, Set<T> remove, Iterable<T> add)
+    {
+        return ImmutableSet.copyOf(replace(identityMap(original), remove, add).keySet());
+    }
+
+    /**
+     * update the contents of an "identity map" with the provided sets, ensuring that the items to remove are
+     * really present, and that the items to add are not (unless we're also removing them)
+     * @return a new identity map with the contents of the provided one modified
+     */
+    static <T> Map<T, T> replace(Map<T, T> original, Set<T> remove, Iterable<T> add)
+    {
+        // ensure the ones being removed are the exact same ones present
+        for (T reader : remove)
+            assert original.get(reader) == reader;
+
+        // ensure we don't already contain any we're adding, that we aren't also removing
+        assert !any(add, and(not(in(remove)), in(original.keySet()))) : String.format("original:%s remove:%s add:%s", original.keySet(), remove, add);
+
+        Map<T, T> result =
+            identityMap(concat(add, filter(original.keySet(), not(in(remove)))));
+
+        assert result.size() == original.size() - remove.size() + Iterables.size(add) :
+        String.format("Expecting new size of %d, got %d while replacing %s by %s in %s",
+                      original.size() - remove.size() + Iterables.size(add), result.size(), remove, add, original.keySet());
+        return result;
+    }
+
+    /**
+     * A convenience method for encapsulating this action over multiple SSTableReader with exception-safety
+     * @return accumulate if not null (with any thrown exception attached), or any thrown exception otherwise
+     */
+    static Throwable setReplaced(Iterable<SSTableReader> readers, Throwable accumulate)
+    {
+        for (SSTableReader reader : readers)
+        {
+            try
+            {
+                reader.setReplaced();
+            }
+            catch (Throwable t)
+            {
+                accumulate = merge(accumulate, t);
+            }
+        }
+        return accumulate;
+    }
+
+    /**
+     * A convenience method for encapsulating this action over multiple SSTableReader with exception-safety
+     * @return accumulate if not null (with any thrown exception attached), or any thrown exception otherwise
+     */
+    static void setupKeycache(Iterable<SSTableReader> readers)
+    {
+        for (SSTableReader reader : readers)
+            reader.setupKeyCache();
+    }
+
+    /**
+     * assert that none of these readers have been replaced
+     */
+    static void checkNotReplaced(Iterable<SSTableReader> readers)
+    {
+        for (SSTableReader reader : readers)
+            assert !reader.isReplaced();
+    }
+
+    /**
+     * A convenience method for encapsulating this action over multiple SSTableReader with exception-safety
+     * @return accumulate if not null (with any thrown exception attached), or any thrown exception otherwise
+     */
+    static Throwable markObsolete(Tracker tracker, Iterable<SSTableReader> readers, Throwable accumulate)
+    {
+        for (SSTableReader reader : readers)
+        {
+            try
+            {
+                boolean firstToCompact = reader.markObsolete(tracker);
+                assert firstToCompact : reader + " was already marked compacted";
+            }
+            catch (Throwable t)
+            {
+                accumulate = merge(accumulate, t);
+            }
+        }
+        return accumulate;
+    }
+
+    /**
+     * @return the identity function, as a Map, with domain of the provided values
+     */
+    static <T> Map<T, T> identityMap(Iterable<T> values)
+    {
+        ImmutableMap.Builder<T, T> builder = ImmutableMap.<T, T>builder();
+        for (T t : values)
+            builder.put(t, t);
+        return builder.build();
+    }
+
+    /**
+     * @return an Iterable of the union if the sets, with duplicates being represented by their first encountered instance
+     * (as defined by the order of set provision)
+     */
+    static <T> Iterable<T> concatUniq(Set<T>... sets)
+    {
+        List<Predicate<T>> notIn = new ArrayList<>(sets.length);
+        for (Set<T> set : sets)
+            notIn.add(not(in(set)));
+        List<Iterable<T>> results = new ArrayList<>(sets.length);
+        for (int i = 0 ; i < sets.length ; i++)
+            results.add(filter(sets[i], and(notIn.subList(0, i))));
+        return concat(results);
+    }
+
+    /**
+     * @return a Predicate yielding true for an item present in NONE of the provided sets
+     */
+    static <T> Predicate<T> notIn(Set<T>... sets)
+    {
+        return not(orIn(sets));
+    }
+
+    /**
+     * @return a Predicate yielding true for an item present in ANY of the provided sets
+     */
+    static <T> Predicate<T> orIn(Collection<T>... sets)
+    {
+        Predicate<T>[] orIn = new Predicate[sets.length];
+        for (int i = 0 ; i < orIn.length ; i++)
+            orIn[i] = in(sets[i]);
+        return or(orIn);
+    }
+
+    /**
+     * filter out (i.e. remove) matching elements
+     * @return filter, filtered to only those elements that *are not* present in *any* of the provided sets (are present in none)
+     */
+    static <T> Iterable<T> filterOut(Iterable<T> filter, Set<T>... inNone)
+    {
+        return filter(filter, notIn(inNone));
+    }
+
+    /**
+     * filter in (i.e. retain)
+     *
+     * @return filter, filtered to only those elements that *are* present in *any* of the provided sets
+     */
+    static <T> Iterable<T> filterIn(Iterable<T> filter, Set<T>... inAny)
+    {
+        return filter(filter, orIn(inAny));
+    }
+
+    static Set<SSTableReader> emptySet()
+    {
+        return Collections.emptySet();
+    }
+
+    static <T> T select(T t, Collection<T> col)
+    {
+        if (col instanceof Set && !col.contains(t))
+            return null;
+        return getFirst(filter(col, equalTo(t)), null);
+    }
+
+    static <T> T selectFirst(T t, Collection<T> ... sets)
+    {
+        for (Collection<T> set : sets)
+        {
+            T select = select(t, set);
+            if (select != null)
+                return select;
+        }
+        return null;
+    }
+
+    static <T> Predicate<T> idIn(Set<T> set)
+    {
+        return idIn(identityMap(set));
+    }
+
+    static <T> Predicate<T> idIn(final Map<T, T> identityMap)
+    {
+        return new Predicate<T>()
+        {
+            public boolean apply(T t)
+            {
+                return identityMap.get(t) == t;
+            }
+        };
+    }
+
+}
diff --git a/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java b/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java
new file mode 100644
index 0000000..a95c4a8
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java
@@ -0,0 +1,529 @@
+/*
+ * 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.db.lifecycle;
+
+import java.util.*;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Function;
+import com.google.common.base.Predicate;
+import com.google.common.collect.*;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader.UniqueIdentifier;
+import org.apache.cassandra.utils.concurrent.Transactional;
+
+import static com.google.common.base.Functions.compose;
+import static com.google.common.base.Predicates.*;
+import static com.google.common.collect.ImmutableSet.copyOf;
+import static com.google.common.collect.Iterables.*;
+import static java.util.Collections.singleton;
+import static org.apache.cassandra.db.lifecycle.Helpers.*;
+import static org.apache.cassandra.db.lifecycle.View.updateCompacting;
+import static org.apache.cassandra.db.lifecycle.View.updateLiveSet;
+import static org.apache.cassandra.utils.Throwables.maybeFail;
+import static org.apache.cassandra.utils.concurrent.Refs.release;
+import static org.apache.cassandra.utils.concurrent.Refs.selfRefs;
+
+public class LifecycleTransaction extends Transactional.AbstractTransactional
+{
+    private static final Logger logger = LoggerFactory.getLogger(LifecycleTransaction.class);
+
+    /**
+     * a class that represents accumulated modifications to the Tracker.
+     * has two instances, one containing modifications that are "staged" (i.e. invisible)
+     * and one containing those "logged" that have been made visible through a call to checkpoint()
+     */
+    private static class State
+    {
+        // readers that are either brand new, update a previous new reader, or update one of the original readers
+        final Set<SSTableReader> update = new HashSet<>();
+        // disjoint from update, represents a subset of originals that is no longer needed
+        final Set<SSTableReader> obsolete = new HashSet<>();
+
+        void log(State staged)
+        {
+            update.removeAll(staged.obsolete);
+            update.removeAll(staged.update);
+            update.addAll(staged.update);
+            obsolete.addAll(staged.obsolete);
+        }
+
+        boolean contains(SSTableReader reader)
+        {
+            return update.contains(reader) || obsolete.contains(reader);
+        }
+
+        boolean isEmpty()
+        {
+            return update.isEmpty() && obsolete.isEmpty();
+        }
+
+        void clear()
+        {
+            update.clear();
+            obsolete.clear();
+        }
+
+        @Override
+        public String toString()
+        {
+            return String.format("[obsolete: %s, update: %s]", obsolete, update);
+        }
+    }
+
+    public final Tracker tracker;
+    private final OperationType operationType;
+    // the original readers this transaction was opened over, and that it guards
+    // (no other transactions may operate over these readers concurrently)
+    private final Set<SSTableReader> originals = new HashSet<>();
+    // the set of readers we've marked as compacting (only updated on creation and in checkpoint())
+    private final Set<SSTableReader> marked = new HashSet<>();
+    // the identity set of readers we've ever encountered; used to ensure we don't accidentally revisit the
+    // same version of a reader. potentially a dangerous property if there are reference counting bugs
+    // as they won't be caught until the transaction's lifespan is over.
+    private final Set<UniqueIdentifier> identities = Collections.newSetFromMap(new IdentityHashMap<UniqueIdentifier, Boolean>());
+
+    // changes that have been made visible
+    private final State logged = new State();
+    // changes that are pending
+    private final State staged = new State();
+
+    /**
+     * construct a Transaction for use in an offline operation
+     */
+    public static LifecycleTransaction offline(OperationType operationType, SSTableReader reader)
+    {
+        return offline(operationType, singleton(reader));
+    }
+
+    /**
+     * construct a Transaction for use in an offline operation
+     */
+    public static LifecycleTransaction offline(OperationType operationType, Iterable<SSTableReader> readers)
+    {
+        // if offline, for simplicity we just use a dummy tracker
+        Tracker dummy = new Tracker(null, false);
+        dummy.addInitialSSTables(readers);
+        dummy.apply(updateCompacting(emptySet(), readers));
+        return new LifecycleTransaction(dummy, operationType, readers);
+    }
+
+    LifecycleTransaction(Tracker tracker, OperationType operationType, Iterable<SSTableReader> readers)
+    {
+        this.tracker = tracker;
+        this.operationType = operationType;
+        for (SSTableReader reader : readers)
+        {
+            originals.add(reader);
+            marked.add(reader);
+            identities.add(reader.instanceId);
+        }
+    }
+
+    public void doPrepare()
+    {
+        // note for future: in anticompaction two different operations use the same Transaction, and both prepareToCommit()
+        // separately: the second prepareToCommit is ignored as a "redundant" transition. since it is only a checkpoint
+        // (and these happen anyway) this is fine but if more logic gets inserted here than is performed in a checkpoint,
+        // it may break this use case, and care is needed
+        checkpoint();
+    }
+
+    /**
+     * point of no return: commit all changes, but leave all readers marked as compacting
+     */
+    public Throwable doCommit(Throwable accumulate)
+    {
+        assert staged.isEmpty() : "must be no actions introduced between prepareToCommit and a commit";
+
+        if (logger.isTraceEnabled())
+            logger.trace("Committing transaction over {} staged: {}, logged: {}", originals, staged, logged);
+
+        // this is now the point of no return; we cannot safely rollback, so we ignore exceptions until we're done
+        // we restore state by obsoleting our obsolete files, releasing our references to them, and updating our size
+        // and notification status for the obsolete and new files
+        accumulate = markObsolete(tracker, logged.obsolete, accumulate);
+        accumulate = tracker.updateSizeTracking(logged.obsolete, logged.update, accumulate);
+        accumulate = release(selfRefs(logged.obsolete), accumulate);
+        accumulate = tracker.notifySSTablesChanged(originals, logged.update, operationType, accumulate);
+        return accumulate;
+    }
+
+    /**
+     * undo all of the changes made by this transaction, resetting the state to its original form
+     */
+    public Throwable doAbort(Throwable accumulate)
+    {
+        if (logger.isTraceEnabled())
+            logger.trace("Aborting transaction over {} staged: {}, logged: {}", originals, staged, logged);
+
+        if (logged.isEmpty() && staged.isEmpty())
+            return accumulate;
+
+        // mark obsolete all readers that are not versions of those present in the original set
+        Iterable<SSTableReader> obsolete = filterOut(concatUniq(staged.update, logged.update), originals);
+        logger.trace("Obsoleting {}", obsolete);
+        // we don't pass the tracker in for the obsoletion, since these readers have never been notified externally
+        // nor had their size accounting affected
+        accumulate = markObsolete(null, obsolete, accumulate);
+
+        // replace all updated readers with a version restored to its original state
+        List<SSTableReader> restored = restoreUpdatedOriginals();
+        List<SSTableReader> invalid = Lists.newArrayList(Iterables.concat(logged.update, logged.obsolete));
+        accumulate = tracker.apply(updateLiveSet(logged.update, restored), accumulate);
+        accumulate = tracker.notifySSTablesChanged(invalid, restored, OperationType.COMPACTION, accumulate);
+        // setReplaced immediately preceding versions that have not been obsoleted
+        accumulate = setReplaced(logged.update, accumulate);
+        // we have replaced all of logged.update and never made visible staged.update,
+        // and the files we have logged as obsolete we clone fresh versions of, so they are no longer needed either
+        // any _staged_ obsoletes should either be in staged.update already, and dealt with there,
+        // or is still in its original form (so left as is); in either case no extra action is needed
+        accumulate = release(selfRefs(concat(staged.update, logged.update, logged.obsolete)), accumulate);
+        logged.clear();
+        staged.clear();
+        return accumulate;
+    }
+
+    @Override
+    protected Throwable doPostCleanup(Throwable accumulate)
+    {
+        return unmarkCompacting(marked, accumulate);
+    }
+
+    public boolean isOffline()
+    {
+        return tracker.isDummy();
+    }
+
+    public void permitRedundantTransitions()
+    {
+        super.permitRedundantTransitions();
+    }
+
+    /**
+     * call when a consistent batch of changes is ready to be made atomically visible
+     * these will be exposed in the Tracker atomically, or an exception will be thrown; in this case
+     * the transaction should be rolled back
+     */
+    public void checkpoint()
+    {
+        maybeFail(checkpoint(null));
+    }
+    private Throwable checkpoint(Throwable accumulate)
+    {
+        if (logger.isTraceEnabled())
+            logger.trace("Checkpointing staged {}", staged);
+
+        if (staged.isEmpty())
+            return accumulate;
+
+        Set<SSTableReader> toUpdate = toUpdate();
+        Set<SSTableReader> fresh = copyOf(fresh());
+
+        // check the current versions of the readers we're replacing haven't somehow been replaced by someone else
+        checkNotReplaced(filterIn(toUpdate, staged.update));
+
+        // ensure any new readers are in the compacting set, since we aren't done with them yet
+        // and don't want anyone else messing with them
+        // apply atomically along with updating the live set of readers
+        tracker.apply(compose(updateCompacting(emptySet(), fresh),
+                              updateLiveSet(toUpdate, staged.update)));
+
+        // log the staged changes and our newly marked readers
+        marked.addAll(fresh);
+        logged.log(staged);
+
+        // setup our tracker, and mark our prior versions replaced, also releasing our references to them
+        // we do not replace/release obsoleted readers, since we may need to restore them on rollback
+        accumulate = setReplaced(filterOut(toUpdate, staged.obsolete), accumulate);
+        accumulate = release(selfRefs(filterOut(toUpdate, staged.obsolete)), accumulate);
+
+        staged.clear();
+        return accumulate;
+    }
+
+    /**
+     * update a reader: if !original, this is a reader that is being introduced by this transaction;
+     * otherwise it must be in the originals() set, i.e. a reader guarded by this transaction
+     */
+    public void update(SSTableReader reader, boolean original)
+    {
+        assert !staged.update.contains(reader) : "each reader may only be updated once per checkpoint: " + reader;
+        assert !identities.contains(reader.instanceId) : "each reader instance may only be provided as an update once: " + reader;
+        // check it isn't obsolete, and that it matches the original flag
+        assert !(logged.obsolete.contains(reader) || staged.obsolete.contains(reader)) : "may not update a reader that has been obsoleted";
+        assert original == originals.contains(reader) : String.format("the 'original' indicator was incorrect (%s provided): %s", original, reader);
+        staged.update.add(reader);
+        identities.add(reader.instanceId);
+        if (!isOffline())
+            reader.setupKeyCache();
+    }
+
+    /**
+     * mark this reader as for obsoletion. this does not actually obsolete the reader until commit() is called,
+     * but on checkpoint() the reader will be removed from the live set
+     */
+    public void obsolete(SSTableReader reader)
+    {
+        logger.trace("Staging for obsolescence {}", reader);
+        // check this is: a reader guarded by the transaction, an instance we have already worked with
+        // and that we haven't already obsoleted it, nor do we have other changes staged for it
+        assert identities.contains(reader.instanceId) : "only reader instances that have previously been provided may be obsoleted: " + reader;
+        assert originals.contains(reader) : "only readers in the 'original' set may be obsoleted: " + reader + " vs " + originals;
+        assert !(logged.obsolete.contains(reader) || staged.obsolete.contains(reader)) : "may not obsolete a reader that has already been obsoleted: " + reader;
+        assert !staged.update.contains(reader) : "may not obsolete a reader that has a staged update (must checkpoint first): " + reader;
+        assert current(reader) == reader : "may only obsolete the latest version of the reader: " + reader;
+        staged.obsolete.add(reader);
+    }
+
+    /**
+     * obsolete every file in the original transaction
+     */
+    public void obsoleteOriginals()
+    {
+        logger.trace("Staging for obsolescence {}", originals);
+        // if we're obsoleting, we should have no staged updates for the original files
+        assert Iterables.isEmpty(filterIn(staged.update, originals)) : staged.update;
+
+        // stage obsoletes for any currently visible versions of any original readers
+        Iterables.addAll(staged.obsolete, filterIn(current(), originals));
+    }
+
+    /**
+     * return the readers we're replacing in checkpoint(), i.e. the currently visible version of those in staged
+     */
+    private Set<SSTableReader> toUpdate()
+    {
+        return copyOf(filterIn(current(), staged.obsolete, staged.update));
+    }
+
+    /**
+     * new readers that haven't appeared previously (either in the original set or the logged updates)
+     */
+    private Iterable<SSTableReader> fresh()
+    {
+        return filterOut(staged.update,
+                         originals, logged.update);
+    }
+
+    /**
+     * returns the currently visible readers managed by this transaction
+     */
+    public Iterable<SSTableReader> current()
+    {
+        // i.e., those that are updates that have been logged (made visible),
+        // and any original readers that have neither been obsoleted nor updated
+        return concat(logged.update, filterOut(originals, logged.update, logged.obsolete));
+    }
+
+    /**
+     * update the current replacement of any original reader back to its original start
+     */
+    private List<SSTableReader> restoreUpdatedOriginals()
+    {
+        Iterable<SSTableReader> torestore = filterIn(originals, logged.update, logged.obsolete);
+        return ImmutableList.copyOf(transform(torestore,
+                                              new Function<SSTableReader, SSTableReader>()
+                                              {
+                                                  public SSTableReader apply(SSTableReader reader)
+                                                  {
+                                                      return current(reader).cloneWithNewStart(reader.first, null);
+                                                  }
+                                              }));
+    }
+
+    /**
+     * the set of readers guarded by this transaction _in their original instance/state_
+     * call current(SSTableReader) on any reader in this set to get the latest instance
+     */
+    public Set<SSTableReader> originals()
+    {
+        return Collections.unmodifiableSet(originals);
+    }
+
+    /**
+     * indicates if the reader has been marked for obsoletion
+     */
+    public boolean isObsolete(SSTableReader reader)
+    {
+        return logged.obsolete.contains(reader) || staged.obsolete.contains(reader);
+    }
+
+    /**
+     * return the current version of the provided reader, whether or not it is visible or staged;
+     * i.e. returns the first version present by testing staged, logged and originals in order.
+     */
+    public SSTableReader current(SSTableReader reader)
+    {
+        Set<SSTableReader> container;
+        if (staged.contains(reader))
+            container = staged.update.contains(reader) ? staged.update : staged.obsolete;
+        else if (logged.contains(reader))
+            container = logged.update.contains(reader) ? logged.update : logged.obsolete;
+        else if (originals.contains(reader))
+            container = originals;
+        else throw new AssertionError();
+        return select(reader, container);
+    }
+
+    /**
+     * remove the reader from the set we're modifying
+     */
+    public void cancel(SSTableReader cancel)
+    {
+        logger.trace("Cancelling {} from transaction", cancel);
+        assert originals.contains(cancel) : "may only cancel a reader in the 'original' set: " + cancel + " vs " + originals;
+        assert !(staged.contains(cancel) || logged.contains(cancel)) : "may only cancel a reader that has not been updated or obsoleted in this transaction: " + cancel;
+        originals.remove(cancel);
+        marked.remove(cancel);
+        identities.remove(cancel.instanceId);
+        maybeFail(unmarkCompacting(singleton(cancel), null));
+    }
+
+    /**
+     * remove the readers from the set we're modifying
+     */
+    public void cancel(Iterable<SSTableReader> cancels)
+    {
+        for (SSTableReader cancel : cancels)
+            cancel(cancel);
+    }
+
+    /**
+     * remove the provided readers from this Transaction, and return a new Transaction to manage them
+     * only permitted to be called if the current Transaction has never been used
+     */
+    public LifecycleTransaction split(Collection<SSTableReader> readers)
+    {
+        logger.trace("Splitting {} into new transaction", readers);
+        checkUnused();
+        for (SSTableReader reader : readers)
+            assert identities.contains(reader.instanceId) : "may only split the same reader instance the transaction was opened with: " + reader;
+
+        for (SSTableReader reader : readers)
+        {
+            identities.remove(reader.instanceId);
+            originals.remove(reader);
+            marked.remove(reader);
+        }
+        return new LifecycleTransaction(tracker, operationType, readers);
+    }
+
+    /**
+     * check this transaction has never been used
+     */
+    private void checkUnused()
+    {
+        assert logged.isEmpty();
+        assert staged.isEmpty();
+        assert identities.size() == originals.size();
+        assert originals.size() == marked.size();
+    }
+
+    private Throwable unmarkCompacting(Set<SSTableReader> unmark, Throwable accumulate)
+    {
+        accumulate = tracker.apply(updateCompacting(unmark, emptySet()), accumulate);
+        // when the CFS is invalidated, it will call unreferenceSSTables().  However, unreferenceSSTables only deals
+        // with sstables that aren't currently being compacted.  If there are ongoing compactions that finish or are
+        // interrupted after the CFS is invalidated, those sstables need to be unreferenced as well, so we do that here.
+        accumulate = tracker.dropSSTablesIfInvalid(accumulate);
+        return accumulate;
+    }
+
+    // convenience method for callers that know only one sstable is involved in the transaction
+    public SSTableReader onlyOne()
+    {
+        assert originals.size() == 1;
+        return getFirst(originals, null);
+    }
+
+    // a class representing the current state of the reader within this transaction, encoding the actions both logged
+    // and pending, and the reader instances that are visible now, and will be after the next checkpoint (with null
+    // indicating either obsolescence, or that the reader does not occur in the transaction; which is defined
+    // by the corresponding Action)
+    @VisibleForTesting
+    public static class ReaderState
+    {
+        public static enum Action
+        {
+            UPDATED, OBSOLETED, NONE;
+            public static Action get(boolean updated, boolean obsoleted)
+            {
+                assert !(updated && obsoleted);
+                return updated ? UPDATED : obsoleted ? OBSOLETED : NONE;
+            }
+        }
+
+        final Action staged;
+        final Action logged;
+        final SSTableReader nextVisible;
+        final SSTableReader currentlyVisible;
+        final boolean original;
+
+        public ReaderState(Action logged, Action staged, SSTableReader currentlyVisible, SSTableReader nextVisible, boolean original)
+        {
+            this.staged = staged;
+            this.logged = logged;
+            this.currentlyVisible = currentlyVisible;
+            this.nextVisible = nextVisible;
+            this.original = original;
+        }
+
+        public boolean equals(Object that)
+        {
+            return that instanceof ReaderState && equals((ReaderState) that);
+        }
+
+        public boolean equals(ReaderState that)
+        {
+            return this.staged == that.staged && this.logged == that.logged && this.original == that.original
+                && this.currentlyVisible == that.currentlyVisible && this.nextVisible == that.nextVisible;
+        }
+
+        public String toString()
+        {
+            return String.format("[logged=%s staged=%s original=%s]", logged, staged, original);
+        }
+
+        public static SSTableReader visible(SSTableReader reader, Predicate<SSTableReader> obsolete, Collection<SSTableReader> ... selectFrom)
+        {
+            return obsolete.apply(reader) ? null : selectFirst(reader, selectFrom);
+        }
+    }
+
+    @VisibleForTesting
+    public ReaderState state(SSTableReader reader)
+    {
+        SSTableReader currentlyVisible = ReaderState.visible(reader, in(logged.obsolete), logged.update, originals);
+        SSTableReader nextVisible = ReaderState.visible(reader, orIn(staged.obsolete, logged.obsolete), staged.update, logged.update, originals);
+        return new ReaderState(ReaderState.Action.get(logged.update.contains(reader), logged.obsolete.contains(reader)),
+                               ReaderState.Action.get(staged.update.contains(reader), staged.obsolete.contains(reader)),
+                               currentlyVisible, nextVisible, originals.contains(reader)
+        );
+    }
+
+    public String toString()
+    {
+        return originals.toString();
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/lifecycle/SSTableIntervalTree.java b/src/java/org/apache/cassandra/db/lifecycle/SSTableIntervalTree.java
new file mode 100644
index 0000000..841fa92
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/lifecycle/SSTableIntervalTree.java
@@ -0,0 +1,60 @@
+/*
+ *
+ * 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.db.lifecycle;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import com.google.common.collect.Iterables;
+
+import org.apache.cassandra.db.RowPosition;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.utils.Interval;
+import org.apache.cassandra.utils.IntervalTree;
+
+public class SSTableIntervalTree extends IntervalTree<RowPosition, SSTableReader, Interval<RowPosition, SSTableReader>>
+{
+    private static final SSTableIntervalTree EMPTY = new SSTableIntervalTree(null);
+
+    SSTableIntervalTree(Collection<Interval<RowPosition, SSTableReader>> intervals)
+    {
+        super(intervals);
+    }
+
+    public static SSTableIntervalTree empty()
+    {
+        return EMPTY;
+    }
+
+    public static SSTableIntervalTree build(Iterable<SSTableReader> sstables)
+    {
+        return new SSTableIntervalTree(buildIntervals(sstables));
+    }
+
+    public static List<Interval<RowPosition, SSTableReader>> buildIntervals(Iterable<SSTableReader> sstables)
+    {
+        List<Interval<RowPosition, SSTableReader>> intervals = new ArrayList<>(Iterables.size(sstables));
+        for (SSTableReader sstable : sstables)
+            intervals.add(Interval.<RowPosition, SSTableReader>create(sstable.first, sstable.last, sstable));
+        return intervals;
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
new file mode 100644
index 0000000..e77ef78
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
@@ -0,0 +1,504 @@
+/*
+ * 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.db.lifecycle;
+
+import java.io.File;
+import java.util.*;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.atomic.AtomicReference;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Function;
+import com.google.common.base.Predicate;
+import com.google.common.base.Predicates;
+import com.google.common.collect.*;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Directories;
+import org.apache.cassandra.db.Memtable;
+import org.apache.cassandra.db.commitlog.ReplayPosition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.metrics.StorageMetrics;
+import org.apache.cassandra.notifications.*;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.concurrent.OpOrder;
+
+import static com.google.common.base.Predicates.and;
+import static com.google.common.base.Predicates.in;
+import static com.google.common.base.Predicates.not;
+import static com.google.common.collect.ImmutableSet.copyOf;
+import static com.google.common.collect.Iterables.filter;
+import static java.util.Collections.singleton;
+import static java.util.Collections.singletonList;
+import static org.apache.cassandra.db.lifecycle.Helpers.*;
+import static org.apache.cassandra.db.lifecycle.View.permitCompacting;
+import static org.apache.cassandra.db.lifecycle.View.updateCompacting;
+import static org.apache.cassandra.db.lifecycle.View.updateLiveSet;
+import static org.apache.cassandra.utils.Throwables.maybeFail;
+import static org.apache.cassandra.utils.Throwables.merge;
+import static org.apache.cassandra.utils.concurrent.Refs.release;
+import static org.apache.cassandra.utils.concurrent.Refs.selfRefs;
+
+/**
+ * Tracker tracks live {@link View} of data store for a table.
+ */
+public class Tracker
+{
+    private static final Logger logger = LoggerFactory.getLogger(Tracker.class);
+
+    private final Collection<INotificationConsumer> subscribers = new CopyOnWriteArrayList<>();
+
+    public final ColumnFamilyStore cfstore;
+    final AtomicReference<View> view;
+    public final boolean loadsstables;
+
+    /**
+     * @param memtable Initial Memtable. Can be null.
+     * @param loadsstables true to indicate to load SSTables (TODO: remove as this is only accessed from 2i)
+     */
+    public Tracker(Memtable memtable, boolean loadsstables)
+    {
+        this.cfstore = memtable != null ? memtable.cfs : null;
+        this.view = new AtomicReference<>();
+        this.loadsstables = loadsstables;
+        this.reset(memtable);
+    }
+
+    public LifecycleTransaction tryModify(SSTableReader sstable, OperationType operationType)
+    {
+        return tryModify(singleton(sstable), operationType);
+    }
+
+    /**
+     * @return a Transaction over the provided sstables if we are able to mark the given @param sstables as compacted, before anyone else
+     */
+    public LifecycleTransaction tryModify(Iterable<SSTableReader> sstables, OperationType operationType)
+    {
+        if (Iterables.isEmpty(sstables))
+            return new LifecycleTransaction(this, operationType, sstables);
+        if (null == apply(permitCompacting(sstables), updateCompacting(emptySet(), sstables)))
+            return null;
+        return new LifecycleTransaction(this, operationType, sstables);
+    }
+
+
+    // METHODS FOR ATOMICALLY MODIFYING THE VIEW
+
+    Pair<View, View> apply(Function<View, View> function)
+    {
+        return apply(Predicates.<View>alwaysTrue(), function);
+    }
+
+    Throwable apply(Function<View, View> function, Throwable accumulate)
+    {
+        try
+        {
+            apply(function);
+        }
+        catch (Throwable t)
+        {
+            accumulate = merge(accumulate, t);
+        }
+        return accumulate;
+    }
+
+    /**
+     * atomically tests permit against the view and applies function to it, if permit yields true, returning the original;
+     * otherwise the method aborts, returning null
+     */
+    Pair<View, View> apply(Predicate<View> permit, Function<View, View> function)
+    {
+        while (true)
+        {
+            View cur = view.get();
+            if (!permit.apply(cur))
+                return null;
+            View updated = function.apply(cur);
+            if (view.compareAndSet(cur, updated))
+                return Pair.create(cur, updated);
+        }
+    }
+
+    Throwable updateSizeTracking(Iterable<SSTableReader> oldSSTables, Iterable<SSTableReader> newSSTables, Throwable accumulate)
+    {
+        if (isDummy())
+            return accumulate;
+
+        long add = 0;
+        for (SSTableReader sstable : newSSTables)
+        {
+            if (logger.isTraceEnabled())
+                logger.trace("adding {} to list of files tracked for {}.{}", sstable.descriptor, cfstore.keyspace.getName(), cfstore.name);
+            try
+            {
+                add += sstable.bytesOnDisk();
+            }
+            catch (Throwable t)
+            {
+                accumulate = merge(accumulate, t);
+            }
+        }
+        long subtract = 0;
+        for (SSTableReader sstable : oldSSTables)
+        {
+            if (logger.isTraceEnabled())
+                logger.trace("removing {} from list of files tracked for {}.{}", sstable.descriptor, cfstore.keyspace.getName(), cfstore.name);
+            try
+            {
+                subtract += sstable.bytesOnDisk();
+            }
+            catch (Throwable t)
+            {
+                accumulate = merge(accumulate, t);
+            }
+        }
+        StorageMetrics.load.inc(add - subtract);
+        cfstore.metric.liveDiskSpaceUsed.inc(add - subtract);
+        // we don't subtract from total until the sstable is deleted
+        cfstore.metric.totalDiskSpaceUsed.inc(add);
+        return accumulate;
+    }
+
+    // SETUP / CLEANUP
+
+    public void addInitialSSTables(Iterable<SSTableReader> sstables)
+    {
+        if (!isDummy())
+            setupKeycache(sstables);
+        apply(updateLiveSet(emptySet(), sstables));
+        maybeFail(updateSizeTracking(emptySet(), sstables, null));
+        // no notifications or backup necessary
+    }
+
+    public void addSSTables(Iterable<SSTableReader> sstables)
+    {
+        addInitialSSTables(sstables);
+        for (SSTableReader sstable : sstables)
+        {
+            maybeIncrementallyBackup(sstable);
+            notifyAdded(sstable);
+        }
+    }
+
+    /** (Re)initializes the tracker, purging all references. */
+    @VisibleForTesting
+    public void reset(Memtable memtable)
+    {
+        view.set(new View(memtable != null ? singletonList(memtable) : Collections.<Memtable>emptyList(),
+                          Collections.<Memtable>emptyList(),
+                          Collections.<SSTableReader, SSTableReader>emptyMap(),
+                          Collections.<SSTableReader>emptySet(),
+                          Collections.<SSTableReader>emptySet(),
+                          SSTableIntervalTree.empty()));
+    }
+
+    public Throwable dropSSTablesIfInvalid(Throwable accumulate)
+    {
+        if (!isDummy() && !cfstore.isValid())
+            accumulate = dropSSTables(accumulate);
+        return accumulate;
+    }
+
+    public void dropSSTables()
+    {
+        maybeFail(dropSSTables(null));
+    }
+
+    public Throwable dropSSTables(Throwable accumulate)
+    {
+        return dropSSTables(Predicates.<SSTableReader>alwaysTrue(), OperationType.UNKNOWN, accumulate);
+    }
+
+    /**
+     * removes all sstables that are not busy compacting.
+     */
+    public Throwable dropSSTables(final Predicate<SSTableReader> remove, OperationType operationType, Throwable accumulate)
+    {
+        Pair<View, View> result = apply(new Function<View, View>()
+        {
+            public View apply(View view)
+            {
+                Set<SSTableReader> toremove = copyOf(filter(view.sstables, and(remove, notIn(view.compacting))));
+                return updateLiveSet(toremove, emptySet()).apply(view);
+            }
+        });
+
+        Set<SSTableReader> removed = Sets.difference(result.left.sstables, result.right.sstables);
+        assert Iterables.all(removed, remove);
+
+        if (!removed.isEmpty())
+        {
+            // notifySSTablesChanged -> LeveledManifest.promote doesn't like a no-op "promotion"
+            accumulate = notifySSTablesChanged(removed, Collections.<SSTableReader>emptySet(), operationType, accumulate);
+            accumulate = updateSizeTracking(removed, emptySet(), accumulate);
+            accumulate = markObsolete(this, removed, accumulate);
+            accumulate = release(selfRefs(removed), accumulate);
+        }
+        return accumulate;
+    }
+
+    /**
+     * Removes every SSTable in the directory from the Tracker's view.
+     * @param directory the unreadable directory, possibly with SSTables in it, but not necessarily.
+     */
+    public void removeUnreadableSSTables(final File directory)
+    {
+        maybeFail(dropSSTables(new Predicate<SSTableReader>()
+        {
+            public boolean apply(SSTableReader reader)
+            {
+                return reader.descriptor.directory.equals(directory);
+            }
+        }, OperationType.UNKNOWN, null));
+    }
+
+
+
+    // FLUSHING
+
+    /**
+     * get the Memtable that the ordered writeOp should be directed to
+     */
+    public Memtable getMemtableFor(OpOrder.Group opGroup, ReplayPosition replayPosition)
+    {
+        // since any new memtables appended to the list after we fetch it will be for operations started
+        // after us, we can safely assume that we will always find the memtable that 'accepts' us;
+        // if the barrier for any memtable is set whilst we are reading the list, it must accept us.
+
+        // there may be multiple memtables in the list that would 'accept' us, however we only ever choose
+        // the oldest such memtable, as accepts() only prevents us falling behind (i.e. ensures we don't
+        // assign operations to a memtable that was retired/queued before we started)
+        for (Memtable memtable : view.get().liveMemtables)
+        {
+            if (memtable.accepts(opGroup, replayPosition))
+                return memtable;
+        }
+        throw new AssertionError(view.get().liveMemtables.toString());
+    }
+
+    /**
+     * Switch the current memtable. This atomically appends a new memtable to the end of the list of active memtables,
+     * returning the previously last memtable. It leaves the previous Memtable in the list of live memtables until
+     * discarding(memtable) is called. These two methods must be synchronized/paired, i.e. m = switchMemtable
+     * must be followed by discarding(m), they cannot be interleaved.
+     *
+     * @return the previously active memtable
+     */
+    public Memtable switchMemtable(boolean truncating, Memtable newMemtable)
+    {
+        Pair<View, View> result = apply(View.switchMemtable(newMemtable));
+        if (truncating)
+            notifyRenewed(newMemtable);
+
+        return result.left.getCurrentMemtable();
+    }
+
+    public void markFlushing(Memtable memtable)
+    {
+        apply(View.markFlushing(memtable));
+    }
+
+    public void replaceFlushed(Memtable memtable, SSTableReader sstable)
+    {
+        assert !isDummy();
+        if (sstable == null)
+        {
+            // sstable may be null if we flushed batchlog and nothing needed to be retained
+            // if it's null, we don't care what state the cfstore is in, we just replace it and continue
+            apply(View.replaceFlushed(memtable, null));
+            return;
+        }
+
+        sstable.setupKeyCache();
+        // back up before creating a new Snapshot (which makes the new one eligible for compaction)
+        maybeIncrementallyBackup(sstable);
+
+        apply(View.replaceFlushed(memtable, sstable));
+
+        Throwable fail;
+        fail = updateSizeTracking(emptySet(), singleton(sstable), null);
+
+        maybeFail(fail);
+    }
+
+    /**
+     * permit compaction of the provided sstable; this translates to notifying compaction
+     * strategies of its existence, and potentially submitting a background task
+     */
+    public void permitCompactionOfFlushed(SSTableReader sstable)
+    {
+        if (sstable == null)
+            return;
+
+        apply(View.permitCompactionOfFlushed(sstable));
+
+        if (isDummy())
+            return;
+
+        if (cfstore.isValid())
+        {
+            notifyAdded(sstable);
+            CompactionManager.instance.submitBackground(cfstore);
+        }
+        else
+        {
+            dropSSTables();
+        }
+    }
+
+
+
+    // MISCELLANEOUS public utility calls
+
+    public Set<SSTableReader> getSSTables()
+    {
+        return view.get().sstables;
+    }
+
+    public Iterable<SSTableReader> getPermittedToCompact()
+    {
+        View view = this.view.get();
+        return filter(view.sstables, not(in(view.premature)));
+    }
+
+    public Set<SSTableReader> getCompacting()
+    {
+        return view.get().compacting;
+    }
+
+    public Set<SSTableReader> getUncompacting()
+    {
+        return view.get().nonCompactingSStables();
+    }
+
+    public Iterable<SSTableReader> getUncompacting(Iterable<SSTableReader> candidates)
+    {
+        return view.get().getUncompacting(candidates);
+    }
+
+    public void maybeIncrementallyBackup(final SSTableReader sstable)
+    {
+        if (!DatabaseDescriptor.isIncrementalBackupsEnabled())
+            return;
+
+        File backupsDir = Directories.getBackupsDirectory(sstable.descriptor);
+        sstable.createLinks(FileUtils.getCanonicalPath(backupsDir));
+    }
+
+
+    // NOTIFICATION
+
+    Throwable notifySSTablesChanged(Collection<SSTableReader> removed, Collection<SSTableReader> added, OperationType compactionType, Throwable accumulate)
+    {
+        INotification notification = new SSTableListChangedNotification(added, removed, compactionType);
+        for (INotificationConsumer subscriber : subscribers)
+        {
+            try
+            {
+                subscriber.handleNotification(notification, this);
+            }
+            catch (Throwable t)
+            {
+                accumulate = merge(accumulate, t);
+            }
+        }
+        return accumulate;
+    }
+
+    Throwable notifyAdded(SSTableReader added, Throwable accumulate)
+    {
+        INotification notification = new SSTableAddedNotification(added);
+        for (INotificationConsumer subscriber : subscribers)
+        {
+            try
+            {
+                subscriber.handleNotification(notification, this);
+            }
+            catch (Throwable t)
+            {
+                accumulate = merge(accumulate, t);
+            }
+        }
+        return accumulate;
+    }
+
+    public void notifyAdded(SSTableReader added)
+    {
+        maybeFail(notifyAdded(added, null));
+    }
+
+    public void notifySSTableRepairedStatusChanged(Collection<SSTableReader> repairStatusesChanged)
+    {
+        INotification notification = new SSTableRepairStatusChanged(repairStatusesChanged);
+        for (INotificationConsumer subscriber : subscribers)
+            subscriber.handleNotification(notification, this);
+    }
+
+    public void notifyDeleting(SSTableReader deleting)
+    {
+        INotification notification = new SSTableDeletingNotification(deleting);
+        for (INotificationConsumer subscriber : subscribers)
+            subscriber.handleNotification(notification, this);
+    }
+
+    public void notifyRenewed(Memtable renewed)
+    {
+        INotification notification = new MemtableRenewedNotification(renewed);
+        for (INotificationConsumer subscriber : subscribers)
+            subscriber.handleNotification(notification, this);
+    }
+
+    public void notifyTruncated(long truncatedAt)
+    {
+        INotification notification = new TruncationNotification(truncatedAt);
+        for (INotificationConsumer subscriber : subscribers)
+            subscriber.handleNotification(notification, this);
+    }
+
+    public boolean isDummy()
+    {
+        return cfstore == null || !DatabaseDescriptor.isDaemonInitialized();
+    }
+
+    public void subscribe(INotificationConsumer consumer)
+    {
+        subscribers.add(consumer);
+    }
+
+    public void unsubscribe(INotificationConsumer consumer)
+    {
+        subscribers.remove(consumer);
+    }
+
+    private static Set<SSTableReader> emptySet()
+    {
+        return Collections.emptySet();
+    }
+
+    public View getView()
+    {
+        return view.get();
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/lifecycle/View.java b/src/java/org/apache/cassandra/db/lifecycle/View.java
new file mode 100644
index 0000000..e303801
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/lifecycle/View.java
@@ -0,0 +1,286 @@
+/*
+ * 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.db.lifecycle;
+
+import java.util.*;
+
+import javax.annotation.Nullable;
+
+import com.google.common.base.Function;
+import com.google.common.base.Functions;
+import com.google.common.base.Predicate;
+import com.google.common.collect.*;
+
+import org.apache.cassandra.db.Memtable;
+import org.apache.cassandra.db.RowPosition;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.utils.Interval;
+
+import static com.google.common.base.Predicates.equalTo;
+import static com.google.common.base.Predicates.not;
+import static com.google.common.collect.ImmutableList.copyOf;
+import static com.google.common.collect.ImmutableList.of;
+import static com.google.common.collect.Iterables.all;
+import static com.google.common.collect.Iterables.concat;
+import static com.google.common.collect.Iterables.filter;
+import static java.util.Collections.singleton;
+import static org.apache.cassandra.db.lifecycle.Helpers.emptySet;
+import static org.apache.cassandra.db.lifecycle.Helpers.replace;
+
+/**
+ * An immutable structure holding the current memtable, the memtables pending
+ * flush, the sstables for a column family, and the sstables that are active
+ * in compaction (a subset of the sstables).
+ *
+ * Modifications to instances are all performed via a Function produced by the static methods in this class.
+ * These are composed as necessary and provided to the Tracker.apply() methods, which atomically reject or
+ * accept and apply the changes to the View.
+ *
+ */
+public class View
+{
+    /**
+     * ordinarily a list of size 1, but when preparing to flush will contain both the memtable we will flush
+     * and the new replacement memtable, until all outstanding write operations on the old table complete.
+     * The last item in the list is always the "current" memtable.
+     */
+    public final List<Memtable> liveMemtables;
+    /**
+     * contains all memtables that are no longer referenced for writing and are queued for / in the process of being
+     * flushed. In chronologically ascending order.
+     */
+    public final List<Memtable> flushingMemtables;
+    public final Set<SSTableReader> compacting;
+    public final Set<SSTableReader> sstables;
+    public final Set<SSTableReader> premature;
+    // we use a Map here so that we can easily perform identity checks as well as equality checks.
+    // When marking compacting, we now  indicate if we expect the sstables to be present (by default we do),
+    // and we then check that not only are they all present in the live set, but that the exact instance present is
+    // the one we made our decision to compact against.
+    public final Map<SSTableReader, SSTableReader> sstablesMap;
+
+    public final SSTableIntervalTree intervalTree;
+
+    View(List<Memtable> liveMemtables, List<Memtable> flushingMemtables, Map<SSTableReader, SSTableReader> sstables, Set<SSTableReader> compacting, Set<SSTableReader> premature, SSTableIntervalTree intervalTree)
+    {
+        assert liveMemtables != null;
+        assert flushingMemtables != null;
+        assert sstables != null;
+        assert compacting != null;
+        assert intervalTree != null;
+
+        this.liveMemtables = liveMemtables;
+        this.flushingMemtables = flushingMemtables;
+
+        this.sstablesMap = sstables;
+        this.sstables = sstablesMap.keySet();
+        this.compacting = compacting;
+        this.premature = premature;
+        this.intervalTree = intervalTree;
+    }
+
+    public Memtable getCurrentMemtable()
+    {
+        return liveMemtables.get(liveMemtables.size() - 1);
+    }
+
+    /**
+     * @return the active memtable and all the memtables that are pending flush.
+     */
+    public Iterable<Memtable> getAllMemtables()
+    {
+        return concat(flushingMemtables, liveMemtables);
+    }
+
+    public Sets.SetView<SSTableReader> nonCompactingSStables()
+    {
+        return Sets.difference(sstables, compacting);
+    }
+
+    public Iterable<SSTableReader> getUncompacting(Iterable<SSTableReader> candidates)
+    {
+        return filter(candidates, new Predicate<SSTableReader>()
+        {
+            public boolean apply(SSTableReader sstable)
+            {
+                return !compacting.contains(sstable);
+            }
+        });
+    }
+
+    @Override
+    public String toString()
+    {
+        return String.format("View(pending_count=%d, sstables=%s, compacting=%s)", liveMemtables.size() + flushingMemtables.size() - 1, sstables, compacting);
+    }
+
+    /**
+      * Returns the sstables that have any partition between {@code left} and {@code right}, when both bounds are taken inclusively.
+      * The interval formed by {@code left} and {@code right} shouldn't wrap.
+      */
+    public List<SSTableReader> sstablesInBounds(RowPosition left, RowPosition right)
+    {
+        return sstablesInBounds(left, right, intervalTree);
+    }
+
+    public static List<SSTableReader> sstablesInBounds(RowPosition left, RowPosition right, SSTableIntervalTree intervalTree)
+    {
+        assert !AbstractBounds.strictlyWrapsAround(left, right);
+
+        if (intervalTree.isEmpty())
+            return Collections.emptyList();
+
+        RowPosition stopInTree = right.isMinimum() ? intervalTree.max() : right;
+        return intervalTree.search(Interval.<RowPosition, SSTableReader>create(left, stopInTree));
+    }
+
+    // METHODS TO CONSTRUCT FUNCTIONS FOR MODIFYING A VIEW:
+
+    // return a function to un/mark the provided readers compacting in a view
+    static Function<View, View> updateCompacting(final Set<SSTableReader> unmark, final Iterable<SSTableReader> mark)
+    {
+        if (unmark.isEmpty() && Iterables.isEmpty(mark))
+            return Functions.identity();
+        return new Function<View, View>()
+        {
+            public View apply(View view)
+            {
+                assert all(mark, Helpers.idIn(view.sstablesMap));
+                return new View(view.liveMemtables, view.flushingMemtables, view.sstablesMap,
+                                replace(view.compacting, unmark, mark),
+                                view.premature, view.intervalTree);
+            }
+        };
+    }
+
+    // construct a predicate to reject views that do not permit us to mark these readers compacting;
+    // i.e. one of them is either already compacting, has been compacted, or has been replaced
+    static Predicate<View> permitCompacting(final Iterable<SSTableReader> readers)
+    {
+        return new Predicate<View>()
+        {
+            public boolean apply(View view)
+            {
+                for (SSTableReader reader : readers)
+                    if (view.compacting.contains(reader) || view.sstablesMap.get(reader) != reader || reader.isMarkedCompacted() || view.premature.contains(reader))
+                        return false;
+                return true;
+            }
+        };
+    }
+
+    // construct a function to change the liveset in a Snapshot
+    static Function<View, View> updateLiveSet(final Set<SSTableReader> remove, final Iterable<SSTableReader> add)
+    {
+        if (remove.isEmpty() && Iterables.isEmpty(add))
+            return Functions.identity();
+        return new Function<View, View>()
+        {
+            public View apply(View view)
+            {
+                Map<SSTableReader, SSTableReader> sstableMap = replace(view.sstablesMap, remove, add);
+                return new View(view.liveMemtables, view.flushingMemtables, sstableMap, view.compacting, view.premature,
+                                SSTableIntervalTree.build(sstableMap.keySet()));
+            }
+        };
+    }
+
+    // called prior to initiating flush: add newMemtable to liveMemtables, making it the latest memtable
+    static Function<View, View> switchMemtable(final Memtable newMemtable)
+    {
+        return new Function<View, View>()
+        {
+            public View apply(View view)
+            {
+                List<Memtable> newLive = ImmutableList.<Memtable>builder().addAll(view.liveMemtables).add(newMemtable).build();
+                assert newLive.size() == view.liveMemtables.size() + 1;
+                return new View(newLive, view.flushingMemtables, view.sstablesMap, view.compacting, view.premature, view.intervalTree);
+            }
+        };
+    }
+
+    // called before flush: move toFlush from liveMemtables to flushingMemtables
+    static Function<View, View> markFlushing(final Memtable toFlush)
+    {
+        return new Function<View, View>()
+        {
+            public View apply(View view)
+            {
+                List<Memtable> live = view.liveMemtables, flushing = view.flushingMemtables;
+                List<Memtable> newLive = copyOf(filter(live, not(equalTo(toFlush))));
+                List<Memtable> newFlushing = copyOf(concat(filter(flushing, lessThan(toFlush)),
+                                                           of(toFlush),
+                                                           filter(flushing, not(lessThan(toFlush)))));
+                assert newLive.size() == live.size() - 1;
+                assert newFlushing.size() == flushing.size() + 1;
+                return new View(newLive, newFlushing, view.sstablesMap, view.compacting, view.premature, view.intervalTree);
+            }
+        };
+    }
+
+    // called after flush: removes memtable from flushingMemtables, and inserts flushed into the live sstable set
+    static Function<View, View> replaceFlushed(final Memtable memtable, final SSTableReader flushed)
+    {
+        return new Function<View, View>()
+        {
+            public View apply(View view)
+            {
+                List<Memtable> flushingMemtables = copyOf(filter(view.flushingMemtables, not(equalTo(memtable))));
+                assert flushingMemtables.size() == view.flushingMemtables.size() - 1;
+
+                if (flushed == null)
+                    return new View(view.liveMemtables, flushingMemtables, view.sstablesMap,
+                                    view.compacting, view.premature, view.intervalTree);
+
+                Map<SSTableReader, SSTableReader> sstableMap = replace(view.sstablesMap, emptySet(), singleton(flushed));
+                Set<SSTableReader> compacting = replace(view.compacting, emptySet(), singleton(flushed));
+                Set<SSTableReader> premature = replace(view.premature, emptySet(), singleton(flushed));
+                return new View(view.liveMemtables, flushingMemtables, sstableMap, compacting, premature,
+                                SSTableIntervalTree.build(sstableMap.keySet()));
+            }
+        };
+    }
+
+    static Function<View, View> permitCompactionOfFlushed(final SSTableReader reader)
+    {
+        return new Function<View, View>()
+        {
+
+            @Nullable
+            public View apply(View view)
+            {
+                Set<SSTableReader> premature = ImmutableSet.copyOf(filter(view.premature, not(equalTo(reader))));
+                Set<SSTableReader> compacting = ImmutableSet.copyOf(filter(view.compacting, not(equalTo(reader))));
+                return new View(view.liveMemtables, view.flushingMemtables, view.sstablesMap, compacting, premature, view.intervalTree);
+            }
+        };
+    }
+
+
+    private static <T extends Comparable<T>> Predicate<T> lessThan(final T lessThan)
+    {
+        return new Predicate<T>()
+        {
+            public boolean apply(T t)
+            {
+                return t.compareTo(lessThan) < 0;
+            }
+        };
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/marshal/AbstractCompositeType.java b/src/java/org/apache/cassandra/db/marshal/AbstractCompositeType.java
index 4e830abb..4baf6a3 100644
--- a/src/java/org/apache/cassandra/db/marshal/AbstractCompositeType.java
+++ b/src/java/org/apache/cassandra/db/marshal/AbstractCompositeType.java
@@ -22,6 +22,7 @@
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.serializers.BytesSerializer;
 import org.apache.cassandra.serializers.MarshalException;
@@ -266,6 +267,18 @@
     }
 
     @Override
+    public Term fromJSONObject(Object parsed)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public String toJSONString(ByteBuffer buffer, int protocolVersion)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
     public void validate(ByteBuffer bytes) throws MarshalException
     {
         ByteBuffer bb = bytes.duplicate();
diff --git a/src/java/org/apache/cassandra/db/marshal/AbstractType.java b/src/java/org/apache/cassandra/db/marshal/AbstractType.java
index bcb33dc..b90e127 100644
--- a/src/java/org/apache/cassandra/db/marshal/AbstractType.java
+++ b/src/java/org/apache/cassandra/db/marshal/AbstractType.java
@@ -18,6 +18,7 @@
 package org.apache.cassandra.db.marshal;
 
 import java.nio.ByteBuffer;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
@@ -25,9 +26,11 @@
 import java.util.Map;
 
 import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.serializers.MarshalException;
+
 import org.github.jamm.Unmetered;
 
 /**
@@ -63,6 +66,14 @@
         };
     }
 
+    public static List<String> asCQLTypeStringList(List<AbstractType<?>> abstractTypes)
+    {
+        List<String> r = new ArrayList<>(abstractTypes.size());
+        for (AbstractType<?> abstractType : abstractTypes)
+            r.add(abstractType.asCQL3Type().toString());
+        return r;
+    }
+
     public T compose(ByteBuffer bytes)
     {
         return getSerializer().deserialize(bytes);
@@ -85,10 +96,15 @@
     /** get a byte representation of the given string. */
     public abstract ByteBuffer fromString(String source) throws MarshalException;
 
-    /** for compatibility with TimeUUID in CQL2. See TimeUUIDType (that overrides it). */
-    public ByteBuffer fromStringCQL2(String source) throws MarshalException
+    /** Given a parsed JSON string, return a byte representation of the object.
+     * @param parsed the result of parsing a json string
+     **/
+    public abstract Term fromJSONObject(Object parsed) throws MarshalException;
+
+    /** Converts a value to a JSON string. */
+    public String toJSONString(ByteBuffer buffer, int protocolVersion)
     {
-        return fromString(source);
+        return '"' + getSerializer().deserialize(buffer).toString() + '"';
     }
 
     /* validate that the byte array is a valid sequence for the type we are supposed to be comparing */
@@ -134,6 +150,16 @@
         return false;
     }
 
+    public boolean isFrozenCollection()
+    {
+        return isCollection() && !isMultiCell();
+    }
+
+    public boolean isReversed()
+    {
+        return false;
+    }
+
     public static AbstractType<?> parseDefaultParameters(AbstractType<?> baseType, TypeParser parser) throws SyntaxException
     {
         Map<String, String> parameters = parser.getKeyValueParameters();
@@ -236,6 +262,14 @@
     }
 
     /**
+     * Returns {@code true} for types where empty should be handled like {@code null} like {@link Int32Type}.
+     */
+    public boolean isEmptyValueMeaningless()
+    {
+        return false;
+    }
+
+    /**
      * @param ignoreFreezing if true, the type string will not be wrapped with FrozenType(...), even if this type is frozen.
      */
     public String toString(boolean ignoreFreezing)
diff --git a/src/java/org/apache/cassandra/db/marshal/AsciiType.java b/src/java/org/apache/cassandra/db/marshal/AsciiType.java
index 891a8ed..2356c1c 100644
--- a/src/java/org/apache/cassandra/db/marshal/AsciiType.java
+++ b/src/java/org/apache/cassandra/db/marshal/AsciiType.java
@@ -23,7 +23,11 @@
 import java.nio.charset.CharsetEncoder;
 import java.nio.charset.CharacterCodingException;
 
+import org.apache.cassandra.cql3.Constants;
+import org.apache.cassandra.cql3.Json;
+
 import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.serializers.AsciiSerializer;
@@ -65,6 +69,33 @@
         }
     }
 
+    @Override
+    public Term fromJSONObject(Object parsed) throws MarshalException
+    {
+        try
+        {
+            return new Constants.Value(fromString((String) parsed));
+        }
+        catch (ClassCastException exc)
+        {
+            throw new MarshalException(String.format(
+                    "Expected an ascii string, but got a %s: %s", parsed.getClass().getSimpleName(), parsed));
+        }
+    }
+
+    @Override
+    public String toJSONString(ByteBuffer buffer, int protocolVersion)
+    {
+        try
+        {
+            return '"' + Json.quoteAsJsonString(ByteBufferUtil.string(buffer, Charset.forName("US-ASCII"))) + '"';
+        }
+        catch (CharacterCodingException exc)
+        {
+            throw new AssertionError("ascii value contained non-ascii characters: ", exc);
+        }
+    }
+
     public CQL3Type asCQL3Type()
     {
         return CQL3Type.Native.ASCII;
diff --git a/src/java/org/apache/cassandra/db/marshal/BooleanType.java b/src/java/org/apache/cassandra/db/marshal/BooleanType.java
index 70d7559..bfe8c34 100644
--- a/src/java/org/apache/cassandra/db/marshal/BooleanType.java
+++ b/src/java/org/apache/cassandra/db/marshal/BooleanType.java
@@ -20,6 +20,8 @@
 import java.nio.ByteBuffer;
 
 import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.Constants;
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.serializers.BooleanSerializer;
 import org.apache.cassandra.serializers.MarshalException;
@@ -35,6 +37,11 @@
 
     BooleanType() {} // singleton
 
+    public boolean isEmptyValueMeaningless()
+    {
+        return true;
+    }
+
     public int compare(ByteBuffer o1, ByteBuffer o2)
     {
         if (!o1.hasRemaining() || !o2.hasRemaining())
@@ -57,7 +64,25 @@
         if (source.equalsIgnoreCase(Boolean.TRUE.toString()))
             return decompose(true);
 
-        throw new MarshalException(String.format("unable to make boolean from '%s'", source));
+        throw new MarshalException(String.format("Unable to make boolean from '%s'", source));
+    }
+
+    @Override
+    public Term fromJSONObject(Object parsed) throws MarshalException
+    {
+        if (parsed instanceof String)
+            return new Constants.Value(fromString((String) parsed));
+        else if (!(parsed instanceof Boolean))
+            throw new MarshalException(String.format(
+                    "Expected a boolean value, but got a %s: %s", parsed.getClass().getSimpleName(), parsed));
+
+        return new Constants.Value(getSerializer().serialize((Boolean) parsed));
+    }
+
+    @Override
+    public String toJSONString(ByteBuffer buffer, int protocolVersion)
+    {
+        return getSerializer().deserialize(buffer).toString();
     }
 
     public CQL3Type asCQL3Type()
diff --git a/src/java/org/apache/cassandra/db/marshal/ByteType.java b/src/java/org/apache/cassandra/db/marshal/ByteType.java
new file mode 100644
index 0000000..e1cacc3
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/marshal/ByteType.java
@@ -0,0 +1,89 @@
+/*
+ * 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.db.marshal;
+
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.Constants;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.serializers.ByteSerializer;
+import org.apache.cassandra.serializers.MarshalException;
+import org.apache.cassandra.serializers.TypeSerializer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class ByteType extends AbstractType<Byte>
+{
+    public static final ByteType instance = new ByteType();
+
+    ByteType()
+    {
+    } // singleton
+
+    public int compare(ByteBuffer o1, ByteBuffer o2)
+    {
+        return o1.get(o1.position()) - o2.get(o2.position());
+    }
+
+    public ByteBuffer fromString(String source) throws MarshalException
+    {
+        // Return an empty ByteBuffer for an empty string.
+        if (source.isEmpty())
+            return ByteBufferUtil.EMPTY_BYTE_BUFFER;
+
+        byte b;
+
+        try
+        {
+            b = Byte.parseByte(source);
+        }
+        catch (Exception e)
+        {
+            throw new MarshalException(String.format("Unable to make byte from '%s'", source), e);
+        }
+
+        return decompose(b);
+    }
+
+    public Term fromJSONObject(Object parsed) throws MarshalException
+    {
+        if (parsed instanceof String || parsed instanceof Number)
+            return new Constants.Value(fromString(String.valueOf(parsed)));
+
+        throw new MarshalException(String.format(
+                "Expected a byte value, but got a %s: %s", parsed.getClass().getSimpleName(), parsed));
+    }
+
+    @Override
+    public String toJSONString(ByteBuffer buffer, int protocolVersion)
+    {
+        return getSerializer().deserialize(buffer).toString();
+    }
+
+    @Override
+    public CQL3Type asCQL3Type()
+    {
+        return CQL3Type.Native.TINYINT;
+    }
+
+    @Override
+    public TypeSerializer<Byte> getSerializer()
+    {
+        return ByteSerializer.instance;
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/marshal/BytesType.java b/src/java/org/apache/cassandra/db/marshal/BytesType.java
index a6a672c..eed3872 100644
--- a/src/java/org/apache/cassandra/db/marshal/BytesType.java
+++ b/src/java/org/apache/cassandra/db/marshal/BytesType.java
@@ -20,6 +20,8 @@
 import java.nio.ByteBuffer;
 
 import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.Constants;
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.serializers.BytesSerializer;
 import org.apache.cassandra.serializers.MarshalException;
@@ -50,6 +52,29 @@
     }
 
     @Override
+    public Term fromJSONObject(Object parsed) throws MarshalException
+    {
+        try
+        {
+            String parsedString = (String) parsed;
+            if (!parsedString.startsWith("0x"))
+                throw new MarshalException(String.format("String representation of blob is missing 0x prefix: %s", parsedString));
+
+            return new Constants.Value(BytesType.instance.fromString(parsedString.substring(2)));
+        }
+        catch (ClassCastException | MarshalException exc)
+        {
+            throw new MarshalException(String.format("Value '%s' is not a valid blob representation: %s", parsed, exc.getMessage()));
+        }
+    }
+
+    @Override
+    public String toJSONString(ByteBuffer buffer, int protocolVersion)
+    {
+        return "\"0x" + ByteBufferUtil.bytesToHex(buffer) + '"';
+    }
+
+    @Override
     public boolean isCompatibleWith(AbstractType<?> previous)
     {
         // Both asciiType and utf8Type really use bytes comparison and
diff --git a/src/java/org/apache/cassandra/db/marshal/CollectionType.java b/src/java/org/apache/cassandra/db/marshal/CollectionType.java
index 8662c1e..1660b2e 100644
--- a/src/java/org/apache/cassandra/db/marshal/CollectionType.java
+++ b/src/java/org/apache/cassandra/db/marshal/CollectionType.java
@@ -27,6 +27,11 @@
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.Lists;
+import org.apache.cassandra.cql3.Maps;
+import org.apache.cassandra.cql3.Sets;
+
 import org.apache.cassandra.serializers.CollectionSerializer;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -44,7 +49,29 @@
 
     public enum Kind
     {
-        MAP, SET, LIST
+        MAP
+        {
+            public ColumnSpecification makeCollectionReceiver(ColumnSpecification collection, boolean isKey)
+            {
+                return isKey ? Maps.keySpecOf(collection) : Maps.valueSpecOf(collection);
+            }
+        },
+        SET
+        {
+            public ColumnSpecification makeCollectionReceiver(ColumnSpecification collection, boolean isKey)
+            {
+                return Sets.valueSpecOf(collection);
+            }
+        },
+        LIST
+        {
+            public ColumnSpecification makeCollectionReceiver(ColumnSpecification collection, boolean isKey)
+            {
+                return Lists.valueSpecOf(collection);
+            }
+        };
+
+        public abstract ColumnSpecification makeCollectionReceiver(ColumnSpecification collection, boolean isKey);
     }
 
     public final Kind kind;
@@ -60,6 +87,11 @@
     @Override
     public abstract CollectionSerializer<T> getSerializer();
 
+    public ColumnSpecification makeCollectionReceiver(ColumnSpecification collection, boolean isKey)
+    {
+        return kind.makeCollectionReceiver(collection, isKey);
+    }
+
     public String getString(ByteBuffer bytes)
     {
         return BytesType.instance.getString(bytes);
diff --git a/src/java/org/apache/cassandra/db/marshal/ColumnToCollectionType.java b/src/java/org/apache/cassandra/db/marshal/ColumnToCollectionType.java
index 6fb32fb..1d2c88c 100644
--- a/src/java/org/apache/cassandra/db/marshal/ColumnToCollectionType.java
+++ b/src/java/org/apache/cassandra/db/marshal/ColumnToCollectionType.java
@@ -23,6 +23,7 @@
 
 import com.google.common.collect.ImmutableMap;
 
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.serializers.TypeSerializer;
@@ -92,6 +93,18 @@
     }
 
     @Override
+    public Term fromJSONObject(Object parsed) throws MarshalException
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public String toJSONString(ByteBuffer buffer, int protocolVersion)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
     public void validate(ByteBuffer bytes)
     {
         throw new UnsupportedOperationException("ColumnToCollectionType should only be used in composite types, never alone");
diff --git a/src/java/org/apache/cassandra/db/marshal/CompositeType.java b/src/java/org/apache/cassandra/db/marshal/CompositeType.java
index f8ac22d..9892118 100644
--- a/src/java/org/apache/cassandra/db/marshal/CompositeType.java
+++ b/src/java/org/apache/cassandra/db/marshal/CompositeType.java
@@ -19,19 +19,20 @@
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.util.Arrays;
 import java.util.ArrayList;
-import java.util.HashMap;
+import java.util.Arrays;
 import java.util.List;
-import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
 
 import com.google.common.collect.ImmutableList;
 
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.cql3.Operator;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.io.util.DataOutputBufferFixed;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
@@ -67,7 +68,7 @@
     public final List<AbstractType<?>> types;
 
     // interning instances
-    private static final Map<List<AbstractType<?>>, CompositeType> instances = new HashMap<List<AbstractType<?>>, CompositeType>();
+    private static final ConcurrentMap<List<AbstractType<?>>, CompositeType> instances = new ConcurrentHashMap<List<AbstractType<?>>, CompositeType>();
 
     public static CompositeType getInstance(TypeParser parser) throws ConfigurationException, SyntaxException
     {
@@ -97,7 +98,7 @@
         return true;
     }
 
-    public static synchronized CompositeType getInstance(List<AbstractType<?>> types)
+    public static CompositeType getInstance(List<AbstractType<?>> types)
     {
         assert types != null && !types.isEmpty();
 
@@ -105,7 +106,11 @@
         if (ct == null)
         {
             ct = new CompositeType(types);
-            instances.put(types, ct);
+            CompositeType previous = instances.putIfAbsent(types, ct);
+            if (previous != null)
+            {
+                ct = previous;
+            }
         }
         return ct;
     }
@@ -412,9 +417,8 @@
 
         public ByteBuffer build()
         {
-            try
+            try (DataOutputBuffer out = new DataOutputBufferFixed(serializedSize))
             {
-                DataOutputBuffer out = new DataOutputBuffer(serializedSize);
                 if (isStatic)
                     out.writeShort(STATIC_MARKER);
 
diff --git a/src/java/org/apache/cassandra/db/marshal/CounterColumnType.java b/src/java/org/apache/cassandra/db/marshal/CounterColumnType.java
index 2bcb4db..4b3ce82 100644
--- a/src/java/org/apache/cassandra/db/marshal/CounterColumnType.java
+++ b/src/java/org/apache/cassandra/db/marshal/CounterColumnType.java
@@ -20,6 +20,7 @@
 import java.nio.ByteBuffer;
 
 import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.db.context.CounterContext;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.serializers.CounterSerializer;
@@ -31,6 +32,11 @@
 
     CounterColumnType() {} // singleton
 
+    public boolean isEmptyValueMeaningless()
+    {
+        return true;
+    }
+
     public boolean isCounter()
     {
         return true;
@@ -68,6 +74,18 @@
         return ByteBufferUtil.hexToBytes(source);
     }
 
+    @Override
+    public Term fromJSONObject(Object parsed)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public String toJSONString(ByteBuffer buffer, int protocolVersion)
+    {
+        return CounterSerializer.instance.deserialize(buffer).toString();
+    }
+
     public CQL3Type asCQL3Type()
     {
         return CQL3Type.Native.COUNTER;
diff --git a/src/java/org/apache/cassandra/db/marshal/DateType.java b/src/java/org/apache/cassandra/db/marshal/DateType.java
index 882e4cf..5186ca8 100644
--- a/src/java/org/apache/cassandra/db/marshal/DateType.java
+++ b/src/java/org/apache/cassandra/db/marshal/DateType.java
@@ -18,9 +18,10 @@
 package org.apache.cassandra.db.marshal;
 
 import java.nio.ByteBuffer;
-import java.text.ParseException;
 import java.util.Date;
 
+import org.apache.cassandra.cql3.Constants;
+import org.apache.cassandra.cql3.Term;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -29,8 +30,12 @@
 import org.apache.cassandra.serializers.TimestampSerializer;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.commons.lang3.time.DateUtils;
 
+/**
+ * This is the old version of TimestampType, but has been replaced as it wasn't comparing pre-epoch timestamps
+ * correctly. This is kept for backward compatibility but shouldn't be used in new code.
+ */
+@Deprecated
 public class DateType extends AbstractType<Date>
 {
     private static final Logger logger = LoggerFactory.getLogger(DateType.class);
@@ -39,6 +44,11 @@
 
     DateType() {} // singleton
 
+    public boolean isEmptyValueMeaningless()
+    {
+        return true;
+    }
+
     public int compare(ByteBuffer o1, ByteBuffer o2)
     {
         if (!o1.hasRemaining() || !o2.hasRemaining())
@@ -57,6 +67,30 @@
     }
 
     @Override
+    public Term fromJSONObject(Object parsed) throws MarshalException
+    {
+        if (parsed instanceof Long)
+            return new Constants.Value(ByteBufferUtil.bytes((Long) parsed));
+
+        try
+        {
+            return new Constants.Value(TimestampType.instance.fromString((String) parsed));
+        }
+        catch (ClassCastException exc)
+        {
+            throw new MarshalException(String.format(
+                    "Expected a long or a datestring representation of a date value, but got a %s: %s",
+                    parsed.getClass().getSimpleName(), parsed));
+        }
+    }
+
+    @Override
+    public String toJSONString(ByteBuffer buffer, int protocolVersion)
+    {
+        return '"' + TimestampSerializer.getJsonDateFormatter().format(TimestampSerializer.instance.deserialize(buffer)) + '"';
+    }
+
+    @Override
     public boolean isCompatibleWith(AbstractType<?> previous)
     {
         if (super.isCompatibleWith(previous))
diff --git a/src/java/org/apache/cassandra/db/marshal/DecimalType.java b/src/java/org/apache/cassandra/db/marshal/DecimalType.java
index b7e481d..4052d70 100644
--- a/src/java/org/apache/cassandra/db/marshal/DecimalType.java
+++ b/src/java/org/apache/cassandra/db/marshal/DecimalType.java
@@ -21,6 +21,8 @@
 import java.nio.ByteBuffer;
 
 import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.Constants;
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.serializers.DecimalSerializer;
 import org.apache.cassandra.serializers.MarshalException;
@@ -32,6 +34,11 @@
 
     DecimalType() {} // singleton
 
+    public boolean isEmptyValueMeaningless()
+    {
+        return true;
+    }
+
     public int compare(ByteBuffer o1, ByteBuffer o2)
     {
         if (!o1.hasRemaining() || !o2.hasRemaining())
@@ -59,6 +66,25 @@
         return decompose(decimal);
     }
 
+    @Override
+    public Term fromJSONObject(Object parsed) throws MarshalException
+    {
+        try
+        {
+            return new Constants.Value(getSerializer().serialize(new BigDecimal(parsed.toString())));
+        }
+        catch (NumberFormatException exc)
+        {
+            throw new MarshalException(String.format("Value '%s' is not a valid representation of a decimal value", parsed));
+        }
+    }
+
+    @Override
+    public String toJSONString(ByteBuffer buffer, int protocolVersion)
+    {
+        return getSerializer().deserialize(buffer).toString();
+    }
+
     public CQL3Type asCQL3Type()
     {
         return CQL3Type.Native.DECIMAL;
diff --git a/src/java/org/apache/cassandra/db/marshal/DoubleType.java b/src/java/org/apache/cassandra/db/marshal/DoubleType.java
index af11a36..661b3c9 100644
--- a/src/java/org/apache/cassandra/db/marshal/DoubleType.java
+++ b/src/java/org/apache/cassandra/db/marshal/DoubleType.java
@@ -20,6 +20,8 @@
 import java.nio.ByteBuffer;
 
 import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.Constants;
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.serializers.DoubleSerializer;
 import org.apache.cassandra.serializers.MarshalException;
@@ -31,6 +33,11 @@
 
     DoubleType() {} // singleton
 
+    public boolean isEmptyValueMeaningless()
+    {
+        return true;
+    }
+
     public int compare(ByteBuffer o1, ByteBuffer o2)
     {
         if (!o1.hasRemaining() || !o2.hasRemaining())
@@ -52,12 +59,35 @@
       }
       catch (NumberFormatException e1)
       {
-          throw new MarshalException(String.format("unable to coerce '%s' to a double", source), e1);
+          throw new MarshalException(String.format("Unable to make double from '%s'", source), e1);
       }
 
       return decompose(d);
     }
 
+    @Override
+    public Term fromJSONObject(Object parsed) throws MarshalException
+    {
+        try
+        {
+            if (parsed instanceof String)
+                return new Constants.Value(fromString((String) parsed));
+            else
+                return new Constants.Value(getSerializer().serialize(((Number) parsed).doubleValue()));
+        }
+        catch (ClassCastException exc)
+        {
+            throw new MarshalException(String.format(
+                    "Expected a double value, but got a %s: %s", parsed.getClass().getSimpleName(), parsed));
+        }
+    }
+
+    @Override
+    public String toJSONString(ByteBuffer buffer, int protocolVersion)
+    {
+        return getSerializer().deserialize(buffer).toString();
+    }
+
     public CQL3Type asCQL3Type()
     {
         return CQL3Type.Native.DOUBLE;
diff --git a/src/java/org/apache/cassandra/db/marshal/DynamicCompositeType.java b/src/java/org/apache/cassandra/db/marshal/DynamicCompositeType.java
index d3b0b90..97d145d 100644
--- a/src/java/org/apache/cassandra/db/marshal/DynamicCompositeType.java
+++ b/src/java/org/apache/cassandra/db/marshal/DynamicCompositeType.java
@@ -22,6 +22,7 @@
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.cassandra.cql3.Term;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -105,14 +106,6 @@
         {
             throw new RuntimeException(e);
         }
-        catch (ConfigurationException e)
-        {
-            throw new RuntimeException(e);
-        }
-        catch (SyntaxException e)
-        {
-            throw new RuntimeException(e);
-        }
     }
 
     protected AbstractType<?> getComparator(int i, ByteBuffer bb)
@@ -180,14 +173,6 @@
         {
             throw new RuntimeException(e);
         }
-        catch (ConfigurationException e)
-        {
-            throw new RuntimeException(e);
-        }
-        catch (SyntaxException e)
-        {
-            throw new RuntimeException(e);
-        }
     }
 
     protected ParsedComparator parseComparator(int i, String part)
@@ -304,11 +289,7 @@
                 }
                 type = t;
             }
-            catch (SyntaxException e)
-            {
-                throw new IllegalArgumentException(e);
-            }
-            catch (ConfigurationException e)
+            catch (SyntaxException | ConfigurationException e)
             {
                 throw new IllegalArgumentException(e);
             }
@@ -393,6 +374,18 @@
         }
 
         @Override
+        public Term fromJSONObject(Object parsed)
+        {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public String toJSONString(ByteBuffer buffer, int protocolVersion)
+        {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
         public void validate(ByteBuffer bytes)
         {
             throw new UnsupportedOperationException();
diff --git a/src/java/org/apache/cassandra/db/marshal/EmptyType.java b/src/java/org/apache/cassandra/db/marshal/EmptyType.java
index 0ddb9ea..f82d767 100644
--- a/src/java/org/apache/cassandra/db/marshal/EmptyType.java
+++ b/src/java/org/apache/cassandra/db/marshal/EmptyType.java
@@ -19,6 +19,8 @@
 
 import java.nio.ByteBuffer;
 
+import org.apache.cassandra.cql3.Constants;
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.serializers.EmptySerializer;
 import org.apache.cassandra.serializers.MarshalException;
@@ -52,6 +54,17 @@
         return ByteBufferUtil.EMPTY_BYTE_BUFFER;
     }
 
+    @Override
+    public Term fromJSONObject(Object parsed) throws MarshalException
+    {
+        if (!(parsed instanceof String))
+            throw new MarshalException(String.format("Expected an empty string, but got: %s", parsed));
+        if (!((String) parsed).isEmpty())
+            throw new MarshalException(String.format("'%s' is not empty", parsed));
+
+        return new Constants.Value(ByteBufferUtil.EMPTY_BYTE_BUFFER);
+    }
+
     public TypeSerializer<Void> getSerializer()
     {
         return EmptySerializer.instance;
diff --git a/src/java/org/apache/cassandra/db/marshal/FloatType.java b/src/java/org/apache/cassandra/db/marshal/FloatType.java
index 9364928..af02cad 100644
--- a/src/java/org/apache/cassandra/db/marshal/FloatType.java
+++ b/src/java/org/apache/cassandra/db/marshal/FloatType.java
@@ -20,6 +20,8 @@
 import java.nio.ByteBuffer;
 
 import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.Constants;
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.serializers.FloatSerializer;
 import org.apache.cassandra.serializers.MarshalException;
@@ -32,6 +34,11 @@
 
     FloatType() {} // singleton
 
+    public boolean isEmptyValueMeaningless()
+    {
+        return true;
+    }
+
     public int compare(ByteBuffer o1, ByteBuffer o2)
     {
         if (!o1.hasRemaining() || !o2.hasRemaining())
@@ -53,10 +60,33 @@
       }
       catch (NumberFormatException e1)
       {
-          throw new MarshalException(String.format("unable to coerce '%s' to a float", source), e1);
+          throw new MarshalException(String.format("Unable to make float from '%s'", source), e1);
       }
     }
 
+    @Override
+    public Term fromJSONObject(Object parsed) throws MarshalException
+    {
+        try
+        {
+            if (parsed instanceof String)
+                return new Constants.Value(fromString((String) parsed));
+            else
+                return new Constants.Value(getSerializer().serialize(((Number) parsed).floatValue()));
+        }
+        catch (ClassCastException exc)
+        {
+            throw new MarshalException(String.format(
+                    "Expected a float value, but got a %s: %s", parsed.getClass().getSimpleName(), parsed));
+        }
+    }
+
+    @Override
+    public String toJSONString(ByteBuffer buffer, int protocolVersion)
+    {
+        return getSerializer().deserialize(buffer).toString();
+    }
+
     public CQL3Type asCQL3Type()
     {
         return CQL3Type.Native.FLOAT;
diff --git a/src/java/org/apache/cassandra/db/marshal/FrozenType.java b/src/java/org/apache/cassandra/db/marshal/FrozenType.java
index f440c90..7713028 100644
--- a/src/java/org/apache/cassandra/db/marshal/FrozenType.java
+++ b/src/java/org/apache/cassandra/db/marshal/FrozenType.java
@@ -20,6 +20,7 @@
 import java.nio.ByteBuffer;
 import java.util.List;
 
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.serializers.TypeSerializer;
@@ -55,6 +56,16 @@
         throw new UnsupportedOperationException();
     }
 
+    public Term fromJSONObject(Object parsed) throws MarshalException
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    public String toJSONString(ByteBuffer buffer, int protocolVersion)
+    {
+        throw new UnsupportedOperationException();
+    }
+
     public TypeSerializer<Void> getSerializer()
     {
         throw new UnsupportedOperationException();
diff --git a/src/java/org/apache/cassandra/db/marshal/InetAddressType.java b/src/java/org/apache/cassandra/db/marshal/InetAddressType.java
index 0473ee8..4901c74 100644
--- a/src/java/org/apache/cassandra/db/marshal/InetAddressType.java
+++ b/src/java/org/apache/cassandra/db/marshal/InetAddressType.java
@@ -21,6 +21,8 @@
 import java.nio.ByteBuffer;
 
 import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.Constants;
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.serializers.InetAddressSerializer;
 import org.apache.cassandra.serializers.MarshalException;
@@ -32,6 +34,11 @@
 
     InetAddressType() {} // singleton
 
+    public boolean isEmptyValueMeaningless()
+    {
+        return true;
+    }
+
     public int compare(ByteBuffer o1, ByteBuffer o2)
     {
         return ByteBufferUtil.compareUnsigned(o1, o2);
@@ -51,12 +58,32 @@
         }
         catch (Exception e)
         {
-            throw new MarshalException(String.format("unable to make inetaddress from '%s'", source), e);
+            throw new MarshalException(String.format("Unable to make inet address from '%s'", source), e);
         }
 
         return decompose(address);
     }
 
+    @Override
+    public Term fromJSONObject(Object parsed) throws MarshalException
+    {
+        try
+        {
+            return new Constants.Value(InetAddressType.instance.fromString((String) parsed));
+        }
+        catch (ClassCastException exc)
+        {
+            throw new MarshalException(String.format(
+                    "Expected a string representation of an inet value, but got a %s: %s", parsed.getClass().getSimpleName(), parsed));
+        }
+    }
+
+    @Override
+    public String toJSONString(ByteBuffer buffer, int protocolVersion)
+    {
+        return '"' + getSerializer().deserialize(buffer).getHostAddress() + '"';
+    }
+
     public CQL3Type asCQL3Type()
     {
         return CQL3Type.Native.INET;
diff --git a/src/java/org/apache/cassandra/db/marshal/Int32Type.java b/src/java/org/apache/cassandra/db/marshal/Int32Type.java
index 976c7a8..67d8142 100644
--- a/src/java/org/apache/cassandra/db/marshal/Int32Type.java
+++ b/src/java/org/apache/cassandra/db/marshal/Int32Type.java
@@ -20,6 +20,8 @@
 import java.nio.ByteBuffer;
 
 import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.Constants;
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.serializers.Int32Serializer;
 import org.apache.cassandra.serializers.MarshalException;
@@ -33,6 +35,11 @@
     {
     } // singleton
 
+    public boolean isEmptyValueMeaningless()
+    {
+        return true;
+    }
+
     public int compare(ByteBuffer o1, ByteBuffer o2)
     {
         if (!o1.hasRemaining() || !o2.hasRemaining())
@@ -59,12 +66,39 @@
         }
         catch (Exception e)
         {
-            throw new MarshalException(String.format("unable to make int from '%s'", source), e);
+            throw new MarshalException(String.format("Unable to make int from '%s'", source), e);
         }
 
         return decompose(int32Type);
     }
 
+    @Override
+    public Term fromJSONObject(Object parsed) throws MarshalException
+    {
+        try
+        {
+            if (parsed instanceof String)
+                return new Constants.Value(fromString((String) parsed));
+
+            Number parsedNumber = (Number) parsed;
+            if (!(parsedNumber instanceof Integer))
+                throw new MarshalException(String.format("Expected an int value, but got a %s: %s", parsed.getClass().getSimpleName(), parsed));
+
+            return new Constants.Value(getSerializer().serialize(parsedNumber.intValue()));
+        }
+        catch (ClassCastException exc)
+        {
+            throw new MarshalException(String.format(
+                    "Expected an int value, but got a %s: %s", parsed.getClass().getSimpleName(), parsed));
+        }
+    }
+
+    @Override
+    public String toJSONString(ByteBuffer buffer, int protocolVersion)
+    {
+        return getSerializer().deserialize(buffer).toString();
+    }
+
     public CQL3Type asCQL3Type()
     {
         return CQL3Type.Native.INT;
diff --git a/src/java/org/apache/cassandra/db/marshal/IntegerType.java b/src/java/org/apache/cassandra/db/marshal/IntegerType.java
index ec1c7ad..a3741d4 100644
--- a/src/java/org/apache/cassandra/db/marshal/IntegerType.java
+++ b/src/java/org/apache/cassandra/db/marshal/IntegerType.java
@@ -21,6 +21,8 @@
 import java.nio.ByteBuffer;
 
 import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.Constants;
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.serializers.IntegerSerializer;
 import org.apache.cassandra.serializers.MarshalException;
@@ -58,8 +60,18 @@
 
     IntegerType() {/* singleton */}
 
+    public boolean isEmptyValueMeaningless()
+    {
+        return true;
+    }
+
     public int compare(ByteBuffer lhs, ByteBuffer rhs)
     {
+        return IntegerType.compareIntegers(lhs, rhs);
+    }
+
+    public static int compareIntegers(ByteBuffer lhs, ByteBuffer rhs)
+    {
         int lhsLen = lhs.remaining();
         int rhsLen = rhs.remaining();
 
@@ -137,6 +149,26 @@
     }
 
     @Override
+    public Term fromJSONObject(Object parsed) throws MarshalException
+    {
+        try
+        {
+            return new Constants.Value(getSerializer().serialize(new BigInteger(parsed.toString())));
+        }
+        catch (NumberFormatException exc)
+        {
+            throw new MarshalException(String.format(
+                    "Value '%s' is not a valid representation of a varint value", parsed));
+        }
+    }
+
+    @Override
+    public String toJSONString(ByteBuffer buffer, int protocolVersion)
+    {
+        return getSerializer().deserialize(buffer).toString();
+    }
+
+    @Override
     public boolean isValueCompatibleWithInternal(AbstractType<?> otherType)
     {
         return this == otherType || Int32Type.instance.isValueCompatibleWith(otherType) || LongType.instance.isValueCompatibleWith(otherType);
diff --git a/src/java/org/apache/cassandra/db/marshal/LexicalUUIDType.java b/src/java/org/apache/cassandra/db/marshal/LexicalUUIDType.java
index 634194f..3e00d71 100644
--- a/src/java/org/apache/cassandra/db/marshal/LexicalUUIDType.java
+++ b/src/java/org/apache/cassandra/db/marshal/LexicalUUIDType.java
@@ -20,6 +20,8 @@
 import java.nio.ByteBuffer;
 import java.util.UUID;
 
+import org.apache.cassandra.cql3.Constants;
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.serializers.UUIDSerializer;
@@ -34,6 +36,11 @@
     {
     } // singleton
 
+    public boolean isEmptyValueMeaningless()
+    {
+        return true;
+    }
+
     public int compare(ByteBuffer o1, ByteBuffer o2)
     {
         if (!o1.hasRemaining() || !o2.hasRemaining())
@@ -58,6 +65,20 @@
         }
     }
 
+    @Override
+    public Term fromJSONObject(Object parsed) throws MarshalException
+    {
+        try
+        {
+            return new Constants.Value(fromString((String) parsed));
+        }
+        catch (ClassCastException exc)
+        {
+            throw new MarshalException(String.format(
+                    "Expected a string representation of a uuid, but got a %s: %s", parsed.getClass().getSimpleName(), parsed));
+        }
+    }
+
     public TypeSerializer<UUID> getSerializer()
     {
         return UUIDSerializer.instance;
diff --git a/src/java/org/apache/cassandra/db/marshal/ListType.java b/src/java/org/apache/cassandra/db/marshal/ListType.java
index c2f36c0..9334ad0 100644
--- a/src/java/org/apache/cassandra/db/marshal/ListType.java
+++ b/src/java/org/apache/cassandra/db/marshal/ListType.java
@@ -20,11 +20,17 @@
 import java.nio.ByteBuffer;
 import java.util.*;
 
+import org.apache.cassandra.cql3.Json;
+import org.apache.cassandra.cql3.Lists;
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.db.Cell;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.serializers.CollectionSerializer;
+import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.serializers.ListSerializer;
+
+import org.apache.cassandra.transport.Server;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -177,4 +183,45 @@
             bbs.add(c.value());
         return bbs;
     }
+
+    @Override
+    public Term fromJSONObject(Object parsed) throws MarshalException
+    {
+        if (parsed instanceof String)
+            parsed = Json.decodeJson((String) parsed);
+
+        if (!(parsed instanceof List))
+            throw new MarshalException(String.format(
+                    "Expected a list, but got a %s: %s", parsed.getClass().getSimpleName(), parsed));
+
+        List list = (List) parsed;
+        List<Term> terms = new ArrayList<>(list.size());
+        for (Object element : list)
+        {
+            if (element == null)
+                throw new MarshalException("Invalid null element in list");
+            terms.add(elements.fromJSONObject(element));
+        }
+
+        return new Lists.DelayedValue(terms);
+    }
+
+    public static String setOrListToJsonString(ByteBuffer buffer, AbstractType elementsType, int protocolVersion)
+    {
+        StringBuilder sb = new StringBuilder("[");
+        int size = CollectionSerializer.readCollectionSize(buffer, protocolVersion);
+        for (int i = 0; i < size; i++)
+        {
+            if (i > 0)
+                sb.append(", ");
+            sb.append(elementsType.toJSONString(CollectionSerializer.readValue(buffer, protocolVersion), protocolVersion));
+        }
+        return sb.append("]").toString();
+    }
+
+    @Override
+    public String toJSONString(ByteBuffer buffer, int protocolVersion)
+    {
+        return setOrListToJsonString(buffer, elements, protocolVersion);
+    }
 }
diff --git a/src/java/org/apache/cassandra/db/marshal/LocalByPartionerType.java b/src/java/org/apache/cassandra/db/marshal/LocalByPartionerType.java
index d1aaac0..427598d 100644
--- a/src/java/org/apache/cassandra/db/marshal/LocalByPartionerType.java
+++ b/src/java/org/apache/cassandra/db/marshal/LocalByPartionerType.java
@@ -19,6 +19,7 @@
 
 import java.nio.ByteBuffer;
 
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.serializers.MarshalException;
 
@@ -59,6 +60,18 @@
         throw new UnsupportedOperationException();
     }
 
+    @Override
+    public Term fromJSONObject(Object parsed)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public String toJSONString(ByteBuffer buffer, int protocolVersion)
+    {
+        throw new UnsupportedOperationException();
+    }
+
     public int compare(ByteBuffer o1, ByteBuffer o2)
     {
         // o1 and o2 can be empty so we need to use RowPosition, not DecoratedKey
diff --git a/src/java/org/apache/cassandra/db/marshal/LongType.java b/src/java/org/apache/cassandra/db/marshal/LongType.java
index feedaeb..d77d7d0 100644
--- a/src/java/org/apache/cassandra/db/marshal/LongType.java
+++ b/src/java/org/apache/cassandra/db/marshal/LongType.java
@@ -20,6 +20,8 @@
 import java.nio.ByteBuffer;
 
 import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.Constants;
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.serializers.LongSerializer;
 import org.apache.cassandra.serializers.MarshalException;
@@ -31,6 +33,11 @@
 
     LongType() {} // singleton
 
+    public boolean isEmptyValueMeaningless()
+    {
+        return true;
+    }
+
     public int compare(ByteBuffer o1, ByteBuffer o2)
     {
         return compareLongs(o1, o2);
@@ -62,13 +69,40 @@
         }
         catch (Exception e)
         {
-            throw new MarshalException(String.format("unable to make long from '%s'", source), e);
+            throw new MarshalException(String.format("Unable to make long from '%s'", source), e);
         }
 
         return decompose(longType);
     }
 
     @Override
+    public Term fromJSONObject(Object parsed) throws MarshalException
+    {
+        try
+        {
+            if (parsed instanceof String)
+                return new Constants.Value(fromString((String) parsed));
+
+            Number parsedNumber = (Number) parsed;
+            if (!(parsedNumber instanceof Integer || parsedNumber instanceof Long))
+                throw new MarshalException(String.format("Expected a bigint value, but got a %s: %s", parsed.getClass().getSimpleName(), parsed));
+
+            return new Constants.Value(getSerializer().serialize(parsedNumber.longValue()));
+        }
+        catch (ClassCastException exc)
+        {
+            throw new MarshalException(String.format(
+                    "Expected a bigint value, but got a %s: %s", parsed.getClass().getSimpleName(), parsed));
+        }
+    }
+
+    @Override
+    public String toJSONString(ByteBuffer buffer, int protocolVersion)
+    {
+        return getSerializer().deserialize(buffer).toString();
+    }
+
+    @Override
     public boolean isValueCompatibleWithInternal(AbstractType<?> otherType)
     {
         return this == otherType || otherType == DateType.instance || otherType == TimestampType.instance;
diff --git a/src/java/org/apache/cassandra/db/marshal/MapType.java b/src/java/org/apache/cassandra/db/marshal/MapType.java
index 0fb545d..434702a 100644
--- a/src/java/org/apache/cassandra/db/marshal/MapType.java
+++ b/src/java/org/apache/cassandra/db/marshal/MapType.java
@@ -20,10 +20,14 @@
 import java.nio.ByteBuffer;
 import java.util.*;
 
+import org.apache.cassandra.cql3.Json;
+import org.apache.cassandra.cql3.Maps;
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.db.Cell;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.serializers.CollectionSerializer;
+import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.serializers.MapSerializer;
 import org.apache.cassandra.transport.Server;
 import org.apache.cassandra.utils.Pair;
@@ -199,4 +203,52 @@
         }
         return bbs;
     }
+
+    @Override
+    public Term fromJSONObject(Object parsed) throws MarshalException
+    {
+        if (parsed instanceof String)
+            parsed = Json.decodeJson((String) parsed);
+
+        if (!(parsed instanceof Map))
+            throw new MarshalException(String.format(
+                    "Expected a map, but got a %s: %s", parsed.getClass().getSimpleName(), parsed));
+
+        Map<Object, Object> map = (Map<Object, Object>) parsed;
+        Map<Term, Term> terms = new HashMap<>(map.size());
+        for (Map.Entry<Object, Object> entry : map.entrySet())
+        {
+            if (entry.getKey() == null)
+                throw new MarshalException("Invalid null key in map");
+
+            if (entry.getValue() == null)
+                throw new MarshalException("Invalid null value in map");
+
+            terms.put(keys.fromJSONObject(entry.getKey()), values.fromJSONObject(entry.getValue()));
+        }
+        return new Maps.DelayedValue(keys, terms);
+    }
+
+    @Override
+    public String toJSONString(ByteBuffer buffer, int protocolVersion)
+    {
+        StringBuilder sb = new StringBuilder("{");
+        int size = CollectionSerializer.readCollectionSize(buffer, protocolVersion);
+        for (int i = 0; i < size; i++)
+        {
+            if (i > 0)
+                sb.append(", ");
+
+            // map keys must be JSON strings, so convert non-string keys to strings
+            String key = keys.toJSONString(CollectionSerializer.readValue(buffer, protocolVersion), protocolVersion);
+            if (key.startsWith("\""))
+                sb.append(key);
+            else
+                sb.append('"').append(Json.quoteAsJsonString(key)).append('"');
+
+            sb.append(": ");
+            sb.append(values.toJSONString(CollectionSerializer.readValue(buffer, protocolVersion), protocolVersion));
+        }
+        return sb.append("}").toString();
+    }
 }
diff --git a/src/java/org/apache/cassandra/db/marshal/ReversedType.java b/src/java/org/apache/cassandra/db/marshal/ReversedType.java
index 53798f8..19bee5f 100644
--- a/src/java/org/apache/cassandra/db/marshal/ReversedType.java
+++ b/src/java/org/apache/cassandra/db/marshal/ReversedType.java
@@ -23,8 +23,10 @@
 import java.util.List;
 
 import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
+import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.serializers.TypeSerializer;
 
 public class ReversedType<T> extends AbstractType<T>
@@ -58,6 +60,11 @@
         this.baseType = baseType;
     }
 
+    public boolean isEmptyValueMeaningless()
+    {
+        return baseType.isEmptyValueMeaningless();
+    }
+
     public int compare(ByteBuffer o1, ByteBuffer o2)
     {
         return baseType.compare(o2, o1);
@@ -74,6 +81,18 @@
     }
 
     @Override
+    public Term fromJSONObject(Object parsed) throws MarshalException
+    {
+        return baseType.fromJSONObject(parsed);
+    }
+
+    @Override
+    public String toJSONString(ByteBuffer buffer, int protocolVersion)
+    {
+        return baseType.toJSONString(buffer, protocolVersion);
+    }
+
+    @Override
     public boolean isCompatibleWith(AbstractType<?> otherType)
     {
         if (!(otherType instanceof ReversedType))
@@ -105,6 +124,12 @@
     }
 
     @Override
+    public boolean isReversed()
+    {
+        return true;
+    }
+
+    @Override
     public String toString()
     {
         return getClass().getName() + "(" + baseType + ")";
diff --git a/src/java/org/apache/cassandra/db/marshal/SetType.java b/src/java/org/apache/cassandra/db/marshal/SetType.java
index b635208..52e3f47 100644
--- a/src/java/org/apache/cassandra/db/marshal/SetType.java
+++ b/src/java/org/apache/cassandra/db/marshal/SetType.java
@@ -20,10 +20,15 @@
 import java.nio.ByteBuffer;
 import java.util.*;
 
+import org.apache.cassandra.cql3.Json;
+import org.apache.cassandra.cql3.Sets;
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.db.Cell;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
+import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.serializers.SetSerializer;
+import org.apache.cassandra.transport.Server;
 
 public class SetType<T> extends CollectionType<Set<T>>
 {
@@ -152,4 +157,32 @@
             bbs.add(c.name().collectionElement());
         return bbs;
     }
+
+    @Override
+    public Term fromJSONObject(Object parsed) throws MarshalException
+    {
+        if (parsed instanceof String)
+            parsed = Json.decodeJson((String) parsed);
+
+        if (!(parsed instanceof List))
+            throw new MarshalException(String.format(
+                    "Expected a list (representing a set), but got a %s: %s", parsed.getClass().getSimpleName(), parsed));
+
+        List list = (List) parsed;
+        Set<Term> terms = new HashSet<>(list.size());
+        for (Object element : list)
+        {
+            if (element == null)
+                throw new MarshalException("Invalid null element in set");
+            terms.add(elements.fromJSONObject(element));
+        }
+
+        return new Sets.DelayedValue(elements, terms);
+    }
+
+    @Override
+    public String toJSONString(ByteBuffer buffer, int protocolVersion)
+    {
+        return ListType.setOrListToJsonString(buffer, elements, protocolVersion);
+    }
 }
diff --git a/src/java/org/apache/cassandra/db/marshal/ShortType.java b/src/java/org/apache/cassandra/db/marshal/ShortType.java
new file mode 100644
index 0000000..2f9ec57
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/marshal/ShortType.java
@@ -0,0 +1,92 @@
+/*
+ * 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.db.marshal;
+
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.Constants;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.serializers.MarshalException;
+import org.apache.cassandra.serializers.ShortSerializer;
+import org.apache.cassandra.serializers.TypeSerializer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class ShortType extends AbstractType<Short>
+{
+    public static final ShortType instance = new ShortType();
+
+    ShortType()
+    {
+    } // singleton
+
+    public int compare(ByteBuffer o1, ByteBuffer o2)
+    {
+        int diff = o1.get(o1.position()) - o2.get(o2.position());
+        if (diff != 0)
+            return diff;
+
+        return ByteBufferUtil.compareUnsigned(o1, o2);
+    }
+
+    public ByteBuffer fromString(String source) throws MarshalException
+    {
+        // Return an empty ByteBuffer for an empty string.
+        if (source.isEmpty())
+            return ByteBufferUtil.EMPTY_BYTE_BUFFER;
+
+        short s;
+
+        try
+        {
+            s = Short.parseShort(source);
+        }
+        catch (Exception e)
+        {
+            throw new MarshalException(String.format("Unable to make short from '%s'", source), e);
+        }
+
+        return decompose(s);
+    }
+
+    public Term fromJSONObject(Object parsed) throws MarshalException
+    {
+        if (parsed instanceof String || parsed instanceof Number)
+            return new Constants.Value(fromString(String.valueOf(parsed)));
+
+        throw new MarshalException(String.format(
+                "Expected a short value, but got a %s: %s", parsed.getClass().getSimpleName(), parsed));
+    }
+
+    @Override
+    public String toJSONString(ByteBuffer buffer, int protocolVersion)
+    {
+        return getSerializer().deserialize(buffer).toString();
+    }
+
+    @Override
+    public CQL3Type asCQL3Type()
+    {
+        return CQL3Type.Native.SMALLINT;
+    }
+
+    public TypeSerializer<Short> getSerializer()
+    {
+        return ShortSerializer.instance;
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/marshal/SimpleDateType.java b/src/java/org/apache/cassandra/db/marshal/SimpleDateType.java
new file mode 100644
index 0000000..cae9707
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/marshal/SimpleDateType.java
@@ -0,0 +1,99 @@
+/*
+ * 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.db.marshal;
+
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.Constants;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.serializers.MarshalException;
+import org.apache.cassandra.serializers.SimpleDateSerializer;
+import org.apache.cassandra.serializers.TypeSerializer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class SimpleDateType extends AbstractType<Integer>
+{
+    public static final SimpleDateType instance = new SimpleDateType();
+
+    SimpleDateType() {} // singleton
+
+    public int compare(ByteBuffer o1, ByteBuffer o2)
+    {
+        // We add Integer.MIN_VALUE to overflow to allow unsigned comparison
+        return ByteBufferUtil.compareUnsigned(o1, o2);
+    }
+
+    @Override
+    public boolean isByteOrderComparable()
+    {
+        return true;
+    }
+
+    public ByteBuffer fromString(String source) throws MarshalException
+    {
+        return ByteBufferUtil.bytes(SimpleDateSerializer.dateStringToDays(source));
+    }
+
+    public ByteBuffer fromTimeInMillis(long millis) throws MarshalException
+    {
+        return ByteBufferUtil.bytes(SimpleDateSerializer.timeInMillisToDay(millis));
+    }
+
+    public long toTimeInMillis(ByteBuffer buffer) throws MarshalException
+    {
+        return SimpleDateSerializer.dayToTimeInMillis(ByteBufferUtil.toInt(buffer));
+    }
+
+    @Override
+    public boolean isValueCompatibleWithInternal(AbstractType<?> otherType)
+    {
+        return this == otherType || otherType == Int32Type.instance;
+    }
+
+    public Term fromJSONObject(Object parsed) throws MarshalException
+    {
+        try
+        {
+            return new Constants.Value(fromString((String) parsed));
+        }
+        catch (ClassCastException exc)
+        {
+            throw new MarshalException(String.format(
+                    "Expected a string representation of a date value, but got a %s: %s",
+                    parsed.getClass().getSimpleName(), parsed));
+        }
+    }
+
+    @Override
+    public String toJSONString(ByteBuffer buffer, int protocolVersion)
+    {
+        return '"' + SimpleDateSerializer.instance.toString(SimpleDateSerializer.instance.deserialize(buffer)) + '"';
+    }
+
+    @Override
+    public CQL3Type asCQL3Type()
+    {
+        return CQL3Type.Native.DATE;
+    }
+
+    public TypeSerializer<Integer> getSerializer()
+    {
+        return SimpleDateSerializer.instance;
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/marshal/TimeType.java b/src/java/org/apache/cassandra/db/marshal/TimeType.java
new file mode 100644
index 0000000..86de574
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/marshal/TimeType.java
@@ -0,0 +1,89 @@
+/*
+ * 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.db.marshal;
+
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.cql3.Constants;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.serializers.TimeSerializer;
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.serializers.TypeSerializer;
+import org.apache.cassandra.serializers.MarshalException;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+/**
+ * Nanosecond resolution time values
+ */
+public class TimeType extends AbstractType<Long>
+{
+    public static final TimeType instance = new TimeType();
+    private TimeType() {} // singleton
+
+    public int compare(ByteBuffer o1, ByteBuffer o2)
+    {
+        return ByteBufferUtil.compareUnsigned(o1, o2);
+    }
+
+    public ByteBuffer fromString(String source) throws MarshalException
+    {
+        return decompose(TimeSerializer.timeStringToLong(source));
+    }
+
+    @Override
+    public boolean isByteOrderComparable()
+    {
+        return true;
+    }
+
+    @Override
+    public boolean isValueCompatibleWithInternal(AbstractType<?> otherType)
+    {
+        return this == otherType || otherType == LongType.instance;
+    }
+
+    public Term fromJSONObject(Object parsed) throws MarshalException
+    {
+        try
+        {
+            return new Constants.Value(fromString((String) parsed));
+        }
+        catch (ClassCastException exc)
+        {
+            throw new MarshalException(String.format(
+                    "Expected a string representation of a time value, but got a %s: %s", parsed.getClass().getSimpleName(), parsed));
+        }
+    }
+
+    @Override
+    public String toJSONString(ByteBuffer buffer, int protocolVersion)
+    {
+        return '"' + TimeSerializer.instance.toString(TimeSerializer.instance.deserialize(buffer)) + '"';
+    }
+
+    @Override
+    public CQL3Type asCQL3Type()
+    {
+        return CQL3Type.Native.TIME;
+    }
+
+    public TypeSerializer<Long> getSerializer()
+    {
+        return TimeSerializer.instance;
+    }
+}
diff --git a/src/java/org/apache/cassandra/db/marshal/TimeUUIDType.java b/src/java/org/apache/cassandra/db/marshal/TimeUUIDType.java
index 88dc211..a1d8d82 100644
--- a/src/java/org/apache/cassandra/db/marshal/TimeUUIDType.java
+++ b/src/java/org/apache/cassandra/db/marshal/TimeUUIDType.java
@@ -19,131 +19,103 @@
 
 import java.nio.ByteBuffer;
 import java.util.UUID;
-import java.util.regex.Pattern;
 
 import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.Constants;
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.serializers.TimeUUIDSerializer;
-import org.apache.cassandra.serializers.TimestampSerializer;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.UUIDGen;
 
 public class TimeUUIDType extends AbstractType<UUID>
 {
     public static final TimeUUIDType instance = new TimeUUIDType();
 
-    static final Pattern regexPattern = Pattern.compile("[A-Fa-f0-9]{8}\\-[A-Fa-f0-9]{4}\\-[A-Fa-f0-9]{4}\\-[A-Fa-f0-9]{4}\\-[A-Fa-f0-9]{12}");
-
     TimeUUIDType()
     {
     } // singleton
 
-    public int compare(ByteBuffer o1, ByteBuffer o2)
+    public boolean isEmptyValueMeaningless()
     {
-        if (!o1.hasRemaining() || !o2.hasRemaining())
-            return o1.hasRemaining() ? 1 : o2.hasRemaining() ? -1 : 0;
-
-        int res = compareTimestampBytes(o1, o2);
-        if (res != 0)
-            return res;
-        return o1.compareTo(o2);
+        return true;
     }
 
-    private static int compareTimestampBytes(ByteBuffer o1, ByteBuffer o2)
+    public int compare(ByteBuffer b1, ByteBuffer b2)
     {
-        int o1Pos = o1.position();
-        int o2Pos = o2.position();
+        // Compare for length
+        int s1 = b1.position(), s2 = b2.position();
+        int l1 = b1.limit(), l2 = b2.limit();
 
-        int d = (o1.get(o1Pos + 6) & 0xF) - (o2.get(o2Pos + 6) & 0xF);
-        if (d != 0) return d;
+        // should we assert exactly 16 bytes (or 0)? seems prudent
+        boolean p1 = l1 - s1 == 16, p2 = l2 - s2 == 16;
+        if (!(p1 & p2))
+        {
+            assert p1 | (l1 == s1);
+            assert p2 | (l2 == s2);
+            return p1 ? 1 : p2 ? -1 : 0;
+        }
 
-        d = (o1.get(o1Pos + 7) & 0xFF) - (o2.get(o2Pos + 7) & 0xFF);
-        if (d != 0) return d;
+        long msb1 = b1.getLong(s1);
+        long msb2 = b2.getLong(s2);
+        msb1 = reorderTimestampBytes(msb1);
+        msb2 = reorderTimestampBytes(msb2);
 
-        d = (o1.get(o1Pos + 4) & 0xFF) - (o2.get(o2Pos + 4) & 0xFF);
-        if (d != 0) return d;
+        assert (msb1 & topbyte(0xf0L)) == topbyte(0x10L);
+        assert (msb2 & topbyte(0xf0L)) == topbyte(0x10L);
 
-        d = (o1.get(o1Pos + 5) & 0xFF) - (o2.get(o2Pos + 5) & 0xFF);
-        if (d != 0) return d;
+        int c = Long.compare(msb1, msb2);
+        if (c != 0)
+            return c;
 
-        d = (o1.get(o1Pos) & 0xFF) - (o2.get(o2Pos) & 0xFF);
-        if (d != 0) return d;
-
-        d = (o1.get(o1Pos + 1) & 0xFF) - (o2.get(o2Pos + 1) & 0xFF);
-        if (d != 0) return d;
-
-        d = (o1.get(o1Pos + 2) & 0xFF) - (o2.get(o2Pos + 2) & 0xFF);
-        if (d != 0) return d;
-
-        return (o1.get(o1Pos + 3) & 0xFF) - (o2.get(o2Pos + 3) & 0xFF);
+        // this has to be a signed per-byte comparison for compatibility
+        // so we transform the bytes so that a simple long comparison is equivalent
+        long lsb1 = signedBytesToNativeLong(b1.getLong(s1 + 8));
+        long lsb2 = signedBytesToNativeLong(b2.getLong(s2 + 8));
+        return Long.compare(lsb1, lsb2);
     }
 
-    // This accepts dates are valid TimeUUID represensation, which is bogus
-    // (see #4936) but kept for CQL2 for compatibility sake.
-    @Override
-    public ByteBuffer fromStringCQL2(String source) throws MarshalException
+    // takes as input 8 signed bytes in native machine order
+    // returns the first byte unchanged, and the following 7 bytes converted to an unsigned representation
+    // which is the same as a 2's complement long in native format
+    private static long signedBytesToNativeLong(long signedBytes)
     {
-        // Return an empty ByteBuffer for an empty string.
-        if (source.isEmpty())
-            return ByteBufferUtil.EMPTY_BYTE_BUFFER;
+        return signedBytes ^ 0x0080808080808080L;
+    }
 
-        ByteBuffer idBytes = null;
+    private static long topbyte(long topbyte)
+    {
+        return topbyte << 56;
+    }
 
-        // ffffffff-ffff-ffff-ffff-ffffffffff
-        if (regexPattern.matcher(source).matches())
-        {
-            UUID uuid = null;
-            try
-            {
-                uuid = UUID.fromString(source);
-                idBytes = decompose(uuid);
-            }
-            catch (IllegalArgumentException e)
-            {
-                throw new MarshalException(String.format("unable to make UUID from '%s'", source), e);
-            }
-
-            if (uuid.version() != 1)
-                throw new MarshalException("TimeUUID supports only version 1 UUIDs");
-        }
-        else
-        {
-            idBytes = ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes(TimestampSerializer.dateStringToTimestamp(source)));
-        }
-
-        return idBytes;
+    protected static long reorderTimestampBytes(long input)
+    {
+        return    (input <<  48)
+                  | ((input <<  16) & 0xFFFF00000000L)
+                  |  (input >>> 32);
     }
 
     public ByteBuffer fromString(String source) throws MarshalException
     {
-        // Return an empty ByteBuffer for an empty string.
-        if (source.isEmpty())
-            return ByteBufferUtil.EMPTY_BYTE_BUFFER;
-
-        ByteBuffer idBytes = null;
-
-        // ffffffff-ffff-ffff-ffff-ffffffffff
-        if (regexPattern.matcher(source).matches())
-        {
-            UUID uuid = null;
-            try
-            {
-                uuid = UUID.fromString(source);
-                idBytes = decompose(uuid);
-            }
-            catch (IllegalArgumentException e)
-            {
-                throw new MarshalException(String.format("Unable to make UUID from '%s'", source), e);
-            }
-
-            if (uuid.version() != 1)
-                throw new MarshalException("TimeUUID supports only version 1 UUIDs");
-        } else
-        {
+        ByteBuffer parsed = UUIDType.parse(source);
+        if (parsed == null)
             throw new MarshalException(String.format("Unknown timeuuid representation: %s", source));
+        if (parsed.remaining() == 16 && UUIDType.version(parsed) != 1)
+            throw new MarshalException("TimeUUID supports only version 1 UUIDs");
+        return parsed;
+    }
+
+    @Override
+    public Term fromJSONObject(Object parsed) throws MarshalException
+    {
+        try
+        {
+            return new Constants.Value(fromString((String) parsed));
         }
-        return idBytes;
+        catch (ClassCastException exc)
+        {
+            throw new MarshalException(
+                    String.format("Expected a string representation of a timeuuid, but got a %s: %s", parsed.getClass().getSimpleName(), parsed));
+        }
     }
 
     public CQL3Type asCQL3Type()
diff --git a/src/java/org/apache/cassandra/db/marshal/TimestampType.java b/src/java/org/apache/cassandra/db/marshal/TimestampType.java
index d7ce47b..1704362 100644
--- a/src/java/org/apache/cassandra/db/marshal/TimestampType.java
+++ b/src/java/org/apache/cassandra/db/marshal/TimestampType.java
@@ -20,6 +20,8 @@
 import java.nio.ByteBuffer;
 import java.util.Date;
 
+import org.apache.cassandra.cql3.Constants;
+import org.apache.cassandra.cql3.Term;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.cassandra.cql3.CQL3Type;
@@ -43,6 +45,11 @@
 
     private TimestampType() {} // singleton
 
+    public boolean isEmptyValueMeaningless()
+    {
+        return true;
+    }
+
     public int compare(ByteBuffer o1, ByteBuffer o2)
     {
         return LongType.compareLongs(o1, o2);
@@ -57,6 +64,35 @@
       return ByteBufferUtil.bytes(TimestampSerializer.dateStringToTimestamp(source));
     }
 
+    public ByteBuffer fromTimeInMillis(long millis) throws MarshalException
+    {
+        return ByteBufferUtil.bytes(millis);
+    }
+
+    @Override
+    public Term fromJSONObject(Object parsed) throws MarshalException
+    {
+        if (parsed instanceof Long)
+            return new Constants.Value(ByteBufferUtil.bytes((Long) parsed));
+
+        try
+        {
+            return new Constants.Value(TimestampType.instance.fromString((String) parsed));
+        }
+        catch (ClassCastException exc)
+        {
+            throw new MarshalException(String.format(
+                    "Expected a long or a datestring representation of a timestamp value, but got a %s: %s",
+                    parsed.getClass().getSimpleName(), parsed));
+        }
+    }
+
+    @Override
+    public String toJSONString(ByteBuffer buffer, int protocolVersion)
+    {
+        return '"' + TimestampSerializer.getJsonDateFormatter().format(TimestampSerializer.instance.deserialize(buffer)) + '"';
+    }
+
     @Override
     public boolean isCompatibleWith(AbstractType<?> previous)
     {
diff --git a/src/java/org/apache/cassandra/db/marshal/TupleType.java b/src/java/org/apache/cassandra/db/marshal/TupleType.java
index dbce0db..0d08a52 100644
--- a/src/java/org/apache/cassandra/db/marshal/TupleType.java
+++ b/src/java/org/apache/cassandra/db/marshal/TupleType.java
@@ -18,12 +18,14 @@
 package org.apache.cassandra.db.marshal;
 
 import java.nio.ByteBuffer;
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Iterator;
 import java.util.List;
 
 import com.google.common.base.Objects;
 
-import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.serializers.*;
@@ -240,6 +242,59 @@
         return buildValue(fields);
     }
 
+    @Override
+    public Term fromJSONObject(Object parsed) throws MarshalException
+    {
+        if (parsed instanceof String)
+            parsed = Json.decodeJson((String) parsed);
+
+        if (!(parsed instanceof List))
+            throw new MarshalException(String.format(
+                    "Expected a list representation of a tuple, but got a %s: %s", parsed.getClass().getSimpleName(), parsed));
+
+        List list = (List) parsed;
+
+        if (list.size() > types.size())
+            throw new MarshalException(String.format("Tuple contains extra items (expected %s): %s", types.size(), parsed));
+        else if (types.size() > list.size())
+            throw new MarshalException(String.format("Tuple is missing items (expected %s): %s", types.size(), parsed));
+
+        List<Term> terms = new ArrayList<>(list.size());
+        Iterator<AbstractType<?>> typeIterator = types.iterator();
+        for (Object element : list)
+        {
+            if (element == null)
+            {
+                typeIterator.next();
+                terms.add(Constants.NULL_VALUE);
+            }
+            else
+            {
+                terms.add(typeIterator.next().fromJSONObject(element));
+            }
+        }
+
+        return new Tuples.DelayedValue(this, terms);
+    }
+
+    @Override
+    public String toJSONString(ByteBuffer buffer, int protocolVersion)
+    {
+        StringBuilder sb = new StringBuilder("[");
+        for (int i = 0; i < types.size(); i++)
+        {
+            if (i > 0)
+                sb.append(", ");
+
+            ByteBuffer value = CollectionSerializer.readValue(buffer, protocolVersion);
+            if (value == null)
+                sb.append("null");
+            else
+                sb.append(types.get(i).toJSONString(value, protocolVersion));
+        }
+        return sb.append("]").toString();
+    }
+
     public TypeSerializer<ByteBuffer> getSerializer()
     {
         return BytesSerializer.instance;
diff --git a/src/java/org/apache/cassandra/db/marshal/TypeParser.java b/src/java/org/apache/cassandra/db/marshal/TypeParser.java
index cdb5679..faa678e 100644
--- a/src/java/org/apache/cassandra/db/marshal/TypeParser.java
+++ b/src/java/org/apache/cassandra/db/marshal/TypeParser.java
@@ -21,14 +21,9 @@
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
+import java.util.*;
 
-import org.apache.commons.lang3.StringUtils;
-
+import org.apache.cassandra.cql3.CQL3Type;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
@@ -43,7 +38,7 @@
     private int idx;
 
     // A cache of parsed string, specially useful for DynamicCompositeType
-    private static final Map<String, AbstractType<?>> cache = new HashMap<String, AbstractType<?>>();
+    private static final Map<String, AbstractType<?>> cache = new HashMap<>();
 
     public static final TypeParser EMPTY_PARSER = new TypeParser("", 0);
 
@@ -99,9 +94,48 @@
         return parse(compareWith == null ? null : compareWith.toString());
     }
 
-    public static String getShortName(AbstractType<?> type)
+    public static String parseCqlNativeType(String str)
     {
-        return type.getClass().getSimpleName();
+        return CQL3Type.Native.valueOf(str.trim().toUpperCase(Locale.ENGLISH)).getType().toString();
+    }
+
+    public static String parseCqlCollectionOrFrozenType(String str) throws SyntaxException
+    {
+        str = str.trim().toLowerCase();
+        switch (str)
+        {
+            case "map": return "MapType";
+            case "set": return "SetType";
+            case "list": return "ListType";
+            case "frozen": return "FrozenType";
+            default: throw new SyntaxException("Invalid type name" + str);
+        }
+    }
+
+    /**
+     * Turns user facing type names into Abstract Types, 'text' -> UTF8Type
+     */
+    public static AbstractType<?> parseCqlName(String str) throws SyntaxException, ConfigurationException
+    {
+        return parse(parseCqlNameRecurse(str));
+    }
+
+    private static String parseCqlNameRecurse(String str) throws SyntaxException
+    {
+        if (str.indexOf(',') >= 0 && (!str.contains("<") || (str.indexOf(',') < str.indexOf('<'))))
+        {
+            String[] parseString = str.split(",", 2);
+            return parseCqlNameRecurse(parseString[0]) + "," + parseCqlNameRecurse(parseString[1]);
+        }
+        else if (str.contains("<"))
+        {
+            String[] parseString = str.trim().split("<", 2);
+            return parseCqlCollectionOrFrozenType(parseString[0]) + "(" + parseCqlNameRecurse(parseString[1].substring(0, parseString[1].length()-1)) + ")";
+        }
+        else
+        {
+            return parseCqlNativeType(str);
+        }
     }
 
     /**
@@ -121,14 +155,13 @@
 
     public Map<String, String> getKeyValueParameters() throws SyntaxException
     {
-        Map<String, String> map = new HashMap<String, String>();
-
         if (isEOS())
-            return map;
+            return Collections.emptyMap();
 
         if (str.charAt(idx) != '(')
             throw new IllegalStateException();
 
+        Map<String, String> map = new HashMap<>();
         ++idx; // skipping '('
 
         while (skipBlankAndComma())
@@ -159,7 +192,7 @@
 
     public List<AbstractType<?>> getTypeParameters() throws SyntaxException, ConfigurationException
     {
-        List<AbstractType<?>> list = new ArrayList<AbstractType<?>>();
+        List<AbstractType<?>> list = new ArrayList<>();
 
         if (isEOS())
             return list;
@@ -193,7 +226,7 @@
 
     public Map<Byte, AbstractType<?>> getAliasParameters() throws SyntaxException, ConfigurationException
     {
-        Map<Byte, AbstractType<?>> map = new HashMap<Byte, AbstractType<?>>();
+        Map<Byte, AbstractType<?>> map = new HashMap<>();
 
         if (isEOS())
             return map;
@@ -271,7 +304,7 @@
             {
                 AbstractType<?> type = parse();
                 if (!(type instanceof CollectionType))
-                    throw new SyntaxException(type.toString() + " is not a collection type");
+                    throw new SyntaxException(type + " is not a collection type");
                 map.put(bb, (CollectionType)type);
             }
             catch (SyntaxException e)
@@ -349,12 +382,7 @@
             Field field = typeClass.getDeclaredField("instance");
             return (AbstractType<?>) field.get(null);
         }
-        catch (NoSuchFieldException e)
-        {
-            // Trying with empty parser
-            return getRawAbstractType(typeClass, EMPTY_PARSER);
-        }
-        catch (IllegalAccessException e)
+        catch (NoSuchFieldException | IllegalAccessException e)
         {
             // Trying with empty parser
             return getRawAbstractType(typeClass, EMPTY_PARSER);
@@ -370,13 +398,7 @@
             Method method = typeClass.getDeclaredMethod("getInstance", TypeParser.class);
             return (AbstractType<?>) method.invoke(null, parser);
         }
-        catch (NoSuchMethodException e)
-        {
-            // Trying to see if we have an instance field and apply the default parameter to it
-            AbstractType<?> type = getRawAbstractType(typeClass);
-            return AbstractType.parseDefaultParameters(type, parser);
-        }
-        catch (IllegalAccessException e)
+        catch (NoSuchMethodException | IllegalAccessException e)
         {
             // Trying to see if we have an instance field and apply the default parameter to it
             AbstractType<?> type = getRawAbstractType(typeClass);
@@ -397,11 +419,7 @@
             Field field = typeClass.getDeclaredField("instance");
             return (AbstractType<?>) field.get(null);
         }
-        catch (NoSuchFieldException e)
-        {
-            throw new ConfigurationException("Invalid comparator class " + typeClass.getName() + ": must define a public static instance field or a public static method getInstance(TypeParser).");
-        }
-        catch (IllegalAccessException e)
+        catch (NoSuchFieldException | IllegalAccessException e)
         {
             throw new ConfigurationException("Invalid comparator class " + typeClass.getName() + ": must define a public static instance field or a public static method getInstance(TypeParser).");
         }
@@ -414,11 +432,7 @@
             Method method = typeClass.getDeclaredMethod("getInstance", TypeParser.class);
             return (AbstractType<?>) method.invoke(null, parser);
         }
-        catch (NoSuchMethodException e)
-        {
-            throw new ConfigurationException("Invalid comparator class " + typeClass.getName() + ": must define a public static instance field or a public static method getInstance(TypeParser).");
-        }
-        catch (IllegalAccessException e)
+        catch (NoSuchMethodException | IllegalAccessException e)
         {
             throw new ConfigurationException("Invalid comparator class " + typeClass.getName() + ": must define a public static instance field or a public static method getInstance(TypeParser).");
         }
@@ -506,12 +520,6 @@
         return str.substring(i, idx);
     }
 
-    public char readNextChar()
-    {
-        skipBlank();
-        return str.charAt(idx++);
-    }
-
     /**
      * Helper function to ease the writing of AbstractType.toString() methods.
      */
@@ -584,7 +592,6 @@
         {
             sb.append(',');
             sb.append(ByteBufferUtil.bytesToHex(columnNames.get(i))).append(":");
-
             // omit FrozenType(...) from fields because it is currently implicit
             sb.append(columnTypes.get(i).toString(true));
         }
diff --git a/src/java/org/apache/cassandra/db/marshal/UTF8Type.java b/src/java/org/apache/cassandra/db/marshal/UTF8Type.java
index 6d58db2..3b93d9a 100644
--- a/src/java/org/apache/cassandra/db/marshal/UTF8Type.java
+++ b/src/java/org/apache/cassandra/db/marshal/UTF8Type.java
@@ -18,8 +18,15 @@
 package org.apache.cassandra.db.marshal;
 
 import java.nio.ByteBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+
+import org.apache.cassandra.cql3.Constants;
+import org.apache.cassandra.cql3.Json;
 
 import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.serializers.UTF8Serializer;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -40,6 +47,34 @@
         return decompose(source);
     }
 
+
+    @Override
+    public Term fromJSONObject(Object parsed) throws MarshalException
+    {
+        try
+        {
+            return new Constants.Value(fromString((String) parsed));
+        }
+        catch (ClassCastException exc)
+        {
+            throw new MarshalException(String.format(
+                    "Expected a UTF-8 string, but got a %s: %s", parsed.getClass().getSimpleName(), parsed));
+        }
+    }
+
+    @Override
+    public String toJSONString(ByteBuffer buffer, int protocolVersion)
+    {
+        try
+        {
+            return '"' + Json.quoteAsJsonString(ByteBufferUtil.string(buffer, Charset.forName("UTF-8"))) + '"';
+        }
+        catch (CharacterCodingException exc)
+        {
+            throw new AssertionError("UTF-8 value contained non-utf8 characters: ", exc);
+        }
+    }
+
     @Override
     public boolean isCompatibleWith(AbstractType<?> previous)
     {
diff --git a/src/java/org/apache/cassandra/db/marshal/UUIDType.java b/src/java/org/apache/cassandra/db/marshal/UUIDType.java
index c4ce1d1..0250eb20 100644
--- a/src/java/org/apache/cassandra/db/marshal/UUIDType.java
+++ b/src/java/org/apache/cassandra/db/marshal/UUIDType.java
@@ -17,19 +17,20 @@
  */
 package org.apache.cassandra.db.marshal;
 
-
 import java.nio.ByteBuffer;
-import java.text.ParseException;
 import java.util.UUID;
+import java.util.regex.Pattern;
+
+import com.google.common.primitives.UnsignedLongs;
 
 import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.Constants;
+import org.apache.cassandra.cql3.Term;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.serializers.UUIDSerializer;
-import org.apache.cassandra.serializers.TimestampSerializer;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.UUIDGen;
-import org.apache.commons.lang3.time.DateUtils;
 
 /**
  * Compares UUIDs using the following criteria:<br>
@@ -49,152 +50,74 @@
     {
     }
 
-    public int compare(ByteBuffer b1, ByteBuffer b2)
+    public boolean isEmptyValueMeaningless()
     {
-
-        // Compare for length
-
-        if ((b1 == null) || (b1.remaining() < 16))
-        {
-            return ((b2 == null) || (b2.remaining() < 16)) ? 0 : -1;
-        }
-        if ((b2 == null) || (b2.remaining() < 16))
-        {
-            return 1;
-        }
-
-        int s1 = b1.position();
-        int s2 = b2.position();
-
-        // Compare versions
-
-        int v1 = (b1.get(s1 + 6) >> 4) & 0x0f;
-        int v2 = (b2.get(s2 + 6) >> 4) & 0x0f;
-
-        if (v1 != v2)
-        {
-            return v1 - v2;
-        }
-
-        // Compare timestamps for version 1
-
-        if (v1 == 1)
-        {
-            // if both time-based, compare as timestamps
-            int c = compareTimestampBytes(b1, b2);
-            if (c != 0)
-            {
-                return c;
-            }
-        }
-
-        // Compare the two byte arrays starting from the first
-        // byte in the sequence until an inequality is
-        // found. This should provide equivalent results
-        // to the comparison performed by the RFC 4122
-        // Appendix A - Sample Implementation.
-        // Note: java.util.UUID.compareTo is not a lexical
-        // comparison
-        for (int i = 0; i < 16; i++)
-        {
-            int c = ((b1.get(s1 + i)) & 0xFF) - ((b2.get(s2 + i)) & 0xFF);
-            if (c != 0)
-            {
-                return c;
-            }
-        }
-
-        return 0;
+        return true;
     }
 
-    private static int compareTimestampBytes(ByteBuffer o1, ByteBuffer o2)
+    public int compare(ByteBuffer b1, ByteBuffer b2)
     {
-        int o1Pos = o1.position();
-        int o2Pos = o2.position();
+        // Compare for length
+        int s1 = b1.position(), s2 = b2.position();
+        int l1 = b1.limit(), l2 = b2.limit();
 
-        int d = (o1.get(o1Pos + 6) & 0xF) - (o2.get(o2Pos + 6) & 0xF);
-        if (d != 0)
+        // should we assert exactly 16 bytes (or 0)? seems prudent
+        boolean p1 = l1 - s1 == 16, p2 = l2 - s2 == 16;
+        if (!(p1 & p2))
         {
-            return d;
+            assert p1 | (l1 == s1);
+            assert p2 | (l2 == s2);
+            return p1 ? 1 : p2 ? -1 : 0;
         }
 
-        d = (o1.get(o1Pos + 7) & 0xFF) - (o2.get(o2Pos + 7) & 0xFF);
-        if (d != 0)
+        // Compare versions
+        long msb1 = b1.getLong(s1);
+        long msb2 = b2.getLong(s2);
+
+        int version1 = (int) ((msb1 >>> 12) & 0xf);
+        int version2 = (int) ((msb2 >>> 12) & 0xf);
+        if (version1 != version2)
+            return version1 - version2;
+
+        // bytes: version is top 4 bits of byte 6
+        // then: [6.5-8), [4-6), [0-4)
+        if (version1 == 1)
         {
-            return d;
+            long reorder1 = TimeUUIDType.reorderTimestampBytes(msb1);
+            long reorder2 = TimeUUIDType.reorderTimestampBytes(msb2);
+            // we know this is >= 0, since the top 3 bits will be 0
+            int c = Long.compare(reorder1, reorder2);
+            if (c != 0)
+                return c;
+        }
+        else
+        {
+            int c = UnsignedLongs.compare(msb1, msb2);
+            if (c != 0)
+                return c;
         }
 
-        d = (o1.get(o1Pos + 4) & 0xFF) - (o2.get(o2Pos + 4) & 0xFF);
-        if (d != 0)
-        {
-            return d;
-        }
+        return UnsignedLongs.compare(b1.getLong(s1 + 8), b2.getLong(s2 + 8));
+    }
 
-        d = (o1.get(o1Pos + 5) & 0xFF) - (o2.get(o2Pos + 5) & 0xFF);
-        if (d != 0)
-        {
-            return d;
-        }
-
-        d = (o1.get(o1Pos) & 0xFF) - (o2.get(o2Pos) & 0xFF);
-        if (d != 0)
-        {
-            return d;
-        }
-
-        d = (o1.get(o1Pos + 1) & 0xFF) - (o2.get(o2Pos + 1) & 0xFF);
-        if (d != 0)
-        {
-            return d;
-        }
-
-        d = (o1.get(o1Pos + 2) & 0xFF) - (o2.get(o2Pos + 2) & 0xFF);
-        if (d != 0)
-        {
-            return d;
-        }
-
-        return (o1.get(o1Pos + 3) & 0xFF) - (o2.get(o2Pos + 3) & 0xFF);
+    @Override
+    public boolean isValueCompatibleWithInternal(AbstractType<?> otherType)
+    {
+        return otherType instanceof UUIDType || otherType instanceof TimeUUIDType;
     }
 
     @Override
     public ByteBuffer fromString(String source) throws MarshalException
     {
         // Return an empty ByteBuffer for an empty string.
-        if (source.isEmpty())
-            return ByteBufferUtil.EMPTY_BYTE_BUFFER;
+        ByteBuffer parsed = parse(source);
+        if (parsed != null)
+            return parsed;
 
-        // ffffffff-ffff-ffff-ffff-ffffffffff
-        if (TimeUUIDType.regexPattern.matcher(source).matches())
-        {
-            UUID uuid;
-            try
-            {
-                uuid = UUID.fromString(source);
-                return ByteBuffer.wrap(UUIDGen.decompose(uuid));
-            }
-            catch (IllegalArgumentException e)
-            {
-                throw new MarshalException(String.format("unable to make UUID from '%s'", source), e);
-            }
-        }
-
-        try
-        {
-            return ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes(TimestampSerializer.dateStringToTimestamp(source)));
-        }
-        catch (MarshalException e)
-        {
-            throw new MarshalException(String.format("unable to make version 1 UUID from '%s'", source), e);
-        }
+        throw new MarshalException(String.format("Unable to make UUID from '%s'", source));
     }
 
     @Override
-    public boolean isValueCompatibleWithInternal(AbstractType<?> otherType)
-    {
-        return this == otherType || otherType == TimeUUIDType.instance;
-    }
-
     public CQL3Type asCQL3Type()
     {
         return CQL3Type.Native.UUID;
@@ -205,4 +128,44 @@
         return UUIDSerializer.instance;
     }
 
+    static final Pattern regexPattern = Pattern.compile("[A-Fa-f0-9]{8}\\-[A-Fa-f0-9]{4}\\-[A-Fa-f0-9]{4}\\-[A-Fa-f0-9]{4}\\-[A-Fa-f0-9]{12}");
+
+    static ByteBuffer parse(String source)
+    {
+        if (source.isEmpty())
+            return ByteBufferUtil.EMPTY_BYTE_BUFFER;
+
+        if (regexPattern.matcher(source).matches())
+        {
+            try
+            {
+                return ByteBuffer.wrap(UUIDGen.decompose(UUID.fromString(source)));
+            }
+            catch (IllegalArgumentException e)
+            {
+                throw new MarshalException(String.format("Unable to make UUID from '%s'", source), e);
+            }
+        }
+
+        return null;
+    }
+
+    @Override
+    public Term fromJSONObject(Object parsed) throws MarshalException
+    {
+        try
+        {
+            return new Constants.Value(fromString((String) parsed));
+        }
+        catch (ClassCastException exc)
+        {
+            throw new MarshalException(String.format(
+                    "Expected a string representation of a uuid, but got a %s: %s", parsed.getClass().getSimpleName(), parsed));
+        }
+    }
+
+    static int version(ByteBuffer uuid)
+    {
+        return (uuid.get(6) & 0xf0) >> 4;
+    }
 }
diff --git a/src/java/org/apache/cassandra/db/marshal/UserType.java b/src/java/org/apache/cassandra/db/marshal/UserType.java
index 180d713..187deeb 100644
--- a/src/java/org/apache/cassandra/db/marshal/UserType.java
+++ b/src/java/org/apache/cassandra/db/marshal/UserType.java
@@ -18,16 +18,16 @@
 package org.apache.cassandra.db.marshal;
 
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.StandardCharsets;
+import java.util.*;
 
 import com.google.common.base.Objects;
 
-import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
-import org.apache.cassandra.serializers.*;
+import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.Pair;
 
@@ -41,6 +41,7 @@
     public final String keyspace;
     public final ByteBuffer name;
     private final List<ByteBuffer> fieldNames;
+    private final List<String> stringFieldNames;
 
     public UserType(String keyspace, ByteBuffer name, List<ByteBuffer> fieldNames, List<AbstractType<?>> fieldTypes)
     {
@@ -49,6 +50,18 @@
         this.keyspace = keyspace;
         this.name = name;
         this.fieldNames = fieldNames;
+        this.stringFieldNames = new ArrayList<>(fieldNames.size());
+        for (ByteBuffer fieldName : fieldNames)
+        {
+            try
+            {
+                stringFieldNames.add(ByteBufferUtil.string(fieldName, StandardCharsets.UTF_8));
+            }
+            catch (CharacterCodingException ex)
+            {
+                throw new AssertionError("Got non-UTF8 field name for user-defined type: " + ByteBufferUtil.bytesToHex(fieldName), ex);
+            }
+        }
     }
 
     public static UserType getInstance(TypeParser parser) throws ConfigurationException, SyntaxException
@@ -81,6 +94,11 @@
         return fieldNames.get(i);
     }
 
+    public String fieldNameAsString(int i)
+    {
+        return stringFieldNames.get(i);
+    }
+
     public List<ByteBuffer> fieldNames()
     {
         return fieldNames;
@@ -103,7 +121,7 @@
                 return;
 
             if (input.remaining() < 4)
-                throw new MarshalException(String.format("Not enough bytes to read size of %dth field %s", i, fieldName(i)));
+                throw new MarshalException(String.format("Not enough bytes to read size of %dth field %s", i, fieldNameAsString(i)));
 
             int size = input.getInt();
 
@@ -112,7 +130,7 @@
                 continue;
 
             if (input.remaining() < size)
-                throw new MarshalException(String.format("Not enough bytes to read %dth field %s", i, fieldName(i)));
+                throw new MarshalException(String.format("Not enough bytes to read %dth field %s", i, fieldNameAsString(i)));
 
             ByteBuffer field = ByteBufferUtil.readBytes(input, size);
             types.get(i).validate(field);
@@ -124,6 +142,81 @@
     }
 
     @Override
+    public Term fromJSONObject(Object parsed) throws MarshalException
+    {
+        if (parsed instanceof String)
+            parsed = Json.decodeJson((String) parsed);
+
+        if (!(parsed instanceof Map))
+            throw new MarshalException(String.format(
+                    "Expected a map, but got a %s: %s", parsed.getClass().getSimpleName(), parsed));
+
+        Map<String, Object> map = (Map<String, Object>) parsed;
+
+        Json.handleCaseSensitivity(map);
+
+        List<Term> terms = new ArrayList<>(types.size());
+
+        Set keys = map.keySet();
+        assert keys.isEmpty() || keys.iterator().next() instanceof String;
+
+        int foundValues = 0;
+        for (int i = 0; i < types.size(); i++)
+        {
+            Object value = map.get(stringFieldNames.get(i));
+            if (value == null)
+            {
+                terms.add(Constants.NULL_VALUE);
+            }
+            else
+            {
+                terms.add(types.get(i).fromJSONObject(value));
+                foundValues += 1;
+            }
+        }
+
+        // check for extra, unrecognized fields
+        if (foundValues != map.size())
+        {
+            for (Object fieldName : keys)
+            {
+                if (!stringFieldNames.contains(fieldName))
+                    throw new MarshalException(String.format(
+                            "Unknown field '%s' in value of user defined type %s", fieldName, getNameAsString()));
+            }
+        }
+
+        return new UserTypes.DelayedValue(this, terms);
+    }
+
+    @Override
+    public String toJSONString(ByteBuffer buffer, int protocolVersion)
+    {
+        ByteBuffer[] buffers = split(buffer);
+        StringBuilder sb = new StringBuilder("{");
+        for (int i = 0; i < types.size(); i++)
+        {
+            if (i > 0)
+                sb.append(", ");
+
+            String name = stringFieldNames.get(i);
+            if (!name.equals(name.toLowerCase(Locale.US)))
+                name = "\"" + name + "\"";
+
+            sb.append('"');
+            sb.append(Json.quoteAsJsonString(name));
+            sb.append("\": ");
+
+            ByteBuffer valueBuffer = (i >= buffers.length) ? null : buffers[i];
+            if (valueBuffer == null)
+                sb.append("null");
+            else
+                sb.append(types.get(i).toJSONString(valueBuffer, protocolVersion));
+        }
+        return sb.append("}").toString();
+    }
+
+    @Override
     public int hashCode()
     {
         return Objects.hashCode(keyspace, name, fieldNames, types);
diff --git a/src/java/org/apache/cassandra/dht/AbstractBounds.java b/src/java/org/apache/cassandra/dht/AbstractBounds.java
index 704d8c2..c33ffc0 100644
--- a/src/java/org/apache/cassandra/dht/AbstractBounds.java
+++ b/src/java/org/apache/cassandra/dht/AbstractBounds.java
@@ -20,20 +20,23 @@
 import java.io.DataInput;
 import java.io.IOException;
 import java.io.Serializable;
-import java.util.*;
+import java.util.Collection;
+import java.util.List;
 
 import org.apache.cassandra.db.DecoratedKey;
-import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.db.RowPosition;
+import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.utils.Pair;
 
 public abstract class AbstractBounds<T extends RingPosition<T>> implements Serializable
 {
     private static final long serialVersionUID = 1L;
-    public static final AbstractBoundsSerializer serializer = new AbstractBoundsSerializer();
+    public static final IPartitionerDependentSerializer<AbstractBounds<Token>> tokenSerializer =
+            new AbstractBoundsSerializer<Token>(Token.serializer);
+    public static final IPartitionerDependentSerializer<AbstractBounds<RowPosition>> rowPositionSerializer =
+            new AbstractBoundsSerializer<RowPosition>(RowPosition.serializer);
 
     private enum Type
     {
@@ -44,13 +47,11 @@
     public final T left;
     public final T right;
 
-    protected transient final IPartitioner partitioner;
-
-    public AbstractBounds(T left, T right, IPartitioner partitioner)
+    public AbstractBounds(T left, T right)
     {
+        assert left.getPartitioner() == right.getPartitioner();
         this.left = left;
         this.right = right;
-        this.partitioner = partitioner;
     }
 
     /**
@@ -71,6 +72,30 @@
     public abstract boolean inclusiveLeft();
     public abstract boolean inclusiveRight();
 
+    /**
+     * Whether {@code left} and {@code right} forms a wrapping interval, that is if unwrapping wouldn't be a no-op.
+     * <p>
+     * Note that the semantic is slightly different from {@link Range#isWrapAround()} in the sense that if both
+     * {@code right} are minimal (for the partitioner), this methods return false (doesn't wrap) while
+     * {@link Range#isWrapAround()} returns true (does wrap). This is confusing and we should fix it by
+     * refactoring/rewriting the whole AbstractBounds hierarchy with cleaner semantics, but we don't want to risk
+     * breaking something by changing {@link Range#isWrapAround()} in the meantime.
+     */
+    public static <T extends RingPosition<T>> boolean strictlyWrapsAround(T left, T right)
+    {
+        return !(left.compareTo(right) <= 0 || right.isMinimum());
+    }
+
+    public static <T extends RingPosition<T>> boolean noneStrictlyWrapsAround(Collection<AbstractBounds<T>> bounds)
+    {
+        for (AbstractBounds<T> b : bounds)
+        {
+            if (strictlyWrapsAround(b.left, b.right))
+                return false;
+        }
+        return true;
+    }
+
     @Override
     public int hashCode()
     {
@@ -112,42 +137,13 @@
     protected abstract String getOpeningString();
     protected abstract String getClosingString();
 
-    /**
-     * Transform this abstract bounds to equivalent covering bounds of row positions.
-     * If this abstract bounds was already an abstractBounds of row positions, this is a noop.
-     */
-    public abstract AbstractBounds<RowPosition> toRowBounds();
-
-    /**
-     * Transform this abstract bounds to a token abstract bounds.
-     * If this abstract bounds was already an abstractBounds of token, this is a noop, otherwise this use the row position tokens.
-     */
-    public abstract AbstractBounds<Token> toTokenBounds();
-
     public abstract AbstractBounds<T> withNewRight(T newRight);
 
-    public static class AbstractBoundsSerializer implements IVersionedSerializer<AbstractBounds<?>>
+    public static class AbstractBoundsSerializer<T extends RingPosition<T>> implements IPartitionerDependentSerializer<AbstractBounds<T>>
     {
-        public void serialize(AbstractBounds<?> range, DataOutputPlus out, int version) throws IOException
-        {
-            /*
-             * The first int tells us if it's a range or bounds (depending on the value) _and_ if it's tokens or keys (depending on the
-             * sign). We use negative kind for keys so as to preserve the serialization of token from older version.
-             */
-            out.writeInt(kindInt(range));
-            if (range.left instanceof Token)
-            {
-                Token.serializer.serialize((Token) range.left, out);
-                Token.serializer.serialize((Token) range.right, out);
-            }
-            else
-            {
-                RowPosition.serializer.serialize((RowPosition) range.left, out);
-                RowPosition.serializer.serialize((RowPosition) range.right, out);
-            }
-        }
+        IPartitionerDependentSerializer<T> serializer;
 
-        private int kindInt(AbstractBounds<?> ab)
+        private static int kindInt(AbstractBounds<?> ab)
         {
             int kind = ab instanceof Range ? Type.RANGE.ordinal() : Type.BOUNDS.ordinal();
             if (!(ab.left instanceof Token))
@@ -155,43 +151,43 @@
             return kind;
         }
 
-        public AbstractBounds<?> deserialize(DataInput in, int version) throws IOException
+        public AbstractBoundsSerializer(IPartitionerDependentSerializer<T> serializer)
+        {
+            this.serializer = serializer;
+        }
+
+        public void serialize(AbstractBounds<T> range, DataOutputPlus out, int version) throws IOException
+        {
+            /*
+             * The first int tells us if it's a range or bounds (depending on the value) _and_ if it's tokens or keys (depending on the
+             * sign). We use negative kind for keys so as to preserve the serialization of token from older version.
+             */
+            out.writeInt(kindInt(range));
+            serializer.serialize(range.left, out, version);
+            serializer.serialize(range.right, out, version);
+        }
+
+        public AbstractBounds<T> deserialize(DataInput in, IPartitioner p, int version) throws IOException
         {
             int kind = in.readInt();
             boolean isToken = kind >= 0;
             if (!isToken)
                 kind = -(kind+1);
 
-            RingPosition<?> left, right;
-            if (isToken)
-            {
-                left = Token.serializer.deserialize(in);
-                right = Token.serializer.deserialize(in);
-            }
-            else
-            {
-                left = RowPosition.serializer.deserialize(in);
-                right = RowPosition.serializer.deserialize(in);
-            }
+            T left = serializer.deserialize(in, p, version);
+            T right = serializer.deserialize(in, p, version);
+            assert isToken == left instanceof Token;
 
             if (kind == Type.RANGE.ordinal())
-                return new Range(left, right);
-            return new Bounds(left, right);
+                return new Range<T>(left, right);
+            return new Bounds<T>(left, right);
         }
 
-        public long serializedSize(AbstractBounds<?> ab, int version)
+        public long serializedSize(AbstractBounds<T> ab, int version)
         {
             int size = TypeSizes.NATIVE.sizeof(kindInt(ab));
-            if (ab.left instanceof Token)
-            {
-                size += Token.serializer.serializedSize((Token) ab.left, TypeSizes.NATIVE);
-                size += Token.serializer.serializedSize((Token) ab.right, TypeSizes.NATIVE);
-            }
-            else
-            {
-                size += RowPosition.serializer.serializedSize((RowPosition) ab.left, TypeSizes.NATIVE);
-                size += RowPosition.serializer.serializedSize((RowPosition) ab.right, TypeSizes.NATIVE);
-            }
+            size += serializer.serializedSize(ab.left, version);
+            size += serializer.serializedSize(ab.right, version);
             return size;
         }
     }
diff --git a/src/java/org/apache/cassandra/dht/AbstractByteOrderedPartitioner.java b/src/java/org/apache/cassandra/dht/AbstractByteOrderedPartitioner.java
deleted file mode 100644
index d3124ef..0000000
--- a/src/java/org/apache/cassandra/dht/AbstractByteOrderedPartitioner.java
+++ /dev/null
@@ -1,206 +0,0 @@
-/*
- * 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.dht;
-
-import java.math.BigInteger;
-import java.nio.ByteBuffer;
-import java.util.*;
-
-import org.apache.cassandra.config.*;
-import org.apache.cassandra.db.BufferDecoratedKey;
-import org.apache.commons.lang3.ArrayUtils;
-
-import org.apache.cassandra.db.DecoratedKey;
-import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.db.marshal.BytesType;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.Hex;
-import org.apache.cassandra.utils.Pair;
-
-public abstract class AbstractByteOrderedPartitioner extends AbstractPartitioner
-{
-    public static final BytesToken MINIMUM = new BytesToken(ArrayUtils.EMPTY_BYTE_ARRAY);
-
-    public static final BigInteger BYTE_MASK = new BigInteger("255");
-
-    public DecoratedKey decorateKey(ByteBuffer key)
-    {
-        return new BufferDecoratedKey(getToken(key), key);
-    }
-
-    public BytesToken midpoint(Token lt, Token rt)
-    {
-        BytesToken ltoken = (BytesToken) lt;
-        BytesToken rtoken = (BytesToken) rt;
-
-        int sigbytes = Math.max(ltoken.token.length, rtoken.token.length);
-        BigInteger left = bigForBytes(ltoken.token, sigbytes);
-        BigInteger right = bigForBytes(rtoken.token, sigbytes);
-
-        Pair<BigInteger,Boolean> midpair = FBUtilities.midpoint(left, right, 8*sigbytes);
-        return new BytesToken(bytesForBig(midpair.left, sigbytes, midpair.right));
-    }
-
-    /**
-     * Convert a byte array containing the most significant of 'sigbytes' bytes
-     * representing a big-endian magnitude into a BigInteger.
-     */
-    private BigInteger bigForBytes(byte[] bytes, int sigbytes)
-    {
-        byte[] b;
-        if (sigbytes != bytes.length)
-        {
-            b = new byte[sigbytes];
-            System.arraycopy(bytes, 0, b, 0, bytes.length);
-        } else
-            b = bytes;
-        return new BigInteger(1, b);
-    }
-
-    /**
-     * Convert a (positive) BigInteger into a byte array representing its magnitude.
-     * If remainder is true, an additional byte with the high order bit enabled
-     * will be added to the end of the array
-     */
-    private byte[] bytesForBig(BigInteger big, int sigbytes, boolean remainder)
-    {
-        byte[] bytes = new byte[sigbytes + (remainder ? 1 : 0)];
-        if (remainder)
-        {
-            // remaining bit is the most significant in the last byte
-            bytes[sigbytes] |= 0x80;
-        }
-        // bitmask for a single byte
-        for (int i = 0; i < sigbytes; i++)
-        {
-            int maskpos = 8 * (sigbytes - (i + 1));
-            // apply bitmask and get byte value
-            bytes[i] = (byte)(big.and(BYTE_MASK.shiftLeft(maskpos)).shiftRight(maskpos).intValue() & 0xFF);
-        }
-        return bytes;
-    }
-
-    public BytesToken getMinimumToken()
-    {
-        return MINIMUM;
-    }
-
-    public BytesToken getRandomToken()
-    {
-        Random r = new Random();
-        byte[] buffer = new byte[16];
-        r.nextBytes(buffer);
-        return new BytesToken(buffer);
-    }
-
-    private final Token.TokenFactory tokenFactory = new Token.TokenFactory() {
-        public ByteBuffer toByteArray(Token token)
-        {
-            BytesToken bytesToken = (BytesToken) token;
-            return ByteBuffer.wrap(bytesToken.token);
-        }
-
-        public Token fromByteArray(ByteBuffer bytes)
-        {
-            return new BytesToken(bytes);
-        }
-
-        public String toString(Token token)
-        {
-            BytesToken bytesToken = (BytesToken) token;
-            return Hex.bytesToHex(bytesToken.token);
-        }
-
-        public void validate(String token) throws ConfigurationException
-        {
-            try
-            {
-                if (token.length() % 2 == 1)
-                    token = "0" + token;
-                Hex.hexToBytes(token);
-            }
-            catch (NumberFormatException e)
-            {
-                throw new ConfigurationException("Token " + token + " contains non-hex digits");
-            }
-        }
-
-        public Token fromString(String string)
-        {
-            if (string.length() % 2 == 1)
-                string = "0" + string;
-            return new BytesToken(Hex.hexToBytes(string));
-        }
-    };
-
-    public Token.TokenFactory getTokenFactory()
-    {
-        return tokenFactory;
-    }
-
-    public boolean preservesOrder()
-    {
-        return true;
-    }
-
-    public abstract BytesToken getToken(ByteBuffer key);
-
-    public Map<Token, Float> describeOwnership(List<Token> sortedTokens)
-    {
-        // allTokens will contain the count and be returned, sorted_ranges is shorthand for token<->token math.
-        Map<Token, Float> allTokens = new HashMap<Token, Float>();
-        List<Range<Token>> sortedRanges = new ArrayList<Range<Token>>(sortedTokens.size());
-
-        // this initializes the counts to 0 and calcs the ranges in order.
-        Token lastToken = sortedTokens.get(sortedTokens.size() - 1);
-        for (Token node : sortedTokens)
-        {
-            allTokens.put(node, new Float(0.0));
-            sortedRanges.add(new Range<Token>(lastToken, node));
-            lastToken = node;
-        }
-
-        for (String ks : Schema.instance.getKeyspaces())
-        {
-            for (CFMetaData cfmd : Schema.instance.getKSMetaData(ks).cfMetaData().values())
-            {
-                for (Range<Token> r : sortedRanges)
-                {
-                    // Looping over every KS:CF:Range, get the splits size and add it to the count
-                    allTokens.put(r.right, allTokens.get(r.right) + StorageService.instance.getSplits(ks, cfmd.cfName, r, 1).size());
-                }
-            }
-        }
-
-        // Sum every count up and divide count/total for the fractional ownership.
-        Float total = new Float(0.0);
-        for (Float f : allTokens.values())
-            total += f;
-        for (Map.Entry<Token, Float> row : allTokens.entrySet())
-            allTokens.put(row.getKey(), row.getValue() / total);
-
-        return allTokens;
-    }
-
-    public AbstractType<?> getTokenValidator()
-    {
-        return BytesType.instance;
-    }
-}
diff --git a/src/java/org/apache/cassandra/dht/AbstractPartitioner.java b/src/java/org/apache/cassandra/dht/AbstractPartitioner.java
deleted file mode 100644
index ed077c9..0000000
--- a/src/java/org/apache/cassandra/dht/AbstractPartitioner.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * 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.dht;
-
-abstract class AbstractPartitioner implements IPartitioner
-{
-    @SuppressWarnings("unchecked")
-    public <R extends RingPosition<R>> R minValue(Class<R> klass)
-    {
-        Token minToken = getMinimumToken();
-        if (minToken.getClass().equals(klass))
-            return (R)minToken;
-        else
-            return (R)minToken.minKeyBound();
-    }
-}
diff --git a/src/java/org/apache/cassandra/dht/BigIntegerToken.java b/src/java/org/apache/cassandra/dht/BigIntegerToken.java
deleted file mode 100644
index f050519..0000000
--- a/src/java/org/apache/cassandra/dht/BigIntegerToken.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * 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.dht;
-
-import java.math.BigInteger;
-
-public class BigIntegerToken extends ComparableObjectToken<BigInteger>
-{
-    static final long serialVersionUID = -5833589141319293006L;
-
-    public BigIntegerToken(BigInteger token)
-    {
-        super(token);
-    }
-
-    // convenience method for testing
-    public BigIntegerToken(String token) {
-        this(new BigInteger(token));
-    }
-}
diff --git a/src/java/org/apache/cassandra/dht/BootStrapper.java b/src/java/org/apache/cassandra/dht/BootStrapper.java
index dfefbe9..26fa6b3 100644
--- a/src/java/org/apache/cassandra/dht/BootStrapper.java
+++ b/src/java/org/apache/cassandra/dht/BootStrapper.java
@@ -21,8 +21,9 @@
 import java.io.IOException;
 import java.net.InetAddress;
 import java.util.*;
-import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicInteger;
 
+import com.google.common.util.concurrent.ListenableFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -37,8 +38,12 @@
 import org.apache.cassandra.locator.AbstractReplicationStrategy;
 import org.apache.cassandra.locator.TokenMetadata;
 import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.streaming.*;
+import org.apache.cassandra.utils.progress.ProgressEvent;
+import org.apache.cassandra.utils.progress.ProgressEventNotifierSupport;
+import org.apache.cassandra.utils.progress.ProgressEventType;
 
-public class BootStrapper
+public class BootStrapper extends ProgressEventNotifierSupport
 {
     private static final Logger logger = LoggerFactory.getLogger(BootStrapper.class);
 
@@ -55,15 +60,20 @@
 
         this.address = address;
         this.tokens = tokens;
-        tokenMetadata = tmd;
+        this.tokenMetadata = tmd;
     }
 
-    public void bootstrap()
+    public ListenableFuture<StreamState> bootstrap(StreamStateStore stateStore, boolean useStrictConsistency)
     {
-        if (logger.isDebugEnabled())
-            logger.debug("Beginning bootstrap process");
+        logger.trace("Beginning bootstrap process");
 
-        RangeStreamer streamer = new RangeStreamer(tokenMetadata, tokens, address, "Bootstrap");
+        RangeStreamer streamer = new RangeStreamer(tokenMetadata,
+                                                   tokens,
+                                                   address,
+                                                   "Bootstrap",
+                                                   useStrictConsistency,
+                                                   DatabaseDescriptor.getEndpointSnitch(),
+                                                   stateStore);
         streamer.addSourceFilter(new RangeStreamer.FailureDetectorSourceFilter(FailureDetector.instance));
         streamer.addSourceFilter(new RangeStreamer.ExcludeLocalNodeFilter());
 
@@ -73,19 +83,70 @@
             streamer.addRanges(keyspaceName, strategy.getPendingAddressRanges(tokenMetadata, tokens, address));
         }
 
-        try
+        StreamResultFuture bootstrapStreamResult = streamer.fetchAsync();
+        bootstrapStreamResult.addEventListener(new StreamEventHandler()
         {
-            streamer.fetchAsync().get();
-            StorageService.instance.finishBootstrapping();
-        }
-        catch (InterruptedException e)
-        {
-            throw new RuntimeException("Interrupted while waiting on boostrap to complete. Bootstrap will have to be restarted.");
-        }
-        catch (ExecutionException e)
-        {
-            throw new RuntimeException("Error during boostrap: " + e.getCause().getMessage(), e.getCause());
-        }
+            private final AtomicInteger receivedFiles = new AtomicInteger();
+            private final AtomicInteger totalFilesToReceive = new AtomicInteger();
+
+            @Override
+            public void handleStreamEvent(StreamEvent event)
+            {
+                switch (event.eventType)
+                {
+                    case STREAM_PREPARED:
+                        StreamEvent.SessionPreparedEvent prepared = (StreamEvent.SessionPreparedEvent) event;
+                        int currentTotal = totalFilesToReceive.addAndGet((int) prepared.session.getTotalFilesToReceive());
+                        ProgressEvent prepareProgress = new ProgressEvent(ProgressEventType.PROGRESS, receivedFiles.get(), currentTotal, "prepare with " + prepared.session.peer + " complete");
+                        fireProgressEvent("bootstrap", prepareProgress);
+                        break;
+
+                    case FILE_PROGRESS:
+                        StreamEvent.ProgressEvent progress = (StreamEvent.ProgressEvent) event;
+                        if (progress.progress.isCompleted())
+                        {
+                            int received = receivedFiles.incrementAndGet();
+                            ProgressEvent currentProgress = new ProgressEvent(ProgressEventType.PROGRESS, received, totalFilesToReceive.get(), "received file " + progress.progress.fileName);
+                            fireProgressEvent("bootstrap", currentProgress);
+                        }
+                        break;
+
+                    case STREAM_COMPLETE:
+                        StreamEvent.SessionCompleteEvent completeEvent = (StreamEvent.SessionCompleteEvent) event;
+                        ProgressEvent completeProgress = new ProgressEvent(ProgressEventType.PROGRESS, receivedFiles.get(), totalFilesToReceive.get(), "session with " + completeEvent.peer + " complete");
+                        fireProgressEvent("bootstrap", completeProgress);
+                        break;
+                }
+            }
+
+            @Override
+            public void onSuccess(StreamState streamState)
+            {
+                ProgressEventType type;
+                String message;
+
+                if (streamState.hasFailedSession())
+                {
+                    type = ProgressEventType.ERROR;
+                    message = "Some bootstrap stream failed";
+                }
+                else
+                {
+                    type = ProgressEventType.SUCCESS;
+                    message = "Bootstrap streaming success";
+                }
+                ProgressEvent currentProgress = new ProgressEvent(type, receivedFiles.get(), totalFilesToReceive.get(), message);
+                fireProgressEvent("bootstrap", currentProgress);
+            }
+
+            @Override
+            public void onFailure(Throwable throwable)
+            {
+                ProgressEvent currentProgress = new ProgressEvent(ProgressEventType.ERROR, receivedFiles.get(), totalFilesToReceive.get(), throwable.getMessage());
+                fireProgressEvent("bootstrap", currentProgress);
+            }
+        });
+        return bootstrapStreamResult;
     }
 
     /**
@@ -99,8 +160,8 @@
         // if user specified tokens, use those
         if (initialTokens.size() > 0)
         {
-            logger.debug("tokens manually specified as {}",  initialTokens);
-            List<Token> tokens = new ArrayList<Token>(initialTokens.size());
+            logger.trace("tokens manually specified as {}",  initialTokens);
+            List<Token> tokens = new ArrayList<>(initialTokens.size());
             for (String tokenString : initialTokens)
             {
                 Token token = StorageService.getPartitioner().getTokenFactory().fromString(tokenString);
@@ -123,7 +184,7 @@
 
     public static Collection<Token> getRandomTokens(TokenMetadata metadata, int numTokens)
     {
-        Set<Token> tokens = new HashSet<Token>(numTokens);
+        Set<Token> tokens = new HashSet<>(numTokens);
         while (tokens.size() < numTokens)
         {
             Token token = StorageService.getPartitioner().getRandomToken();
diff --git a/src/java/org/apache/cassandra/dht/Bounds.java b/src/java/org/apache/cassandra/dht/Bounds.java
index 5ffde42..73414cd 100644
--- a/src/java/org/apache/cassandra/dht/Bounds.java
+++ b/src/java/org/apache/cassandra/dht/Bounds.java
@@ -30,7 +30,6 @@
 import com.google.common.collect.Sets;
 
 import org.apache.cassandra.db.RowPosition;
-import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.Pair;
 
 /**
@@ -40,14 +39,9 @@
 {
     public Bounds(T left, T right)
     {
-        this(left, right, StorageService.getPartitioner());
-    }
-
-    public Bounds(T left, T right, IPartitioner partitioner)
-    {
-        super(left, right, partitioner);
+        super(left, right);
         // unlike a Range, a Bounds may not wrap
-        assert left.compareTo(right) <= 0 || right.isMinimum(partitioner) : "[" + left + "," + right + "]";
+        assert !strictlyWrapsAround(left, right) : "[" + left + "," + right + "]";
     }
 
     public boolean contains(T position)
@@ -55,7 +49,7 @@
         // Range.contains doesnt work correctly if left == right (unless both
         // are minimum) because for Range that means a wrapping range that select
         // the whole ring. So we must explicitely handle this case
-        return left.equals(position) || ((right.isMinimum(partitioner) || !left.equals(right)) && Range.contains(left, right, position));
+        return left.equals(position) || ((right.isMinimum() || !left.equals(right)) && Range.contains(left, right, position));
     }
 
     public Pair<AbstractBounds<T>, AbstractBounds<T>> split(T position)
@@ -65,8 +59,8 @@
         if (position.equals(right))
             return null;
 
-        AbstractBounds<T> lb = new Bounds<T>(left, position, partitioner);
-        AbstractBounds<T> rb = new Range<T>(position, right, partitioner);
+        AbstractBounds<T> lb = new Bounds<T>(left, position);
+        AbstractBounds<T> rb = new Range<T>(position, right);
         return Pair.create(lb, rb);
     }
 
@@ -134,21 +128,9 @@
     /**
      * Compute a bounds of keys corresponding to a given bounds of token.
      */
-    public static Bounds<RowPosition> makeRowBounds(Token left, Token right, IPartitioner partitioner)
+    public static Bounds<RowPosition> makeRowBounds(Token left, Token right)
     {
-        return new Bounds<RowPosition>(left.minKeyBound(partitioner), right.maxKeyBound(partitioner), partitioner);
-    }
-
-    @SuppressWarnings("unchecked")
-    public AbstractBounds<RowPosition> toRowBounds()
-    {
-        return (left instanceof Token) ? makeRowBounds((Token)left, (Token)right, partitioner) : (Bounds<RowPosition>)this;
-    }
-
-    @SuppressWarnings("unchecked")
-    public AbstractBounds<Token> toTokenBounds()
-    {
-        return (left instanceof RowPosition) ? new Bounds<Token>(((RowPosition)left).getToken(), ((RowPosition)right).getToken(), partitioner) : (Bounds<Token>)this;
+        return new Bounds<RowPosition>(left.minKeyBound(), right.maxKeyBound());
     }
 
     public AbstractBounds<T> withNewRight(T newRight)
diff --git a/src/java/org/apache/cassandra/dht/ByteOrderedPartitioner.java b/src/java/org/apache/cassandra/dht/ByteOrderedPartitioner.java
index 888ef35..297e5a6 100644
--- a/src/java/org/apache/cassandra/dht/ByteOrderedPartitioner.java
+++ b/src/java/org/apache/cassandra/dht/ByteOrderedPartitioner.java
@@ -17,14 +17,107 @@
  */
 package org.apache.cassandra.dht;
 
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.db.BufferDecoratedKey;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Hex;
 import org.apache.cassandra.utils.ObjectSizes;
+import org.apache.cassandra.utils.Pair;
 
+import org.apache.commons.lang3.ArrayUtils;
+
+import java.math.BigInteger;
 import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
 
-public class ByteOrderedPartitioner extends AbstractByteOrderedPartitioner
+public class ByteOrderedPartitioner implements IPartitioner
 {
+    public static final BytesToken MINIMUM = new BytesToken(ArrayUtils.EMPTY_BYTE_ARRAY);
+
+    public static final BigInteger BYTE_MASK = new BigInteger("255");
+
     private static final long EMPTY_SIZE = ObjectSizes.measure(MINIMUM);
 
+    public static final ByteOrderedPartitioner instance = new ByteOrderedPartitioner();
+
+    public static class BytesToken extends Token
+    {
+        static final long serialVersionUID = -2630749093733680626L;
+
+        final byte[] token;
+
+        public BytesToken(ByteBuffer token)
+        {
+            this(ByteBufferUtil.getArray(token));
+        }
+
+        public BytesToken(byte[] token)
+        {
+            this.token = token;
+        }
+
+        @Override
+        public String toString()
+        {
+            return Hex.bytesToHex(token);
+        }
+
+        public int compareTo(Token other)
+        {
+            BytesToken o = (BytesToken) other;
+            return FBUtilities.compareUnsigned(token, o.token, 0, 0, token.length, o.token.length);
+        }
+
+        @Override
+        public int hashCode()
+        {
+            final int prime = 31;
+            return prime + Arrays.hashCode(token);
+        }
+
+        @Override
+        public boolean equals(Object obj)
+        {
+            if (this == obj)
+                return true;
+            if (!(obj instanceof BytesToken))
+                return false;
+            BytesToken other = (BytesToken) obj;
+
+            return Arrays.equals(token, other.token);
+        }
+
+        @Override
+        public IPartitioner getPartitioner()
+        {
+            return instance;
+        }
+
+        @Override
+        public long getHeapSize()
+        {
+            return EMPTY_SIZE + ObjectSizes.sizeOfArray(token);
+        }
+
+        @Override
+        public Object getTokenValue()
+        {
+            return token;
+        }
+    }
+
     public BytesToken getToken(ByteBuffer key)
     {
         if (key.remaining() == 0)
@@ -32,9 +125,165 @@
         return new BytesToken(key);
     }
 
-    @Override
-    public long getHeapSizeOf(Token token)
+    public DecoratedKey decorateKey(ByteBuffer key)
     {
-        return EMPTY_SIZE + ObjectSizes.sizeOfArray(((BytesToken) token).token);
+        return new BufferDecoratedKey(getToken(key), key);
+    }
+
+    public BytesToken midpoint(Token lt, Token rt)
+    {
+        BytesToken ltoken = (BytesToken) lt;
+        BytesToken rtoken = (BytesToken) rt;
+
+        int sigbytes = Math.max(ltoken.token.length, rtoken.token.length);
+        BigInteger left = bigForBytes(ltoken.token, sigbytes);
+        BigInteger right = bigForBytes(rtoken.token, sigbytes);
+
+        Pair<BigInteger,Boolean> midpair = FBUtilities.midpoint(left, right, 8*sigbytes);
+        return new BytesToken(bytesForBig(midpair.left, sigbytes, midpair.right));
+    }
+
+    /**
+     * Convert a byte array containing the most significant of 'sigbytes' bytes
+     * representing a big-endian magnitude into a BigInteger.
+     */
+    private BigInteger bigForBytes(byte[] bytes, int sigbytes)
+    {
+        byte[] b;
+        if (sigbytes != bytes.length)
+        {
+            b = new byte[sigbytes];
+            System.arraycopy(bytes, 0, b, 0, bytes.length);
+        } else
+            b = bytes;
+        return new BigInteger(1, b);
+    }
+
+    /**
+     * Convert a (positive) BigInteger into a byte array representing its magnitude.
+     * If remainder is true, an additional byte with the high order bit enabled
+     * will be added to the end of the array
+     */
+    private byte[] bytesForBig(BigInteger big, int sigbytes, boolean remainder)
+    {
+        byte[] bytes = new byte[sigbytes + (remainder ? 1 : 0)];
+        if (remainder)
+        {
+            // remaining bit is the most significant in the last byte
+            bytes[sigbytes] |= 0x80;
+        }
+        // bitmask for a single byte
+        for (int i = 0; i < sigbytes; i++)
+        {
+            int maskpos = 8 * (sigbytes - (i + 1));
+            // apply bitmask and get byte value
+            bytes[i] = (byte)(big.and(BYTE_MASK.shiftLeft(maskpos)).shiftRight(maskpos).intValue() & 0xFF);
+        }
+        return bytes;
+    }
+
+    public BytesToken getMinimumToken()
+    {
+        return MINIMUM;
+    }
+
+    public BytesToken getRandomToken()
+    {
+        Random r = new Random();
+        byte[] buffer = new byte[16];
+        r.nextBytes(buffer);
+        return new BytesToken(buffer);
+    }
+
+    private final Token.TokenFactory tokenFactory = new Token.TokenFactory() {
+        public ByteBuffer toByteArray(Token token)
+        {
+            BytesToken bytesToken = (BytesToken) token;
+            return ByteBuffer.wrap(bytesToken.token);
+        }
+
+        public Token fromByteArray(ByteBuffer bytes)
+        {
+            return new BytesToken(bytes);
+        }
+
+        public String toString(Token token)
+        {
+            BytesToken bytesToken = (BytesToken) token;
+            return Hex.bytesToHex(bytesToken.token);
+        }
+
+        public void validate(String token) throws ConfigurationException
+        {
+            try
+            {
+                if (token.length() % 2 == 1)
+                    token = "0" + token;
+                Hex.hexToBytes(token);
+            }
+            catch (NumberFormatException e)
+            {
+                throw new ConfigurationException("Token " + token + " contains non-hex digits");
+            }
+        }
+
+        public Token fromString(String string)
+        {
+            if (string.length() % 2 == 1)
+                string = "0" + string;
+            return new BytesToken(Hex.hexToBytes(string));
+        }
+    };
+
+    public Token.TokenFactory getTokenFactory()
+    {
+        return tokenFactory;
+    }
+
+    public boolean preservesOrder()
+    {
+        return true;
+    }
+
+    public Map<Token, Float> describeOwnership(List<Token> sortedTokens)
+    {
+        // allTokens will contain the count and be returned, sorted_ranges is shorthand for token<->token math.
+        Map<Token, Float> allTokens = new HashMap<Token, Float>();
+        List<Range<Token>> sortedRanges = new ArrayList<Range<Token>>(sortedTokens.size());
+
+        // this initializes the counts to 0 and calcs the ranges in order.
+        Token lastToken = sortedTokens.get(sortedTokens.size() - 1);
+        for (Token node : sortedTokens)
+        {
+            allTokens.put(node, new Float(0.0));
+            sortedRanges.add(new Range<Token>(lastToken, node));
+            lastToken = node;
+        }
+
+        for (String ks : Schema.instance.getKeyspaces())
+        {
+            for (CFMetaData cfmd : Schema.instance.getKSMetaData(ks).cfMetaData().values())
+            {
+                for (Range<Token> r : sortedRanges)
+                {
+                    // Looping over every KS:CF:Range, get the splits size and add it to the count
+                    allTokens.put(r.right, allTokens.get(r.right) + StorageService.instance.getSplits(ks, cfmd.cfName, r, 1).size());
+                }
+            }
+        }
+
+        // Sum every count up and divide count/total for the fractional ownership.
+        Float total = new Float(0.0);
+        for (Float f : allTokens.values())
+            total += f;
+        for (Map.Entry<Token, Float> row : allTokens.entrySet())
+            allTokens.put(row.getKey(), row.getValue() / total);
+
+        return allTokens;
+    }
+
+    public AbstractType<?> getTokenValidator()
+    {
+        return BytesType.instance;
     }
 }
diff --git a/src/java/org/apache/cassandra/dht/BytesToken.java b/src/java/org/apache/cassandra/dht/BytesToken.java
deleted file mode 100644
index ad1fd10..0000000
--- a/src/java/org/apache/cassandra/dht/BytesToken.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * 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.dht;
-
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.Hex;
-
-public class BytesToken extends Token
-{
-    static final long serialVersionUID = -2630749093733680626L;
-
-    final byte[] token;
-
-    public BytesToken(ByteBuffer token)
-    {
-        this(ByteBufferUtil.getArray(token));
-    }
-
-    public BytesToken(byte[] token)
-    {
-        this.token = token;
-    }
-
-    @Override
-    public String toString()
-    {
-        return Hex.bytesToHex(token);
-    }
-
-    public int compareTo(Token other)
-    {
-        BytesToken o = (BytesToken) other;
-        return FBUtilities.compareUnsigned(token, o.token, 0, 0, token.length, o.token.length);
-    }
-
-
-    @Override
-    public int hashCode()
-    {
-        final int prime = 31;
-        return prime + Arrays.hashCode(token);
-    }
-
-    @Override
-    public boolean equals(Object obj)
-    {
-        if (this == obj)
-            return true;
-        if (!(obj instanceof BytesToken))
-            return false;
-        BytesToken other = (BytesToken) obj;
-
-        return Arrays.equals(token, other.token);
-    }
-
-    @Override
-    public byte[] getTokenValue()
-    {
-        return token;
-    }
-}
diff --git a/src/java/org/apache/cassandra/dht/ExcludingBounds.java b/src/java/org/apache/cassandra/dht/ExcludingBounds.java
index 0d09f08..7319356 100644
--- a/src/java/org/apache/cassandra/dht/ExcludingBounds.java
+++ b/src/java/org/apache/cassandra/dht/ExcludingBounds.java
@@ -20,8 +20,6 @@
 import java.util.Collections;
 import java.util.List;
 
-import org.apache.cassandra.db.RowPosition;
-import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.Pair;
 
 /**
@@ -31,14 +29,9 @@
 {
     public ExcludingBounds(T left, T right)
     {
-        this(left, right, StorageService.getPartitioner());
-    }
-
-    ExcludingBounds(T left, T right, IPartitioner partitioner)
-    {
-        super(left, right, partitioner);
+        super(left, right);
         // unlike a Range, an ExcludingBounds may not wrap, nor be empty
-        assert left.compareTo(right) < 0 || right.isMinimum(partitioner) : "(" + left + "," + right + ")";
+        assert !strictlyWrapsAround(left, right) && (right.isMinimum() || left.compareTo(right) != 0) : "(" + left + "," + right + ")";
     }
 
     public boolean contains(T position)
@@ -51,8 +44,8 @@
         assert contains(position) || left.equals(position);
         if (left.equals(position))
             return null;
-        AbstractBounds<T> lb = new Range<T>(left, position, partitioner);
-        AbstractBounds<T> rb = new ExcludingBounds<T>(position, right, partitioner);
+        AbstractBounds<T> lb = new Range<T>(left, position);
+        AbstractBounds<T> rb = new ExcludingBounds<T>(position, right);
         return Pair.create(lb, rb);
     }
 
@@ -97,26 +90,6 @@
         return ")";
     }
 
-    /**
-     * Compute a bounds of keys corresponding to a given bounds of token.
-     */
-    private static ExcludingBounds<RowPosition> makeRowBounds(Token left, Token right, IPartitioner partitioner)
-    {
-        return new ExcludingBounds<RowPosition>(left.maxKeyBound(partitioner), right.minKeyBound(partitioner), partitioner);
-    }
-
-    @SuppressWarnings("unchecked")
-    public AbstractBounds<RowPosition> toRowBounds()
-    {
-        return (left instanceof Token) ? makeRowBounds((Token)left, (Token)right, partitioner) : (ExcludingBounds<RowPosition>)this;
-    }
-
-    @SuppressWarnings("unchecked")
-    public AbstractBounds<Token> toTokenBounds()
-    {
-        return (left instanceof RowPosition) ? new ExcludingBounds<Token>(((RowPosition)left).getToken(), ((RowPosition)right).getToken(), partitioner) : (ExcludingBounds<Token>)this;
-    }
-
     public AbstractBounds<T> withNewRight(T newRight)
     {
         return new ExcludingBounds<T>(left, newRight);
diff --git a/src/java/org/apache/cassandra/dht/IPartitioner.java b/src/java/org/apache/cassandra/dht/IPartitioner.java
index 6451535..b22da66 100644
--- a/src/java/org/apache/cassandra/dht/IPartitioner.java
+++ b/src/java/org/apache/cassandra/dht/IPartitioner.java
@@ -56,13 +56,6 @@
     public Token getToken(ByteBuffer key);
 
     /**
-     *
-     * @param token
-     * @return the on-heap memory used by the provided token
-     */
-    public long getHeapSizeOf(Token token);
-
-    /**
      * @return a randomly generated token
      */
     public Token getRandomToken();
@@ -85,6 +78,4 @@
     public Map<Token, Float> describeOwnership(List<Token> sortedTokens);
 
     public AbstractType<?> getTokenValidator();
-
-    public <R extends RingPosition<R>> R minValue(Class<R> klass);
 }
diff --git a/src/java/org/apache/cassandra/dht/IPartitionerDependentSerializer.java b/src/java/org/apache/cassandra/dht/IPartitionerDependentSerializer.java
new file mode 100644
index 0000000..3a9a768
--- /dev/null
+++ b/src/java/org/apache/cassandra/dht/IPartitionerDependentSerializer.java
@@ -0,0 +1,61 @@
+/*
+ * 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.dht;
+
+import java.io.DataInput;
+import java.io.IOException;
+
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * Versioned serializer where the serialization depends on partitioner.
+ *
+ * On serialization the partitioner is given by the entity being serialized. To deserialize the partitioner used must
+ * be known to the calling method.
+ */
+public interface IPartitionerDependentSerializer<T>
+{
+    /**
+     * Serialize the specified type into the specified DataOutputStream instance.
+     *
+     * @param t type that needs to be serialized
+     * @param out DataOutput into which serialization needs to happen.
+     * @param version protocol version
+     * @throws java.io.IOException if serialization fails
+     */
+    public void serialize(T t, DataOutputPlus out, int version) throws IOException;
+
+    /**
+     * Deserialize into the specified DataInputStream instance.
+     * @param in DataInput from which deserialization needs to happen.
+     * @param p Partitioner that will be used to construct tokens. Needs to match the partitioner that was used to
+     *     serialize the token.
+     * @param version protocol version
+     * @return the type that was deserialized
+     * @throws IOException if deserialization fails
+     */
+    public T deserialize(DataInput in, IPartitioner p, int version) throws IOException;
+
+    /**
+     * Calculate serialized size of object without actually serializing.
+     * @param t object to calculate serialized size
+     * @param version protocol version
+     * @return serialized size of object t
+     */
+    public long serializedSize(T t, int version);
+}
diff --git a/src/java/org/apache/cassandra/dht/IncludingExcludingBounds.java b/src/java/org/apache/cassandra/dht/IncludingExcludingBounds.java
index 278a806..abcf87b 100644
--- a/src/java/org/apache/cassandra/dht/IncludingExcludingBounds.java
+++ b/src/java/org/apache/cassandra/dht/IncludingExcludingBounds.java
@@ -20,8 +20,6 @@
 import java.util.Collections;
 import java.util.List;
 
-import org.apache.cassandra.db.RowPosition;
-import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.Pair;
 
 /**
@@ -31,15 +29,10 @@
 {
     public IncludingExcludingBounds(T left, T right)
     {
-        this(left, right, StorageService.getPartitioner());
-    }
-
-    IncludingExcludingBounds(T left, T right, IPartitioner partitioner)
-    {
-        super(left, right, partitioner);
+        super(left, right);
         // unlike a Range, an IncludingExcludingBounds may not wrap, nor have
         // right == left unless the right is the min token
-        assert left.compareTo(right) < 0 || right.isMinimum(partitioner) : "[" + left + "," + right + ")";
+        assert !strictlyWrapsAround(left, right) && (right.isMinimum() || left.compareTo(right) != 0) : "(" + left + "," + right + ")";
     }
 
     public boolean contains(T position)
@@ -50,8 +43,8 @@
     public Pair<AbstractBounds<T>, AbstractBounds<T>> split(T position)
     {
         assert contains(position);
-        AbstractBounds<T> lb = new Bounds<T>(left, position, partitioner);
-        AbstractBounds<T> rb = new ExcludingBounds<T>(position, right, partitioner);
+        AbstractBounds<T> lb = new Bounds<T>(left, position);
+        AbstractBounds<T> rb = new ExcludingBounds<T>(position, right);
         return Pair.create(lb, rb);
     }
 
@@ -96,26 +89,6 @@
         return ")";
     }
 
-    /**
-     * Compute a bounds of keys corresponding to a given bounds of token.
-     */
-    private static IncludingExcludingBounds<RowPosition> makeRowBounds(Token left, Token right, IPartitioner partitioner)
-    {
-        return new IncludingExcludingBounds<RowPosition>(left.maxKeyBound(partitioner), right.minKeyBound(partitioner), partitioner);
-    }
-
-    @SuppressWarnings("unchecked")
-    public AbstractBounds<RowPosition> toRowBounds()
-    {
-        return (left instanceof Token) ? makeRowBounds((Token)left, (Token)right, partitioner) : (IncludingExcludingBounds<RowPosition>)this;
-    }
-
-    @SuppressWarnings("unchecked")
-    public AbstractBounds<Token> toTokenBounds()
-    {
-        return (left instanceof RowPosition) ? new IncludingExcludingBounds<Token>(((RowPosition)left).getToken(), ((RowPosition)right).getToken(), partitioner) : (IncludingExcludingBounds<Token>)this;
-    }
-
     public AbstractBounds<T> withNewRight(T newRight)
     {
         return new IncludingExcludingBounds<T>(left, newRight);
diff --git a/src/java/org/apache/cassandra/dht/LocalPartitioner.java b/src/java/org/apache/cassandra/dht/LocalPartitioner.java
index 1cbeba4..01dc75e 100644
--- a/src/java/org/apache/cassandra/dht/LocalPartitioner.java
+++ b/src/java/org/apache/cassandra/dht/LocalPartitioner.java
@@ -22,17 +22,17 @@
 import java.util.List;
 import java.util.Map;
 
-import org.apache.cassandra.db.BufferDecoratedKey;
 import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.CachedHashDecoratedKey;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.ObjectSizes;
 
-public class LocalPartitioner extends AbstractPartitioner
+public class LocalPartitioner implements IPartitioner
 {
-    private static final long EMPTY_SIZE = ObjectSizes.measure(new LocalToken(null, null));
+    private static final long EMPTY_SIZE = ObjectSizes.measure(new LocalPartitioner(null).new LocalToken(null));
 
-    private final AbstractType<?> comparator;
+    final AbstractType<?> comparator;   // package-private to avoid access workarounds in embedded LocalToken.
 
     public LocalPartitioner(AbstractType<?> comparator)
     {
@@ -41,7 +41,7 @@
 
     public DecoratedKey decorateKey(ByteBuffer key)
     {
-        return new BufferDecoratedKey(getToken(key), key);
+        return new CachedHashDecoratedKey(getToken(key), key);
     }
 
     public Token midpoint(Token left, Token right)
@@ -51,17 +51,12 @@
 
     public LocalToken getMinimumToken()
     {
-        return new LocalToken(comparator, ByteBufferUtil.EMPTY_BYTE_BUFFER);
+        return new LocalToken(ByteBufferUtil.EMPTY_BYTE_BUFFER);
     }
 
     public LocalToken getToken(ByteBuffer key)
     {
-        return new LocalToken(comparator, key);
-    }
-
-    public long getHeapSizeOf(Token token)
-    {
-        return EMPTY_SIZE + ObjectSizes.sizeOnHeapOf(((LocalToken) token).token);
+        return new LocalToken(key);
     }
 
     public LocalToken getRandomToken()
@@ -88,4 +83,57 @@
     {
         return comparator;
     }
+
+    public class LocalToken extends ComparableObjectToken<ByteBuffer>
+    {
+        static final long serialVersionUID = 8437543776403014875L;
+
+        public LocalToken(ByteBuffer token)
+        {
+            super(token);
+        }
+
+        @Override
+        public String toString()
+        {
+            return comparator.getString(token);
+        }
+
+        @Override
+        public int compareTo(Token o)
+        {
+            assert getPartitioner() == o.getPartitioner();
+            return comparator.compare(token, ((LocalToken) o).token);
+        }
+
+        @Override
+        public int hashCode()
+        {
+            final int prime = 31;
+            return prime + token.hashCode();
+        }
+
+        @Override
+        public boolean equals(Object obj)
+        {
+            if (this == obj)
+                return true;
+            if (!(obj instanceof LocalToken))
+                return false;
+            LocalToken other = (LocalToken) obj;
+            return token.equals(other.token);
+        }
+
+        @Override
+        public IPartitioner getPartitioner()
+        {
+            return LocalPartitioner.this;
+        }
+
+        @Override
+        public long getHeapSize()
+        {
+            return EMPTY_SIZE + ObjectSizes.sizeOnHeapOf(token);
+        }
+    }
 }
diff --git a/src/java/org/apache/cassandra/dht/LocalToken.java b/src/java/org/apache/cassandra/dht/LocalToken.java
deleted file mode 100644
index 29e98eb..0000000
--- a/src/java/org/apache/cassandra/dht/LocalToken.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * 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.dht;
-
-import java.nio.ByteBuffer;
-
-import org.apache.cassandra.db.marshal.AbstractType;
-
-public class LocalToken extends ComparableObjectToken<ByteBuffer>
-{
-    static final long serialVersionUID = 8437543776403014875L;
-
-    private final AbstractType<?> comparator;
-
-    public LocalToken(AbstractType<?> comparator, ByteBuffer token)
-    {
-        super(token);
-        this.comparator = comparator;
-    }
-
-    @Override
-    public String toString()
-    {
-        return comparator.getString(token);
-    }
-
-    public int compareTo(Token o)
-    {
-        return comparator.compare(token, ((LocalToken) o).token);
-    }
-}
diff --git a/src/java/org/apache/cassandra/dht/LongToken.java b/src/java/org/apache/cassandra/dht/LongToken.java
deleted file mode 100644
index 8d60653..0000000
--- a/src/java/org/apache/cassandra/dht/LongToken.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/**
- * 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.dht;
-
-import com.google.common.primitives.Longs;
-
-public class LongToken extends Token
-{
-    static final long serialVersionUID = -5833580143318243006L;
-
-    final long token;
-
-    public LongToken(long token)
-    {
-        this.token = token;
-    }
-
-    public String toString()
-    {
-        return Long.toString(token);
-    }
-
-    public boolean equals(Object obj)
-    {
-        if (this == obj)
-            return true;
-        if (obj == null || this.getClass() != obj.getClass())
-            return false;
-
-        return token == (((LongToken)obj).token);
-    }
-
-    public int hashCode()
-    {
-        return Longs.hashCode(token);
-    }
-
-    public int compareTo(Token o)
-    {
-        return Long.compare(token, ((LongToken) o).token);
-    }
-
-    public Long getTokenValue()
-    {
-        return token;
-    }
-}
diff --git a/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java b/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java
index cf91c86..96c603e 100644
--- a/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java
+++ b/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java
@@ -26,8 +26,8 @@
 import java.util.Map;
 import java.util.concurrent.ThreadLocalRandom;
 
-import org.apache.cassandra.db.BufferDecoratedKey;
 import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PreHashedDecoratedKey;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.LongType;
 import org.apache.cassandra.exceptions.ConfigurationException;
@@ -35,19 +35,24 @@
 import org.apache.cassandra.utils.MurmurHash;
 import org.apache.cassandra.utils.ObjectSizes;
 
+import com.google.common.primitives.Longs;
+
 /**
  * This class generates a BigIntegerToken using a Murmur3 hash.
  */
-public class Murmur3Partitioner extends AbstractPartitioner
+public class Murmur3Partitioner implements IPartitioner
 {
     public static final LongToken MINIMUM = new LongToken(Long.MIN_VALUE);
     public static final long MAXIMUM = Long.MAX_VALUE;
 
     private static final int HEAP_SIZE = (int) ObjectSizes.measureDeep(MINIMUM);
 
+    public static final Murmur3Partitioner instance = new Murmur3Partitioner();
+
     public DecoratedKey decorateKey(ByteBuffer key)
     {
-        return new BufferDecoratedKey(getToken(key), key);
+        long[] hash = getHash(key);
+        return new PreHashedDecoratedKey(getToken(key, hash), key, hash[0], hash[1]);
     }
 
     public Token midpoint(Token lToken, Token rToken)
@@ -82,6 +87,61 @@
         return MINIMUM;
     }
 
+    public static class LongToken extends Token
+    {
+        static final long serialVersionUID = -5833580143318243006L;
+
+        final long token;
+
+        public LongToken(long token)
+        {
+            this.token = token;
+        }
+
+        public String toString()
+        {
+            return Long.toString(token);
+        }
+
+        public boolean equals(Object obj)
+        {
+            if (this == obj)
+                return true;
+            if (obj == null || this.getClass() != obj.getClass())
+                return false;
+
+            return token == (((LongToken)obj).token);
+        }
+
+        public int hashCode()
+        {
+            return Longs.hashCode(token);
+        }
+
+        public int compareTo(Token o)
+        {
+            return Long.compare(token, ((LongToken) o).token);
+        }
+
+        @Override
+        public IPartitioner getPartitioner()
+        {
+            return instance;
+        }
+
+        @Override
+        public long getHeapSize()
+        {
+            return HEAP_SIZE;
+        }
+
+        @Override
+        public Object getTokenValue()
+        {
+            return token;
+        }
+    }
+
     /**
      * Generate the token of a key.
      * Note that we need to ensure all generated token are strictly bigger than MINIMUM.
@@ -90,17 +150,22 @@
      */
     public LongToken getToken(ByteBuffer key)
     {
+        return getToken(key, getHash(key));
+    }
+
+    private LongToken getToken(ByteBuffer key, long[] hash)
+    {
         if (key.remaining() == 0)
             return MINIMUM;
 
-        long[] hash = new long[2];
-        MurmurHash.hash3_x64_128(key, key.position(), key.remaining(), 0, hash);
         return new LongToken(normalize(hash[0]));
     }
 
-    public long getHeapSizeOf(Token token)
+    private long[] getHash(ByteBuffer key)
     {
-        return HEAP_SIZE;
+        long[] hash = new long[2];
+        MurmurHash.hash3_x64_128(key, key.position(), key.remaining(), 0, hash);
+        return hash;
     }
 
     public LongToken getRandomToken()
@@ -129,7 +194,7 @@
             throw new RuntimeException("No nodes present in the cluster. Has this node finished starting up?");
         // 1-case
         if (sortedTokens.size() == 1)
-            ownerships.put((Token) i.next(), new Float(1.0));
+            ownerships.put(i.next(), new Float(1.0));
         // n-case
         else
         {
@@ -140,7 +205,7 @@
 
             while (i.hasNext())
             {
-                t = (Token) i.next(); ti = BigInteger.valueOf(((LongToken) t).token); // The next token and its value
+                t = i.next(); ti = BigInteger.valueOf(((LongToken) t).token); // The next token and its value
                 float age = new BigDecimal(ti.subtract(tim1).add(ri).mod(ri)).divide(r, 6, BigDecimal.ROUND_HALF_EVEN).floatValue(); // %age = ((T(i) - T(i-1) + R) % R) / R
                 ownerships.put(t, age);                           // save (T(i) -> %age)
                 tim1 = ti;                                        // -> advance loop
@@ -193,7 +258,7 @@
         {
             try
             {
-                return new LongToken(Long.valueOf(string));
+                return new LongToken(Long.parseLong(string));
             }
             catch (NumberFormatException e)
             {
diff --git a/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java b/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
index 44d98e0..cffa4fc 100644
--- a/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
+++ b/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
@@ -23,8 +23,8 @@
 import java.util.*;
 
 import org.apache.cassandra.config.*;
-import org.apache.cassandra.db.BufferDecoratedKey;
 import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.CachedHashDecoratedKey;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.exceptions.ConfigurationException;
@@ -35,7 +35,7 @@
 import org.apache.cassandra.utils.ObjectSizes;
 import org.apache.cassandra.utils.Pair;
 
-public class OrderPreservingPartitioner extends AbstractPartitioner
+public class OrderPreservingPartitioner implements IPartitioner
 {
     public static final StringToken MINIMUM = new StringToken("");
 
@@ -43,9 +43,11 @@
 
     private static final long EMPTY_SIZE = ObjectSizes.measure(MINIMUM);
 
+    public static final OrderPreservingPartitioner instance = new OrderPreservingPartitioner();
+
     public DecoratedKey decorateKey(ByteBuffer key)
     {
-        return new BufferDecoratedKey(getToken(key), key);
+        return new CachedHashDecoratedKey(getToken(key), key);
     }
 
     public StringToken midpoint(Token ltoken, Token rtoken)
@@ -161,6 +163,28 @@
         return true;
     }
 
+    public static class StringToken extends ComparableObjectToken<String>
+    {
+        static final long serialVersionUID = 5464084395277974963L;
+
+        public StringToken(String token)
+        {
+            super(token);
+        }
+
+        @Override
+        public IPartitioner getPartitioner()
+        {
+            return instance;
+        }
+
+        @Override
+        public long getHeapSize()
+        {
+            return EMPTY_SIZE + ObjectSizes.sizeOf(token);
+        }
+    }
+
     public StringToken getToken(ByteBuffer key)
     {
         String skey;
@@ -175,11 +199,6 @@
         return new StringToken(skey);
     }
 
-    public long getHeapSizeOf(Token token)
-    {
-        return EMPTY_SIZE + ObjectSizes.sizeOf(((StringToken) token).token);
-    }
-
     public Map<Token, Float> describeOwnership(List<Token> sortedTokens)
     {
         // allTokens will contain the count and be returned, sorted_ranges is shorthand for token<->token math.
diff --git a/src/java/org/apache/cassandra/dht/RandomPartitioner.java b/src/java/org/apache/cassandra/dht/RandomPartitioner.java
index fa44f47..71a0a99 100644
--- a/src/java/org/apache/cassandra/dht/RandomPartitioner.java
+++ b/src/java/org/apache/cassandra/dht/RandomPartitioner.java
@@ -22,7 +22,9 @@
 import java.nio.ByteBuffer;
 import java.util.*;
 
-import org.apache.cassandra.db.BufferDecoratedKey;
+import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.cassandra.db.CachedHashDecoratedKey;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.marshal.AbstractType;
@@ -36,17 +38,19 @@
 /**
  * This class generates a BigIntegerToken using MD5 hash.
  */
-public class RandomPartitioner extends AbstractPartitioner
+public class RandomPartitioner implements IPartitioner
 {
     public static final BigInteger ZERO = new BigInteger("0");
     public static final BigIntegerToken MINIMUM = new BigIntegerToken("-1");
     public static final BigInteger MAXIMUM = new BigInteger("2").pow(127);
 
-    private static final int EMPTY_SIZE = (int) ObjectSizes.measureDeep(new BigIntegerToken(FBUtilities.hashToBigInteger(ByteBuffer.allocate(1))));
+    private static final int HEAP_SIZE = (int) ObjectSizes.measureDeep(new BigIntegerToken(FBUtilities.hashToBigInteger(ByteBuffer.allocate(1))));
+
+    public static final RandomPartitioner instance = new RandomPartitioner();
 
     public DecoratedKey decorateKey(ByteBuffer key)
     {
-        return new BufferDecoratedKey(getToken(key), key);
+        return new CachedHashDecoratedKey(getToken(key), key);
     }
 
     public Token midpoint(Token ltoken, Token rtoken)
@@ -122,6 +126,34 @@
         return false;
     }
 
+    public static class BigIntegerToken extends ComparableObjectToken<BigInteger>
+    {
+        static final long serialVersionUID = -5833589141319293006L;
+
+        public BigIntegerToken(BigInteger token)
+        {
+            super(token);
+        }
+
+        // convenience method for testing
+        @VisibleForTesting
+        public BigIntegerToken(String token) {
+            this(new BigInteger(token));
+        }
+
+        @Override
+        public IPartitioner getPartitioner()
+        {
+            return instance;
+        }
+
+        @Override
+        public long getHeapSize()
+        {
+            return HEAP_SIZE;
+        }
+    }
+
     public BigIntegerToken getToken(ByteBuffer key)
     {
         if (key.remaining() == 0)
@@ -129,11 +161,6 @@
         return new BigIntegerToken(FBUtilities.hashToBigInteger(key));
     }
 
-    public long getHeapSizeOf(Token token)
-    {
-        return EMPTY_SIZE;
-    }
-
     public Map<Token, Float> describeOwnership(List<Token> sortedTokens)
     {
         Map<Token, Float> ownerships = new HashMap<Token, Float>();
diff --git a/src/java/org/apache/cassandra/dht/Range.java b/src/java/org/apache/cassandra/dht/Range.java
index 618a3f4..34e91ea 100644
--- a/src/java/org/apache/cassandra/dht/Range.java
+++ b/src/java/org/apache/cassandra/dht/Range.java
@@ -24,7 +24,6 @@
 
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.RowPosition;
-import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.Pair;
 
 /**
@@ -33,6 +32,8 @@
  * A Range is responsible for the tokens between (left, right].
  *
  * Used by the partitioner and by map/reduce by-token range scans.
+ *
+ * Note: this class has a natural ordering that is inconsistent with equals
  */
 public class Range<T extends RingPosition<T>> extends AbstractBounds<T> implements Comparable<Range<T>>, Serializable
 {
@@ -40,12 +41,7 @@
 
     public Range(T left, T right)
     {
-        this(left, right, StorageService.getPartitioner());
-    }
-
-    public Range(T left, T right, IPartitioner partitioner)
-    {
-        super(left, right, partitioner);
+        super(left, right);
     }
 
     public static <T extends RingPosition<T>> boolean contains(T left, T right, T point)
@@ -175,8 +171,7 @@
             if (!(left.compareTo(that.right) < 0 && that.left.compareTo(right) < 0))
                 return Collections.emptySet();
             return rangeSet(new Range<T>(ObjectUtils.max(this.left, that.left),
-                                         ObjectUtils.min(this.right, that.right),
-                                         partitioner));
+                                         ObjectUtils.min(this.right, that.right)));
         }
         if (thiswraps && thatwraps)
         {
@@ -204,8 +199,8 @@
     {
         Set<Range<T>> intersection = new HashSet<Range<T>>(2);
         if (that.right.compareTo(first.left) > 0)
-            intersection.add(new Range<T>(first.left, that.right, first.partitioner));
-        intersection.add(new Range<T>(that.left, first.right, first.partitioner));
+            intersection.add(new Range<T>(first.left, that.right));
+        intersection.add(new Range<T>(that.left, first.right));
         return Collections.unmodifiableSet(intersection);
     }
 
@@ -213,10 +208,10 @@
     {
         Set<Range<T>> intersection = new HashSet<Range<T>>(2);
         if (other.contains(wrapping.right))
-            intersection.add(new Range<T>(other.left, wrapping.right, wrapping.partitioner));
+            intersection.add(new Range<T>(other.left, wrapping.right));
         // need the extra compareto here because ranges are asymmetrical; wrapping.left _is not_ contained by the wrapping range
         if (other.contains(wrapping.left) && wrapping.left.compareTo(other.right) < 0)
-            intersection.add(new Range<T>(wrapping.left, other.right, wrapping.partitioner));
+            intersection.add(new Range<T>(wrapping.left, other.right));
         return Collections.unmodifiableSet(intersection);
     }
 
@@ -227,8 +222,8 @@
         if (position.equals(left) || position.equals(right))
             return null;
 
-        AbstractBounds<T> lb = new Range<T>(left, position, partitioner);
-        AbstractBounds<T> rb = new Range<T>(position, right, partitioner);
+        AbstractBounds<T> lb = new Range<T>(left, position);
+        AbstractBounds<T> rb = new Range<T>(position, right);
         return Pair.create(lb, rb);
     }
 
@@ -244,13 +239,12 @@
 
     public List<Range<T>> unwrap()
     {
-        @SuppressWarnings("unchecked")
-        T minValue = (T) partitioner.minValue(right.getClass());
+        T minValue = right.minValue();
         if (!isWrapAround() || right.equals(minValue))
             return Arrays.asList(this);
         List<Range<T>> unwrapped = new ArrayList<Range<T>>(2);
-        unwrapped.add(new Range<T>(left, minValue, partitioner));
-        unwrapped.add(new Range<T>(minValue, right, partitioner));
+        unwrapped.add(new Range<T>(left, minValue));
+        unwrapped.add(new Range<T>(minValue, right));
         return unwrapped;
     }
 
@@ -262,18 +256,18 @@
        return left.compareTo(right) >= 0;
     }
 
+    /**
+     * Note: this class has a natural ordering that is inconsistent with equals
+     */
     public int compareTo(Range<T> rhs)
     {
-        /*
-         * If the range represented by the "this" pointer
-         * is a wrap around then it is the smaller one.
-         */
-        if ( isWrapAround(left, right) )
-            return -1;
+        boolean lhsWrap = isWrapAround(left, right);
+        boolean rhsWrap = isWrapAround(rhs.left, rhs.right);
 
-        if ( isWrapAround(rhs.left, rhs.right) )
-            return 1;
-
+        // if one of the two wraps, that's the smaller one.
+        if (lhsWrap != rhsWrap)
+            return Boolean.compare(!lhsWrap, !rhsWrap);
+        // otherwise compare by right.
         return right.compareTo(rhs.right);
     }
 
@@ -289,9 +283,9 @@
         ArrayList<Range<T>> difference = new ArrayList<Range<T>>(2);
 
         if (!left.equals(contained.left))
-            difference.add(new Range<T>(left, contained.left, partitioner));
+            difference.add(new Range<T>(left, contained.left));
         if (!right.equals(contained.right))
-            difference.add(new Range<T>(contained.right, right, partitioner));
+            difference.add(new Range<T>(contained.right, right));
         return difference;
     }
 
@@ -342,7 +336,7 @@
         else
         {
             @SuppressWarnings("unchecked")
-            Range<T>[] intersections = (Range<T>[]) new Range[intersectionSet.size()];
+            Range<T>[] intersections = new Range[intersectionSet.size()];
             intersectionSet.toArray(intersections);
             if (intersections.length == 1)
             {
@@ -453,8 +447,7 @@
         Iterator<Range<T>> iter = ranges.iterator();
         Range<T> current = iter.next();
 
-        @SuppressWarnings("unchecked")
-        T min = (T) current.partitioner.minValue(current.left.getClass());
+        T min = current.left.minValue();
         while (iter.hasNext())
         {
             // If current goes to the end of the ring, we're done
@@ -489,30 +482,22 @@
         return output;
     }
 
+    public AbstractBounds<T> withNewRight(T newRight)
+    {
+        return new Range<T>(left, newRight);
+    }
 
     /**
      * Compute a range of keys corresponding to a given range of token.
      */
-    public static Range<RowPosition> makeRowRange(Token left, Token right, IPartitioner partitioner)
+    public static Range<RowPosition> makeRowRange(Token left, Token right)
     {
-        return new Range<RowPosition>(left.maxKeyBound(partitioner), right.maxKeyBound(partitioner), partitioner);
+        return new Range<RowPosition>(left.maxKeyBound(), right.maxKeyBound());
     }
 
-    @SuppressWarnings("unchecked")
-    public AbstractBounds<RowPosition> toRowBounds()
+    public static Range<RowPosition> makeRowRange(Range<Token> tokenBounds)
     {
-        return (left instanceof Token) ? makeRowRange((Token)left, (Token)right, partitioner) : (Range<RowPosition>)this;
-    }
-
-    @SuppressWarnings("unchecked")
-    public AbstractBounds<Token> toTokenBounds()
-    {
-        return (left instanceof RowPosition) ? new Range<Token>(((RowPosition)left).getToken(), ((RowPosition)right).getToken(), partitioner) : (Range<Token>)this;
-    }
-
-    public AbstractBounds<T> withNewRight(T newRight)
-    {
-        return new Range<T>(left, newRight);
+        return makeRowRange(tokenBounds.left, tokenBounds.right);
     }
 
     /**
diff --git a/src/java/org/apache/cassandra/dht/RangeStreamer.java b/src/java/org/apache/cassandra/dht/RangeStreamer.java
index 121a351..aef588e 100644
--- a/src/java/org/apache/cassandra/dht/RangeStreamer.java
+++ b/src/java/org/apache/cassandra/dht/RangeStreamer.java
@@ -20,24 +20,24 @@
 import java.net.InetAddress;
 import java.util.*;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.Multimap;
 import com.google.common.collect.Sets;
-import org.apache.cassandra.gms.EndpointState;
 import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.SystemKeyspace;
-import org.apache.cassandra.gms.FailureDetector;
+import org.apache.cassandra.gms.EndpointState;
 import org.apache.cassandra.gms.Gossiper;
 import org.apache.cassandra.gms.IFailureDetector;
 import org.apache.cassandra.locator.AbstractReplicationStrategy;
 import org.apache.cassandra.locator.IEndpointSnitch;
 import org.apache.cassandra.locator.TokenMetadata;
+import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.streaming.StreamPlan;
 import org.apache.cassandra.streaming.StreamResultFuture;
 import org.apache.cassandra.utils.FBUtilities;
@@ -48,15 +48,21 @@
 public class RangeStreamer
 {
     private static final Logger logger = LoggerFactory.getLogger(RangeStreamer.class);
-    public static final boolean useStrictConsistency = Boolean.valueOf(System.getProperty("cassandra.consistent.rangemovement","true"));
-    public static final boolean allowSimultaneousMoves = Boolean.valueOf(System.getProperty("cassandra.consistent.simultaneousmoves.allow","false"));
+
+    /* bootstrap tokens. can be null if replacing the node. */
     private final Collection<Token> tokens;
+    /* current token ring */
     private final TokenMetadata metadata;
+    /* address of this node */
     private final InetAddress address;
+    /* streaming description */
     private final String description;
     private final Multimap<String, Map.Entry<InetAddress, Collection<Range<Token>>>> toFetch = HashMultimap.create();
-    private final Set<ISourceFilter> sourceFilters = new HashSet<ISourceFilter>();
+    private final Set<ISourceFilter> sourceFilters = new HashSet<>();
     private final StreamPlan streamPlan;
+    private final boolean useStrictConsistency;
+    private final IEndpointSnitch snitch;
+    private final StreamStateStore stateStore;
 
     /**
      * A filter applied to sources to stream from when constructing a fetch map.
@@ -116,22 +122,23 @@
         }
     }
 
-    public RangeStreamer(TokenMetadata metadata, Collection<Token> tokens, InetAddress address, String description)
+    public RangeStreamer(TokenMetadata metadata,
+                         Collection<Token> tokens,
+                         InetAddress address,
+                         String description,
+                         boolean useStrictConsistency,
+                         IEndpointSnitch snitch,
+                         StreamStateStore stateStore)
     {
         this.metadata = metadata;
         this.tokens = tokens;
         this.address = address;
         this.description = description;
-        this.streamPlan = new StreamPlan(description);
-    }
-
-    public RangeStreamer(TokenMetadata metadata, InetAddress address, String description)
-    {
-        this.metadata = metadata;
-        this.tokens = null;
-        this.address = address;
-        this.description = description;
-        this.streamPlan = new StreamPlan(description);
+        this.streamPlan = new StreamPlan(description, true);
+        this.useStrictConsistency = useStrictConsistency;
+        this.snitch = snitch;
+        this.stateStore = stateStore;
+        streamPlan.listeners(this.stateStore);
     }
 
     public void addSourceFilter(ISourceFilter filter)
@@ -139,42 +146,51 @@
         sourceFilters.add(filter);
     }
 
+    /**
+     * Add ranges to be streamed for given keyspace.
+     *
+     * @param keyspaceName keyspace name
+     * @param ranges ranges to be streamed
+     */
     public void addRanges(String keyspaceName, Collection<Range<Token>> ranges)
     {
         Multimap<Range<Token>, InetAddress> rangesForKeyspace = useStrictSourcesForRanges(keyspaceName)
                 ? getAllRangesWithStrictSourcesFor(keyspaceName, ranges) : getAllRangesWithSourcesFor(keyspaceName, ranges);
 
-        if (logger.isDebugEnabled())
+        if (logger.isTraceEnabled())
         {
             for (Map.Entry<Range<Token>, InetAddress> entry : rangesForKeyspace.entries())
-                logger.debug(String.format("%s: range %s exists on %s", description, entry.getKey(), entry.getValue()));
+                logger.trace(String.format("%s: range %s exists on %s", description, entry.getKey(), entry.getValue()));
         }
 
-        for (Map.Entry<InetAddress, Collection<Range<Token>>> entry : getRangeFetchMap(rangesForKeyspace, sourceFilters, keyspaceName).asMap().entrySet())
+        for (Map.Entry<InetAddress, Collection<Range<Token>>> entry : getRangeFetchMap(rangesForKeyspace, sourceFilters, keyspaceName, useStrictConsistency).asMap().entrySet())
         {
-            if (logger.isDebugEnabled())
+            if (logger.isTraceEnabled())
             {
                 for (Range<Token> r : entry.getValue())
-                    logger.debug(String.format("%s: range %s from source %s for keyspace %s", description, r, entry.getKey(), keyspaceName));
+                    logger.trace(String.format("%s: range %s from source %s for keyspace %s", description, r, entry.getKey(), keyspaceName));
             }
             toFetch.put(keyspaceName, entry);
         }
     }
 
+    /**
+     * @param keyspaceName keyspace name to check
+     * @return true when the node is bootstrapping, useStrictConsistency is true and # of nodes in the cluster is more than # of replica
+     */
     private boolean useStrictSourcesForRanges(String keyspaceName)
     {
         AbstractReplicationStrategy strat = Keyspace.open(keyspaceName).getReplicationStrategy();
-        return isNotReplacingAndUsesStrictConsistency() && tokens != null && metadata.getAllEndpoints().size() != strat.getReplicationFactor();
-    }
-
-    private static boolean isNotReplacingAndUsesStrictConsistency()
-    {
-        return !DatabaseDescriptor.isReplacing() && useStrictConsistency;
+        return useStrictConsistency
+                && tokens != null
+                && metadata.getAllEndpoints().size() != strat.getReplicationFactor();
     }
 
     /**
      * Get a map of all ranges and their respective sources that are candidates for streaming the given ranges
      * to us. For each range, the list of sources is sorted by proximity relative to the given destAddress.
+     *
+     * @throws java.lang.IllegalStateException when there is no source to get data streamed
      */
     private Multimap<Range<Token>, InetAddress> getAllRangesWithSourcesFor(String keyspaceName, Collection<Range<Token>> desiredRanges)
     {
@@ -188,7 +204,7 @@
             {
                 if (range.contains(desiredRange))
                 {
-                    List<InetAddress> preferred = DatabaseDescriptor.getEndpointSnitch().getSortedListByProximity(address, rangeAddresses.get(range));
+                    List<InetAddress> preferred = snitch.getSortedListByProximity(address, rangeAddresses.get(range));
                     rangeSources.putAll(desiredRange, preferred);
                     break;
                 }
@@ -205,22 +221,23 @@
      * Get a map of all ranges and the source that will be cleaned up once this bootstrapped node is added for the given ranges.
      * For each range, the list should only contain a single source. This allows us to consistently migrate data without violating
      * consistency.
+     *
+     * @throws java.lang.IllegalStateException when there is no source to get data streamed, or more than 1 source found.
      */
-    private Multimap<Range<Token>, InetAddress> getAllRangesWithStrictSourcesFor(String table, Collection<Range<Token>> desiredRanges)
+    private Multimap<Range<Token>, InetAddress> getAllRangesWithStrictSourcesFor(String keyspace, Collection<Range<Token>> desiredRanges)
     {
-
         assert tokens != null;
-        AbstractReplicationStrategy strat = Keyspace.open(table).getReplicationStrategy();
+        AbstractReplicationStrategy strat = Keyspace.open(keyspace).getReplicationStrategy();
 
-        //Active ranges
+        // Active ranges
         TokenMetadata metadataClone = metadata.cloneOnlyTokenMap();
-        Multimap<Range<Token>,InetAddress> addressRanges = strat.getRangeAddresses(metadataClone);
+        Multimap<Range<Token>, InetAddress> addressRanges = strat.getRangeAddresses(metadataClone);
 
-        //Pending ranges
+        // Pending ranges
         metadataClone.updateNormalTokens(tokens, address);
-        Multimap<Range<Token>,InetAddress> pendingRangeAddresses = strat.getRangeAddresses(metadataClone);
+        Multimap<Range<Token>, InetAddress> pendingRangeAddresses = strat.getRangeAddresses(metadataClone);
 
-        //Collects the source that will have its range moved to the new node
+        // Collects the source that will have its range moved to the new node
         Multimap<Range<Token>, InetAddress> rangeSources = ArrayListMultimap.create();
 
         for (Range<Token> desiredRange : desiredRanges)
@@ -232,8 +249,8 @@
                     Set<InetAddress> oldEndpoints = Sets.newHashSet(preEntry.getValue());
                     Set<InetAddress> newEndpoints = Sets.newHashSet(pendingRangeAddresses.get(desiredRange));
 
-                    //Due to CASSANDRA-5953 we can have a higher RF then we have endpoints.
-                    //So we need to be careful to only be strict when endpoints == RF
+                    // Due to CASSANDRA-5953 we can have a higher RF then we have endpoints.
+                    // So we need to be careful to only be strict when endpoints == RF
                     if (oldEndpoints.size() == strat.getReplicationFactor())
                     {
                         oldEndpoints.removeAll(newEndpoints);
@@ -244,7 +261,7 @@
                 }
             }
 
-            //Validate
+            // Validate
             Collection<InetAddress> addressList = rangeSources.get(desiredRange);
             if (addressList == null || addressList.isEmpty())
                 throw new IllegalStateException("No sources found for " + desiredRange);
@@ -255,7 +272,8 @@
             InetAddress sourceIp = addressList.iterator().next();
             EndpointState sourceState = Gossiper.instance.getEndpointStateForEndpoint(sourceIp);
             if (Gossiper.instance.isEnabled() && (sourceState == null || !sourceState.isAlive()))
-                throw new RuntimeException("A node required to move the data consistently is down ("+sourceIp+").  If you wish to move the data from a potentially inconsistent replica, restart the node with -Dcassandra.consistent.rangemovement=false");
+                throw new RuntimeException("A node required to move the data consistently is down (" + sourceIp + "). " +
+                                           "If you wish to move the data from a potentially inconsistent replica, restart the node with -Dcassandra.consistent.rangemovement=false");
         }
 
         return rangeSources;
@@ -265,10 +283,12 @@
      * @param rangesWithSources The ranges we want to fetch (key) and their potential sources (value)
      * @param sourceFilters A (possibly empty) collection of source filters to apply. In addition to any filters given
      *                      here, we always exclude ourselves.
-     * @return
+     * @param keyspace keyspace name
+     * @return Map of source endpoint to collection of ranges
      */
     private static Multimap<InetAddress, Range<Token>> getRangeFetchMap(Multimap<Range<Token>, InetAddress> rangesWithSources,
-                                                                        Collection<ISourceFilter> sourceFilters, String keyspace)
+                                                                        Collection<ISourceFilter> sourceFilters, String keyspace,
+                                                                        boolean useStrictConsistency)
     {
         Multimap<InetAddress, Range<Token>> rangeFetchMapMap = HashMultimap.create();
         for (Range<Token> range : rangesWithSources.keySet())
@@ -301,7 +321,7 @@
                 AbstractReplicationStrategy strat = Keyspace.open(keyspace).getReplicationStrategy();
                 if (strat != null && strat.getReplicationFactor() == 1)
                 {
-                    if (isNotReplacingAndUsesStrictConsistency())
+                    if (useStrictConsistency)
                         throw new IllegalStateException("Unable to find sufficient sources for streaming range " + range + " in keyspace " + keyspace + " with RF=1." +
                                                         "If you want to ignore this, consider using system property -Dcassandra.consistent.rangemovement=false.");
                     else
@@ -316,12 +336,14 @@
         return rangeFetchMapMap;
     }
 
-    public static Multimap<InetAddress, Range<Token>> getWorkMap(Multimap<Range<Token>, InetAddress> rangesWithSourceTarget, String keyspace)
+    public static Multimap<InetAddress, Range<Token>> getWorkMap(Multimap<Range<Token>, InetAddress> rangesWithSourceTarget, String keyspace,
+                                                                 IFailureDetector fd, boolean useStrictConsistency)
     {
-        return getRangeFetchMap(rangesWithSourceTarget, Collections.<ISourceFilter>singleton(new FailureDetectorSourceFilter(FailureDetector.instance)), keyspace);
+        return getRangeFetchMap(rangesWithSourceTarget, Collections.<ISourceFilter>singleton(new FailureDetectorSourceFilter(fd)), keyspace, useStrictConsistency);
     }
 
     // For testing purposes
+    @VisibleForTesting
     Multimap<String, Map.Entry<InetAddress, Collection<Range<Token>>>> toFetch()
     {
         return toFetch;
@@ -335,9 +357,17 @@
             InetAddress source = entry.getValue().getKey();
             InetAddress preferred = SystemKeyspace.getPreferredIP(source);
             Collection<Range<Token>> ranges = entry.getValue().getValue();
+
+            // filter out already streamed ranges
+            Set<Range<Token>> availableRanges = stateStore.getAvailableRanges(keyspace, StorageService.getPartitioner());
+            if (ranges.removeAll(availableRanges))
+            {
+                logger.info("Some ranges of {} are already available. Skipping streaming those ranges.", availableRanges);
+            }
+
+            if (logger.isTraceEnabled())
+                logger.trace("{}ing from {} ranges {}", description, source, StringUtils.join(ranges, ", "));
             /* Send messages to respective folks to stream data over to me */
-            if (logger.isDebugEnabled())
-                logger.debug("{}ing from {} ranges {}", description, source, StringUtils.join(ranges, ", "));
             streamPlan.requestRanges(source, preferred, keyspace, ranges);
         }
 
diff --git a/src/java/org/apache/cassandra/dht/RingPosition.java b/src/java/org/apache/cassandra/dht/RingPosition.java
index fefc3b3..f2d83ad 100644
--- a/src/java/org/apache/cassandra/dht/RingPosition.java
+++ b/src/java/org/apache/cassandra/dht/RingPosition.java
@@ -25,5 +25,7 @@
 public interface RingPosition<C extends RingPosition<C>> extends Comparable<C>
 {
     public Token getToken();
-    public boolean isMinimum(IPartitioner partitioner);
+    public IPartitioner getPartitioner();
+    public boolean isMinimum();
+    public C minValue();
 }
diff --git a/src/java/org/apache/cassandra/dht/StreamStateStore.java b/src/java/org/apache/cassandra/dht/StreamStateStore.java
new file mode 100644
index 0000000..f6046aa
--- /dev/null
+++ b/src/java/org/apache/cassandra/dht/StreamStateStore.java
@@ -0,0 +1,82 @@
+/*
+ * 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.dht;
+
+import java.util.Set;
+
+import org.apache.cassandra.db.SystemKeyspace;
+import org.apache.cassandra.streaming.StreamEvent;
+import org.apache.cassandra.streaming.StreamEventHandler;
+import org.apache.cassandra.streaming.StreamRequest;
+import org.apache.cassandra.streaming.StreamState;
+
+/**
+ * Store and update available ranges (data already received) to system keyspace.
+ */
+public class StreamStateStore implements StreamEventHandler
+{
+    public Set<Range<Token>> getAvailableRanges(String keyspace, IPartitioner partitioner)
+    {
+        return SystemKeyspace.getAvailableRanges(keyspace, partitioner);
+    }
+
+    /**
+     * Check if given token's data is available in this node.
+     *
+     * @param keyspace keyspace name
+     * @param token token to check
+     * @return true if given token in the keyspace is already streamed and ready to be served.
+     */
+    public boolean isDataAvailable(String keyspace, Token token)
+    {
+        Set<Range<Token>> availableRanges = getAvailableRanges(keyspace, token.getPartitioner());
+        for (Range<Token> range : availableRanges)
+        {
+            if (range.contains(token))
+                return true;
+        }
+        return false;
+    }
+
+    /**
+     * When StreamSession completes, make all keyspaces/ranges in session available to be served.
+     *
+     * @param event Stream event.
+     */
+    @Override
+    public void handleStreamEvent(StreamEvent event)
+    {
+        if (event.eventType == StreamEvent.Type.STREAM_COMPLETE)
+        {
+            StreamEvent.SessionCompleteEvent se = (StreamEvent.SessionCompleteEvent) event;
+            if (se.success)
+            {
+                for (StreamRequest request : se.requests)
+                {
+                    SystemKeyspace.updateAvailableRanges(request.keyspace, request.ranges);
+                }
+            }
+        }
+    }
+
+    @Override
+    public void onSuccess(StreamState streamState) {}
+
+    @Override
+    public void onFailure(Throwable throwable) {}
+}
diff --git a/src/java/org/apache/cassandra/dht/StringToken.java b/src/java/org/apache/cassandra/dht/StringToken.java
deleted file mode 100644
index f541313..0000000
--- a/src/java/org/apache/cassandra/dht/StringToken.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * 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.dht;
-
-
-public class StringToken extends ComparableObjectToken<String>
-{
-    static final long serialVersionUID = 5464084395277974963L;
-
-    public StringToken(String token)
-    {
-        super(token);
-    }
-}
diff --git a/src/java/org/apache/cassandra/dht/Token.java b/src/java/org/apache/cassandra/dht/Token.java
index 98babf5..76918a7 100644
--- a/src/java/org/apache/cassandra/dht/Token.java
+++ b/src/java/org/apache/cassandra/dht/Token.java
@@ -22,12 +22,10 @@
 import java.io.Serializable;
 import java.nio.ByteBuffer;
 
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.db.RowPosition;
-import org.apache.cassandra.io.ISerializer;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 public abstract class Token implements RingPosition<Token>, Serializable
@@ -46,32 +44,33 @@
         public abstract void validate(String token) throws ConfigurationException;
     }
 
-    public static class TokenSerializer implements ISerializer<Token>
+    public static class TokenSerializer implements IPartitionerDependentSerializer<Token>
     {
-        public void serialize(Token token, DataOutputPlus out) throws IOException
+        public void serialize(Token token, DataOutputPlus out, int version) throws IOException
         {
-            IPartitioner p = StorageService.getPartitioner();
+            IPartitioner p = token.getPartitioner();
             ByteBuffer b = p.getTokenFactory().toByteArray(token);
             ByteBufferUtil.writeWithLength(b, out);
         }
 
-        public Token deserialize(DataInput in) throws IOException
+        public Token deserialize(DataInput in, IPartitioner p, int version) throws IOException
         {
-            IPartitioner p = StorageService.getPartitioner();
             int size = in.readInt();
             byte[] bytes = new byte[size];
             in.readFully(bytes);
             return p.getTokenFactory().fromByteArray(ByteBuffer.wrap(bytes));
         }
 
-        public long serializedSize(Token object, TypeSizes typeSizes)
+        public long serializedSize(Token object, int version)
         {
-            IPartitioner p = StorageService.getPartitioner();
+            IPartitioner p = object.getPartitioner();
             ByteBuffer b = p.getTokenFactory().toByteArray(object);
             return TypeSizes.NATIVE.sizeof(b.remaining()) + b.remaining();
         }
     }
 
+    abstract public IPartitioner getPartitioner();
+    abstract public long getHeapSize();
     abstract public Object getTokenValue();
 
     public Token getToken()
@@ -79,14 +78,14 @@
         return this;
     }
 
-    public boolean isMinimum(IPartitioner partitioner)
+    public Token minValue()
     {
-        return this.equals(partitioner.getMinimumToken());
+        return getPartitioner().getMinimumToken();
     }
 
     public boolean isMinimum()
     {
-        return isMinimum(StorageService.getPartitioner());
+        return this.equals(minValue());
     }
 
     /*
@@ -103,17 +102,12 @@
      * Note that those are "fake" keys and should only be used for comparison
      * of other keys, for selection of keys when only a token is known.
      */
-    public KeyBound minKeyBound(IPartitioner partitioner)
+    public KeyBound minKeyBound()
     {
         return new KeyBound(this, true);
     }
 
-    public KeyBound minKeyBound()
-    {
-        return minKeyBound(null);
-    }
-
-    public KeyBound maxKeyBound(IPartitioner partitioner)
+    public KeyBound maxKeyBound()
     {
         /*
          * For each token, we needs both minKeyBound and maxKeyBound
@@ -122,16 +116,11 @@
          * simpler to associate the same value for minKeyBound and
          * maxKeyBound for the minimun token.
          */
-        if (isMinimum(partitioner))
+        if (isMinimum())
             return minKeyBound();
         return new KeyBound(this, false);
     }
 
-    public KeyBound maxKeyBound()
-    {
-        return maxKeyBound(StorageService.getPartitioner());
-    }
-
     @SuppressWarnings("unchecked")
     public <R extends RingPosition<R>> R upperBound(Class<R> klass)
     {
@@ -172,14 +161,19 @@
                 return ((pos instanceof KeyBound) && !((KeyBound)pos).isMinimumBound) ? 0 : 1;
         }
 
-        public boolean isMinimum(IPartitioner partitioner)
+        public IPartitioner getPartitioner()
         {
-            return getToken().isMinimum(partitioner);
+            return getToken().getPartitioner();
+        }
+
+        public KeyBound minValue()
+        {
+            return getPartitioner().getMinimumToken().minKeyBound();
         }
 
         public boolean isMinimum()
         {
-            return isMinimum(StorageService.getPartitioner());
+            return getToken().isMinimum();
         }
 
         public RowPosition.Kind kind()
diff --git a/src/java/org/apache/cassandra/exceptions/AlreadyExistsException.java b/src/java/org/apache/cassandra/exceptions/AlreadyExistsException.java
index 4530568..1829c5c 100644
--- a/src/java/org/apache/cassandra/exceptions/AlreadyExistsException.java
+++ b/src/java/org/apache/cassandra/exceptions/AlreadyExistsException.java
@@ -31,7 +31,7 @@
 
     public AlreadyExistsException(String ksName, String cfName)
     {
-        this(ksName, cfName, String.format("Cannot add already existing column family \"%s\" to keyspace \"%s\"", cfName, ksName));
+        this(ksName, cfName, String.format("Cannot add already existing table \"%s\" to keyspace \"%s\"", cfName, ksName));
     }
 
     public AlreadyExistsException(String ksName)
diff --git a/src/java/org/apache/cassandra/exceptions/CassandraException.java b/src/java/org/apache/cassandra/exceptions/CassandraException.java
index aaa8055..58521df 100644
--- a/src/java/org/apache/cassandra/exceptions/CassandraException.java
+++ b/src/java/org/apache/cassandra/exceptions/CassandraException.java
@@ -17,7 +17,7 @@
  */
 package org.apache.cassandra.exceptions;
 
-public abstract class CassandraException extends Exception implements TransportException
+public abstract class CassandraException extends RuntimeException implements TransportException
 {
     private final ExceptionCode code;
 
diff --git a/src/java/org/apache/cassandra/exceptions/ConfigurationException.java b/src/java/org/apache/cassandra/exceptions/ConfigurationException.java
index 8f0bb1a..f28fa51 100644
--- a/src/java/org/apache/cassandra/exceptions/ConfigurationException.java
+++ b/src/java/org/apache/cassandra/exceptions/ConfigurationException.java
@@ -19,18 +19,34 @@
 
 public class ConfigurationException extends RequestValidationException
 {
+    /*
+     * If the error is logged should a stack trace be included.
+     * For expected errors with an informative message no stack trace needs to be logged.
+     * This is just a suggestion to exception handlers as to how they should format the exception.
+     */
+    public final boolean logStackTrace;
+
     public ConfigurationException(String msg)
     {
         super(ExceptionCode.CONFIG_ERROR, msg);
+        logStackTrace = true;
+    }
+
+    public ConfigurationException(String msg, boolean logStackTrace)
+    {
+        super(ExceptionCode.CONFIG_ERROR, msg);
+        this.logStackTrace = logStackTrace;
     }
 
     public ConfigurationException(String msg, Throwable e)
     {
         super(ExceptionCode.CONFIG_ERROR, msg, e);
+        logStackTrace = true;
     }
 
     protected ConfigurationException(ExceptionCode code, String msg)
     {
         super(code, msg);
+        logStackTrace = true;
     }
 }
diff --git a/src/java/org/apache/cassandra/exceptions/ExceptionCode.java b/src/java/org/apache/cassandra/exceptions/ExceptionCode.java
index ce082a7..6ad0577 100644
--- a/src/java/org/apache/cassandra/exceptions/ExceptionCode.java
+++ b/src/java/org/apache/cassandra/exceptions/ExceptionCode.java
@@ -39,6 +39,9 @@
     TRUNCATE_ERROR  (0x1003),
     WRITE_TIMEOUT   (0x1100),
     READ_TIMEOUT    (0x1200),
+    READ_FAILURE    (0x1300),
+    FUNCTION_FAILURE(0x1400),
+    WRITE_FAILURE   (0x1500),
 
     // 2xx: problem validating the request
     SYNTAX_ERROR    (0x2000),
@@ -49,7 +52,7 @@
     UNPREPARED      (0x2500);
 
     public final int value;
-    private static final Map<Integer, ExceptionCode> valueToCode = new HashMap<Integer, ExceptionCode>(ExceptionCode.values().length);
+    private static final Map<Integer, ExceptionCode> valueToCode = new HashMap<>(ExceptionCode.values().length);
     static
     {
         for (ExceptionCode code : ExceptionCode.values())
diff --git a/src/java/org/apache/cassandra/exceptions/FunctionExecutionException.java b/src/java/org/apache/cassandra/exceptions/FunctionExecutionException.java
new file mode 100644
index 0000000..e743fde
--- /dev/null
+++ b/src/java/org/apache/cassandra/exceptions/FunctionExecutionException.java
@@ -0,0 +1,47 @@
+/*
+ * 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.exceptions;
+
+import java.util.List;
+
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.functions.FunctionName;
+import org.apache.cassandra.db.marshal.AbstractType;
+
+public class FunctionExecutionException extends RequestExecutionException
+{
+    public final FunctionName functionName;
+    public final List<String> argTypes;
+    public final String detail;
+
+    public static FunctionExecutionException create(Function function, Throwable cause)
+    {
+        List<String> cqlTypes = AbstractType.asCQLTypeStringList(function.argTypes());
+        FunctionExecutionException fee = new FunctionExecutionException(function.name(), cqlTypes, cause.toString());
+        fee.initCause(cause);
+        return fee;
+    }
+
+    public FunctionExecutionException(FunctionName functionName, List<String> argTypes, String detail)
+    {
+        super(ExceptionCode.FUNCTION_FAILURE, "execution of '" + functionName + argTypes + "' failed: " + detail);
+        this.functionName = functionName;
+        this.argTypes = argTypes;
+        this.detail = detail;
+    }
+}
diff --git a/src/java/org/apache/cassandra/exceptions/ReadFailureException.java b/src/java/org/apache/cassandra/exceptions/ReadFailureException.java
new file mode 100644
index 0000000..91cf580
--- /dev/null
+++ b/src/java/org/apache/cassandra/exceptions/ReadFailureException.java
@@ -0,0 +1,31 @@
+/*
+ * 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.exceptions;
+
+import org.apache.cassandra.db.ConsistencyLevel;
+
+public class ReadFailureException extends RequestFailureException
+{
+    public final boolean dataPresent;
+
+    public ReadFailureException(ConsistencyLevel consistency, int received, int failures, int blockFor, boolean dataPresent)
+    {
+        super(ExceptionCode.READ_FAILURE, consistency, received, failures, blockFor);
+        this.dataPresent = dataPresent;
+    }
+}
diff --git a/src/java/org/apache/cassandra/exceptions/RepairException.java b/src/java/org/apache/cassandra/exceptions/RepairException.java
index 832a6d8..2f5f2c1 100644
--- a/src/java/org/apache/cassandra/exceptions/RepairException.java
+++ b/src/java/org/apache/cassandra/exceptions/RepairException.java
@@ -41,6 +41,6 @@
     @Override
     public String getMessage()
     {
-        return desc.toString() + " " + super.getMessage();
+        return desc + " " + super.getMessage();
     }
 }
diff --git a/src/java/org/apache/cassandra/exceptions/RequestFailureException.java b/src/java/org/apache/cassandra/exceptions/RequestFailureException.java
new file mode 100644
index 0000000..6b8b40f
--- /dev/null
+++ b/src/java/org/apache/cassandra/exceptions/RequestFailureException.java
@@ -0,0 +1,37 @@
+/*
+ * 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.exceptions;
+
+import org.apache.cassandra.db.ConsistencyLevel;
+
+public class RequestFailureException extends RequestExecutionException
+{
+    public final ConsistencyLevel consistency;
+    public final int received;
+    public final int failures;
+    public final int blockFor;
+
+    protected RequestFailureException(ExceptionCode code, ConsistencyLevel consistency, int received, int failures, int blockFor)
+    {
+        super(code, String.format("Operation failed - received %d responses and %d failures", received, failures));
+        this.consistency = consistency;
+        this.received = received;
+        this.failures = failures;
+        this.blockFor = blockFor;
+    }
+}
diff --git a/src/java/org/apache/cassandra/exceptions/StartupException.java b/src/java/org/apache/cassandra/exceptions/StartupException.java
new file mode 100644
index 0000000..ec4890f
--- /dev/null
+++ b/src/java/org/apache/cassandra/exceptions/StartupException.java
@@ -0,0 +1,39 @@
+/*
+ * 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.exceptions;
+
+/**
+ * An exception thrown during system startup, indicating the environment or system
+ * is not in a valid state to startup.
+ */
+public class StartupException extends Exception
+{
+    public final int returnCode;
+
+    public StartupException(int returnCode, String message)
+    {
+        super(message);
+        this.returnCode = returnCode;
+    }
+
+    public StartupException(int returnCode, String message, Throwable cause)
+    {
+        super(message, cause);
+        this.returnCode = returnCode;
+    }
+}
diff --git a/src/java/org/apache/cassandra/exceptions/UnrecognizedEntityException.java b/src/java/org/apache/cassandra/exceptions/UnrecognizedEntityException.java
new file mode 100644
index 0000000..e8392e9
--- /dev/null
+++ b/src/java/org/apache/cassandra/exceptions/UnrecognizedEntityException.java
@@ -0,0 +1,49 @@
+/*
+ * 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.exceptions;
+
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.Relation;
+
+/**
+ * Exception thrown when an entity is not recognized within a relation.
+ */
+public final class UnrecognizedEntityException extends InvalidRequestException
+{
+    /**
+     * The unrecognized entity.
+     */
+    public final ColumnIdentifier entity;
+
+    /**
+     * The entity relation.
+     */
+    public final Relation relation;
+
+    /**
+     * Creates a new <code>UnrecognizedEntityException</code>.
+     * @param entity the unrecognized entity
+     * @param relation the entity relation
+     */
+    public UnrecognizedEntityException(ColumnIdentifier entity, Relation relation)
+    {
+        super(String.format("Undefined name %s in where clause ('%s')", entity, relation));
+        this.entity = entity;
+        this.relation = relation;
+    }
+}
diff --git a/src/java/org/apache/cassandra/exceptions/WriteFailureException.java b/src/java/org/apache/cassandra/exceptions/WriteFailureException.java
new file mode 100644
index 0000000..24de9b1
--- /dev/null
+++ b/src/java/org/apache/cassandra/exceptions/WriteFailureException.java
@@ -0,0 +1,32 @@
+/*
+ * 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.exceptions;
+
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.db.WriteType;
+
+public class WriteFailureException extends RequestFailureException
+{
+    public final WriteType writeType;
+
+    public WriteFailureException(ConsistencyLevel consistency, int received, int failures, int blockFor, WriteType writeType)
+    {
+        super(ExceptionCode.WRITE_FAILURE, consistency, received, failures, blockFor);
+        this.writeType = writeType;
+    }
+}
diff --git a/src/java/org/apache/cassandra/gms/ApplicationState.java b/src/java/org/apache/cassandra/gms/ApplicationState.java
index 777dfc5..ade9208 100644
--- a/src/java/org/apache/cassandra/gms/ApplicationState.java
+++ b/src/java/org/apache/cassandra/gms/ApplicationState.java
@@ -33,6 +33,7 @@
     NET_VERSION,
     HOST_ID,
     TOKENS,
+    RPC_READY,
     // pad to allow adding new states to existing cluster
     X1,
     X2,
diff --git a/src/java/org/apache/cassandra/gms/EchoMessage.java b/src/java/org/apache/cassandra/gms/EchoMessage.java
index 444278f..2d4c095 100644
--- a/src/java/org/apache/cassandra/gms/EchoMessage.java
+++ b/src/java/org/apache/cassandra/gms/EchoMessage.java
@@ -27,10 +27,16 @@
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataOutputPlus;
 
-public class EchoMessage
+public final class EchoMessage
 {
-    public static IVersionedSerializer<EchoMessage> serializer = new EchoMessageSerializer();
+	public static final EchoMessage instance = new EchoMessage();
+	
+    public static final IVersionedSerializer<EchoMessage> serializer = new EchoMessageSerializer();
 
+	private EchoMessage()
+	{
+	}
+	
     public static class EchoMessageSerializer implements IVersionedSerializer<EchoMessage>
     {
         public void serialize(EchoMessage t, DataOutputPlus out, int version) throws IOException
@@ -39,7 +45,7 @@
 
         public EchoMessage deserialize(DataInput in, int version) throws IOException
         {
-            return new EchoMessage();
+            return EchoMessage.instance;
         }
 
         public long serializedSize(EchoMessage t, int version)
diff --git a/src/java/org/apache/cassandra/gms/EndpointState.java b/src/java/org/apache/cassandra/gms/EndpointState.java
index 3e29295..931da8d 100644
--- a/src/java/org/apache/cassandra/gms/EndpointState.java
+++ b/src/java/org/apache/cassandra/gms/EndpointState.java
@@ -138,6 +138,23 @@
         isAlive = false;
     }
 
+    public boolean isRpcReady()
+    {
+        VersionedValue rpcState = getApplicationState(ApplicationState.RPC_READY);
+        return rpcState != null && Boolean.parseBoolean(rpcState.value);
+    }
+
+    public String getStatus()
+    {
+        VersionedValue status = getApplicationState(ApplicationState.STATUS);
+        if (status == null)
+            return "";
+
+        String[] pieces = status.value.split(VersionedValue.DELIMITER_STR, -1);
+        assert (pieces.length > 0);
+        return pieces[0];
+    }
+
     public String toString()
     {
         return "EndpointState: HeartBeatState = " + hbState + ", AppStateMap = " + applicationState.get();
diff --git a/src/java/org/apache/cassandra/gms/FailureDetector.java b/src/java/org/apache/cassandra/gms/FailureDetector.java
index b2fe9d3..b9b7944 100644
--- a/src/java/org/apache/cassandra/gms/FailureDetector.java
+++ b/src/java/org/apache/cassandra/gms/FailureDetector.java
@@ -22,10 +22,13 @@
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.TimeUnit;
 import javax.management.MBeanServer;
 import javax.management.ObjectName;
+import javax.management.openmbean.CompositeData;
+import javax.management.openmbean.*;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -46,6 +49,7 @@
     public static final String MBEAN_NAME = "org.apache.cassandra.net:type=FailureDetector";
     private static final int SAMPLE_SIZE = 1000;
     protected static final long INITIAL_VALUE_NANOS = TimeUnit.NANOSECONDS.convert(getInitialValue(), TimeUnit.MILLISECONDS);
+    private static final int DEBUG_PERCENTAGE = 80; // if the phi is larger than this percentage of the max, log a debug message
     private static final long DEFAULT_MAX_PAUSE = 5000L * 1000000L; // 5 seconds
     private static final long MAX_LOCAL_PAUSE_IN_NANOS = getMaxLocalPause();
     private long lastInterpret = System.nanoTime();
@@ -71,8 +75,8 @@
     // change.
     private final double PHI_FACTOR = 1.0 / Math.log(10.0); // 0.434...
 
-    private final Map<InetAddress, ArrivalWindow> arrivalSamples = new Hashtable<InetAddress, ArrivalWindow>();
-    private final List<IFailureDetectionEventListener> fdEvntListeners = new CopyOnWriteArrayList<IFailureDetectionEventListener>();
+    private final ConcurrentHashMap<InetAddress, ArrivalWindow> arrivalSamples = new ConcurrentHashMap<>();
+    private final List<IFailureDetectionEventListener> fdEvntListeners = new CopyOnWriteArrayList<>();
 
     public FailureDetector()
     {
@@ -148,6 +152,34 @@
         return count;
     }
 
+    @Override
+    public TabularData getPhiValues() throws OpenDataException
+    {
+        final CompositeType ct = new CompositeType("Node", "Node",
+                new String[]{"Endpoint", "PHI"},
+                new String[]{"IP of the endpoint", "PHI value"},
+                new OpenType[]{SimpleType.STRING, SimpleType.DOUBLE});
+        final TabularDataSupport results = new TabularDataSupport(new TabularType("PhiList", "PhiList", ct, new String[]{"Endpoint"}));
+
+        for (final Map.Entry<InetAddress, ArrivalWindow> entry : arrivalSamples.entrySet())
+        {
+            final ArrivalWindow window = entry.getValue();
+            if (window.mean() > 0)
+            {
+                final double phi = window.getLastReportedPhi();
+                if (phi != Double.MIN_VALUE)
+                {
+                    // returned values are scaled by PHI_FACTOR so that the are on the same scale as PhiConvictThreshold
+                    final CompositeData data = new CompositeDataSupport(ct,
+                            new String[]{"Endpoint", "PHI"},
+                            new Object[]{entry.getKey().toString(), phi * PHI_FACTOR});
+                    results.put(data);
+                }
+            }
+        }
+        return results;
+    }
+
     public String getEndpointState(String address) throws UnknownHostException
     {
         StringBuilder sb = new StringBuilder();
@@ -184,20 +216,14 @@
     {
         File file = FileUtils.createTempFile("failuredetector-", ".dat");
 
-        OutputStream os = null;
-        try
+        try (OutputStream os = new BufferedOutputStream(new FileOutputStream(file, true)))
         {
-            os = new BufferedOutputStream(new FileOutputStream(file, true));
             os.write(toString().getBytes());
         }
         catch (IOException e)
         {
             throw new FSWriteError(e, file);
         }
-        finally
-        {
-            FileUtils.closeQuietly(os);
-        }
     }
 
     public void setPhiConvictThreshold(double phi)
@@ -226,8 +252,6 @@
 
     public void report(InetAddress ep)
     {
-        if (logger.isTraceEnabled())
-            logger.trace("reporting {}", ep);
         long now = System.nanoTime();
         ArrivalWindow heartbeatWindow = arrivalSamples.get(ep);
         if (heartbeatWindow == null)
@@ -235,12 +259,17 @@
             // avoid adding an empty ArrivalWindow to the Map
             heartbeatWindow = new ArrivalWindow(SAMPLE_SIZE);
             heartbeatWindow.add(now, ep);
-            arrivalSamples.put(ep, heartbeatWindow);
+            heartbeatWindow = arrivalSamples.putIfAbsent(ep, heartbeatWindow);
+            if (heartbeatWindow != null)
+                heartbeatWindow.add(now, ep);
         }
         else
         {
             heartbeatWindow.add(now, ep);
         }
+
+        if (logger.isTraceEnabled() && heartbeatWindow != null)
+            logger.trace("Average for {} is {}", ep, heartbeatWindow.mean());
     }
 
     public void interpret(InetAddress ep)
@@ -266,17 +295,26 @@
         }
         double phi = hbWnd.phi(now);
         if (logger.isTraceEnabled())
-            logger.trace("PHI for " + ep + " : " + phi);
+            logger.trace("PHI for {} : {}", ep, phi);
 
         if (PHI_FACTOR * phi > getPhiConvictThreshold())
         {
-            logger.trace("notifying listeners that {} is down", ep);
-            logger.trace("intervals: {} mean: {}", hbWnd, hbWnd.mean());
+            if (logger.isTraceEnabled())
+                logger.trace("Node {} phi {} > {}; intervals: {} mean: {}", new Object[]{ep, PHI_FACTOR * phi, getPhiConvictThreshold(), hbWnd, hbWnd.mean()});
             for (IFailureDetectionEventListener listener : fdEvntListeners)
             {
                 listener.convict(ep, phi);
             }
         }
+        else if (logger.isDebugEnabled() && (PHI_FACTOR * phi * DEBUG_PERCENTAGE / 100.0 > getPhiConvictThreshold()))
+        {
+            logger.debug("PHI for {} : {}", ep, phi);
+        }
+        else if (logger.isTraceEnabled())
+        {
+            logger.trace("PHI for {} : {}", ep, phi);
+            logger.trace("mean for {} : {}", ep, hbWnd.mean());
+        }
     }
 
     public void forceConviction(InetAddress ep)
@@ -313,16 +351,12 @@
         {
             ArrivalWindow hWnd = arrivalSamples.get(ep);
             sb.append(ep + " : ");
-            sb.append(hWnd.toString());
+            sb.append(hWnd);
             sb.append(System.getProperty("line.separator"));
         }
         sb.append("-----------------------------------------------------------------------");
         return sb.toString();
     }
-
-    public static void main(String[] args)
-    {
-    }
 }
 
 /*
@@ -379,12 +413,7 @@
     private static final Logger logger = LoggerFactory.getLogger(ArrivalWindow.class);
     private long tLast = 0L;
     private final ArrayBackedBoundedStats arrivalIntervals;
-
-    // this is useless except to provide backwards compatibility in phi_convict_threshold,
-    // because everyone seems pretty accustomed to the default of 8, and users who have
-    // already tuned their phi_convict_threshold for their own environments won't need to
-    // change.
-    private final double PHI_FACTOR = 1.0 / Math.log(10.0);
+    private double lastReportedPhi = Double.MIN_VALUE;
 
     // in the event of a long partition, never record an interval longer than the rpc timeout,
     // since if a host is regularly experiencing connectivity problems lasting this long we'd
@@ -418,9 +447,14 @@
         {
             long interArrivalTime = (value - tLast);
             if (interArrivalTime <= MAX_INTERVAL_IN_NANO)
+            {
                 arrivalIntervals.add(interArrivalTime);
+                logger.trace("Reporting interval time of {} for {}", interArrivalTime, ep);
+            }
             else
+            {
                 logger.debug("Ignoring interval time of {} for {}", interArrivalTime, ep);
+            }
         }
         else
         {
@@ -442,7 +476,13 @@
     {
         assert arrivalIntervals.mean() > 0 && tLast > 0; // should not be called before any samples arrive
         long t = tnow - tLast;
-        return t / mean();
+        lastReportedPhi = t / mean();
+        return lastReportedPhi;
+    }
+
+    double getLastReportedPhi()
+    {
+        return lastReportedPhi;
     }
 
     public String toString()
diff --git a/src/java/org/apache/cassandra/gms/FailureDetectorMBean.java b/src/java/org/apache/cassandra/gms/FailureDetectorMBean.java
index 45250b4..23fae3a 100644
--- a/src/java/org/apache/cassandra/gms/FailureDetectorMBean.java
+++ b/src/java/org/apache/cassandra/gms/FailureDetectorMBean.java
@@ -20,6 +20,9 @@
 import java.net.UnknownHostException;
 import java.util.Map;
 
+import javax.management.openmbean.OpenDataException;
+import javax.management.openmbean.TabularData;
+
 public interface FailureDetectorMBean
 {
     public void dumpInterArrivalTimes();
@@ -37,4 +40,6 @@
     public int getDownEndpointCount();
 
     public int getUpEndpointCount();
+
+    public TabularData getPhiValues() throws OpenDataException;
 }
diff --git a/src/java/org/apache/cassandra/gms/GossipDigestAckVerbHandler.java b/src/java/org/apache/cassandra/gms/GossipDigestAckVerbHandler.java
index 9f69a94..59060f8 100644
--- a/src/java/org/apache/cassandra/gms/GossipDigestAckVerbHandler.java
+++ b/src/java/org/apache/cassandra/gms/GossipDigestAckVerbHandler.java
@@ -51,21 +51,31 @@
         Map<InetAddress, EndpointState> epStateMap = gDigestAckMessage.getEndpointStateMap();
         logger.trace("Received ack with {} digests and {} states", gDigestList.size(), epStateMap.size());
 
-        if (epStateMap.size() > 0)
-        {
-            /* Notify the Failure Detector */
-            Gossiper.instance.notifyFailureDetector(epStateMap);
-            Gossiper.instance.applyStateLocally(epStateMap);
-        }
-
         if (Gossiper.instance.isInShadowRound())
         {
             if (logger.isDebugEnabled())
                 logger.debug("Finishing shadow round with {}", from);
-            Gossiper.instance.finishShadowRound();
+            Gossiper.instance.finishShadowRound(epStateMap);
             return; // don't bother doing anything else, we have what we came for
         }
 
+        if (epStateMap.size() > 0)
+        {
+            // Ignore any GossipDigestAck messages that we handle before a regular GossipDigestSyn has been send.
+            // This will prevent Acks from leaking over from the shadow round that are not actual part of
+            // the regular gossip conversation.
+            if ((System.nanoTime() - Gossiper.instance.firstSynSendAt) < 0 || Gossiper.instance.firstSynSendAt == 0)
+            {
+                if (logger.isTraceEnabled())
+                    logger.trace("Ignoring unrequested GossipDigestAck from {}", from);
+                return;
+            }
+
+            /* Notify the Failure Detector */
+            Gossiper.instance.notifyFailureDetector(epStateMap);
+            Gossiper.instance.applyStateLocally(epStateMap);
+        }
+
         /* Get the state required to send to this gossipee - construct GossipDigestAck2Message */
         Map<InetAddress, EndpointState> deltaEpStateMap = new HashMap<InetAddress, EndpointState>();
         for (GossipDigest gDigest : gDigestList)
diff --git a/src/java/org/apache/cassandra/gms/GossipDigestSynVerbHandler.java b/src/java/org/apache/cassandra/gms/GossipDigestSynVerbHandler.java
index 6b175de..1c67570 100644
--- a/src/java/org/apache/cassandra/gms/GossipDigestSynVerbHandler.java
+++ b/src/java/org/apache/cassandra/gms/GossipDigestSynVerbHandler.java
@@ -68,7 +68,7 @@
                 sb.append(gDigest);
                 sb.append(" ");
             }
-            logger.trace("Gossip syn digests are : " + sb.toString());
+            logger.trace("Gossip syn digests are : {}", sb);
         }
 
         doSort(gDigestList);
diff --git a/src/java/org/apache/cassandra/gms/Gossiper.java b/src/java/org/apache/cassandra/gms/Gossiper.java
index 889806c..c2eccba 100644
--- a/src/java/org/apache/cassandra/gms/Gossiper.java
+++ b/src/java/org/apache/cassandra/gms/Gossiper.java
@@ -30,6 +30,7 @@
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
 import com.google.common.util.concurrent.Uninterruptibles;
 
 import org.apache.cassandra.utils.Pair;
@@ -65,7 +66,7 @@
 
 public class Gossiper implements IFailureDetectionEventListener, GossiperMBean
 {
-    private static final String MBEAN_NAME = "org.apache.cassandra.net:type=Gossiper";
+    public static final String MBEAN_NAME = "org.apache.cassandra.net:type=Gossiper";
 
     private static final DebuggableScheduledThreadPoolExecutor executor = new DebuggableScheduledThreadPoolExecutor("GossipTasks");
 
@@ -76,6 +77,7 @@
     static {
         SILENT_SHUTDOWN_STATES.addAll(DEAD_STATES);
         SILENT_SHUTDOWN_STATES.add(VersionedValue.STATUS_BOOTSTRAPPING);
+        SILENT_SHUTDOWN_STATES.add(VersionedValue.STATUS_BOOTSTRAPPING_REPLACE);
     }
 
     private volatile ScheduledFuture<?> scheduledGossipTask;
@@ -85,11 +87,14 @@
     private static final Logger logger = LoggerFactory.getLogger(Gossiper.class);
     public static final Gossiper instance = new Gossiper();
 
+    // Timestamp to prevent processing any in-flight messages for we've not send any SYN yet, see CASSANDRA-12653.
+    volatile long firstSynSendAt = 0L;
+
     public static final long aVeryLongTime = 259200 * 1000; // 3 days
 
     // Maximimum difference between generation value and local time we are willing to accept about a peer
     static final int MAX_GENERATION_DIFFERENCE = 86400 * 365;
-    private long FatClientTimeout;
+    private long fatClientTimeout;
     private final Random random = new Random();
     private final Comparator<InetAddress> inetcomparator = new Comparator<InetAddress>()
     {
@@ -124,6 +129,9 @@
 
     private volatile boolean inShadowRound = false;
 
+    // endpoint states as gathered during shadow round
+    private final Map<InetAddress, EndpointState> endpointShadowStateMap = new ConcurrentHashMap<>();
+
     private volatile long lastProcessedMessageAt = System.currentTimeMillis();
 
     private class GossipTask implements Runnable
@@ -140,7 +148,7 @@
                 /* Update the local heartbeat counter. */
                 endpointStateMap.get(FBUtilities.getBroadcastAddress()).getHeartBeatState().updateHeartBeat();
                 if (logger.isTraceEnabled())
-                    logger.trace("My heartbeat is now " + endpointStateMap.get(FBUtilities.getBroadcastAddress()).getHeartBeatState().getHeartBeatVersion());
+                    logger.trace("My heartbeat is now {}", endpointStateMap.get(FBUtilities.getBroadcastAddress()).getHeartBeatState().getHeartBeatVersion());
                 final List<GossipDigest> gDigests = new ArrayList<GossipDigest>();
                 Gossiper.instance.makeRandomGossipDigest(gDigests);
 
@@ -156,7 +164,7 @@
                     boolean gossipedToSeed = doGossipToLiveMember(message);
 
                     /* Gossip to some unreachable member with some probability to check if he is back up */
-                    doGossipToUnreachableMember(message);
+                    maybeGossipToUnreachableMember(message);
 
                     /* Gossip to a seed if we did not do so above, or we have seen less nodes
                        than there are seeds.  This prevents partitions where each group of nodes
@@ -175,7 +183,7 @@
 
                        See CASSANDRA-150 for more exposition. */
                     if (!gossipedToSeed || liveEndpoints.size() < seeds.size())
-                        doGossipToSeed(message);
+                        maybeGossipToSeed(message);
 
                     doStatusCheck();
                 }
@@ -195,7 +203,7 @@
     private Gossiper()
     {
         // half of QUARATINE_DELAY, to ensure justRemovedEndpoints has enough leeway to prevent re-gossip
-        FatClientTimeout = (long) (QUARANTINE_DELAY / 2);
+        fatClientTimeout = (QUARANTINE_DELAY / 2);
         /* register with the Failure Detector for receiving Failure detector events */
         FailureDetector.instance.registerFailureDetectionEventListener(this);
 
@@ -333,10 +341,12 @@
         if (epState == null)
             return;
 
-        logger.debug("Convicting {} with status {} - alive {}", endpoint, getGossipStatus(epState), epState.isAlive());
         if (!epState.isAlive())
             return;
 
+        logger.debug("Convicting {} with status {} - alive {}", endpoint, getGossipStatus(epState), epState.isAlive());
+
+
         if (isShutdown(endpoint))
         {
             markAsShutdown(endpoint);
@@ -496,7 +506,7 @@
                 sb.append(gDigest);
                 sb.append(" ");
             }
-            logger.trace("Gossip Digests are : " + sb.toString());
+            logger.trace("Gossip Digests are : {}", sb);
         }
     }
 
@@ -552,16 +562,21 @@
         Uninterruptibles.sleepUninterruptibly(intervalInMillis * 2, TimeUnit.MILLISECONDS);
     }
 
+    public void unsafeAssassinateEndpoint(String address) throws UnknownHostException
+    {
+        logger.warn("Gossiper.unsafeAssassinateEndpoint is deprecated and will be removed in the next release; use assassinateEndpoint instead");
+        assassinateEndpoint(address);
+    }
+
     /**
      * Do not call this method unless you know what you are doing.
      * It will try extremely hard to obliterate any endpoint from the ring,
      * even if it does not know about it.
-     * This should only ever be called by human via JMX.
      *
      * @param address
      * @throws UnknownHostException
      */
-    public void unsafeAssassinateEndpoint(String address) throws UnknownHostException
+    public void assassinateEndpoint(String address) throws UnknownHostException
     {
         InetAddress endpoint = InetAddress.getByName(address);
         EndpointState epState = endpointStateMap.get(endpoint);
@@ -586,6 +601,7 @@
                 tokens = Collections.singletonList(StorageService.getPartitioner().getRandomToken());
             }
             int generation = epState.getHeartBeatState().getGeneration();
+            int heartbeat = epState.getHeartBeatState().getHeartBeatVersion();
             logger.info("Sleeping for {}ms to ensure {} does not change", StorageService.RING_DELAY, endpoint);
             Uninterruptibles.sleepUninterruptibly(StorageService.RING_DELAY, TimeUnit.MILLISECONDS);
             // make sure it did not change
@@ -593,7 +609,9 @@
             if (newState == null)
                 logger.warn("Endpoint {} disappeared while trying to assassinate, continuing anyway", endpoint);
             else if (newState.getHeartBeatState().getGeneration() != generation)
-                throw new RuntimeException("Endpoint " + endpoint + " generation changed while trying to remove it");
+                throw new RuntimeException("Endpoint still alive: " + endpoint + " generation changed while trying to assassinate it");
+            else if (newState.getHeartBeatState().getHeartBeatVersion() != heartbeat)
+                throw new RuntimeException("Endpoint still alive: " + endpoint + " heartbeat changed while trying to assassinate it");
             epState.updateTimestamp(); // make sure we don't evict it too soon
             epState.getHeartBeatState().forceNewerGenerationUnsafe();
         }
@@ -634,6 +652,8 @@
         InetAddress to = liveEndpoints.get(index);
         if (logger.isTraceEnabled())
             logger.trace("Sending a GossipDigestSyn to {} ...", to);
+        if (firstSynSendAt == 0)
+            firstSynSendAt = System.nanoTime();
         MessagingService.instance().sendOneWay(message, to);
         return seeds.contains(to);
     }
@@ -648,7 +668,7 @@
     }
 
     /* Sends a Gossip message to an unreachable member */
-    private void doGossipToUnreachableMember(MessageOut<GossipDigestSyn> message)
+    private void maybeGossipToUnreachableMember(MessageOut<GossipDigestSyn> message)
     {
         double liveEndpointCount = liveEndpoints.size();
         double unreachableEndpointCount = unreachableEndpoints.size();
@@ -662,8 +682,8 @@
         }
     }
 
-    /* Gossip to a seed for facilitating partition healing */
-    private void doGossipToSeed(MessageOut<GossipDigestSyn> prod)
+    /* Possibly gossip to a seed for facilitating partition healing */
+    private void maybeGossipToSeed(MessageOut<GossipDigestSyn> prod)
     {
         int size = seeds.size();
         if (size > 0)
@@ -688,13 +708,7 @@
         }
     }
 
-    /**
-     * A fat client is a node that has not joined the ring, therefore acting as a coordinator only.
-     *
-     * @param endpoint - the endpoint to check
-     * @return true if it is a fat client
-     */
-    public boolean isFatClient(InetAddress endpoint)
+    public boolean isGossipOnlyMember(InetAddress endpoint)
     {
         EndpointState epState = endpointStateMap.get(endpoint);
         if (epState == null)
@@ -708,11 +722,12 @@
      * Check if this endpoint can safely bootstrap into the cluster.
      *
      * @param endpoint - the endpoint to check
+     * @param epStates - endpoint states in the cluster
      * @return true if the endpoint can join the cluster
      */
-    public boolean isSafeForBootstrap(InetAddress endpoint)
+    public boolean isSafeForBootstrap(InetAddress endpoint, Map<InetAddress, EndpointState> epStates)
     {
-        EndpointState epState = endpointStateMap.get(endpoint);
+        EndpointState epState = epStates.get(endpoint);
 
         // if there's no previous state, or the node was previously removed from the cluster, we're good
         if (epState == null || isDeadState(epState))
@@ -736,7 +751,7 @@
         long now = System.currentTimeMillis();
         long nowNano = System.nanoTime();
 
-        long pending = ((JMXEnabledThreadPoolExecutor) StageManager.getStage(Stage.GOSSIP)).getPendingTasks();
+        long pending = ((JMXEnabledThreadPoolExecutor) StageManager.getStage(Stage.GOSSIP)).metrics.pendingTasks.getValue();
         if (pending > 0 && lastProcessedMessageAt < now - 1000)
         {
             // if some new messages just arrived, give the executor some time to work on them
@@ -762,11 +777,11 @@
             {
                 // check if this is a fat client. fat clients are removed automatically from
                 // gossip after FatClientTimeout.  Do not remove dead states here.
-                if (isFatClient(endpoint)
+                if (isGossipOnlyMember(endpoint)
                     && !justRemovedEndpoints.containsKey(endpoint)
-                    && TimeUnit.NANOSECONDS.toMillis(nowNano - epState.getUpdateTimestamp()) > FatClientTimeout)
+                    && TimeUnit.NANOSECONDS.toMillis(nowNano - epState.getUpdateTimestamp()) > fatClientTimeout)
                 {
-                    logger.info("FatClient {} has been silent for {}ms, removing from gossip", endpoint, FatClientTimeout);
+                    logger.info("FatClient {} has been silent for {}ms, removing from gossip", endpoint, fatClientTimeout);
                     removeEndpoint(endpoint); // will put it in justRemovedEndpoints to respect quarantine delay
                     evictFromMembership(endpoint); // can get rid of the state immediately
                 }
@@ -811,38 +826,19 @@
         return endpointStateMap.get(ep);
     }
 
-    // removes ALL endpoint states; should only be called after shadow gossip
-    public void resetEndpointStateMap()
-    {
-        endpointStateMap.clear();
-        unreachableEndpoints.clear();
-        liveEndpoints.clear();
-    }
-
     public Set<Entry<InetAddress, EndpointState>> getEndpointStates()
     {
         return endpointStateMap.entrySet();
     }
 
-    public boolean usesHostId(InetAddress endpoint)
-    {
-        if (MessagingService.instance().knowsVersion(endpoint))
-            return true;
-        else if (getEndpointStateForEndpoint(endpoint).getApplicationState(ApplicationState.NET_VERSION) != null)
-            return true;
-        return false;
-    }
-
-    public boolean usesVnodes(InetAddress endpoint)
-    {
-        return usesHostId(endpoint) && getEndpointStateForEndpoint(endpoint).getApplicationState(ApplicationState.TOKENS) != null;
-    }
-
     public UUID getHostId(InetAddress endpoint)
     {
-        if (!usesHostId(endpoint))
-            throw new RuntimeException("Host " + endpoint + " does not use new-style tokens!");
-        return UUID.fromString(getEndpointStateForEndpoint(endpoint).getApplicationState(ApplicationState.HOST_ID).value);
+        return getHostId(endpoint, endpointStateMap);
+    }
+
+    public UUID getHostId(InetAddress endpoint, Map<InetAddress, EndpointState> epStates)
+    {
+        return UUID.fromString(epStates.get(endpoint).getApplicationState(ApplicationState.HOST_ID).value);
     }
 
     EndpointState getStateForVersionBiggerThan(InetAddress forEndpoint, int version)
@@ -865,7 +861,7 @@
             {
                 reqdEndpointState = new EndpointState(epState.getHeartBeatState());
                 if (logger.isTraceEnabled())
-                    logger.trace("local heartbeat version " + localHbVersion + " greater than " + version + " for " + forEndpoint);
+                    logger.trace("local heartbeat version {} greater than {} for {}", localHbVersion, version, forEndpoint);
             }
             /* Accumulate all application states whose versions are greater than "version" variable */
             Map<ApplicationState, VersionedValue> states = new EnumMap<>(ApplicationState.class);
@@ -880,7 +876,7 @@
                     }
                     final ApplicationState key = entry.getKey();
                     if (logger.isTraceEnabled())
-                        logger.trace("Adding state " + key + ": " + value.value);
+                        logger.trace("Adding state {}: {}" , key, value.value);
 
                     states.put(key, value);
                 }
@@ -961,7 +957,7 @@
 
         localState.markDead();
 
-        MessageOut<EchoMessage> echoMessage = new MessageOut<EchoMessage>(MessagingService.Verb.ECHO, new EchoMessage(), EchoMessage.serializer);
+        MessageOut<EchoMessage> echoMessage = new MessageOut<EchoMessage>(MessagingService.Verb.ECHO, EchoMessage.instance, EchoMessage.serializer);
         logger.trace("Sending a EchoMessage to {}", addr);
         IAsyncCallback echoHandler = new IAsyncCallback()
         {
@@ -989,12 +985,12 @@
         liveEndpoints.add(addr);
         unreachableEndpoints.remove(addr);
         expireTimeEndpointMap.remove(addr);
-        logger.debug("removing expire time for endpoint : " + addr);
+        logger.debug("removing expire time for endpoint : {}", addr);
         logger.info("InetAddress {} is now UP", addr);
         for (IEndpointStateChangeSubscriber subscriber : subscribers)
             subscriber.onAlive(addr, localState);
         if (logger.isTraceEnabled())
-            logger.trace("Notified " + subscribers);
+            logger.trace("Notified {}", subscribers);
     }
 
     @VisibleForTesting
@@ -1009,7 +1005,7 @@
         for (IEndpointStateChangeSubscriber subscriber : subscribers)
             subscriber.onDead(addr, localState);
         if (logger.isTraceEnabled())
-            logger.trace("Notified " + subscribers);
+            logger.trace("Notified {}", subscribers);
     }
 
     /**
@@ -1029,7 +1025,7 @@
                 logger.info("Node {} is now part of the cluster", ep);
         }
         if (logger.isTraceEnabled())
-            logger.trace("Adding endpoint state for " + ep);
+            logger.trace("Adding endpoint state for {}", ep);
         endpointStateMap.put(ep, epState);
 
         if (localEpState != null)
@@ -1052,6 +1048,14 @@
             markAsShutdown(ep);
     }
 
+    public boolean isAlive(InetAddress endpoint)
+    {
+        EndpointState epState = getEndpointStateForEndpoint(endpoint);
+        if (epState == null)
+            return false;
+        return epState.isAlive() && !isDeadState(epState);
+    }
+
     public boolean isDeadState(EndpointState epState)
     {
         String status = getGossipStatus(epState);
@@ -1091,7 +1095,7 @@
             if (justRemovedEndpoints.containsKey(ep))
             {
                 if (logger.isTraceEnabled())
-                    logger.trace("Ignoring gossip for " + ep + " because it is quarantined");
+                    logger.trace("Ignoring gossip for {} because it is quarantined", ep);
                 continue;
             }
 
@@ -1108,7 +1112,7 @@
                 int remoteGeneration = remoteState.getHeartBeatState().getGeneration();
                 long localTime = System.currentTimeMillis()/1000;
                 if (logger.isTraceEnabled())
-                    logger.trace(ep + "local generation " + localGeneration + ", remote generation " + remoteGeneration);
+                    logger.trace("{} local generation {}, remote generation {}", ep, localGeneration, remoteGeneration);
 
                 // We measure generation drift against local time, based on the fact that generation is initialized by time
                 if (remoteGeneration > localTime + MAX_GENERATION_DIFFERENCE)
@@ -1119,7 +1123,7 @@
                 else if (remoteGeneration > localGeneration)
                 {
                     if (logger.isTraceEnabled())
-                        logger.trace("Updating heartbeat state generation to " + remoteGeneration + " from " + localGeneration + " for " + ep);
+                        logger.trace("Updating heartbeat state generation to {} from {} for {}", remoteGeneration, localGeneration, ep);
                     // major state change will handle the update by inserting the remote state directly
                     handleMajorStateChange(ep, remoteState);
                 }
@@ -1134,7 +1138,7 @@
                         applyNewStates(ep, localEpStatePtr, remoteState);
                     }
                     else if (logger.isTraceEnabled())
-                            logger.trace("Ignoring remote version " + remoteMaxVersion + " <= " + localMaxVersion + " for " + ep);
+                            logger.trace("Ignoring remote version {} <= {} for {}", remoteMaxVersion, localMaxVersion, ep);
 
                     if (!localEpStatePtr.isAlive() && !isDeadState(localEpStatePtr)) // unless of course, it was dead
                         markAlive(ep, localEpStatePtr);
@@ -1142,7 +1146,7 @@
                 else
                 {
                     if (logger.isTraceEnabled())
-                        logger.trace("Ignoring remote generation " + remoteGeneration + " < " + localGeneration);
+                        logger.trace("Ignoring remote generation {} < {}", remoteGeneration, localGeneration);
                 }
             }
             else
@@ -1161,7 +1165,7 @@
 
         localState.setHeartBeatState(remoteState.getHeartBeatState());
         if (logger.isTraceEnabled())
-            logger.trace("Updating heartbeat state version to " + localState.getHeartBeatState().getHeartBeatVersion() + " from " + oldVersion + " for " + addr + " ...");
+            logger.trace("Updating heartbeat state version to {} from {} for {} ...", localState.getHeartBeatState().getHeartBeatVersion(), oldVersion, addr);
 
         Set<Entry<ApplicationState, VersionedValue>> remoteStates = remoteState.states();
         assert remoteState.getHeartBeatState().getGeneration() == localState.getHeartBeatState().getGeneration();
@@ -1195,7 +1199,7 @@
         /* We are here since we have no data for this endpoint locally so request everthing. */
         deltaGossipDigestList.add(new GossipDigest(gDigest.getEndpoint(), remoteGeneration, 0));
         if (logger.isTraceEnabled())
-            logger.trace("requestAll for " + gDigest.getEndpoint());
+            logger.trace("requestAll for {}", gDigest.getEndpoint());
     }
 
     /* Send all the data with version greater than maxRemoteVersion */
@@ -1299,7 +1303,7 @@
         //notify snitches that Gossiper is about to start
         DatabaseDescriptor.getEndpointSnitch().gossiperStarting();
         if (logger.isTraceEnabled())
-            logger.trace("gossip started with generation " + localState.getHeartBeatState().getGeneration());
+            logger.trace("gossip started with generation {}", localState.getHeartBeatState().getGeneration());
 
         scheduledGossipTask = executor.scheduleWithFixedDelay(new GossipTask(),
                                                               Gossiper.intervalInMillis,
@@ -1308,12 +1312,32 @@
     }
 
     /**
-     *  Do a single 'shadow' round of gossip, where we do not modify any state
-     *  Only used when replacing a node, to get and assume its states
+     * Do a single 'shadow' round of gossip by retrieving endpoint states that will be stored exclusively in the
+     * map return value, instead of endpointStateMap.
+     *
+     * Used when preparing to join the ring:
+     * <ul>
+     *     <li>when replacing a node, to get and assume its tokens</li>
+     *     <li>when joining, to check that the local host id matches any previous id for the endpoint address</li>
+     * </ul>
+     *
+     * Method is synchronized, as we use an in-progress flag to indicate that shadow round must be cleared
+     * again by calling {@link Gossiper#finishShadowRound(Map)}. This will update
+     * {@link Gossiper#endpointShadowStateMap} with received values, in order to return an immutable copy to the
+     * caller of {@link Gossiper#doShadowRound()}. Therefor only a single shadow round execution is permitted at
+     * the same time.
+     *
+     * @return endpoint states gathered during shadow round or empty map
      */
-    public void doShadowRound()
+    public synchronized Map<InetAddress, EndpointState> doShadowRound()
     {
         buildSeedsList();
+        // it may be that the local address is the only entry in the seed
+        // list in which case, attempting a shadow round is pointless
+        if (seeds.isEmpty())
+            return endpointShadowStateMap;
+
+        endpointShadowStateMap.clear();
         // send a completely empty syn
         List<GossipDigest> gDigests = new ArrayList<GossipDigest>();
         GossipDigestSyn digestSynMessage = new GossipDigestSyn(DatabaseDescriptor.getClusterName(),
@@ -1349,6 +1373,8 @@
         {
             throw new RuntimeException(wtf);
         }
+
+        return ImmutableMap.copyOf(endpointShadowStateMap);
     }
 
     private void buildSeedsList()
@@ -1404,7 +1430,7 @@
         endpointStateMap.put(ep, epState);
         unreachableEndpoints.put(ep, System.nanoTime());
         if (logger.isTraceEnabled())
-            logger.trace("Adding saved endpoint " + ep + " " + epState.getHeartBeatState().getGeneration());
+            logger.trace("Adding saved endpoint {} {}", ep, epState.getHeartBeatState().getGeneration());
     }
 
     private void addLocalApplicationStateInternal(ApplicationState state, VersionedValue value)
@@ -1449,7 +1475,7 @@
     public void stop()
     {
         EndpointState mystate = endpointStateMap.get(FBUtilities.getBroadcastAddress());
-        if (mystate != null && !isSilentShutdownState(mystate))
+        if (mystate != null && !isSilentShutdownState(mystate) && StorageService.instance.isJoined())
         {
             logger.info("Announcing shutdown");
             addLocalApplicationState(ApplicationState.STATUS, StorageService.instance.valueFactory.shutdown(true));
@@ -1459,7 +1485,7 @@
             Uninterruptibles.sleepUninterruptibly(Integer.getInteger("cassandra.shutdown_announce_in_ms", 2000), TimeUnit.MILLISECONDS);
         }
         else
-            logger.warn("No local state or state is in silent shutdown, not announcing shutdown");
+            logger.warn("No local state, state is in silent shutdown, or node hasn't joined, not announcing shutdown");
         if (scheduledGossipTask != null)
             scheduledGossipTask.cancel(false);
     }
@@ -1469,10 +1495,13 @@
         return (scheduledGossipTask != null) && (!scheduledGossipTask.isCancelled());
     }
 
-    protected void finishShadowRound()
+    protected void finishShadowRound(Map<InetAddress, EndpointState> epStateMap)
     {
         if (inShadowRound)
+        {
+            endpointShadowStateMap.putAll(epStateMap);
             inShadowRound = false;
+        }
     }
 
     protected boolean isInShadowRound()
diff --git a/src/java/org/apache/cassandra/gms/GossiperMBean.java b/src/java/org/apache/cassandra/gms/GossiperMBean.java
index 521fd21..c4b244c 100644
--- a/src/java/org/apache/cassandra/gms/GossiperMBean.java
+++ b/src/java/org/apache/cassandra/gms/GossiperMBean.java
@@ -27,4 +27,6 @@
 
     public void unsafeAssassinateEndpoint(String address) throws UnknownHostException;
 
+    public void assassinateEndpoint(String address) throws UnknownHostException;
+
 }
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/gms/IFailureDetector.java b/src/java/org/apache/cassandra/gms/IFailureDetector.java
index 92338e9..a860c7c 100644
--- a/src/java/org/apache/cassandra/gms/IFailureDetector.java
+++ b/src/java/org/apache/cassandra/gms/IFailureDetector.java
@@ -41,7 +41,7 @@
      * This method is invoked by any entity wanting to interrogate the status of an endpoint.
      * In our case it would be the Gossiper. The Failure Detector will then calculate Phi and
      * deem an endpoint as suspicious or alive as explained in the Hayashibara paper.
-     * <p/>
+     *
      * param ep endpoint for which we interpret the inter arrival times.
      */
     public void interpret(InetAddress ep);
@@ -50,7 +50,7 @@
      * This method is invoked by the receiver of the heartbeat. In our case it would be
      * the Gossiper. Gossiper inform the Failure Detector on receipt of a heartbeat. The
      * FailureDetector will then sample the arrival time as explained in the paper.
-     * <p/>
+     *
      * param ep endpoint being reported.
      */
     public void report(InetAddress ep);
diff --git a/src/java/org/apache/cassandra/gms/VersionedValue.java b/src/java/org/apache/cassandra/gms/VersionedValue.java
index 810a668..661d3ba 100644
--- a/src/java/org/apache/cassandra/gms/VersionedValue.java
+++ b/src/java/org/apache/cassandra/gms/VersionedValue.java
@@ -43,11 +43,15 @@
  * application wants to make available to the rest of the nodes in the cluster.
  * Whenever a piece of state needs to be disseminated to the rest of cluster wrap
  * the state in an instance of <i>ApplicationState</i> and add it to the Gossiper.
- * <p/>
+ * <p>
  * e.g. if we want to disseminate load information for node A do the following:
- * <p/>
+ * </p>
+ * <pre>
+ * {@code
  * ApplicationState loadState = new ApplicationState(<string representation of load>);
  * Gossiper.instance.addApplicationState("LOAD STATE", loadState);
+ * }
+ * </pre>
  */
 
 public class VersionedValue implements Comparable<VersionedValue>
@@ -61,6 +65,7 @@
 
     // values for ApplicationState.STATUS
     public final static String STATUS_BOOTSTRAPPING = "BOOT";
+    public final static String STATUS_BOOTSTRAPPING_REPLACE = "BOOT_REPLACE";
     public final static String STATUS_NORMAL = "NORMAL";
     public final static String STATUS_LEAVING = "LEAVING";
     public final static String STATUS_LEFT = "LEFT";
@@ -129,6 +134,11 @@
             return new VersionedValue(value.value);
         }
 
+        public VersionedValue bootReplacing(InetAddress oldNode)
+        {
+            return new VersionedValue(versionString(VersionedValue.STATUS_BOOTSTRAPPING_REPLACE, oldNode.getHostAddress()));
+        }
+
         public VersionedValue bootstrapping(Collection<Token> tokens)
         {
             return new VersionedValue(versionString(VersionedValue.STATUS_BOOTSTRAPPING,
@@ -214,6 +224,11 @@
             return new VersionedValue(VersionedValue.HIBERNATE + VersionedValue.DELIMITER + value);
         }
 
+        public VersionedValue rpcReady(boolean value)
+        {
+            return new VersionedValue(String.valueOf(value));
+        }
+
         public VersionedValue shutdown(boolean value)
         {
             return new VersionedValue(VersionedValue.SHUTDOWN + VersionedValue.DELIMITER + value);
diff --git a/src/java/org/apache/cassandra/hadoop/AbstractBulkOutputFormat.java b/src/java/org/apache/cassandra/hadoop/AbstractBulkOutputFormat.java
deleted file mode 100644
index e893ba6..0000000
--- a/src/java/org/apache/cassandra/hadoop/AbstractBulkOutputFormat.java
+++ /dev/null
@@ -1,105 +0,0 @@
-/*
- * 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.hadoop;
-
-
-import java.io.IOException;
-import java.util.Collection;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.*;
-
-public abstract class AbstractBulkOutputFormat<K, V> extends OutputFormat<K, V>
-    implements org.apache.hadoop.mapred.OutputFormat<K, V>
-{
-    @Override
-    public void checkOutputSpecs(JobContext context)
-    {
-        checkOutputSpecs(HadoopCompat.getConfiguration(context));
-    }
-
-    private void checkOutputSpecs(Configuration conf)
-    {
-        if (ConfigHelper.getOutputKeyspace(conf) == null)
-        {
-            throw new UnsupportedOperationException("you must set the keyspace with setColumnFamily()");
-        }
-    }
-
-    @Override
-    public OutputCommitter getOutputCommitter(TaskAttemptContext context) throws IOException, InterruptedException
-    {
-        return new NullOutputCommitter();
-    }
-
-    /** Fills the deprecated OutputFormat interface for streaming. */
-    @Deprecated
-    public void checkOutputSpecs(org.apache.hadoop.fs.FileSystem filesystem, org.apache.hadoop.mapred.JobConf job) throws IOException
-    {
-        checkOutputSpecs(job);
-    }
-
-    public static class NullOutputCommitter extends OutputCommitter
-    {
-        public void abortTask(TaskAttemptContext taskContext) { }
-
-        public void cleanupJob(JobContext jobContext) { }
-
-        public void commitTask(TaskAttemptContext taskContext) { }
-
-        public boolean needsTaskCommit(TaskAttemptContext taskContext)
-        {
-            return false;
-        }
-
-        public void setupJob(JobContext jobContext) { }
-
-        public void setupTask(TaskAttemptContext taskContext) { }
-    }
-
-    /**
-     * Set the hosts to ignore as comma delimited values.
-     * Data will not be bulk loaded onto the ignored nodes.
-     * @param conf job configuration
-     * @param ignoreNodesCsv a comma delimited list of nodes to ignore
-     */
-    public static void setIgnoreHosts(Configuration conf, String ignoreNodesCsv)
-    {
-        conf.set(AbstractBulkRecordWriter.IGNORE_HOSTS, ignoreNodesCsv);
-    }
-
-    /**
-     * Set the hosts to ignore. Data will not be bulk loaded onto the ignored nodes.
-     * @param conf job configuration
-     * @param ignoreNodes the nodes to ignore
-     */
-    public static void setIgnoreHosts(Configuration conf, String... ignoreNodes)
-    {
-        conf.setStrings(AbstractBulkRecordWriter.IGNORE_HOSTS, ignoreNodes);
-    }
-
-    /**
-     * Get the hosts to ignore as a collection of strings
-     * @param conf job configuration
-     * @return the nodes to ignore as a collection of stirngs
-     */
-    public static Collection<String> getIgnoreHosts(Configuration conf)
-    {
-        return conf.getStringCollection(AbstractBulkRecordWriter.IGNORE_HOSTS);
-    }
-}
diff --git a/src/java/org/apache/cassandra/hadoop/AbstractBulkRecordWriter.java b/src/java/org/apache/cassandra/hadoop/AbstractBulkRecordWriter.java
deleted file mode 100644
index f9322c7..0000000
--- a/src/java/org/apache/cassandra/hadoop/AbstractBulkRecordWriter.java
+++ /dev/null
@@ -1,263 +0,0 @@
-/*
- * 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.hadoop;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-import org.apache.cassandra.auth.IAuthenticator;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.Config;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.io.sstable.SSTableLoader;
-import org.apache.cassandra.streaming.StreamState;
-import org.apache.cassandra.thrift.AuthenticationRequest;
-import org.apache.cassandra.thrift.Cassandra;
-import org.apache.cassandra.thrift.CfDef;
-import org.apache.cassandra.thrift.KsDef;
-import org.apache.cassandra.thrift.TokenRange;
-import org.apache.cassandra.utils.OutputHandler;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.util.Progressable;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public abstract class AbstractBulkRecordWriter<K, V> extends RecordWriter<K, V>
-implements org.apache.hadoop.mapred.RecordWriter<K, V>
-{
-    public final static String OUTPUT_LOCATION = "mapreduce.output.bulkoutputformat.localdir";
-    public final static String BUFFER_SIZE_IN_MB = "mapreduce.output.bulkoutputformat.buffersize";
-    public final static String STREAM_THROTTLE_MBITS = "mapreduce.output.bulkoutputformat.streamthrottlembits";
-    public final static String MAX_FAILED_HOSTS = "mapreduce.output.bulkoutputformat.maxfailedhosts";
-    public static final String IGNORE_HOSTS = "mapreduce.output.bulkoutputformat.ignorehosts";
-    
-    private final Logger logger = LoggerFactory.getLogger(AbstractBulkRecordWriter.class);
-    
-    protected final Configuration conf;
-    protected final int maxFailures;
-    protected final int bufferSize;
-    protected final Set<InetAddress> ignores = new HashSet<>();
-    protected Closeable writer;
-    protected SSTableLoader loader;
-    protected Progressable progress;
-    protected TaskAttemptContext context;
-    
-    protected AbstractBulkRecordWriter(TaskAttemptContext context)
-    {
-        this(HadoopCompat.getConfiguration(context));
-        this.context = context;
-    }
-
-    protected AbstractBulkRecordWriter(Configuration conf, Progressable progress)
-    {
-        this(conf);
-        this.progress = progress;
-    }
-
-    protected AbstractBulkRecordWriter(Configuration conf)
-    {
-        Config.setClientMode(true);
-        Config.setOutboundBindAny(true);
-        this.conf = conf;
-        DatabaseDescriptor.setStreamThroughputOutboundMegabitsPerSec(Integer.parseInt(conf.get(STREAM_THROTTLE_MBITS, "0")));
-        maxFailures = Integer.parseInt(conf.get(MAX_FAILED_HOSTS, "0"));
-        bufferSize = Integer.parseInt(conf.get(BUFFER_SIZE_IN_MB, "64"));
-        try
-        {
-            for (String hostToIgnore : AbstractBulkOutputFormat.getIgnoreHosts(conf))
-                ignores.add(InetAddress.getByName(hostToIgnore));
-        }
-        catch (UnknownHostException e)
-        {
-            throw new RuntimeException(("Unknown host: " + e.getMessage()));
-        }
-    }
-
-    protected String getOutputLocation() throws IOException
-    {
-        String dir = conf.get(OUTPUT_LOCATION, System.getProperty("java.io.tmpdir"));
-        if (dir == null)
-            throw new IOException("Output directory not defined, if hadoop is not setting java.io.tmpdir then define " + OUTPUT_LOCATION);
-        return dir;
-    }
-
-    @Override
-    public void close(TaskAttemptContext context) throws IOException, InterruptedException
-    {
-        close();
-    }
-
-    /** Fills the deprecated RecordWriter interface for streaming. */
-    @Deprecated
-    public void close(org.apache.hadoop.mapred.Reporter reporter) throws IOException
-    {
-        close();
-    }
-
-    private void close() throws IOException
-    {
-        if (writer != null)
-        {
-            writer.close();
-            Future<StreamState> future = loader.stream(ignores);
-            while (true)
-            {
-                try
-                {
-                    future.get(1000, TimeUnit.MILLISECONDS);
-                    break;
-                }
-                catch (ExecutionException | TimeoutException te)
-                {
-                    if (null != progress)
-                        progress.progress();
-                    if (null != context)
-                        HadoopCompat.progress(context);
-                }
-                catch (InterruptedException e)
-                {
-                    throw new IOException(e);
-                }
-            }
-            if (loader.getFailedHosts().size() > 0)
-            {
-                if (loader.getFailedHosts().size() > maxFailures)
-                    throw new IOException("Too many hosts failed: " + loader.getFailedHosts());
-                else
-                    logger.warn("Some hosts failed: {}", loader.getFailedHosts());
-            }
-        }
-    }
-
-    public static class ExternalClient extends SSTableLoader.Client
-    {
-        private final Map<String, Map<String, CFMetaData>> knownCfs = new HashMap<>();
-        private final Configuration conf;
-        private final String hostlist;
-        private final int rpcPort;
-        private final String username;
-        private final String password;
-
-        public ExternalClient(Configuration conf)
-        {
-          super();
-          this.conf = conf;
-          this.hostlist = ConfigHelper.getOutputInitialAddress(conf);
-          this.rpcPort = ConfigHelper.getOutputRpcPort(conf);
-          this.username = ConfigHelper.getOutputKeyspaceUserName(conf);
-          this.password = ConfigHelper.getOutputKeyspacePassword(conf);
-        }
-
-        public void init(String keyspace)
-        {
-            Set<InetAddress> hosts = new HashSet<InetAddress>();
-            String[] nodes = hostlist.split(",");
-            for (String node : nodes)
-            {
-                try
-                {
-                    hosts.add(InetAddress.getByName(node));
-                }
-                catch (UnknownHostException e)
-                {
-                    throw new RuntimeException(e);
-                }
-            }
-            Iterator<InetAddress> hostiter = hosts.iterator();
-            while (hostiter.hasNext())
-            {
-                try
-                {
-                    InetAddress host = hostiter.next();
-                    Cassandra.Client client = ConfigHelper.createConnection(conf, host.getHostAddress(), rpcPort);
-
-                    // log in
-                    client.set_keyspace(keyspace);
-                    if (username != null)
-                    {
-                        Map<String, String> creds = new HashMap<String, String>();
-                        creds.put(IAuthenticator.USERNAME_KEY, username);
-                        creds.put(IAuthenticator.PASSWORD_KEY, password);
-                        AuthenticationRequest authRequest = new AuthenticationRequest(creds);
-                        client.login(authRequest);
-                    }
-
-                    List<TokenRange> tokenRanges = client.describe_ring(keyspace);
-                    List<KsDef> ksDefs = client.describe_keyspaces();
-
-                    setPartitioner(client.describe_partitioner());
-                    Token.TokenFactory tkFactory = getPartitioner().getTokenFactory();
-
-                    for (TokenRange tr : tokenRanges)
-                    {
-                        Range<Token> range = new Range<Token>(tkFactory.fromString(tr.start_token), tkFactory.fromString(tr.end_token));
-                        for (String ep : tr.endpoints)
-                        {
-                            addRangeForEndpoint(range, InetAddress.getByName(ep));
-                        }
-                    }
-
-                    for (KsDef ksDef : ksDefs)
-                    {
-                        Map<String, CFMetaData> cfs = new HashMap<>(ksDef.cf_defs.size());
-                        for (CfDef cfDef : ksDef.cf_defs)
-                            cfs.put(cfDef.name, CFMetaData.fromThrift(cfDef));
-                        knownCfs.put(ksDef.name, cfs);
-                    }
-                    break;
-                }
-                catch (Exception e)
-                {
-                    if (!hostiter.hasNext())
-                        throw new RuntimeException("Could not retrieve endpoint ranges: ", e);
-                }
-            }
-        }
-
-        public CFMetaData getCFMetaData(String keyspace, String cfName)
-        {
-            Map<String, CFMetaData> cfs = knownCfs.get(keyspace);
-            return cfs != null ? cfs.get(cfName) : null;
-        } 
-    }
-
-    public static class NullOutputHandler implements OutputHandler
-    {
-        public void output(String msg) {}
-        public void debug(String msg) {}
-        public void warn(String msg) {}
-        public void warn(String msg, Throwable th) {}
-    }
-}
diff --git a/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java b/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java
index e8de0f2..d55f205 100644
--- a/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java
+++ b/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyInputFormat.java
@@ -18,8 +18,13 @@
 package org.apache.cassandra.hadoop;
 
 import java.io.IOException;
-import java.net.InetAddress;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
@@ -27,22 +32,22 @@
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.auth.IAuthenticator;
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.Host;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.TokenRange;
+import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.thrift.AuthenticationRequest;
-import org.apache.cassandra.thrift.Cassandra;
-import org.apache.cassandra.thrift.CfSplit;
-import org.apache.cassandra.thrift.InvalidRequestException;
+import org.apache.cassandra.hadoop.cql3.CqlConfigHelper;
 import org.apache.cassandra.thrift.KeyRange;
-import org.apache.cassandra.thrift.TokenRange;
-import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapreduce.InputFormat;
@@ -50,13 +55,6 @@
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.thrift.TApplicationException;
-import org.apache.thrift.TException;
-import org.apache.thrift.protocol.TBinaryProtocol;
-import org.apache.thrift.protocol.TProtocol;
-import org.apache.thrift.transport.TTransport;
-import org.apache.thrift.transport.TTransportException;
-
 
 public abstract class AbstractColumnFamilyInputFormat<K, Y> extends InputFormat<K, Y> implements org.apache.hadoop.mapred.InputFormat<K, Y>
 {
@@ -65,7 +63,7 @@
     public static final String MAPRED_TASK_ID = "mapred.task.id";
     // The simple fact that we need this is because the old Hadoop API wants us to "write"
     // to the key and value whereas the new asks for it.
-    // I choose 8kb as the default max key size (instanciated only once), but you can
+    // I choose 8kb as the default max key size (instantiated only once), but you can
     // override it in your jobConf with this setting.
     public static final String CASSANDRA_HADOOP_MAX_KEY_SIZE = "cassandra.hadoop.max_key_size";
     public static final int    CASSANDRA_HADOOP_MAX_KEY_SIZE_DEFAULT = 8192;
@@ -78,7 +76,7 @@
     {
         if (ConfigHelper.getInputKeyspace(conf) == null || ConfigHelper.getInputColumnFamily(conf) == null)
         {
-            throw new UnsupportedOperationException("you must set the keyspace and columnfamily with setInputColumnFamily()");
+            throw new UnsupportedOperationException("you must set the keyspace and table with setInputColumnFamily()");
         }
         if (ConfigHelper.getInputInitialAddress(conf) == null)
             throw new UnsupportedOperationException("You must set the initial output address to a Cassandra node with setInputInitialAddress");
@@ -86,106 +84,72 @@
             throw new UnsupportedOperationException("You must set the Cassandra partitioner class with setInputPartitioner");
     }
 
-    public static Cassandra.Client createAuthenticatedClient(String location, int port, Configuration conf) throws Exception
-    {
-        logger.debug("Creating authenticated client for CF input format");
-        TTransport transport;
-        try
-        {
-            transport = ConfigHelper.getClientTransportFactory(conf).openTransport(location, port);
-        }
-        catch (Exception e)
-        {
-            throw new TTransportException("Failed to open a transport to " + location + ":" + port + ".", e);
-        }
-        TProtocol binaryProtocol = new TBinaryProtocol(transport, true, true);
-        Cassandra.Client client = new Cassandra.Client(binaryProtocol);
-
-        // log in
-        client.set_keyspace(ConfigHelper.getInputKeyspace(conf));
-        if ((ConfigHelper.getInputKeyspaceUserName(conf) != null) && (ConfigHelper.getInputKeyspacePassword(conf) != null))
-        {
-            Map<String, String> creds = new HashMap<String, String>();
-            creds.put(IAuthenticator.USERNAME_KEY, ConfigHelper.getInputKeyspaceUserName(conf));
-            creds.put(IAuthenticator.PASSWORD_KEY, ConfigHelper.getInputKeyspacePassword(conf));
-            AuthenticationRequest authRequest = new AuthenticationRequest(creds);
-            client.login(authRequest);
-        }
-        logger.debug("Authenticated client for CF input format created successfully");
-        return client;
-    }
-
     public List<InputSplit> getSplits(JobContext context) throws IOException
     {
-        Configuration conf = HadoopCompat.getConfiguration(context);;
+        Configuration conf = HadoopCompat.getConfiguration(context);
 
         validateConfiguration(conf);
 
-        // cannonical ranges and nodes holding replicas
-        List<TokenRange> masterRangeNodes = getRangeMap(conf);
-
         keyspace = ConfigHelper.getInputKeyspace(conf);
         cfName = ConfigHelper.getInputColumnFamily(conf);
         partitioner = ConfigHelper.getInputPartitioner(conf);
-        logger.debug("partitioner is {}", partitioner);
+        logger.trace("partitioner is {}", partitioner);
 
+        // canonical ranges and nodes holding replicas
+        Map<TokenRange, Set<Host>> masterRangeNodes = getRangeMap(conf, keyspace);
 
-        // cannonical ranges, split into pieces, fetching the splits in parallel
+        // canonical ranges, split into pieces, fetching the splits in parallel
         ExecutorService executor = new ThreadPoolExecutor(0, 128, 60L, TimeUnit.SECONDS, new LinkedBlockingQueue<Runnable>());
-        List<InputSplit> splits = new ArrayList<InputSplit>();
+        List<InputSplit> splits = new ArrayList<>();
 
-        try
+        List<Future<List<InputSplit>>> splitfutures = new ArrayList<>();
+        KeyRange jobKeyRange = ConfigHelper.getInputKeyRange(conf);
+        Range<Token> jobRange = null;
+        if (jobKeyRange != null)
         {
-            List<Future<List<InputSplit>>> splitfutures = new ArrayList<Future<List<InputSplit>>>();
-            KeyRange jobKeyRange = ConfigHelper.getInputKeyRange(conf);
-            Range<Token> jobRange = null;
-            if (jobKeyRange != null)
+            if (jobKeyRange.start_key != null)
             {
-                if (jobKeyRange.start_key != null)
-                {
-                    if (!partitioner.preservesOrder())
-                        throw new UnsupportedOperationException("KeyRange based on keys can only be used with a order preserving paritioner");
-                    if (jobKeyRange.start_token != null)
-                        throw new IllegalArgumentException("only start_key supported");
-                    if (jobKeyRange.end_token != null)
-                        throw new IllegalArgumentException("only start_key supported");
-                    jobRange = new Range<>(partitioner.getToken(jobKeyRange.start_key),
-                                           partitioner.getToken(jobKeyRange.end_key),
-                                           partitioner);
-                }
-                else if (jobKeyRange.start_token != null)
-                {
-                    jobRange = new Range<>(partitioner.getTokenFactory().fromString(jobKeyRange.start_token),
-                                           partitioner.getTokenFactory().fromString(jobKeyRange.end_token),
-                                           partitioner);
-                }
-                else
-                {
-                    logger.warn("ignoring jobKeyRange specified without start_key or start_token");
-                }
+                if (!partitioner.preservesOrder())
+                    throw new UnsupportedOperationException("KeyRange based on keys can only be used with a order preserving partitioner");
+                if (jobKeyRange.start_token != null)
+                    throw new IllegalArgumentException("only start_key supported");
+                if (jobKeyRange.end_token != null)
+                    throw new IllegalArgumentException("only start_key supported");
+                jobRange = new Range<>(partitioner.getToken(jobKeyRange.start_key),
+                                       partitioner.getToken(jobKeyRange.end_key));
             }
+            else if (jobKeyRange.start_token != null)
+            {
+                jobRange = new Range<>(partitioner.getTokenFactory().fromString(jobKeyRange.start_token),
+                                       partitioner.getTokenFactory().fromString(jobKeyRange.end_token));
+            }
+            else
+            {
+                logger.warn("ignoring jobKeyRange specified without start_key or start_token");
+            }
+        }
 
-            for (TokenRange range : masterRangeNodes)
+        try (Cluster cluster = CqlConfigHelper.getInputCluster(ConfigHelper.getInputInitialAddress(conf).split(","), conf);
+             Session session = cluster.connect())
+        {
+            Metadata metadata = session.getCluster().getMetadata();
+
+            for (TokenRange range : masterRangeNodes.keySet())
             {
                 if (jobRange == null)
                 {
-                    // for each range, pick a live owner and ask it to compute bite-sized splits
-                    splitfutures.add(executor.submit(new SplitCallable(range, conf)));
+                    // for each tokenRange, pick a live owner and ask it to compute bite-sized splits
+                    splitfutures.add(executor.submit(new SplitCallable(range, masterRangeNodes.get(range), conf, session)));
                 }
                 else
                 {
-                    Range<Token> dhtRange = new Range<Token>(partitioner.getTokenFactory().fromString(range.start_token),
-                                                             partitioner.getTokenFactory().fromString(range.end_token),
-                                                             partitioner);
-
-                    if (dhtRange.intersects(jobRange))
+                    TokenRange jobTokenRange = rangeToTokenRange(metadata, jobRange);
+                    if (range.intersects(jobTokenRange))
                     {
-                        for (Range<Token> intersection: dhtRange.intersectionWith(jobRange))
+                        for (TokenRange intersection: range.intersectWith(jobTokenRange))
                         {
-                            range.start_token = partitioner.getTokenFactory().toString(intersection.left);
-                            range.end_token = partitioner.getTokenFactory().toString(intersection.right);
-                            // for each range, pick a live owner and ask it to compute bite-sized splits
-                            splitfutures.add(executor.submit(new SplitCallable(range, conf)));
+                            // for each tokenRange, pick a live owner and ask it to compute bite-sized splits
+                            splitfutures.add(executor.submit(new SplitCallable(intersection,  masterRangeNodes.get(range), conf, session)));
                         }
                     }
                 }
@@ -214,56 +178,60 @@
         return splits;
     }
 
+    private TokenRange rangeToTokenRange(Metadata metadata, Range<Token> range)
+    {
+        return metadata.newTokenRange(metadata.newToken(partitioner.getTokenFactory().toString(range.left)),
+                metadata.newToken(partitioner.getTokenFactory().toString(range.right)));
+    }
+
     /**
-     * Gets a token range and splits it up according to the suggested
+     * Gets a token tokenRange and splits it up according to the suggested
      * size into input splits that Hadoop can use.
      */
     class SplitCallable implements Callable<List<InputSplit>>
     {
 
-        private final TokenRange range;
+        private final TokenRange tokenRange;
+        private final Set<Host> hosts;
         private final Configuration conf;
+        private final Session session;
 
-        public SplitCallable(TokenRange tr, Configuration conf)
+        public SplitCallable(TokenRange tr, Set<Host> hosts, Configuration conf, Session session)
         {
-            this.range = tr;
+            this.tokenRange = tr;
+            this.hosts = hosts;
             this.conf = conf;
+            this.session = session;
         }
 
         public List<InputSplit> call() throws Exception
         {
-            ArrayList<InputSplit> splits = new ArrayList<InputSplit>();
-            List<CfSplit> subSplits = getSubSplits(keyspace, cfName, range, conf);
-            assert range.rpc_endpoints.size() == range.endpoints.size() : "rpc_endpoints size must match endpoints size";
+            ArrayList<InputSplit> splits = new ArrayList<>();
+            Map<TokenRange, Long> subSplits;
+            subSplits = getSubSplits(keyspace, cfName, tokenRange, conf, session);
             // turn the sub-ranges into InputSplits
-            String[] endpoints = range.endpoints.toArray(new String[range.endpoints.size()]);
+            String[] endpoints = new String[hosts.size()];
+
             // hadoop needs hostname, not ip
             int endpointIndex = 0;
-            for (String endpoint: range.rpc_endpoints)
-            {
-                String endpoint_address = endpoint;
-                if (endpoint_address == null || endpoint_address.equals("0.0.0.0"))
-                    endpoint_address = range.endpoints.get(endpointIndex);
-                endpoints[endpointIndex++] = InetAddress.getByName(endpoint_address).getHostName();
-            }
+            for (Host endpoint : hosts)
+                endpoints[endpointIndex++] = endpoint.getAddress().getHostName();
 
-            Token.TokenFactory factory = partitioner.getTokenFactory();
-            for (CfSplit subSplit : subSplits)
+            for (TokenRange subSplit : subSplits.keySet())
             {
-                Token left = factory.fromString(subSplit.getStart_token());
-                Token right = factory.fromString(subSplit.getEnd_token());
-                Range<Token> range = new Range<Token>(left, right, partitioner);
-                List<Range<Token>> ranges = range.isWrapAround() ? range.unwrap() : ImmutableList.of(range);
-                for (Range<Token> subrange : ranges)
+                List<TokenRange> ranges = subSplit.unwrap();
+                for (TokenRange subrange : ranges)
                 {
                     ColumnFamilySplit split =
                             new ColumnFamilySplit(
-                                    factory.toString(subrange.left),
-                                    factory.toString(subrange.right),
-                                    subSplit.getRow_count(),
+                                    partitioner.preservesOrder() ?
+                                            subrange.getStart().toString().substring(2) : subrange.getStart().toString(),
+                                    partitioner.preservesOrder() ?
+                                            subrange.getEnd().toString().substring(2) : subrange.getEnd().toString(),
+                                    subSplits.get(subSplit),
                                     endpoints);
 
-                    logger.debug("adding {}", split);
+                    logger.trace("adding {}", split);
                     splits.add(split);
                 }
             }
@@ -271,83 +239,69 @@
         }
     }
 
-    private List<CfSplit> getSubSplits(String keyspace, String cfName, TokenRange range, Configuration conf) throws IOException
+    private Map<TokenRange, Long> getSubSplits(String keyspace, String cfName, TokenRange range, Configuration conf, Session session) throws IOException
     {
-        int splitsize = ConfigHelper.getInputSplitSize(conf);
-        for (int i = 0; i < range.rpc_endpoints.size(); i++)
-        {
-            String host = range.rpc_endpoints.get(i);
-
-            if (host == null || host.equals("0.0.0.0"))
-                host = range.endpoints.get(i);
-
-            try
-            {
-                Cassandra.Client client = ConfigHelper.createConnection(conf, host, ConfigHelper.getInputRpcPort(conf));
-                client.set_keyspace(keyspace);
-
-                try
-                {
-                    return client.describe_splits_ex(cfName, range.start_token, range.end_token, splitsize);
-                }
-                catch (TApplicationException e)
-                {
-                    // fallback to guessing split size if talking to a server without describe_splits_ex method
-                    if (e.getType() == TApplicationException.UNKNOWN_METHOD)
-                    {
-                        List<String> splitPoints = client.describe_splits(cfName, range.start_token, range.end_token, splitsize);
-                        return tokenListToSplits(splitPoints, splitsize);
-                    }
-                    throw e;
-                }
-            }
-            catch (IOException e)
-            {
-                logger.debug("failed connect to endpoint {}", host, e);
-            }
-            catch (InvalidRequestException e)
-            {
-                throw new RuntimeException(e);
-            }
-            catch (TException e)
-            {
-                throw new RuntimeException(e);
-            }
-        }
-        throw new IOException("failed connecting to all endpoints " + StringUtils.join(range.endpoints, ","));
-    }
-
-    private List<CfSplit> tokenListToSplits(List<String> splitTokens, int splitsize)
-    {
-        List<CfSplit> splits = Lists.newArrayListWithExpectedSize(splitTokens.size() - 1);
-        for (int j = 0; j < splitTokens.size() - 1; j++)
-            splits.add(new CfSplit(splitTokens.get(j), splitTokens.get(j + 1), splitsize));
-        return splits;
-    }
-
-    private List<TokenRange> getRangeMap(Configuration conf) throws IOException
-    {
-        Cassandra.Client client = ConfigHelper.getClientFromInputAddressList(conf);
-
-        List<TokenRange> map;
+        int splitSize = ConfigHelper.getInputSplitSize(conf);
+        int splitSizeMb = ConfigHelper.getInputSplitSizeInMb(conf);
         try
         {
-            map = client.describe_local_ring(ConfigHelper.getInputKeyspace(conf));
+            return describeSplits(keyspace, cfName, range, splitSize, splitSizeMb, session);
         }
-        catch (InvalidRequestException e)
+        catch (Exception e)
         {
             throw new RuntimeException(e);
         }
-        catch (TException e)
-        {
-            throw new RuntimeException(e);
-        }
-        return map;
     }
 
-    //
+    private Map<TokenRange, Set<Host>> getRangeMap(Configuration conf, String keyspace)
+    {
+        try (Cluster cluster = CqlConfigHelper.getInputCluster(ConfigHelper.getInputInitialAddress(conf).split(","), conf))
+        {
+            Map<TokenRange, Set<Host>> map = new HashMap<>();
+            Metadata metadata = cluster.connect().getCluster().getMetadata();
+            for (TokenRange tokenRange : metadata.getTokenRanges())
+                map.put(tokenRange, metadata.getReplicas('"' + keyspace + '"', tokenRange));
+            return map;
+        }
+    }
+
+    private Map<TokenRange, Long> describeSplits(String keyspace, String table, TokenRange tokenRange, int splitSize, int splitSizeMb, Session session)
+    {
+        String query = String.format("SELECT mean_partition_size, partitions_count " +
+                                     "FROM %s.%s " +
+                                     "WHERE keyspace_name = ? AND table_name = ? AND range_start = ? AND range_end = ?",
+                                     SystemKeyspace.NAME,
+                                     SystemKeyspace.SIZE_ESTIMATES);
+
+        ResultSet resultSet = session.execute(query, keyspace, table, tokenRange.getStart().toString(), tokenRange.getEnd().toString());
+
+        Row row = resultSet.one();
+        // If we have no data on this split, return the full split i.e., do not sub-split
+        // Assume smallest granularity of partition count available from CASSANDRA-7688
+        if (row == null)
+        {
+            Map<TokenRange, Long> wrappedTokenRange = new HashMap<>();
+            wrappedTokenRange.put(tokenRange, (long) 128);
+            return wrappedTokenRange;
+        }
+
+        long meanPartitionSize = row.getLong("mean_partition_size");
+        long partitionCount = row.getLong("partitions_count");
+
+        int splitCount = splitSizeMb > 0
+            ? (int)(meanPartitionSize * partitionCount / splitSizeMb / 1024 / 1024)
+            : (int)(partitionCount / splitSize);
+
+        if (splitCount <= 0) splitCount = 1;
+        List<TokenRange> splitRanges = tokenRange.splitEvenly(splitCount);
+        Map<TokenRange, Long> rangesWithLength = new HashMap<>();
+        for (TokenRange range : splitRanges)
+            rangesWithLength.put(range, partitionCount/splitCount);
+
+        return rangesWithLength;
+    }
+
     // Old Hadoop API
-    //
     public org.apache.hadoop.mapred.InputSplit[] getSplits(JobConf jobConf, int numSplits) throws IOException
     {
         TaskAttemptContext tac = HadoopCompat.newTaskAttemptContext(jobConf, new TaskAttemptID());
diff --git a/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyOutputFormat.java b/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyOutputFormat.java
deleted file mode 100644
index f4963fb..0000000
--- a/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyOutputFormat.java
+++ /dev/null
@@ -1,165 +0,0 @@
-/*
- * 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.hadoop;
-
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.auth.IAuthenticator;
-import org.apache.cassandra.thrift.*;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.*;
-import org.apache.thrift.protocol.TBinaryProtocol;
-import org.apache.thrift.protocol.TProtocol;
-import org.apache.thrift.transport.TTransport;
-
-/**
- * The <code>ColumnFamilyOutputFormat</code> acts as a Hadoop-specific
- * OutputFormat that allows reduce tasks to store keys (and corresponding
- * values) as Cassandra rows (and respective columns) in a given
- * ColumnFamily.
- *
- * <p>
- * As is the case with the {@link ColumnFamilyInputFormat}, you need to set the
- * Keyspace and ColumnFamily in your
- * Hadoop job Configuration. The {@link ConfigHelper} class, through its
- * {@link ConfigHelper#setOutputColumnFamily} method, is provided to make this
- * simple.
- * </p>
- *
- * <p>
- * For the sake of performance, this class employs a lazy write-back caching
- * mechanism, where its record writer batches mutations created based on the
- * reduce's inputs (in a task-specific map), and periodically makes the changes
- * official by sending a batch mutate request to Cassandra.
- * </p>
- * @param <Y>
- */
-public abstract class AbstractColumnFamilyOutputFormat<K, Y> extends OutputFormat<K, Y> implements org.apache.hadoop.mapred.OutputFormat<K, Y>
-{
-    public static final String BATCH_THRESHOLD = "mapreduce.output.columnfamilyoutputformat.batch.threshold";
-    public static final String QUEUE_SIZE = "mapreduce.output.columnfamilyoutputformat.queue.size";
-    private static final Logger logger = LoggerFactory.getLogger(AbstractColumnFamilyOutputFormat.class);
-
-
-    /**
-     * Check for validity of the output-specification for the job.
-     *
-     * @param context
-     *            information about the job
-     * @throws IOException
-     *             when output should not be attempted
-     */
-    public void checkOutputSpecs(JobContext context)
-    {
-        checkOutputSpecs(HadoopCompat.getConfiguration(context));
-    }
-
-    protected void checkOutputSpecs(Configuration conf)
-    {
-        if (ConfigHelper.getOutputKeyspace(conf) == null)
-            throw new UnsupportedOperationException("You must set the keyspace with setOutputKeyspace()");
-        if (ConfigHelper.getOutputPartitioner(conf) == null)
-            throw new UnsupportedOperationException("You must set the output partitioner to the one used by your Cassandra cluster");
-        if (ConfigHelper.getOutputInitialAddress(conf) == null)
-            throw new UnsupportedOperationException("You must set the initial output address to a Cassandra node");
-    }
-
-    /** Fills the deprecated OutputFormat interface for streaming. */
-    @Deprecated
-    public void checkOutputSpecs(org.apache.hadoop.fs.FileSystem filesystem, org.apache.hadoop.mapred.JobConf job) throws IOException
-    {
-        checkOutputSpecs(job);
-    }
-
-    /**
-     * The OutputCommitter for this format does not write any data to the DFS.
-     *
-     * @param context
-     *            the task context
-     * @return an output committer
-     * @throws IOException
-     * @throws InterruptedException
-     */
-    public OutputCommitter getOutputCommitter(TaskAttemptContext context) throws IOException, InterruptedException
-    {
-        return new NullOutputCommitter();
-    }
-
-    /**
-     * Connects to the given server:port and returns a client based on the given socket that points to the configured
-     * keyspace, and is logged in with the configured credentials.
-     *
-     * @param host fully qualified host name to connect to
-     * @param port RPC port of the server
-     * @param conf a job configuration
-     * @return a cassandra client
-     * @throws Exception set of thrown exceptions may be implementation defined,
-     *                   depending on the used transport factory
-     */
-    public static Cassandra.Client createAuthenticatedClient(String host, int port, Configuration conf) throws Exception
-    {
-        logger.debug("Creating authenticated client for CF output format");
-        TTransport transport = ConfigHelper.getClientTransportFactory(conf).openTransport(host, port);
-        TProtocol binaryProtocol = new TBinaryProtocol(transport, true, true);
-        Cassandra.Client client = new Cassandra.Client(binaryProtocol);
-        client.set_keyspace(ConfigHelper.getOutputKeyspace(conf));
-        String user = ConfigHelper.getOutputKeyspaceUserName(conf);
-        String password = ConfigHelper.getOutputKeyspacePassword(conf);
-        if ((user != null) && (password != null))
-            login(user, password, client);
-
-        logger.debug("Authenticated client for CF output format created successfully");
-        return client;
-    }
-
-    public static void login(String user, String password, Cassandra.Client client) throws Exception
-    {
-        Map<String, String> creds = new HashMap<String, String>();
-        creds.put(IAuthenticator.USERNAME_KEY, user);
-        creds.put(IAuthenticator.PASSWORD_KEY, password);
-        AuthenticationRequest authRequest = new AuthenticationRequest(creds);
-        client.login(authRequest);
-    }
-
-    /**
-     * An {@link OutputCommitter} that does nothing.
-     */
-    private static class NullOutputCommitter extends OutputCommitter
-    {
-        public void abortTask(TaskAttemptContext taskContext) { }
-
-        public void cleanupJob(JobContext jobContext) { }
-
-        public void commitTask(TaskAttemptContext taskContext) { }
-
-        public boolean needsTaskCommit(TaskAttemptContext taskContext)
-        {
-            return false;
-        }
-
-        public void setupJob(JobContext jobContext) { }
-
-        public void setupTask(TaskAttemptContext taskContext) { }
-    }
-}
diff --git a/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyRecordWriter.java b/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyRecordWriter.java
deleted file mode 100644
index 501ca65..0000000
--- a/src/java/org/apache/cassandra/hadoop/AbstractColumnFamilyRecordWriter.java
+++ /dev/null
@@ -1,193 +0,0 @@
-/*
- * 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.hadoop;
-
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.util.*;
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.cassandra.client.RingCache;
-import org.apache.cassandra.thrift.*;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.thrift.transport.TTransport;
-import org.apache.hadoop.util.Progressable;
-
-
-/**
- * The <code>ColumnFamilyRecordWriter</code> maps the output &lt;key, value&gt;
- * pairs to a Cassandra column family. In particular, it applies all mutations
- * in the value, which it associates with the key, and in turn the responsible
- * endpoint.
- *
- * <p>
- * Furthermore, this writer groups the mutations by the endpoint responsible for
- * the rows being affected. This allows the mutations to be executed in parallel,
- * directly to a responsible endpoint.
- * </p>
- *
- * @see ColumnFamilyOutputFormat
- */
-public abstract class AbstractColumnFamilyRecordWriter<K, Y> extends RecordWriter<K, Y> implements org.apache.hadoop.mapred.RecordWriter<K, Y>
-{
-    // The configuration this writer is associated with.
-    protected final Configuration conf;
-
-    // The ring cache that describes the token ranges each node in the ring is
-    // responsible for. This is what allows us to group the mutations by
-    // the endpoints they should be targeted at. The targeted endpoint
-    // essentially
-    // acts as the primary replica for the rows being affected by the mutations.
-    protected final RingCache ringCache;
-
-    // The number of mutations to buffer per endpoint
-    protected final int queueSize;
-
-    protected final long batchThreshold;
-
-    protected final ConsistencyLevel consistencyLevel;
-    protected Progressable progressable;
-    protected TaskAttemptContext context;
-
-    protected AbstractColumnFamilyRecordWriter(Configuration conf)
-    {
-        this.conf = conf;
-        this.ringCache = new RingCache(conf);
-        this.queueSize = conf.getInt(AbstractColumnFamilyOutputFormat.QUEUE_SIZE, 32 * FBUtilities.getAvailableProcessors());
-        batchThreshold = conf.getLong(AbstractColumnFamilyOutputFormat.BATCH_THRESHOLD, 32);
-        consistencyLevel = ConsistencyLevel.valueOf(ConfigHelper.getWriteConsistencyLevel(conf));
-    }
-    
-    /**
-     * Close this <code>RecordWriter</code> to future operations, but not before
-     * flushing out the batched mutations.
-     *
-     * @param context the context of the task
-     * @throws IOException
-     */
-    public void close(TaskAttemptContext context) throws IOException, InterruptedException
-    {
-        close();
-    }
-
-    /** Fills the deprecated RecordWriter interface for streaming. */
-    @Deprecated
-    public void close(org.apache.hadoop.mapred.Reporter reporter) throws IOException
-    {
-        close();
-    }
-    
-    protected abstract void close() throws IOException;
-
-    /**
-     * A client that runs in a threadpool and connects to the list of endpoints for a particular
-     * range. Mutations for keys in that range are sent to this client via a queue.
-     */
-    public abstract class AbstractRangeClient<K> extends Thread
-    {
-        // The list of endpoints for this range
-        protected final List<InetAddress> endpoints;
-        // A bounded queue of incoming mutations for this range
-        protected final BlockingQueue<K> queue = new ArrayBlockingQueue<K>(queueSize);
-
-        protected volatile boolean run = true;
-        // we want the caller to know if something went wrong, so we record any unrecoverable exception while writing
-        // so we can throw it on the caller's stack when he calls put() again, or if there are no more put calls,
-        // when the client is closed.
-        protected volatile IOException lastException;
-
-        protected Cassandra.Client client;
-
-        /**
-         * Constructs an {@link AbstractRangeClient} for the given endpoints.
-         * @param endpoints the possible endpoints to execute the mutations on
-         */
-        public AbstractRangeClient(List<InetAddress> endpoints)
-        {
-            super("client-" + endpoints);
-            this.endpoints = endpoints;
-         }
-
-        /**
-         * enqueues the given value to Cassandra
-         */
-        public void put(K value) throws IOException
-        {
-            while (true)
-            {
-                if (lastException != null)
-                    throw lastException;
-                try
-                {
-                    if (queue.offer(value, 100, TimeUnit.MILLISECONDS))
-                        break;
-                }
-                catch (InterruptedException e)
-                {
-                    throw new AssertionError(e);
-                }
-            }
-        }
-
-        public void close() throws IOException
-        {
-            // stop the run loop.  this will result in closeInternal being called by the time join() finishes.
-            run = false;
-            interrupt();
-            try
-            {
-                this.join();
-            }
-            catch (InterruptedException e)
-            {
-                throw new AssertionError(e);
-            }
-
-            if (lastException != null)
-                throw lastException;
-        }
-
-        protected void closeInternal()
-        {
-            if (client != null)
-            {
-                TTransport transport = client.getOutputProtocol().getTransport();
-                if (transport.isOpen())
-                    transport.close();
-            }
-        }
-
-        /**
-         * Loops collecting mutations from the queue and sending to Cassandra
-         */
-        public abstract void run();
-
-        @Override
-        public String toString()
-        {
-            return "#<Client for " + endpoints.toString() + ">";
-        }
-    }
-}
-
diff --git a/src/java/org/apache/cassandra/hadoop/BulkOutputFormat.java b/src/java/org/apache/cassandra/hadoop/BulkOutputFormat.java
index f5a5a8d..5282279 100644
--- a/src/java/org/apache/cassandra/hadoop/BulkOutputFormat.java
+++ b/src/java/org/apache/cassandra/hadoop/BulkOutputFormat.java
@@ -23,9 +23,12 @@
 import java.util.List;
 
 import org.apache.cassandra.thrift.Mutation;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.*;
 
-public class BulkOutputFormat extends AbstractBulkOutputFormat<ByteBuffer,List<Mutation>>
+@Deprecated
+public class BulkOutputFormat extends OutputFormat<ByteBuffer,List<Mutation>>
+        implements org.apache.hadoop.mapred.OutputFormat<ByteBuffer,List<Mutation>>
 {
     /** Fills the deprecated OutputFormat interface for streaming. */
     @Deprecated
@@ -39,4 +42,50 @@
     {
         return new BulkRecordWriter(context);
     }
+
+
+    @Override
+    public void checkOutputSpecs(JobContext context)
+    {
+        checkOutputSpecs(HadoopCompat.getConfiguration(context));
+    }
+
+    private void checkOutputSpecs(Configuration conf)
+    {
+        if (ConfigHelper.getOutputKeyspace(conf) == null)
+        {
+            throw new UnsupportedOperationException("you must set the keyspace with setColumnFamily()");
+        }
+    }
+
+    @Override
+    public OutputCommitter getOutputCommitter(TaskAttemptContext context) throws IOException, InterruptedException
+    {
+        return new NullOutputCommitter();
+    }
+
+    /** Fills the deprecated OutputFormat interface for streaming. */
+    @Deprecated
+    public void checkOutputSpecs(org.apache.hadoop.fs.FileSystem filesystem, org.apache.hadoop.mapred.JobConf job) throws IOException
+    {
+        checkOutputSpecs(job);
+    }
+
+    public static class NullOutputCommitter extends OutputCommitter
+    {
+        public void abortTask(TaskAttemptContext taskContext) { }
+
+        public void cleanupJob(JobContext jobContext) { }
+
+        public void commitTask(TaskAttemptContext taskContext) { }
+
+        public boolean needsTaskCommit(TaskAttemptContext taskContext)
+        {
+            return false;
+        }
+
+        public void setupJob(JobContext jobContext) { }
+
+        public void setupTask(TaskAttemptContext taskContext) { }
+    }
 }
diff --git a/src/java/org/apache/cassandra/hadoop/BulkRecordWriter.java b/src/java/org/apache/cassandra/hadoop/BulkRecordWriter.java
index d67b856..6b9ecb5 100644
--- a/src/java/org/apache/cassandra/hadoop/BulkRecordWriter.java
+++ b/src/java/org/apache/cassandra/hadoop/BulkRecordWriter.java
@@ -17,24 +17,57 @@
  */
 package org.apache.cassandra.hadoop;
 
+import java.io.Closeable;
 import java.io.File;
 import java.io.IOException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
-import java.util.List;
+import java.util.*;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.hadoop.cql3.CqlConfigHelper;
 import org.apache.cassandra.io.sstable.SSTableLoader;
 import org.apache.cassandra.io.sstable.SSTableSimpleUnsortedWriter;
+import org.apache.cassandra.streaming.StreamState;
 import org.apache.cassandra.thrift.Column;
 import org.apache.cassandra.thrift.CounterColumn;
 import org.apache.cassandra.thrift.Mutation;
+import org.apache.cassandra.utils.NativeSSTableLoaderClient;
+import org.apache.cassandra.utils.OutputHandler;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.RecordWriter;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.util.Progressable;
 
-public final class BulkRecordWriter extends AbstractBulkRecordWriter<ByteBuffer, List<Mutation>>
+@Deprecated
+public final class BulkRecordWriter extends RecordWriter<ByteBuffer, List<Mutation>>
+        implements org.apache.hadoop.mapred.RecordWriter<ByteBuffer, List<Mutation>>
 {
+    public final static String OUTPUT_LOCATION = "mapreduce.output.bulkoutputformat.localdir";
+    public final static String BUFFER_SIZE_IN_MB = "mapreduce.output.bulkoutputformat.buffersize";
+    public final static String STREAM_THROTTLE_MBITS = "mapreduce.output.bulkoutputformat.streamthrottlembits";
+    public final static String MAX_FAILED_HOSTS = "mapreduce.output.bulkoutputformat.maxfailedhosts";
+
+    private final Logger logger = LoggerFactory.getLogger(BulkRecordWriter.class);
+
+    protected final Configuration conf;
+    protected final int maxFailures;
+    protected final int bufferSize;
+    protected Closeable writer;
+    protected SSTableLoader loader;
+    protected Progressable progress;
+    protected TaskAttemptContext context;
     private File outputDir;
     
     
@@ -55,17 +88,32 @@
 
     BulkRecordWriter(TaskAttemptContext context)
     {
-        super(context);
+
+        this(HadoopCompat.getConfiguration(context));
+        this.context = context;
     }
 
     BulkRecordWriter(Configuration conf, Progressable progress)
     {
-        super(conf, progress);
+        this(conf);
+        this.progress = progress;
     }
 
     BulkRecordWriter(Configuration conf)
     {
-        super(conf);
+        Config.setOutboundBindAny(true);
+        this.conf = conf;
+        DatabaseDescriptor.setStreamThroughputOutboundMegabitsPerSec(Integer.parseInt(conf.get(STREAM_THROTTLE_MBITS, "0")));
+        maxFailures = Integer.parseInt(conf.get(MAX_FAILED_HOSTS, "0"));
+        bufferSize = Integer.parseInt(conf.get(BUFFER_SIZE_IN_MB, "64"));
+    }
+
+    protected String getOutputLocation() throws IOException
+    {
+        String dir = conf.get(OUTPUT_LOCATION, System.getProperty("java.io.tmpdir"));
+        if (dir == null)
+            throw new IOException("Output directory not defined, if hadoop is not setting java.io.tmpdir then define " + OUTPUT_LOCATION);
+        return dir;
     }
 
     private void setTypes(Mutation mutation)
@@ -115,6 +163,54 @@
     }
 
     @Override
+    public void close(TaskAttemptContext context) throws IOException, InterruptedException
+    {
+        close();
+    }
+
+    /** Fills the deprecated RecordWriter interface for streaming. */
+    @Deprecated
+    public void close(org.apache.hadoop.mapred.Reporter reporter) throws IOException
+    {
+        close();
+    }
+
+    private void close() throws IOException
+    {
+        if (writer != null)
+        {
+            writer.close();
+            Future<StreamState> future = loader.stream();
+            while (true)
+            {
+                try
+                {
+                    future.get(1000, TimeUnit.MILLISECONDS);
+                    break;
+                }
+                catch (ExecutionException | TimeoutException te)
+                {
+                    if (null != progress)
+                        progress.progress();
+                    if (null != context)
+                        HadoopCompat.progress(context);
+                }
+                catch (InterruptedException e)
+                {
+                    throw new IOException(e);
+                }
+            }
+            if (loader.getFailedHosts().size() > 0)
+            {
+                if (loader.getFailedHosts().size() > maxFailures)
+                    throw new IOException("Too many hosts failed: " + loader.getFailedHosts());
+                else
+                    logger.warn("Some hosts failed: {}", loader.getFailedHosts());
+            }
+        }
+    }
+
+    @Override
     public void write(ByteBuffer keybuff, List<Mutation> value) throws IOException
     {
         setTypes(value.get(0));
@@ -158,4 +254,43 @@
                 HadoopCompat.progress(context);
         }
     }
+
+    public static class ExternalClient extends NativeSSTableLoaderClient
+    {
+        public ExternalClient(Configuration conf)
+        {
+            super(resolveHostAddresses(conf),
+                  CqlConfigHelper.getOutputNativePort(conf),
+                  ConfigHelper.getOutputKeyspaceUserName(conf),
+                  ConfigHelper.getOutputKeyspacePassword(conf),
+                  CqlConfigHelper.getSSLOptions(conf).orNull());
+        }
+
+        private static Collection<InetAddress> resolveHostAddresses(Configuration conf)
+        {
+            Set<InetAddress> addresses = new HashSet<>();
+
+            for (String host : ConfigHelper.getOutputInitialAddress(conf).split(","))
+            {
+                try
+                {
+                    addresses.add(InetAddress.getByName(host));
+                }
+                catch (UnknownHostException e)
+                {
+                    throw new RuntimeException(e);
+                }
+            }
+
+            return addresses;
+        }
+    }
+
+    public static class NullOutputHandler implements OutputHandler
+    {
+        public void output(String msg) {}
+        public void debug(String msg) {}
+        public void warn(String msg) {}
+        public void warn(String msg, Throwable th) {}
+    }
 }
diff --git a/src/java/org/apache/cassandra/hadoop/ColumnFamilyInputFormat.java b/src/java/org/apache/cassandra/hadoop/ColumnFamilyInputFormat.java
index 686d486..87cb791 100644
--- a/src/java/org/apache/cassandra/hadoop/ColumnFamilyInputFormat.java
+++ b/src/java/org/apache/cassandra/hadoop/ColumnFamilyInputFormat.java
@@ -21,11 +21,22 @@
 import java.nio.ByteBuffer;
 import java.util.*;
 
-import org.apache.cassandra.db.Cell;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.auth.PasswordAuthenticator;
+import org.apache.cassandra.thrift.*;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.mapreduce.*;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.transport.TTransportException;
 
 /**
  * Hadoop InputFormat allowing map/reduce against Cassandra rows within one ColumnFamily.
@@ -44,15 +55,47 @@
  *
  * The default split size is 64k rows.
  */
-public class ColumnFamilyInputFormat extends AbstractColumnFamilyInputFormat<ByteBuffer, SortedMap<ByteBuffer, Cell>>
+@Deprecated
+public class ColumnFamilyInputFormat extends AbstractColumnFamilyInputFormat<ByteBuffer, SortedMap<ByteBuffer, ColumnFamilyRecordReader.Column>>
 {
-    
-    public RecordReader<ByteBuffer, SortedMap<ByteBuffer, Cell>> createRecordReader(InputSplit inputSplit, TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException
+    private static final Logger logger = LoggerFactory.getLogger(ColumnFamilyInputFormat.class);
+
+    @SuppressWarnings("resource")
+    public static Cassandra.Client createAuthenticatedClient(String location, int port, Configuration conf) throws Exception
+    {
+        logger.trace("Creating authenticated client for CF input format");
+        TTransport transport;
+        try
+        {
+            transport = ConfigHelper.getClientTransportFactory(conf).openTransport(location, port);
+        }
+        catch (Exception e)
+        {
+            throw new TTransportException("Failed to open a transport to " + location + ":" + port + ".", e);
+        }
+        TProtocol binaryProtocol = new TBinaryProtocol(transport, true, true);
+        Cassandra.Client client = new Cassandra.Client(binaryProtocol);
+
+        // log in
+        client.set_keyspace(ConfigHelper.getInputKeyspace(conf));
+        if ((ConfigHelper.getInputKeyspaceUserName(conf) != null) && (ConfigHelper.getInputKeyspacePassword(conf) != null))
+        {
+            Map<String, String> creds = new HashMap<String, String>();
+            creds.put(PasswordAuthenticator.USERNAME_KEY, ConfigHelper.getInputKeyspaceUserName(conf));
+            creds.put(PasswordAuthenticator.PASSWORD_KEY, ConfigHelper.getInputKeyspacePassword(conf));
+            AuthenticationRequest authRequest = new AuthenticationRequest(creds);
+            client.login(authRequest);
+        }
+        logger.trace("Authenticated client for CF input format created successfully");
+        return client;
+    }
+
+    public RecordReader<ByteBuffer, SortedMap<ByteBuffer, ColumnFamilyRecordReader.Column>> createRecordReader(InputSplit inputSplit, TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException
     {
         return new ColumnFamilyRecordReader();
     }
 
-    public org.apache.hadoop.mapred.RecordReader<ByteBuffer, SortedMap<ByteBuffer, Cell>> getRecordReader(org.apache.hadoop.mapred.InputSplit split, JobConf jobConf, final Reporter reporter) throws IOException
+    public org.apache.hadoop.mapred.RecordReader<ByteBuffer, SortedMap<ByteBuffer, ColumnFamilyRecordReader.Column>> getRecordReader(org.apache.hadoop.mapred.InputSplit split, JobConf jobConf, final Reporter reporter) throws IOException
     {
         TaskAttemptContext tac = HadoopCompat.newMapContext(
                 jobConf,
diff --git a/src/java/org/apache/cassandra/hadoop/ColumnFamilyOutputFormat.java b/src/java/org/apache/cassandra/hadoop/ColumnFamilyOutputFormat.java
index 49aaf99..edc988b 100644
--- a/src/java/org/apache/cassandra/hadoop/ColumnFamilyOutputFormat.java
+++ b/src/java/org/apache/cassandra/hadoop/ColumnFamilyOutputFormat.java
@@ -18,11 +18,18 @@
 package org.apache.cassandra.hadoop;
 
 
-import java.io.IOException;
+import java.io.*;
 import java.nio.ByteBuffer;
-import java.util.List;
+import java.util.*;
+
+import org.slf4j.*;
+
+import org.apache.cassandra.auth.*;
 import org.apache.cassandra.thrift.*;
+import org.apache.hadoop.conf.*;
 import org.apache.hadoop.mapreduce.*;
+import org.apache.thrift.protocol.*;
+import org.apache.thrift.transport.*;
 
 /**
  * The <code>ColumnFamilyOutputFormat</code> acts as a Hadoop-specific
@@ -45,8 +52,94 @@
  * official by sending a batch mutate request to Cassandra.
  * </p>
  */
-public class ColumnFamilyOutputFormat extends AbstractColumnFamilyOutputFormat<ByteBuffer,List<Mutation>>
+@Deprecated
+public class ColumnFamilyOutputFormat extends OutputFormat<ByteBuffer,List<Mutation>>
+        implements org.apache.hadoop.mapred.OutputFormat<ByteBuffer,List<Mutation>>
 {
+    public static final String BATCH_THRESHOLD = "mapreduce.output.columnfamilyoutputformat.batch.threshold";
+    public static final String QUEUE_SIZE = "mapreduce.output.columnfamilyoutputformat.queue.size";
+
+    private static final Logger logger = LoggerFactory.getLogger(ColumnFamilyOutputFormat.class);
+
+    /**
+     * The OutputCommitter for this format does not write any data to the DFS.
+     *
+     * @param context
+     *            the task context
+     * @return an output committer
+     * @throws IOException
+     * @throws InterruptedException
+     */
+    public OutputCommitter getOutputCommitter(TaskAttemptContext context) throws IOException, InterruptedException
+    {
+        return new NullOutputCommitter();
+    }
+
+    /**
+     * Check for validity of the output-specification for the job.
+     *
+     * @param context
+     *            information about the job
+     */
+    public void checkOutputSpecs(JobContext context)
+    {
+        checkOutputSpecs(HadoopCompat.getConfiguration(context));
+    }
+
+    protected void checkOutputSpecs(Configuration conf)
+    {
+        if (ConfigHelper.getOutputKeyspace(conf) == null)
+            throw new UnsupportedOperationException("You must set the keyspace with setOutputKeyspace()");
+        if (ConfigHelper.getOutputPartitioner(conf) == null)
+            throw new UnsupportedOperationException("You must set the output partitioner to the one used by your Cassandra cluster");
+        if (ConfigHelper.getOutputInitialAddress(conf) == null)
+            throw new UnsupportedOperationException("You must set the initial output address to a Cassandra node");
+    }
+
+    /** Fills the deprecated OutputFormat interface for streaming. */
+    @Deprecated
+    public void checkOutputSpecs(org.apache.hadoop.fs.FileSystem filesystem, org.apache.hadoop.mapred.JobConf job) throws IOException
+    {
+        checkOutputSpecs(job);
+    }
+
+    /**
+     * Connects to the given server:port and returns a client based on the given socket that points to the configured
+     * keyspace, and is logged in with the configured credentials.
+     *
+     * @param host fully qualified host name to connect to
+     * @param port RPC port of the server
+     * @param conf a job configuration
+     * @return a cassandra client
+     * @throws Exception set of thrown exceptions may be implementation defined,
+     *                   depending on the used transport factory
+     */
+    @SuppressWarnings("resource")
+    public static Cassandra.Client createAuthenticatedClient(String host, int port, Configuration conf) throws Exception
+    {
+        logger.trace("Creating authenticated client for CF output format");
+        TTransport transport = ConfigHelper.getClientTransportFactory(conf).openTransport(host, port);
+        TProtocol binaryProtocol = new TBinaryProtocol(transport, true, true);
+        Cassandra.Client client = new Cassandra.Client(binaryProtocol);
+        client.set_keyspace(ConfigHelper.getOutputKeyspace(conf));
+        String user = ConfigHelper.getOutputKeyspaceUserName(conf);
+        String password = ConfigHelper.getOutputKeyspacePassword(conf);
+        if ((user != null) && (password != null))
+            login(user, password, client);
+
+        logger.trace("Authenticated client for CF output format created successfully");
+        return client;
+    }
+
+    public static void login(String user, String password, Cassandra.Client client) throws Exception
+    {
+        Map<String, String> creds = new HashMap<String, String>();
+        creds.put(PasswordAuthenticator.USERNAME_KEY, user);
+        creds.put(PasswordAuthenticator.PASSWORD_KEY, password);
+        AuthenticationRequest authRequest = new AuthenticationRequest(creds);
+        client.login(authRequest);
+    }
+
     /** Fills the deprecated OutputFormat interface for streaming. */
     @Deprecated
     public ColumnFamilyRecordWriter getRecordWriter(org.apache.hadoop.fs.FileSystem filesystem, org.apache.hadoop.mapred.JobConf job, String name, org.apache.hadoop.util.Progressable progress)
@@ -60,10 +153,31 @@
      * @param context
      *            the information about the current task.
      * @return a {@link RecordWriter} to write the output for the job.
-     * @throws IOException
      */
     public ColumnFamilyRecordWriter getRecordWriter(final TaskAttemptContext context) throws InterruptedException
     {
         return new ColumnFamilyRecordWriter(context);
     }
+
+    /**
+     * An {@link OutputCommitter} that does nothing.
+     */
+    private static class NullOutputCommitter extends OutputCommitter
+    {
+        public void abortTask(TaskAttemptContext taskContext) { }
+
+        public void cleanupJob(JobContext jobContext) { }
+
+        public void commitTask(TaskAttemptContext taskContext) { }
+
+        public boolean needsTaskCommit(TaskAttemptContext taskContext)
+        {
+            return false;
+        }
+
+        public void setupJob(JobContext jobContext) { }
+
+        public void setupTask(TaskAttemptContext taskContext) { }
+    }
+
 }
diff --git a/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java b/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
index dc44a43..9d1d10c 100644
--- a/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
+++ b/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
@@ -24,17 +24,16 @@
 import java.util.*;
 
 import com.google.common.collect.*;
-import org.apache.cassandra.db.BufferCell;
-import org.apache.cassandra.db.Cell;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.db.composites.CellNames;
+import org.apache.cassandra.schema.LegacySchemaTables;
+import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.CompositeType;
 import org.apache.cassandra.db.marshal.TypeParser;
 import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.thrift.*;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
@@ -46,8 +45,9 @@
 import org.apache.thrift.TException;
 import org.apache.thrift.transport.TTransport;
 
-public class ColumnFamilyRecordReader extends RecordReader<ByteBuffer, SortedMap<ByteBuffer, Cell>>
-    implements org.apache.hadoop.mapred.RecordReader<ByteBuffer, SortedMap<ByteBuffer, Cell>>
+@Deprecated
+public class ColumnFamilyRecordReader extends RecordReader<ByteBuffer, SortedMap<ByteBuffer, ColumnFamilyRecordReader.Column>>
+    implements org.apache.hadoop.mapred.RecordReader<ByteBuffer, SortedMap<ByteBuffer, ColumnFamilyRecordReader.Column>>
 {
     private static final Logger logger = LoggerFactory.getLogger(ColumnFamilyRecordReader.class);
 
@@ -55,7 +55,7 @@
 
     private ColumnFamilySplit split;
     private RowIterator iter;
-    private Pair<ByteBuffer, SortedMap<ByteBuffer, Cell>> currentRow;
+    private Pair<ByteBuffer, SortedMap<ByteBuffer, Column>> currentRow;
     private SlicePredicate predicate;
     private boolean isEmptyPredicate;
     private int totalRowCount; // total number of rows to fetch
@@ -79,6 +79,7 @@
         this.keyBufferSize = keyBufferSize;
     }
 
+    @SuppressWarnings("resource")
     public void close()
     {
         if (client != null)
@@ -94,7 +95,7 @@
         return currentRow.left;
     }
 
-    public SortedMap<ByteBuffer, Cell> getCurrentValue()
+    public SortedMap<ByteBuffer, Column> getCurrentValue()
     {
         return currentRow.right;
     }
@@ -172,14 +173,14 @@
             throw new RuntimeException(lastException);
 
         iter = widerows ? new WideRowIterator() : new StaticRowIterator();
-        logger.debug("created {}", iter);
+        logger.trace("created {}", iter);
     }
 
     public boolean nextKeyValue() throws IOException
     {
         if (!iter.hasNext())
         {
-            logger.debug("Finished scanning {} rows (estimate was: {})", iter.rowsRead(), totalRowCount);
+            logger.trace("Finished scanning {} rows (estimate was: {})", iter.rowsRead(), totalRowCount);
             return false;
         }
 
@@ -215,7 +216,7 @@
         return split.getLocations();
     }
 
-    private abstract class RowIterator extends AbstractIterator<Pair<ByteBuffer, SortedMap<ByteBuffer, Cell>>>
+    private abstract class RowIterator extends AbstractIterator<Pair<ByteBuffer, SortedMap<ByteBuffer, Column>>>
     {
         protected List<KeySlice> rows;
         protected int totalRead = 0;
@@ -231,17 +232,15 @@
             {
                 partitioner = FBUtilities.newPartitioner(client.describe_partitioner());           
                 // get CF meta data
-                String query = "SELECT comparator," +
-                               "       subcomparator," +
-                               "       type " +
-                               "FROM system.schema_columnfamilies " +
-                               "WHERE keyspace_name = '%s' " +
-                               "  AND columnfamily_name = '%s' ";
+                String query = String.format("SELECT comparator, subcomparator, type " +
+                                             "FROM %s.%s " +
+                                             "WHERE keyspace_name = '%s' AND columnfamily_name = '%s'",
+                                             SystemKeyspace.NAME,
+                                             LegacySchemaTables.COLUMNFAMILIES,
+                                             keyspace,
+                                             cfName);
 
-                CqlResult result = client.execute_cql3_query(
-                                        ByteBufferUtil.bytes(String.format(query, keyspace, cfName)),
-                                        Compression.NONE,
-                                        ConsistencyLevel.ONE);
+                CqlResult result = client.execute_cql3_query(ByteBufferUtil.bytes(query), Compression.NONE, ConsistencyLevel.ONE);
 
                 Iterator<CqlRow> iteraRow = result.rows.iterator();
 
@@ -284,50 +283,48 @@
             return totalRead;
         }
 
-        protected List<Cell> unthriftify(ColumnOrSuperColumn cosc)
+        protected List<Pair<ByteBuffer, Column>> unthriftify(ColumnOrSuperColumn cosc)
         {
             if (cosc.counter_column != null)
-                return Collections.<Cell>singletonList(unthriftifyCounter(cosc.counter_column));
+                return Collections.singletonList(unthriftifyCounter(cosc.counter_column));
             if (cosc.counter_super_column != null)
                 return unthriftifySuperCounter(cosc.counter_super_column);
             if (cosc.super_column != null)
                 return unthriftifySuper(cosc.super_column);
             assert cosc.column != null;
-            return Collections.<Cell>singletonList(unthriftifySimple(cosc.column));
+            return Collections.singletonList(unthriftifySimple(cosc.column));
         }
 
-        private List<Cell> unthriftifySuper(SuperColumn super_column)
+        private List<Pair<ByteBuffer, Column>> unthriftifySuper(SuperColumn super_column)
         {
-            List<Cell> cells = new ArrayList<Cell>(super_column.columns.size());
+            List<Pair<ByteBuffer, Column>> columns = new ArrayList<>(super_column.columns.size());
             for (org.apache.cassandra.thrift.Column column : super_column.columns)
             {
-                Cell c = unthriftifySimple(column);
-                cells.add(c.withUpdatedName(CellNames.simpleDense(CompositeType.build(super_column.name, c.name().toByteBuffer()))));
+                Pair<ByteBuffer, Column> c = unthriftifySimple(column);
+                columns.add(Pair.create(CompositeType.build(super_column.name, c.left), c.right));
             }
-            return cells;
+            return columns;
         }
 
-        protected Cell unthriftifySimple(org.apache.cassandra.thrift.Column column)
+        protected Pair<ByteBuffer, Column> unthriftifySimple(org.apache.cassandra.thrift.Column column)
         {
-            return new BufferCell(CellNames.simpleDense(column.name), column.value, column.timestamp);
+            return Pair.create(column.name, Column.fromRegularColumn(column));
         }
 
-        private Cell unthriftifyCounter(CounterColumn column)
+        private Pair<ByteBuffer, Column> unthriftifyCounter(CounterColumn column)
         {
-            //CounterColumns read the counterID from the System keyspace, so need the StorageService running and access
-            //to cassandra.yaml. To avoid a Hadoop needing access to yaml return a regular Cell.
-            return new BufferCell(CellNames.simpleDense(column.name), ByteBufferUtil.bytes(column.value), 0);
+            return Pair.create(column.name, Column.fromCounterColumn(column));
         }
 
-        private List<Cell> unthriftifySuperCounter(CounterSuperColumn super_column)
+        private List<Pair<ByteBuffer, Column>> unthriftifySuperCounter(CounterSuperColumn super_column)
         {
-            List<Cell> cells = new ArrayList<Cell>(super_column.columns.size());
+            List<Pair<ByteBuffer, Column>> columns = new ArrayList<>(super_column.columns.size());
             for (CounterColumn column : super_column.columns)
             {
-                Cell c = unthriftifyCounter(column);
-                cells.add(c.withUpdatedName(CellNames.simpleDense(CompositeType.build(super_column.name, c.name().toByteBuffer()))));
+                Pair<ByteBuffer, Column> c = unthriftifyCounter(column);
+                columns.add(Pair.create(CompositeType.build(super_column.name, c.left), c.right));
             }
-            return cells;
+            return columns;
         }
     }
 
@@ -406,7 +403,7 @@
             }
         }
 
-        protected Pair<ByteBuffer, SortedMap<ByteBuffer, Cell>> computeNext()
+        protected Pair<ByteBuffer, SortedMap<ByteBuffer, Column>> computeNext()
         {
             maybeInit();
             if (rows == null)
@@ -415,12 +412,12 @@
             totalRead++;
             KeySlice ks = rows.get(i++);
             AbstractType<?> comp = isSuper ? CompositeType.getInstance(comparator, subComparator) : comparator;
-            SortedMap<ByteBuffer, Cell> map = new TreeMap<ByteBuffer, Cell>(comp);
+            SortedMap<ByteBuffer, Column> map = new TreeMap<>(comp);
             for (ColumnOrSuperColumn cosc : ks.columns)
             {
-                List<Cell> cells = unthriftify(cosc);
-                for (Cell cell : cells)
-                    map.put(cell.name().toByteBuffer(), cell);
+                List<Pair<ByteBuffer, Column>> columns = unthriftify(cosc);
+                for (Pair<ByteBuffer, Column> column : columns)
+                    map.put(column.left, column.right);
             }
             return Pair.create(ks.key, map);
         }
@@ -428,7 +425,7 @@
 
     private class WideRowIterator extends RowIterator
     {
-        private PeekingIterator<Pair<ByteBuffer, SortedMap<ByteBuffer, Cell>>> wideColumns;
+        private PeekingIterator<Pair<ByteBuffer, SortedMap<ByteBuffer, Column>>> wideColumns;
         private ByteBuffer lastColumn = ByteBufferUtil.EMPTY_BYTE_BUFFER;
         private ByteBuffer lastCountedKey = ByteBufferUtil.EMPTY_BYTE_BUFFER;
 
@@ -449,7 +446,7 @@
             else
             {
                 KeySlice lastRow = Iterables.getLast(rows);
-                logger.debug("Starting with last-seen row {}", lastRow.key);
+                logger.trace("Starting with last-seen row {}", lastRow.key);
                 keyRange = new KeyRange(batchSize)
                           .setStart_key(lastRow.key)
                           .setEnd_token(split.getEndToken())
@@ -462,7 +459,7 @@
                 int n = 0;
                 for (KeySlice row : rows)
                     n += row.columns.size();
-                logger.debug("read {} columns in {} rows for {} starting with {}",
+                logger.trace("read {} columns in {} rows for {} starting with {}",
                              new Object[]{ n, rows.size(), keyRange, lastColumn });
 
                 wideColumns = Iterators.peekingIterator(new WideColumnIterator(rows));
@@ -477,13 +474,13 @@
             }
         }
 
-        protected Pair<ByteBuffer, SortedMap<ByteBuffer, Cell>> computeNext()
+        protected Pair<ByteBuffer, SortedMap<ByteBuffer, Column>> computeNext()
         {
             maybeInit();
             if (rows == null)
                 return endOfData();
 
-            Pair<ByteBuffer, SortedMap<ByteBuffer, Cell>> next = wideColumns.next();
+            Pair<ByteBuffer, SortedMap<ByteBuffer, Column>> next = wideColumns.next();
             lastColumn = next.right.keySet().iterator().next().duplicate();
 
             maybeIncreaseRowCounter(next);
@@ -495,7 +492,7 @@
          * Increases the row counter only if we really moved to the next row.
          * @param next just fetched row slice
          */
-        private void maybeIncreaseRowCounter(Pair<ByteBuffer, SortedMap<ByteBuffer, Cell>> next)
+        private void maybeIncreaseRowCounter(Pair<ByteBuffer, SortedMap<ByteBuffer, Column>> next)
         {
             ByteBuffer currentKey = next.left;
             if (!currentKey.equals(lastCountedKey))
@@ -505,7 +502,7 @@
             }
         }
 
-        private class WideColumnIterator extends AbstractIterator<Pair<ByteBuffer, SortedMap<ByteBuffer, Cell>>>
+        private class WideColumnIterator extends AbstractIterator<Pair<ByteBuffer, SortedMap<ByteBuffer, Column>>>
         {
             private final Iterator<KeySlice> rows;
             private Iterator<ColumnOrSuperColumn> columns;
@@ -526,7 +523,7 @@
                 columns = currentRow.columns.iterator();
             }
 
-            protected Pair<ByteBuffer, SortedMap<ByteBuffer, Cell>> computeNext()
+            protected Pair<ByteBuffer, SortedMap<ByteBuffer, Column>> computeNext()
             {
                 AbstractType<?> comp = isSuper ? CompositeType.getInstance(comparator, subComparator) : comparator;
                 while (true)
@@ -534,20 +531,20 @@
                     if (columns.hasNext())
                     {
                         ColumnOrSuperColumn cosc = columns.next();
-                        SortedMap<ByteBuffer, Cell> map;
-                        List<Cell> cells = unthriftify(cosc);
-                        if (cells.size() == 1)
+                        SortedMap<ByteBuffer, Column> map;
+                        List<Pair<ByteBuffer, Column>> columns = unthriftify(cosc);
+                        if (columns.size() == 1)
                         {
-                            map = ImmutableSortedMap.of(cells.get(0).name().toByteBuffer(), cells.get(0));
+                            map = ImmutableSortedMap.of(columns.get(0).left, columns.get(0).right);
                         }
                         else
                         {
                             assert isSuper;
-                            map = new TreeMap<ByteBuffer, Cell>(comp);
-                            for (Cell cell : cells)
-                                map.put(cell.name().toByteBuffer(), cell);
+                            map = new TreeMap<>(comp);
+                            for (Pair<ByteBuffer, Column> column : columns)
+                                map.put(column.left, column.right);
                         }
-                        return Pair.<ByteBuffer, SortedMap<ByteBuffer, Cell>>create(currentRow.key, map);
+                        return Pair.create(currentRow.key, map);
                     }
 
                     if (!rows.hasNext())
@@ -564,7 +561,7 @@
     // to the old. Thus, expect a small performance hit.
     // And obviously this wouldn't work for wide rows. But since ColumnFamilyInputFormat
     // and ColumnFamilyRecordReader don't support them, it should be fine for now.
-    public boolean next(ByteBuffer key, SortedMap<ByteBuffer, Cell> value) throws IOException
+    public boolean next(ByteBuffer key, SortedMap<ByteBuffer, Column> value) throws IOException
     {
         if (this.nextKeyValue())
         {
@@ -585,13 +582,37 @@
         return ByteBuffer.wrap(new byte[this.keyBufferSize]);
     }
 
-    public SortedMap<ByteBuffer, Cell> createValue()
+    public SortedMap<ByteBuffer, Column> createValue()
     {
-        return new TreeMap<ByteBuffer, Cell>();
+        return new TreeMap<>();
     }
 
     public long getPos() throws IOException
     {
-        return (long)iter.rowsRead();
+        return iter.rowsRead();
+    }
+
+    public static final class Column
+    {
+        public final ByteBuffer name;
+        public final ByteBuffer value;
+        public final long timestamp;
+
+        private Column(ByteBuffer name, ByteBuffer value, long timestamp)
+        {
+            this.name = name;
+            this.value = value;
+            this.timestamp = timestamp;
+        }
+
+        static Column fromRegularColumn(org.apache.cassandra.thrift.Column input)
+        {
+            return new Column(input.name, input.value, input.timestamp);
+        }
+
+        static Column fromCounterColumn(org.apache.cassandra.thrift.CounterColumn input)
+        {
+            return new Column(input.name, ByteBufferUtil.bytes(input.value), 0);
+        }
     }
 }
diff --git a/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordWriter.java b/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordWriter.java
index d6a873b..f06f03d 100644
--- a/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordWriter.java
+++ b/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordWriter.java
@@ -22,15 +22,19 @@
 import java.net.InetAddress;
 import java.nio.ByteBuffer;
 import java.util.*;
+import java.util.concurrent.*;
 
+import org.apache.cassandra.client.*;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.thrift.*;
-import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.thrift.ConsistencyLevel;
+import org.apache.cassandra.utils.*;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.*;
 import org.apache.thrift.TException;
 import org.apache.hadoop.util.Progressable;
+import org.apache.thrift.transport.*;
 
 
 /**
@@ -47,10 +51,30 @@
  *
  * @see ColumnFamilyOutputFormat
  */
-final class ColumnFamilyRecordWriter extends AbstractColumnFamilyRecordWriter<ByteBuffer, List<Mutation>>
+@Deprecated
+final class ColumnFamilyRecordWriter extends RecordWriter<ByteBuffer, List<Mutation>> implements
+        org.apache.hadoop.mapred.RecordWriter<ByteBuffer, List<Mutation>>
 {
+    // The configuration this writer is associated with.
+    protected final Configuration conf;
+
+    // The number of mutations to buffer per endpoint
+    protected final int queueSize;
+
+    protected final long batchThreshold;
+
+    protected final ConsistencyLevel consistencyLevel;
+    protected Progressable progressable;
+    protected TaskAttemptContext context;
     // handles for clients for each range running in the threadpool
     private final Map<Range, RangeClient> clients;
+
+    // The ring cache that describes the token ranges each node in the ring is
+    // responsible for. This is what allows us to group the mutations by
+    // the endpoints they should be targeted at. The targeted endpoint
+    // essentially
+    // acts as the primary replica for the rows being affected by the mutations.
+    private final RingCache ringCache;
     
     /**
      * Upon construction, obtain the map that this writer will use to collect
@@ -73,11 +97,33 @@
 
     ColumnFamilyRecordWriter(Configuration conf)
     {
-        super(conf);
+        this.conf = conf;
+        this.queueSize = conf.getInt(ColumnFamilyOutputFormat.QUEUE_SIZE, 32 * FBUtilities.getAvailableProcessors());
+        batchThreshold = conf.getLong(ColumnFamilyOutputFormat.BATCH_THRESHOLD, 32);
+        consistencyLevel = ConsistencyLevel.valueOf(ConfigHelper.getWriteConsistencyLevel(conf));
+        this.ringCache = new RingCache(conf);
         this.clients = new HashMap<Range, RangeClient>();
     }
-    
-    @Override
+
+    /**
+     * Close this <code>RecordWriter</code> to future operations, but not before
+     * flushing out the batched mutations.
+     *
+     * @param context the context of the task
+     * @throws IOException
+     */
+    public void close(TaskAttemptContext context) throws IOException, InterruptedException
+    {
+        close();
+    }
+
+    /** Fills the deprecated RecordWriter interface for streaming. */
+    @Deprecated
+    public void close(org.apache.hadoop.mapred.Reporter reporter) throws IOException
+    {
+        close();
+    }
+
     public void close() throws IOException
     {
         // close all the clients before throwing anything
@@ -138,8 +184,20 @@
      * A client that runs in a threadpool and connects to the list of endpoints for a particular
      * range. Mutations for keys in that range are sent to this client via a queue.
      */
-    public class RangeClient extends AbstractRangeClient<Pair<ByteBuffer, Mutation>>
+    public class RangeClient extends Thread
     {
+        // The list of endpoints for this range
+        protected final List<InetAddress> endpoints;
+        // A bounded queue of incoming mutations for this range
+        protected final BlockingQueue<Pair<ByteBuffer, Mutation>> queue = new ArrayBlockingQueue<>(queueSize);
+
+        protected volatile boolean run = true;
+        // we want the caller to know if something went wrong, so we record any unrecoverable exception while writing
+        // so we can throw it on the caller's stack when he calls put() again, or if there are no more put calls,
+        // when the client is closed.
+        protected volatile IOException lastException;
+
+        protected Cassandra.Client client;
         public final String columnFamily = ConfigHelper.getOutputColumnFamily(conf);
         
         /**
@@ -148,8 +206,59 @@
         */
         public RangeClient(List<InetAddress> endpoints)
         {
-            super(endpoints);
+            super("client-" + endpoints);
+            this.endpoints = endpoints;
          }
+
+        /**
+         * enqueues the given value to Cassandra
+         */
+        public void put(Pair<ByteBuffer, Mutation> value) throws IOException
+        {
+            while (true)
+            {
+                if (lastException != null)
+                    throw lastException;
+                try
+                {
+                    if (queue.offer(value, 100, TimeUnit.MILLISECONDS))
+                        break;
+                }
+                catch (InterruptedException e)
+                {
+                    throw new AssertionError(e);
+                }
+            }
+        }
+
+        public void close() throws IOException
+        {
+            // stop the run loop.  this will result in closeInternal being called by the time join() finishes.
+            run = false;
+            interrupt();
+            try
+            {
+                this.join();
+            }
+            catch (InterruptedException e)
+            {
+                throw new AssertionError(e);
+            }
+
+            if (lastException != null)
+                throw lastException;
+        }
+
+        @SuppressWarnings("resource")
+        protected void closeInternal()
+        {
+            if (client != null)
+            {
+                TTransport transport = client.getOutputProtocol().getTransport();
+                if (transport.isOpen())
+                    transport.close();
+            }
+        }
         
         /**
          * Loops collecting mutations from the queue and sending to Cassandra
diff --git a/src/java/org/apache/cassandra/hadoop/ColumnFamilySplit.java b/src/java/org/apache/cassandra/hadoop/ColumnFamilySplit.java
index 0ebadc0..3625685 100644
--- a/src/java/org/apache/cassandra/hadoop/ColumnFamilySplit.java
+++ b/src/java/org/apache/cassandra/hadoop/ColumnFamilySplit.java
@@ -22,6 +22,7 @@
 
 import java.io.DataInput;
 import java.io.DataOutput;
+import java.io.EOFException;
 import java.io.IOException;
 import java.util.Arrays;
 
@@ -84,6 +85,7 @@
         {
             out.writeUTF(endpoint);
         }
+        out.writeLong(length);
     }
 
     public void readFields(DataInput in) throws IOException
@@ -96,6 +98,14 @@
         {
             dataNodes[i] = in.readUTF();
         }
+        try
+        {
+            length = in.readLong();
+        }
+        catch (EOFException e)
+        {
+            //We must be deserializing in a mixed-version cluster.
+        }
     }
 
     @Override
diff --git a/src/java/org/apache/cassandra/hadoop/ConfigHelper.java b/src/java/org/apache/cassandra/hadoop/ConfigHelper.java
index 10cfe8e..376c250 100644
--- a/src/java/org/apache/cassandra/hadoop/ConfigHelper.java
+++ b/src/java/org/apache/cassandra/hadoop/ConfigHelper.java
@@ -26,7 +26,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.io.compress.CompressionParameters;
 import org.apache.cassandra.thrift.*;
@@ -55,6 +54,7 @@
     private static final String INPUT_PREDICATE_CONFIG = "cassandra.input.predicate";
     private static final String INPUT_KEYRANGE_CONFIG = "cassandra.input.keyRange";
     private static final String INPUT_SPLIT_SIZE_CONFIG = "cassandra.input.split.size";
+    private static final String INPUT_SPLIT_SIZE_IN_MB_CONFIG = "cassandra.input.split.size_mb";
     private static final String INPUT_WIDEROWS_CONFIG = "cassandra.input.widerows";
     private static final int DEFAULT_SPLIT_SIZE = 64 * 1024;
     private static final String RANGE_BATCH_SIZE_CONFIG = "cassandra.range.batch.size";
@@ -86,7 +86,7 @@
             throw new UnsupportedOperationException("keyspace may not be null");
 
         if (columnFamily == null)
-            throw new UnsupportedOperationException("columnfamily may not be null");
+            throw new UnsupportedOperationException("table may not be null");
 
         conf.set(INPUT_KEYSPACE_CONFIG, keyspace);
         conf.set(INPUT_COLUMNFAMILY_CONFIG, columnFamily);
@@ -177,7 +177,7 @@
      * the overhead of each map will take up the bulk of the job time.
      *
      * @param conf      Job configuration you are about to run
-     * @param splitsize Size of the input split
+     * @param splitsize Number of partitions in the input split
      */
     public static void setInputSplitSize(Configuration conf, int splitsize)
     {
@@ -190,6 +190,29 @@
     }
 
     /**
+     * Set the size of the input split. getInputSplitSize value is used if this is not set.
+     * This affects the number of maps created, if the number is too small
+     * the overhead of each map will take up the bulk of the job time.
+     *
+     * @param conf        Job configuration you are about to run
+     * @param splitSizeMb Input split size in MB
+     */
+    public static void setInputSplitSizeInMb(Configuration conf, int splitSizeMb)
+    {
+        conf.setInt(INPUT_SPLIT_SIZE_IN_MB_CONFIG, splitSizeMb);
+    }
+
+    /**
+     * cassandra.input.split.size will be used if the value is undefined or negative.
+     * @param conf  Job configuration you are about to run
+     * @return      split size in MB or -1 if it is undefined.
+     */
+    public static int getInputSplitSizeInMb(Configuration conf)
+    {
+        return conf.getInt(INPUT_SPLIT_SIZE_IN_MB_CONFIG, -1);
+    }
+
+    /**
      * Set the predicate that determines what columns will be selected from each row.
      *
      * @param conf      Job configuration you are about to run
@@ -417,14 +440,7 @@
 
     public static IPartitioner getInputPartitioner(Configuration conf)
     {
-        try
-        {
-            return FBUtilities.newPartitioner(conf.get(INPUT_PARTITIONER_CONFIG));
-        }
-        catch (ConfigurationException e)
-        {
-            throw new RuntimeException(e);
-        }
+        return FBUtilities.newPartitioner(conf.get(INPUT_PARTITIONER_CONFIG));
     }
 
     public static int getOutputRpcPort(Configuration conf)
@@ -454,14 +470,7 @@
 
     public static IPartitioner getOutputPartitioner(Configuration conf)
     {
-        try
-        {
-            return FBUtilities.newPartitioner(conf.get(OUTPUT_PARTITIONER_CONFIG));
-        }
-        catch (ConfigurationException e)
-        {
-            throw new RuntimeException(e);
-        }
+        return FBUtilities.newPartitioner(conf.get(OUTPUT_PARTITIONER_CONFIG));
     }
 
     public static String getOutputCompressionClass(Configuration conf)
@@ -503,15 +512,11 @@
         if (getOutputCompressionClass(conf) == null)
             return new CompressionParameters(null);
 
-        Map<String, String> options = new HashMap<String, String>();
+        Map<String, String> options = new HashMap<String, String>(2);
         options.put(CompressionParameters.SSTABLE_COMPRESSION, getOutputCompressionClass(conf));
         options.put(CompressionParameters.CHUNK_LENGTH_KB, getOutputCompressionChunkLength(conf));
 
-        try {
-            return CompressionParameters.create(options);
-        } catch (ConfigurationException e) {
-            throw new RuntimeException(e);
-        }
+        return CompressionParameters.create(options);
     }
 
     public static boolean getOutputLocalDCOnly(Configuration conf)
@@ -562,6 +567,7 @@
         return client;
     }
 
+    @SuppressWarnings("resource")
     public static Cassandra.Client createConnection(Configuration conf, String host, Integer port) throws IOException
     {
         try
diff --git a/src/java/org/apache/cassandra/hadoop/HadoopCompat.java b/src/java/org/apache/cassandra/hadoop/HadoopCompat.java
index f2f7033..bcfb952 100644
--- a/src/java/org/apache/cassandra/hadoop/HadoopCompat.java
+++ b/src/java/org/apache/cassandra/hadoop/HadoopCompat.java
@@ -234,11 +234,7 @@
         try {
             return (Counter)
                     GENERIC_COUNTER_CONSTRUCTOR.newInstance(name, displayName, value);
-        } catch (InstantiationException e) {
-            throw new IllegalArgumentException("Can't instantiate Counter", e);
-        } catch (IllegalAccessException e) {
-            throw new IllegalArgumentException("Can't instantiate Counter", e);
-        } catch (InvocationTargetException e) {
+        } catch (InstantiationException | IllegalAccessException | InvocationTargetException e) {
             throw new IllegalArgumentException("Can't instantiate Counter", e);
         }
     }
@@ -249,9 +245,7 @@
     private static Object invoke(Method method, Object obj, Object... args) {
         try {
             return method.invoke(obj, args);
-        } catch (IllegalAccessException e) {
-            throw new IllegalArgumentException("Can't invoke method " + method.getName(), e);
-        } catch (InvocationTargetException e) {
+        } catch (IllegalAccessException | InvocationTargetException e) {
             throw new IllegalArgumentException("Can't invoke method " + method.getName(), e);
         }
     }
diff --git a/src/java/org/apache/cassandra/hadoop/cql3/CqlBulkOutputFormat.java b/src/java/org/apache/cassandra/hadoop/cql3/CqlBulkOutputFormat.java
index 7fedb41..051447c 100644
--- a/src/java/org/apache/cassandra/hadoop/cql3/CqlBulkOutputFormat.java
+++ b/src/java/org/apache/cassandra/hadoop/cql3/CqlBulkOutputFormat.java
@@ -20,14 +20,17 @@
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.util.Collection;
 import java.util.List;
 
-import org.apache.cassandra.config.EncryptionOptions;
-import org.apache.cassandra.hadoop.AbstractBulkOutputFormat;
 import org.apache.cassandra.hadoop.ConfigHelper;
+import org.apache.cassandra.hadoop.HadoopCompat;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.OutputCommitter;
+import org.apache.hadoop.mapreduce.OutputFormat;
 import org.apache.hadoop.mapreduce.RecordWriter;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.util.Progressable;
@@ -36,10 +39,10 @@
  * The <code>CqlBulkOutputFormat</code> acts as a Hadoop-specific
  * OutputFormat that allows reduce tasks to store keys (and corresponding
  * bound variable values) as CQL rows (and respective columns) in a given
- * ColumnFamily.
+ * table.
  *
  * <p>
- * As is the case with the {@link org.apache.cassandra.hadoop.CqlOutputFormat}, 
+ * As is the case with the {@link org.apache.cassandra.hadoop.cql3.CqlOutputFormat}, 
  * you need to set the prepared statement in your
  * Hadoop job Configuration. The {@link CqlConfigHelper} class, through its
  * {@link ConfigHelper#setOutputPreparedStatement} method, is provided to make this
@@ -49,22 +52,14 @@
  * simple.
  * </p>
  */
-public class CqlBulkOutputFormat extends AbstractBulkOutputFormat<Object, List<ByteBuffer>>
+public class CqlBulkOutputFormat extends OutputFormat<Object, List<ByteBuffer>>
+        implements org.apache.hadoop.mapred.OutputFormat<Object, List<ByteBuffer>>
 {   
   
-    private static final String OUTPUT_CQL_SCHEMA_PREFIX = "cassandra.columnfamily.schema.";
-    private static final String OUTPUT_CQL_INSERT_PREFIX = "cassandra.columnfamily.insert.";
+    private static final String OUTPUT_CQL_SCHEMA_PREFIX = "cassandra.table.schema.";
+    private static final String OUTPUT_CQL_INSERT_PREFIX = "cassandra.table.insert.";
     private static final String DELETE_SOURCE = "cassandra.output.delete.source";
-    private static final String OUTPUT_CQL_STORAGE_PORT = "cassandra.storage.port";
-    private static final String OUTPUT_CQL_SSL_STORAGE_PORT = "cassandra.ssl.storage.port";
-    private static final String INTERNODE_ENCRYPTION = "cassandra.internode.encryption";
-    private static final String SERVER_KEYSTORE = "cassandra.server.keystore";
-    private static final String SERVER_KEYSTORE_PASSWORD = "cassandra.server.keystore.password";
-    private static final String SERVER_TRUSTSTORE = "cassandra.server.truststore";
-    private static final String SERVER_TRUSTSTORE_PASSWORD = "cassandra.server.truststore.password";
-    private static final String SERVER_CIPHER_SUITES = "cassandra.server.truststore.password";
-    public static final int DEFAULT_STORAGE_PORT = 7000;
-    public static final int DEFAULT_SSL_STORAGE_PORT = 7001;
+    private static final String TABLE_ALIAS_PREFIX = "cqlbulkoutputformat.table.alias.";
   
     /** Fills the deprecated OutputFormat interface for streaming. */
     @Deprecated
@@ -85,113 +80,60 @@
     {
         return new CqlBulkRecordWriter(context);
     }
+
+    @Override
+    public void checkOutputSpecs(JobContext context)
+    {
+        checkOutputSpecs(HadoopCompat.getConfiguration(context));
+    }
+
+    private void checkOutputSpecs(Configuration conf)
+    {
+        if (ConfigHelper.getOutputKeyspace(conf) == null)
+        {
+            throw new UnsupportedOperationException("you must set the keyspace with setTable()");
+        }
+    }
+
+    /** Fills the deprecated OutputFormat interface for streaming. */
+    @Deprecated
+    public void checkOutputSpecs(org.apache.hadoop.fs.FileSystem filesystem, org.apache.hadoop.mapred.JobConf job) throws IOException
+    {
+        checkOutputSpecs(job);
+    }
+
+    @Override
+    public OutputCommitter getOutputCommitter(TaskAttemptContext context) throws IOException, InterruptedException
+    {
+        return new NullOutputCommitter();
+    }
     
-    public static void setColumnFamilySchema(Configuration conf, String columnFamily, String schema)
+    public static void setTableSchema(Configuration conf, String columnFamily, String schema)
     {
         conf.set(OUTPUT_CQL_SCHEMA_PREFIX + columnFamily, schema);
     }
 
-    public static void setColumnFamilyInsertStatement(Configuration conf, String columnFamily, String insertStatement)
+    public static void setTableInsertStatement(Configuration conf, String columnFamily, String insertStatement)
     {
         conf.set(OUTPUT_CQL_INSERT_PREFIX + columnFamily, insertStatement);
     }
-
-    public static void setStoragePort(Configuration conf, int port)
-    {
-        conf.set(OUTPUT_CQL_STORAGE_PORT, "" + port);
-    }
-
-    public static void setSSLStoragePort(Configuration conf, int port)
-    {
-        conf.set(OUTPUT_CQL_SSL_STORAGE_PORT, "" + port);
-    }
-
-    public static void setInternodeEncryption(Configuration conf, String encrypt)
-    {
-        conf.set(INTERNODE_ENCRYPTION, encrypt);
-    }
-
-    public static void setServerKeystore(Configuration conf, String keystore)
-    {
-        conf.set(SERVER_KEYSTORE, keystore);
-    }
-
-    public static void setServerKeystorePassword(Configuration conf, String keystorePass)
-    {
-        conf.set(SERVER_KEYSTORE_PASSWORD, keystorePass);
-    }
-
-    public static void setServerTruststore(Configuration conf, String truststore)
-    {
-        conf.set(SERVER_TRUSTSTORE, truststore);
-    }
-
-    public static void setServerTruststorePassword(Configuration conf, String truststorePass)
-    {
-        conf.set(SERVER_TRUSTSTORE_PASSWORD, truststorePass);
-    }
-
-    public static void setServerCipherSuites(Configuration conf, String cipherSuites)
-    {
-        conf.set(SERVER_CIPHER_SUITES, cipherSuites);
-    }
-
-    public static int getStoragePort(Configuration conf)
-    {
-        return conf.getInt(OUTPUT_CQL_STORAGE_PORT, DEFAULT_STORAGE_PORT);
-    }
-
-    public static int getSSLStoragePort(Configuration conf)
-    {
-        return conf.getInt(OUTPUT_CQL_SSL_STORAGE_PORT, DEFAULT_SSL_STORAGE_PORT);
-    }
-
-    public static String getInternodeEncryption(Configuration conf)
-    {
-        return conf.get(INTERNODE_ENCRYPTION, EncryptionOptions.ServerEncryptionOptions.InternodeEncryption.none.name());
-    }
-
-    public static String getServerKeystore(Configuration conf)
-    {
-        return conf.get(SERVER_KEYSTORE);
-    }
-
-    public static String getServerTruststore(Configuration conf)
-    {
-        return conf.get(SERVER_TRUSTSTORE);
-    }
-
-    public static String getServerKeystorePassword(Configuration conf)
-    {
-        return conf.get(SERVER_KEYSTORE_PASSWORD);
-    }
-
-    public static String getServerTruststorePassword(Configuration conf)
-    {
-        return conf.get(SERVER_TRUSTSTORE_PASSWORD);
-    }
-
-    public static String getServerCipherSuites(Configuration conf)
-    {
-        return conf.get(SERVER_CIPHER_SUITES);
-    }
-
-    public static String getColumnFamilySchema(Configuration conf, String columnFamily)
+    
+    public static String getTableSchema(Configuration conf, String columnFamily)
     {
         String schema = conf.get(OUTPUT_CQL_SCHEMA_PREFIX + columnFamily);
         if (schema == null)
         { 
-            throw new UnsupportedOperationException("You must set the ColumnFamily schema using setColumnFamilySchema.");
+            throw new UnsupportedOperationException("You must set the Table schema using setTableSchema.");
         }
         return schema; 
     }
     
-    public static String getColumnFamilyInsertStatement(Configuration conf, String columnFamily)
+    public static String getTableInsertStatement(Configuration conf, String columnFamily)
     {
         String insert = conf.get(OUTPUT_CQL_INSERT_PREFIX + columnFamily); 
         if (insert == null)
         {
-            throw new UnsupportedOperationException("You must set the ColumnFamily insert statement using setColumnFamilySchema.");
+            throw new UnsupportedOperationException("You must set the Table insert statement using setTableSchema.");
         }
         return insert;
     }
@@ -205,4 +147,63 @@
     {
         return conf.getBoolean(DELETE_SOURCE, false);
     }
+    
+    public static void setTableAlias(Configuration conf, String alias, String columnFamily)
+    {
+        conf.set(TABLE_ALIAS_PREFIX + alias, columnFamily);
+    }
+    
+    public static String getTableForAlias(Configuration conf, String alias)
+    {
+        return conf.get(TABLE_ALIAS_PREFIX + alias);
+    }
+
+    /**
+     * Set the hosts to ignore as comma delimited values.
+     * Data will not be bulk loaded onto the ignored nodes.
+     * @param conf job configuration
+     * @param ignoreNodesCsv a comma delimited list of nodes to ignore
+     */
+    public static void setIgnoreHosts(Configuration conf, String ignoreNodesCsv)
+    {
+        conf.set(CqlBulkRecordWriter.IGNORE_HOSTS, ignoreNodesCsv);
+    }
+
+    /**
+     * Set the hosts to ignore. Data will not be bulk loaded onto the ignored nodes.
+     * @param conf job configuration
+     * @param ignoreNodes the nodes to ignore
+     */
+    public static void setIgnoreHosts(Configuration conf, String... ignoreNodes)
+    {
+        conf.setStrings(CqlBulkRecordWriter.IGNORE_HOSTS, ignoreNodes);
+    }
+
+    /**
+     * Get the hosts to ignore as a collection of strings
+     * @param conf job configuration
+     * @return the nodes to ignore as a collection of stirngs
+     */
+    public static Collection<String> getIgnoreHosts(Configuration conf)
+    {
+        return conf.getStringCollection(CqlBulkRecordWriter.IGNORE_HOSTS);
+    }
+
+    public static class NullOutputCommitter extends OutputCommitter
+    {
+        public void abortTask(TaskAttemptContext taskContext) { }
+
+        public void cleanupJob(JobContext jobContext) { }
+
+        public void commitTask(TaskAttemptContext taskContext) { }
+
+        public boolean needsTaskCommit(TaskAttemptContext taskContext)
+        {
+            return false;
+        }
+
+        public void setupJob(JobContext jobContext) { }
+
+        public void setupTask(TaskAttemptContext taskContext) { }
+    }
 }
diff --git a/src/java/org/apache/cassandra/hadoop/cql3/CqlBulkRecordWriter.java b/src/java/org/apache/cassandra/hadoop/cql3/CqlBulkRecordWriter.java
index ced8aa9..d064e27 100644
--- a/src/java/org/apache/cassandra/hadoop/cql3/CqlBulkRecordWriter.java
+++ b/src/java/org/apache/cassandra/hadoop/cql3/CqlBulkRecordWriter.java
@@ -17,20 +17,23 @@
  */
 package org.apache.cassandra.hadoop.cql3;
 
+import java.io.Closeable;
 import java.io.File;
 import java.io.IOException;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.UUID;
+import java.util.*;
+import java.util.concurrent.*;
 
-import org.apache.cassandra.config.EncryptionOptions;
-import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.Murmur3Partitioner;
 import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.hadoop.AbstractBulkRecordWriter;
 import org.apache.cassandra.hadoop.BulkRecordWriter;
 import org.apache.cassandra.hadoop.ConfigHelper;
 import org.apache.cassandra.hadoop.HadoopCompat;
@@ -38,10 +41,9 @@
 import org.apache.cassandra.io.sstable.SSTableLoader;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.streaming.StreamState;
-import org.apache.cassandra.thrift.ITransportFactory;
-import org.apache.cassandra.tools.BulkLoader;
-import org.apache.commons.lang.StringUtils;
+import org.apache.cassandra.utils.NativeSSTableLoaderClient;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.RecordWriter;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.util.Progressable;
 
@@ -59,30 +61,55 @@
  *
  * @see CqlBulkOutputFormat
  */
-public class CqlBulkRecordWriter extends AbstractBulkRecordWriter<Object, List<ByteBuffer>>
+public class CqlBulkRecordWriter extends RecordWriter<Object, List<ByteBuffer>>
+        implements org.apache.hadoop.mapred.RecordWriter<Object, List<ByteBuffer>>
 {
+    public final static String OUTPUT_LOCATION = "mapreduce.output.bulkoutputformat.localdir";
+    public final static String BUFFER_SIZE_IN_MB = "mapreduce.output.bulkoutputformat.buffersize";
+    public final static String STREAM_THROTTLE_MBITS = "mapreduce.output.bulkoutputformat.streamthrottlembits";
+    public final static String MAX_FAILED_HOSTS = "mapreduce.output.bulkoutputformat.maxfailedhosts";
+    public final static String IGNORE_HOSTS = "mapreduce.output.bulkoutputformat.ignorehosts";
+
+    private final Logger logger = LoggerFactory.getLogger(CqlBulkRecordWriter.class);
+
+    protected final Configuration conf;
+    protected final int maxFailures;
+    protected final int bufferSize;
+    protected Closeable writer;
+    protected SSTableLoader loader;
+    protected Progressable progress;
+    protected TaskAttemptContext context;
+    protected final Set<InetAddress> ignores = new HashSet<>();
+
     private String keyspace;
-    private String columnFamily;
+    private String table;
     private String schema;
     private String insertStatement;
     private File outputDir;
     private boolean deleteSrc;
+    private IPartitioner partitioner;
 
     CqlBulkRecordWriter(TaskAttemptContext context) throws IOException
     {
-        super(context);
+        this(HadoopCompat.getConfiguration(context));
+        this.context = context;
         setConfigs();
     }
 
     CqlBulkRecordWriter(Configuration conf, Progressable progress) throws IOException
     {
-        super(conf, progress);
+        this(conf);
+        this.progress = progress;
         setConfigs();
     }
 
     CqlBulkRecordWriter(Configuration conf) throws IOException
     {
-        super(conf);
+        Config.setOutboundBindAny(true);
+        this.conf = conf;
+        DatabaseDescriptor.setStreamThroughputOutboundMegabitsPerSec(Integer.parseInt(conf.get(STREAM_THROTTLE_MBITS, "0")));
+        maxFailures = Integer.parseInt(conf.get(MAX_FAILED_HOSTS, "0"));
+        bufferSize = Integer.parseInt(conf.get(BUFFER_SIZE_IN_MB, "64"));
         setConfigs();
     }
     
@@ -90,48 +117,77 @@
     {
         // if anything is missing, exceptions will be thrown here, instead of on write()
         keyspace = ConfigHelper.getOutputKeyspace(conf);
-        columnFamily = ConfigHelper.getOutputColumnFamily(conf);
-        schema = CqlBulkOutputFormat.getColumnFamilySchema(conf, columnFamily);
-        insertStatement = CqlBulkOutputFormat.getColumnFamilyInsertStatement(conf, columnFamily);
-        outputDir = getColumnFamilyDirectory();
+        table = ConfigHelper.getOutputColumnFamily(conf);
+        
+        // check if table is aliased
+        String aliasedCf = CqlBulkOutputFormat.getTableForAlias(conf, table);
+        if (aliasedCf != null)
+            table = aliasedCf;
+        
+        schema = CqlBulkOutputFormat.getTableSchema(conf, table);
+        insertStatement = CqlBulkOutputFormat.getTableInsertStatement(conf, table);
+        outputDir = getTableDirectory();
         deleteSrc = CqlBulkOutputFormat.getDeleteSourceOnSuccess(conf);
-    }
-
-    
-    private void prepareWriter() throws IOException
-    {
         try
         {
-            if (writer == null)
-            {
-                writer = CQLSSTableWriter.builder()
-                    .forTable(schema)
-                    .using(insertStatement)
-                    .withPartitioner(ConfigHelper.getOutputPartitioner(conf))
-                    .inDirectory(outputDir)
-                    .withBufferSizeInMB(Integer.parseInt(conf.get(BUFFER_SIZE_IN_MB, "64")))
-                    .build();
-            }
-            if (loader == null)
-            {
-                BulkLoader.ExternalClient externalClient = getExternalClient(conf);
-                this.loader = new SSTableLoader(outputDir, externalClient, new BulkRecordWriter.NullOutputHandler()) {
-                    @Override
-                    public void onSuccess(StreamState finalState)
-                    {
-                        if (deleteSrc)
-                            FileUtils.deleteRecursive(outputDir);
-                    }
-                };
-            }
+            partitioner = ConfigHelper.getInputPartitioner(conf);
         }
         catch (Exception e)
         {
-            throw new IOException(e);
-        }      
+            partitioner = Murmur3Partitioner.instance;
+        }
+        try
+        {
+            for (String hostToIgnore : CqlBulkOutputFormat.getIgnoreHosts(conf))
+                ignores.add(InetAddress.getByName(hostToIgnore));
+        }
+        catch (UnknownHostException e)
+        {
+            throw new RuntimeException(("Unknown host: " + e.getMessage()));
+        }
+    }
+
+    protected String getOutputLocation() throws IOException
+    {
+        String dir = conf.get(OUTPUT_LOCATION, System.getProperty("java.io.tmpdir"));
+        if (dir == null)
+            throw new IOException("Output directory not defined, if hadoop is not setting java.io.tmpdir then define " + OUTPUT_LOCATION);
+        return dir;
+    }
+
+    private void prepareWriter() throws IOException
+    {
+        if (writer == null)
+        {
+            writer = CQLSSTableWriter.builder()
+                                     .forTable(schema)
+                                     .using(insertStatement)
+                                     .withPartitioner(ConfigHelper.getOutputPartitioner(conf))
+                                     .inDirectory(outputDir)
+                                     .withBufferSizeInMB(Integer.parseInt(conf.get(BUFFER_SIZE_IN_MB, "64")))
+                                     .withPartitioner(partitioner)
+                                     .build();
+        }
+
+        if (loader == null)
+        {
+            ExternalClient externalClient = new ExternalClient(conf);
+            externalClient.setTableMetadata(CFMetaData.compile(schema, keyspace));
+
+            loader = new SSTableLoader(outputDir, externalClient, new BulkRecordWriter.NullOutputHandler())
+            {
+                @Override
+                public void onSuccess(StreamState finalState)
+                {
+                    if (deleteSrc)
+                        FileUtils.deleteRecursive(outputDir);
+                }
+            };
+        }
     }
     
     /**
+     * <p>
      * The column values must correspond to the order in which
      * they appear in the insert stored procedure. 
      * 
@@ -163,9 +219,9 @@
         }
     }
     
-    private File getColumnFamilyDirectory() throws IOException
+    private File getTableDirectory() throws IOException
     {
-        File dir = new File(String.format("%s%s%s%s%s-%s", getOutputLocation(), File.separator, keyspace, File.separator, columnFamily, UUID.randomUUID().toString()));
+        File dir = new File(String.format("%s%s%s%s%s-%s", getOutputLocation(), File.separator, keyspace, File.separator, table, UUID.randomUUID().toString()));
         
         if (!dir.exists() && !dir.mkdirs())
         {
@@ -175,52 +231,82 @@
         return dir;
     }
 
-    private BulkLoader.ExternalClient getExternalClient(Configuration conf)
+    @Override
+    public void close(TaskAttemptContext context) throws IOException, InterruptedException
     {
-        Set<InetAddress> hosts = new HashSet<InetAddress>();
-        String outputAddress = ConfigHelper.getOutputInitialAddress(conf);
-        if (outputAddress == null) outputAddress = "localhost";
-        String[] nodes = outputAddress.split(",");
-        for (String node : nodes)
-        {
-            try
-            {
-                hosts.add(InetAddress.getByName(node));
-            }
-            catch (UnknownHostException e)
-            {
-                throw new RuntimeException(e);
-            }
-        }
-        int rpcPort = ConfigHelper.getOutputRpcPort(conf);
-        String username = ConfigHelper.getOutputKeyspaceUserName(conf);
-        String password = ConfigHelper.getOutputKeyspacePassword(conf);
-        ITransportFactory transportFactory = ConfigHelper.getClientTransportFactory(conf);
-        return new BulkLoader.ExternalClient(hosts,
-                rpcPort,
-                username,
-                password,
-                transportFactory,
-                CqlBulkOutputFormat.getStoragePort(conf),
-                CqlBulkOutputFormat.getSSLStoragePort(conf),
-                getServerEncryptOpt(conf));
+        close();
     }
 
-    private ServerEncryptionOptions getServerEncryptOpt(Configuration conf)
+    /** Fills the deprecated RecordWriter interface for streaming. */
+    @Deprecated
+    public void close(org.apache.hadoop.mapred.Reporter reporter) throws IOException
     {
-        ServerEncryptionOptions encryptOpt = new ServerEncryptionOptions();
-        String internodeEncrypt = CqlBulkOutputFormat.getInternodeEncryption(conf);
-        if (StringUtils.isEmpty(internodeEncrypt))
-            return encryptOpt;
+        close();
+    }
 
-        encryptOpt.internode_encryption = EncryptionOptions.ServerEncryptionOptions.InternodeEncryption.valueOf(internodeEncrypt);
-        encryptOpt.keystore = CqlBulkOutputFormat.getServerKeystore(conf);
-        encryptOpt.truststore = CqlBulkOutputFormat.getServerTruststore(conf);
-        encryptOpt.keystore_password = CqlBulkOutputFormat.getServerKeystorePassword(conf);
-        encryptOpt.truststore_password = CqlBulkOutputFormat.getServerTruststorePassword(conf);
-        String cipherSuites = CqlBulkOutputFormat.getServerCipherSuites(conf);
-        if (!StringUtils.isEmpty(cipherSuites))
-            encryptOpt.cipher_suites = cipherSuites.replace(" ", "").split(",");
-        return encryptOpt;
+    private void close() throws IOException
+    {
+        if (writer != null)
+        {
+            writer.close();
+            Future<StreamState> future = loader.stream(ignores);
+            while (true)
+            {
+                try
+                {
+                    future.get(1000, TimeUnit.MILLISECONDS);
+                    break;
+                }
+                catch (ExecutionException | TimeoutException te)
+                {
+                    if (null != progress)
+                        progress.progress();
+                    if (null != context)
+                        HadoopCompat.progress(context);
+                }
+                catch (InterruptedException e)
+                {
+                    throw new IOException(e);
+                }
+            }
+            if (loader.getFailedHosts().size() > 0)
+            {
+                if (loader.getFailedHosts().size() > maxFailures)
+                    throw new IOException("Too many hosts failed: " + loader.getFailedHosts());
+                else
+                    logger.warn("Some hosts failed: {}", loader.getFailedHosts());
+            }
+        }
+    }
+    
+    public static class ExternalClient extends NativeSSTableLoaderClient
+    {
+        public ExternalClient(Configuration conf)
+        {
+            super(resolveHostAddresses(conf),
+                  CqlConfigHelper.getOutputNativePort(conf),
+                  ConfigHelper.getOutputKeyspaceUserName(conf),
+                  ConfigHelper.getOutputKeyspacePassword(conf),
+                  CqlConfigHelper.getSSLOptions(conf).orNull());
+        }
+
+        private static Collection<InetAddress> resolveHostAddresses(Configuration conf)
+        {
+            Set<InetAddress> addresses = new HashSet<>();
+
+            for (String host : ConfigHelper.getOutputInitialAddress(conf).split(","))
+            {
+                try
+                {
+                    addresses.add(InetAddress.getByName(host));
+                }
+                catch (UnknownHostException e)
+                {
+                    throw new RuntimeException(e);
+                }
+            }
+
+            return addresses;
+        }
     }
 }
diff --git a/src/java/org/apache/cassandra/hadoop/cql3/CqlConfigHelper.java b/src/java/org/apache/cassandra/hadoop/cql3/CqlConfigHelper.java
index 7d65663..35cdca8 100644
--- a/src/java/org/apache/cassandra/hadoop/cql3/CqlConfigHelper.java
+++ b/src/java/org/apache/cassandra/hadoop/cql3/CqlConfigHelper.java
@@ -34,22 +34,23 @@
 import javax.net.ssl.SSLContext;
 import javax.net.ssl.TrustManagerFactory;
 
-import org.apache.cassandra.hadoop.ConfigHelper;
-import org.apache.cassandra.io.util.FileUtils;
+import com.google.common.base.Optional;
 import org.apache.commons.lang3.StringUtils;
-import org.apache.hadoop.conf.Configuration;
 
 import com.datastax.driver.core.AuthProvider;
-import com.datastax.driver.core.PlainTextAuthProvider;
 import com.datastax.driver.core.Cluster;
 import com.datastax.driver.core.HostDistance;
+import com.datastax.driver.core.PlainTextAuthProvider;
+import com.datastax.driver.core.ProtocolVersion;
+import com.datastax.driver.core.policies.LoadBalancingPolicy;
 import com.datastax.driver.core.PoolingOptions;
 import com.datastax.driver.core.ProtocolOptions;
 import com.datastax.driver.core.QueryOptions;
 import com.datastax.driver.core.SSLOptions;
 import com.datastax.driver.core.SocketOptions;
-import com.datastax.driver.core.policies.LoadBalancingPolicy;
-import com.google.common.base.Optional;
+import org.apache.cassandra.hadoop.ConfigHelper;
+import org.apache.hadoop.conf.Configuration;
+
 
 public class CqlConfigHelper
 {
@@ -64,7 +65,6 @@
     private static final String INPUT_NATIVE_PORT = "cassandra.input.native.port";
     private static final String INPUT_NATIVE_CORE_CONNECTIONS_PER_HOST = "cassandra.input.native.core.connections.per.host";
     private static final String INPUT_NATIVE_MAX_CONNECTIONS_PER_HOST = "cassandra.input.native.max.connections.per.host";
-    private static final String INPUT_NATIVE_MIN_SIMULT_REQ_PER_CONNECTION = "cassandra.input.native.min.simult.reqs.per.connection"; 
     private static final String INPUT_NATIVE_MAX_SIMULT_REQ_PER_CONNECTION = "cassandra.input.native.max.simult.reqs.per.connection";
     private static final String INPUT_NATIVE_CONNECTION_TIMEOUT = "cassandra.input.native.connection.timeout";
     private static final String INPUT_NATIVE_READ_CONNECTION_TIMEOUT = "cassandra.input.native.read.connection.timeout";
@@ -84,6 +84,7 @@
     private static final String INPUT_NATIVE_PROTOCOL_VERSION = "cassandra.input.native.protocol.version";
 
     private static final String OUTPUT_CQL = "cassandra.output.cql";
+    private static final String OUTPUT_NATIVE_PORT = "cassandra.output.native.port";
     
     /**
      * Set the CQL columns for the input of this job.
@@ -176,9 +177,9 @@
         return Integer.parseInt(conf.get(INPUT_NATIVE_PORT, "9042"));
     }
 
-    public static Optional<Integer> getInputMinSimultReqPerConnections(Configuration conf)
+    public static int getOutputNativePort(Configuration conf)
     {
-        return getIntSetting(INPUT_NATIVE_MIN_SIMULT_REQ_PER_CONNECTION, conf);
+        return Integer.parseInt(conf.get(OUTPUT_NATIVE_PORT, "9042"));
     }
 
     public static Optional<Integer> getInputMaxSimultReqPerConnections(Configuration conf)
@@ -294,18 +295,34 @@
     public static Cluster getInputCluster(String[] hosts, Configuration conf)
     {
         int port = getInputNativePort(conf);
+        return getCluster(hosts, conf, port);
+    }
+
+    public static Cluster getOutputCluster(String host, Configuration conf)
+    {
+        return getOutputCluster(new String[]{host}, conf);
+    }
+
+    public static Cluster getOutputCluster(String[] hosts, Configuration conf)
+    {
+        int port = getOutputNativePort(conf);
+        return getCluster(hosts, conf, port);
+    }
+
+    public static Cluster getCluster(String[] hosts, Configuration conf, int port)
+    {
         Optional<AuthProvider> authProvider = getAuthProvider(conf);
         Optional<SSLOptions> sslOptions = getSSLOptions(conf);
         Optional<Integer> protocolVersion = getProtocolVersion(conf);
-        LoadBalancingPolicy loadBalancingPolicy = getReadLoadBalancingPolicy(conf, hosts);
+        LoadBalancingPolicy loadBalancingPolicy = getReadLoadBalancingPolicy(hosts);
         SocketOptions socketOptions = getReadSocketOptions(conf);
         QueryOptions queryOptions = getReadQueryOptions(conf);
         PoolingOptions poolingOptions = getReadPoolingOptions(conf);
-        
+
         Cluster.Builder builder = Cluster.builder()
-                                         .addContactPoints(hosts)
-                                         .withPort(port)
-                                         .withCompression(ProtocolOptions.Compression.NONE);
+                .addContactPoints(hosts)
+                .withPort(port)
+                .withCompression(ProtocolOptions.Compression.NONE);
 
         if (authProvider.isPresent())
             builder.withAuthProvider(authProvider.get());
@@ -313,17 +330,16 @@
             builder.withSSL(sslOptions.get());
 
         if (protocolVersion.isPresent()) {
-            builder.withProtocolVersion(protocolVersion.get());
+            builder.withProtocolVersion(ProtocolVersion.fromInt(protocolVersion.get()));
         }
         builder.withLoadBalancingPolicy(loadBalancingPolicy)
-               .withSocketOptions(socketOptions)
-               .withQueryOptions(queryOptions)
-               .withPoolingOptions(poolingOptions);
+                .withSocketOptions(socketOptions)
+                .withQueryOptions(queryOptions)
+                .withPoolingOptions(poolingOptions);
 
         return builder.build();
     }
 
-
     public static void setInputCoreConnections(Configuration conf, String connections)
     {
         conf.set(INPUT_NATIVE_CORE_CONNECTIONS_PER_HOST, connections);
@@ -334,11 +350,6 @@
         conf.set(INPUT_NATIVE_MAX_CONNECTIONS_PER_HOST, connections);
     }
 
-    public static void setInputMinSimultReqPerConnections(Configuration conf, String reqs)
-    {
-        conf.set(INPUT_NATIVE_MIN_SIMULT_REQ_PER_CONNECTION, reqs);
-    }
-
     public static void setInputMaxSimultReqPerConnections(Configuration conf, String reqs)
     {
         conf.set(INPUT_NATIVE_MAX_SIMULT_REQ_PER_CONNECTION, reqs);
@@ -424,8 +435,7 @@
         Optional<Integer> coreConnections = getInputCoreConnections(conf);
         Optional<Integer> maxConnections = getInputMaxConnections(conf);
         Optional<Integer> maxSimultaneousRequests = getInputMaxSimultReqPerConnections(conf);
-        Optional<Integer> minSimultaneousRequests = getInputMinSimultReqPerConnections(conf);
-        
+
         PoolingOptions poolingOptions = new PoolingOptions();
 
         for (HostDistance hostDistance : Arrays.asList(HostDistance.LOCAL, HostDistance.REMOTE))
@@ -434,8 +444,6 @@
                 poolingOptions.setCoreConnectionsPerHost(hostDistance, coreConnections.get());
             if (maxConnections.isPresent())
                 poolingOptions.setMaxConnectionsPerHost(hostDistance, maxConnections.get());
-            if (minSimultaneousRequests.isPresent())
-                poolingOptions.setMinSimultaneousRequestsPerConnectionThreshold(hostDistance, minSimultaneousRequests.get());
             if (maxSimultaneousRequests.isPresent())
                 poolingOptions.setMaxSimultaneousRequestsPerConnectionThreshold(hostDistance, maxSimultaneousRequests.get());
         }
@@ -488,7 +496,7 @@
         return socketOptions;
     }
 
-    private static LoadBalancingPolicy getReadLoadBalancingPolicy(Configuration conf, final String[] stickHosts)
+    private static LoadBalancingPolicy getReadLoadBalancingPolicy(final String[] stickHosts)
     {
         return new LimitedLocalNodeFirstLocalBalancingPolicy(stickHosts);
     }
@@ -502,20 +510,20 @@
         return Optional.of(getClientAuthProvider(authProvider.get(), conf));
     }
 
-    private static Optional<SSLOptions> getSSLOptions(Configuration conf)
+    public static Optional<SSLOptions> getSSLOptions(Configuration conf)
     {
         Optional<String> truststorePath = getInputNativeSSLTruststorePath(conf);
         Optional<String> keystorePath = getInputNativeSSLKeystorePath(conf);
         Optional<String> truststorePassword = getInputNativeSSLTruststorePassword(conf);
         Optional<String> keystorePassword = getInputNativeSSLKeystorePassword(conf);
         Optional<String> cipherSuites = getInputNativeSSLCipherSuites(conf);
-        
-        if (truststorePath.isPresent() && keystorePath.isPresent() && truststorePassword.isPresent() && keystorePassword.isPresent())
+
+        if (truststorePath.isPresent())
         {
             SSLContext context;
             try
             {
-                context = getSSLContext(truststorePath.get(), truststorePassword.get(), keystorePath.get(), keystorePassword.get());
+                context = getSSLContext(truststorePath, truststorePassword, keystorePath, keystorePassword);
             }
             catch (UnrecoverableKeyException | KeyManagementException |
                     NoSuchAlgorithmException | KeyStoreException | CertificateException | IOException e)
@@ -535,7 +543,7 @@
         String setting = conf.get(parameter);
         if (setting == null)
             return Optional.absent();
-        return Optional.of(Integer.parseInt(setting));  
+        return Optional.of(Integer.valueOf(setting));  
     }
 
     private static Optional<Boolean> getBooleanSetting(String parameter, Configuration conf)
@@ -543,7 +551,7 @@
         String setting = conf.get(parameter);
         if (setting == null)
             return Optional.absent();
-        return Optional.of(Boolean.parseBoolean(setting));  
+        return Optional.of(Boolean.valueOf(setting));  
     }
 
     private static Optional<String> getStringSetting(String parameter, Configuration conf)
@@ -577,35 +585,46 @@
         }
     }
 
-    private static SSLContext getSSLContext(String truststorePath, String truststorePassword, String keystorePath, String keystorePassword)
-            throws NoSuchAlgorithmException, KeyStoreException, CertificateException, IOException, UnrecoverableKeyException, KeyManagementException
+    private static SSLContext getSSLContext(Optional<String> truststorePath,
+                                            Optional<String> truststorePassword,
+                                            Optional<String> keystorePath,
+                                            Optional<String> keystorePassword)
+    throws NoSuchAlgorithmException,
+           KeyStoreException,
+           CertificateException,
+           IOException,
+           UnrecoverableKeyException,
+           KeyManagementException
     {
-        FileInputStream tsf = null;
-        FileInputStream ksf = null;
-        SSLContext ctx = null;
-        try
+        SSLContext ctx = SSLContext.getInstance("SSL");
+
+        TrustManagerFactory tmf = null;
+        if (truststorePath.isPresent())
         {
-            tsf = new FileInputStream(truststorePath);
-            ksf = new FileInputStream(keystorePath);
-            ctx = SSLContext.getInstance("SSL");
-
-            KeyStore ts = KeyStore.getInstance("JKS");
-            ts.load(tsf, truststorePassword.toCharArray());
-            TrustManagerFactory tmf = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm());
-            tmf.init(ts);
-
-            KeyStore ks = KeyStore.getInstance("JKS");
-            ks.load(ksf, keystorePassword.toCharArray());
-            KeyManagerFactory kmf = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
-            kmf.init(ks, keystorePassword.toCharArray());
-
-            ctx.init(kmf.getKeyManagers(), tmf.getTrustManagers(), new SecureRandom());
+            try (FileInputStream tsf = new FileInputStream(truststorePath.get()))
+            {
+                KeyStore ts = KeyStore.getInstance("JKS");
+                ts.load(tsf, truststorePassword.isPresent() ? truststorePassword.get().toCharArray() : null);
+                tmf = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm());
+                tmf.init(ts);
+            }
         }
-        finally
+
+        KeyManagerFactory kmf = null;
+        if (keystorePath.isPresent())
         {
-            FileUtils.closeQuietly(tsf);
-            FileUtils.closeQuietly(ksf);
+            try (FileInputStream ksf = new FileInputStream(keystorePath.get()))
+            {
+                KeyStore ks = KeyStore.getInstance("JKS");
+                ks.load(ksf, keystorePassword.isPresent() ? keystorePassword.get().toCharArray() : null);
+                kmf = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
+                kmf.init(ks, keystorePassword.isPresent() ? keystorePassword.get().toCharArray() : null);
+            }
         }
+
+        ctx.init(kmf != null ? kmf.getKeyManagers() : null,
+                 tmf != null ? tmf.getTrustManagers() : null,
+                 new SecureRandom());
         return ctx;
     }
 }
diff --git a/src/java/org/apache/cassandra/hadoop/cql3/CqlInputFormat.java b/src/java/org/apache/cassandra/hadoop/cql3/CqlInputFormat.java
index 36da92d..c46ceb8 100644
--- a/src/java/org/apache/cassandra/hadoop/cql3/CqlInputFormat.java
+++ b/src/java/org/apache/cassandra/hadoop/cql3/CqlInputFormat.java
@@ -40,10 +40,12 @@
  *   ConfigHelper.setInputColumnFamily
  *
  * You can also configure the number of rows per InputSplit with
- *   ConfigHelper.setInputSplitSize. The default split size is 64k rows.
+ *   1: ConfigHelper.setInputSplitSize. The default split size is 64k rows.
+ *   or
+ *   2: ConfigHelper.setInputSplitSizeInMb. InputSplit size in MB with new, more precise method
+ *   If no value is provided for InputSplitSizeInMb, InputSplitSize will be used.
  *
- *   the number of CQL rows per page
- *   CQLConfigHelper.setInputCQLPageRowSize. The default page row size is 1000. You 
+ *   CQLConfigHelper.setInputCQLPageRowSize. The default page row size is 1000. You
  *   should set it to "as big as possible, but no bigger." It set the LIMIT for the CQL 
  *   query, so you need set it big enough to minimize the network overhead, and also
  *   not too big to avoid out of memory issue.
diff --git a/src/java/org/apache/cassandra/hadoop/cql3/CqlOutputFormat.java b/src/java/org/apache/cassandra/hadoop/cql3/CqlOutputFormat.java
index 0d09ca2..9a1cda6 100644
--- a/src/java/org/apache/cassandra/hadoop/cql3/CqlOutputFormat.java
+++ b/src/java/org/apache/cassandra/hadoop/cql3/CqlOutputFormat.java
@@ -23,15 +23,15 @@
 import java.util.List;
 import java.util.Map;
 
-import org.apache.cassandra.hadoop.AbstractColumnFamilyOutputFormat;
-import org.apache.cassandra.hadoop.ConfigHelper;
+import org.apache.cassandra.hadoop.*;
+import org.apache.hadoop.conf.*;
 import org.apache.hadoop.mapreduce.*;
 
 /**
- * The <code>ColumnFamilyOutputFormat</code> acts as a Hadoop-specific
+ * The <code>CqlOutputFormat</code> acts as a Hadoop-specific
  * OutputFormat that allows reduce tasks to store keys (and corresponding
  * bound variable values) as CQL rows (and respective columns) in a given
- * ColumnFamily.
+ * table.
  *
  * <p>
  * As is the case with the {@link org.apache.cassandra.hadoop.ColumnFamilyInputFormat}, 
@@ -52,8 +52,51 @@
  * to Cassandra.
  * </p>
  */
-public class CqlOutputFormat extends AbstractColumnFamilyOutputFormat<Map<String, ByteBuffer>, List<ByteBuffer>>
-{   
+public class CqlOutputFormat extends OutputFormat<Map<String, ByteBuffer>, List<ByteBuffer>>
+        implements org.apache.hadoop.mapred.OutputFormat<Map<String, ByteBuffer>, List<ByteBuffer>>
+{
+    /**
+     * Check for validity of the output-specification for the job.
+     *
+     * @param context
+     *            information about the job
+     */
+    public void checkOutputSpecs(JobContext context)
+    {
+        checkOutputSpecs(HadoopCompat.getConfiguration(context));
+    }
+
+    protected void checkOutputSpecs(Configuration conf)
+    {
+        if (ConfigHelper.getOutputKeyspace(conf) == null)
+            throw new UnsupportedOperationException("You must set the keyspace with setOutputKeyspace()");
+        if (ConfigHelper.getOutputPartitioner(conf) == null)
+            throw new UnsupportedOperationException("You must set the output partitioner to the one used by your Cassandra cluster");
+        if (ConfigHelper.getOutputInitialAddress(conf) == null)
+            throw new UnsupportedOperationException("You must set the initial output address to a Cassandra node");
+    }
+
+    /** Fills the deprecated OutputFormat interface for streaming. */
+    @Deprecated
+    public void checkOutputSpecs(org.apache.hadoop.fs.FileSystem filesystem, org.apache.hadoop.mapred.JobConf job) throws IOException
+    {
+        checkOutputSpecs(job);
+    }
+
+    /**
+     * The OutputCommitter for this format does not write any data to the DFS.
+     *
+     * @param context
+     *            the task context
+     * @return an output committer
+     * @throws IOException
+     * @throws InterruptedException
+     */
+    public OutputCommitter getOutputCommitter(TaskAttemptContext context) throws IOException, InterruptedException
+    {
+        return new NullOutputCommitter();
+    }
+
     /** Fills the deprecated OutputFormat interface for streaming. */
     @Deprecated
     public CqlRecordWriter getRecordWriter(org.apache.hadoop.fs.FileSystem filesystem, org.apache.hadoop.mapred.JobConf job, String name, org.apache.hadoop.util.Progressable progress) throws IOException
@@ -73,4 +116,25 @@
     {
         return new CqlRecordWriter(context);
     }
+
+    /**
+     * An {@link OutputCommitter} that does nothing.
+     */
+    private static class NullOutputCommitter extends OutputCommitter
+    {
+        public void abortTask(TaskAttemptContext taskContext) { }
+
+        public void cleanupJob(JobContext jobContext) { }
+
+        public void commitTask(TaskAttemptContext taskContext) { }
+
+        public boolean needsTaskCommit(TaskAttemptContext taskContext)
+        {
+            return false;
+        }
+
+        public void setupJob(JobContext jobContext) { }
+
+        public void setupTask(TaskAttemptContext taskContext) { }
+    }
 }
diff --git a/src/java/org/apache/cassandra/hadoop/cql3/CqlRecordReader.java b/src/java/org/apache/cassandra/hadoop/cql3/CqlRecordReader.java
index 234cba3..b3e440d 100644
--- a/src/java/org/apache/cassandra/hadoop/cql3/CqlRecordReader.java
+++ b/src/java/org/apache/cassandra/hadoop/cql3/CqlRecordReader.java
@@ -30,16 +30,28 @@
 import com.google.common.collect.AbstractIterator;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Maps;
-
 import org.apache.commons.lang3.StringUtils;
-
-import org.apache.cassandra.hadoop.HadoopCompat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnDefinitions;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.LocalDate;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.TableMetadata;
+import com.datastax.driver.core.Token;
+import com.datastax.driver.core.TupleValue;
+import com.datastax.driver.core.UDTValue;
+import com.google.common.reflect.TypeToken;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.hadoop.ColumnFamilySplit;
 import org.apache.cassandra.hadoop.ConfigHelper;
+import org.apache.cassandra.hadoop.HadoopCompat;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.Pair;
 import org.apache.hadoop.conf.Configuration;
@@ -47,25 +59,23 @@
 import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 
-import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.ColumnDefinitions;
-import com.datastax.driver.core.ColumnMetadata;
-import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.Row;
-import com.datastax.driver.core.Session;
 /**
+ * <p>
  * CqlRecordReader reads the rows return from the CQL query
  * It uses CQL auto-paging.
- * <p/>
+ * </p>
+ * <p>
  * Return a Long as a local CQL row key starts from 0;
- * <p/>
+ * </p>
+ * {@code
  * Row as C* java driver CQL result set row
  * 1) select clause must include partition key columns (to calculate the progress based on the actual CF row processed)
  * 2) where clause must include token(partition_key1, ...  , partition_keyn) > ? and 
  *       token(partition_key1, ... , partition_keyn) <= ?  (in the right order) 
+ * }
  */
 public class CqlRecordReader extends RecordReader<Long, Row>
-        implements org.apache.hadoop.mapred.RecordReader<Long, Row>
+        implements org.apache.hadoop.mapred.RecordReader<Long, Row>, AutoCloseable
 {
     private static final Logger logger = LoggerFactory.getLogger(CqlRecordReader.class);
 
@@ -129,7 +139,7 @@
           throw new RuntimeException("Can't create connection session");
 
         //get negotiated serialization protocol
-        nativeProtocolVersion = cluster.getConfiguration().getProtocolOptions().getProtocolVersion();
+        nativeProtocolVersion = cluster.getConfiguration().getProtocolOptions().getProtocolVersion().toInt();
 
         // If the user provides a CQL query then we will use it without validation
         // otherwise we will fall back to building a query using the:
@@ -146,10 +156,10 @@
 
         if (StringUtils.isEmpty(cqlQuery))
             cqlQuery = buildQuery();
-        logger.debug("cqlQuery {}", cqlQuery);
+        logger.trace("cqlQuery {}", cqlQuery);
 
         rowIterator = new RowIterator();
-        logger.debug("created {}", rowIterator);
+        logger.trace("created {}", rowIterator);
     }
 
     public void close()
@@ -184,7 +194,7 @@
     {
         if (!rowIterator.hasNext())
         {
-            logger.debug("Finished scanning {} rows (estimate was: {})", rowIterator.totalRead, totalRowCount);
+            logger.trace("Finished scanning {} rows (estimate was: {})", rowIterator.totalRead, totalRowCount);
             return false;
         }
 
@@ -219,12 +229,12 @@
 
     public long getPos() throws IOException
     {
-        return (long) rowIterator.totalRead;
+        return rowIterator.totalRead;
     }
 
     public Long createKey()
     {
-        return new Long(0L);
+        return Long.valueOf(0L);
     }
 
     public Row createValue()
@@ -268,7 +278,7 @@
                 return endOfData();
 
             Row row = rows.next();
-            Map<String, ByteBuffer> keyColumns = new HashMap<String, ByteBuffer>(); 
+            Map<String, ByteBuffer> keyColumns = new HashMap<String, ByteBuffer>(partitionBoundColumns.size()); 
             for (String column : partitionBoundColumns.keySet())
                 keyColumns.put(column, row.getBytesUnsafe(column));
 
@@ -324,6 +334,18 @@
         }
 
         @Override
+        public Object getObject(int i)
+        {
+            return row.getObject(i);
+        }
+
+        @Override
+        public Object getObject(String s)
+        {
+            return row.getObject(s);
+        }
+
+        @Override
         public boolean getBool(int i)
         {
             return row.getBool(i);
@@ -336,6 +358,30 @@
         }
 
         @Override
+        public short getShort(int i)
+        {
+            return row.getShort(i);
+        }
+
+        @Override
+        public short getShort(String s)
+        {
+            return row.getShort(s);
+        }
+
+        @Override
+        public byte getByte(int i)
+        {
+            return row.getByte(i);
+        }
+
+        @Override
+        public byte getByte(String s)
+        {
+            return row.getByte(s);
+        }
+
+        @Override
         public int getInt(int i)
         {
             return row.getInt(i);
@@ -360,15 +406,39 @@
         }
 
         @Override
-        public Date getDate(int i)
+        public Date getTimestamp(int i)
+        {
+            return row.getTimestamp(i);
+        }
+
+        @Override
+        public Date getTimestamp(String s)
+        {
+            return row.getTimestamp(s);
+        }
+
+        @Override
+        public LocalDate getDate(int i)
         {
             return row.getDate(i);
         }
 
         @Override
-        public Date getDate(String name)
+        public LocalDate getDate(String s)
         {
-            return row.getDate(name);
+            return row.getDate(s);
+        }
+
+        @Override
+        public long getTime(int i)
+        {
+            return row.getTime(i);
+        }
+
+        @Override
+        public long getTime(String s)
+        {
+            return row.getTime(s);
         }
 
         @Override
@@ -486,34 +556,112 @@
         }
 
         @Override
+        public <T> List<T> getList(int i, TypeToken<T> typeToken)
+        {
+            return row.getList(i, typeToken);
+        }
+
+        @Override
         public <T> List<T> getList(String name, Class<T> elementsClass)
         {
             return row.getList(name, elementsClass);
         }
 
         @Override
+        public <T> List<T> getList(String s, TypeToken<T> typeToken)
+        {
+            return row.getList(s, typeToken);
+        }
+
+        @Override
         public <T> Set<T> getSet(int i, Class<T> elementsClass)
         {
             return row.getSet(i, elementsClass);
         }
 
         @Override
+        public <T> Set<T> getSet(int i, TypeToken<T> typeToken)
+        {
+            return row.getSet(i, typeToken);
+        }
+
+        @Override
         public <T> Set<T> getSet(String name, Class<T> elementsClass)
         {
             return row.getSet(name, elementsClass);
         }
 
         @Override
+        public <T> Set<T> getSet(String s, TypeToken<T> typeToken)
+        {
+            return row.getSet(s, typeToken);
+        }
+
+        @Override
         public <K, V> Map<K, V> getMap(int i, Class<K> keysClass, Class<V> valuesClass)
         {
             return row.getMap(i, keysClass, valuesClass);
         }
 
         @Override
+        public <K, V> Map<K, V> getMap(int i, TypeToken<K> typeToken, TypeToken<V> typeToken1)
+        {
+            return row.getMap(i, typeToken, typeToken1);
+        }
+
+        @Override
         public <K, V> Map<K, V> getMap(String name, Class<K> keysClass, Class<V> valuesClass)
         {
             return row.getMap(name, keysClass, valuesClass);
         }
+
+        @Override
+        public <K, V> Map<K, V> getMap(String s, TypeToken<K> typeToken, TypeToken<V> typeToken1)
+        {
+            return row.getMap(s, typeToken, typeToken1);
+        }
+
+        @Override
+        public UDTValue getUDTValue(int i)
+        {
+            return row.getUDTValue(i);
+        }
+
+        @Override
+        public UDTValue getUDTValue(String name)
+        {
+            return row.getUDTValue(name);
+        }
+
+        @Override
+        public TupleValue getTupleValue(int i)
+        {
+            return row.getTupleValue(i);
+        }
+
+        @Override
+        public TupleValue getTupleValue(String name)
+        {
+            return row.getTupleValue(name);
+        }
+
+        @Override
+        public Token getToken(int i)
+        {
+            return row.getToken(i);
+        }
+
+        @Override
+        public Token getToken(String name)
+        {
+            return row.getToken(name);
+        }
+
+        @Override
+        public Token getPartitionKeyToken()
+        {
+            return row.getPartitionKeyToken();
+        }
     }
 
     /**
@@ -573,29 +721,21 @@
 
     private void fetchKeys()
     {
-        String query = "SELECT column_name, component_index, type FROM system.schema_columns WHERE keyspace_name='" +
-                       keyspace + "' and columnfamily_name='" + cfName + "'";
-        List<Row> rows = session.execute(query).all();
-        if (rows.isEmpty())
+        // get CF meta data
+        TableMetadata tableMetadata = session.getCluster()
+                                             .getMetadata()
+                                             .getKeyspace(Metadata.quote(keyspace))
+                                             .getTable(Metadata.quote(cfName));
+        if (tableMetadata == null)
         {
             throw new RuntimeException("No table metadata found for " + keyspace + "." + cfName);
         }
-        int numberOfPartitionKeys = 0;
-        for (Row row : rows)
-            if (row.getString(2).equals("partition_key"))
-                numberOfPartitionKeys++;
-        String[] partitionKeyArray = new String[numberOfPartitionKeys];
-        for (Row row : rows)
+        //Here we assume that tableMetadata.getPartitionKey() always
+        //returns the list of columns in order of component_index
+        for (ColumnMetadata partitionKey : tableMetadata.getPartitionKey())
         {
-            String type = row.getString(2);
-            String column = row.getString(0);
-            if (type.equals("partition_key"))
-            {
-                int componentIndex = row.isNull(1) ? 0 : row.getInt(1);
-                partitionKeyArray[componentIndex] = column;
-            }
+            partitionKeys.add(partitionKey.getName());
         }
-        partitionKeys.addAll(Arrays.asList(partitionKeyArray));
     }
 
     private String quote(String identifier)
diff --git a/src/java/org/apache/cassandra/hadoop/cql3/CqlRecordWriter.java b/src/java/org/apache/cassandra/hadoop/cql3/CqlRecordWriter.java
index 2096055..e3d1772 100644
--- a/src/java/org/apache/cassandra/hadoop/cql3/CqlRecordWriter.java
+++ b/src/java/org/apache/cassandra/hadoop/cql3/CqlRecordWriter.java
@@ -21,35 +21,29 @@
 import java.net.InetAddress;
 import java.nio.ByteBuffer;
 import java.util.*;
-import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.*;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.cassandra.db.marshal.AbstractType;
+
+import com.datastax.driver.core.*;
+import com.datastax.driver.core.exceptions.*;
 import org.apache.cassandra.db.marshal.CompositeType;
-import org.apache.cassandra.db.marshal.LongType;
-import org.apache.cassandra.db.marshal.TypeParser;
-import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.exceptions.SyntaxException;
-import org.apache.cassandra.hadoop.AbstractColumnFamilyOutputFormat;
-import org.apache.cassandra.hadoop.AbstractColumnFamilyRecordWriter;
+import org.apache.cassandra.hadoop.ColumnFamilyOutputFormat;
 import org.apache.cassandra.hadoop.ConfigHelper;
 import org.apache.cassandra.hadoop.HadoopCompat;
-import org.apache.cassandra.thrift.*;
-import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.RecordWriter;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.util.Progressable;
-import org.apache.thrift.TException;
-import org.apache.thrift.transport.TTransport;
 
 /**
- * The <code>ColumnFamilyRecordWriter</code> maps the output &lt;key, value&gt;
- * pairs to a Cassandra column family. In particular, it applies the binded variables
- * in the value to the prepared statement, which it associates with the key, and in 
+ * The <code>CqlRecordWriter</code> maps the output &lt;key, value&gt;
+ * pairs to a Cassandra table. In particular, it applies the binded variables
+ * in the value to the prepared statement, which it associates with the key, and in
  * turn the responsible endpoint.
  *
  * <p>
@@ -60,21 +54,38 @@
  *
  * @see CqlOutputFormat
  */
-class CqlRecordWriter extends AbstractColumnFamilyRecordWriter<Map<String, ByteBuffer>, List<ByteBuffer>>
+class CqlRecordWriter extends RecordWriter<Map<String, ByteBuffer>, List<ByteBuffer>> implements
+        org.apache.hadoop.mapred.RecordWriter<Map<String, ByteBuffer>, List<ByteBuffer>>, AutoCloseable
 {
     private static final Logger logger = LoggerFactory.getLogger(CqlRecordWriter.class);
 
+    // The configuration this writer is associated with.
+    protected final Configuration conf;
+    // The number of mutations to buffer per endpoint
+    protected final int queueSize;
+
+    protected final long batchThreshold;
+
+    protected Progressable progressable;
+    protected TaskAttemptContext context;
+
+    // The ring cache that describes the token ranges each node in the ring is
+    // responsible for. This is what allows us to group the mutations by
+    // the endpoints they should be targeted at. The targeted endpoint
+    // essentially
+    // acts as the primary replica for the rows being affected by the mutations.
+    private final NativeRingCache ringCache;
+
     // handles for clients for each range running in the threadpool
     protected final Map<InetAddress, RangeClient> clients;
 
     // host to prepared statement id mappings
-    protected final ConcurrentHashMap<Cassandra.Client, Integer> preparedStatements = new ConcurrentHashMap<Cassandra.Client, Integer>();
+    protected final ConcurrentHashMap<Session, PreparedStatement> preparedStatements = new ConcurrentHashMap<Session, PreparedStatement>();
 
     protected final String cql;
 
-    protected AbstractType<?> keyValidator;
-    protected String [] partitionKeyColumns;
-    protected List<String> clusterColumns;
+    protected List<ColumnMetadata> partitionKeyColumns;
+    protected List<ColumnMetadata> clusterColumns;
 
     /**
      * Upon construction, obtain the map that this writer will use to collect
@@ -89,7 +100,7 @@
         this.context = context;
     }
 
-    CqlRecordWriter(Configuration conf, Progressable progressable) throws IOException
+    CqlRecordWriter(Configuration conf, Progressable progressable)
     {
         this(conf);
         this.progressable = progressable;
@@ -97,28 +108,26 @@
 
     CqlRecordWriter(Configuration conf)
     {
-        super(conf);
+        this.conf = conf;
+        this.queueSize = conf.getInt(ColumnFamilyOutputFormat.QUEUE_SIZE, 32 * FBUtilities.getAvailableProcessors());
+        batchThreshold = conf.getLong(ColumnFamilyOutputFormat.BATCH_THRESHOLD, 32);
         this.clients = new HashMap<>();
+        String keyspace = ConfigHelper.getOutputKeyspace(conf);
 
-        try
+        try (Cluster cluster = CqlConfigHelper.getOutputCluster(ConfigHelper.getOutputInitialAddress(conf), conf);
+             Session client = cluster.connect(keyspace))
         {
-            Cassandra.Client client = ConfigHelper.getClientFromOutputAddressList(conf);
+            ringCache = new NativeRingCache(conf);
             if (client != null)
             {
-                client.set_keyspace(ConfigHelper.getOutputKeyspace(conf));
-                String user = ConfigHelper.getOutputKeyspaceUserName(conf);
-                String password = ConfigHelper.getOutputKeyspacePassword(conf);
-                if ((user != null) && (password != null))
-                    AbstractColumnFamilyOutputFormat.login(user, password, client);
-                retrievePartitionKeyValidator(client);
+                TableMetadata tableMetadata = client.getCluster().getMetadata().getKeyspace(client.getLoggedKeyspace()).getTable(ConfigHelper.getOutputColumnFamily(conf));
+                clusterColumns = tableMetadata.getClusteringColumns();
+                partitionKeyColumns = tableMetadata.getPartitionKey();
+
                 String cqlQuery = CqlConfigHelper.getOutputCql(conf).trim();
                 if (cqlQuery.toLowerCase().startsWith("insert"))
                     throw new UnsupportedOperationException("INSERT with CqlRecordWriter is not supported, please use UPDATE/DELETE statement");
                 cql = appendKeyWhereClauses(cqlQuery);
-
-                TTransport transport = client.getOutputProtocol().getTransport();
-                if (transport.isOpen())
-                    transport.close();
             }
             else
             {
@@ -130,7 +139,26 @@
             throw new RuntimeException(e);
         }
     }
-    
+
+    /**
+     * Close this <code>RecordWriter</code> to future operations, but not before
+     * flushing out the batched mutations.
+     *
+     * @param context the context of the task
+     * @throws IOException
+     */
+    public void close(TaskAttemptContext context) throws IOException, InterruptedException
+    {
+        close();
+    }
+
+    /** Fills the deprecated RecordWriter interface for streaming. */
+    @Deprecated
+    public void close(org.apache.hadoop.mapred.Reporter reporter) throws IOException
+    {
+        close();
+    }
+
     @Override
     public void close() throws IOException
     {
@@ -151,10 +179,10 @@
         if (clientException != null)
             throw clientException;
     }
-    
+
     /**
      * If the key is to be associated with a valid value, a mutation is created
-     * for it with the given column family and columns. In the event the value
+     * for it with the given table and columns. In the event the value
      * in the column is missing (i.e., null), then it is marked for
      * {@link Deletion}. Similarly, if the entire value for a key is missing
      * (i.e., null), then the entire key is marked for {@link Deletion}.
@@ -169,25 +197,25 @@
     @Override
     public void write(Map<String, ByteBuffer> keyColumns, List<ByteBuffer> values) throws IOException
     {
-        Range<Token> range = ringCache.getRange(getPartitionKey(keyColumns));
+        TokenRange range = ringCache.getRange(getPartitionKey(keyColumns));
 
         // get the client for the given range, or create a new one
-	final InetAddress address = ringCache.getEndpoint(range).get(0);
+        final InetAddress address = ringCache.getEndpoints(range).get(0);
         RangeClient client = clients.get(address);
         if (client == null)
         {
             // haven't seen keys for this range: create new client
-            client = new RangeClient(ringCache.getEndpoint(range));
+            client = new RangeClient(ringCache.getEndpoints(range));
             client.start();
             clients.put(address, client);
         }
 
         // add primary key columns to the bind variables
         List<ByteBuffer> allValues = new ArrayList<ByteBuffer>(values);
-        for (String column : partitionKeyColumns)
-            allValues.add(keyColumns.get(column));
-        for (String column : clusterColumns)
-            allValues.add(keyColumns.get(column));
+        for (ColumnMetadata column : partitionKeyColumns)
+            allValues.add(keyColumns.get(column.getName()));
+        for (ColumnMetadata column : clusterColumns)
+            allValues.add(keyColumns.get(column.getName()));
 
         client.put(allValues);
 
@@ -197,194 +225,255 @@
             HadoopCompat.progress(context);
     }
 
+    private static void closeSession(Session session)
+    {
+        //Close the session to satisfy to avoid warnings for the resource not being closed
+        try
+        {
+            if (session != null)
+                session.getCluster().closeAsync();
+        }
+        catch (Throwable t)
+        {
+            logger.warn("Error closing connection", t);
+        }
+    }
+
     /**
      * A client that runs in a threadpool and connects to the list of endpoints for a particular
      * range. Bound variables for keys in that range are sent to this client via a queue.
      */
-    public class RangeClient extends AbstractRangeClient<List<ByteBuffer>>
+    public class RangeClient extends Thread
     {
+        // The list of endpoints for this range
+        protected final List<InetAddress> endpoints;
+        protected Cluster cluster = null;
+        // A bounded queue of incoming mutations for this range
+        protected final BlockingQueue<List<ByteBuffer>> queue = new ArrayBlockingQueue<List<ByteBuffer>>(queueSize);
+
+        protected volatile boolean run = true;
+        // we want the caller to know if something went wrong, so we record any unrecoverable exception while writing
+        // so we can throw it on the caller's stack when he calls put() again, or if there are no more put calls,
+        // when the client is closed.
+        protected volatile IOException lastException;
+
         /**
          * Constructs an {@link RangeClient} for the given endpoints.
          * @param endpoints the possible endpoints to execute the mutations on
          */
         public RangeClient(List<InetAddress> endpoints)
         {
-            super(endpoints);
-         }
-        
+            super("client-" + endpoints);
+            this.endpoints = endpoints;
+        }
+
         /**
-         * Loops collecting cql binded variable values from the queue and sending to Cassandra
+         * enqueues the given value to Cassandra
          */
-        public void run()
+        public void put(List<ByteBuffer> value) throws IOException
         {
-            outer:
-            while (run || !queue.isEmpty())
+            while (true)
             {
-                List<ByteBuffer> bindVariables;
+                if (lastException != null)
+                    throw lastException;
                 try
                 {
-                    bindVariables = queue.take();
+                    if (queue.offer(value, 100, TimeUnit.MILLISECONDS))
+                        break;
                 }
                 catch (InterruptedException e)
                 {
-                    // re-check loop condition after interrupt
-                    continue;
+                    throw new AssertionError(e);
                 }
+            }
+        }
 
-                Iterator<InetAddress> iter = endpoints.iterator();
-                while (true)
+        /**
+         * Loops collecting cql binded variable values from the queue and sending to Cassandra
+         */
+        @SuppressWarnings("resource")
+        public void run()
+        {
+            Session session = null;
+            try
+            {
+                outer:
+                while (run || !queue.isEmpty())
                 {
-                    // send the mutation to the last-used endpoint.  first time through, this will NPE harmlessly.
+                    List<ByteBuffer> bindVariables;
                     try
                     {
-                        int i = 0;
-                        int itemId = preparedStatement(client);
-                        while (bindVariables != null)
-                        {
-                            client.execute_prepared_cql3_query(itemId, bindVariables, ConsistencyLevel.ONE);
-                            i++;
-                            
-                            if (i >= batchThreshold)
-                                break;
-                            
-                            bindVariables = queue.poll();
-                        }
-                        
-                        break;
+                        bindVariables = queue.take();
                     }
-                    catch (Exception e)
+                    catch (InterruptedException e)
                     {
-                        closeInternal();
-                        if (!iter.hasNext())
-                        {
-                            lastException = new IOException(e);
-                            break outer;
-                        }
+                        // re-check loop condition after interrupt
+                        continue;
                     }
 
-                    // attempt to connect to a different endpoint
-                    try
+                    ListIterator<InetAddress> iter = endpoints.listIterator();
+                    while (true)
                     {
-                        InetAddress address = iter.next();
-                        String host = address.getHostName();
-                        int port = ConfigHelper.getOutputRpcPort(conf);
-                        client = CqlOutputFormat.createAuthenticatedClient(host, port, conf);
-                    }
-                    catch (Exception e)
-                    {
-                        closeInternal();
-                        // TException means something unexpected went wrong to that endpoint, so
-                        // we should try again to another.  Other exceptions (auth or invalid request) are fatal.
-                        if ((!(e instanceof TException)) || !iter.hasNext())
+                        // send the mutation to the last-used endpoint.  first time through, this will NPE harmlessly.
+                        if (session != null)
                         {
-                            lastException = new IOException(e);
-                            break outer;
+                            try
+                            {
+                                int i = 0;
+                                PreparedStatement statement = preparedStatement(session);
+                                while (bindVariables != null)
+                                {
+                                    BoundStatement boundStatement = new BoundStatement(statement);
+                                    for (int columnPosition = 0; columnPosition < bindVariables.size(); columnPosition++)
+                                    {
+                                        boundStatement.setBytesUnsafe(columnPosition, bindVariables.get(columnPosition));
+                                    }
+                                    session.execute(boundStatement);
+                                    i++;
+
+                                    if (i >= batchThreshold)
+                                        break;
+                                    bindVariables = queue.poll();
+                                }
+                                break;
+                            }
+                            catch (Exception e)
+                            {
+                                closeInternal();
+                                if (!iter.hasNext())
+                                {
+                                    lastException = new IOException(e);
+                                    break outer;
+                                }
+                            }
+                        }
+
+                        // attempt to connect to a different endpoint
+                        try
+                        {
+                            InetAddress address = iter.next();
+                            String host = address.getHostName();
+                            cluster = CqlConfigHelper.getOutputCluster(host, conf);
+                            closeSession(session);
+                            session = cluster.connect();
+                        }
+                        catch (Exception e)
+                        {
+                            //If connection died due to Interrupt, just try connecting to the endpoint again.
+                            //There are too many ways for the Thread.interrupted() state to be cleared, so
+                            //we can't rely on that here. Until the java driver gives us a better way of knowing
+                            //that this exception came from an InterruptedException, this is the best solution.
+                            if (canRetryDriverConnection(e))
+                            {
+                                iter.previous();
+                            }
+                            closeInternal();
+
+                            // Most exceptions mean something unexpected went wrong to that endpoint, so
+                            // we should try again to another.  Other exceptions (auth or invalid request) are fatal.
+                            if ((e instanceof AuthenticationException || e instanceof InvalidQueryException) || !iter.hasNext())
+                            {
+                                lastException = new IOException(e);
+                                break outer;
+                            }
                         }
                     }
                 }
             }
+            finally
+            {
+                closeSession(session);
+            }
 
             // close all our connections once we are done.
             closeInternal();
         }
 
         /** get prepared statement id from cache, otherwise prepare it from Cassandra server*/
-        private int preparedStatement(Cassandra.Client client)
+        private PreparedStatement preparedStatement(Session client)
         {
-            Integer itemId = preparedStatements.get(client);
-            if (itemId == null)
+            PreparedStatement statement = preparedStatements.get(client);
+            if (statement == null)
             {
-                CqlPreparedResult result;
+                PreparedStatement result;
                 try
                 {
-                    result = client.prepare_cql3_query(ByteBufferUtil.bytes(cql), Compression.NONE);
+                    result = client.prepare(cql);
                 }
-                catch (InvalidRequestException e)
-                {
-                    throw new RuntimeException("failed to prepare cql query " + cql, e);
-                }
-                catch (TException e)
+                catch (NoHostAvailableException e)
                 {
                     throw new RuntimeException("failed to prepare cql query " + cql, e);
                 }
 
-                Integer previousId = preparedStatements.putIfAbsent(client, Integer.valueOf(result.itemId));
-                itemId = previousId == null ? result.itemId : previousId;
+                PreparedStatement previousId = preparedStatements.putIfAbsent(client, result);
+                statement = previousId == null ? result : previousId;
             }
-            return itemId;
+            return statement;
+        }
+
+        public void close() throws IOException
+        {
+            // stop the run loop.  this will result in closeInternal being called by the time join() finishes.
+            run = false;
+            interrupt();
+            try
+            {
+                this.join();
+            }
+            catch (InterruptedException e)
+            {
+                throw new AssertionError(e);
+            }
+
+            if (lastException != null)
+                throw lastException;
+        }
+
+        protected void closeInternal()
+        {
+            if (cluster != null)
+            {
+                cluster.close();
+            }
+        }
+
+        private boolean canRetryDriverConnection(Exception e)
+        {
+            if (e instanceof DriverException && e.getMessage().contains("Connection thread interrupted"))
+                return true;
+            if (e instanceof NoHostAvailableException)
+            {
+                if (((NoHostAvailableException) e).getErrors().values().size() == 1)
+                {
+                    Throwable cause = ((NoHostAvailableException) e).getErrors().values().iterator().next();
+                    if (cause != null && cause.getCause() instanceof java.nio.channels.ClosedByInterruptException)
+                    {
+                        return true;
+                    }
+                }
+            }
+            return false;
         }
     }
 
     private ByteBuffer getPartitionKey(Map<String, ByteBuffer> keyColumns)
     {
         ByteBuffer partitionKey;
-        if (keyValidator instanceof CompositeType)
+        if (partitionKeyColumns.size() > 1)
         {
-            ByteBuffer[] keys = new ByteBuffer[partitionKeyColumns.length];
+            ByteBuffer[] keys = new ByteBuffer[partitionKeyColumns.size()];
             for (int i = 0; i< keys.length; i++)
-                keys[i] = keyColumns.get(partitionKeyColumns[i]);
+                keys[i] = keyColumns.get(partitionKeyColumns.get(i).getName());
 
             partitionKey = CompositeType.build(keys);
         }
         else
         {
-            partitionKey = keyColumns.get(partitionKeyColumns[0]);
+            partitionKey = keyColumns.get(partitionKeyColumns.get(0).getName());
         }
         return partitionKey;
     }
 
-    /** retrieve the key validator from system.schema_columnfamilies table */
-    private void retrievePartitionKeyValidator(Cassandra.Client client) throws Exception
-    {
-        String keyspace = ConfigHelper.getOutputKeyspace(conf);
-        String cfName = ConfigHelper.getOutputColumnFamily(conf);
-        String query = "SELECT key_validator," +
-        		       "       key_aliases," +
-        		       "       column_aliases " +
-                       "FROM system.schema_columnfamilies " +
-                       "WHERE keyspace_name='%s' and columnfamily_name='%s'";
-        String formatted = String.format(query, keyspace, cfName);
-        CqlResult result = client.execute_cql3_query(ByteBufferUtil.bytes(formatted), Compression.NONE, ConsistencyLevel.ONE);
-
-        Column rawKeyValidator = result.rows.get(0).columns.get(0);
-        String validator = ByteBufferUtil.string(ByteBuffer.wrap(rawKeyValidator.getValue()));
-        keyValidator = parseType(validator);
-        
-        Column rawPartitionKeys = result.rows.get(0).columns.get(1);
-        String keyString = ByteBufferUtil.string(ByteBuffer.wrap(rawPartitionKeys.getValue()));
-        logger.debug("partition keys: {}", keyString);
-
-        List<String> keys = FBUtilities.fromJsonList(keyString);
-        partitionKeyColumns = new String[keys.size()];
-        int i = 0;
-        for (String key : keys)
-        {
-            partitionKeyColumns[i] = key;
-            i++;
-        }
-
-        Column rawClusterColumns = result.rows.get(0).columns.get(2);
-        String clusterColumnString = ByteBufferUtil.string(ByteBuffer.wrap(rawClusterColumns.getValue()));
-
-        logger.debug("cluster columns: {}", clusterColumnString);
-        clusterColumns = FBUtilities.fromJsonList(clusterColumnString);
-    }
-
-    private AbstractType<?> parseType(String type) throws ConfigurationException
-    {
-        try
-        {
-            // always treat counters like longs, specifically CCT.serialize is not what we need
-            if (type != null && type.equals("org.apache.cassandra.db.marshal.CounterColumnType"))
-                return LongType.instance;
-            return TypeParser.parse(type);
-        }
-        catch (SyntaxException e)
-        {
-            throw new ConfigurationException(e.getMessage(), e);
-        }
-    }
-
     /**
      * add where clauses for partition keys and cluster columns
      */
@@ -392,10 +481,10 @@
     {
         String keyWhereClause = "";
 
-        for (String partitionKey : partitionKeyColumns)
-            keyWhereClause += String.format("%s = ?", keyWhereClause.isEmpty() ? quote(partitionKey) : (" AND " + quote(partitionKey)));
-        for (String clusterColumn : clusterColumns)
-            keyWhereClause += " AND " + quote(clusterColumn) + " = ?";
+        for (ColumnMetadata partitionKey : partitionKeyColumns)
+            keyWhereClause += String.format("%s = ?", keyWhereClause.isEmpty() ? quote(partitionKey.getName()) : (" AND " + quote(partitionKey.getName())));
+        for (ColumnMetadata clusterColumn : clusterColumns)
+            keyWhereClause += " AND " + quote(clusterColumn.getName()) + " = ?";
 
         return cqlQuery + " WHERE " + keyWhereClause;
     }
@@ -405,4 +494,60 @@
     {
         return "\"" + identifier.replaceAll("\"", "\"\"") + "\"";
     }
+
+    class NativeRingCache
+    {
+        private Map<TokenRange, Set<Host>> rangeMap;
+        private Metadata metadata;
+        private final IPartitioner partitioner;
+        private final Configuration conf;
+
+        public NativeRingCache(Configuration conf)
+        {
+            this.conf = conf;
+            this.partitioner = ConfigHelper.getOutputPartitioner(conf);
+            refreshEndpointMap();
+        }
+
+
+        private void refreshEndpointMap()
+        {
+            String keyspace = ConfigHelper.getOutputKeyspace(conf);
+            try (Cluster cluster = CqlConfigHelper.getOutputCluster(ConfigHelper.getOutputInitialAddress(conf), conf);
+                 Session session = cluster.connect(keyspace))
+            {
+                rangeMap = new HashMap<>();
+                metadata = session.getCluster().getMetadata();
+                Set<TokenRange> ranges = metadata.getTokenRanges();
+                for (TokenRange range : ranges)
+                    rangeMap.put(range, metadata.getReplicas(keyspace, range));
+            }
+        }
+
+        public TokenRange getRange(ByteBuffer key)
+        {
+            Token t = partitioner.getToken(key);
+            com.datastax.driver.core.Token driverToken = metadata.newToken(partitioner.getTokenFactory().toString(t));
+            for (TokenRange range : rangeMap.keySet())
+            {
+                if (range.contains(driverToken))
+                {
+                    return range;
+                }
+            }
+
+            throw new RuntimeException("Invalid token information returned by describe_ring: " + rangeMap);
+        }
+
+        public List<InetAddress> getEndpoints(TokenRange range)
+        {
+            Set<Host> hostSet = rangeMap.get(range);
+            List<InetAddress> addresses = new ArrayList<>(hostSet.size());
+            for (Host host: hostSet)
+            {
+                addresses.add(host.getAddress());
+            }
+            return addresses;
+        }
+    }
 }
diff --git a/src/java/org/apache/cassandra/hadoop/cql3/LimitedLocalNodeFirstLocalBalancingPolicy.java b/src/java/org/apache/cassandra/hadoop/cql3/LimitedLocalNodeFirstLocalBalancingPolicy.java
index 8949892..5c8d3c5 100644
--- a/src/java/org/apache/cassandra/hadoop/cql3/LimitedLocalNodeFirstLocalBalancingPolicy.java
+++ b/src/java/org/apache/cassandra/hadoop/cql3/LimitedLocalNodeFirstLocalBalancingPolicy.java
@@ -71,7 +71,7 @@
                 logger.warn("Invalid replica host name: {}, skipping it", replica);
             }
         }
-        logger.debug("Created instance with the following replicas: {}", Arrays.asList(replicas));
+        logger.trace("Created instance with the following replicas: {}", Arrays.asList(replicas));
     }
 
     @Override
@@ -86,7 +86,13 @@
             }
         }
         liveReplicaHosts.addAll(replicaHosts);
-        logger.debug("Initialized with replica hosts: {}", replicaHosts);
+        logger.trace("Initialized with replica hosts: {}", replicaHosts);
+    }
+
+    @Override
+    public void close()
+    {
+        //
     }
 
     @Override
@@ -121,7 +127,7 @@
 
         Collections.shuffle(remote);
 
-        logger.debug("Using the following hosts order for the new query plan: {} | {}", local, remote);
+        logger.trace("Using the following hosts order for the new query plan: {} | {}", local, remote);
 
         return Iterators.concat(local.iterator(), remote.iterator());
     }
@@ -132,7 +138,7 @@
         if (replicaAddresses.contains(host.getAddress()))
         {
             liveReplicaHosts.add(host);
-            logger.debug("Added a new host {}", host);
+            logger.trace("Added a new host {}", host);
         }
     }
 
@@ -142,7 +148,7 @@
         if (replicaAddresses.contains(host.getAddress()))
         {
             liveReplicaHosts.add(host);
-            logger.debug("The host {} is now up", host);
+            logger.trace("The host {} is now up", host);
         }
     }
 
@@ -151,7 +157,7 @@
     {
         if (liveReplicaHosts.remove(host))
         {
-            logger.debug("The host {} is now down", host);
+            logger.trace("The host {} is now down", host);
         }
     }
 
@@ -161,11 +167,10 @@
     {
         if (liveReplicaHosts.remove(host))
         {
-            logger.debug("Removed the host {}", host);
+            logger.trace("Removed the host {}", host);
         }
     }
 
-    @Override
     public void onSuspected(Host host)
     {
         // not supported by this load balancing policy
diff --git a/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java b/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
deleted file mode 100644
index 5884f29..0000000
--- a/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
+++ /dev/null
@@ -1,812 +0,0 @@
-/*
- * 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.hadoop.pig;
-
-import java.io.IOException;
-import java.io.UnsupportedEncodingException;
-import java.math.BigInteger;
-import java.net.URLDecoder;
-import java.nio.ByteBuffer;
-import java.nio.charset.CharacterCodingException;
-import java.util.*;
-
-import com.google.common.collect.Iterables;
-
-import org.apache.cassandra.db.Cell;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.exceptions.SyntaxException;
-import org.apache.cassandra.auth.IAuthenticator;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.db.marshal.*;
-import org.apache.cassandra.db.marshal.AbstractCompositeType.CompositeComponent;
-import org.apache.cassandra.serializers.CollectionSerializer;
-import org.apache.cassandra.hadoop.*;
-import org.apache.cassandra.thrift.*;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.Hex;
-import org.apache.cassandra.utils.UUIDGen;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapreduce.*;
-import org.apache.pig.*;
-import org.apache.pig.backend.executionengine.ExecException;
-import org.apache.pig.data.*;
-import org.apache.pig.impl.util.UDFContext;
-import org.apache.thrift.TDeserializer;
-import org.apache.thrift.TException;
-import org.apache.thrift.TSerializer;
-import org.apache.thrift.protocol.TBinaryProtocol;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A LoadStoreFunc for retrieving data from and storing data to Cassandra
- */
-public abstract class AbstractCassandraStorage extends LoadFunc implements StoreFuncInterface, LoadMetadata
-{
-
-    protected enum MarshallerType { COMPARATOR, DEFAULT_VALIDATOR, KEY_VALIDATOR, SUBCOMPARATOR };
-
-    // system environment variables that can be set to configure connection info:
-    // alternatively, Hadoop JobConf variables can be set using keys from ConfigHelper
-    public final static String PIG_INPUT_RPC_PORT = "PIG_INPUT_RPC_PORT";
-    public final static String PIG_INPUT_INITIAL_ADDRESS = "PIG_INPUT_INITIAL_ADDRESS";
-    public final static String PIG_INPUT_PARTITIONER = "PIG_INPUT_PARTITIONER";
-    public final static String PIG_OUTPUT_RPC_PORT = "PIG_OUTPUT_RPC_PORT";
-    public final static String PIG_OUTPUT_INITIAL_ADDRESS = "PIG_OUTPUT_INITIAL_ADDRESS";
-    public final static String PIG_OUTPUT_PARTITIONER = "PIG_OUTPUT_PARTITIONER";
-    public final static String PIG_RPC_PORT = "PIG_RPC_PORT";
-    public final static String PIG_INITIAL_ADDRESS = "PIG_INITIAL_ADDRESS";
-    public final static String PIG_PARTITIONER = "PIG_PARTITIONER";
-    public final static String PIG_INPUT_FORMAT = "PIG_INPUT_FORMAT";
-    public final static String PIG_OUTPUT_FORMAT = "PIG_OUTPUT_FORMAT";
-    public final static String PIG_INPUT_SPLIT_SIZE = "PIG_INPUT_SPLIT_SIZE";
-
-    protected String DEFAULT_INPUT_FORMAT;
-    protected String DEFAULT_OUTPUT_FORMAT;
-
-    public final static String PARTITION_FILTER_SIGNATURE = "cassandra.partition.filter";
-
-    private static final Logger logger = LoggerFactory.getLogger(AbstractCassandraStorage.class);
-
-    protected String username;
-    protected String password;
-    protected String keyspace;
-    protected String column_family;
-    protected String loadSignature;
-    protected String storeSignature;
-
-    protected Configuration conf;
-    protected String inputFormatClass;
-    protected String outputFormatClass;
-    protected int splitSize = 64 * 1024;
-    protected String partitionerClass;
-    protected boolean usePartitionFilter = false;
-    protected String initHostAddress;
-    protected String rpcPort;
-    protected int nativeProtocolVersion = 1;
-
-
-    public AbstractCassandraStorage()
-    {
-        super();
-    }
-
-    /** Deconstructs a composite type to a Tuple. */
-    protected Tuple composeComposite(AbstractCompositeType comparator, ByteBuffer name) throws IOException
-    {
-        List<CompositeComponent> result = comparator.deconstruct(name);
-        Tuple t = TupleFactory.getInstance().newTuple(result.size());
-        for (int i=0; i<result.size(); i++)
-            setTupleValue(t, i, cassandraToObj(result.get(i).comparator, result.get(i).value));
-
-        return t;
-    }
-
-    /** convert a column to a tuple */
-    protected Tuple columnToTuple(Cell col, CfInfo cfInfo, AbstractType comparator) throws IOException
-    {
-        CfDef cfDef = cfInfo.cfDef;
-        Tuple pair = TupleFactory.getInstance().newTuple(2);
-
-        ByteBuffer colName = col.name().toByteBuffer();
-
-        // name
-        if(comparator instanceof AbstractCompositeType)
-            setTupleValue(pair, 0, composeComposite((AbstractCompositeType)comparator,colName));
-        else
-            setTupleValue(pair, 0, cassandraToObj(comparator, colName));
-
-        // value
-        Map<ByteBuffer,AbstractType> validators = getValidatorMap(cfDef);
-        if (cfInfo.cql3Table && !cfInfo.compactCqlTable)
-        {
-            ByteBuffer[] names = ((AbstractCompositeType) parseType(cfDef.comparator_type)).split(colName);
-            colName = names[names.length-1];
-        }
-        if (validators.get(colName) == null)
-        {
-            Map<MarshallerType, AbstractType> marshallers = getDefaultMarshallers(cfDef);
-            setTupleValue(pair, 1, cassandraToObj(marshallers.get(MarshallerType.DEFAULT_VALIDATOR), col.value()));
-        }
-        else
-            setTupleValue(pair, 1, cassandraToObj(validators.get(colName), col.value()));
-        return pair;
-    }
-
-    /** set the value to the position of the tuple */
-    protected void setTupleValue(Tuple pair, int position, Object value) throws ExecException
-    {
-       if (value instanceof BigInteger)
-           pair.set(position, ((BigInteger) value).intValue());
-       else if (value instanceof ByteBuffer)
-           pair.set(position, new DataByteArray(ByteBufferUtil.getArray((ByteBuffer) value)));
-       else if (value instanceof UUID)
-           pair.set(position, new DataByteArray(UUIDGen.decompose((java.util.UUID) value)));
-       else if (value instanceof Date)
-           pair.set(position, TimestampType.instance.decompose((Date) value).getLong());
-       else
-           pair.set(position, value);
-    }
-
-    /** get the columnfamily definition for the signature */
-    protected CfInfo getCfInfo(String signature) throws IOException
-    {
-        UDFContext context = UDFContext.getUDFContext();
-        Properties property = context.getUDFProperties(AbstractCassandraStorage.class);
-        String prop = property.getProperty(signature);
-        CfInfo cfInfo = new CfInfo();
-        cfInfo.cfDef = cfdefFromString(prop.substring(2));
-        cfInfo.compactCqlTable = prop.charAt(0) == '1' ? true : false;
-        cfInfo.cql3Table = prop.charAt(1) == '1' ? true : false;
-        return cfInfo;
-    }
-
-    /** construct a map to store the mashaller type to cassandra data type mapping */
-    protected Map<MarshallerType, AbstractType> getDefaultMarshallers(CfDef cfDef) throws IOException
-    {
-        Map<MarshallerType, AbstractType> marshallers = new EnumMap<MarshallerType, AbstractType>(MarshallerType.class);
-        AbstractType comparator;
-        AbstractType subcomparator;
-        AbstractType default_validator;
-        AbstractType key_validator;
-
-        comparator = parseType(cfDef.getComparator_type());
-        subcomparator = parseType(cfDef.getSubcomparator_type());
-        default_validator = parseType(cfDef.getDefault_validation_class());
-        key_validator = parseType(cfDef.getKey_validation_class());
-
-        marshallers.put(MarshallerType.COMPARATOR, comparator);
-        marshallers.put(MarshallerType.DEFAULT_VALIDATOR, default_validator);
-        marshallers.put(MarshallerType.KEY_VALIDATOR, key_validator);
-        marshallers.put(MarshallerType.SUBCOMPARATOR, subcomparator);
-        return marshallers;
-    }
-
-    /** get the validators */
-    protected Map<ByteBuffer, AbstractType> getValidatorMap(CfDef cfDef) throws IOException
-    {
-        Map<ByteBuffer, AbstractType> validators = new HashMap<ByteBuffer, AbstractType>();
-        for (ColumnDef cd : cfDef.getColumn_metadata())
-        {
-            if (cd.getValidation_class() != null && !cd.getValidation_class().isEmpty())
-            {
-                AbstractType validator = null;
-                try
-                {
-                    validator = TypeParser.parse(cd.getValidation_class());
-                    if (validator instanceof CounterColumnType)
-                        validator = LongType.instance; 
-                    validators.put(cd.name, validator);
-                }
-                catch (ConfigurationException e)
-                {
-                    throw new IOException(e);
-                }
-                catch (SyntaxException e)
-                {
-                    throw new IOException(e);
-                }
-            }
-        }
-        return validators;
-    }
-
-    /** parse the string to a cassandra data type */
-    protected AbstractType parseType(String type) throws IOException
-    {
-        try
-        {
-            // always treat counters like longs, specifically CCT.compose is not what we need
-            if (type != null && type.equals("org.apache.cassandra.db.marshal.CounterColumnType"))
-                    return LongType.instance;
-            return TypeParser.parse(type);
-        }
-        catch (ConfigurationException e)
-        {
-            throw new IOException(e);
-        }
-        catch (SyntaxException e)
-        {
-            throw new IOException(e);
-        }
-    }
-
-    @Override
-    public InputFormat getInputFormat() throws IOException
-    {
-        try
-        {
-            return FBUtilities.construct(inputFormatClass, "inputformat");
-        }
-        catch (ConfigurationException e)
-        {
-            throw new IOException(e);
-        }
-    }
-
-    /** decompose the query to store the parameters in a map */
-    public static Map<String, String> getQueryMap(String query) throws UnsupportedEncodingException 
-    {
-        String[] params = query.split("&");
-        Map<String, String> map = new HashMap<String, String>();
-        for (String param : params)
-        {
-            String[] keyValue = param.split("=");
-            map.put(keyValue[0], URLDecoder.decode(keyValue[1],"UTF-8"));
-        }
-        return map;
-    }
-
-    /** set hadoop cassandra connection settings */
-    protected void setConnectionInformation() throws IOException
-    {
-        if (System.getenv(PIG_RPC_PORT) != null)
-        {
-            ConfigHelper.setInputRpcPort(conf, System.getenv(PIG_RPC_PORT));
-            ConfigHelper.setOutputRpcPort(conf, System.getenv(PIG_RPC_PORT));
-        }
-
-        if (System.getenv(PIG_INPUT_RPC_PORT) != null)
-            ConfigHelper.setInputRpcPort(conf, System.getenv(PIG_INPUT_RPC_PORT));
-        if (System.getenv(PIG_OUTPUT_RPC_PORT) != null)
-            ConfigHelper.setOutputRpcPort(conf, System.getenv(PIG_OUTPUT_RPC_PORT));
-
-        if (System.getenv(PIG_INITIAL_ADDRESS) != null)
-        {
-            ConfigHelper.setInputInitialAddress(conf, System.getenv(PIG_INITIAL_ADDRESS));
-            ConfigHelper.setOutputInitialAddress(conf, System.getenv(PIG_INITIAL_ADDRESS));
-        }
-        if (System.getenv(PIG_INPUT_INITIAL_ADDRESS) != null)
-            ConfigHelper.setInputInitialAddress(conf, System.getenv(PIG_INPUT_INITIAL_ADDRESS));
-        if (System.getenv(PIG_OUTPUT_INITIAL_ADDRESS) != null)
-            ConfigHelper.setOutputInitialAddress(conf, System.getenv(PIG_OUTPUT_INITIAL_ADDRESS));
-
-        if (System.getenv(PIG_PARTITIONER) != null)
-        {
-            ConfigHelper.setInputPartitioner(conf, System.getenv(PIG_PARTITIONER));
-            ConfigHelper.setOutputPartitioner(conf, System.getenv(PIG_PARTITIONER));
-        }
-        if(System.getenv(PIG_INPUT_PARTITIONER) != null)
-            ConfigHelper.setInputPartitioner(conf, System.getenv(PIG_INPUT_PARTITIONER));
-        if(System.getenv(PIG_OUTPUT_PARTITIONER) != null)
-            ConfigHelper.setOutputPartitioner(conf, System.getenv(PIG_OUTPUT_PARTITIONER));
-        if (System.getenv(PIG_INPUT_FORMAT) != null)
-            inputFormatClass = getFullyQualifiedClassName(System.getenv(PIG_INPUT_FORMAT));
-        else
-            inputFormatClass = DEFAULT_INPUT_FORMAT;
-        if (System.getenv(PIG_OUTPUT_FORMAT) != null)
-            outputFormatClass = getFullyQualifiedClassName(System.getenv(PIG_OUTPUT_FORMAT));
-        else
-            outputFormatClass = DEFAULT_OUTPUT_FORMAT;
-    }
-
-    /** get the full class name */
-    protected String getFullyQualifiedClassName(String classname)
-    {
-        return classname.contains(".") ? classname : "org.apache.cassandra.hadoop." + classname;
-    }
-
-    /** get pig type for the cassandra data type*/
-    protected byte getPigType(AbstractType type)
-    {
-        if (type instanceof LongType || type instanceof DateType || type instanceof TimestampType) // DateType is bad and it should feel bad
-            return DataType.LONG;
-        else if (type instanceof IntegerType || type instanceof Int32Type) // IntegerType will overflow at 2**31, but is kept for compatibility until pig has a BigInteger
-            return DataType.INTEGER;
-        else if (type instanceof AsciiType || type instanceof UTF8Type || type instanceof DecimalType || type instanceof InetAddressType)
-            return DataType.CHARARRAY;
-        else if (type instanceof FloatType)
-            return DataType.FLOAT;
-        else if (type instanceof DoubleType)
-            return DataType.DOUBLE;
-        else if (type instanceof AbstractCompositeType || type instanceof CollectionType)
-            return DataType.TUPLE;
-
-        return DataType.BYTEARRAY;
-    }
-
-    public ResourceStatistics getStatistics(String location, Job job)
-    {
-        return null;
-    }
-
-    @Override
-    public String relativeToAbsolutePath(String location, Path curDir) throws IOException
-    {
-        return location;
-    }
-
-    @Override
-    public void setUDFContextSignature(String signature)
-    {
-        this.loadSignature = signature;
-    }
-
-    /** StoreFunc methods */
-    public void setStoreFuncUDFContextSignature(String signature)
-    {
-        this.storeSignature = signature;
-    }
-
-    public String relToAbsPathForStoreLocation(String location, Path curDir) throws IOException
-    {
-        return relativeToAbsolutePath(location, curDir);
-    }
-
-    /** output format */
-    public OutputFormat getOutputFormat() throws IOException
-    {
-        try
-        {
-            return FBUtilities.construct(outputFormatClass, "outputformat");
-        }
-        catch (ConfigurationException e)
-        {
-            throw new IOException(e);
-        }
-    }
-
-    public void checkSchema(ResourceSchema schema) throws IOException
-    {
-        // we don't care about types, they all get casted to ByteBuffers
-    }
-
-    protected abstract ByteBuffer nullToBB();
-
-    /** convert object to ByteBuffer */
-    protected ByteBuffer objToBB(Object o)
-    {
-        if (o == null)
-            return nullToBB();
-        if (o instanceof java.lang.String)
-            return ByteBuffer.wrap(new DataByteArray((String)o).get());
-        if (o instanceof Integer)
-            return Int32Type.instance.decompose((Integer)o);
-        if (o instanceof Long)
-            return LongType.instance.decompose((Long)o);
-        if (o instanceof Float)
-            return FloatType.instance.decompose((Float)o);
-        if (o instanceof Double)
-            return DoubleType.instance.decompose((Double)o);
-        if (o instanceof UUID)
-            return ByteBuffer.wrap(UUIDGen.decompose((UUID) o));
-        if(o instanceof Tuple) {
-            List<Object> objects = ((Tuple)o).getAll();
-            //collections
-            if (objects.size() > 0 && objects.get(0) instanceof String)
-            {
-                String collectionType = (String) objects.get(0);
-                if ("set".equalsIgnoreCase(collectionType) ||
-                        "list".equalsIgnoreCase(collectionType))
-                    return objToListOrSetBB(objects.subList(1, objects.size()));
-                else if ("map".equalsIgnoreCase(collectionType))
-                    return objToMapBB(objects.subList(1, objects.size()));
-                   
-            }
-            return objToCompositeBB(objects);
-        }
-
-        return ByteBuffer.wrap(((DataByteArray) o).get());
-    }
-
-    private ByteBuffer objToListOrSetBB(List<Object> objects)
-    {
-        List<ByteBuffer> serialized = new ArrayList<ByteBuffer>(objects.size());
-        for(Object sub : objects)
-        {
-            ByteBuffer buffer = objToBB(sub);
-            serialized.add(buffer);
-        }
-        // NOTE: using protocol v1 serialization format for collections so as to not break
-        // compatibility. Not sure if that's the right thing.
-        return CollectionSerializer.pack(serialized, objects.size(), 1);
-    }
-
-    private ByteBuffer objToMapBB(List<Object> objects)
-    {
-        List<ByteBuffer> serialized = new ArrayList<ByteBuffer>(objects.size() * 2);
-        for(Object sub : objects)
-        {
-            List<Object> keyValue = ((Tuple)sub).getAll();
-            for (Object entry: keyValue)
-            {
-                ByteBuffer buffer = objToBB(entry);
-                serialized.add(buffer);
-            }
-        } 
-        // NOTE: using protocol v1 serialization format for collections so as to not break
-        // compatibility. Not sure if that's the right thing.
-        return CollectionSerializer.pack(serialized, objects.size(), 1);
-    }
-
-    private ByteBuffer objToCompositeBB(List<Object> objects)
-    {
-        List<ByteBuffer> serialized = new ArrayList<ByteBuffer>(objects.size());
-        int totalLength = 0;
-        for(Object sub : objects)
-        {
-            ByteBuffer buffer = objToBB(sub);
-            serialized.add(buffer);
-            totalLength += 2 + buffer.remaining() + 1;
-        }
-        ByteBuffer out = ByteBuffer.allocate(totalLength);
-        for (ByteBuffer bb : serialized)
-        {
-            int length = bb.remaining();
-            out.put((byte) ((length >> 8) & 0xFF));
-            out.put((byte) (length & 0xFF));
-            out.put(bb);
-            out.put((byte) 0);
-        }
-        out.flip();
-        return out;
-    }
-
-    public void cleanupOnFailure(String failure, Job job)
-    {
-    }
-
-    public void cleanupOnSuccess(String location, Job job) throws IOException {
-    }
-
-
-    /** Methods to get the column family schema from Cassandra */
-    protected void initSchema(String signature) throws IOException
-    {
-        Properties properties = UDFContext.getUDFContext().getUDFProperties(AbstractCassandraStorage.class);
-
-        // Only get the schema if we haven't already gotten it
-        if (!properties.containsKey(signature))
-        {
-            try
-            {
-                Cassandra.Client client = ConfigHelper.getClientFromInputAddressList(conf);
-                client.set_keyspace(keyspace);
-
-                if (username != null && password != null)
-                {
-                    Map<String, String> credentials = new HashMap<String, String>(2);
-                    credentials.put(IAuthenticator.USERNAME_KEY, username);
-                    credentials.put(IAuthenticator.PASSWORD_KEY, password);
-
-                    try
-                    {
-                        client.login(new AuthenticationRequest(credentials));
-                    }
-                    catch (AuthenticationException e)
-                    {
-                        logger.error("Authentication exception: invalid username and/or password");
-                        throw new IOException(e);
-                    }
-                    catch (AuthorizationException e)
-                    {
-                        throw new AssertionError(e); // never actually throws AuthorizationException.
-                    }
-                }
-
-                // compose the CfDef for the columfamily
-                CfInfo cfInfo = getCfInfo(client);
-
-                if (cfInfo.cfDef != null)
-                {
-                    StringBuilder sb = new StringBuilder();
-                    sb.append(cfInfo.compactCqlTable ? 1 : 0).append(cfInfo.cql3Table ? 1: 0).append(cfdefToString(cfInfo.cfDef));
-                    properties.setProperty(signature, sb.toString());
-                }
-                else
-                    throw new IOException(String.format("Column family '%s' not found in keyspace '%s'",
-                                                             column_family,
-                                                             keyspace));
-            }
-            catch (Exception e)
-            {
-                throw new IOException(e);
-            }
-        }
-    }
-
-    /** convert CfDef to string */
-    protected static String cfdefToString(CfDef cfDef) throws IOException
-    {
-        assert cfDef != null;
-        // this is so awful it's kind of cool!
-        TSerializer serializer = new TSerializer(new TBinaryProtocol.Factory());
-        try
-        {
-            return Hex.bytesToHex(serializer.serialize(cfDef));
-        }
-        catch (TException e)
-        {
-            throw new IOException(e);
-        }
-    }
-
-    /** convert string back to CfDef */
-    protected static CfDef cfdefFromString(String st) throws IOException
-    {
-        assert st != null;
-        TDeserializer deserializer = new TDeserializer(new TBinaryProtocol.Factory());
-        CfDef cfDef = new CfDef();
-        try
-        {
-            deserializer.deserialize(cfDef, Hex.hexToBytes(st));
-        }
-        catch (TException e)
-        {
-            throw new IOException(e);
-        }
-        return cfDef;
-    }
-
-    /** return the CfInfo for the column family */
-    protected CfInfo getCfInfo(Cassandra.Client client)
-            throws InvalidRequestException,
-                   UnavailableException,
-                   TimedOutException,
-                   SchemaDisagreementException,
-                   TException,
-                   NotFoundException,
-                   org.apache.cassandra.exceptions.InvalidRequestException,
-                   ConfigurationException,
-                   IOException
-    {
-        // get CF meta data
-        String query = "SELECT type," +
-                       "       comparator," +
-                       "       subcomparator," +
-                       "       default_validator," +
-                       "       key_validator," +
-                       "       key_aliases " +
-                       "FROM system.schema_columnfamilies " +
-                       "WHERE keyspace_name = '%s' " +
-                       "  AND columnfamily_name = '%s' ";
-
-        CqlResult result = client.execute_cql3_query(
-                                ByteBufferUtil.bytes(String.format(query, keyspace, column_family)),
-                                Compression.NONE,
-                                ConsistencyLevel.ONE);
-
-        if (result == null || result.rows == null || result.rows.isEmpty())
-            return null;
-
-        Iterator<CqlRow> iteraRow = result.rows.iterator();
-        CfDef cfDef = new CfDef();
-        cfDef.keyspace = keyspace;
-        cfDef.name = column_family;
-        boolean cql3Table = false;
-        if (iteraRow.hasNext())
-        {
-            CqlRow cqlRow = iteraRow.next();
-
-            cfDef.column_type = ByteBufferUtil.string(cqlRow.columns.get(0).value);
-            cfDef.comparator_type = ByteBufferUtil.string(cqlRow.columns.get(1).value);
-            ByteBuffer subComparator = cqlRow.columns.get(2).value;
-            if (subComparator != null)
-                cfDef.subcomparator_type = ByteBufferUtil.string(subComparator);
-            cfDef.default_validation_class = ByteBufferUtil.string(cqlRow.columns.get(3).value);
-            cfDef.key_validation_class = ByteBufferUtil.string(cqlRow.columns.get(4).value);
-            String keyAliases = ByteBufferUtil.string(cqlRow.columns.get(5).value);
-            if (FBUtilities.fromJsonList(keyAliases).size() > 0)
-                cql3Table = true;
-        }
-        cfDef.column_metadata = getColumnMetadata(client);
-        CfInfo cfInfo = new CfInfo();
-        cfInfo.cfDef = cfDef;
-        if (cql3Table && !(parseType(cfDef.comparator_type) instanceof AbstractCompositeType))
-            cfInfo.compactCqlTable = true;
-        if (cql3Table)
-            cfInfo.cql3Table = true;; 
-        return cfInfo;
-    }
-
-    /** get a list of columns */
-    protected abstract List<ColumnDef> getColumnMetadata(Cassandra.Client client)
-            throws InvalidRequestException,
-            UnavailableException,
-            TimedOutException,
-            SchemaDisagreementException,
-            TException,
-            CharacterCodingException,
-            org.apache.cassandra.exceptions.InvalidRequestException,
-            ConfigurationException,
-            NotFoundException;
-
-    /** get column meta data */
-    protected List<ColumnDef> getColumnMeta(Cassandra.Client client, boolean cassandraStorage, boolean includeCompactValueColumn)
-            throws InvalidRequestException,
-            UnavailableException,
-            TimedOutException,
-            SchemaDisagreementException,
-            TException,
-            CharacterCodingException,
-            org.apache.cassandra.exceptions.InvalidRequestException,
-            ConfigurationException,
-            NotFoundException
-    {
-        String query = "SELECT column_name, " +
-                       "       validator, " +
-                       "       index_type, " +
-                       "       type " +
-                       "FROM system.schema_columns " +
-                       "WHERE keyspace_name = '%s' " +
-                       "  AND columnfamily_name = '%s'";
-
-        CqlResult result = client.execute_cql3_query(
-                                   ByteBufferUtil.bytes(String.format(query, keyspace, column_family)),
-                                   Compression.NONE,
-                                   ConsistencyLevel.ONE);
-
-        List<CqlRow> rows = result.rows;
-        List<ColumnDef> columnDefs = new ArrayList<ColumnDef>();
-        if (rows == null || rows.isEmpty())
-        {
-            // if CassandraStorage, just return the empty list
-            if (cassandraStorage)
-                return columnDefs;
-
-            // otherwise for CqlNativeStorage, check metadata for classic thrift tables
-            CFMetaData cfm = getCFMetaData(keyspace, column_family, client);
-            for (ColumnDefinition def : cfm.regularAndStaticColumns())
-            {
-                ColumnDef cDef = new ColumnDef();
-                String columnName = def.name.toString();
-                String type = def.type.toString();
-                logger.debug("name: {}, type: {} ", columnName, type);
-                cDef.name = ByteBufferUtil.bytes(columnName);
-                cDef.validation_class = type;
-                columnDefs.add(cDef);
-            }
-            // we may not need to include the value column for compact tables as we 
-            // could have already processed it as schema_columnfamilies.value_alias
-            if (columnDefs.size() == 0 && includeCompactValueColumn && cfm.compactValueColumn() != null)
-            {
-                ColumnDefinition def = cfm.compactValueColumn();
-                if ("value".equals(def.name.toString()))
-                {
-                    ColumnDef cDef = new ColumnDef();
-                    cDef.name = def.name.bytes;
-                    cDef.validation_class = def.type.toString();
-                    columnDefs.add(cDef);
-                }
-            }
-            return columnDefs;
-        }
-
-        Iterator<CqlRow> iterator = rows.iterator();
-        while (iterator.hasNext())
-        {
-            CqlRow row = iterator.next();
-            ColumnDef cDef = new ColumnDef();
-            String type = ByteBufferUtil.string(row.getColumns().get(3).value);
-            if (!type.equals("regular"))
-                continue;
-            cDef.setName(ByteBufferUtil.clone(row.getColumns().get(0).value));
-            cDef.validation_class = ByteBufferUtil.string(row.getColumns().get(1).value);
-            ByteBuffer indexType = row.getColumns().get(2).value;
-            if (indexType != null)
-                cDef.index_type = getIndexType(ByteBufferUtil.string(indexType));
-            columnDefs.add(cDef);
-        }
-        return columnDefs;
-    }
-
-    /** get index type from string */
-    protected IndexType getIndexType(String type)
-    {
-        type = type.toLowerCase();
-        if ("keys".equals(type))
-            return IndexType.KEYS;
-        else if("custom".equals(type))
-            return IndexType.CUSTOM;
-        else if("composites".equals(type))
-            return IndexType.COMPOSITES;
-        else
-            return null;
-    }
-
-    /** return partition keys */
-    public String[] getPartitionKeys(String location, Job job) throws IOException
-    {
-        if (!usePartitionFilter)
-            return null;
-        List<ColumnDef> indexes = getIndexes();
-        String[] partitionKeys = new String[indexes.size()];
-        for (int i = 0; i < indexes.size(); i++)
-        {
-            partitionKeys[i] = new String(indexes.get(i).getName());
-        }
-        return partitionKeys;
-    }
-
-    /** get a list of columns with defined index*/
-    protected List<ColumnDef> getIndexes() throws IOException
-    {
-        CfDef cfdef = getCfInfo(loadSignature).cfDef;
-        List<ColumnDef> indexes = new ArrayList<ColumnDef>();
-        for (ColumnDef cdef : cfdef.column_metadata)
-        {
-            if (cdef.index_type != null)
-                indexes.add(cdef);
-        }
-        return indexes;
-    }
-
-    /** get CFMetaData of a column family */
-    protected CFMetaData getCFMetaData(String ks, String cf, Cassandra.Client client)
-            throws NotFoundException,
-            InvalidRequestException,
-            TException,
-            org.apache.cassandra.exceptions.InvalidRequestException,
-            ConfigurationException
-    {
-        KsDef ksDef = client.describe_keyspace(ks);
-        for (CfDef cfDef : ksDef.cf_defs)
-        {
-            if (cfDef.name.equalsIgnoreCase(cf))
-                return CFMetaData.fromThrift(cfDef);
-        }
-        return null;
-    }
-
-    protected Object cassandraToObj(AbstractType validator, ByteBuffer value)
-    {
-        if (validator instanceof DecimalType || validator instanceof InetAddressType)
-            return validator.getString(value);
-
-        if (validator instanceof CollectionType)
-        {
-            // For CollectionType, the compose() method assumes the v3 protocol format of collection, which
-            // is not correct here since we query using the CQL-over-thrift interface which use the pre-v3 format
-            return ((CollectionSerializer)validator.getSerializer()).deserializeForNativeProtocol(value, nativeProtocolVersion);
-        }
-
-        return validator.compose(value);
-    }
-
-    protected static class CfInfo
-    {
-        boolean compactCqlTable = false;
-        boolean cql3Table = false;
-        CfDef cfDef;
-    }
-}
-
diff --git a/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java b/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
index 1e6802b..71fe037 100644
--- a/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
+++ b/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
@@ -18,30 +18,46 @@
 package org.apache.cassandra.hadoop.pig;
 
 import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
 import java.nio.ByteBuffer;
 import java.nio.charset.CharacterCodingException;
 import java.util.*;
 
-import org.apache.cassandra.hadoop.HadoopCompat;
-import org.apache.cassandra.db.Cell;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.auth.PasswordAuthenticator;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.hadoop.*;
+import org.apache.cassandra.exceptions.SyntaxException;
+import org.apache.cassandra.hadoop.ColumnFamilyRecordReader;
+import org.apache.cassandra.hadoop.ConfigHelper;
+import org.apache.cassandra.hadoop.HadoopCompat;
+import org.apache.cassandra.schema.LegacySchemaTables;
+import org.apache.cassandra.serializers.CollectionSerializer;
 import org.apache.cassandra.thrift.*;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Hex;
+import org.apache.cassandra.utils.UUIDGen;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.*;
 import org.apache.pig.Expression;
+import org.apache.pig.LoadFunc;
+import org.apache.pig.LoadMetadata;
 import org.apache.pig.ResourceSchema;
-import org.apache.pig.ResourceSchema.ResourceFieldSchema;
+import org.apache.pig.ResourceStatistics;
+import org.apache.pig.StoreFuncInterface;
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigSplit;
 import org.apache.pig.data.*;
 import org.apache.pig.impl.util.UDFContext;
+import org.apache.pig.ResourceSchema.ResourceFieldSchema;
 import org.apache.thrift.TDeserializer;
 import org.apache.thrift.TException;
 import org.apache.thrift.TSerializer;
@@ -52,7 +68,8 @@
  *
  * A row from a standard CF will be returned as nested tuples: (key, ((name1, val1), (name2, val2))).
  */
-public class CassandraStorage extends AbstractCassandraStorage
+@Deprecated
+public class CassandraStorage extends LoadFunc implements StoreFuncInterface, LoadMetadata
 {
     public final static String PIG_ALLOW_DELETES = "PIG_ALLOW_DELETES";
     public final static String PIG_WIDEROW_INPUT = "PIG_WIDEROW_INPUT";
@@ -66,15 +83,37 @@
     private boolean slice_reverse = false;
     private boolean allow_deletes = false;
 
-    private RecordReader<ByteBuffer, Map<ByteBuffer, Cell>> reader;
+    private RecordReader<ByteBuffer, Map<ByteBuffer, ColumnFamilyRecordReader.Column>> reader;
     private RecordWriter<ByteBuffer, List<Mutation>> writer;
 
     private boolean widerows = false;
     private int limit;
+
+    protected String DEFAULT_INPUT_FORMAT;
+    protected String DEFAULT_OUTPUT_FORMAT;
+
+    protected enum MarshallerType { COMPARATOR, DEFAULT_VALIDATOR, KEY_VALIDATOR, SUBCOMPARATOR };
+
+    protected String username;
+    protected String password;
+    protected String keyspace;
+    protected String column_family;
+    protected String loadSignature;
+    protected String storeSignature;
+
+    protected Configuration conf;
+    protected String inputFormatClass;
+    protected String outputFormatClass;
+    protected int splitSize = 64 * 1024;
+    protected String partitionerClass;
+    protected boolean usePartitionFilter = false;
+    protected String initHostAddress;
+    protected String rpcPort;
+    protected int nativeProtocolVersion = 1;
     
     // wide row hacks
     private ByteBuffer lastKey;
-    private Map<ByteBuffer, Cell> lastRow;
+    private Map<ByteBuffer, ColumnFamilyRecordReader.Column> lastRow;
     private boolean hasNext = true;
 
     public CassandraStorage()
@@ -104,8 +143,7 @@
     /** read wide row*/
     public Tuple getNextWide() throws IOException
     {
-        CfInfo cfInfo = getCfInfo(loadSignature);
-        CfDef cfDef = cfInfo.cfDef;
+        CfDef cfDef = getCfDef(loadSignature);
         ByteBuffer key = null;
         Tuple tuple = null; 
         DefaultDataBag bag = new DefaultDataBag();
@@ -123,12 +161,12 @@
                     {
                         if (tuple.size() == 0) // lastRow is a new one
                         {
-                            key = (ByteBuffer)reader.getCurrentKey();
+                            key = reader.getCurrentKey();
                             tuple = keyToTuple(key, cfDef, parseType(cfDef.getKey_validation_class()));
                         }
-                        for (Map.Entry<ByteBuffer, Cell> entry : lastRow.entrySet())
+                        for (Map.Entry<ByteBuffer, ColumnFamilyRecordReader.Column> entry : lastRow.entrySet())
                         {
-                            bag.add(columnToTuple(entry.getValue(), cfInfo, parseType(cfDef.getComparator_type())));
+                            bag.add(columnToTuple(entry.getValue(), cfDef, parseType(cfDef.getComparator_type())));
                         }
                         lastKey = null;
                         lastRow = null;
@@ -146,31 +184,31 @@
                             return null;
                     }
                 }
-                if (key != null && !((ByteBuffer)reader.getCurrentKey()).equals(key)) // key changed
+                if (key != null && !(reader.getCurrentKey()).equals(key)) // key changed
                 {
                     // read too much, hold on to it for next time
-                    lastKey = (ByteBuffer)reader.getCurrentKey();
-                    lastRow = (SortedMap<ByteBuffer, Cell>)reader.getCurrentValue();
+                    lastKey = reader.getCurrentKey();
+                    lastRow = reader.getCurrentValue();
                     // but return what we have so far
                     tuple.append(bag);
                     return tuple;
                 }
                 if (key == null) // only set the key on the first iteration
                 {
-                    key = (ByteBuffer)reader.getCurrentKey();
+                    key = reader.getCurrentKey();
                     if (lastKey != null && !(key.equals(lastKey))) // last key only had one value
                     {
                         if (tuple == null)
                             tuple = keyToTuple(lastKey, cfDef, parseType(cfDef.getKey_validation_class()));
                         else
                             addKeyToTuple(tuple, lastKey, cfDef, parseType(cfDef.getKey_validation_class()));
-                        for (Map.Entry<ByteBuffer, Cell> entry : lastRow.entrySet())
+                        for (Map.Entry<ByteBuffer, ColumnFamilyRecordReader.Column> entry : lastRow.entrySet())
                         {
-                            bag.add(columnToTuple(entry.getValue(), cfInfo, parseType(cfDef.getComparator_type())));
+                            bag.add(columnToTuple(entry.getValue(), cfDef, parseType(cfDef.getComparator_type())));
                         }
                         tuple.append(bag);
                         lastKey = key;
-                        lastRow = (SortedMap<ByteBuffer, Cell>)reader.getCurrentValue();
+                        lastRow = reader.getCurrentValue();
                         return tuple;
                     }
                     if (tuple == null)
@@ -178,19 +216,20 @@
                     else
                         addKeyToTuple(tuple, lastKey, cfDef, parseType(cfDef.getKey_validation_class()));
                 }
-                SortedMap<ByteBuffer, Cell> row = (SortedMap<ByteBuffer, Cell>)reader.getCurrentValue();
+                SortedMap<ByteBuffer, ColumnFamilyRecordReader.Column> row =
+                    (SortedMap<ByteBuffer, ColumnFamilyRecordReader.Column>)reader.getCurrentValue();
                 if (lastRow != null) // prepend what was read last time
                 {
-                    for (Map.Entry<ByteBuffer, Cell> entry : lastRow.entrySet())
+                    for (Map.Entry<ByteBuffer, ColumnFamilyRecordReader.Column> entry : lastRow.entrySet())
                     {
-                        bag.add(columnToTuple(entry.getValue(), cfInfo, parseType(cfDef.getComparator_type())));
+                        bag.add(columnToTuple(entry.getValue(), cfDef, parseType(cfDef.getComparator_type())));
                     }
                     lastKey = null;
                     lastRow = null;
                 }
-                for (Map.Entry<ByteBuffer, Cell> entry : row.entrySet())
+                for (Map.Entry<ByteBuffer, ColumnFamilyRecordReader.Column> entry : row.entrySet())
                 {
-                    bag.add(columnToTuple(entry.getValue(), cfInfo, parseType(cfDef.getComparator_type())));
+                    bag.add(columnToTuple(entry.getValue(), cfDef, parseType(cfDef.getComparator_type())));
                 }
             }
         }
@@ -200,7 +239,6 @@
         }
     }
 
-    @Override
     /** read next row */
     public Tuple getNext() throws IOException
     {
@@ -212,10 +250,9 @@
             if (!reader.nextKeyValue())
                 return null;
 
-            CfInfo cfInfo = getCfInfo(loadSignature);
-            CfDef cfDef = cfInfo.cfDef;
+            CfDef cfDef = getCfDef(loadSignature);
             ByteBuffer key = reader.getCurrentKey();
-            Map<ByteBuffer, Cell> cf = reader.getCurrentValue();
+            Map<ByteBuffer, ColumnFamilyRecordReader.Column> cf = reader.getCurrentValue();
             assert key != null && cf != null;
 
             // output tuple, will hold the key, each indexed column in a tuple, then a bag of the rest
@@ -224,7 +261,7 @@
             Tuple tuple = keyToTuple(key, cfDef, parseType(cfDef.getKey_validation_class()));
             DefaultDataBag bag = new DefaultDataBag();
             // we must add all the indexed columns first to match the schema
-            Map<ByteBuffer, Boolean> added = new HashMap<ByteBuffer, Boolean>();
+            Map<ByteBuffer, Boolean> added = new HashMap<ByteBuffer, Boolean>(cfDef.column_metadata.size());
             // take care to iterate these in the same order as the schema does
             for (ColumnDef cdef : cfDef.column_metadata)
             {
@@ -240,7 +277,7 @@
                 }
                 if (hasColumn)
                 {
-                    tuple.append(columnToTuple(cf.get(cdef.name), cfInfo, parseType(cfDef.getComparator_type())));
+                    tuple.append(columnToTuple(cf.get(cdef.name), cfDef, parseType(cfDef.getComparator_type())));
                 }
                 else if (!cql3Table)
                 {   // otherwise, we need to add an empty tuple to take its place
@@ -249,10 +286,10 @@
                 added.put(cdef.name, true);
             }
             // now add all the other columns
-            for (Map.Entry<ByteBuffer, Cell> entry : cf.entrySet())
+            for (Map.Entry<ByteBuffer, ColumnFamilyRecordReader.Column> entry : cf.entrySet())
             {
                 if (!added.containsKey(entry.getKey()))
-                    bag.add(columnToTuple(entry.getValue(), cfInfo, parseType(cfDef.getComparator_type())));
+                    bag.add(columnToTuple(entry.getValue(), cfDef, parseType(cfDef.getComparator_type())));
             }
             tuple.append(bag);
             // finally, special top-level indexes if needed
@@ -260,7 +297,7 @@
             {
                 for (ColumnDef cdef : getIndexes())
                 {
-                    Tuple throwaway = columnToTuple(cf.get(cdef.name), cfInfo, parseType(cfDef.getComparator_type()));
+                    Tuple throwaway = columnToTuple(cf.get(cdef.name), cfDef, parseType(cfDef.getComparator_type()));
                     tuple.append(throwaway.get(1));
                 }
             }
@@ -272,14 +309,57 @@
         }
     }
 
+    /** write next row */
+    public void putNext(Tuple t) throws IOException
+    {
+        /*
+        We support two cases for output:
+        First, the original output:
+            (key, (name, value), (name,value), {(name,value)}) (tuples or bag is optional)
+        For supers, we only accept the original output.
+        */
+
+        if (t.size() < 1)
+        {
+            // simply nothing here, we can't even delete without a key
+            logger.warn("Empty output skipped, filter empty tuples to suppress this warning");
+            return;
+        }
+        ByteBuffer key = objToBB(t.get(0));
+        if (t.getType(1) == DataType.TUPLE)
+            writeColumnsFromTuple(key, t, 1);
+        else if (t.getType(1) == DataType.BAG)
+        {
+            if (t.size() > 2)
+                throw new IOException("No arguments allowed after bag");
+            writeColumnsFromBag(key, (DataBag) t.get(1));
+        }
+        else
+            throw new IOException("Second argument in output must be a tuple or bag");
+    }
+
     /** set hadoop cassandra connection settings */
     protected void setConnectionInformation() throws IOException
     {
-        super.setConnectionInformation();
+        StorageHelper.setConnectionInformation(conf);
+        if (System.getenv(StorageHelper.PIG_INPUT_FORMAT) != null)
+            inputFormatClass = getFullyQualifiedClassName(System.getenv(StorageHelper.PIG_INPUT_FORMAT));
+        else
+            inputFormatClass = DEFAULT_INPUT_FORMAT;
+        if (System.getenv(StorageHelper.PIG_OUTPUT_FORMAT) != null)
+            outputFormatClass = getFullyQualifiedClassName(System.getenv(StorageHelper.PIG_OUTPUT_FORMAT));
+        else
+            outputFormatClass = DEFAULT_OUTPUT_FORMAT;
         if (System.getenv(PIG_ALLOW_DELETES) != null)
             allow_deletes = Boolean.parseBoolean(System.getenv(PIG_ALLOW_DELETES));
     }
 
+    /** get the full class name */
+    protected String getFullyQualifiedClassName(String classname)
+    {
+        return classname.contains(".") ? classname : "org.apache.cassandra.hadoop." + classname;
+    }
+
     /** set read configuration settings */
     public void setLocation(String location, Job job) throws IOException
     {
@@ -296,11 +376,11 @@
             widerows = Boolean.parseBoolean(System.getenv(PIG_WIDEROW_INPUT));
         if (System.getenv(PIG_USE_SECONDARY) != null)
             usePartitionFilter = Boolean.parseBoolean(System.getenv(PIG_USE_SECONDARY));
-        if (System.getenv(PIG_INPUT_SPLIT_SIZE) != null)
+        if (System.getenv(StorageHelper.PIG_INPUT_SPLIT_SIZE) != null)
         {
             try
             {
-                ConfigHelper.setInputSplitSize(conf, Integer.parseInt(System.getenv(PIG_INPUT_SPLIT_SIZE)));
+                ConfigHelper.setInputSplitSize(conf, Integer.parseInt(System.getenv(StorageHelper.PIG_INPUT_SPLIT_SIZE)));
             }
             catch (NumberFormatException e)
             {
@@ -380,12 +460,67 @@
         initSchema(storeSignature);
     }
 
+    /** Methods to get the column family schema from Cassandra */
+    protected void initSchema(String signature) throws IOException
+    {
+        Properties properties = UDFContext.getUDFContext().getUDFProperties(CassandraStorage.class);
+
+        // Only get the schema if we haven't already gotten it
+        if (!properties.containsKey(signature))
+        {
+            try
+            {
+                Cassandra.Client client = ConfigHelper.getClientFromInputAddressList(conf);
+                client.set_keyspace(keyspace);
+
+                if (username != null && password != null)
+                {
+                    Map<String, String> credentials = new HashMap<String, String>(2);
+                    credentials.put(PasswordAuthenticator.USERNAME_KEY, username);
+                    credentials.put(PasswordAuthenticator.PASSWORD_KEY, password);
+
+                    try
+                    {
+                        client.login(new AuthenticationRequest(credentials));
+                    }
+                    catch (AuthenticationException e)
+                    {
+                        logger.error("Authentication exception: invalid username and/or password");
+                        throw new IOException(e);
+                    }
+                }
+
+                // compose the CfDef for the columfamily
+                CfDef cfDef = getCfDef(client);
+
+                if (cfDef != null)
+                {
+                    StringBuilder sb = new StringBuilder();
+                    sb.append(cfdefToString(cfDef));
+                    properties.setProperty(signature, sb.toString());
+                }
+                else
+                    throw new IOException(String.format("Table '%s' not found in keyspace '%s'",
+                            column_family,
+                            keyspace));
+            }
+            catch (Exception e)
+            {
+                throw new IOException(e);
+            }
+        }
+    }
+
+    public void checkSchema(ResourceSchema schema) throws IOException
+    {
+        // we don't care about types, they all get casted to ByteBuffers
+    }
+
     /** define the schema */
     public ResourceSchema getSchema(String location, Job job) throws IOException
     {
         setLocation(location, job);
-        CfInfo cfInfo = getCfInfo(loadSignature);
-        CfDef cfDef = cfInfo.cfDef;
+        CfDef cfDef = getCfDef(loadSignature);
         if (cfDef.column_type.equals("Super"))
             return null;
         /*
@@ -405,7 +540,7 @@
         // add key
         ResourceFieldSchema keyFieldSchema = new ResourceFieldSchema();
         keyFieldSchema.setName("key");
-        keyFieldSchema.setType(getPigType(marshallers.get(MarshallerType.KEY_VALIDATOR)));
+        keyFieldSchema.setType(StorageHelper.getPigType(marshallers.get(MarshallerType.KEY_VALIDATOR)));
 
         ResourceSchema bagSchema = new ResourceSchema();
         ResourceFieldSchema bagField = new ResourceFieldSchema();
@@ -419,8 +554,8 @@
         ResourceFieldSchema bagvalSchema = new ResourceFieldSchema();
         bagcolSchema.setName("name");
         bagvalSchema.setName("value");
-        bagcolSchema.setType(getPigType(marshallers.get(MarshallerType.COMPARATOR)));
-        bagvalSchema.setType(getPigType(marshallers.get(MarshallerType.DEFAULT_VALIDATOR)));
+        bagcolSchema.setType(StorageHelper.getPigType(marshallers.get(MarshallerType.COMPARATOR)));
+        bagvalSchema.setType(StorageHelper.getPigType(marshallers.get(MarshallerType.DEFAULT_VALIDATOR)));
         bagTupleSchema.setFields(new ResourceFieldSchema[] { bagcolSchema, bagvalSchema });
         bagTupleField.setSchema(bagTupleSchema);
         bagSchema.setFields(new ResourceFieldSchema[] { bagTupleField });
@@ -431,7 +566,7 @@
         // add the key first, then the indexed columns, and finally the bag
         allSchemaFields.add(keyFieldSchema);
 
-        if (!widerows && (cfInfo.compactCqlTable || !cfInfo.cql3Table))
+        if (!widerows)
         {
             // defined validators/indexes
             for (ColumnDef cdef : cfDef.column_metadata)
@@ -445,14 +580,14 @@
 
                 ResourceFieldSchema idxColSchema = new ResourceFieldSchema();
                 idxColSchema.setName("name");
-                idxColSchema.setType(getPigType(marshallers.get(MarshallerType.COMPARATOR)));
+                idxColSchema.setType(StorageHelper.getPigType(marshallers.get(MarshallerType.COMPARATOR)));
 
                 ResourceFieldSchema valSchema = new ResourceFieldSchema();
                 AbstractType validator = validators.get(cdef.name);
                 if (validator == null)
                     validator = marshallers.get(MarshallerType.DEFAULT_VALIDATOR);
                 valSchema.setName("value");
-                valSchema.setType(getPigType(validator));
+                valSchema.setType(StorageHelper.getPigType(validator));
 
                 innerTupleSchema.setFields(new ResourceFieldSchema[] { idxColSchema, valSchema });
                 allSchemaFields.add(innerTupleField);
@@ -472,7 +607,7 @@
                 AbstractType validator = validators.get(cdef.name);
                 if (validator == null)
                     validator = marshallers.get(MarshallerType.DEFAULT_VALIDATOR);
-                idxSchema.setType(getPigType(validator));
+                idxSchema.setType(StorageHelper.getPigType(validator));
                 allSchemaFields.add(idxSchema);
             }
         }
@@ -485,8 +620,8 @@
     public void setPartitionFilter(Expression partitionFilter) throws IOException
     {
         UDFContext context = UDFContext.getUDFContext();
-        Properties property = context.getUDFProperties(AbstractCassandraStorage.class);
-        property.setProperty(PARTITION_FILTER_SIGNATURE, indexExpressionsToString(filterToIndexExpressions(partitionFilter)));
+        Properties property = context.getUDFProperties(CassandraStorage.class);
+        property.setProperty(StorageHelper.PARTITION_FILTER_SIGNATURE, indexExpressionsToString(filterToIndexExpressions(partitionFilter)));
     }
 
     /** prepare writer */
@@ -495,33 +630,93 @@
         this.writer = writer;
     }
 
-    /** write next row */
-    public void putNext(Tuple t) throws IOException
+    /** convert object to ByteBuffer */
+    protected ByteBuffer objToBB(Object o)
     {
-        /*
-        We support two cases for output:
-        First, the original output:
-            (key, (name, value), (name,value), {(name,value)}) (tuples or bag is optional)
-        For supers, we only accept the original output.
-        */
+        if (o == null)
+            return nullToBB();
+        if (o instanceof java.lang.String)
+            return ByteBuffer.wrap(new DataByteArray((String)o).get());
+        if (o instanceof Integer)
+            return Int32Type.instance.decompose((Integer)o);
+        if (o instanceof Long)
+            return LongType.instance.decompose((Long)o);
+        if (o instanceof Float)
+            return FloatType.instance.decompose((Float)o);
+        if (o instanceof Double)
+            return DoubleType.instance.decompose((Double)o);
+        if (o instanceof UUID)
+            return ByteBuffer.wrap(UUIDGen.decompose((UUID) o));
+        if(o instanceof Tuple) {
+            List<Object> objects = ((Tuple)o).getAll();
+            //collections
+            if (objects.size() > 0 && objects.get(0) instanceof String)
+            {
+                String collectionType = (String) objects.get(0);
+                if ("set".equalsIgnoreCase(collectionType) ||
+                        "list".equalsIgnoreCase(collectionType))
+                    return objToListOrSetBB(objects.subList(1, objects.size()));
+                else if ("map".equalsIgnoreCase(collectionType))
+                    return objToMapBB(objects.subList(1, objects.size()));
 
-        if (t.size() < 1)
-        {
-            // simply nothing here, we can't even delete without a key
-            logger.warn("Empty output skipped, filter empty tuples to suppress this warning");
-            return;
+            }
+            return objToCompositeBB(objects);
         }
-        ByteBuffer key = objToBB(t.get(0));
-        if (t.getType(1) == DataType.TUPLE)
-            writeColumnsFromTuple(key, t, 1);
-        else if (t.getType(1) == DataType.BAG)
+
+        return ByteBuffer.wrap(((DataByteArray) o).get());
+    }
+
+    private ByteBuffer objToListOrSetBB(List<Object> objects)
+    {
+        List<ByteBuffer> serialized = new ArrayList<ByteBuffer>(objects.size());
+        for(Object sub : objects)
         {
-            if (t.size() > 2)
-                throw new IOException("No arguments allowed after bag");
-            writeColumnsFromBag(key, (DataBag) t.get(1));
+            ByteBuffer buffer = objToBB(sub);
+            serialized.add(buffer);
         }
-        else
-            throw new IOException("Second argument in output must be a tuple or bag");
+        // NOTE: using protocol v1 serialization format for collections so as to not break
+        // compatibility. Not sure if that's the right thing.
+        return CollectionSerializer.pack(serialized, objects.size(), 1);
+    }
+
+    private ByteBuffer objToMapBB(List<Object> objects)
+    {
+        List<ByteBuffer> serialized = new ArrayList<ByteBuffer>(objects.size() * 2);
+        for(Object sub : objects)
+        {
+            List<Object> keyValue = ((Tuple)sub).getAll();
+            for (Object entry: keyValue)
+            {
+                ByteBuffer buffer = objToBB(entry);
+                serialized.add(buffer);
+            }
+        }
+        // NOTE: using protocol v1 serialization format for collections so as to not break
+        // compatibility. Not sure if that's the right thing.
+        return CollectionSerializer.pack(serialized, objects.size(), 1);
+    }
+
+    private ByteBuffer objToCompositeBB(List<Object> objects)
+    {
+        List<ByteBuffer> serialized = new ArrayList<ByteBuffer>(objects.size());
+        int totalLength = 0;
+        for(Object sub : objects)
+        {
+            ByteBuffer buffer = objToBB(sub);
+            serialized.add(buffer);
+            totalLength += 2 + buffer.remaining() + 1;
+        }
+        ByteBuffer out = ByteBuffer.allocate(totalLength);
+        for (ByteBuffer bb : serialized)
+        {
+            int length = bb.remaining();
+            out.put((byte) ((length >> 8) & 0xFF));
+            out.put((byte) (length & 0xFF));
+            out.put(bb);
+            out.put((byte) 0);
+        }
+        out.flip();
+        return out;
     }
 
     /** write tuple data to cassandra */
@@ -643,6 +838,19 @@
         }
     }
 
+    /** get a list of columns with defined index*/
+    protected List<ColumnDef> getIndexes() throws IOException
+    {
+        CfDef cfdef = getCfDef(loadSignature);
+        List<ColumnDef> indexes = new ArrayList<ColumnDef>();
+        for (ColumnDef cdef : cfdef.column_metadata)
+        {
+            if (cdef.index_type != null)
+                indexes.add(cdef);
+        }
+        return indexes;
+    }
+
     /** get a list of Cassandra IndexExpression from Pig expression */
     private List<IndexExpression> filterToIndexExpressions(Expression expression) throws IOException
     {
@@ -713,13 +921,64 @@
         return indexClause.getExpressions();
     }
 
+    public ResourceStatistics getStatistics(String location, Job job)
+    {
+        return null;
+    }
+
+    public void cleanupOnFailure(String failure, Job job)
+    {
+    }
+
+    public void cleanupOnSuccess(String location, Job job) throws IOException {
+    }
+
+
+    /** StoreFunc methods */
+    public void setStoreFuncUDFContextSignature(String signature)
+    {
+        this.storeSignature = signature;
+    }
+
+    public String relToAbsPathForStoreLocation(String location, Path curDir) throws IOException
+    {
+        return relativeToAbsolutePath(location, curDir);
+    }
+
+    /** output format */
+    public OutputFormat getOutputFormat() throws IOException
+    {
+        try
+        {
+            return FBUtilities.construct(outputFormatClass, "outputformat");
+        }
+        catch (ConfigurationException e)
+        {
+            throw new IOException(e);
+        }
+    }
+
+
+    @Override
+    public InputFormat getInputFormat() throws IOException
+    {
+        try
+        {
+            return FBUtilities.construct(inputFormatClass, "inputformat");
+        }
+        catch (ConfigurationException e)
+        {
+            throw new IOException(e);
+        }
+    }
+
     /** get a list of index expression */
     private List<IndexExpression> getIndexExpressions() throws IOException
     {
         UDFContext context = UDFContext.getUDFContext();
-        Properties property = context.getUDFProperties(AbstractCassandraStorage.class);
-        if (property.getProperty(PARTITION_FILTER_SIGNATURE) != null)
-            return indexExpressionsFromString(property.getProperty(PARTITION_FILTER_SIGNATURE));
+        Properties property = context.getUDFProperties(CassandraStorage.class);
+        if (property.getProperty(StorageHelper.PARTITION_FILTER_SIGNATURE) != null)
+            return indexExpressionsFromString(property.getProperty(StorageHelper.PARTITION_FILTER_SIGNATURE));
         else
             return null;
     }
@@ -731,6 +990,129 @@
         return getColumnMeta(client, true, true);
     }
 
+
+    /** get column meta data */
+    protected List<ColumnDef> getColumnMeta(Cassandra.Client client, boolean cassandraStorage, boolean includeCompactValueColumn)
+            throws org.apache.cassandra.thrift.InvalidRequestException,
+            UnavailableException,
+            TimedOutException,
+            SchemaDisagreementException,
+            TException,
+            CharacterCodingException,
+            org.apache.cassandra.exceptions.InvalidRequestException,
+            ConfigurationException,
+            NotFoundException
+    {
+        String query = String.format("SELECT column_name, validator, index_type, type " +
+                        "FROM %s.%s " +
+                        "WHERE keyspace_name = '%s' AND columnfamily_name = '%s'",
+                SystemKeyspace.NAME,
+                LegacySchemaTables.COLUMNS,
+                keyspace,
+                column_family);
+
+        CqlResult result = client.execute_cql3_query(ByteBufferUtil.bytes(query), Compression.NONE, ConsistencyLevel.ONE);
+
+        List<CqlRow> rows = result.rows;
+        List<ColumnDef> columnDefs = new ArrayList<ColumnDef>();
+        if (rows == null || rows.isEmpty())
+        {
+            // if CassandraStorage, just return the empty list
+            if (cassandraStorage)
+                return columnDefs;
+
+            // otherwise for CqlNativeStorage, check metadata for classic thrift tables
+            CFMetaData cfm = getCFMetaData(keyspace, column_family, client);
+            for (ColumnDefinition def : cfm.regularAndStaticColumns())
+            {
+                ColumnDef cDef = new ColumnDef();
+                String columnName = def.name.toString();
+                String type = def.type.toString();
+                logger.trace("name: {}, type: {} ", columnName, type);
+                cDef.name = ByteBufferUtil.bytes(columnName);
+                cDef.validation_class = type;
+                columnDefs.add(cDef);
+            }
+            // we may not need to include the value column for compact tables as we
+            // could have already processed it as schema_columnfamilies.value_alias
+            if (columnDefs.size() == 0 && includeCompactValueColumn && cfm.compactValueColumn() != null)
+            {
+                ColumnDefinition def = cfm.compactValueColumn();
+                if ("value".equals(def.name.toString()))
+                {
+                    ColumnDef cDef = new ColumnDef();
+                    cDef.name = def.name.bytes;
+                    cDef.validation_class = def.type.toString();
+                    columnDefs.add(cDef);
+                }
+            }
+            return columnDefs;
+        }
+
+        Iterator<CqlRow> iterator = rows.iterator();
+        while (iterator.hasNext())
+        {
+            CqlRow row = iterator.next();
+            ColumnDef cDef = new ColumnDef();
+            String type = ByteBufferUtil.string(row.getColumns().get(3).value);
+            if (!type.equals("regular"))
+                continue;
+            cDef.setName(ByteBufferUtil.clone(row.getColumns().get(0).value));
+            cDef.validation_class = ByteBufferUtil.string(row.getColumns().get(1).value);
+            ByteBuffer indexType = row.getColumns().get(2).value;
+            if (indexType != null)
+                cDef.index_type = getIndexType(ByteBufferUtil.string(indexType));
+            columnDefs.add(cDef);
+        }
+        return columnDefs;
+    }
+
+
+    /** get CFMetaData of a column family */
+    protected CFMetaData getCFMetaData(String ks, String cf, Cassandra.Client client)
+            throws NotFoundException,
+            org.apache.cassandra.thrift.InvalidRequestException,
+            TException,
+            org.apache.cassandra.exceptions.InvalidRequestException,
+            ConfigurationException
+    {
+        KsDef ksDef = client.describe_keyspace(ks);
+        for (CfDef cfDef : ksDef.cf_defs)
+        {
+            if (cfDef.name.equalsIgnoreCase(cf))
+                return ThriftConversion.fromThrift(cfDef);
+        }
+        return null;
+    }
+
+    /** get index type from string */
+    protected IndexType getIndexType(String type)
+    {
+        type = type.toLowerCase();
+        if ("keys".equals(type))
+            return IndexType.KEYS;
+        else if("custom".equals(type))
+            return IndexType.CUSTOM;
+        else if("composites".equals(type))
+            return IndexType.COMPOSITES;
+        else
+            return null;
+    }
+
+    /** return partition keys */
+    public String[] getPartitionKeys(String location, Job job) throws IOException
+    {
+        if (!usePartitionFilter)
+            return null;
+        List<ColumnDef> indexes = getIndexes();
+        String[] partitionKeys = new String[indexes.size()];
+        for (int i = 0; i < indexes.size(); i++)
+        {
+            partitionKeys[i] = new String(indexes.get(i).getName());
+        }
+        return partitionKeys;
+    }
+
     /** convert key to a tuple */
     private Tuple keyToTuple(ByteBuffer key, CfDef cfDef, AbstractType comparator) throws IOException
     {
@@ -744,15 +1126,26 @@
     {
         if( comparator instanceof AbstractCompositeType )
         {
-            setTupleValue(tuple, 0, composeComposite((AbstractCompositeType)comparator,key));
+            StorageHelper.setTupleValue(tuple, 0, composeComposite((AbstractCompositeType) comparator, key));
         }
         else
         {
-            setTupleValue(tuple, 0, cassandraToObj(getDefaultMarshallers(cfDef).get(MarshallerType.KEY_VALIDATOR), key));
+            StorageHelper.setTupleValue(tuple, 0, StorageHelper.cassandraToObj(getDefaultMarshallers(cfDef).get(MarshallerType.KEY_VALIDATOR), key, nativeProtocolVersion));
         }
 
     }
 
+    /** Deconstructs a composite type to a Tuple. */
+    protected Tuple composeComposite(AbstractCompositeType comparator, ByteBuffer name) throws IOException
+    {
+        List<AbstractCompositeType.CompositeComponent> result = comparator.deconstruct(name);
+        Tuple t = TupleFactory.getInstance().newTuple(result.size());
+        for (int i=0; i<result.size(); i++)
+            StorageHelper.setTupleValue(t, i, StorageHelper.cassandraToObj(result.get(i).comparator, result.get(i).value, nativeProtocolVersion));
+
+        return t;
+    }
+
     /** cassandra://[username:password@]<keyspace>/<columnfamily>[?slice_start=<start>&slice_end=<end>
      * [&reversed=true][&limit=1][&allow_deletes=true][&widerows=true]
      * [&use_secondary=true][&comparator=<comparator>][&partitioner=<partitioner>]]*/
@@ -810,17 +1203,211 @@
         }
         catch (Exception e)
         {
-            throw new IOException("Expected 'cassandra://[username:password@]<keyspace>/<columnfamily>" +
+            throw new IOException("Expected 'cassandra://[username:password@]<keyspace>/<table>" +
                     "[?slice_start=<start>&slice_end=<end>[&reversed=true][&limit=1]" +
                     "[&allow_deletes=true][&widerows=true][&use_secondary=true]" +
                     "[&comparator=<comparator>][&split_size=<size>][&partitioner=<partitioner>]" +
                     "[&init_address=<host>][&rpc_port=<port>]]': " + e.getMessage());
         }
     }
-    
+
+
+    /** decompose the query to store the parameters in a map */
+    public static Map<String, String> getQueryMap(String query) throws UnsupportedEncodingException
+    {
+        String[] params = query.split("&");
+        Map<String, String> map = new HashMap<String, String>(params.length);
+        for (String param : params)
+        {
+            String[] keyValue = param.split("=");
+            map.put(keyValue[0], URLDecoder.decode(keyValue[1], "UTF-8"));
+        }
+        return map;
+    }
+
     public ByteBuffer nullToBB()
     {
-        return (ByteBuffer) null;
+        return null;
+    }
+
+    /** return the CfInfo for the column family */
+    protected CfDef getCfDef(Cassandra.Client client)
+            throws org.apache.cassandra.thrift.InvalidRequestException,
+            UnavailableException,
+            TimedOutException,
+            SchemaDisagreementException,
+            TException,
+            NotFoundException,
+            org.apache.cassandra.exceptions.InvalidRequestException,
+            ConfigurationException,
+            IOException
+    {
+        // get CF meta data
+        String query = String.format("SELECT type, comparator, subcomparator, default_validator, key_validator " +
+                        "FROM %s.%s " +
+                        "WHERE keyspace_name = '%s' AND columnfamily_name = '%s'",
+                SystemKeyspace.NAME,
+                LegacySchemaTables.COLUMNFAMILIES,
+                keyspace,
+                column_family);
+
+        CqlResult result = client.execute_cql3_query(ByteBufferUtil.bytes(query), Compression.NONE, ConsistencyLevel.ONE);
+
+        if (result == null || result.rows == null || result.rows.isEmpty())
+            return null;
+
+        Iterator<CqlRow> iteraRow = result.rows.iterator();
+        CfDef cfDef = new CfDef();
+        cfDef.keyspace = keyspace;
+        cfDef.name = column_family;
+        if (iteraRow.hasNext())
+        {
+            CqlRow cqlRow = iteraRow.next();
+
+            cfDef.column_type = ByteBufferUtil.string(cqlRow.columns.get(0).value);
+            cfDef.comparator_type = ByteBufferUtil.string(cqlRow.columns.get(1).value);
+            ByteBuffer subComparator = cqlRow.columns.get(2).value;
+            if (subComparator != null)
+                cfDef.subcomparator_type = ByteBufferUtil.string(subComparator);
+            cfDef.default_validation_class = ByteBufferUtil.string(cqlRow.columns.get(3).value);
+            cfDef.key_validation_class = ByteBufferUtil.string(cqlRow.columns.get(4).value);
+        }
+        cfDef.column_metadata = getColumnMetadata(client);
+        return cfDef;
+    }
+
+    /** get the columnfamily definition for the signature */
+    protected CfDef getCfDef(String signature) throws IOException
+    {
+        UDFContext context = UDFContext.getUDFContext();
+        Properties property = context.getUDFProperties(CassandraStorage.class);
+        String prop = property.getProperty(signature);
+        return cfdefFromString(prop);
+    }
+
+    /** convert string back to CfDef */
+    protected static CfDef cfdefFromString(String st) throws IOException
+    {
+        assert st != null;
+        TDeserializer deserializer = new TDeserializer(new TBinaryProtocol.Factory());
+        CfDef cfDef = new CfDef();
+        try
+        {
+            deserializer.deserialize(cfDef, Hex.hexToBytes(st));
+        }
+        catch (TException e)
+        {
+            throw new IOException(e);
+        }
+        return cfDef;
+    }
+
+    /** convert CfDef to string */
+    protected static String cfdefToString(CfDef cfDef) throws IOException
+    {
+        assert cfDef != null;
+        // this is so awful it's kind of cool!
+        TSerializer serializer = new TSerializer(new TBinaryProtocol.Factory());
+        try
+        {
+            return Hex.bytesToHex(serializer.serialize(cfDef));
+        }
+        catch (TException e)
+        {
+            throw new IOException(e);
+        }
+    }
+
+    /** parse the string to a cassandra data type */
+    protected AbstractType parseType(String type) throws IOException
+    {
+        try
+        {
+            // always treat counters like longs, specifically CCT.compose is not what we need
+            if (type != null && type.equals("org.apache.cassandra.db.marshal.CounterColumnType"))
+                return LongType.instance;
+            return TypeParser.parse(type);
+        }
+        catch (ConfigurationException e)
+        {
+            throw new IOException(e);
+        }
+        catch (SyntaxException e)
+        {
+            throw new IOException(e);
+        }
+    }
+
+    /** convert a column to a tuple */
+    protected Tuple columnToTuple(ColumnFamilyRecordReader.Column column, CfDef cfDef, AbstractType comparator) throws IOException
+    {
+        Tuple pair = TupleFactory.getInstance().newTuple(2);
+
+        // name
+        if(comparator instanceof AbstractCompositeType)
+            StorageHelper.setTupleValue(pair, 0, composeComposite((AbstractCompositeType) comparator, column.name));
+        else
+            StorageHelper.setTupleValue(pair, 0, StorageHelper.cassandraToObj(comparator, column.name, nativeProtocolVersion));
+
+        // value
+        Map<ByteBuffer,AbstractType> validators = getValidatorMap(cfDef);
+        if (validators.get(column.name) == null)
+        {
+            Map<MarshallerType, AbstractType> marshallers = getDefaultMarshallers(cfDef);
+            StorageHelper.setTupleValue(pair, 1, StorageHelper.cassandraToObj(marshallers.get(MarshallerType.DEFAULT_VALIDATOR), column.value, nativeProtocolVersion));
+        }
+        else
+            StorageHelper.setTupleValue(pair, 1, StorageHelper.cassandraToObj(validators.get(column.name), column.value, nativeProtocolVersion));
+        return pair;
+    }
+
+    /** construct a map to store the mashaller type to cassandra data type mapping */
+    protected Map<MarshallerType, AbstractType> getDefaultMarshallers(CfDef cfDef) throws IOException
+    {
+        Map<MarshallerType, AbstractType> marshallers = new EnumMap<MarshallerType, AbstractType>(MarshallerType.class);
+        AbstractType comparator;
+        AbstractType subcomparator;
+        AbstractType default_validator;
+        AbstractType key_validator;
+
+        comparator = parseType(cfDef.getComparator_type());
+        subcomparator = parseType(cfDef.getSubcomparator_type());
+        default_validator = parseType(cfDef.getDefault_validation_class());
+        key_validator = parseType(cfDef.getKey_validation_class());
+
+        marshallers.put(MarshallerType.COMPARATOR, comparator);
+        marshallers.put(MarshallerType.DEFAULT_VALIDATOR, default_validator);
+        marshallers.put(MarshallerType.KEY_VALIDATOR, key_validator);
+        marshallers.put(MarshallerType.SUBCOMPARATOR, subcomparator);
+        return marshallers;
+    }
+
+    /** get the validators */
+    protected Map<ByteBuffer, AbstractType> getValidatorMap(CfDef cfDef) throws IOException
+    {
+        Map<ByteBuffer, AbstractType> validators = new HashMap<ByteBuffer, AbstractType>();
+        for (ColumnDef cd : cfDef.getColumn_metadata())
+        {
+            if (cd.getValidation_class() != null && !cd.getValidation_class().isEmpty())
+            {
+                AbstractType validator = null;
+                try
+                {
+                    validator = TypeParser.parse(cd.getValidation_class());
+                    if (validator instanceof CounterColumnType)
+                        validator = LongType.instance;
+                    validators.put(cd.name, validator);
+                }
+                catch (ConfigurationException e)
+                {
+                    throw new IOException(e);
+                }
+                catch (SyntaxException e)
+                {
+                    throw new IOException(e);
+                }
+            }
+        }
+        return validators;
     }
 }
-
diff --git a/src/java/org/apache/cassandra/hadoop/pig/CqlNativeStorage.java b/src/java/org/apache/cassandra/hadoop/pig/CqlNativeStorage.java
index 5287bf5..8831cf2 100644
--- a/src/java/org/apache/cassandra/hadoop/pig/CqlNativeStorage.java
+++ b/src/java/org/apache/cassandra/hadoop/pig/CqlNativeStorage.java
@@ -17,59 +17,86 @@
  */
 package org.apache.cassandra.hadoop.pig;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
 import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
 import java.nio.ByteBuffer;
-import java.nio.charset.CharacterCodingException;
 import java.util.*;
 
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions.InternodeEncryption;
-import org.apache.cassandra.db.BufferCell;
-import org.apache.cassandra.db.Cell;
-import org.apache.cassandra.db.composites.CellNames;
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnMetadata;
+import com.datastax.driver.core.Metadata;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.TableMetadata;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+
 import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.exceptions.AuthenticationException;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.hadoop.AbstractBulkRecordWriter;
+import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.hadoop.ConfigHelper;
 import org.apache.cassandra.hadoop.HadoopCompat;
 import org.apache.cassandra.hadoop.cql3.CqlBulkOutputFormat;
+import org.apache.cassandra.hadoop.cql3.CqlBulkRecordWriter;
 import org.apache.cassandra.hadoop.cql3.CqlConfigHelper;
 import org.apache.cassandra.hadoop.cql3.CqlRecordReader;
-import org.apache.cassandra.thrift.*;
+import org.apache.cassandra.serializers.CollectionSerializer;
 import org.apache.cassandra.utils.*;
-import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.*;
-import org.apache.pig.Expression;
-import org.apache.pig.ResourceSchema;
+import org.apache.pig.*;
 import org.apache.pig.Expression.OpType;
 import org.apache.pig.ResourceSchema.ResourceFieldSchema;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigSplit;
 import org.apache.pig.data.*;
 import org.apache.pig.impl.util.UDFContext;
-import org.apache.thrift.TException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.yaml.snakeyaml.external.biz.base64Coder.Base64Coder;
 
-import com.datastax.driver.core.Row;
-
-public class CqlNativeStorage extends AbstractCassandraStorage
+@Deprecated
+public class CqlNativeStorage extends LoadFunc implements StoreFuncInterface, LoadMetadata
 {
+    protected String DEFAULT_INPUT_FORMAT;
+    protected String DEFAULT_OUTPUT_FORMAT;
+
+    protected String username;
+    protected String password;
+    protected String keyspace;
+    protected String column_family;
+    protected String loadSignature;
+    protected String storeSignature;
+
+    protected Configuration conf;
+    protected String inputFormatClass;
+    protected String outputFormatClass;
+    protected int splitSize = 64 * 1024;
+    protected String partitionerClass;
+    protected boolean usePartitionFilter = false;
+    protected String initHostAddress;
+    protected String rpcPort;
+    protected int nativeProtocolVersion = 1;
+
     private static final Logger logger = LoggerFactory.getLogger(CqlNativeStorage.class);
-    public static String BULK_OUTPUT_FORMAT = "org.apache.cassandra.hadoop.cql3.CqlBulkOutputFormat";
+    private static String BULK_OUTPUT_FORMAT = "org.apache.cassandra.hadoop.cql3.CqlBulkOutputFormat";
     private int pageSize = 1000;
     private String columns;
     private String outputQuery;
     private String whereClause;
-    private boolean hasCompactValueAlias = false;
 
     private RecordReader<Long, Row> reader;
     private RecordWriter<Map<String, ByteBuffer>, List<ByteBuffer>> writer;
     private String nativePort;
     private String nativeCoreConnections;
     private String nativeMaxConnections;
-    private String nativeMinSimultReqs;
     private String nativeMaxSimultReqs;
     private String nativeConnectionTimeout;
     private String nativeReadConnectionTimeout;
@@ -94,14 +121,8 @@
     private int bulkBuffSize = -1;
     private int bulkStreamThrottle = -1;
     private int bulkMaxFailedHosts = -1;
-    private int storagePort = CqlBulkOutputFormat.DEFAULT_STORAGE_PORT;
-    private int sslStoragePort = CqlBulkOutputFormat.DEFAULT_SSL_STORAGE_PORT;
-    private String serverKeystore;
-    private String serverKeystorePassword;
-    private String serverTruststore;
-    private String serverTruststorePassword;
-    private String serverCipherSuites;
-    private String internodeEncrypt;
+    private boolean bulkDeleteSourceOnSuccess = true;
+    private String bulkTableAlias;
 
     public CqlNativeStorage()
     {
@@ -139,21 +160,20 @@
             if (!reader.nextKeyValue())
                 return null;
 
-            CfInfo cfInfo = getCfInfo(loadSignature);
-            CfDef cfDef = cfInfo.cfDef;
+            TableInfo tableMetadata = getCfInfo(loadSignature);
             Row row = reader.getCurrentValue();
-            Tuple tuple = TupleFactory.getInstance().newTuple(cfDef.column_metadata.size());
-            Iterator<ColumnDef> itera = cfDef.column_metadata.iterator();
+            Tuple tuple = TupleFactory.getInstance().newTuple(tableMetadata.getColumns().size());
+            Iterator<ColumnInfo> itera = tableMetadata.getColumns().iterator();
             int i = 0;
             while (itera.hasNext())
             {
-                ColumnDef cdef = itera.next();
-                ByteBuffer columnValue = row.getBytesUnsafe(ByteBufferUtil.string(cdef.name.duplicate()));
+                ColumnInfo cdef = itera.next();
+                ByteBuffer columnValue = row.getBytesUnsafe(cdef.getName());
                 if (columnValue != null)
                 {
-                    Cell cell = new BufferCell(CellNames.simpleDense(cdef.name), columnValue);
-                    AbstractType<?> validator = getValidatorMap(cfDef).get(cdef.name);
-                    setTupleValue(tuple, i, cqlColumnToObj(cell, cfDef), validator);
+                    AbstractType<?> validator = getValidatorMap(tableMetadata).get(ByteBufferUtil.bytes(cdef.getName()));
+                    setTupleValue(tuple, i, cqlColumnToObj(ByteBufferUtil.bytes(cdef.getName()), columnValue,
+                                                           tableMetadata), validator);
                 }
                 else
                     tuple.set(i, null);
@@ -168,15 +188,11 @@
     }
 
     /** convert a cql column to an object */
-    private Object cqlColumnToObj(Cell col, CfDef cfDef) throws IOException
+    private Object cqlColumnToObj(ByteBuffer name, ByteBuffer columnValue, TableInfo cfDef) throws IOException
     {
         // standard
         Map<ByteBuffer,AbstractType> validators = getValidatorMap(cfDef);
-        ByteBuffer cellName = col.name().toByteBuffer();
-        if (validators.get(cellName) == null)
-            return cassandraToObj(getDefaultMarshallers(cfDef).get(MarshallerType.DEFAULT_VALIDATOR), col.value());
-        else
-            return cassandraToObj(validators.get(cellName), col.value());
+        return StorageHelper.cassandraToObj(validators.get(name), columnValue, nativeProtocolVersion);
     }
 
     /** set the value to the position of the tuple */
@@ -185,7 +201,7 @@
         if (validator instanceof CollectionType)
             setCollectionTupleValues(tuple, position, value, validator);
         else
-           setTupleValue(tuple, position, value);
+           StorageHelper.setTupleValue(tuple, position, value);
     }
 
     /** set the values of set/list at and after the position of the tuple */
@@ -240,170 +256,31 @@
         return obj;
     }
 
-    /** include key columns */
-    protected List<ColumnDef> getColumnMetadata(Cassandra.Client client)
-            throws InvalidRequestException,
-            UnavailableException,
-            TimedOutException,
-            SchemaDisagreementException,
-            TException,
-            CharacterCodingException,
-            org.apache.cassandra.exceptions.InvalidRequestException,
-            ConfigurationException,
-            NotFoundException
+    /** get the columnfamily definition for the signature */
+    protected TableInfo getCfInfo(String signature) throws IOException
     {
-        List<ColumnDef> keyColumns = null;
-        // get key columns
+        UDFContext context = UDFContext.getUDFContext();
+        Properties property = context.getUDFProperties(CqlNativeStorage.class);
+        TableInfo cfInfo;
         try
         {
-            keyColumns = getKeysMeta(client);
+            cfInfo = cfdefFromString(property.getProperty(signature));
         }
-        catch(Exception e)
+        catch (ClassNotFoundException e)
         {
-            logger.error("Error in retrieving key columns" , e);
+            throw new IOException(e);
         }
-
-        // get other columns
-        List<ColumnDef> columns = getColumnMeta(client, false, !hasCompactValueAlias);
-
-        // combine all columns in a list
-        if (keyColumns != null && columns != null)
-            keyColumns.addAll(columns);
-
-        return keyColumns;
+        return cfInfo;
     }
 
-    /** get keys meta data */
-    private List<ColumnDef> getKeysMeta(Cassandra.Client client)
-            throws Exception
+    /** return the CfInfo for the column family */
+    protected TableMetadata getCfInfo(Session client)
+            throws NoHostAvailableException,
+            AuthenticationException,
+            IllegalStateException
     {
-        String query = "SELECT key_aliases, " +
-                "       column_aliases, " +
-                "       key_validator, " +
-                "       comparator, " +
-                "       keyspace_name, " +
-                "       value_alias, " +
-                "       default_validator " +
-                "FROM system.schema_columnfamilies " +
-                "WHERE keyspace_name = '%s'" +
-                "  AND columnfamily_name = '%s' ";
-
-        CqlResult result = client.execute_cql3_query(
-                ByteBufferUtil.bytes(String.format(query, keyspace, column_family)),
-                Compression.NONE,
-                ConsistencyLevel.ONE);
-
-        if (result == null || result.rows == null || result.rows.isEmpty())
-            return null;
-
-        Iterator<CqlRow> iteraRow = result.rows.iterator();
-        List<ColumnDef> keys = new ArrayList<ColumnDef>();
-        if (iteraRow.hasNext())
-        {
-            CqlRow cqlRow = iteraRow.next();
-            String name = ByteBufferUtil.string(cqlRow.columns.get(4).value);
-            logger.debug("Found ksDef name: {}", name);
-            String keyString = ByteBufferUtil.string(ByteBuffer.wrap(cqlRow.columns.get(0).getValue()));
-
-            logger.debug("partition keys: {}", keyString);
-            List<String> keyNames = FBUtilities.fromJsonList(keyString);
-
-            Iterator<String> iterator = keyNames.iterator();
-            while (iterator.hasNext())
-            {
-                ColumnDef cDef = new ColumnDef();
-                cDef.name = ByteBufferUtil.bytes(iterator.next());
-                keys.add(cDef);
-            }
-            // classic thrift tables
-            if (keys.size() == 0)
-            {
-                CFMetaData cfm = getCFMetaData(keyspace, column_family, client);
-                for (ColumnDefinition def : cfm.partitionKeyColumns())
-                {
-                    String key = def.name.toString();
-                    logger.debug("name: {} ", key);
-                    ColumnDef cDef = new ColumnDef();
-                    cDef.name = ByteBufferUtil.bytes(key);
-                    keys.add(cDef);
-                }
-                for (ColumnDefinition def : cfm.clusteringColumns())
-                {
-                    String key = def.name.toString();
-                    logger.debug("name: {} ", key);
-                    ColumnDef cDef = new ColumnDef();
-                    cDef.name = ByteBufferUtil.bytes(key);
-                    keys.add(cDef);
-                }
-            }
-
-            keyString = ByteBufferUtil.string(ByteBuffer.wrap(cqlRow.columns.get(1).getValue()));
-
-            logger.debug("cluster keys: {}", keyString);
-            keyNames = FBUtilities.fromJsonList(keyString);
-
-            iterator = keyNames.iterator();
-            while (iterator.hasNext())
-            {
-                ColumnDef cDef = new ColumnDef();
-                cDef.name = ByteBufferUtil.bytes(iterator.next());
-                keys.add(cDef);
-            }
-
-            String validator = ByteBufferUtil.string(ByteBuffer.wrap(cqlRow.columns.get(2).getValue()));
-            logger.debug("row key validator: {}", validator);
-            AbstractType<?> keyValidator = parseType(validator);
-
-            Iterator<ColumnDef> keyItera = keys.iterator();
-            if (keyValidator instanceof CompositeType)
-            {
-                Iterator<AbstractType<?>> typeItera = ((CompositeType) keyValidator).types.iterator();
-                while (typeItera.hasNext())
-                    keyItera.next().validation_class = typeItera.next().toString();
-            }
-            else
-                keyItera.next().validation_class = keyValidator.toString();
-
-            validator = ByteBufferUtil.string(ByteBuffer.wrap(cqlRow.columns.get(3).getValue()));
-            logger.debug("cluster key validator: {}", validator);
-
-            if (keyItera.hasNext() && validator != null && !validator.isEmpty())
-            {
-                AbstractType<?> clusterKeyValidator = parseType(validator);
-
-                if (clusterKeyValidator instanceof CompositeType)
-                {
-                    Iterator<AbstractType<?>> typeItera = ((CompositeType) clusterKeyValidator).types.iterator();
-                    while (keyItera.hasNext())
-                        keyItera.next().validation_class = typeItera.next().toString();
-                }
-                else
-                    keyItera.next().validation_class = clusterKeyValidator.toString();
-            }
-
-            // compact value_alias column
-            if (cqlRow.columns.get(5).value != null)
-            {
-                try
-                {
-                    String compactValidator = ByteBufferUtil.string(ByteBuffer.wrap(cqlRow.columns.get(6).getValue()));
-                    logger.debug("default validator: {}", compactValidator);
-                    AbstractType<?> defaultValidator = parseType(compactValidator);
-
-                    ColumnDef cDef = new ColumnDef();
-                    cDef.name = cqlRow.columns.get(5).value;
-                    cDef.validation_class = defaultValidator.toString();
-                    keys.add(cDef);
-                    hasCompactValueAlias = true;
-                }
-                catch (Exception e)
-                {
-                    // no compact column at value_alias
-                }
-            }
-
-        }
-        return keys;
+        // get CF meta data
+        return client.getCluster().getMetadata().getKeyspace(Metadata.quote(keyspace)).getTable(Metadata.quote(column_family));
     }
 
     /** convert key tuple to key map */
@@ -414,7 +291,6 @@
         {
             if (t.getType(i) != DataType.TUPLE)
                 throw new IOException("keys was not a tuple");
-
             Tuple inner = (Tuple) t.get(i);
             if (inner.size() != 2)
                 throw new IOException("Keys were not in name and value pairs");
@@ -426,6 +302,91 @@
         return keys;
     }
 
+    /** convert object to ByteBuffer */
+    protected ByteBuffer objToBB(Object o)
+    {
+        if (o == null)
+            return nullToBB();
+        if (o instanceof java.lang.String)
+            return ByteBuffer.wrap(new DataByteArray((String)o).get());
+        if (o instanceof Integer)
+            return Int32Type.instance.decompose((Integer)o);
+        if (o instanceof Long)
+            return LongType.instance.decompose((Long)o);
+        if (o instanceof Float)
+            return FloatType.instance.decompose((Float)o);
+        if (o instanceof Double)
+            return DoubleType.instance.decompose((Double)o);
+        if (o instanceof UUID)
+            return ByteBuffer.wrap(UUIDGen.decompose((UUID) o));
+        if(o instanceof Tuple) {
+            List<Object> objects = ((Tuple)o).getAll();
+            //collections
+            if (objects.size() > 0 && objects.get(0) instanceof String)
+            {
+                String collectionType = (String) objects.get(0);
+                if ("set".equalsIgnoreCase(collectionType) ||
+                        "list".equalsIgnoreCase(collectionType))
+                    return objToListOrSetBB(objects.subList(1, objects.size()));
+                else if ("map".equalsIgnoreCase(collectionType))
+                    return objToMapBB(objects.subList(1, objects.size()));
+
+            }
+            return objToCompositeBB(objects);
+        }
+
+        return ByteBuffer.wrap(((DataByteArray) o).get());
+    }
+
+    private ByteBuffer objToListOrSetBB(List<Object> objects)
+    {
+        List<ByteBuffer> serialized = new ArrayList<ByteBuffer>(objects.size());
+        for(Object sub : objects)
+        {
+            ByteBuffer buffer = objToBB(sub);
+            serialized.add(buffer);
+        }
+        return CollectionSerializer.pack(serialized, objects.size(), 3);
+    }
+
+    private ByteBuffer objToMapBB(List<Object> objects)
+    {
+        List<ByteBuffer> serialized = new ArrayList<ByteBuffer>(objects.size() * 2);
+        for(Object sub : objects)
+        {
+            List<Object> keyValue = ((Tuple)sub).getAll();
+            for (Object entry: keyValue)
+            {
+                ByteBuffer buffer = objToBB(entry);
+                serialized.add(buffer);
+            }
+        }
+        return CollectionSerializer.pack(serialized, objects.size(), 3);
+    }
+
+    private ByteBuffer objToCompositeBB(List<Object> objects)
+    {
+        List<ByteBuffer> serialized = new ArrayList<ByteBuffer>(objects.size());
+        int totalLength = 0;
+        for(Object sub : objects)
+        {
+            ByteBuffer buffer = objToBB(sub);
+            serialized.add(buffer);
+            totalLength += 2 + buffer.remaining() + 1;
+        }
+        ByteBuffer out = ByteBuffer.allocate(totalLength);
+        for (ByteBuffer bb : serialized)
+        {
+            int length = bb.remaining();
+            out.put((byte) ((length >> 8) & 0xFF));
+            out.put((byte) (length & 0xFF));
+            out.put(bb);
+            out.put((byte) 0);
+        }
+        out.flip();
+        return out;
+    }
+
     /** send CQL query request using data from tuple */
     private void cqlQueryFromTuple(Map<String, ByteBuffer> key, Tuple t, int offset) throws IOException
     {
@@ -467,30 +428,50 @@
         }
     }
 
+    /** get the validators */
+    protected Map<ByteBuffer, AbstractType> getValidatorMap(TableInfo cfDef) throws IOException
+    {
+        Map<ByteBuffer, AbstractType> validators = new HashMap<>();
+        for (ColumnInfo cd : cfDef.getColumns())
+        {
+            if (cd.getTypeName() != null)
+            {
+                try
+                {
+                    AbstractType validator = TypeParser.parseCqlName(cd.getTypeName());
+                    if (validator instanceof CounterColumnType)
+                        validator = LongType.instance;
+                    validators.put(ByteBufferUtil.bytes(cd.getName()), validator);
+                }
+                catch (ConfigurationException | SyntaxException e)
+                {
+                    throw new IOException(e);
+                }
+            }
+        }
+        return validators;
+    }
+
     /** schema: (value, value, value) where keys are in the front. */
     public ResourceSchema getSchema(String location, Job job) throws IOException
     {
         setLocation(location, job);
-        CfInfo cfInfo = getCfInfo(loadSignature);
-        CfDef cfDef = cfInfo.cfDef;
+        TableInfo cfInfo = getCfInfo(loadSignature);
         // top-level schema, no type
         ResourceSchema schema = new ResourceSchema();
 
-        // get default marshallers and validators
-        Map<MarshallerType, AbstractType> marshallers = getDefaultMarshallers(cfDef);
-        Map<ByteBuffer, AbstractType> validators = getValidatorMap(cfDef);
+        // get default validators
+        Map<ByteBuffer, AbstractType> validators = getValidatorMap(cfInfo);
 
         // will contain all fields for this schema
         List<ResourceFieldSchema> allSchemaFields = new ArrayList<ResourceFieldSchema>();
 
-        for (ColumnDef cdef : cfDef.column_metadata)
+        for (ColumnInfo cdef : cfInfo.getColumns())
         {
             ResourceFieldSchema valSchema = new ResourceFieldSchema();
-            AbstractType validator = validators.get(cdef.name);
-            if (validator == null)
-                validator = marshallers.get(MarshallerType.DEFAULT_VALIDATOR);
-            valSchema.setName(new String(cdef.getName()));
-            valSchema.setType(getPigType(validator));
+            AbstractType<?> validator = validators.get(ByteBufferUtil.bytes(cdef.getName()));
+            valSchema.setName(cdef.getName());
+            valSchema.setType(StorageHelper.getPigType(validator));
             allSchemaFields.add(valSchema);
         }
 
@@ -502,8 +483,8 @@
     public void setPartitionFilter(Expression partitionFilter) throws IOException
     {
         UDFContext context = UDFContext.getUDFContext();
-        Properties property = context.getUDFProperties(AbstractCassandraStorage.class);
-        property.setProperty(PARTITION_FILTER_SIGNATURE, partitionFilterToWhereClauseString(partitionFilter));
+        Properties property = context.getUDFProperties(CqlNativeStorage.class);
+        property.setProperty(StorageHelper.PARTITION_FILTER_SIGNATURE, partitionFilterToWhereClauseString(partitionFilter));
     }
 
     /**
@@ -537,8 +518,8 @@
     private String getWhereClauseForPartitionFilter()
     {
         UDFContext context = UDFContext.getUDFContext();
-        Properties property = context.getUDFProperties(AbstractCassandraStorage.class);
-        return property.getProperty(PARTITION_FILTER_SIGNATURE);
+        Properties property = context.getUDFProperties(CqlNativeStorage.class);
+        return property.getProperty(StorageHelper.PARTITION_FILTER_SIGNATURE);
     }
 
     /**
@@ -597,8 +578,6 @@
             CqlConfigHelper.setInputCoreConnections(conf, nativeCoreConnections);
         if (nativeMaxConnections != null)
             CqlConfigHelper.setInputMaxConnections(conf, nativeMaxConnections);
-        if (nativeMinSimultReqs != null)
-            CqlConfigHelper.setInputMinSimultReqPerConnections(conf, nativeMinSimultReqs);
         if (nativeMaxSimultReqs != null)
             CqlConfigHelper.setInputMaxSimultReqPerConnections(conf, nativeMaxSimultReqs);
         if (nativeConnectionTimeout != null)
@@ -642,25 +621,25 @@
             CqlConfigHelper.setInputWhereClauses(conf, whereClause);
 
         String whereClauseForPartitionFilter = getWhereClauseForPartitionFilter();
-        String wc = whereClause != null && !whereClause.trim().isEmpty() 
+        String wc = whereClause != null && !whereClause.trim().isEmpty()
                                ? whereClauseForPartitionFilter == null ? whereClause: String.format("%s AND %s", whereClause.trim(), whereClauseForPartitionFilter)
                                : whereClauseForPartitionFilter;
 
         if (wc != null)
         {
-            logger.debug("where clause: {}", wc);
+            logger.trace("where clause: {}", wc);
             CqlConfigHelper.setInputWhereClauses(conf, wc);
-        } 
-        if (System.getenv(PIG_INPUT_SPLIT_SIZE) != null)
+        }
+        if (System.getenv(StorageHelper.PIG_INPUT_SPLIT_SIZE) != null)
         {
             try
             {
-                ConfigHelper.setInputSplitSize(conf, Integer.parseInt(System.getenv(PIG_INPUT_SPLIT_SIZE)));
+                ConfigHelper.setInputSplitSize(conf, Integer.parseInt(System.getenv(StorageHelper.PIG_INPUT_SPLIT_SIZE)));
             }
             catch (NumberFormatException e)
             {
                 throw new IOException("PIG_INPUT_SPLIT_SIZE is not a number", e);
-            }           
+            }
         }
 
         if (ConfigHelper.getInputInitialAddress(conf) == null)
@@ -703,36 +682,26 @@
         {
             DEFAULT_OUTPUT_FORMAT = BULK_OUTPUT_FORMAT;
             if (bulkCfSchema != null)
-                CqlBulkOutputFormat.setColumnFamilySchema(conf, column_family, bulkCfSchema);
+                CqlBulkOutputFormat.setTableSchema(conf, column_family, bulkCfSchema);
             else
                 throw new IOException("bulk_cf_schema is missing in input url parameter");
             if (bulkInsertStatement != null)
-                CqlBulkOutputFormat.setColumnFamilyInsertStatement(conf, column_family, bulkInsertStatement);
+                CqlBulkOutputFormat.setTableInsertStatement(conf, column_family, bulkInsertStatement);
             else
                 throw new IOException("bulk_insert_statement is missing in input url parameter");
+            if (bulkTableAlias != null)
+                CqlBulkOutputFormat.setTableAlias(conf, bulkTableAlias, column_family);
+            CqlBulkOutputFormat.setDeleteSourceOnSuccess(conf, bulkDeleteSourceOnSuccess);
             if (bulkOutputLocation != null)
-                conf.set(AbstractBulkRecordWriter.OUTPUT_LOCATION, bulkOutputLocation);
+                conf.set(CqlBulkRecordWriter.OUTPUT_LOCATION, bulkOutputLocation);
             if (bulkBuffSize > 0)
-                conf.set(AbstractBulkRecordWriter.BUFFER_SIZE_IN_MB, String.valueOf(bulkBuffSize));
+                conf.set(CqlBulkRecordWriter.BUFFER_SIZE_IN_MB, String.valueOf(bulkBuffSize));
             if (bulkStreamThrottle > 0)
-                conf.set(AbstractBulkRecordWriter.STREAM_THROTTLE_MBITS, String.valueOf(bulkStreamThrottle));
+                conf.set(CqlBulkRecordWriter.STREAM_THROTTLE_MBITS, String.valueOf(bulkStreamThrottle));
             if (bulkMaxFailedHosts > 0)
-                conf.set(AbstractBulkRecordWriter.MAX_FAILED_HOSTS, String.valueOf(bulkMaxFailedHosts));
-            CqlBulkOutputFormat.setSSLStoragePort(conf, sslStoragePort);
-            CqlBulkOutputFormat.setStoragePort(conf, storagePort);
-            if (serverEncrypted())
-            {
-                if (!StringUtils.isEmpty(serverKeystore))
-                    CqlBulkOutputFormat.setServerKeystore(conf, serverKeystore);
-                if (!StringUtils.isEmpty(serverTruststore))
-                    CqlBulkOutputFormat.setServerTruststore(conf, serverTruststore);
-                if (!StringUtils.isEmpty(serverKeystorePassword))
-                    CqlBulkOutputFormat.setServerKeystorePassword(conf, serverKeystorePassword);
-                if (!StringUtils.isEmpty(serverTruststorePassword))
-                    CqlBulkOutputFormat.setServerTruststorePassword(conf, serverTruststorePassword);
-                if (!StringUtils.isEmpty(serverCipherSuites))
-                    CqlBulkOutputFormat.setServerCipherSuites(conf, serverCipherSuites);
-            }
+                conf.set(CqlBulkRecordWriter.MAX_FAILED_HOSTS, String.valueOf(bulkMaxFailedHosts));
+            if (partitionerClass!= null)
+                ConfigHelper.setInputPartitioner(conf, partitionerClass);
         }
 
         setConnectionInformation();
@@ -747,10 +716,74 @@
         initSchema(storeSignature);
     }
 
-    private boolean serverEncrypted()
+    /** Methods to get the column family schema from Cassandra */
+    protected void initSchema(String signature) throws IOException
     {
-        return !StringUtils.isEmpty(internodeEncrypt) && 
-                InternodeEncryption.none != InternodeEncryption.valueOf(internodeEncrypt.toLowerCase());
+        Properties properties = UDFContext.getUDFContext().getUDFProperties(CqlNativeStorage.class);
+
+        // Only get the schema if we haven't already gotten it
+        if (!properties.containsKey(signature))
+        {
+            try (Cluster cluster = CqlConfigHelper.getInputCluster(ConfigHelper.getInputInitialAddress(conf), conf);
+                 Session client = cluster.connect())
+            {
+                client.execute("USE " + keyspace);
+
+                // compose the CfDef for the columfamily
+                TableMetadata cfInfo = getCfInfo(client);
+
+                if (cfInfo != null)
+                {
+                    properties.setProperty(signature, cfdefToString(cfInfo));
+                }
+                else
+                    throw new IOException(String.format("Table '%s' not found in keyspace '%s'",
+                            column_family,
+                            keyspace));
+            }
+            catch (Exception e)
+            {
+                throw new IOException(e);
+            }
+        }
+    }
+
+
+    /** convert CfDef to string */
+    protected static String cfdefToString(TableMetadata cfDef) throws IOException
+    {
+        TableInfo tableInfo = new TableInfo(cfDef);
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        try (ObjectOutputStream oos = new ObjectOutputStream( baos ))
+        {
+            oos.writeObject(tableInfo);
+        }
+
+        return new String( Base64Coder.encode(baos.toByteArray()) );
+    }
+
+    /** convert string back to CfDef */
+    protected static TableInfo cfdefFromString(String st) throws IOException, ClassNotFoundException
+    {
+        byte [] data = Base64Coder.decode( st );
+        try (ObjectInputStream ois = new ObjectInputStream(new ByteArrayInputStream(data)))
+        {
+            Object o = ois.readObject();
+            return (TableInfo)o;
+        }
+    }
+
+    /** decompose the query to store the parameters in a map */
+    public static Map<String, String> getQueryMap(String query) throws UnsupportedEncodingException
+    {
+        String[] params = query.split("&");
+        Map<String, String> map = new HashMap<String, String>(params.length);
+        for (String param : params)
+        {
+            String[] keyValue = param.split("=");
+            map.put(keyValue[0], URLDecoder.decode(keyValue[1], "UTF-8"));
+        }
+        return map;
     }
 
     private void setLocationFromUri(String location) throws IOException
@@ -787,22 +820,10 @@
                     bulkStreamThrottle = Integer.valueOf(urlQuery.get("bulk_stream_throttle"));
                 if (urlQuery.containsKey("bulk_max_failed_hosts"))
                     bulkMaxFailedHosts = Integer.valueOf(urlQuery.get("bulk_max_failed_hosts"));
-                if (urlQuery.containsKey("storage_port"))
-                    storagePort = Integer.valueOf(urlQuery.get("storage_port"));
-                if (urlQuery.containsKey("ssl_storage_port"))
-                    sslStoragePort = Integer.valueOf(urlQuery.get("ssl_storage_port"));
-                if (urlQuery.containsKey("internode_encrypt"))
-                    internodeEncrypt = urlQuery.get("internode_encrypt");
-                if (urlQuery.containsKey("server_keystore"))
-                    serverKeystore = urlQuery.get("server_keystore");
-                if (urlQuery.containsKey("server_truststore"))
-                    serverTruststore = urlQuery.get("server_truststore");
-                if (urlQuery.containsKey("server_keystore_pass"))
-                    serverKeystorePassword = urlQuery.get("server_keystore_pass");
-                if (urlQuery.containsKey("server_truststore_pass"))
-                    serverTruststorePassword = urlQuery.get("server_truststore_pass");
-                if (urlQuery.containsKey("server_cipher_suites"))
-                    serverCipherSuites = urlQuery.get("server_cipher_suites");
+                if (urlQuery.containsKey("bulk_delete_source"))
+                    bulkDeleteSourceOnSuccess = Boolean.parseBoolean(urlQuery.get("bulk_delete_source"));
+                if (urlQuery.containsKey("bulk_table_alias"))
+                    bulkTableAlias = urlQuery.get("bulk_table_alias");
 
                 //split size
                 if (urlQuery.containsKey("split_size"))
@@ -820,8 +841,6 @@
                     nativeCoreConnections = urlQuery.get("core_conns");
                 if (urlQuery.containsKey("max_conns"))
                     nativeMaxConnections = urlQuery.get("max_conns");
-                if (urlQuery.containsKey("min_simult_reqs"))
-                    nativeMinSimultReqs = urlQuery.get("min_simult_reqs");
                 if (urlQuery.containsKey("max_simult_reqs"))
                     nativeMaxSimultReqs = urlQuery.get("max_simult_reqs");
                 if (urlQuery.containsKey("native_timeout"))
@@ -889,21 +908,177 @@
                     "[&key_store_path=<key_store_path>][&trust_store_password=<trust_store_password>]" +
                     "[&key_store_password=<key_store_password>][&cipher_suites=<cipher_suites>][&input_cql=<input_cql>]" +
                     "[columns=<columns>][where_clause=<where_clause>]" +
-                    "[&bulk_cf_schema=bulk_cf_schema][&bulk_insert_statement=bulk_insert_statement]" +
-                    "[&bulk_output_location=<bulk_output_location>][&bulk_buff_size=<bulk_buff_size>]" +
-                    "[&storage_port=<storage_port>][&ssl_storage_port=<ssl_storage_port>]" +
-                    "[&server_keystore=<server_keystore>][&server_keystore_pass=<server_keystore_pass>]" +
-                    "[&server_truststore=<server_truststore>][&server_truststore_pass=<server_truststore_pass>]" +
-                    "[&server_cipher_suites=<server_cipher_suites>][&internode_encrypt=<internode_encrypt>]" +
+                    "[&bulk_cf_schema=bulk_cf_schema][&bulk_insert_statement=bulk_insert_statement][&bulk_table_alias=<bulk_table_alias>]" +
+                    "[&bulk_output_location=<bulk_output_location>][&bulk_buff_size=<bulk_buff_size>][&bulk_delete_source=<bulk_delete_source>]" +
                     "[&bulk_stream_throttle=<bulk_stream_throttle>][&bulk_max_failed_hosts=<bulk_max_failed_hosts>]]': " +  e.getMessage());
          }
     }
 
-    /**
-     * Thrift API can't handle null, so use empty byte array
-     */
     public ByteBuffer nullToBB()
     {
         return ByteBuffer.wrap(new byte[0]);
     }
+
+    /** output format */
+    public OutputFormat getOutputFormat() throws IOException
+    {
+        try
+        {
+            return FBUtilities.construct(outputFormatClass, "outputformat");
+        }
+        catch (ConfigurationException e)
+        {
+            throw new IOException(e);
+        }
+    }
+
+    public void cleanupOnFailure(String failure, Job job)
+    {
+    }
+
+    public void cleanupOnSuccess(String location, Job job) throws IOException {
+    }
+
+    /** return partition keys */
+    public String[] getPartitionKeys(String location, Job job) throws IOException
+    {
+        if (!usePartitionFilter)
+            return null;
+        TableInfo tableMetadata = getCfInfo(loadSignature);
+        String[] partitionKeys = new String[tableMetadata.getPartitionKey().size()];
+        for (int i = 0; i < tableMetadata.getPartitionKey().size(); i++)
+        {
+            partitionKeys[i] = tableMetadata.getPartitionKey().get(i).getName();
+        }
+        return partitionKeys;
+    }
+
+    public void checkSchema(ResourceSchema schema) throws IOException
+    {
+        // we don't care about types, they all get casted to ByteBuffers
+    }
+
+    public ResourceStatistics getStatistics(String location, Job job)
+    {
+        return null;
+    }
+
+    @Override
+    public InputFormat getInputFormat() throws IOException
+    {
+        try
+        {
+            return FBUtilities.construct(inputFormatClass, "inputformat");
+        }
+        catch (ConfigurationException e)
+        {
+            throw new IOException(e);
+        }
+    }
+
+    public String relToAbsPathForStoreLocation(String location, Path curDir) throws IOException
+    {
+        return relativeToAbsolutePath(location, curDir);
+    }
+
+    @Override
+    public String relativeToAbsolutePath(String location, Path curDir) throws IOException
+    {
+        return location;
+    }
+
+    @Override
+    public void setUDFContextSignature(String signature)
+    {
+        this.loadSignature = signature;
+    }
+
+    /** StoreFunc methods */
+    public void setStoreFuncUDFContextSignature(String signature)
+    {
+        this.storeSignature = signature;
+    }
+
+    /** set hadoop cassandra connection settings */
+    protected void setConnectionInformation() throws IOException
+    {
+        StorageHelper.setConnectionInformation(conf);
+        if (System.getenv(StorageHelper.PIG_INPUT_FORMAT) != null)
+            inputFormatClass = getFullyQualifiedClassName(System.getenv(StorageHelper.PIG_INPUT_FORMAT));
+        else
+            inputFormatClass = DEFAULT_INPUT_FORMAT;
+        if (System.getenv(StorageHelper.PIG_OUTPUT_FORMAT) != null)
+            outputFormatClass = getFullyQualifiedClassName(System.getenv(StorageHelper.PIG_OUTPUT_FORMAT));
+        else
+            outputFormatClass = DEFAULT_OUTPUT_FORMAT;
+    }
+
+    /** get the full class name */
+    protected String getFullyQualifiedClassName(String classname)
+    {
+        return classname.contains(".") ? classname : "org.apache.cassandra.hadoop." + classname;
+    }
 }
+
+class TableInfo implements Serializable
+{
+    private final List<ColumnInfo> columns;
+    private final List<ColumnInfo> partitionKey;
+    private final String name;
+
+    public TableInfo(TableMetadata tableMetadata)
+    {
+        List<ColumnMetadata> cmColumns = tableMetadata.getColumns();
+        columns = new ArrayList<>(cmColumns.size());
+        for (ColumnMetadata cm : cmColumns)
+        {
+            columns.add(new ColumnInfo(this, cm));
+        }
+        List<ColumnMetadata> cmPartitionKey = tableMetadata.getPartitionKey();
+        partitionKey = new ArrayList<>(cmPartitionKey.size());
+        for (ColumnMetadata cm : cmPartitionKey)
+        {
+            partitionKey.add(new ColumnInfo(this, cm));
+        }
+        name = tableMetadata.getName();
+    }
+
+    public List<ColumnInfo> getPartitionKey()
+    {
+        return partitionKey;
+    }
+
+    public List<ColumnInfo> getColumns()
+    {
+        return columns;
+    }
+
+    public String getName()
+    {
+        return name;
+    }
+}
+
+class ColumnInfo implements Serializable
+{
+    private final TableInfo table;
+    private final String name;
+    private final String typeName;
+
+    public ColumnInfo(TableInfo tableInfo, ColumnMetadata columnMetadata)
+    {
+        table = tableInfo;
+        name = columnMetadata.getName();
+        typeName = columnMetadata.getType().toString();
+    }
+
+    public String getName()
+    {
+        return name;
+    }
+
+    public String getTypeName()
+    {
+        return typeName;
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java b/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java
deleted file mode 100644
index 66583ec..0000000
--- a/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * 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.hadoop.pig;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-/**
- * @deprecated use CqlNativeStorage instead. CqlStorage will be removed.
- */
-public class CqlStorage extends CqlNativeStorage
-{
-    private static final Logger logger = LoggerFactory.getLogger(CqlNativeStorage.class);
-
-    public CqlStorage()
-    {
-        this(1000);
-        logger.warn("CqlStorage is deprecated and will be removed in the next release, use CqlNativeStorage instead.");
-    }
-
-    /** @param pageSize limit number of CQL rows to fetch in a thrift request */
-    public CqlStorage(int pageSize)
-    {
-        super(pageSize);
-    }
-}
diff --git a/src/java/org/apache/cassandra/hadoop/pig/StorageHelper.java b/src/java/org/apache/cassandra/hadoop/pig/StorageHelper.java
new file mode 100644
index 0000000..74f734e
--- /dev/null
+++ b/src/java/org/apache/cassandra/hadoop/pig/StorageHelper.java
@@ -0,0 +1,142 @@
+/*
+ *
+ * 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.hadoop.pig;
+
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.util.Date;
+import java.util.UUID;
+
+import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.hadoop.ConfigHelper;
+import org.apache.cassandra.serializers.CollectionSerializer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.UUIDGen;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.data.DataByteArray;
+import org.apache.pig.data.DataType;
+import org.apache.pig.data.Tuple;
+
+@Deprecated
+public class StorageHelper
+{
+    // system environment variables that can be set to configure connection info:
+    // alternatively, Hadoop JobConf variables can be set using keys from ConfigHelper
+    public final static String PIG_INPUT_RPC_PORT = "PIG_INPUT_RPC_PORT";
+    public final static String PIG_INPUT_INITIAL_ADDRESS = "PIG_INPUT_INITIAL_ADDRESS";
+    public final static String PIG_INPUT_PARTITIONER = "PIG_INPUT_PARTITIONER";
+    public final static String PIG_OUTPUT_RPC_PORT = "PIG_OUTPUT_RPC_PORT";
+    public final static String PIG_OUTPUT_INITIAL_ADDRESS = "PIG_OUTPUT_INITIAL_ADDRESS";
+    public final static String PIG_OUTPUT_PARTITIONER = "PIG_OUTPUT_PARTITIONER";
+    public final static String PIG_RPC_PORT = "PIG_RPC_PORT";
+    public final static String PIG_INITIAL_ADDRESS = "PIG_INITIAL_ADDRESS";
+    public final static String PIG_PARTITIONER = "PIG_PARTITIONER";
+    public final static String PIG_INPUT_FORMAT = "PIG_INPUT_FORMAT";
+    public final static String PIG_OUTPUT_FORMAT = "PIG_OUTPUT_FORMAT";
+    public final static String PIG_INPUT_SPLIT_SIZE = "PIG_INPUT_SPLIT_SIZE";
+
+
+    public final static String PARTITION_FILTER_SIGNATURE = "cassandra.partition.filter";
+
+    protected static void setConnectionInformation(Configuration conf)
+    {
+        if (System.getenv(PIG_RPC_PORT) != null)
+        {
+            ConfigHelper.setInputRpcPort(conf, System.getenv(PIG_RPC_PORT));
+            ConfigHelper.setOutputRpcPort(conf, System.getenv(PIG_RPC_PORT));
+        }
+
+        if (System.getenv(PIG_INPUT_RPC_PORT) != null)
+            ConfigHelper.setInputRpcPort(conf, System.getenv(PIG_INPUT_RPC_PORT));
+        if (System.getenv(PIG_OUTPUT_RPC_PORT) != null)
+            ConfigHelper.setOutputRpcPort(conf, System.getenv(PIG_OUTPUT_RPC_PORT));
+
+        if (System.getenv(PIG_INITIAL_ADDRESS) != null)
+        {
+            ConfigHelper.setInputInitialAddress(conf, System.getenv(PIG_INITIAL_ADDRESS));
+            ConfigHelper.setOutputInitialAddress(conf, System.getenv(PIG_INITIAL_ADDRESS));
+        }
+        if (System.getenv(PIG_INPUT_INITIAL_ADDRESS) != null)
+            ConfigHelper.setInputInitialAddress(conf, System.getenv(PIG_INPUT_INITIAL_ADDRESS));
+        if (System.getenv(PIG_OUTPUT_INITIAL_ADDRESS) != null)
+            ConfigHelper.setOutputInitialAddress(conf, System.getenv(PIG_OUTPUT_INITIAL_ADDRESS));
+
+        if (System.getenv(PIG_PARTITIONER) != null)
+        {
+            ConfigHelper.setInputPartitioner(conf, System.getenv(PIG_PARTITIONER));
+            ConfigHelper.setOutputPartitioner(conf, System.getenv(PIG_PARTITIONER));
+        }
+        if(System.getenv(PIG_INPUT_PARTITIONER) != null)
+            ConfigHelper.setInputPartitioner(conf, System.getenv(PIG_INPUT_PARTITIONER));
+        if(System.getenv(PIG_OUTPUT_PARTITIONER) != null)
+            ConfigHelper.setOutputPartitioner(conf, System.getenv(PIG_OUTPUT_PARTITIONER));
+    }
+
+    protected static Object cassandraToObj(AbstractType validator, ByteBuffer value, int nativeProtocolVersion)
+    {
+        if (validator instanceof DecimalType || validator instanceof InetAddressType)
+            return validator.getString(value);
+
+        if (validator instanceof CollectionType)
+        {
+            // For CollectionType, the compose() method assumes the v3 protocol format of collection, which
+            // is not correct here since we query using the CQL-over-thrift interface which use the pre-v3 format
+            return ((CollectionSerializer)validator.getSerializer()).deserializeForNativeProtocol(value, nativeProtocolVersion);
+        }
+
+        return validator.compose(value);
+    }
+
+    /** set the value to the position of the tuple */
+    protected static void setTupleValue(Tuple pair, int position, Object value) throws ExecException
+    {
+        if (value instanceof BigInteger)
+            pair.set(position, ((BigInteger) value).intValue());
+        else if (value instanceof ByteBuffer)
+            pair.set(position, new DataByteArray(ByteBufferUtil.getArray((ByteBuffer) value)));
+        else if (value instanceof UUID)
+            pair.set(position, new DataByteArray(UUIDGen.decompose((java.util.UUID) value)));
+        else if (value instanceof Date)
+            pair.set(position, TimestampType.instance.decompose((Date) value).getLong());
+        else
+            pair.set(position, value);
+    }
+
+    /** get pig type for the cassandra data type*/
+    protected static byte getPigType(AbstractType type)
+    {
+        if (type instanceof LongType || type instanceof DateType || type instanceof TimestampType) // DateType is bad and it should feel bad
+            return DataType.LONG;
+        else if (type instanceof IntegerType || type instanceof Int32Type) // IntegerType will overflow at 2**31, but is kept for compatibility until pig has a BigInteger
+            return DataType.INTEGER;
+        else if (type instanceof AsciiType || type instanceof UTF8Type || type instanceof DecimalType || type instanceof InetAddressType)
+            return DataType.CHARARRAY;
+        else if (type instanceof FloatType)
+            return DataType.FLOAT;
+        else if (type instanceof DoubleType)
+            return DataType.DOUBLE;
+        else if (type instanceof AbstractCompositeType || type instanceof CollectionType)
+            return DataType.TUPLE;
+
+        return DataType.BYTEARRAY;
+    }
+}
diff --git a/src/java/org/apache/cassandra/io/ISSTableSerializer.java b/src/java/org/apache/cassandra/io/ISSTableSerializer.java
index 20ee352..96a38ac 100644
--- a/src/java/org/apache/cassandra/io/ISSTableSerializer.java
+++ b/src/java/org/apache/cassandra/io/ISSTableSerializer.java
@@ -20,7 +20,7 @@
 import java.io.DataInput;
 import java.io.IOException;
 
-import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.format.Version;
 import org.apache.cassandra.io.util.DataOutputPlus;
 
 public interface ISSTableSerializer<T>
@@ -43,5 +43,5 @@
      * @throws IOException
      * @return the type that was deserialized
      */
-    public T deserializeFromSSTable(DataInput in, Descriptor.Version version) throws IOException;
+    public T deserializeFromSSTable(DataInput in, Version version) throws IOException;
 }
diff --git a/src/java/org/apache/cassandra/io/IVersionedSerializer.java b/src/java/org/apache/cassandra/io/IVersionedSerializer.java
index 46494e1..2572840 100644
--- a/src/java/org/apache/cassandra/io/IVersionedSerializer.java
+++ b/src/java/org/apache/cassandra/io/IVersionedSerializer.java
@@ -30,7 +30,7 @@
      * @param t type that needs to be serialized
      * @param out DataOutput into which serialization needs to happen.
      * @param version protocol version
-     * @throws java.io.IOException
+     * @throws java.io.IOException if serialization fails
      */
     public void serialize(T t, DataOutputPlus out, int version) throws IOException;
 
@@ -39,7 +39,7 @@
      * @param in DataInput from which deserialization needs to happen.
      * @param version protocol version
      * @return the type that was deserialized
-     * @throws IOException
+     * @throws IOException if deserialization fails
      */
     public T deserialize(DataInput in, int version) throws IOException;
 
diff --git a/src/java/org/apache/cassandra/io/compress/BufferType.java b/src/java/org/apache/cassandra/io/compress/BufferType.java
new file mode 100644
index 0000000..8817802
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/compress/BufferType.java
@@ -0,0 +1,45 @@
+/*
+ * 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.io.compress;
+
+import java.nio.ByteBuffer;
+
+public enum BufferType
+{
+    ON_HEAP
+    {
+        public ByteBuffer allocate(int size)
+        {
+            return ByteBuffer.allocate(size);
+        }
+    },
+    OFF_HEAP
+    {
+        public ByteBuffer allocate(int size)
+        {
+            return ByteBuffer.allocateDirect(size);
+        }
+    };
+
+    public abstract ByteBuffer allocate(int size);
+
+    public static BufferType typeOf(ByteBuffer buffer)
+    {
+        return buffer.isDirect() ? OFF_HEAP : ON_HEAP;
+    }
+}
diff --git a/src/java/org/apache/cassandra/io/compress/CompressedRandomAccessReader.java b/src/java/org/apache/cassandra/io/compress/CompressedRandomAccessReader.java
index 184db9c..ccfa5e7 100644
--- a/src/java/org/apache/cassandra/io/compress/CompressedRandomAccessReader.java
+++ b/src/java/org/apache/cassandra/io/compress/CompressedRandomAccessReader.java
@@ -19,16 +19,18 @@
 
 import java.io.*;
 import java.nio.ByteBuffer;
+import java.nio.MappedByteBuffer;
+import java.util.Map;
+import java.util.TreeMap;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.zip.Adler32;
-import java.util.zip.CRC32;
-import java.util.zip.Checksum;
+
+
+import com.google.common.primitives.Ints;
 
 import org.apache.cassandra.io.FSReadError;
 import org.apache.cassandra.io.sstable.CorruptSSTableException;
-import org.apache.cassandra.io.util.CompressedPoolingSegmentedFile;
-import org.apache.cassandra.io.util.PoolingSegmentedFile;
-import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.io.util.*;
 import org.apache.cassandra.utils.FBUtilities;
 
 /**
@@ -37,22 +39,20 @@
  */
 public class CompressedRandomAccessReader extends RandomAccessReader
 {
-    public static CompressedRandomAccessReader open(String dataFilePath, CompressionMetadata metadata)
+    public static CompressedRandomAccessReader open(ChannelProxy channel, CompressionMetadata metadata)
     {
-        return open(dataFilePath, metadata, null);
-    }
-    public static CompressedRandomAccessReader open(String path, CompressionMetadata metadata, CompressedPoolingSegmentedFile owner)
-    {
-        try
-        {
-            return new CompressedRandomAccessReader(path, metadata, owner);
-        }
-        catch (FileNotFoundException e)
-        {
-            throw new RuntimeException(e);
-        }
+        return new CompressedRandomAccessReader(channel, metadata, null, null);
     }
 
+    public static CompressedRandomAccessReader open(ICompressedFile file)
+    {
+        return new CompressedRandomAccessReader(file.channel(),
+                                                file.getMetadata(),
+                                                file,
+                                                file instanceof PoolingSegmentedFile ? (PoolingSegmentedFile) file : null);
+    }
+
+    private final TreeMap<Long, MappedByteBuffer> chunkSegments;
 
     private final CompressionMetadata metadata;
 
@@ -60,25 +60,93 @@
     private ByteBuffer compressed;
 
     // re-use single crc object
-    private final Checksum checksum;
+    private final Adler32 checksum;
 
     // raw checksum bytes
-    private final ByteBuffer checksumBytes = ByteBuffer.wrap(new byte[4]);
+    private ByteBuffer checksumBytes;
 
-    protected CompressedRandomAccessReader(String dataFilePath, CompressionMetadata metadata, PoolingSegmentedFile owner) throws FileNotFoundException
+    protected CompressedRandomAccessReader(ChannelProxy channel, CompressionMetadata metadata, ICompressedFile file, PoolingSegmentedFile owner)
     {
-        super(new File(dataFilePath), metadata.chunkLength(), metadata.compressedFileLength, owner);
+        super(channel, metadata.chunkLength(), metadata.compressedFileLength, metadata.compressor().preferredBufferType(), owner);
         this.metadata = metadata;
-        checksum = metadata.hasPostCompressionAdlerChecksums ? new Adler32() : new CRC32();
-        compressed = ByteBuffer.wrap(new byte[metadata.compressor().initialCompressedBufferLength(metadata.chunkLength())]);
+        checksum = new Adler32();
+
+        chunkSegments = file == null ? null : file.chunkSegments();
+        if (chunkSegments == null)
+        {
+            compressed = super.allocateBuffer(metadata.compressor().initialCompressedBufferLength(metadata.chunkLength()), metadata.compressor().preferredBufferType());
+            checksumBytes = ByteBuffer.wrap(new byte[4]);
+        }
     }
 
     @Override
-    protected void reBuffer()
+    protected ByteBuffer allocateBuffer(int bufferSize, BufferType bufferType)
+    {
+        assert Integer.bitCount(bufferSize) == 1;
+        return bufferType.allocate(bufferSize);
+    }
+
+    @Override
+    public void deallocate()
+    {
+        super.deallocate();
+        if (compressed != null)
+            FileUtils.clean(compressed);
+        compressed = null;
+    }
+
+    private void reBufferStandard()
     {
         try
         {
-            decompressChunk(metadata.chunkFor(current));
+            long position = current();
+            assert position < metadata.dataLength;
+
+            CompressionMetadata.Chunk chunk = metadata.chunkFor(position);
+
+            if (compressed.capacity() < chunk.length)
+                compressed = allocateBuffer(chunk.length, metadata.compressor().preferredBufferType());
+            else
+                compressed.clear();
+            compressed.limit(chunk.length);
+
+            if (channel.read(compressed, chunk.offset) != chunk.length)
+                throw new CorruptBlockException(getPath(), chunk);
+            compressed.flip();
+            buffer.clear();
+
+            try
+            {
+                metadata.compressor().uncompress(compressed, buffer);
+            }
+            catch (IOException e)
+            {
+                throw new CorruptBlockException(getPath(), chunk);
+            }
+            finally
+            {
+                buffer.flip();
+            }
+
+            if (metadata.parameters.getCrcCheckChance() > ThreadLocalRandom.current().nextDouble())
+            {
+                compressed.rewind();
+                FBUtilities.directCheckSum(checksum, compressed);
+
+                if (checksum(chunk) != (int) checksum.getValue())
+                    throw new CorruptBlockException(getPath(), chunk);
+
+                // reset checksum object back to the original (blank) state
+                checksum.reset();
+            }
+
+            // buffer offset is always aligned
+            bufferOffset = position & ~(buffer.capacity() - 1);
+            buffer.position((int) (position - bufferOffset));
+            // the length() can be provided at construction time, to override the true (uncompressed) length of the file;
+            // this is permitted to occur within a compressed segment, so we truncate validBufferBytes if we cross the imposed length
+            if (bufferOffset + buffer.limit() > length())
+                buffer.limit((int)(length() - bufferOffset));
         }
         catch (CorruptBlockException e)
         {
@@ -90,71 +158,91 @@
         }
     }
 
-    private void decompressChunk(CompressionMetadata.Chunk chunk) throws IOException
+    private void reBufferMmap()
     {
-        if (channel.position() != chunk.offset)
-            channel.position(chunk.offset);
-
-        if (compressed.capacity() < chunk.length)
-            compressed = ByteBuffer.wrap(new byte[chunk.length]);
-        else
-            compressed.clear();
-        compressed.limit(chunk.length);
-
-        if (channel.read(compressed) != chunk.length)
-            throw new CorruptBlockException(getPath(), chunk);
-
-        // technically flip() is unnecessary since all the remaining work uses the raw array, but if that changes
-        // in the future this will save a lot of hair-pulling
-        compressed.flip();
         try
         {
-            validBufferBytes = metadata.compressor().uncompress(compressed.array(), 0, chunk.length, buffer, 0);
-        }
-        catch (IOException e)
-        {
-            throw new CorruptBlockException(getPath(), chunk, e);
-        }
+            long position = current();
+            assert position < metadata.dataLength;
 
-        if (metadata.parameters.getCrcCheckChance() > ThreadLocalRandom.current().nextDouble())
-        {
+            CompressionMetadata.Chunk chunk = metadata.chunkFor(position);
 
-            if (metadata.hasPostCompressionAdlerChecksums)
+            Map.Entry<Long, MappedByteBuffer> entry = chunkSegments.floorEntry(chunk.offset);
+            long segmentOffset = entry.getKey();
+            int chunkOffset = Ints.checkedCast(chunk.offset - segmentOffset);
+            ByteBuffer compressedChunk = entry.getValue().duplicate(); // TODO: change to slice(chunkOffset) when we upgrade LZ4-java
+
+            compressedChunk.position(chunkOffset).limit(chunkOffset + chunk.length);
+
+            buffer.clear();
+
+            try
             {
-                checksum.update(compressed.array(), 0, chunk.length);
+                metadata.compressor().uncompress(compressedChunk, buffer);
             }
-            else
+            catch (IOException e)
             {
-                checksum.update(buffer, 0, validBufferBytes);
-            }
-
-            if (checksum(chunk) != (int) checksum.getValue())
                 throw new CorruptBlockException(getPath(), chunk);
+            }
+            finally
+            {
+                buffer.flip();
+            }
 
-            // reset checksum object back to the original (blank) state
-            checksum.reset();
+            if (metadata.parameters.getCrcCheckChance() > ThreadLocalRandom.current().nextDouble())
+            {
+                compressedChunk.position(chunkOffset).limit(chunkOffset + chunk.length);
+
+                FBUtilities.directCheckSum(checksum, compressedChunk);
+
+                compressedChunk.limit(compressedChunk.capacity());
+                if (compressedChunk.getInt() != (int) checksum.getValue())
+                    throw new CorruptBlockException(getPath(), chunk);
+
+                // reset checksum object back to the original (blank) state
+                checksum.reset();
+            }
+
+            // buffer offset is always aligned
+            bufferOffset = position & ~(buffer.capacity() - 1);
+            buffer.position((int) (position - bufferOffset));
+            // the length() can be provided at construction time, to override the true (uncompressed) length of the file;
+            // this is permitted to occur within a compressed segment, so we truncate validBufferBytes if we cross the imposed length
+            if (bufferOffset + buffer.limit() > length())
+                buffer.limit((int)(length() - bufferOffset));
+        }
+        catch (CorruptBlockException e)
+        {
+            throw new CorruptSSTableException(e, getPath());
         }
 
-        // buffer offset is always aligned
-        bufferOffset = current & ~(buffer.length - 1);
-        // the length() can be provided at construction time, to override the true (uncompressed) length of the file;
-        // this is permitted to occur within a compressed segment, so we truncate validBufferBytes if we cross the imposed length
-        if (bufferOffset + validBufferBytes > length())
-            validBufferBytes = (int)(length() - bufferOffset);
+    }
+
+    @Override
+    protected void reBuffer()
+    {
+        if (chunkSegments != null)
+        {
+            reBufferMmap();
+        }
+        else
+        {
+            reBufferStandard();
+        }
     }
 
     private int checksum(CompressionMetadata.Chunk chunk) throws IOException
     {
-        assert channel.position() == chunk.offset + chunk.length;
+        long position = chunk.offset + chunk.length;
         checksumBytes.clear();
-        if (channel.read(checksumBytes) != checksumBytes.capacity())
+        if (channel.read(checksumBytes, position) != checksumBytes.capacity())
             throw new CorruptBlockException(getPath(), chunk);
         return checksumBytes.getInt(0);
     }
 
     public int getTotalBufferSize()
     {
-        return super.getTotalBufferSize() + compressed.capacity();
+        return super.getTotalBufferSize() + (chunkSegments != null ? 0 : compressed.capacity());
     }
 
     @Override
diff --git a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
index c004232..9c7c776 100644
--- a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
+++ b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
@@ -17,25 +17,22 @@
  */
 package org.apache.cassandra.io.compress;
 
+import java.io.DataOutputStream;
 import java.io.EOFException;
 import java.io.File;
 import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
 import java.util.zip.Adler32;
-import java.util.zip.Checksum;
 
 import org.apache.cassandra.io.FSReadError;
 import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.sstable.CorruptSSTableException;
-import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.sstable.SSTableWriter;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 import org.apache.cassandra.io.util.DataIntegrityMetadata;
 import org.apache.cassandra.io.util.FileMark;
 import org.apache.cassandra.io.util.SequentialWriter;
-
-import static org.apache.cassandra.io.compress.CompressionMetadata.Writer.OpenType.FINAL;
-import static org.apache.cassandra.io.compress.CompressionMetadata.Writer.OpenType.SHARED;
-import static org.apache.cassandra.io.compress.CompressionMetadata.Writer.OpenType.SHARED_FINAL;
+import org.apache.cassandra.utils.FBUtilities;
 
 public class CompressedSequentialWriter extends SequentialWriter
 {
@@ -50,31 +47,33 @@
     private final ICompressor compressor;
 
     // used to store compressed data
-    private final ICompressor.WrappedArray compressed;
+    private ByteBuffer compressed;
 
     // holds a number of already written chunks
     private int chunkCount = 0;
 
-    private long originalSize = 0, compressedSize = 0;
+    private long uncompressedSize = 0, compressedSize = 0;
 
     private final MetadataCollector sstableMetadataCollector;
 
+    private final ByteBuffer crcCheckBuffer = ByteBuffer.allocate(4);
+
     public CompressedSequentialWriter(File file,
                                       String offsetsPath,
                                       CompressionParameters parameters,
                                       MetadataCollector sstableMetadataCollector)
     {
-        super(file, parameters.chunkLength());
+        super(file, parameters.chunkLength(), parameters.sstableCompressor.preferredBufferType());
         this.compressor = parameters.sstableCompressor;
 
         // buffer for compression should be the same size as buffer itself
-        compressed = new ICompressor.WrappedArray(new byte[compressor.initialCompressedBufferLength(buffer.length)]);
+        compressed = compressor.preferredBufferType().allocate(compressor.initialCompressedBufferLength(buffer.capacity()));
 
         /* Index File (-CompressionInfo.db component) and it's header */
         metadataWriter = CompressionMetadata.Writer.open(parameters, offsetsPath);
 
         this.sstableMetadataCollector = sstableMetadataCollector;
-        crcMetadata = new DataIntegrityMetadata.ChecksumWriter(out);
+        crcMetadata = new DataIntegrityMetadata.ChecksumWriter(new DataOutputStream(Channels.newOutputStream(channel)));
     }
 
     @Override
@@ -82,7 +81,7 @@
     {
         try
         {
-            return out.getFilePointer();
+            return channel.position();
         }
         catch (IOException e)
         {
@@ -91,12 +90,6 @@
     }
 
     @Override
-    public void sync()
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    @Override
     public void flush()
     {
         throw new UnsupportedOperationException();
@@ -107,18 +100,20 @@
     {
         seekToChunkStart(); // why is this necessary? seems like it should always be at chunk start in normal operation
 
-        int compressedLength;
         try
         {
             // compressing data with buffer re-use
-            compressedLength = compressor.compress(buffer, 0, validBufferBytes, compressed, 0);
+            buffer.flip();
+            compressed.clear();
+            compressor.compress(buffer, compressed);
         }
         catch (IOException e)
         {
             throw new RuntimeException("Compression exception", e); // shouldn't happen
         }
 
-        originalSize += validBufferBytes;
+        int compressedLength = compressed.position();
+        uncompressedSize += buffer.position();
         compressedSize += compressedLength;
 
         try
@@ -127,13 +122,17 @@
             metadataWriter.addOffset(chunkOffset);
             chunkCount++;
 
-            assert compressedLength <= compressed.buffer.length;
+            // write out the compressed data
+            compressed.flip();
+            channel.write(compressed);
 
-            // write data itself
-            out.write(compressed.buffer, 0, compressedLength);
             // write corresponding checksum
-            crcMetadata.append(compressed.buffer, 0, compressedLength, true);
+            compressed.rewind();
+            crcMetadata.appendDirect(compressed, true);
             lastFlushOffset += compressedLength + 4;
+
+            // adjust our bufferOffset to account for the new uncompressed data we've now written out
+            resetBuffer();
         }
         catch (IOException e)
         {
@@ -146,19 +145,17 @@
             runPostFlush.run();
     }
 
-    public CompressionMetadata open(long overrideLength, boolean isFinal)
+    public CompressionMetadata open(long overrideLength)
     {
         if (overrideLength <= 0)
-            return metadataWriter.open(originalSize, chunkOffset, isFinal ? FINAL : SHARED_FINAL);
-        // we are early opening the file, make sure we open metadata with the correct size
-        assert !isFinal;
-        return metadataWriter.open(overrideLength, chunkOffset, SHARED);
+            overrideLength = uncompressedSize;
+        return metadataWriter.open(overrideLength, chunkOffset);
     }
 
     @Override
     public FileMark mark()
     {
-        return new CompressedFileWriterMark(chunkOffset, current, validBufferBytes, chunkCount + 1);
+        return new CompressedFileWriterMark(chunkOffset, current(), buffer.position(), chunkCount + 1);
     }
 
     @Override
@@ -169,46 +166,52 @@
         CompressedFileWriterMark realMark = (CompressedFileWriterMark) mark;
 
         // reset position
-        current = realMark.uncDataOffset;
+        long truncateTarget = realMark.uncDataOffset;
 
-        if (realMark.chunkOffset == chunkOffset) // current buffer
+        if (realMark.chunkOffset == chunkOffset)
         {
-            // just reset a buffer offset and return
-            validBufferBytes = realMark.bufferOffset;
+            // simply drop bytes to the right of our mark
+            buffer.position(realMark.validBufferBytes);
             return;
         }
 
-        // synchronize current buffer with disk
-        // because we don't want any data loss
+        // synchronize current buffer with disk - we don't want any data loss
         syncInternal();
 
-        // setting marker as a current offset
         chunkOffset = realMark.chunkOffset;
 
         // compressed chunk size (- 4 bytes reserved for checksum)
         int chunkSize = (int) (metadataWriter.chunkOffsetBy(realMark.nextChunkIndex) - chunkOffset - 4);
-        if (compressed.buffer.length < chunkSize)
-            compressed.buffer = new byte[chunkSize];
+        if (compressed.capacity() < chunkSize)
+            compressed = compressor.preferredBufferType().allocate(chunkSize);
 
         try
         {
-            out.seek(chunkOffset);
-            out.readFully(compressed.buffer, 0, chunkSize);
+            compressed.clear();
+            compressed.limit(chunkSize);
+            channel.position(chunkOffset);
+            channel.read(compressed);
 
             try
             {
-                // repopulate buffer
-                compressor.uncompress(compressed.buffer, 0, chunkSize, buffer, 0);
+                // Repopulate buffer from compressed data
+                buffer.clear();
+                compressed.flip();
+                compressor.uncompress(compressed, buffer);
             }
             catch (IOException e)
             {
                 throw new CorruptBlockException(getPath(), chunkOffset, chunkSize);
             }
 
-            Checksum checksum = new Adler32();
-            checksum.update(compressed.buffer, 0, chunkSize);
+            Adler32 checksum = new Adler32();
+            compressed.rewind();
+            FBUtilities.directCheckSum(checksum, compressed);
 
-            if (out.readInt() != (int) checksum.getValue())
+            crcCheckBuffer.clear();
+            channel.read(crcCheckBuffer);
+            crcCheckBuffer.flip();
+            if (crcCheckBuffer.getInt() != (int) checksum.getValue())
                 throw new CorruptBlockException(getPath(), chunkOffset, chunkSize);
         }
         catch (CorruptBlockException e)
@@ -224,14 +227,13 @@
             throw new FSReadError(e, getPath());
         }
 
-        // reset buffer
-        validBufferBytes = realMark.bufferOffset;
-        bufferOffset = current - validBufferBytes;
-        chunkCount = realMark.nextChunkIndex - 1;
-
-        // mark as dirty so we don't lose the bytes on subsequent reBuffer calls
+        // Mark as dirty so we can guarantee the newly buffered bytes won't be lost on a rebuffer
+        buffer.position(realMark.validBufferBytes);
         isDirty = true;
 
+        bufferOffset = truncateTarget - buffer.position();
+        chunkCount = realMark.nextChunkIndex - 1;
+
         // truncate data and index file
         truncate(chunkOffset);
         metadataWriter.resetAndTruncate(realMark.nextChunkIndex - 1);
@@ -246,7 +248,7 @@
         {
             try
             {
-                out.seek(chunkOffset);
+                channel.position(chunkOffset);
             }
             catch (IOException e)
             {
@@ -255,34 +257,36 @@
         }
     }
 
-    @Override
-    public void close()
+    protected class TransactionalProxy extends SequentialWriter.TransactionalProxy
     {
-        if (buffer == null)
-            return; // already closed
-
-        super.close();
-        sstableMetadataCollector.addCompressionRatio(compressedSize, originalSize);
-        try
+        @Override
+        protected Throwable doCommit(Throwable accumulate)
         {
-            metadataWriter.close(current, chunkCount);
+            return metadataWriter.commit(accumulate);
         }
-        catch (IOException e)
+
+        @Override
+        protected Throwable doAbort(Throwable accumulate)
         {
-            throw new FSWriteError(e, getPath());
+            return super.doAbort(metadataWriter.abort(accumulate));
+        }
+
+        @Override
+        protected void doPrepare()
+        {
+            syncInternal();
+            if (descriptor != null)
+                crcMetadata.writeFullChecksum(descriptor);
+            releaseFileHandle();
+            sstableMetadataCollector.addCompressionRatio(compressedSize, uncompressedSize);
+            metadataWriter.finalizeLength(current(), chunkCount).prepareToCommit();
         }
     }
 
-    public void abort()
-    {
-        super.abort();
-        metadataWriter.abort();
-    }
-
     @Override
-    public void writeFullChecksum(Descriptor descriptor)
+    protected SequentialWriter.TransactionalProxy txnProxy()
     {
-        crcMetadata.writeFullChecksum(descriptor);
+        return new TransactionalProxy();
     }
 
     /**
@@ -295,14 +299,14 @@
         // uncompressed data offset (real data offset)
         final long uncDataOffset;
 
-        final int bufferOffset;
+        final int validBufferBytes;
         final int nextChunkIndex;
 
-        public CompressedFileWriterMark(long chunkOffset, long uncDataOffset, int bufferOffset, int nextChunkIndex)
+        public CompressedFileWriterMark(long chunkOffset, long uncDataOffset, int validBufferBytes, int nextChunkIndex)
         {
             this.chunkOffset = chunkOffset;
             this.uncDataOffset = uncDataOffset;
-            this.bufferOffset = bufferOffset;
+            this.validBufferBytes = validBufferBytes;
             this.nextChunkIndex = nextChunkIndex;
         }
     }
diff --git a/src/java/org/apache/cassandra/io/compress/CompressedThrottledReader.java b/src/java/org/apache/cassandra/io/compress/CompressedThrottledReader.java
index c5ae795..2b07c50 100644
--- a/src/java/org/apache/cassandra/io/compress/CompressedThrottledReader.java
+++ b/src/java/org/apache/cassandra/io/compress/CompressedThrottledReader.java
@@ -21,35 +21,29 @@
  */
 
 
-import java.io.FileNotFoundException;
-
 import com.google.common.util.concurrent.RateLimiter;
 
+import org.apache.cassandra.io.util.ChannelProxy;
+import org.apache.cassandra.io.util.ICompressedFile;
+
 public class CompressedThrottledReader extends CompressedRandomAccessReader
 {
     private final RateLimiter limiter;
 
-    public CompressedThrottledReader(String file, CompressionMetadata metadata, RateLimiter limiter) throws FileNotFoundException
+    public CompressedThrottledReader(ChannelProxy channel, CompressionMetadata metadata, ICompressedFile file, RateLimiter limiter)
     {
-        super(file, metadata, null);
+        super(channel, metadata, file, null);
         this.limiter = limiter;
     }
 
     protected void reBuffer()
     {
-        limiter.acquire(buffer.length);
+        limiter.acquire(buffer.capacity());
         super.reBuffer();
     }
 
-    public static CompressedThrottledReader open(String file, CompressionMetadata metadata, RateLimiter limiter)
+    public static CompressedThrottledReader open(ICompressedFile file, RateLimiter limiter)
     {
-        try
-        {
-            return new CompressedThrottledReader(file, metadata, limiter);
-        }
-        catch (FileNotFoundException e)
-        {
-            throw new RuntimeException(e);
-        }
+        return new CompressedThrottledReader(file.channel(), file.getMetadata(), file, limiter);
     }
 }
diff --git a/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java b/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java
index 0de69a6..45fb0e0 100644
--- a/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java
+++ b/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java
@@ -36,6 +36,7 @@
 import java.util.TreeSet;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Throwables;
 import com.google.common.primitives.Longs;
 
 import org.apache.cassandra.db.TypeSizes;
@@ -51,6 +52,7 @@
 import org.apache.cassandra.io.util.Memory;
 import org.apache.cassandra.io.util.SafeMemory;
 import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.concurrent.Transactional;
 
 /**
  * Holds metadata about compressed file
@@ -62,7 +64,6 @@
     // (when early opening, we want to ensure readers cannot read past fully written sections)
     public final long dataLength;
     public final long compressedFileLength;
-    public final boolean hasPostCompressionAdlerChecksums;
     private final Memory chunkOffsets;
     private final long chunkOffsetsSize;
     public final String indexFilePath;
@@ -82,30 +83,19 @@
     public static CompressionMetadata create(String dataFilePath)
     {
         Descriptor desc = Descriptor.fromFilename(dataFilePath);
-        return new CompressionMetadata(desc.filenameFor(Component.COMPRESSION_INFO), new File(dataFilePath).length(), desc.version.hasPostCompressionAdlerChecksums);
+        return new CompressionMetadata(desc.filenameFor(Component.COMPRESSION_INFO), new File(dataFilePath).length());
     }
 
     @VisibleForTesting
-    CompressionMetadata(String indexFilePath, long compressedLength, boolean hasPostCompressionAdlerChecksums)
+    CompressionMetadata(String indexFilePath, long compressedLength)
     {
         this.indexFilePath = indexFilePath;
-        this.hasPostCompressionAdlerChecksums = hasPostCompressionAdlerChecksums;
 
-        DataInputStream stream;
-        try
-        {
-            stream = new DataInputStream(new FileInputStream(indexFilePath));
-        }
-        catch (FileNotFoundException e)
-        {
-            throw new RuntimeException(e);
-        }
-
-        try
+        try (DataInputStream stream = new DataInputStream(new FileInputStream(indexFilePath)))
         {
             String compressorName = stream.readUTF();
             int optionCount = stream.readInt();
-            Map<String, String> options = new HashMap<>();
+            Map<String, String> options = new HashMap<>(optionCount);
             for (int i = 0; i < optionCount; ++i)
             {
                 String key = stream.readUTF();
@@ -126,24 +116,24 @@
             compressedFileLength = compressedLength;
             chunkOffsets = readChunkOffsets(stream);
         }
+        catch (FileNotFoundException e)
+        {
+            throw new RuntimeException(e);
+        }
         catch (IOException e)
         {
             throw new CorruptSSTableException(e, indexFilePath);
         }
-        finally
-        {
-            FileUtils.closeQuietly(stream);
-        }
+
         this.chunkOffsetsSize = chunkOffsets.size();
     }
 
-    private CompressionMetadata(String filePath, CompressionParameters parameters, SafeMemory offsets, long offsetsSize, long dataLength, long compressedLength, boolean hasPostCompressionAdlerChecksums)
+    private CompressionMetadata(String filePath, CompressionParameters parameters, SafeMemory offsets, long offsetsSize, long dataLength, long compressedLength)
     {
         this.indexFilePath = filePath;
         this.parameters = parameters;
         this.dataLength = dataLength;
         this.compressedFileLength = compressedLength;
-        this.hasPostCompressionAdlerChecksums = hasPostCompressionAdlerChecksums;
         this.chunkOffsets = offsets;
         this.chunkOffsetsSize = offsetsSize;
     }
@@ -176,32 +166,42 @@
      */
     private Memory readChunkOffsets(DataInput input)
     {
+        final int chunkCount;
         try
         {
-            int chunkCount = input.readInt();
+            chunkCount = input.readInt();
             if (chunkCount <= 0)
                 throw new IOException("Compressed file with 0 chunks encountered: " + input);
+        }
+        catch (IOException e)
+        {
+            throw new FSReadError(e, indexFilePath);
+        }
 
-            Memory offsets = Memory.allocate(chunkCount * 8L);
+        @SuppressWarnings("resource")
+        Memory offsets = Memory.allocate(chunkCount * 8L);
+        int i = 0;
+        try
+        {
 
-            for (int i = 0; i < chunkCount; i++)
+            for (i = 0; i < chunkCount; i++)
             {
-                try
-                {
-                    offsets.setLong(i * 8, input.readLong());
-                }
-                catch (EOFException e)
-                {
-                    String msg = String.format("Corrupted Index File %s: read %d but expected %d chunks.",
-                                               indexFilePath, i, chunkCount);
-                    throw new CorruptSSTableException(new IOException(msg, e), indexFilePath);
-                }
+                offsets.setLong(i * 8L, input.readLong());
             }
 
             return offsets;
         }
         catch (IOException e)
         {
+            if (offsets != null)
+                offsets.close();
+
+            if (e instanceof EOFException)
+            {
+                String msg = String.format("Corrupted Index File %s: read %d but expected %d chunks.",
+                                           indexFilePath, i, chunkCount);
+                throw new CorruptSSTableException(new IOException(msg, e), indexFilePath);
+            }
             throw new FSReadError(e, indexFilePath);
         }
     }
@@ -295,7 +295,7 @@
         chunkOffsets.close();
     }
 
-    public static class Writer
+    public static class Writer extends Transactional.AbstractTransactional implements Transactional
     {
         // path to the file
         private final CompressionParameters parameters;
@@ -304,6 +304,9 @@
         private SafeMemory offsets = new SafeMemory(maxCount * 8L);
         private int count = 0;
 
+        // provided by user when setDescriptor
+        private long dataLength, chunkCount;
+
         private Writer(CompressionParameters parameters, String path)
         {
             this.parameters = parameters;
@@ -319,7 +322,7 @@
         {
             if (count == maxCount)
             {
-                SafeMemory newOffsets = offsets.copy((maxCount *= 2L) * 8);
+                SafeMemory newOffsets = offsets.copy((maxCount *= 2L) * 8L);
                 offsets.close();
                 offsets = newOffsets;
             }
@@ -350,62 +353,60 @@
             }
         }
 
-        static enum OpenType
+        // we've written everything; wire up some final metadata state
+        public Writer finalizeLength(long dataLength, int chunkCount)
         {
-            // i.e. FinishType == EARLY; we will use the RefCountedMemory in possibly multiple instances
-            SHARED,
-            // i.e. FinishType == EARLY, but the sstable has been completely written, so we can
-            // finalise the contents and size of the memory, but must retain a reference to it
-            SHARED_FINAL,
-            // i.e. FinishType == NORMAL or FINISH_EARLY, i.e. we have actually finished writing the table
-            // and will never need to open the metadata again, so we can release any references to it here
-            FINAL
+            this.dataLength = dataLength;
+            this.chunkCount = chunkCount;
+            return this;
         }
 
-        public CompressionMetadata open(long dataLength, long compressedLength, OpenType type)
+        public void doPrepare()
         {
-            SafeMemory offsets;
-            int count = this.count;
-            switch (type)
+            assert chunkCount == count;
+
+            // finalize the size of memory used if it won't now change;
+            // unnecessary if already correct size
+            if (offsets.size() != count * 8L)
             {
-                case FINAL: case SHARED_FINAL:
-                    if (this.offsets.size() != count * 8L)
-                    {
-                        // finalize the size of memory used if it won't now change;
-                        // unnecessary if already correct size
-                        SafeMemory tmp = this.offsets.copy(count * 8L);
-                        this.offsets.free();
-                        this.offsets = tmp;
-                    }
-
-                    if (type == OpenType.SHARED_FINAL)
-                    {
-                        offsets = this.offsets.sharedCopy();
-                    }
-                    else
-                    {
-                        offsets = this.offsets;
-                        // null out our reference to the original shared data to catch accidental reuse
-                        // note that since noone is writing to this Writer while we open it, null:ing out this.offsets is safe
-                        this.offsets = null;
-                    }
-                    break;
-
-                case SHARED:
-                    offsets = this.offsets.sharedCopy();
-                    // we should only be opened on a compression data boundary; truncate our size to this boundary
-                    count = (int) (dataLength / parameters.chunkLength());
-                    if (dataLength % parameters.chunkLength() != 0)
-                        count++;
-                    // grab our actual compressed length from the next offset from our the position we're opened to
-                    if (count < this.count)
-                        compressedLength = offsets.getLong(count * 8L);
-                    break;
-
-                default:
-                    throw new AssertionError();
+                SafeMemory tmp = offsets;
+                offsets = offsets.copy(count * 8L);
+                tmp.free();
             }
-            return new CompressionMetadata(filePath, parameters, offsets, count * 8L, dataLength, compressedLength, Descriptor.Version.CURRENT.hasPostCompressionAdlerChecksums);
+
+            // flush the data to disk
+            try (FileOutputStream fos = new FileOutputStream(filePath);
+                 DataOutputStream out = new DataOutputStream(new BufferedOutputStream(fos)))
+            {
+                writeHeader(out, dataLength, count);
+                for (int i = 0; i < count; i++)
+                    out.writeLong(offsets.getLong(i * 8L));
+
+                out.flush();
+                fos.getFD().sync();
+            }
+            catch (IOException e)
+            {
+                throw Throwables.propagate(e);
+            }
+        }
+
+        @SuppressWarnings("resource")
+        public CompressionMetadata open(long dataLength, long compressedLength)
+        {
+            SafeMemory offsets = this.offsets.sharedCopy();
+
+            // calculate how many entries we need, if our dataLength is truncated
+            int count = (int) (dataLength / parameters.chunkLength());
+            if (dataLength % parameters.chunkLength() != 0)
+                count++;
+
+            assert count > 0;
+            // grab our actual compressed length from the next offset from our the position we're opened to
+            if (count < this.count)
+                compressedLength = offsets.getLong(count * 8L);
+
+            return new CompressionMetadata(filePath, parameters, offsets, count * 8L, dataLength, compressedLength);
         }
 
         /**
@@ -423,38 +424,27 @@
         /**
          * Reset the writer so that the next chunk offset written will be the
          * one of {@code chunkIndex}.
+         * 
+         * @param chunkIndex the next index to write
          */
         public void resetAndTruncate(int chunkIndex)
         {
             count = chunkIndex;
         }
 
-        public void close(long dataLength, int chunks) throws IOException
+        protected Throwable doPreCleanup(Throwable failed)
         {
-            FileOutputStream fos = null;
-            DataOutputStream out = null;
-            try
-            {
-                fos = new FileOutputStream(filePath);
-                out = new DataOutputStream(new BufferedOutputStream(fos));
-                assert chunks == count;
-                writeHeader(out, dataLength, chunks);
-                for (int i = 0 ; i < count ; i++)
-                    out.writeLong(offsets.getLong(i * 8L));
-
-                out.flush();
-                fos.getFD().sync();
-            }
-            finally
-            {
-                FileUtils.closeQuietly(out);
-            }
+            return offsets.close(failed);
         }
 
-        public void abort()
+        protected Throwable doCommit(Throwable accumulate)
         {
-            if (offsets != null)
-                offsets.close();
+            return accumulate;
+        }
+
+        protected Throwable doAbort(Throwable accumulate)
+        {
+            return FileUtils.deleteWithConfirm(filePath, false, accumulate);
         }
     }
 
diff --git a/src/java/org/apache/cassandra/io/compress/CompressionParameters.java b/src/java/org/apache/cassandra/io/compress/CompressionParameters.java
index 35191da..b114826 100644
--- a/src/java/org/apache/cassandra/io/compress/CompressionParameters.java
+++ b/src/java/org/apache/cassandra/io/compress/CompressionParameters.java
@@ -21,20 +21,21 @@
 import java.io.IOException;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
-import java.util.Collections;
 import java.util.AbstractSet;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
 
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Sets;
-import org.apache.cassandra.config.CFMetaData;
+
 import org.apache.commons.lang3.builder.EqualsBuilder;
 import org.apache.commons.lang3.builder.HashCodeBuilder;
-
-import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ParameterizedClass;
 import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataOutputPlus;
 
@@ -93,14 +94,7 @@
 
     public CompressionParameters copy()
     {
-        try
-        {
-            return new CompressionParameters(sstableCompressor, chunkLength, new HashMap<>(otherOptions));
-        }
-        catch (ConfigurationException e)
-        {
-            throw new AssertionError(e); // can't happen at this point.
-        }
+        return new CompressionParameters(sstableCompressor, chunkLength, new HashMap<>(otherOptions));
     }
 
     public void setLiveMetadata(final CFMetaData liveMetadata)
@@ -150,7 +144,7 @@
         return chunkLength == null ? DEFAULT_CHUNK_LENGTH : chunkLength;
     }
 
-    private static Class<? extends ICompressor> parseCompressorClass(String className) throws ConfigurationException
+    private static Class<?> parseCompressorClass(String className) throws ConfigurationException
     {
         if (className == null || className.isEmpty())
             return null;
@@ -158,7 +152,7 @@
         className = className.contains(".") ? className : "org.apache.cassandra.io.compress." + className;
         try
         {
-            return (Class<? extends ICompressor>)Class.forName(className);
+            return Class.forName(className);
         }
         catch (Exception e)
         {
@@ -166,7 +160,7 @@
         }
     }
 
-    private static ICompressor createCompressor(Class<? extends ICompressor> compressorClass, Map<String, String> compressionOptions) throws ConfigurationException
+    private static ICompressor createCompressor(Class<?> compressorClass, Map<String, String> compressionOptions) throws ConfigurationException
     {
         if (compressorClass == null)
         {
@@ -212,6 +206,10 @@
         }
     }
 
+    public static ICompressor createCompressor(ParameterizedClass compression) throws ConfigurationException {
+        return createCompressor(parseCompressorClass(compression.class_name), copyOptions(compression.parameters));
+    }
+
     private static Map<String, String> copyOptions(Map<? extends CharSequence, ? extends CharSequence> co)
     {
         if (co == null || co.isEmpty())
@@ -227,6 +225,10 @@
 
     /**
      * Parse the chunk length (in KB) and returns it as bytes.
+     * 
+     * @param chLengthKB the length of the chunk to parse
+     * @return the chunk length in bytes
+     * @throws ConfigurationException if the chunk size is too large
      */
     public static Integer parseChunkLength(String chLengthKB) throws ConfigurationException
     {
diff --git a/src/java/org/apache/cassandra/io/compress/DeflateCompressor.java b/src/java/org/apache/cassandra/io/compress/DeflateCompressor.java
index 125a08f..f2ccd64 100644
--- a/src/java/org/apache/cassandra/io/compress/DeflateCompressor.java
+++ b/src/java/org/apache/cassandra/io/compress/DeflateCompressor.java
@@ -17,7 +17,10 @@
  */
 package org.apache.cassandra.io.compress;
 
+import org.apache.cassandra.utils.FBUtilities;
+
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.Collections;
 import java.util.Map;
 import java.util.Set;
@@ -63,12 +66,26 @@
         return Collections.emptySet();
     }
 
-    public int initialCompressedBufferLength(int chunkLength)
+    public int initialCompressedBufferLength(int sourceLen)
     {
-        return chunkLength;
+        // Taken from zlib deflateBound(). See http://www.zlib.net/zlib_tech.html.
+        return sourceLen + (sourceLen >> 12) + (sourceLen >> 14) + (sourceLen >> 25) + 13;
     }
 
-    public int compress(byte[] input, int inputOffset, int inputLength, ICompressor.WrappedArray output, int outputOffset)
+    public void compress(ByteBuffer input, ByteBuffer output)
+    {
+        if (input.hasArray() && output.hasArray())
+        {
+            int length = compressArray(input.array(), input.arrayOffset() + input.position(), input.remaining(),
+                                       output.array(), output.arrayOffset() + output.position(), output.remaining());
+            input.position(input.limit());
+            output.position(output.position() + length);
+        }
+        else
+            compressBuffer(input, output);
+    }
+
+    public int compressArray(byte[] input, int inputOffset, int inputLength, byte[] output, int outputOffset, int maxOutputLength)
     {
         Deflater def = deflater.get();
         def.reset();
@@ -77,26 +94,96 @@
         if (def.needsInput())
             return 0;
 
-        int offs = outputOffset;
-        while (true)
+        int len = def.deflate(output, outputOffset, maxOutputLength);
+        assert def.finished();
+        return len;
+    }
+
+    public void compressBuffer(ByteBuffer input, ByteBuffer output)
+    {
+        Deflater def = deflater.get();
+        def.reset();
+
+        byte[] buffer = FBUtilities.getThreadLocalScratchBuffer();
+        // Use half the buffer for input, half for output.
+        int chunkLen = buffer.length / 2;
+        while (input.remaining() > chunkLen)
         {
-            offs += def.deflate(output.buffer, offs, output.buffer.length - offs);
-            if (def.finished())
+            input.get(buffer, 0, chunkLen);
+            def.setInput(buffer, 0, chunkLen);
+            while (!def.needsInput())
             {
-                return offs - outputOffset;
+                int len = def.deflate(buffer, chunkLen, chunkLen);
+                output.put(buffer, chunkLen, len);
             }
-            else
+        }
+        int inputLength = input.remaining();
+        input.get(buffer, 0, inputLength);
+        def.setInput(buffer, 0, inputLength);
+        def.finish();
+        while (!def.finished())
+        {
+            int len = def.deflate(buffer, chunkLen, chunkLen);
+            output.put(buffer, chunkLen, len);
+        }
+    }
+
+
+    public void uncompress(ByteBuffer input, ByteBuffer output) throws IOException
+    {
+        if (input.hasArray() && output.hasArray())
+        {
+            int length = uncompress(input.array(), input.arrayOffset() + input.position(), input.remaining(),
+                                    output.array(), output.arrayOffset() + output.position(), output.remaining());
+            input.position(input.limit());
+            output.position(output.position() + length);
+        }
+        else
+            uncompressBuffer(input, output);
+    }
+
+    public void uncompressBuffer(ByteBuffer input, ByteBuffer output) throws IOException
+    {
+        try
+        {
+            Inflater inf = inflater.get();
+            inf.reset();
+
+            byte[] buffer = FBUtilities.getThreadLocalScratchBuffer();
+            // Use half the buffer for input, half for output.
+            int chunkLen = buffer.length / 2;
+            while (input.remaining() > chunkLen)
             {
-                // We're not done, output was too small. Increase it and continue
-                byte[] newBuffer = new byte[(output.buffer.length*4)/3 + 1];
-                System.arraycopy(output.buffer, 0, newBuffer, 0, offs);
-                output.buffer = newBuffer;
+                input.get(buffer, 0, chunkLen);
+                inf.setInput(buffer, 0, chunkLen);
+                while (!inf.needsInput())
+                {
+                    int len = inf.inflate(buffer, chunkLen, chunkLen);
+                    output.put(buffer, chunkLen, len);
+                }
             }
+            int inputLength = input.remaining();
+            input.get(buffer, 0, inputLength);
+            inf.setInput(buffer, 0, inputLength);
+            while (!inf.needsInput())
+            {
+                int len = inf.inflate(buffer, chunkLen, chunkLen);
+                output.put(buffer, chunkLen, len);
+            }
+        }
+        catch (DataFormatException e)
+        {
+            throw new IOException(e);
         }
     }
 
     public int uncompress(byte[] input, int inputOffset, int inputLength, byte[] output, int outputOffset) throws IOException
     {
+        return uncompress(input, inputOffset, inputLength, output, outputOffset, output.length - outputOffset);
+    }
+
+    public int uncompress(byte[] input, int inputOffset, int inputLength, byte[] output, int outputOffset, int maxOutputLength) throws IOException
+    {
         Inflater inf = inflater.get();
         inf.reset();
         inf.setInput(input, inputOffset, inputLength);
@@ -106,11 +193,22 @@
         // We assume output is big enough
         try
         {
-            return inf.inflate(output, outputOffset, output.length - outputOffset);
+            return inf.inflate(output, outputOffset, maxOutputLength);
         }
         catch (DataFormatException e)
         {
             throw new IOException(e);
         }
     }
+
+    public boolean supports(BufferType bufferType)
+    {
+        return true;
+    }
+
+    public BufferType preferredBufferType()
+    {
+        // Prefer array-backed buffers.
+        return BufferType.ON_HEAP;
+    }
 }
diff --git a/src/java/org/apache/cassandra/io/compress/ICompressor.java b/src/java/org/apache/cassandra/io/compress/ICompressor.java
index be76bc5..5719834 100644
--- a/src/java/org/apache/cassandra/io/compress/ICompressor.java
+++ b/src/java/org/apache/cassandra/io/compress/ICompressor.java
@@ -18,33 +18,43 @@
 package org.apache.cassandra.io.compress;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.Set;
 
 public interface ICompressor
 {
     public int initialCompressedBufferLength(int chunkLength);
 
-    public int compress(byte[] input, int inputOffset, int inputLength, WrappedArray output, int outputOffset) throws IOException;
-
     public int uncompress(byte[] input, int inputOffset, int inputLength, byte[] output, int outputOffset) throws IOException;
 
-    public Set<String> supportedOptions();
+    /**
+     * Compression for ByteBuffers.
+     *
+     * The data between input.position() and input.limit() is compressed and placed into output starting from output.position().
+     * Positions in both buffers are moved to reflect the bytes read and written. Limits are not changed.
+     */
+    public void compress(ByteBuffer input, ByteBuffer output) throws IOException;
 
     /**
-     * A simple wrapper of a byte array.
-     * Not all implementation allows to know what is the maximum size after
-     * compression. This make it hard to size the ouput buffer for compress
-     * (and we want to reuse the buffer).  Instead we use this wrapped buffer
-     * so that compress can have the liberty to resize underlying array if
-     * need be.
+     * Decompression for DirectByteBuffers.
+     *
+     * The data between input.position() and input.limit() is uncompressed and placed into output starting from output.position().
+     * Positions in both buffers are moved to reflect the bytes read and written. Limits are not changed.
      */
-    public static class WrappedArray
-    {
-        public byte[] buffer;
+    public void uncompress(ByteBuffer input, ByteBuffer output) throws IOException;
 
-        public WrappedArray(byte[] buffer)
-        {
-            this.buffer = buffer;
-        }
-    }
+    /**
+     * Returns the preferred (most efficient) buffer type for this compressor.
+     */
+    public BufferType preferredBufferType();
+
+    /**
+     * Checks if the given buffer would be supported by the compressor. If a type is supported the compressor must be
+     * able to use it in combination with all other supported types.
+     *
+     * Direct and memory-mapped buffers must be supported by all compressors.
+     */
+    public boolean supports(BufferType bufferType);
+
+    public Set<String> supportedOptions();
 }
diff --git a/src/java/org/apache/cassandra/io/compress/LZ4Compressor.java b/src/java/org/apache/cassandra/io/compress/LZ4Compressor.java
index 6f56f30..5fd4309 100644
--- a/src/java/org/apache/cassandra/io/compress/LZ4Compressor.java
+++ b/src/java/org/apache/cassandra/io/compress/LZ4Compressor.java
@@ -18,19 +18,18 @@
 package org.apache.cassandra.io.compress;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 
 import com.google.common.annotations.VisibleForTesting;
-
 import net.jpountz.lz4.LZ4Exception;
 import net.jpountz.lz4.LZ4Factory;
 
 public class LZ4Compressor implements ICompressor
 {
-
     private static final int INTEGER_BYTES = 4;
 
     @VisibleForTesting
@@ -42,13 +41,13 @@
     }
 
     private final net.jpountz.lz4.LZ4Compressor compressor;
-    private final net.jpountz.lz4.LZ4Decompressor decompressor;
+    private final net.jpountz.lz4.LZ4FastDecompressor decompressor;
 
     private LZ4Compressor()
     {
         final LZ4Factory lz4Factory = LZ4Factory.fastestInstance();
         compressor = lz4Factory.fastCompressor();
-        decompressor = lz4Factory.decompressor();
+        decompressor = lz4Factory.fastDecompressor();
     }
 
     public int initialCompressedBufferLength(int chunkLength)
@@ -56,18 +55,17 @@
         return INTEGER_BYTES + compressor.maxCompressedLength(chunkLength);
     }
 
-    public int compress(byte[] input, int inputOffset, int inputLength, WrappedArray output, int outputOffset) throws IOException
+    public void compress(ByteBuffer input, ByteBuffer output) throws IOException
     {
-        final byte[] dest = output.buffer;
-        dest[outputOffset] = (byte) inputLength;
-        dest[outputOffset + 1] = (byte) (inputLength >>> 8);
-        dest[outputOffset + 2] = (byte) (inputLength >>> 16);
-        dest[outputOffset + 3] = (byte) (inputLength >>> 24);
-        final int maxCompressedLength = compressor.maxCompressedLength(inputLength);
+        int len = input.remaining();
+        output.put((byte) len);
+        output.put((byte) (len >>> 8));
+        output.put((byte) (len >>> 16));
+        output.put((byte) (len >>> 24));
+
         try
         {
-            return INTEGER_BYTES + compressor.compress(input, inputOffset, inputLength,
-                                                       dest, outputOffset + INTEGER_BYTES, maxCompressedLength);
+            compressor.compress(input, output);
         }
         catch (LZ4Exception e)
         {
@@ -82,6 +80,7 @@
                 | ((input[inputOffset + 1] & 0xFF) << 8)
                 | ((input[inputOffset + 2] & 0xFF) << 16)
                 | ((input[inputOffset + 3] & 0xFF) << 24);
+
         final int compressedLength;
         try
         {
@@ -101,8 +100,42 @@
         return decompressedLength;
     }
 
+    public void uncompress(ByteBuffer input, ByteBuffer output) throws IOException
+    {
+        final int decompressedLength = (input.get() & 0xFF)
+                | ((input.get() & 0xFF) << 8)
+                | ((input.get() & 0xFF) << 16)
+                | ((input.get() & 0xFF) << 24);
+
+        try
+        {
+            int compressedLength = decompressor.decompress(input, input.position(), output, output.position(), decompressedLength);
+            input.position(input.position() + compressedLength);
+            output.position(output.position() + decompressedLength);
+        }
+        catch (LZ4Exception e)
+        {
+            throw new IOException(e);
+        }
+
+        if (input.remaining() > 0)
+        {
+            throw new IOException("Compressed lengths mismatch - "+input.remaining()+" bytes remain");
+        }
+    }
+
     public Set<String> supportedOptions()
     {
-        return new HashSet<String>(Arrays.asList(CompressionParameters.CRC_CHECK_CHANCE));
+        return new HashSet<>(Arrays.asList(CompressionParameters.CRC_CHECK_CHANCE));
+    }
+
+    public BufferType preferredBufferType()
+    {
+        return BufferType.OFF_HEAP;
+    }
+
+    public boolean supports(BufferType bufferType)
+    {
+        return true;
     }
 }
diff --git a/src/java/org/apache/cassandra/io/compress/SnappyCompressor.java b/src/java/org/apache/cassandra/io/compress/SnappyCompressor.java
index 3583201..9fc170a 100644
--- a/src/java/org/apache/cassandra/io/compress/SnappyCompressor.java
+++ b/src/java/org/apache/cassandra/io/compress/SnappyCompressor.java
@@ -18,6 +18,7 @@
 package org.apache.cassandra.io.compress;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.Collections;
 import java.util.Map;
 import java.util.Set;
@@ -62,15 +63,7 @@
             JVMStabilityInspector.inspectThrowable(e);
             return false;
         }
-        catch (NoClassDefFoundError e)
-        {
-            return false;
-        }
-        catch (SnappyError e)
-        {
-            return false;
-        }
-        catch (UnsatisfiedLinkError e)
+        catch (NoClassDefFoundError | SnappyError | UnsatisfiedLinkError e)
         {
             return false;
         }
@@ -86,13 +79,43 @@
         return Snappy.maxCompressedLength(chunkLength);
     }
 
-    public int compress(byte[] input, int inputOffset, int inputLength, ICompressor.WrappedArray output, int outputOffset) throws IOException
+    public void compress(ByteBuffer input, ByteBuffer output) throws IOException
     {
-        return Snappy.rawCompress(input, inputOffset, inputLength, output.buffer, outputOffset);
+        int dlimit = output.limit();
+        Snappy.compress(input, output);
+
+        // Snappy doesn't match the ICompressor contract w/regards to state it leaves dest ByteBuffer's counters in
+        output.position(output.limit());
+        output.limit(dlimit);
+        input.position(input.limit());
     }
 
     public int uncompress(byte[] input, int inputOffset, int inputLength, byte[] output, int outputOffset) throws IOException
     {
         return Snappy.rawUncompress(input, inputOffset, inputLength, output, outputOffset);
     }
+
+    public void uncompress(ByteBuffer input, ByteBuffer output)
+            throws IOException
+    {
+        int dlimit = output.limit();
+        Snappy.uncompress(input, output);
+
+        // Snappy doesn't match the ICompressor contract w/regards to state it leaves dest ByteBuffer's counters in
+        output.position(output.limit());
+        output.limit(dlimit);
+        input.position(input.limit());
+    }
+
+    public BufferType preferredBufferType()
+    {
+        return BufferType.OFF_HEAP;
+    }
+
+    public boolean supports(BufferType bufferType)
+    {
+        // Snappy can't deal with different input and output buffer types.
+        // To avoid possible problems, pretend it can't support array-backed at all.
+        return bufferType == BufferType.OFF_HEAP;
+    }
 }
diff --git a/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java b/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
index 557c3de..6896062 100644
--- a/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
@@ -31,7 +31,8 @@
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.context.CounterContext;
 import org.apache.cassandra.dht.IPartitioner;
-import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
 import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.utils.CounterId;
 import org.apache.cassandra.utils.Pair;
@@ -44,8 +45,10 @@
     protected ColumnFamily columnFamily;
     protected ByteBuffer currentSuperColumn;
     protected final CounterId counterid = CounterId.generate();
+    private SSTableFormat.Type formatType = DatabaseDescriptor.getSSTableFormat();
     protected static AtomicInteger generation = new AtomicInteger(0);
 
+
     public AbstractSSTableSimpleWriter(File directory, CFMetaData metadata, IPartitioner partitioner)
     {
         this.metadata = metadata;
@@ -53,21 +56,25 @@
         DatabaseDescriptor.setPartitioner(partitioner);
     }
 
-    protected SSTableWriter getWriter()
+    protected void setSSTableFormatType(SSTableFormat.Type type)
     {
-        return new SSTableWriter(
-            makeFilename(directory, metadata.ksName, metadata.cfName),
-            0, // We don't care about the bloom filter
-            ActiveRepairService.UNREPAIRED_SSTABLE,
-            metadata,
-            DatabaseDescriptor.getPartitioner(),
-            new MetadataCollector(metadata.comparator));
+        this.formatType = type;
     }
 
-    // find available generation and pick up filename from that
-    protected static String makeFilename(File directory, final String keyspace, final String columnFamily)
+    protected SSTableWriter getWriter()
     {
-        final Set<Descriptor> existing = new HashSet<Descriptor>();
+        return SSTableWriter.create(createDescriptor(directory, metadata.ksName, metadata.cfName, formatType), 0, ActiveRepairService.UNREPAIRED_SSTABLE);
+    }
+
+    protected static Descriptor createDescriptor(File directory, final String keyspace, final String columnFamily, final SSTableFormat.Type fmt)
+    {
+        int maxGen = getNextGeneration(directory, columnFamily);
+        return new Descriptor(directory, keyspace, columnFamily, maxGen + 1, Descriptor.Type.TEMP, fmt);
+    }
+
+    private static int getNextGeneration(File directory, final String columnFamily)
+    {
+        final Set<Descriptor> existing = new HashSet<>();
         directory.list(new FilenameFilter()
         {
             public boolean accept(File dir, String name)
@@ -91,8 +98,7 @@
                 maxGen = generation.getAndIncrement();
             }
         }
-
-        return new Descriptor(directory, keyspace, columnFamily, maxGen + 1, Descriptor.Type.TEMP).filenameFor(Component.DATA);
+        return maxGen;
     }
 
     /**
@@ -115,7 +121,7 @@
     public void newSuperColumn(ByteBuffer name)
     {
         if (!columnFamily.metadata().isSuper())
-            throw new IllegalStateException("Cannot add a super column to a standard column family");
+            throw new IllegalStateException("Cannot add a super column to a standard table");
 
         currentSuperColumn = name;
     }
diff --git a/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java b/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
index 838ea48..33ddbe0 100644
--- a/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
@@ -21,26 +21,28 @@
 import java.io.File;
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.util.Arrays;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
+import java.util.*;
 
 import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Iterables;
 
-import org.apache.cassandra.cql3.statements.*;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.*;
-import org.apache.cassandra.config.*;
-import org.apache.cassandra.db.*;
+import org.apache.cassandra.cql3.statements.CreateTableStatement;
+import org.apache.cassandra.cql3.statements.ParsedStatement;
+import org.apache.cassandra.cql3.statements.UpdateStatement;
+import org.apache.cassandra.db.ArrayBackedSortedColumns;
+import org.apache.cassandra.db.Cell;
+import org.apache.cassandra.db.ColumnFamily;
 import org.apache.cassandra.db.composites.Composite;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Murmur3Partitioner;
-import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.RequestValidationException;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
 import org.apache.cassandra.locator.AbstractReplicationStrategy;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.utils.Pair;
@@ -291,7 +293,9 @@
     public static class Builder
     {
         private File directory;
-        private IPartitioner partitioner = new Murmur3Partitioner();
+        private IPartitioner partitioner = Murmur3Partitioner.instance;
+
+        protected SSTableFormat.Type formatType = null;
 
         private CFMetaData schema;
         private UpdateStatement insert;
@@ -300,7 +304,7 @@
         private boolean sorted = false;
         private long bufferSizeInMB = 128;
 
-        private Builder() {}
+        protected Builder() {}
 
         /**
          * The directory where to write the sstables.
@@ -386,32 +390,32 @@
         }
 
         /**
-         * Adds the specified column family to the specified keyspace.
+         * Creates the keyspace with the specified table.
          *
-         * @param ksm the keyspace meta data
-         * @param cfm the column family meta data
+         * @param the table the table that must be created.
          */
-        private static void addTableToKeyspace(KSMetaData ksm, CFMetaData cfm)
+        private static void createKeyspaceWithTable(CFMetaData table)
         {
-            ksm = KSMetaData.cloneWith(ksm, Iterables.concat(ksm.cfMetaData().values(), Collections.singleton(cfm)));
-            Schema.instance.load(cfm);
-            Schema.instance.setKeyspaceDefinition(ksm);
+            KSMetaData ksm;
+            ksm = KSMetaData.newKeyspace(table.ksName,
+                                         AbstractReplicationStrategy.getClass("org.apache.cassandra.locator.SimpleStrategy"),
+                                         ImmutableMap.of("replication_factor", "1"),
+                                         true,
+                                         Collections.singleton(table));
+            Schema.instance.load(ksm);
         }
 
         /**
-         * Creates a keyspace for the specified column family.
+         * Adds the table to the to the specified keyspace.
          *
-         * @param cfm the column family
-         * @throws ConfigurationException if a problem occurs while creating the keyspace.
+         * @param keyspace the keyspace to add to
+         * @param table the table to add
          */
-        private static void createKeyspaceWithTable(CFMetaData cfm) throws ConfigurationException
+        private static void addTableToKeyspace(KSMetaData keyspace, CFMetaData table)
         {
-            KSMetaData ksm = KSMetaData.newKeyspace(cfm.ksName,
-                                                    AbstractReplicationStrategy.getClass("org.apache.cassandra.locator.SimpleStrategy"),
-                                                    ImmutableMap.of("replication_factor", "1"),
-                                                    true,
-                                                    Collections.singleton(cfm));
-            Schema.instance.load(ksm);
+            KSMetaData clone = keyspace.cloneWithTableAdded(table);
+            Schema.instance.load(table);
+            Schema.instance.setKeyspaceDefinition(clone);
         }
 
         /**
@@ -526,6 +530,7 @@
             }
         }
 
+        @SuppressWarnings("resource")
         public CQLSSTableWriter build()
         {
             if (directory == null)
@@ -538,6 +543,10 @@
             AbstractSSTableSimpleWriter writer = sorted
                                                ? new SSTableSimpleWriter(directory, schema, partitioner)
                                                : new BufferedWriter(directory, schema, partitioner, bufferSizeInMB);
+
+            if (formatType != null)
+                writer.setSSTableFormatType(formatType);
+
             return new CQLSSTableWriter(writer, insert, boundNames);
         }
     }
diff --git a/src/java/org/apache/cassandra/io/sstable/ColumnNameHelper.java b/src/java/org/apache/cassandra/io/sstable/ColumnNameHelper.java
index 436975b..846634a 100644
--- a/src/java/org/apache/cassandra/io/sstable/ColumnNameHelper.java
+++ b/src/java/org/apache/cassandra/io/sstable/ColumnNameHelper.java
@@ -21,7 +21,6 @@
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.composites.CellNameType;
 import org.apache.cassandra.db.composites.Composite;
 import org.apache.cassandra.db.marshal.AbstractType;
diff --git a/src/java/org/apache/cassandra/io/sstable/ColumnStats.java b/src/java/org/apache/cassandra/io/sstable/ColumnStats.java
index 83a16c9..a1cb199 100644
--- a/src/java/org/apache/cassandra/io/sstable/ColumnStats.java
+++ b/src/java/org/apache/cassandra/io/sstable/ColumnStats.java
@@ -83,7 +83,7 @@
             }
             else
             {
-                if (value <this.value)
+                if (value < this.value)
                     this.value = value;
             }
         }
@@ -149,7 +149,7 @@
             }
             else
             {
-                if (value >this.value)
+                if (value > this.value)
                     this.value = value;
             }
         }
diff --git a/src/java/org/apache/cassandra/io/sstable/Component.java b/src/java/org/apache/cassandra/io/sstable/Component.java
index 3eacd07..a431f29 100644
--- a/src/java/org/apache/cassandra/io/sstable/Component.java
+++ b/src/java/org/apache/cassandra/io/sstable/Component.java
@@ -47,8 +47,8 @@
         COMPRESSION_INFO("CompressionInfo.db"),
         // statistical metadata about the content of the sstable
         STATS("Statistics.db"),
-        // holds sha1 sum of the data file (to be checked by sha1sum)
-        DIGEST("Digest.sha1"),
+        // holds adler32 checksum of the data file
+        DIGEST("Digest.adler32"),
         // holds the CRC32 for chunks in an a uncompressed file.
         CRC("CRC.db"),
         // holds SSTable Index Summary (sampling of Index component)
@@ -111,7 +111,9 @@
     }
 
     /**
+     * {@code
      * Filename of the form "<ksname>/<cfname>-[tmp-][<version>-]<gen>-<component>",
+     * }
      * @return A Descriptor for the SSTable, and a Component for this particular file.
      * TODO move descriptor into Component field
      */
diff --git a/src/java/org/apache/cassandra/io/sstable/Descriptor.java b/src/java/org/apache/cassandra/io/sstable/Descriptor.java
index 4415db4..ed81616 100644
--- a/src/java/org/apache/cassandra/io/sstable/Descriptor.java
+++ b/src/java/org/apache/cassandra/io/sstable/Descriptor.java
@@ -18,10 +18,19 @@
 package org.apache.cassandra.io.sstable;
 
 import java.io.File;
+import java.io.IOError;
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.Deque;
 import java.util.StringTokenizer;
 
+import com.google.common.base.CharMatcher;
 import com.google.common.base.Objects;
 
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.Directories;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.Version;
 import org.apache.cassandra.io.sstable.metadata.IMetadataSerializer;
 import org.apache.cassandra.io.sstable.metadata.LegacyMetadataSerializer;
 import org.apache.cassandra.io.sstable.metadata.MetadataSerializer;
@@ -38,89 +47,6 @@
  */
 public class Descriptor
 {
-    // versions are denoted as [major][minor].  Minor versions must be forward-compatible:
-    // new fields are allowed in e.g. the metadata component, but fields can't be removed
-    // or have their size changed.
-    //
-    // Minor versions were introduced with version "hb" for Cassandra 1.0.3; prior to that,
-    // we always incremented the major version.
-    public static class Version
-    {
-        // This needs to be at the begining for initialization sake
-        public static final String current_version = "ka";
-
-        // ja (2.0.0): super columns are serialized as composites (note that there is no real format change,
-        //               this is mostly a marker to know if we should expect super columns or not. We do need
-        //               a major version bump however, because we should not allow streaming of super columns
-        //               into this new format)
-        //             tracks max local deletiontime in sstable metadata
-        //             records bloom_filter_fp_chance in metadata component
-        //             remove data size and column count from data file (CASSANDRA-4180)
-        //             tracks max/min column values (according to comparator)
-        // jb (2.0.1): switch from crc32 to adler32 for compression checksums
-        //             checksum the compressed data
-        // ka (2.1.0): new Statistics.db file format
-        //             index summaries can be downsampled and the sampling level is persisted
-        //             switch uncompressed checksums to adler32
-        //             tracks presense of legacy (local and remote) counter shards
-
-        public static final Version CURRENT = new Version(current_version);
-
-        private final String version;
-
-        public final boolean isLatestVersion;
-        public final boolean hasPostCompressionAdlerChecksums;
-        public final boolean hasSamplingLevel;
-        public final boolean newStatsFile;
-        public final boolean hasAllAdlerChecksums;
-        public final boolean hasRepairedAt;
-        public final boolean tracksLegacyCounterShards;
-
-        public Version(String version)
-        {
-            this.version = version;
-            isLatestVersion = version.compareTo(current_version) == 0;
-            hasPostCompressionAdlerChecksums = version.compareTo("jb") >= 0;
-            hasSamplingLevel = version.compareTo("ka") >= 0;
-            newStatsFile = version.compareTo("ka") >= 0;
-            hasAllAdlerChecksums = version.compareTo("ka") >= 0;
-            hasRepairedAt = version.compareTo("ka") >= 0;
-            tracksLegacyCounterShards = version.compareTo("ka") >= 0;
-        }
-
-        /**
-         * @param ver SSTable version
-         * @return True if the given version string matches the format.
-         * @see #version
-         */
-        static boolean validate(String ver)
-        {
-            return ver != null && ver.matches("[a-z]+");
-        }
-
-        public boolean isCompatible()
-        {
-            return version.compareTo("ja") >= 0 && version.charAt(0) <= CURRENT.version.charAt(0);
-        }
-
-        @Override
-        public String toString()
-        {
-            return version;
-        }
-
-        @Override
-        public boolean equals(Object o)
-        {
-            return o == this || o instanceof Version && version.equals(((Version) o).version);
-        }
-
-        @Override
-        public int hashCode()
-        {
-            return version.hashCode();
-        }
-    }
 
     public static enum Type
     {
@@ -134,6 +60,7 @@
         }
     }
 
+    /** canonicalized path to the directory where SSTable resides */
     public final File directory;
     /** version has the following format: <code>[a-z]+</code> */
     public final Version version;
@@ -141,6 +68,7 @@
     public final String cfname;
     public final int generation;
     public final Type type;
+    public final SSTableFormat.Type formatType;
     private final int hashCode;
 
     /**
@@ -148,29 +76,48 @@
      */
     public Descriptor(File directory, String ksname, String cfname, int generation, Type temp)
     {
-        this(Version.CURRENT, directory, ksname, cfname, generation, temp);
+        this(DatabaseDescriptor.getSSTableFormat().info.getLatestVersion(), directory, ksname, cfname, generation, temp, DatabaseDescriptor.getSSTableFormat());
     }
 
-    public Descriptor(String version, File directory, String ksname, String cfname, int generation, Type temp)
+    public Descriptor(File directory, String ksname, String cfname, int generation, Type temp, SSTableFormat.Type formatType)
     {
-        this(new Version(version), directory, ksname, cfname, generation, temp);
+        this(formatType.info.getLatestVersion(), directory, ksname, cfname, generation, temp, formatType);
     }
 
-    public Descriptor(Version version, File directory, String ksname, String cfname, int generation, Type temp)
+    public Descriptor(String version, File directory, String ksname, String cfname, int generation, Type temp, SSTableFormat.Type formatType)
     {
-        assert version != null && directory != null && ksname != null && cfname != null;
+        this(formatType.info.getVersion(version), directory, ksname, cfname, generation, temp, formatType);
+    }
+
+    public Descriptor(Version version, File directory, String ksname, String cfname, int generation, Type temp, SSTableFormat.Type formatType)
+    {
+        assert version != null && directory != null && ksname != null && cfname != null && formatType.info.getLatestVersion().getClass().equals(version.getClass());
         this.version = version;
-        this.directory = directory;
+        try
+        {
+            this.directory = directory.getCanonicalFile();
+        }
+        catch (IOException e)
+        {
+            throw new IOError(e);
+        }
         this.ksname = ksname;
         this.cfname = cfname;
         this.generation = generation;
-        type = temp;
-        hashCode = Objects.hashCode(directory, generation, ksname, cfname, temp);
+        this.type = temp;
+        this.formatType = formatType;
+
+        hashCode = Objects.hashCode(version, this.directory, generation, ksname, cfname, temp, formatType);
     }
 
     public Descriptor withGeneration(int newGeneration)
     {
-        return new Descriptor(version, directory, ksname, cfname, newGeneration, type);
+        return new Descriptor(version, directory, ksname, cfname, newGeneration, type, formatType);
+    }
+
+    public Descriptor withFormatType(SSTableFormat.Type newType)
+    {
+        return new Descriptor(newType.info.getLatestVersion(), directory, ksname, cfname, generation, type, newType);
     }
 
     public String filenameFor(Component component)
@@ -188,12 +135,17 @@
 
     private void appendFileName(StringBuilder buff)
     {
-        buff.append(ksname).append(separator);
-        buff.append(cfname).append(separator);
+        if (!version.hasNewFileName())
+        {
+            buff.append(ksname).append(separator);
+            buff.append(cfname).append(separator);
+        }
         if (type.isTemporary)
             buff.append(type.marker).append(separator);
         buff.append(version).append(separator);
         buff.append(generation);
+        if (formatType != SSTableFormat.Type.LEGACY)
+            buff.append(separator).append(formatType.name);
     }
 
     public String relativeFilenameFor(Component component)
@@ -204,6 +156,11 @@
         return buff.toString();
     }
 
+    public SSTableFormat getFormat()
+    {
+        return formatType.info;
+    }
+
     /**
      * @param suffix A component suffix, such as 'Data.db'/'Index.db'/etc
      * @return A filename for this descriptor with the given suffix.
@@ -220,23 +177,34 @@
      */
     public static Descriptor fromFilename(String filename)
     {
-        File file = new File(filename);
-        return fromFilename(file.getParentFile(), file.getName(), false).left;
+        return fromFilename(filename, false);
+    }
+
+    public static Descriptor fromFilename(String filename, SSTableFormat.Type formatType)
+    {
+        return fromFilename(filename).withFormatType(formatType);
     }
 
     public static Descriptor fromFilename(String filename, boolean skipComponent)
     {
-        File file = new File(filename);
+        File file = new File(filename).getAbsoluteFile();
         return fromFilename(file.getParentFile(), file.getName(), skipComponent).left;
     }
 
-    public static Pair<Descriptor,String> fromFilename(File directory, String name)
+    public static Pair<Descriptor, String> fromFilename(File directory, String name)
     {
         return fromFilename(directory, name, false);
     }
 
     /**
-     * Filename of the form "<ksname>-<cfname>-[tmp-][<version>-]<gen>-<component>"
+     * Filename of the form is vary by version:
+     *
+     * <ul>
+     *     <li>&lt;ksname&gt;-&lt;cfname&gt;-(tmp-)?&lt;version&gt;-&lt;gen&gt;-&lt;component&gt; for cassandra 2.0 and before</li>
+     *     <li>(&lt;tmp marker&gt;-)?&lt;version&gt;-&lt;gen&gt;-&lt;component&gt; for cassandra 3.0 and later</li>
+     * </ul>
+     *
+     * If this is for SSTable of secondary index, directory should ends with index name for 2.1+.
      *
      * @param directory The directory of the SSTable files
      * @param name The name of the SSTable file
@@ -244,43 +212,89 @@
      *
      * @return A Descriptor for the SSTable, and the Component remainder.
      */
-    public static Pair<Descriptor,String> fromFilename(File directory, String name, boolean skipComponent)
+    public static Pair<Descriptor, String> fromFilename(File directory, String name, boolean skipComponent)
     {
+        File parentDirectory = directory != null ? directory : new File(".");
+
         // tokenize the filename
         StringTokenizer st = new StringTokenizer(name, String.valueOf(separator));
         String nexttok;
 
-        // all filenames must start with keyspace and column family
-        String ksname = st.nextToken();
-        String cfname = st.nextToken();
-
-        // optional temporary marker
-        nexttok = st.nextToken();
-        Type type = Type.FINAL;
-        if (nexttok.equals(Type.TEMP.marker))
+        // read tokens backwards to determine version
+        Deque<String> tokenStack = new ArrayDeque<>();
+        while (st.hasMoreTokens())
         {
-            type = Type.TEMP;
-            nexttok = st.nextToken();
+            tokenStack.push(st.nextToken());
         }
-        else if (nexttok.equals(Type.TEMPLINK.marker))
-        {
-            type = Type.TEMPLINK;
-            nexttok = st.nextToken();
-        }
-
-        if (!Version.validate(nexttok))
-            throw new UnsupportedOperationException("SSTable " + name + " is too old to open.  Upgrade to 2.0 first, and run upgradesstables");
-        Version version = new Version(nexttok);
-
-        nexttok = st.nextToken();
-        int generation = Integer.parseInt(nexttok);
 
         // component suffix
-        String component = null;
-        if (!skipComponent)
-            component = st.nextToken();
-        directory = directory != null ? directory : new File(".");
-        return Pair.create(new Descriptor(version, directory, ksname, cfname, generation, type), component);
+        String component = skipComponent ? null : tokenStack.pop();
+
+        nexttok = tokenStack.pop();
+        // generation OR Type
+        SSTableFormat.Type fmt = SSTableFormat.Type.LEGACY;
+        if (!CharMatcher.DIGIT.matchesAllOf(nexttok))
+        {
+            fmt = SSTableFormat.Type.validate(nexttok);
+            nexttok = tokenStack.pop();
+        }
+
+        // generation
+        int generation = Integer.parseInt(nexttok);
+
+        // version
+        nexttok = tokenStack.pop();
+        Version version = fmt.info.getVersion(nexttok);
+
+        if (!version.validate(nexttok))
+            throw new UnsupportedOperationException("SSTable " + name + " is too old to open.  Upgrade to 2.0 first, and run upgradesstables");
+
+        // optional temporary marker
+        Type type = Descriptor.Type.FINAL;
+        nexttok = tokenStack.peek();
+        if (Descriptor.Type.TEMP.marker.equals(nexttok))
+        {
+            type = Descriptor.Type.TEMP;
+            tokenStack.pop();
+        }
+        else if (Descriptor.Type.TEMPLINK.marker.equals(nexttok))
+        {
+            type = Descriptor.Type.TEMPLINK;
+            tokenStack.pop();
+        }
+
+        // ks/cf names
+        String ksname, cfname;
+        if (version.hasNewFileName())
+        {
+            // for 2.1+ read ks and cf names from directory
+            File cfDirectory = parentDirectory;
+            // check if this is secondary index
+            String indexName = "";
+            if (cfDirectory.getName().startsWith(Directories.SECONDARY_INDEX_NAME_SEPARATOR))
+            {
+                indexName = cfDirectory.getName();
+                cfDirectory = cfDirectory.getParentFile();
+            }
+            if (cfDirectory.getName().equals(Directories.BACKUPS_SUBDIR))
+            {
+                cfDirectory = cfDirectory.getParentFile();
+            }
+            else if (cfDirectory.getParentFile().getName().equals(Directories.SNAPSHOT_SUBDIR))
+            {
+                cfDirectory = cfDirectory.getParentFile().getParentFile();
+            }
+            cfname = cfDirectory.getName().split("-")[0] + indexName;
+            ksname = cfDirectory.getParentFile().getName();
+        }
+        else
+        {
+            cfname = tokenStack.pop();
+            ksname = tokenStack.pop();
+        }
+        assert tokenStack.isEmpty() : "Invalid file name " + name + " in " + directory;
+
+        return Pair.create(new Descriptor(version, parentDirectory, ksname, cfname, generation, type, fmt), component);
     }
 
     /**
@@ -289,12 +303,12 @@
      */
     public Descriptor asType(Type type)
     {
-        return new Descriptor(version, directory, ksname, cfname, generation, type);
+        return new Descriptor(version, directory, ksname, cfname, generation, type, formatType);
     }
 
     public IMetadataSerializer getMetadataSerializer()
     {
-        if (version.newStatsFile)
+        if (version.hasNewStatsFile())
             return new MetadataSerializer();
         else
             return new LegacyMetadataSerializer();
@@ -326,6 +340,7 @@
                        && that.generation == this.generation
                        && that.ksname.equals(this.ksname)
                        && that.cfname.equals(this.cfname)
+                       && that.formatType == this.formatType
                        && that.type == this.type;
     }
 
diff --git a/src/java/org/apache/cassandra/io/sstable/IndexHelper.java b/src/java/org/apache/cassandra/io/sstable/IndexHelper.java
index b0bbfc4..3d304c5 100644
--- a/src/java/org/apache/cassandra/io/sstable/IndexHelper.java
+++ b/src/java/org/apache/cassandra/io/sstable/IndexHelper.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.io.sstable;
 
 import java.io.*;
-import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.List;
@@ -29,7 +28,6 @@
 import org.apache.cassandra.io.ISerializer;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.io.util.FileDataInput;
-import org.apache.cassandra.io.util.FileMark;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.utils.*;
 
@@ -67,32 +65,6 @@
     }
 
     /**
-     * Deserialize the index into a structure and return it
-     *
-     * @param in input source
-     * @param type the comparator type for the column family
-     *
-     * @return ArrayList<IndexInfo> - list of de-serialized indexes
-     * @throws IOException if an I/O error occurs.
-     */
-    public static List<IndexInfo> deserializeIndex(FileDataInput in, CType type) throws IOException
-    {
-        int columnIndexSize = in.readInt();
-        if (columnIndexSize == 0)
-            return Collections.<IndexInfo>emptyList();
-        ArrayList<IndexInfo> indexList = new ArrayList<IndexInfo>();
-        FileMark mark = in.mark();
-        ISerializer<IndexInfo> serializer = type.indexSerializer();
-        while (in.bytesPastMark(mark) < columnIndexSize)
-        {
-            indexList.add(serializer.deserialize(in));
-        }
-        assert in.bytesPastMark(mark) == columnIndexSize;
-
-        return indexList;
-    }
-
-    /**
      * The index of the IndexInfo in which a scan starting with @name should begin.
      *
      * @param name
diff --git a/src/java/org/apache/cassandra/io/sstable/IndexSummary.java b/src/java/org/apache/cassandra/io/sstable/IndexSummary.java
index 0ea0b48..7df7349 100644
--- a/src/java/org/apache/cassandra/io/sstable/IndexSummary.java
+++ b/src/java/org/apache/cassandra/io/sstable/IndexSummary.java
@@ -18,18 +18,23 @@
 package org.apache.cassandra.io.sstable;
 
 import java.io.DataInputStream;
+import java.io.File;
+import java.io.FileOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.RowPosition;
 import org.apache.cassandra.dht.IPartitioner;
-import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.io.util.Memory;
-import org.apache.cassandra.io.util.MemoryOutputStream;
+import org.apache.cassandra.io.util.*;
+import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.concurrent.WrappedSharedCloseable;
+import org.apache.cassandra.utils.memory.MemoryUtil;
 
 import static org.apache.cassandra.io.sstable.Downsampling.BASE_SAMPLING_LEVEL;
 
@@ -45,6 +50,7 @@
  */
 public class IndexSummary extends WrappedSharedCloseable
 {
+    private static final Logger logger = LoggerFactory.getLogger(IndexSummary.class);
     public static final IndexSummarySerializer serializer = new IndexSummarySerializer();
 
     /**
@@ -106,11 +112,14 @@
     // Harmony's Collections implementation
     public int binarySearch(RowPosition key)
     {
+        // We will be comparing non-native Keys, so use a buffer with appropriate byte order
+        ByteBuffer hollow = MemoryUtil.getHollowDirectByteBuffer().order(ByteOrder.BIG_ENDIAN);
         int low = 0, mid = offsetCount, high = mid - 1, result = -1;
         while (low <= high)
         {
             mid = (low + high) >> 1;
-            result = -DecoratedKey.compareTo(partitioner, ByteBuffer.wrap(getKey(mid)), key);
+            fillTemporaryKey(mid, hollow);
+            result = -DecoratedKey.compareTo(partitioner, hollow, key);
             if (result > 0)
             {
                 low = mid + 1;
@@ -148,6 +157,13 @@
         return key;
     }
 
+    private void fillTemporaryKey(int index, ByteBuffer buffer)
+    {
+        long start = getPositionInSummary(index);
+        int keySize = (int) (calculateEnd(index) - start - 8L);
+        entries.setByteBuffer(buffer, start, keySize);
+    }
+
     public long getPosition(int index)
     {
         return entries.getLong(calculateEnd(index) - 8);
@@ -219,7 +235,7 @@
         return entries;
     }
 
-    long getOffHeapSize()
+    public long getOffHeapSize()
     {
         return offsetCount * 4 + entriesLength;
     }
@@ -273,6 +289,7 @@
             out.write(t.entries, 0, t.entriesLength);
         }
 
+        @SuppressWarnings("resource")
         public IndexSummary deserialize(DataInputStream in, IPartitioner partitioner, boolean haveSamplingLevel, int expectedMinIndexInterval, int maxIndexInterval) throws IOException
         {
             int minIndexInterval = in.readInt();
@@ -305,8 +322,17 @@
 
             Memory offsets = Memory.allocate(offsetCount * 4);
             Memory entries = Memory.allocate(offheapSize - offsets.size());
-            FBUtilities.copy(in, new MemoryOutputStream(offsets), offsets.size());
-            FBUtilities.copy(in, new MemoryOutputStream(entries), entries.size());
+            try
+            {
+                FBUtilities.copy(in, new MemoryOutputStream(offsets), offsets.size());
+                FBUtilities.copy(in, new MemoryOutputStream(entries), entries.size());
+            }
+            catch (IOException ioe)
+            {
+                offsets.free();
+                entries.free();
+                throw ioe;
+            }
             // our on-disk representation treats the offsets and the summary data as one contiguous structure,
             // in which the offsets are based from the start of the structure. i.e., if the offsets occupy
             // X bytes, the value of the first offset will be X. In memory we split the two regions up, so that
diff --git a/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java b/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java
index ff06c10..6110afe 100644
--- a/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java
+++ b/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java
@@ -17,6 +17,7 @@
  */
 package org.apache.cassandra.io.sstable;
 
+import java.io.IOException;
 import java.nio.ByteOrder;
 import java.util.Map;
 import java.util.TreeMap;
@@ -69,11 +70,11 @@
      */
     public static class ReadableBoundary
     {
-        final DecoratedKey lastKey;
-        final long indexLength;
-        final long dataLength;
-        final int summaryCount;
-        final long entriesLength;
+        public final DecoratedKey lastKey;
+        public final long indexLength;
+        public final long dataLength;
+        public final int summaryCount;
+        public final long entriesLength;
         public ReadableBoundary(DecoratedKey lastKey, long indexLength, long dataLength, int summaryCount, long entriesLength)
         {
             this.lastKey = lastKey;
@@ -107,8 +108,8 @@
 
         // for initializing data structures, adjust our estimates based on the sampling level
         maxExpectedEntries = Math.max(1, (maxExpectedEntries * samplingLevel) / BASE_SAMPLING_LEVEL);
-        offsets = new SafeMemoryWriter(4 * maxExpectedEntries).withByteOrder(ByteOrder.nativeOrder());
-        entries = new SafeMemoryWriter(40 * maxExpectedEntries).withByteOrder(ByteOrder.nativeOrder());
+        offsets = new SafeMemoryWriter(4 * maxExpectedEntries).order(ByteOrder.nativeOrder());
+        entries = new SafeMemoryWriter(40 * maxExpectedEntries).order(ByteOrder.nativeOrder());
 
         // the summary will always contain the first index entry (downsampling will never remove it)
         nextSamplePosition = 0;
@@ -151,7 +152,7 @@
         return lastReadableBoundary;
     }
 
-    public IndexSummaryBuilder maybeAddEntry(DecoratedKey decoratedKey, long indexStart)
+    public IndexSummaryBuilder maybeAddEntry(DecoratedKey decoratedKey, long indexStart) throws IOException
     {
         return maybeAddEntry(decoratedKey, indexStart, 0, 0);
     }
@@ -164,7 +165,7 @@
      * @param dataEnd the position in the data file we need to be able to read to (exclusive) to read this record
      *                a value of 0 indicates we are not tracking readable boundaries
      */
-    public IndexSummaryBuilder maybeAddEntry(DecoratedKey decoratedKey, long indexStart, long indexEnd, long dataEnd)
+    public IndexSummaryBuilder maybeAddEntry(DecoratedKey decoratedKey, long indexStart, long indexEnd, long dataEnd) throws IOException
     {
         if (keysWritten == nextSamplePosition)
         {
@@ -202,12 +203,16 @@
         }
     }
 
-    public IndexSummary build(IPartitioner partitioner)
+    public void prepareToCommit()
     {
         // this method should only be called when we've finished appending records, so we truncate the
         // memory we're using to the exact amount required to represent it before building our summary
         entries.setCapacity(entries.length());
         offsets.setCapacity(offsets.length());
+    }
+
+    public IndexSummary build(IPartitioner partitioner)
+    {
         return build(partitioner, null);
     }
 
@@ -239,6 +244,13 @@
         offsets.close();
     }
 
+    public Throwable close(Throwable accumulate)
+    {
+        accumulate = entries.close(accumulate);
+        accumulate = offsets.close(accumulate);
+        return accumulate;
+    }
+
     public static int entriesAtSamplingLevel(int samplingLevel, int maxSummarySize)
     {
         return (int) Math.ceil((samplingLevel * maxSummarySize) / (double) BASE_SAMPLING_LEVEL);
@@ -269,6 +281,7 @@
      * @param partitioner the partitioner used for the index summary
      * @return a new IndexSummary
      */
+    @SuppressWarnings("resource")
     public static IndexSummary downsample(IndexSummary existing, int newSamplingLevel, int minIndexInterval, IPartitioner partitioner)
     {
         // To downsample the old index summary, we'll go through (potentially) several rounds of downsampling.
diff --git a/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java b/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java
index be5cc3c..0e9073f 100644
--- a/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java
+++ b/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java
@@ -19,20 +19,14 @@
 
 import java.io.IOException;
 import java.lang.management.ManagementFactory;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
+import java.util.*;
 import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.TimeUnit;
 import javax.management.MBeanServer;
 import javax.management.ObjectName;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.HashMultimap;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Multimap;
+import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Sets;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -40,9 +34,12 @@
 import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.DataTracker;
+import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.db.lifecycle.View;
 import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.WrappedRunnable;
 
@@ -59,13 +56,6 @@
     private int resizeIntervalInMinutes = 0;
     private long memoryPoolBytes;
 
-    // The target (or ideal) number of index summary entries must differ from the actual number of
-    // entries by this ratio in order to trigger an upsample or downsample of the summary.  Because
-    // upsampling requires reading the primary index in order to rebuild the summary, the threshold
-    // for upsampling is is higher.
-    static final double UPSAMPLE_THRESHOLD = 1.5;
-    static final double DOWNSAMPLE_THESHOLD = 0.75;
-
     private final DebuggableScheduledThreadPoolExecutor executor;
 
     // our next scheduled resizing run
@@ -204,22 +194,26 @@
      * Returns a Pair of all compacting and non-compacting sstables.  Non-compacting sstables will be marked as
      * compacting.
      */
-    private Pair<List<SSTableReader>, Multimap<DataTracker, SSTableReader>> getCompactingAndNonCompactingSSTables()
+    @SuppressWarnings("resource")
+    private Pair<List<SSTableReader>, Map<UUID, LifecycleTransaction>> getCompactingAndNonCompactingSSTables()
     {
         List<SSTableReader> allCompacting = new ArrayList<>();
-        Multimap<DataTracker, SSTableReader> allNonCompacting = HashMultimap.create();
+        Map<UUID, LifecycleTransaction> allNonCompacting = new HashMap<>();
         for (Keyspace ks : Keyspace.all())
         {
             for (ColumnFamilyStore cfStore: ks.getColumnFamilyStores())
             {
                 Set<SSTableReader> nonCompacting, allSSTables;
+                LifecycleTransaction txn = null;
                 do
                 {
-                    allSSTables = cfStore.getDataTracker().getSSTables();
-                    nonCompacting = Sets.newHashSet(cfStore.getDataTracker().getUncompactingSSTables(allSSTables));
+                    View view = cfStore.getTracker().getView();
+                    allSSTables = view.sstables;
+                    nonCompacting = ImmutableSet.copyOf(view.getUncompacting(allSSTables));
                 }
-                while (!(nonCompacting.isEmpty() || cfStore.getDataTracker().markCompacting(nonCompacting)));
-                allNonCompacting.putAll(cfStore.getDataTracker(), nonCompacting);
+                while (null == (txn = cfStore.getTracker().tryModify(nonCompacting, OperationType.UNKNOWN)));
+
+                allNonCompacting.put(cfStore.metadata.cfId, txn);
                 allCompacting.addAll(Sets.difference(allSSTables, nonCompacting));
             }
         }
@@ -228,29 +222,31 @@
 
     public void redistributeSummaries() throws IOException
     {
-        Pair<List<SSTableReader>, Multimap<DataTracker, SSTableReader>> compactingAndNonCompacting = getCompactingAndNonCompactingSSTables();
+        Pair<List<SSTableReader>, Map<UUID, LifecycleTransaction>> compactingAndNonCompacting = getCompactingAndNonCompactingSSTables();
         try
         {
-            redistributeSummaries(compactingAndNonCompacting.left, Lists.newArrayList(compactingAndNonCompacting.right.values()), this.memoryPoolBytes);
+            redistributeSummaries(new IndexSummaryRedistribution(compactingAndNonCompacting.left,
+                                                                 compactingAndNonCompacting.right,
+                                                                 this.memoryPoolBytes));
         }
         finally
         {
-            for(DataTracker tracker : compactingAndNonCompacting.right.keySet())
-                tracker.unmarkCompacting(compactingAndNonCompacting.right.get(tracker));
+            for (LifecycleTransaction modifier : compactingAndNonCompacting.right.values())
+                modifier.close();
         }
     }
 
     /**
      * Attempts to fairly distribute a fixed pool of memory for index summaries across a set of SSTables based on
      * their recent read rates.
-     * @param nonCompacting a list of sstables to share the memory pool across
-     * @param memoryPoolBytes a size (in bytes) that the total index summary space usage should stay close to or
-     *                        under, if possible
+     * @param redistribution encapsulating the transactions containing the sstables we are to redistribute the
+     *                       memory pool across and a size (in bytes) that the total index summary space usage
+     *                       should stay close to or under, if possible
      * @return a list of new SSTableReader instances
      */
     @VisibleForTesting
-    public static List<SSTableReader> redistributeSummaries(List<SSTableReader> compacting, List<SSTableReader> nonCompacting, long memoryPoolBytes) throws IOException
+    public static List<SSTableReader> redistributeSummaries(IndexSummaryRedistribution redistribution) throws IOException
     {
-        return CompactionManager.instance.runIndexSummaryRedistribution(new IndexSummaryRedistribution(compacting, nonCompacting, memoryPoolBytes));
+        return CompactionManager.instance.runIndexSummaryRedistribution(redistribution);
     }
 }
diff --git a/src/java/org/apache/cassandra/io/sstable/IndexSummaryRedistribution.java b/src/java/org/apache/cassandra/io/sstable/IndexSummaryRedistribution.java
index adb3e4e..aad479b 100644
--- a/src/java/org/apache/cassandra/io/sstable/IndexSummaryRedistribution.java
+++ b/src/java/org/apache/cassandra/io/sstable/IndexSummaryRedistribution.java
@@ -25,20 +25,21 @@
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.UUID;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.HashMultimap;
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
-import com.google.common.collect.Multimap;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.DataTracker;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.compaction.CompactionInfo;
 import org.apache.cassandra.db.compaction.CompactionInterruptedException;
 import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.utils.Pair;
 
 import static org.apache.cassandra.io.sstable.Downsampling.BASE_SAMPLING_LEVEL;
@@ -47,41 +48,58 @@
 {
     private static final Logger logger = LoggerFactory.getLogger(IndexSummaryRedistribution.class);
 
+    // The target (or ideal) number of index summary entries must differ from the actual number of
+    // entries by this ratio in order to trigger an upsample or downsample of the summary.  Because
+    // upsampling requires reading the primary index in order to rebuild the summary, the threshold
+    // for upsampling is is higher.
+    static final double UPSAMPLE_THRESHOLD = 1.5;
+    static final double DOWNSAMPLE_THESHOLD = 0.75;
+
     private final List<SSTableReader> compacting;
-    private final List<SSTableReader> nonCompacting;
+    private final Map<UUID, LifecycleTransaction> transactions;
     private final long memoryPoolBytes;
+    private final UUID compactionId;
     private volatile long remainingSpace;
 
-    public IndexSummaryRedistribution(List<SSTableReader> compacting, List<SSTableReader> nonCompacting, long memoryPoolBytes)
+    public IndexSummaryRedistribution(List<SSTableReader> compacting, Map<UUID, LifecycleTransaction> transactions, long memoryPoolBytes)
     {
         this.compacting = compacting;
-        this.nonCompacting = nonCompacting;
+        this.transactions = transactions;
         this.memoryPoolBytes = memoryPoolBytes;
+        this.compactionId = UUID.randomUUID();
     }
 
     public List<SSTableReader> redistributeSummaries() throws IOException
     {
+        logger.info("Redistributing index summaries");
+        List<SSTableReader> oldFormatSSTables = new ArrayList<>();
+        List<SSTableReader> redistribute = new ArrayList<>();
+        for (LifecycleTransaction txn : transactions.values())
+        {
+            for (SSTableReader sstable : ImmutableList.copyOf(txn.originals()))
+            {
+                // We can't change the sampling level of sstables with the old format, because the serialization format
+                // doesn't include the sampling level.  Leave this one as it is.  (See CASSANDRA-8993 for details.)
+                logger.trace("SSTable {} cannot be re-sampled due to old sstable format", sstable);
+                if (!sstable.descriptor.version.hasSamplingLevel())
+                {
+                    oldFormatSSTables.add(sstable);
+                    txn.cancel(sstable);
+                }
+            }
+            redistribute.addAll(txn.originals());
+        }
+
         long total = 0;
-        for (SSTableReader sstable : Iterables.concat(compacting, nonCompacting))
+        for (SSTableReader sstable : Iterables.concat(compacting, redistribute))
             total += sstable.getIndexSummaryOffHeapSize();
 
-        List<SSTableReader> oldFormatSSTables = new ArrayList<>();
-        for (SSTableReader sstable : nonCompacting)
-        {
-            // We can't change the sampling level of sstables with the old format, because the serialization format
-            // doesn't include the sampling level.  Leave this one as it is.  (See CASSANDRA-8993 for details.)
-            logger.trace("SSTable {} cannot be re-sampled due to old sstable format", sstable);
-            if (!sstable.descriptor.version.hasSamplingLevel)
-                oldFormatSSTables.add(sstable);
-        }
-        nonCompacting.removeAll(oldFormatSSTables);
+        logger.trace("Beginning redistribution of index summaries for {} sstables with memory pool size {} MB; current spaced used is {} MB",
+                     redistribute.size(), memoryPoolBytes / 1024L / 1024L, total / 1024.0 / 1024.0);
 
-        logger.debug("Beginning redistribution of index summaries for {} sstables with memory pool size {} MB; current spaced used is {} MB",
-                     nonCompacting.size(), memoryPoolBytes / 1024L / 1024L, total / 1024.0 / 1024.0);
-
-        final Map<SSTableReader, Double> readRates = new HashMap<>(nonCompacting.size());
+        final Map<SSTableReader, Double> readRates = new HashMap<>(redistribute.size());
         double totalReadsPerSec = 0.0;
-        for (SSTableReader sstable : nonCompacting)
+        for (SSTableReader sstable : redistribute)
         {
             if (isStopRequested())
                 throw new CompactionInterruptedException(getCompactionInfo());
@@ -96,7 +114,7 @@
         logger.trace("Total reads/sec across all sstables in index summary resize process: {}", totalReadsPerSec);
 
         // copy and sort by read rates (ascending)
-        List<SSTableReader> sstablesByHotness = new ArrayList<>(nonCompacting);
+        List<SSTableReader> sstablesByHotness = new ArrayList<>(redistribute);
         Collections.sort(sstablesByHotness, new ReadRateComparator(readRates));
 
         long remainingBytes = memoryPoolBytes;
@@ -105,21 +123,24 @@
 
         logger.trace("Index summaries for compacting SSTables are using {} MB of space",
                      (memoryPoolBytes - remainingBytes) / 1024.0 / 1024.0);
-        List<SSTableReader> newSSTables = adjustSamplingLevels(sstablesByHotness, totalReadsPerSec, remainingBytes);
+        List<SSTableReader> newSSTables = adjustSamplingLevels(sstablesByHotness, transactions, totalReadsPerSec, remainingBytes);
+
+        for (LifecycleTransaction txn : transactions.values())
+            txn.finish();
 
         total = 0;
         for (SSTableReader sstable : Iterables.concat(compacting, oldFormatSSTables, newSSTables))
             total += sstable.getIndexSummaryOffHeapSize();
-        logger.debug("Completed resizing of index summaries; current approximate memory used: {} MB",
+        logger.trace("Completed resizing of index summaries; current approximate memory used: {} MB",
                      total / 1024.0 / 1024.0);
 
         return newSSTables;
     }
 
     private List<SSTableReader> adjustSamplingLevels(List<SSTableReader> sstables,
+                                                     Map<UUID, LifecycleTransaction> transactions,
                                                      double totalReadsPerSec, long memoryPoolCapacity) throws IOException
     {
-
         List<ResampleEntry> toDownsample = new ArrayList<>(sstables.size() / 4);
         List<ResampleEntry> toUpsample = new ArrayList<>(sstables.size() / 4);
         List<ResampleEntry> forceResample = new ArrayList<>();
@@ -158,21 +179,21 @@
             }
 
             int newSamplingLevel = IndexSummaryBuilder.calculateSamplingLevel(currentSamplingLevel, currentNumEntries, targetNumEntries,
-                                                                              minIndexInterval, maxIndexInterval);
+                    minIndexInterval, maxIndexInterval);
             int numEntriesAtNewSamplingLevel = IndexSummaryBuilder.entriesAtSamplingLevel(newSamplingLevel, maxSummarySize);
             double effectiveIndexInterval = sstable.getEffectiveIndexInterval();
 
             logger.trace("{} has {} reads/sec; ideal space for index summary: {} bytes ({} entries); considering moving " +
-                         "from level {} ({} entries, {} bytes) to level {} ({} entries, {} bytes)",
-                         sstable.getFilename(), readsPerSec, idealSpace, targetNumEntries, currentSamplingLevel, currentNumEntries,
-                         currentNumEntries * avgEntrySize, newSamplingLevel, numEntriesAtNewSamplingLevel,
-                         numEntriesAtNewSamplingLevel * avgEntrySize);
+                    "from level {} ({} entries, {} bytes) to level {} ({} entries, {} bytes)",
+                    sstable.getFilename(), readsPerSec, idealSpace, targetNumEntries, currentSamplingLevel, currentNumEntries,
+                    currentNumEntries * avgEntrySize, newSamplingLevel, numEntriesAtNewSamplingLevel,
+                    numEntriesAtNewSamplingLevel * avgEntrySize);
 
             if (effectiveIndexInterval < minIndexInterval)
             {
                 // The min_index_interval was changed; re-sample to match it.
-                logger.debug("Forcing resample of {} because the current index interval ({}) is below min_index_interval ({})",
-                             sstable, effectiveIndexInterval, minIndexInterval);
+                logger.trace("Forcing resample of {} because the current index interval ({}) is below min_index_interval ({})",
+                        sstable, effectiveIndexInterval, minIndexInterval);
                 long spaceUsed = (long) Math.ceil(avgEntrySize * numEntriesAtNewSamplingLevel);
                 forceResample.add(new ResampleEntry(sstable, spaceUsed, newSamplingLevel));
                 remainingSpace -= spaceUsed;
@@ -180,21 +201,21 @@
             else if (effectiveIndexInterval > maxIndexInterval)
             {
                 // The max_index_interval was lowered; force an upsample to the effective minimum sampling level
-                logger.debug("Forcing upsample of {} because the current index interval ({}) is above max_index_interval ({})",
-                             sstable, effectiveIndexInterval, maxIndexInterval);
+                logger.trace("Forcing upsample of {} because the current index interval ({}) is above max_index_interval ({})",
+                        sstable, effectiveIndexInterval, maxIndexInterval);
                 newSamplingLevel = Math.max(1, (BASE_SAMPLING_LEVEL * minIndexInterval) / maxIndexInterval);
                 numEntriesAtNewSamplingLevel = IndexSummaryBuilder.entriesAtSamplingLevel(newSamplingLevel, sstable.getMaxIndexSummarySize());
                 long spaceUsed = (long) Math.ceil(avgEntrySize * numEntriesAtNewSamplingLevel);
                 forceUpsample.add(new ResampleEntry(sstable, spaceUsed, newSamplingLevel));
                 remainingSpace -= avgEntrySize * numEntriesAtNewSamplingLevel;
             }
-            else if (targetNumEntries >= currentNumEntries * IndexSummaryManager.UPSAMPLE_THRESHOLD && newSamplingLevel > currentSamplingLevel)
+            else if (targetNumEntries >= currentNumEntries * UPSAMPLE_THRESHOLD && newSamplingLevel > currentSamplingLevel)
             {
                 long spaceUsed = (long) Math.ceil(avgEntrySize * numEntriesAtNewSamplingLevel);
                 toUpsample.add(new ResampleEntry(sstable, spaceUsed, newSamplingLevel));
                 remainingSpace -= avgEntrySize * numEntriesAtNewSamplingLevel;
             }
-            else if (targetNumEntries < currentNumEntries * IndexSummaryManager.DOWNSAMPLE_THESHOLD && newSamplingLevel < currentSamplingLevel)
+            else if (targetNumEntries < currentNumEntries * DOWNSAMPLE_THESHOLD && newSamplingLevel < currentSamplingLevel)
             {
                 long spaceUsed = (long) Math.ceil(avgEntrySize * numEntriesAtNewSamplingLevel);
                 toDownsample.add(new ResampleEntry(sstable, spaceUsed, newSamplingLevel));
@@ -206,6 +227,7 @@
                 logger.trace("SSTable {} is within thresholds of ideal sampling", sstable);
                 remainingSpace -= sstable.getIndexSummaryOffHeapSize();
                 newSSTables.add(sstable);
+                transactions.get(sstable.metadata.cfId).cancel(sstable);
             }
             totalReadsPerSec -= readsPerSec;
         }
@@ -215,38 +237,27 @@
             Pair<List<SSTableReader>, List<ResampleEntry>> result = distributeRemainingSpace(toDownsample, remainingSpace);
             toDownsample = result.right;
             newSSTables.addAll(result.left);
+            for (SSTableReader sstable : result.left)
+                transactions.get(sstable.metadata.cfId).cancel(sstable);
         }
 
         // downsample first, then upsample
         toDownsample.addAll(forceResample);
         toDownsample.addAll(toUpsample);
         toDownsample.addAll(forceUpsample);
-        Multimap<DataTracker, SSTableReader> replacedByTracker = HashMultimap.create();
-        Multimap<DataTracker, SSTableReader> replacementsByTracker = HashMultimap.create();
-
-        try
+        for (ResampleEntry entry : toDownsample)
         {
-            for (ResampleEntry entry : toDownsample)
-            {
-                if (isStopRequested())
-                    throw new CompactionInterruptedException(getCompactionInfo());
+            if (isStopRequested())
+                throw new CompactionInterruptedException(getCompactionInfo());
 
-                SSTableReader sstable = entry.sstable;
-                logger.debug("Re-sampling index summary for {} from {}/{} to {}/{} of the original number of entries",
-                             sstable, sstable.getIndexSummarySamplingLevel(), Downsampling.BASE_SAMPLING_LEVEL,
-                             entry.newSamplingLevel, Downsampling.BASE_SAMPLING_LEVEL);
-                ColumnFamilyStore cfs = Keyspace.open(sstable.getKeyspaceName()).getColumnFamilyStore(sstable.getColumnFamilyName());
-                DataTracker tracker = cfs.getDataTracker();
-                SSTableReader replacement = sstable.cloneWithNewSummarySamplingLevel(cfs, entry.newSamplingLevel);
-                newSSTables.add(replacement);
-                replacedByTracker.put(tracker, sstable);
-                replacementsByTracker.put(tracker, replacement);
-            }
-        }
-        finally
-        {
-            for (DataTracker tracker : replacedByTracker.keySet())
-                tracker.replaceWithNewInstances(replacedByTracker.get(tracker), replacementsByTracker.get(tracker));
+            SSTableReader sstable = entry.sstable;
+            logger.trace("Re-sampling index summary for {} from {}/{} to {}/{} of the original number of entries",
+                         sstable, sstable.getIndexSummarySamplingLevel(), Downsampling.BASE_SAMPLING_LEVEL,
+                         entry.newSamplingLevel, Downsampling.BASE_SAMPLING_LEVEL);
+            ColumnFamilyStore cfs = Keyspace.open(sstable.metadata.ksName).getColumnFamilyStore(sstable.metadata.cfId);
+            SSTableReader replacement = sstable.cloneWithNewSummarySamplingLevel(cfs, entry.newSamplingLevel);
+            newSSTables.add(replacement);
+            transactions.get(sstable.metadata.cfId).update(replacement, true);
         }
 
         return newSSTables;
@@ -293,7 +304,7 @@
 
     public CompactionInfo getCompactionInfo()
     {
-        return new CompactionInfo(OperationType.INDEX_SUMMARY, (remainingSpace - memoryPoolBytes), memoryPoolBytes, "bytes");
+        return new CompactionInfo(OperationType.INDEX_SUMMARY, (memoryPoolBytes - remainingSpace), memoryPoolBytes, "bytes", compactionId);
     }
 
     /** Utility class for sorting sstables by their read rates. */
diff --git a/src/java/org/apache/cassandra/io/sstable/ReducingKeyIterator.java b/src/java/org/apache/cassandra/io/sstable/ReducingKeyIterator.java
index 6cb8653..a1fda57 100644
--- a/src/java/org/apache/cassandra/io/sstable/ReducingKeyIterator.java
+++ b/src/java/org/apache/cassandra/io/sstable/ReducingKeyIterator.java
@@ -23,6 +23,7 @@
 import java.util.Iterator;
 
 import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.utils.CloseableIterator;
 import org.apache.cassandra.utils.IMergeIterator;
 import org.apache.cassandra.utils.MergeIterator;
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTable.java b/src/java/org/apache/cassandra/io/sstable/SSTable.java
index ac0741d..b0aa89e 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTable.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTable.java
@@ -115,7 +115,7 @@
         }
         FileUtils.delete(desc.filenameFor(Component.SUMMARY));
 
-        logger.debug("Deleted {}", desc);
+        logger.trace("Deleted {}", desc);
         return true;
     }
 
@@ -151,18 +151,17 @@
     }
 
     /**
-     * @return A Descriptor,Component pair. If component is of unknown type, returns CUSTOM component.
+     * @return Descriptor and Component pair. null if given file is not acceptable as SSTable component.
+     *         If component is of unknown type, returns CUSTOM component.
      */
-    public static Pair<Descriptor,Component> tryComponentFromFilename(File dir, String name)
+    public static Pair<Descriptor, Component> tryComponentFromFilename(File dir, String name)
     {
         try
         {
             return Component.fromFilename(dir, name);
         }
-        catch (NoSuchElementException e)
+        catch (Throwable e)
         {
-            // A NoSuchElementException is thrown if the name does not match the Descriptor format
-            // This is the less impacting change (all calls to this method test for null return)
             return null;
         }
     }
@@ -196,7 +195,7 @@
         }
     }
 
-    private static Set<Component> discoverComponentsFor(Descriptor desc)
+    public static Set<Component> discoverComponentsFor(Descriptor desc)
     {
         Set<Component.Type> knownTypes = Sets.difference(Component.TYPES, Collections.singleton(Component.Type.CUSTOM));
         Set<Component> components = Sets.newHashSetWithExpectedSize(knownTypes.size());
@@ -210,7 +209,7 @@
     }
 
     /** @return An estimate of the number of keys contained in the given index file. */
-    long estimateRowsFromIndex(RandomAccessReader ifile) throws IOException
+    protected long estimateRowsFromIndex(RandomAccessReader ifile) throws IOException
     {
         // collect sizes for the first 10000 keys, or first 10 megabytes of data
         final int SAMPLES_CAP = 10000, BYTES_CAP = (int)Math.min(10000000, ifile.length());
@@ -271,10 +270,8 @@
     protected static void appendTOC(Descriptor descriptor, Collection<Component> components)
     {
         File tocFile = new File(descriptor.filenameFor(Component.TOC));
-        PrintWriter w = null;
-        try
+        try (PrintWriter w = new PrintWriter(new FileWriter(tocFile, true)))
         {
-            w = new PrintWriter(new FileWriter(tocFile, true));
             for (Component component : components)
                 w.println(component.name);
         }
@@ -282,10 +279,6 @@
         {
             throw new FSWriteError(e, tocFile);
         }
-        finally
-        {
-            FileUtils.closeQuietly(w);
-        }
     }
 
     /**
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableDeletingTask.java b/src/java/org/apache/cassandra/io/sstable/SSTableDeletingTask.java
index 8703494..ef16b5c 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableDeletingTask.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableDeletingTask.java
@@ -19,19 +19,20 @@
 
 import java.io.File;
 import java.util.Collections;
+import java.util.Queue;
 import java.util.Set;
-import java.util.concurrent.CopyOnWriteArraySet;
+import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.TimeUnit;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Sets;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.yammer.metrics.core.Counter;
+import com.codahale.metrics.Counter;
 import org.apache.cassandra.concurrent.ScheduledExecutors;
-import org.apache.cassandra.db.DataTracker;
-import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.concurrent.Blocker;
 
 public class SSTableDeletingTask implements Runnable
 {
@@ -41,7 +42,8 @@
     // and delete will fail (on Windows) until it is (we only force the unmapping on SUN VMs).
     // Additionally, we need to make sure to delete the data file first, so on restart the others
     // will be recognized as GCable.
-    private static final Set<SSTableDeletingTask> failedTasks = new CopyOnWriteArraySet<>();
+    private static final Queue<Runnable> failedTasks = new ConcurrentLinkedQueue<>();
+    private static final Blocker blocker = new Blocker();
 
     private final Descriptor desc;
     private final Set<Component> components;
@@ -71,13 +73,9 @@
         }
     }
 
-    public void schedule()
-    {
-        ScheduledExecutors.nonPeriodicTasks.submit(this);
-    }
-
     public void run()
     {
+        blocker.ask();
         // If we can't successfully delete the DATA component, set the task to be retried later: see above
         File datafile = new File(desc.filenameFor(Component.DATA));
         if (!datafile.delete())
@@ -98,14 +96,16 @@
      */
     public static void rescheduleFailedTasks()
     {
-        for (SSTableDeletingTask task : failedTasks)
-        {
-            failedTasks.remove(task);
-            task.schedule();
-        }
+        Runnable task;
+        while ( null != (task = failedTasks.poll()))
+            ScheduledExecutors.nonPeriodicTasks.submit(task);
+
+        // On Windows, snapshots cannot be deleted so long as a segment of the root element is memory-mapped in NTFS.
+        SnapshotDeletingTask.rescheduleFailedTasks();
     }
 
     /** for tests */
+    @VisibleForTesting
     public static void waitForDeletions()
     {
         Runnable runnable = new Runnable()
@@ -117,5 +117,11 @@
 
         FBUtilities.waitOnFuture(ScheduledExecutors.nonPeriodicTasks.schedule(runnable, 0, TimeUnit.MILLISECONDS));
     }
+
+    @VisibleForTesting
+    public static void pauseDeletions(boolean stop)
+    {
+        blocker.block(stop);
+    }
 }
 
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java b/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java
index 45994d0..8c02ee7 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableIdentityIterator.java
@@ -23,18 +23,20 @@
 import com.google.common.collect.AbstractIterator;
 
 import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
 import org.apache.cassandra.db.composites.CellNameType;
-import org.apache.cassandra.io.sstable.Descriptor.Version;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.io.util.FileDataInput;
 import org.apache.cassandra.io.util.RandomAccessReader;
 import org.apache.cassandra.serializers.MarshalException;
 
-public class SSTableIdentityIterator implements Comparable<SSTableIdentityIterator>, OnDiskAtomIterator
+    public class SSTableIdentityIterator implements Comparable<SSTableIdentityIterator>, OnDiskAtomIterator
 {
     private final DecoratedKey key;
     private final DataInput in;
-    public final long dataSize; // we [still] require this so compaction can tell if it's safe to read the row into memory
     public final ColumnSerializer.Flag flag;
 
     private final ColumnFamily columnFamily;
@@ -50,11 +52,10 @@
      * @param sstable SSTable we are reading ffrom.
      * @param file Reading using this file.
      * @param key Key of this row.
-     * @param dataSize length of row data
      */
-    public SSTableIdentityIterator(SSTableReader sstable, RandomAccessReader file, DecoratedKey key, long dataSize)
+    public SSTableIdentityIterator(SSTableReader sstable, RandomAccessReader file, DecoratedKey key)
     {
-        this(sstable, file, key, dataSize, false);
+        this(sstable, file, key, false);
     }
 
     /**
@@ -62,12 +63,11 @@
      * @param sstable SSTable we are reading ffrom.
      * @param file Reading using this file.
      * @param key Key of this row.
-     * @param dataSize length of row data
      * @param checkData if true, do its best to deserialize and check the coherence of row data
      */
-    public SSTableIdentityIterator(SSTableReader sstable, RandomAccessReader file, DecoratedKey key, long dataSize, boolean checkData)
+    public SSTableIdentityIterator(SSTableReader sstable, RandomAccessReader file, DecoratedKey key, boolean checkData)
     {
-        this(sstable.metadata, file, file.getPath(), key, dataSize, checkData, sstable, ColumnSerializer.Flag.LOCAL);
+        this(sstable.metadata, file, file.getPath(), key, checkData, sstable, ColumnSerializer.Flag.LOCAL);
     }
 
     /**
@@ -80,7 +80,7 @@
         final int expireBefore = (int) (System.currentTimeMillis() / 1000);
         final Version version = sstable.descriptor.version;
         final long dataEnd = file.getFilePointer() + dataSize;
-        return new SSTableIdentityIterator(sstable.metadata, file, file.getPath(), key, dataSize, checkData, sstable, flag, DeletionTime.LIVE,
+        return new SSTableIdentityIterator(sstable.metadata, file, file.getPath(), key, checkData, sstable, flag, DeletionTime.LIVE,
                                            new AbstractIterator<OnDiskAtom>()
                                                    {
                                                        protected OnDiskAtom computeNext()
@@ -102,16 +102,16 @@
     // sstable may be null *if* checkData is false
     // If it is null, we assume the data is in the current file format
     private SSTableIdentityIterator(CFMetaData metadata,
-                                    DataInput in,
+                                    FileDataInput in,
                                     String filename,
                                     DecoratedKey key,
-                                    long dataSize,
                                     boolean checkData,
                                     SSTableReader sstable,
                                     ColumnSerializer.Flag flag)
     {
-        this(metadata, in, filename, key, dataSize, checkData, sstable, flag, readDeletionTime(in, sstable, filename),
-             metadata.getOnDiskIterator(in, flag, (int) (System.currentTimeMillis() / 1000), sstable == null ? Descriptor.Version.CURRENT : sstable.descriptor.version));
+        this(metadata, in, filename, key, checkData, sstable, flag, readDeletionTime(in, sstable, filename),
+             metadata.getOnDiskIterator(in, flag, (int) (System.currentTimeMillis() / 1000),
+                                        sstable == null ? DatabaseDescriptor.getSSTableFormat().info.getLatestVersion() : sstable.descriptor.version));
     }
 
     private static DeletionTime readDeletionTime(DataInput in, SSTableReader sstable, String filename)
@@ -134,7 +134,6 @@
                                     DataInput in,
                                     String filename,
                                     DecoratedKey key,
-                                    long dataSize,
                                     boolean checkData,
                                     SSTableReader sstable,
                                     ColumnSerializer.Flag flag,
@@ -145,7 +144,6 @@
         this.in = in;
         this.filename = filename;
         this.key = key;
-        this.dataSize = dataSize;
         this.flag = flag;
         this.validateColumns = checkData;
         this.sstable = sstable;
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java b/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
index 249c084..b99003b 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
@@ -27,12 +27,12 @@
 import com.google.common.collect.Multimap;
 
 import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.Config;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.streaming.*;
 import org.apache.cassandra.utils.FBUtilities;
@@ -71,6 +71,7 @@
         this.connectionsPerHost = connectionsPerHost;
     }
 
+    @SuppressWarnings("resource")
     protected Collection<SSTableReader> openSSTables(final Map<InetAddress, Collection<Range<Token>>> ranges)
     {
         outputHandler.output("Opening sstables and calculating sections to stream");
@@ -92,10 +93,10 @@
                     return false;
                 }
 
-                CFMetaData metadata = client.getCFMetaData(keyspace, desc.cfname);
+                CFMetaData metadata = client.getTableMetadata(desc.cfname);
                 if (metadata == null)
                 {
-                    outputHandler.output(String.format("Skipping file %s: column family %s.%s doesn't exist", name, keyspace, desc.cfname));
+                    outputHandler.output(String.format("Skipping file %s: table %s.%s doesn't exist", name, keyspace, desc.cfname));
                     return false;
                 }
 
@@ -126,9 +127,7 @@
 
                         List<Pair<Long, Long>> sstableSections = sstable.getPositionsForRanges(tokenRanges);
                         long estimatedKeys = sstable.estimatedKeysForRanges(tokenRanges);
-                        Ref ref = sstable.tryRef();
-                        if (ref == null)
-                            throw new IllegalStateException("Could not acquire ref for "+sstable);
+                        Ref<SSTableReader> ref = sstable.ref();
                         StreamSession.SSTableStreamingSections details = new StreamSession.SSTableStreamingSections(ref, sstableSections, estimatedKeys, ActiveRepairService.UNREPAIRED_SSTABLE);
                         streamingDetails.put(endpoint, details);
                     }
@@ -156,7 +155,7 @@
         client.init(keyspace);
         outputHandler.output("Established connection to initial hosts");
 
-        StreamPlan plan = new StreamPlan("Bulk Load", 0, connectionsPerHost).connectionFactory(client.getConnectionFactory());
+        StreamPlan plan = new StreamPlan("Bulk Load", 0, connectionsPerHost, false, false).connectionFactory(client.getConnectionFactory());
 
         Map<InetAddress, Collection<Range<Token>>> endpointToRanges = client.getEndpointToRangesMap();
         openSSTables(endpointToRanges);
@@ -251,7 +250,9 @@
         /**
          * Stop the client.
          */
-        public void stop() {}
+        public void stop()
+        {
+        }
 
         /**
          * Provides connection factory.
@@ -268,7 +269,12 @@
          * Validate that {@code keyspace} is an existing keyspace and {@code
          * cfName} one of its existing column family.
          */
-        public abstract CFMetaData getCFMetaData(String keyspace, String cfName);
+        public abstract CFMetaData getTableMetadata(String tableName);
+
+        public void setTableMetadata(CFMetaData cfm)
+        {
+            throw new RuntimeException();
+        }
 
         public Map<InetAddress, Collection<Range<Token>>> getEndpointToRangesMap()
         {
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
deleted file mode 100644
index 2680125..0000000
--- a/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
+++ /dev/null
@@ -1,2434 +0,0 @@
-/*
- * 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.io.sstable;
-
-import java.io.*;
-import java.nio.ByteBuffer;
-import java.util.*;
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicLong;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Predicate;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Iterators;
-import com.google.common.collect.Ordering;
-import com.google.common.primitives.Longs;
-import com.google.common.util.concurrent.RateLimiter;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.clearspring.analytics.stream.cardinality.CardinalityMergeException;
-import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus;
-import com.clearspring.analytics.stream.cardinality.ICardinality;
-import com.yammer.metrics.core.Counter;
-import org.apache.cassandra.cache.CachingOptions;
-import org.apache.cassandra.cache.InstrumentingCache;
-import org.apache.cassandra.cache.KeyCacheKey;
-import org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor;
-import org.apache.cassandra.concurrent.ScheduledExecutors;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.config.Config;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.DataRange;
-import org.apache.cassandra.db.DataTracker;
-import org.apache.cassandra.db.DecoratedKey;
-import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.RowIndexEntry;
-import org.apache.cassandra.db.RowPosition;
-import org.apache.cassandra.db.SystemKeyspace;
-import org.apache.cassandra.db.commitlog.ReplayPosition;
-import org.apache.cassandra.db.index.SecondaryIndex;
-import org.apache.cassandra.dht.AbstractBounds;
-import org.apache.cassandra.dht.IPartitioner;
-import org.apache.cassandra.dht.LocalPartitioner;
-import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.io.FSError;
-import org.apache.cassandra.io.FSReadError;
-import org.apache.cassandra.io.compress.CompressionMetadata;
-import org.apache.cassandra.io.sstable.metadata.CompactionMetadata;
-import org.apache.cassandra.io.sstable.metadata.MetadataComponent;
-import org.apache.cassandra.io.sstable.metadata.MetadataType;
-import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
-import org.apache.cassandra.io.sstable.metadata.ValidationMetadata;
-import org.apache.cassandra.io.util.*;
-import org.apache.cassandra.metrics.RestorableMeter;
-import org.apache.cassandra.metrics.StorageMetrics;
-import org.apache.cassandra.service.ActiveRepairService;
-import org.apache.cassandra.service.CacheService;
-import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.tracing.Tracing;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.CLibrary;
-import org.apache.cassandra.utils.EstimatedHistogram;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.FilterFactory;
-import org.apache.cassandra.utils.IFilter;
-import org.apache.cassandra.utils.Pair;
-import org.apache.cassandra.utils.concurrent.OpOrder;
-import org.apache.cassandra.utils.concurrent.Ref;
-import org.apache.cassandra.utils.concurrent.SelfRefCounted;
-
-import static org.apache.cassandra.db.Directories.SECONDARY_INDEX_NAME_SEPARATOR;
-
-/**
- * An SSTableReader can be constructed in a number of places, but typically is either
- * read from disk at startup, or constructed from a flushed memtable, or after compaction
- * to replace some existing sstables. However once created, an sstablereader may also be modified.
- *
- * A reader's OpenReason describes its current stage in its lifecycle, as follows:
- *
- * NORMAL
- * From:       None        => Reader has been read from disk, either at startup or from a flushed memtable
- *             EARLY       => Reader is the final result of a compaction
- *             MOVED_START => Reader WAS being compacted, but this failed and it has been restored to NORMAL status
- *
- * EARLY
- * From:       None        => Reader is a compaction replacement that is either incomplete and has been opened
- *                            to represent its partial result status, or has been finished but the compaction
- *                            it is a part of has not yet completed fully
- *             EARLY       => Same as from None, only it is not the first time it has been
- *
- * MOVED_START
- * From:       NORMAL      => Reader is being compacted. This compaction has not finished, but the compaction result
- *                            is either partially or fully opened, to either partially or fully replace this reader.
- *                            This reader's start key has been updated to represent this, so that reads only hit
- *                            one or the other reader.
- *
- * METADATA_CHANGE
- * From:       NORMAL      => Reader has seen low traffic and the amount of memory available for index summaries is
- *                            constrained, so its index summary has been downsampled.
- *         METADATA_CHANGE => Same
- *
- * Note that in parallel to this, there are two different Descriptor types; TMPLINK and FINAL; the latter corresponds
- * to NORMAL state readers and all readers that replace a NORMAL one. TMPLINK is used for EARLY state readers and
- * no others.
- *
- * When a reader is being compacted, if the result is large its replacement may be opened as EARLY before compaction
- * completes in order to present the result to consumers earlier. In this case the reader will itself be changed to
- * a MOVED_START state, where its start no longer represents its on-disk minimum key. This is to permit reads to be
- * directed to only one reader when the two represent the same data. The EARLY file can represent a compaction result
- * that is either partially complete and still in-progress, or a complete and immutable sstable that is part of a larger
- * macro compaction action that has not yet fully completed.
- *
- * Currently ALL compaction results at least briefly go through an EARLY open state prior to completion, regardless
- * of if early opening is enabled.
- *
- * Since a reader can be created multiple times over the same shared underlying resources, and the exact resources
- * it shares between each instance differ subtly, we track the lifetime of any underlying resource with its own
- * reference count, which each instance takes a Ref to. Each instance then tracks references to itself, and once these
- * all expire it releases its Refs to these underlying resources.
- *
- * There is some shared cleanup behaviour needed only once all sstablereaders in a certain stage of their lifecycle
- * (i.e. EARLY or NORMAL opening), and some that must only occur once all readers of any kind over a single logical
- * sstable have expired. These are managed by the TypeTidy and GlobalTidy classes at the bottom, and are effectively
- * managed as another resource each instance tracks its own Ref instance to, to ensure all of these resources are
- * cleaned up safely and can be debugged otherwise.
- *
- * TODO: fill in details about DataTracker and lifecycle interactions for tools, and for compaction strategies
- */
-public class SSTableReader extends SSTable implements SelfRefCounted<SSTableReader>
-{
-    private static final Logger logger = LoggerFactory.getLogger(SSTableReader.class);
-    private static final int ACCURATE_BOUNDARIES_MAGIC_NUMBER = 248923458;
-
-    private static final ScheduledThreadPoolExecutor syncExecutor = new ScheduledThreadPoolExecutor(1);
-    static
-    {
-        // Immediately remove readMeter sync task when cancelled.
-        syncExecutor.setRemoveOnCancelPolicy(true);
-    }
-    private static final RateLimiter meterSyncThrottle = RateLimiter.create(100.0);
-
-    public static final Comparator<SSTableReader> maxTimestampComparator = new Comparator<SSTableReader>()
-    {
-        public int compare(SSTableReader o1, SSTableReader o2)
-        {
-            long ts1 = o1.getMaxTimestamp();
-            long ts2 = o2.getMaxTimestamp();
-            return (ts1 > ts2 ? -1 : (ts1 == ts2 ? 0 : 1));
-        }
-    };
-
-    public static final Comparator<SSTableReader> sstableComparator = new Comparator<SSTableReader>()
-    {
-        public int compare(SSTableReader o1, SSTableReader o2)
-        {
-            return o1.first.compareTo(o2.first);
-        }
-    };
-
-    public static final Ordering<SSTableReader> sstableOrdering = Ordering.from(sstableComparator);
-
-    /**
-     * maxDataAge is a timestamp in local server time (e.g. System.currentTimeMilli) which represents an upper bound
-     * to the newest piece of data stored in the sstable. In other words, this sstable does not contain items created
-     * later than maxDataAge.
-     *
-     * The field is not serialized to disk, so relying on it for more than what truncate does is not advised.
-     *
-     * When a new sstable is flushed, maxDataAge is set to the time of creation.
-     * When a sstable is created from compaction, maxDataAge is set to max of all merged sstables.
-     *
-     * The age is in milliseconds since epoc and is local to this host.
-     */
-    public final long maxDataAge;
-
-    public enum OpenReason
-    {
-        NORMAL,
-        EARLY,
-        METADATA_CHANGE,
-        MOVED_START,
-        SHADOWED // => MOVED_START past end
-    }
-
-    public final OpenReason openReason;
-
-    // indexfile and datafile: might be null before a call to load()
-    private SegmentedFile ifile;
-    private SegmentedFile dfile;
-    private IndexSummary indexSummary;
-    private IFilter bf;
-
-    private InstrumentingCache<KeyCacheKey, RowIndexEntry> keyCache;
-
-    private final BloomFilterTracker bloomFilterTracker = new BloomFilterTracker();
-
-    // technically isCompacted is not necessary since it should never be unreferenced unless it is also compacted,
-    // but it seems like a good extra layer of protection against reference counting bugs to not delete data based on that alone
-    private AtomicBoolean isSuspect = new AtomicBoolean(false);
-
-    // not final since we need to be able to change level on a file.
-    private volatile StatsMetadata sstableMetadata;
-
-    private final AtomicLong keyCacheHit = new AtomicLong(0);
-    private final AtomicLong keyCacheRequest = new AtomicLong(0);
-
-    private final InstanceTidier tidy = new InstanceTidier(descriptor, metadata);
-    private final Ref<SSTableReader> selfRef = new Ref<>(this, tidy);
-
-    private RestorableMeter readMeter;
-
-    /**
-     * Calculate approximate key count.
-     * If cardinality estimator is available on all given sstables, then this method use them to estimate
-     * key count.
-     * If not, then this uses index summaries.
-     *
-     * @param sstables SSTables to calculate key count
-     * @return estimated key count
-     */
-    public static long getApproximateKeyCount(Collection<SSTableReader> sstables)
-    {
-        long count = -1;
-
-        // check if cardinality estimator is available for all SSTables
-        boolean cardinalityAvailable = !sstables.isEmpty() && Iterators.all(sstables.iterator(), new Predicate<SSTableReader>()
-        {
-            public boolean apply(SSTableReader sstable)
-            {
-                return sstable.descriptor.version.newStatsFile;
-            }
-        });
-
-        // if it is, load them to estimate key count
-        if (cardinalityAvailable)
-        {
-            boolean failed = false;
-            ICardinality cardinality = null;
-            for (SSTableReader sstable : sstables)
-            {
-                if (sstable.openReason == OpenReason.EARLY)
-                    continue;
-
-                try
-                {
-                    CompactionMetadata metadata = (CompactionMetadata) sstable.descriptor.getMetadataSerializer().deserialize(sstable.descriptor, MetadataType.COMPACTION);
-                    // If we can't load the CompactionMetadata, we are forced to estimate the keys using the index
-                    // summary. (CASSANDRA-10676)
-                    if (metadata == null)
-                    {
-                        logger.warn("Reading cardinality from Statistics.db failed for {}", sstable.getFilename());
-                        failed = true;
-                        break;
-                    }
-
-                    if (cardinality == null)
-                        cardinality = metadata.cardinalityEstimator;
-                    else
-                        cardinality = cardinality.merge(metadata.cardinalityEstimator);
-                }
-                catch (IOException e)
-                {
-                    logger.warn("Reading cardinality from Statistics.db failed.", e);
-                    failed = true;
-                    break;
-                }
-                catch (CardinalityMergeException e)
-                {
-                    logger.warn("Cardinality merge failed.", e);
-                    failed = true;
-                    break;
-                }
-            }
-            if (cardinality != null && !failed)
-                count = cardinality.cardinality();
-        }
-
-        // if something went wrong above or cardinality is not available, calculate using index summary
-        if (count < 0)
-        {
-            for (SSTableReader sstable : sstables)
-                count += sstable.estimatedKeys();
-        }
-        return count;
-    }
-
-    /**
-     * Estimates how much of the keys we would keep if the sstables were compacted together
-     */
-    public static double estimateCompactionGain(Set<SSTableReader> overlapping)
-    {
-        Set<ICardinality> cardinalities = new HashSet<>(overlapping.size());
-        for (SSTableReader sstable : overlapping)
-        {
-            try
-            {
-                ICardinality cardinality = ((CompactionMetadata) sstable.descriptor.getMetadataSerializer().deserialize(sstable.descriptor, MetadataType.COMPACTION)).cardinalityEstimator;
-                if (cardinality != null)
-                    cardinalities.add(cardinality);
-                else
-                    logger.debug("Got a null cardinality estimator in: "+sstable.getFilename());
-            }
-            catch (IOException e)
-            {
-                logger.warn("Could not read up compaction metadata for " + sstable, e);
-            }
-        }
-        long totalKeyCountBefore = 0;
-        for (ICardinality cardinality : cardinalities)
-        {
-            totalKeyCountBefore += cardinality.cardinality();
-        }
-        if (totalKeyCountBefore == 0)
-            return 1;
-
-        long totalKeyCountAfter = mergeCardinalities(cardinalities).cardinality();
-        logger.debug("Estimated compaction gain: {}/{}={}", totalKeyCountAfter, totalKeyCountBefore, ((double)totalKeyCountAfter)/totalKeyCountBefore);
-        return ((double)totalKeyCountAfter)/totalKeyCountBefore;
-    }
-
-    private static ICardinality mergeCardinalities(Collection<ICardinality> cardinalities)
-    {
-        ICardinality base = new HyperLogLogPlus(13, 25); // see MetadataCollector.cardinality
-        try
-        {
-            base = base.merge(cardinalities.toArray(new ICardinality[cardinalities.size()]));
-        }
-        catch (CardinalityMergeException e)
-        {
-            logger.warn("Could not merge cardinalities", e);
-        }
-        return base;
-    }
-
-    public static SSTableReader open(Descriptor descriptor) throws IOException
-    {
-        CFMetaData metadata;
-        if (descriptor.cfname.contains(SECONDARY_INDEX_NAME_SEPARATOR))
-        {
-            int i = descriptor.cfname.indexOf(SECONDARY_INDEX_NAME_SEPARATOR);
-            String parentName = descriptor.cfname.substring(0, i);
-            CFMetaData parent = Schema.instance.getCFMetaData(descriptor.ksname, parentName);
-            ColumnDefinition def = parent.getColumnDefinitionForIndex(descriptor.cfname.substring(i + 1));
-            metadata = CFMetaData.newIndexMetadata(parent, def, SecondaryIndex.getIndexComparator(parent, def));
-        }
-        else
-        {
-            metadata = Schema.instance.getCFMetaData(descriptor.ksname, descriptor.cfname);
-        }
-        return open(descriptor, metadata);
-    }
-
-    public static SSTableReader open(Descriptor desc, CFMetaData metadata) throws IOException
-    {
-        IPartitioner p = desc.cfname.contains(SECONDARY_INDEX_NAME_SEPARATOR)
-                       ? new LocalPartitioner(metadata.getKeyValidator())
-                       : StorageService.getPartitioner();
-        return open(desc, componentsFor(desc), metadata, p);
-    }
-
-    public static SSTableReader open(Descriptor descriptor, Set<Component> components, CFMetaData metadata, IPartitioner partitioner) throws IOException
-    {
-        return open(descriptor, components, metadata, partitioner, true, true);
-    }
-
-    // use only for offline or "Standalone" operations
-    public static SSTableReader openNoValidation(Descriptor descriptor, Set<Component> components, CFMetaData metadata) throws IOException
-    {
-        return open(descriptor, components, metadata, StorageService.getPartitioner(), false, false); // do not track hotness
-    }
-
-    /**
-     * Open SSTable reader to be used in batch mode(such as sstableloader).
-     *
-     * @param descriptor
-     * @param components
-     * @param metadata
-     * @param partitioner
-     * @return opened SSTableReader
-     * @throws IOException
-     */
-    public static SSTableReader openForBatch(Descriptor descriptor, Set<Component> components, CFMetaData metadata, IPartitioner partitioner) throws IOException
-    {
-        // Minimum components without which we can't do anything
-        assert components.contains(Component.DATA) : "Data component is missing for sstable" + descriptor;
-        assert components.contains(Component.PRIMARY_INDEX) : "Primary index component is missing for sstable " + descriptor;
-
-        Map<MetadataType, MetadataComponent> sstableMetadata = descriptor.getMetadataSerializer().deserialize(descriptor,
-                                                                                                               EnumSet.of(MetadataType.VALIDATION, MetadataType.STATS));
-        ValidationMetadata validationMetadata = (ValidationMetadata) sstableMetadata.get(MetadataType.VALIDATION);
-        StatsMetadata statsMetadata = (StatsMetadata) sstableMetadata.get(MetadataType.STATS);
-
-        // Check if sstable is created using same partitioner.
-        // Partitioner can be null, which indicates older version of sstable or no stats available.
-        // In that case, we skip the check.
-        String partitionerName = partitioner.getClass().getCanonicalName();
-        if (validationMetadata != null && !partitionerName.equals(validationMetadata.partitioner))
-        {
-            logger.error(String.format("Cannot open %s; partitioner %s does not match system partitioner %s.  Note that the default partitioner starting with Cassandra 1.2 is Murmur3Partitioner, so you will need to edit that to match your old partitioner if upgrading.",
-                                              descriptor, validationMetadata.partitioner, partitionerName));
-            System.exit(1);
-        }
-
-        logger.info("Opening {} ({} bytes)", descriptor, new File(descriptor.filenameFor(Component.DATA)).length());
-        SSTableReader sstable = new SSTableReader(descriptor,
-                                                  components,
-                                                  metadata,
-                                                  partitioner,
-                                                  System.currentTimeMillis(),
-                                                  statsMetadata,
-                                                  OpenReason.NORMAL);
-
-        // special implementation of load to use non-pooled SegmentedFile builders
-        SegmentedFile.Builder ibuilder = new BufferedSegmentedFile.Builder();
-        SegmentedFile.Builder dbuilder = sstable.compression
-                                       ? new CompressedSegmentedFile.Builder(null)
-                                       : new BufferedSegmentedFile.Builder();
-        if (!sstable.loadSummary(ibuilder, dbuilder))
-            sstable.buildSummary(false, ibuilder, dbuilder, false, Downsampling.BASE_SAMPLING_LEVEL);
-        sstable.ifile = ibuilder.complete(sstable.descriptor.filenameFor(Component.PRIMARY_INDEX));
-        sstable.dfile = dbuilder.complete(sstable.descriptor.filenameFor(Component.DATA));
-        sstable.bf = FilterFactory.AlwaysPresent;
-        sstable.setup(false);
-        return sstable;
-    }
-
-    public static SSTableReader open(Descriptor descriptor,
-                                      Set<Component> components,
-                                      CFMetaData metadata,
-                                      IPartitioner partitioner,
-                                      boolean validate,
-                                      boolean trackHotness) throws IOException
-    {
-        // Minimum components without which we can't do anything
-        assert components.contains(Component.DATA) : "Data component is missing for sstable" + descriptor;
-        assert !validate || components.contains(Component.PRIMARY_INDEX) : "Primary index component is missing for sstable " + descriptor;
-
-        Map<MetadataType, MetadataComponent> sstableMetadata = descriptor.getMetadataSerializer().deserialize(descriptor,
-                                                                                                              EnumSet.of(MetadataType.VALIDATION, MetadataType.STATS));
-        ValidationMetadata validationMetadata = (ValidationMetadata) sstableMetadata.get(MetadataType.VALIDATION);
-        StatsMetadata statsMetadata = (StatsMetadata) sstableMetadata.get(MetadataType.STATS);
-
-        // Check if sstable is created using same partitioner.
-        // Partitioner can be null, which indicates older version of sstable or no stats available.
-        // In that case, we skip the check.
-        String partitionerName = partitioner.getClass().getCanonicalName();
-        if (validationMetadata != null && !partitionerName.equals(validationMetadata.partitioner))
-        {
-            logger.error(String.format("Cannot open %s; partitioner %s does not match system partitioner %s.  Note that the default partitioner starting with Cassandra 1.2 is Murmur3Partitioner, so you will need to edit that to match your old partitioner if upgrading.",
-                                              descriptor, validationMetadata.partitioner, partitionerName));
-            System.exit(1);
-        }
-
-        logger.info("Opening {} ({} bytes)", descriptor, new File(descriptor.filenameFor(Component.DATA)).length());
-        SSTableReader sstable = new SSTableReader(descriptor,
-                                                  components,
-                                                  metadata,
-                                                  partitioner,
-                                                  System.currentTimeMillis(),
-                                                  statsMetadata,
-                                                  OpenReason.NORMAL);
-
-        try
-        {
-            // load index and filter
-            long start = System.nanoTime();
-            sstable.load(validationMetadata);
-            logger.debug("INDEX LOAD TIME for {}: {} ms.", descriptor, TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start));
-
-            sstable.setup(trackHotness);
-            if (validate)
-                sstable.validate();
-
-            if (sstable.getKeyCache() != null)
-                logger.debug("key cache contains {}/{} keys", sstable.getKeyCache().size(), sstable.getKeyCache().getCapacity());
-
-            return sstable;
-        }
-        catch (Throwable t)
-        {
-            sstable.selfRef().release();
-            throw t;
-        }
-    }
-
-    public static void logOpenException(Descriptor descriptor, IOException e)
-    {
-        if (e instanceof FileNotFoundException)
-            logger.error("Missing sstable component in {}; skipped because of {}", descriptor, e.getMessage());
-        else
-            logger.error("Corrupt sstable {}; skipped", descriptor, e);
-    }
-
-    public static Collection<SSTableReader> openAll(Set<Map.Entry<Descriptor, Set<Component>>> entries,
-                                                      final CFMetaData metadata,
-                                                      final IPartitioner partitioner)
-    {
-        final Collection<SSTableReader> sstables = new LinkedBlockingQueue<>();
-
-        ExecutorService executor = DebuggableThreadPoolExecutor.createWithFixedPoolSize("SSTableBatchOpen", FBUtilities.getAvailableProcessors());
-        for (final Map.Entry<Descriptor, Set<Component>> entry : entries)
-        {
-            Runnable runnable = new Runnable()
-            {
-                public void run()
-                {
-                    SSTableReader sstable;
-                    try
-                    {
-                        sstable = open(entry.getKey(), entry.getValue(), metadata, partitioner);
-                    }
-                    catch (CorruptSSTableException ex)
-                    {
-                        FileUtils.handleCorruptSSTable(ex);
-                        logger.error("Corrupt sstable {}; skipping table", entry, ex);
-                        return;
-                    }
-                    catch (FSError ex)
-                    {
-                        FileUtils.handleFSError(ex);
-                        logger.error("Cannot read sstable {}; file system error, skipping table", entry, ex);
-                        return;
-                    }
-                    catch (IOException ex)
-                    {
-                        logger.error("Cannot read sstable {}; other IO error, skipping table", entry, ex);
-                        return;
-                    }
-                    sstables.add(sstable);
-                }
-            };
-            executor.submit(runnable);
-        }
-
-        executor.shutdown();
-        try
-        {
-            executor.awaitTermination(7, TimeUnit.DAYS);
-        }
-        catch (InterruptedException e)
-        {
-            throw new AssertionError(e);
-        }
-
-        return sstables;
-
-    }
-
-    /**
-     * Open a RowIndexedReader which already has its state initialized (by SSTableWriter).
-     */
-    static SSTableReader internalOpen(Descriptor desc,
-                                      Set<Component> components,
-                                      CFMetaData metadata,
-                                      IPartitioner partitioner,
-                                      SegmentedFile ifile,
-                                      SegmentedFile dfile,
-                                      IndexSummary isummary,
-                                      IFilter bf,
-                                      long maxDataAge,
-                                      StatsMetadata sstableMetadata,
-                                      OpenReason openReason)
-    {
-        assert desc != null && partitioner != null && ifile != null && dfile != null && isummary != null && bf != null && sstableMetadata != null;
-        return new SSTableReader(desc,
-                                 components,
-                                 metadata,
-                                 partitioner,
-                                 ifile, dfile,
-                                 isummary,
-                                 bf,
-                                 maxDataAge,
-                                 sstableMetadata,
-                                 openReason);
-    }
-
-
-    private SSTableReader(final Descriptor desc,
-                          Set<Component> components,
-                          CFMetaData metadata,
-                          IPartitioner partitioner,
-                          long maxDataAge,
-                          StatsMetadata sstableMetadata,
-                          OpenReason openReason)
-    {
-        super(desc, components, metadata, partitioner);
-        this.sstableMetadata = sstableMetadata;
-        this.maxDataAge = maxDataAge;
-        this.openReason = openReason;
-    }
-
-    private SSTableReader(Descriptor desc,
-                          Set<Component> components,
-                          CFMetaData metadata,
-                          IPartitioner partitioner,
-                          SegmentedFile ifile,
-                          SegmentedFile dfile,
-                          IndexSummary indexSummary,
-                          IFilter bloomFilter,
-                          long maxDataAge,
-                          StatsMetadata sstableMetadata,
-                          OpenReason openReason)
-    {
-        this(desc, components, metadata, partitioner, maxDataAge, sstableMetadata, openReason);
-        this.ifile = ifile;
-        this.dfile = dfile;
-        this.indexSummary = indexSummary;
-        this.bf = bloomFilter;
-        this.setup(true);
-    }
-
-    public static long getTotalBytes(Iterable<SSTableReader> sstables)
-    {
-        long sum = 0;
-        for (SSTableReader sstable : sstables)
-            sum += sstable.onDiskLength();
-        return sum;
-    }
-
-    public static long getTotalUncompressedBytes(Iterable<SSTableReader> sstables)
-    {
-        long sum = 0;
-        for (SSTableReader sstable : sstables)
-            sum += sstable.uncompressedLength();
-
-        return sum;
-    }
-
-    public boolean equals(Object that)
-    {
-        return that instanceof SSTableReader && ((SSTableReader) that).descriptor.equals(this.descriptor);
-    }
-
-    public int hashCode()
-    {
-        return this.descriptor.hashCode();
-    }
-
-    public String getFilename()
-    {
-        return dfile.path;
-    }
-
-    public void setupKeyCache()
-    {
-        // under normal operation we can do this at any time, but SSTR is also used outside C* proper,
-        // e.g. by BulkLoader, which does not initialize the cache.  As a kludge, we set up the cache
-        // here when we know we're being wired into the rest of the server infrastructure.
-        keyCache = CacheService.instance.keyCache;
-    }
-
-    private void load(ValidationMetadata validation) throws IOException
-    {
-        if (metadata.getBloomFilterFpChance() == 1.0)
-        {
-            // bf is disabled.
-            load(false, true);
-            bf = FilterFactory.AlwaysPresent;
-        }
-        else if (!components.contains(Component.PRIMARY_INDEX))
-        {
-            // avoid any reading of the missing primary index component.
-            // this should only happen for standalone tools
-            load(false, false);
-        }
-        else if (!components.contains(Component.FILTER) || validation == null)
-        {
-            // bf is enabled, but filter component is missing.
-            load(true, true);
-        }
-        else if (validation.bloomFilterFPChance != metadata.getBloomFilterFpChance())
-        {
-            // bf fp chance in sstable metadata and it has changed since compaction.
-            load(true, true);
-        }
-        else
-        {
-            // bf is enabled and fp chance matches the currently configured value.
-            load(false, true);
-            loadBloomFilter();
-        }
-    }
-
-    /**
-     * Load bloom filter from Filter.db file.
-     *
-     * @throws IOException
-     */
-    private void loadBloomFilter() throws IOException
-    {
-        DataInputStream stream = null;
-        try
-        {
-            stream = new DataInputStream(new BufferedInputStream(new FileInputStream(descriptor.filenameFor(Component.FILTER))));
-            bf = FilterFactory.deserialize(stream, true);
-        }
-        finally
-        {
-            FileUtils.closeQuietly(stream);
-        }
-    }
-
-    /**
-     * Loads ifile, dfile and indexSummary, and optionally recreates the bloom filter.
-     * @param saveSummaryIfCreated for bulk loading purposes, if the summary was absent and needed to be built, you can
-     *                             avoid persisting it to disk by setting this to false
-     */
-    private void load(boolean recreateBloomFilter, boolean saveSummaryIfCreated) throws IOException
-    {
-        try
-        {
-            SegmentedFile.Builder ibuilder = SegmentedFile.getBuilder(DatabaseDescriptor.getIndexAccessMode());
-            SegmentedFile.Builder dbuilder = compression
-                                         ? SegmentedFile.getCompressedBuilder()
-                                         : SegmentedFile.getBuilder(DatabaseDescriptor.getDiskAccessMode());
-
-            boolean summaryLoaded = loadSummary(ibuilder, dbuilder);
-            boolean builtSummary = false;
-            if (recreateBloomFilter || !summaryLoaded)
-            {
-                buildSummary(recreateBloomFilter, ibuilder, dbuilder, summaryLoaded, Downsampling.BASE_SAMPLING_LEVEL);
-                builtSummary = true;
-            }
-
-            if (components.contains(Component.PRIMARY_INDEX))
-                ifile = ibuilder.complete(descriptor.filenameFor(Component.PRIMARY_INDEX));
-
-            dfile = dbuilder.complete(descriptor.filenameFor(Component.DATA));
-
-            // Check for an index summary that was downsampled even though the serialization format doesn't support
-            // that.  If it was downsampled, rebuild it.  See CASSANDRA-8993 for details.
-            if (!descriptor.version.hasSamplingLevel && !builtSummary && !validateSummarySamplingLevel() && ifile != null)
-            {
-                indexSummary.close();
-                ifile.close();
-                dfile.close();
-
-                logger.info("Detected erroneously downsampled index summary; will rebuild summary at full sampling");
-                FileUtils.deleteWithConfirm(new File(descriptor.filenameFor(Component.SUMMARY)));
-                ibuilder = SegmentedFile.getBuilder(DatabaseDescriptor.getIndexAccessMode());
-                dbuilder = compression
-                           ? SegmentedFile.getCompressedBuilder()
-                           : SegmentedFile.getBuilder(DatabaseDescriptor.getDiskAccessMode());
-                buildSummary(false, ibuilder, dbuilder, false, Downsampling.BASE_SAMPLING_LEVEL);
-                ifile = ibuilder.complete(descriptor.filenameFor(Component.PRIMARY_INDEX));
-                dfile = dbuilder.complete(descriptor.filenameFor(Component.DATA));
-                saveSummary(ibuilder, dbuilder);
-            }
-            else if (saveSummaryIfCreated && builtSummary)
-            {
-                saveSummary(ibuilder, dbuilder);
-            }
-        }
-        catch (Throwable t)
-        { // Because the tidier has not been set-up yet in SSTableReader.open(), we must release the files in case of error
-            if (ifile != null)
-            {
-                ifile.close();
-                ifile = null;
-            }
-
-            if (dfile != null)
-            {
-                dfile.close();
-                dfile = null;
-            }
-
-            if (indexSummary != null)
-            {
-                indexSummary.close();
-                indexSummary = null;
-            }
-
-            throw t;
-        }
-    }
-
-    /**
-     * Build index summary(and optionally bloom filter) by reading through Index.db file.
-     *
-     * @param recreateBloomFilter true if recreate bloom filter
-     * @param ibuilder
-     * @param dbuilder
-     * @param summaryLoaded true if index summary is already loaded and not need to build again
-     * @throws IOException
-     */
-    private void buildSummary(boolean recreateBloomFilter, SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder, boolean summaryLoaded, int samplingLevel) throws IOException
-    {
-         if (!components.contains(Component.PRIMARY_INDEX))
-             return;
-
-        // we read the positions in a BRAF so we don't have to worry about an entry spanning a mmap boundary.
-        RandomAccessReader primaryIndex = RandomAccessReader.open(new File(descriptor.filenameFor(Component.PRIMARY_INDEX)));
-
-        try
-        {
-            long indexSize = primaryIndex.length();
-            long histogramCount = sstableMetadata.estimatedRowSize.count();
-            long estimatedKeys = histogramCount > 0 && !sstableMetadata.estimatedRowSize.isOverflowed()
-                                 ? histogramCount
-                                 : estimateRowsFromIndex(primaryIndex); // statistics is supposed to be optional
-
-            try(IndexSummaryBuilder summaryBuilder = summaryLoaded ? null : new IndexSummaryBuilder(estimatedKeys, metadata.getMinIndexInterval(), samplingLevel))
-            {
-
-                if (recreateBloomFilter)
-                    bf = FilterFactory.getFilter(estimatedKeys, metadata.getBloomFilterFpChance(), true);
-
-                long indexPosition;
-                while ((indexPosition = primaryIndex.getFilePointer()) != indexSize)
-                {
-                    ByteBuffer key = ByteBufferUtil.readWithShortLength(primaryIndex);
-                    RowIndexEntry indexEntry = metadata.comparator.rowIndexEntrySerializer().deserialize(primaryIndex, descriptor.version);
-                    DecoratedKey decoratedKey = partitioner.decorateKey(key);
-                    if (first == null)
-                        first = decoratedKey;
-                    last = decoratedKey;
-
-                    if (recreateBloomFilter)
-                        bf.add(decoratedKey.getKey());
-
-                    // if summary was already read from disk we don't want to re-populate it using primary index
-                    if (!summaryLoaded)
-                    {
-                        summaryBuilder.maybeAddEntry(decoratedKey, indexPosition);
-                        ibuilder.addPotentialBoundary(indexPosition);
-                        dbuilder.addPotentialBoundary(indexEntry.position);
-                    }
-                }
-
-                if (!summaryLoaded)
-                    indexSummary = summaryBuilder.build(partitioner);
-            }
-        }
-        finally
-        {
-            FileUtils.closeQuietly(primaryIndex);
-        }
-
-        first = getMinimalKey(first);
-        last = getMinimalKey(last);
-    }
-
-    /**
-     * Load index summary from Summary.db file if it exists.
-     *
-     * if loaded index summary has different index interval from current value stored in schema,
-     * then Summary.db file will be deleted and this returns false to rebuild summary.
-     *
-     * @param ibuilder
-     * @param dbuilder
-     * @return true if index summary is loaded successfully from Summary.db file.
-     */
-    public boolean loadSummary(SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder)
-    {
-        File summariesFile = new File(descriptor.filenameFor(Component.SUMMARY));
-        if (!summariesFile.exists())
-            return false;
-
-        DataInputStream iStream = null;
-        try
-        {
-            iStream = new DataInputStream(new FileInputStream(summariesFile));
-            indexSummary = IndexSummary.serializer.deserialize(
-                    iStream, partitioner, descriptor.version.hasSamplingLevel,
-                    metadata.getMinIndexInterval(), metadata.getMaxIndexInterval());
-            first = partitioner.decorateKey(ByteBufferUtil.readWithLength(iStream));
-            last = partitioner.decorateKey(ByteBufferUtil.readWithLength(iStream));
-            ibuilder.deserializeBounds(iStream);
-            dbuilder.deserializeBounds(iStream);
-
-            boolean checkForRepair = true;
-            try
-            {
-                int v = iStream.readInt();
-                // check for our magic number, indicating this summary has been sampled correctly
-                checkForRepair = v != ACCURATE_BOUNDARIES_MAGIC_NUMBER;
-            }
-            catch (Throwable t) {}
-
-            // fix CASSANDRA-10357 on-the-fly
-            if (checkForRepair && MmappedSegmentedFile.maybeRepair(metadata, descriptor, indexSummary, ibuilder, dbuilder))
-                saveSummary(ibuilder, dbuilder);
-        }
-        catch (IOException e)
-        {
-            if (indexSummary != null)
-                indexSummary.close();
-            logger.debug("Cannot deserialize SSTable Summary File {}: {}", summariesFile.getPath(), e.getMessage());
-            // corrupted; delete it and fall back to creating a new summary
-            FileUtils.closeQuietly(iStream);
-            // delete it and fall back to creating a new summary
-            FileUtils.deleteWithConfirm(summariesFile);
-            return false;
-        }
-        finally
-        {
-            FileUtils.closeQuietly(iStream);
-        }
-
-        return true;
-    }
-
-    /**
-     * Validates that an index summary has full sampling, as expected when the serialization format does not support
-     * persisting the sampling level.
-     * @return true if the summary has full sampling, false otherwise
-     */
-    private boolean validateSummarySamplingLevel()
-    {
-        // We need to check index summary entries against the index to verify that none of them were dropped due to
-        // downsampling.  Downsampling can drop any of the first BASE_SAMPLING_LEVEL entries (repeating that drop pattern
-        // for the remainder of the summary).  Unfortunately, the first entry to be dropped is the entry at
-        // index (BASE_SAMPLING_LEVEL - 1), so we need to check a full set of BASE_SAMPLING_LEVEL entries.
-        if (ifile == null)
-            return false;
-
-        Iterator<FileDataInput> segments = ifile.iterator(0);
-        int i = 0;
-        int summaryEntriesChecked = 0;
-        int expectedIndexInterval = getMinIndexInterval();
-        while (segments.hasNext())
-        {
-            FileDataInput in = segments.next();
-            try
-            {
-                while (!in.isEOF())
-                {
-                    ByteBuffer indexKey = ByteBufferUtil.readWithShortLength(in);
-                    if (i % expectedIndexInterval == 0)
-                    {
-                        ByteBuffer summaryKey = ByteBuffer.wrap(indexSummary.getKey(i / expectedIndexInterval));
-                        if (!summaryKey.equals(indexKey))
-                            return false;
-                        summaryEntriesChecked++;
-
-                        if (summaryEntriesChecked == Downsampling.BASE_SAMPLING_LEVEL)
-                            return true;
-                    }
-                    RowIndexEntry.Serializer.skip(in);
-                    i++;
-                }
-            }
-            catch (IOException e)
-            {
-                markSuspect();
-                throw new CorruptSSTableException(e, in.getPath());
-            }
-            finally
-            {
-                FileUtils.closeQuietly(in);
-            }
-        }
-
-        return true;
-    }
-
-    /**
-     * Save index summary to Summary.db file.
-     *
-     * @param ibuilder
-     * @param dbuilder
-     */
-    public void saveSummary(SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder)
-    {
-        saveSummary(ibuilder, dbuilder, indexSummary);
-    }
-
-    private void saveSummary(SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder, IndexSummary summary)
-    {
-        File summariesFile = new File(descriptor.filenameFor(Component.SUMMARY));
-        if (summariesFile.exists())
-            FileUtils.deleteWithConfirm(summariesFile);
-
-        DataOutputStreamAndChannel oStream = null;
-        try
-        {
-            oStream = new DataOutputStreamAndChannel(new FileOutputStream(summariesFile));
-            IndexSummary.serializer.serialize(summary, oStream, descriptor.version.hasSamplingLevel);
-            ByteBufferUtil.writeWithLength(first.getKey(), oStream);
-            ByteBufferUtil.writeWithLength(last.getKey(), oStream);
-            ibuilder.serializeBounds(oStream);
-            dbuilder.serializeBounds(oStream);
-            // write a magic number, to indicate this summary has been sampled correctly
-            oStream.writeInt(ACCURATE_BOUNDARIES_MAGIC_NUMBER);
-        }
-        catch (IOException e)
-        {
-            logger.debug("Cannot save SSTable Summary: ", e);
-
-            // corrupted hence delete it and let it load it now.
-            if (summariesFile.exists())
-                FileUtils.deleteWithConfirm(summariesFile);
-        }
-        finally
-        {
-            FileUtils.closeQuietly(oStream);
-        }
-    }
-
-    public void setReplacedBy(SSTableReader replacement)
-    {
-        synchronized (tidy.global)
-        {
-            assert replacement != null;
-            assert !tidy.isReplaced;
-            tidy.isReplaced = true;
-        }
-    }
-
-    public boolean isReplaced()
-    {
-        return tidy.isReplaced;
-    }
-
-    /**
-     * Clone this reader with the provided start and open reason, and set the clone as replacement.
-     *
-     * @param newFirst the first key for the replacement (which can be different from the original due to the pre-emptive
-     * opening of compaction results).
-     * @param reason the {@code OpenReason} for the replacement.
-     *
-     * @return the cloned reader. That reader is set as a replacement by the method.
-     */
-    private SSTableReader cloneAndReplace(DecoratedKey newFirst, OpenReason reason)
-    {
-        return cloneAndReplace(newFirst, reason, indexSummary.sharedCopy());
-    }
-
-    /**
-     * Clone this reader with the new values and set the clone as replacement.
-     *
-     * @param newFirst the first key for the replacement (which can be different from the original due to the pre-emptive
-     * opening of compaction results).
-     * @param reason the {@code OpenReason} for the replacement.
-     * @param newSummary the index summary for the replacement.
-     *
-     * @return the cloned reader. That reader is set as a replacement by the method.
-     */
-    private SSTableReader cloneAndReplace(DecoratedKey newFirst, OpenReason reason, IndexSummary newSummary)
-    {
-        SSTableReader replacement = internalOpen(descriptor,
-                                                 components,
-                                                 metadata,
-                                                 partitioner,
-                                                 ifile != null ? ifile.sharedCopy() : null,
-                                                 dfile.sharedCopy(),
-                                                 newSummary,
-                                                 bf.sharedCopy(),
-                                                 maxDataAge,
-                                                 sstableMetadata,
-                                                 reason);
-        replacement.first = newFirst;
-        replacement.last = last;
-        replacement.isSuspect.set(isSuspect.get());
-        setReplacedBy(replacement);
-        return replacement;
-    }
-
-    // runOnClose must NOT be an anonymous or non-static inner class, nor must it retain a reference chain to this reader
-    public SSTableReader cloneWithNewStart(DecoratedKey newStart, final Runnable runOnClose)
-    {
-        synchronized (tidy.global)
-        {
-            assert openReason != OpenReason.EARLY;
-            // TODO: merge with caller's firstKeyBeyond() work,to save time
-            if (newStart.compareTo(first) > 0)
-            {
-                final long dataStart = getPosition(newStart, Operator.EQ).position;
-                final long indexStart = getIndexScanPosition(newStart);
-                this.tidy.runOnClose = new DropPageCache(dfile, dataStart, ifile, indexStart, runOnClose);
-            }
-
-            return cloneAndReplace(newStart, OpenReason.MOVED_START);
-        }
-    }
-
-    // runOnClose must NOT be an anonymous or non-static inner class, nor must it retain a reference chain to this reader
-    public SSTableReader cloneAsShadowed(final Runnable runOnClose)
-    {
-        synchronized (tidy.global)
-        {
-            assert openReason != OpenReason.EARLY;
-            this.tidy.runOnClose = new DropPageCache(dfile, 0, ifile, 0, runOnClose);
-            return cloneAndReplace(first, OpenReason.SHADOWED);
-        }
-    }
-
-    private static class DropPageCache implements Runnable
-    {
-        final SegmentedFile dfile;
-        final long dfilePosition;
-        final SegmentedFile ifile;
-        final long ifilePosition;
-        final Runnable andThen;
-
-        private DropPageCache(SegmentedFile dfile, long dfilePosition, SegmentedFile ifile, long ifilePosition, Runnable andThen)
-        {
-            this.dfile = dfile;
-            this.dfilePosition = dfilePosition;
-            this.ifile = ifile;
-            this.ifilePosition = ifilePosition;
-            this.andThen = andThen;
-        }
-
-        public void run()
-        {
-            dfile.dropPageCache(dfilePosition);
-            if (ifile != null)
-                ifile.dropPageCache(ifilePosition);
-            andThen.run();
-        }
-    }
-
-    /**
-     * Returns a new SSTableReader with the same properties as this SSTableReader except that a new IndexSummary will
-     * be built at the target samplingLevel.  This (original) SSTableReader instance will be marked as replaced, have
-     * its DeletingTask removed, and have its periodic read-meter sync task cancelled.
-     * @param samplingLevel the desired sampling level for the index summary on the new SSTableReader
-     * @return a new SSTableReader
-     * @throws IOException
-     */
-    public SSTableReader cloneWithNewSummarySamplingLevel(ColumnFamilyStore parent, int samplingLevel) throws IOException
-    {
-        assert descriptor.version.hasSamplingLevel;
-
-        synchronized (tidy.global)
-        {
-            assert openReason != OpenReason.EARLY;
-
-            int minIndexInterval = metadata.getMinIndexInterval();
-            int maxIndexInterval = metadata.getMaxIndexInterval();
-            double effectiveInterval = indexSummary.getEffectiveIndexInterval();
-
-            IndexSummary newSummary;
-            long oldSize = bytesOnDisk();
-
-            // We have to rebuild the summary from the on-disk primary index in three cases:
-            // 1. The sampling level went up, so we need to read more entries off disk
-            // 2. The min_index_interval changed (in either direction); this changes what entries would be in the summary
-            //    at full sampling (and consequently at any other sampling level)
-            // 3. The max_index_interval was lowered, forcing us to raise the sampling level
-            if (samplingLevel > indexSummary.getSamplingLevel() || indexSummary.getMinIndexInterval() != minIndexInterval || effectiveInterval > maxIndexInterval)
-            {
-                newSummary = buildSummaryAtLevel(samplingLevel);
-            }
-            else if (samplingLevel < indexSummary.getSamplingLevel())
-            {
-                // we can use the existing index summary to make a smaller one
-                newSummary = IndexSummaryBuilder.downsample(indexSummary, samplingLevel, minIndexInterval, partitioner);
-
-                SegmentedFile.Builder ibuilder = SegmentedFile.getBuilder(DatabaseDescriptor.getIndexAccessMode());
-                SegmentedFile.Builder dbuilder = compression
-                                                 ? SegmentedFile.getCompressedBuilder()
-                                                 : SegmentedFile.getBuilder(DatabaseDescriptor.getDiskAccessMode());
-                saveSummary(ibuilder, dbuilder, newSummary);
-            }
-            else
-            {
-                throw new AssertionError("Attempted to clone SSTableReader with the same index summary sampling level and " +
-                                         "no adjustments to min/max_index_interval");
-            }
-
-            long newSize = bytesOnDisk();
-            StorageMetrics.load.inc(newSize - oldSize);
-            parent.metric.liveDiskSpaceUsed.inc(newSize - oldSize);
-
-            return cloneAndReplace(first, OpenReason.METADATA_CHANGE, newSummary);
-        }
-    }
-
-    private IndexSummary buildSummaryAtLevel(int newSamplingLevel) throws IOException
-    {
-        // we read the positions in a BRAF so we don't have to worry about an entry spanning a mmap boundary.
-        RandomAccessReader primaryIndex = RandomAccessReader.open(new File(descriptor.filenameFor(Component.PRIMARY_INDEX)));
-        try
-        {
-            long indexSize = primaryIndex.length();
-            try (IndexSummaryBuilder summaryBuilder = new IndexSummaryBuilder(estimatedKeys(), metadata.getMinIndexInterval(), newSamplingLevel))
-            {
-                long indexPosition;
-                while ((indexPosition = primaryIndex.getFilePointer()) != indexSize)
-                {
-                    summaryBuilder.maybeAddEntry(partitioner.decorateKey(ByteBufferUtil.readWithShortLength(primaryIndex)), indexPosition);
-                    RowIndexEntry.Serializer.skip(primaryIndex);
-                }
-
-                return summaryBuilder.build(partitioner);
-            }
-        }
-        finally
-        {
-            FileUtils.closeQuietly(primaryIndex);
-        }
-    }
-
-    public RestorableMeter getReadMeter()
-    {
-        return readMeter;
-    }
-
-    public int getIndexSummarySamplingLevel()
-    {
-        return indexSummary.getSamplingLevel();
-    }
-
-    public long getIndexSummaryOffHeapSize()
-    {
-        return indexSummary.getOffHeapSize();
-    }
-
-    public int getMinIndexInterval()
-    {
-        return indexSummary.getMinIndexInterval();
-    }
-
-    public double getEffectiveIndexInterval()
-    {
-        return indexSummary.getEffectiveIndexInterval();
-    }
-
-    public void releaseSummary() throws IOException
-    {
-        tidy.releaseSummary();
-        indexSummary = null;
-    }
-
-    private void validate()
-    {
-        if (this.first.compareTo(this.last) > 0)
-        {
-            selfRef().release();
-            throw new IllegalStateException(String.format("SSTable first key %s > last key %s", this.first, this.last));
-        }
-    }
-
-    /**
-     * Gets the position in the index file to start scanning to find the given key (at most indexInterval keys away,
-     * modulo downsampling of the index summary). Always returns a value >= 0
-     */
-    public long getIndexScanPosition(RowPosition key)
-    {
-        if (openReason == OpenReason.MOVED_START && key.compareTo(first) < 0)
-            key = first;
-
-        return getIndexScanPositionFromBinarySearchResult(indexSummary.binarySearch(key), indexSummary);
-    }
-
-    @VisibleForTesting
-    public static long getIndexScanPositionFromBinarySearchResult(int binarySearchResult, IndexSummary referencedIndexSummary)
-    {
-        if (binarySearchResult == -1)
-            return 0;
-        else
-            return referencedIndexSummary.getPosition(getIndexSummaryIndexFromBinarySearchResult(binarySearchResult));
-    }
-
-    public static int getIndexSummaryIndexFromBinarySearchResult(int binarySearchResult)
-    {
-        if (binarySearchResult < 0)
-        {
-            // binary search gives us the first index _greater_ than the key searched for,
-            // i.e., its insertion position
-            int greaterThan = (binarySearchResult + 1) * -1;
-            if (greaterThan == 0)
-                return -1;
-            return greaterThan - 1;
-        }
-        else
-        {
-            return binarySearchResult;
-        }
-    }
-
-    /**
-     * Returns the compression metadata for this sstable.
-     * @throws IllegalStateException if the sstable is not compressed
-     */
-    public CompressionMetadata getCompressionMetadata()
-    {
-        if (!compression)
-            throw new IllegalStateException(this + " is not compressed");
-
-        CompressionMetadata cmd = ((ICompressedFile) dfile).getMetadata();
-
-        //We need the parent cf metadata
-        String cfName = metadata.isSecondaryIndex() ? metadata.getParentColumnFamilyName() : metadata.cfName;
-        cmd.parameters.setLiveMetadata(Schema.instance.getCFMetaData(metadata.ksName, cfName));
-
-        return cmd;
-    }
-
-    /**
-     * Returns the amount of memory in bytes used off heap by the compression meta-data.
-     * @return the amount of memory in bytes used off heap by the compression meta-data
-     */
-    public long getCompressionMetadataOffHeapSize()
-    {
-        if (!compression)
-            return 0;
-
-        return getCompressionMetadata().offHeapSize();
-    }
-
-    /**
-     * For testing purposes only.
-     */
-    public void forceFilterFailures()
-    {
-        bf = FilterFactory.AlwaysPresent;
-    }
-
-    public IFilter getBloomFilter()
-    {
-        return bf;
-    }
-
-    public long getBloomFilterSerializedSize()
-    {
-        return bf.serializedSize();
-    }
-
-    /**
-     * Returns the amount of memory in bytes used off heap by the bloom filter.
-     * @return the amount of memory in bytes used off heap by the bloom filter
-     */
-    public long getBloomFilterOffHeapSize()
-    {
-        return bf.offHeapSize();
-    }
-
-    /**
-     * @return An estimate of the number of keys in this SSTable based on the index summary.
-     */
-    public long estimatedKeys()
-    {
-        return indexSummary.getEstimatedKeyCount();
-    }
-
-    /**
-     * @param ranges
-     * @return An estimate of the number of keys for given ranges in this SSTable.
-     */
-    public long estimatedKeysForRanges(Collection<Range<Token>> ranges)
-    {
-        long sampleKeyCount = 0;
-        List<Pair<Integer, Integer>> sampleIndexes = getSampleIndexesForRanges(indexSummary, ranges);
-        for (Pair<Integer, Integer> sampleIndexRange : sampleIndexes)
-            sampleKeyCount += (sampleIndexRange.right - sampleIndexRange.left + 1);
-
-        // adjust for the current sampling level: (BSL / SL) * index_interval_at_full_sampling
-        long estimatedKeys = sampleKeyCount * ((long) Downsampling.BASE_SAMPLING_LEVEL * indexSummary.getMinIndexInterval()) / indexSummary.getSamplingLevel();
-        return Math.max(1, estimatedKeys);
-    }
-
-    /**
-     * Returns the number of entries in the IndexSummary.  At full sampling, this is approximately 1/INDEX_INTERVALth of
-     * the keys in this SSTable.
-     */
-    public int getIndexSummarySize()
-    {
-        return indexSummary.size();
-    }
-
-    /**
-     * Returns the approximate number of entries the IndexSummary would contain if it were at full sampling.
-     */
-    public int getMaxIndexSummarySize()
-    {
-        return indexSummary.getMaxNumberOfEntries();
-    }
-
-    /**
-     * Returns the key for the index summary entry at `index`.
-     */
-    public byte[] getIndexSummaryKey(int index)
-    {
-        return indexSummary.getKey(index);
-    }
-
-    private static List<Pair<Integer,Integer>> getSampleIndexesForRanges(IndexSummary summary, Collection<Range<Token>> ranges)
-    {
-        // use the index to determine a minimal section for each range
-        List<Pair<Integer,Integer>> positions = new ArrayList<>();
-
-        for (Range<Token> range : Range.normalize(ranges))
-        {
-            RowPosition leftPosition = range.left.maxKeyBound();
-            RowPosition rightPosition = range.right.maxKeyBound();
-
-            int left = summary.binarySearch(leftPosition);
-            if (left < 0)
-                left = (left + 1) * -1;
-            else
-                // left range are start exclusive
-                left = left + 1;
-            if (left == summary.size())
-                // left is past the end of the sampling
-                continue;
-
-            int right = Range.isWrapAround(range.left, range.right)
-                      ? summary.size() - 1
-                      : summary.binarySearch(rightPosition);
-            if (right < 0)
-            {
-                // range are end inclusive so we use the previous index from what binarySearch give us
-                // since that will be the last index we will return
-                right = (right + 1) * -1;
-                if (right == 0)
-                    // Means the first key is already stricly greater that the right bound
-                    continue;
-                right--;
-            }
-
-            if (left > right)
-                // empty range
-                continue;
-            positions.add(Pair.create(left, right));
-        }
-        return positions;
-    }
-
-    public Iterable<DecoratedKey> getKeySamples(final Range<Token> range)
-    {
-        final List<Pair<Integer, Integer>> indexRanges = getSampleIndexesForRanges(indexSummary, Collections.singletonList(range));
-
-        if (indexRanges.isEmpty())
-            return Collections.emptyList();
-
-        return new Iterable<DecoratedKey>()
-        {
-            public Iterator<DecoratedKey> iterator()
-            {
-                return new Iterator<DecoratedKey>()
-                {
-                    private Iterator<Pair<Integer, Integer>> rangeIter = indexRanges.iterator();
-                    private Pair<Integer, Integer> current;
-                    private int idx;
-
-                    public boolean hasNext()
-                    {
-                        if (current == null || idx > current.right)
-                        {
-                            if (rangeIter.hasNext())
-                            {
-                                current = rangeIter.next();
-                                idx = current.left;
-                                return true;
-                            }
-                            return false;
-                        }
-
-                        return true;
-                    }
-
-                    public DecoratedKey next()
-                    {
-                        byte[] bytes = indexSummary.getKey(idx++);
-                        return partitioner.decorateKey(ByteBuffer.wrap(bytes));
-                    }
-
-                    public void remove()
-                    {
-                        throw new UnsupportedOperationException();
-                    }
-                };
-            }
-        };
-    }
-
-    /**
-     * Determine the minimal set of sections that can be extracted from this SSTable to cover the given ranges.
-     * @return A sorted list of (offset,end) pairs that cover the given ranges in the datafile for this SSTable.
-     */
-    public List<Pair<Long,Long>> getPositionsForRanges(Collection<Range<Token>> ranges)
-    {
-        // use the index to determine a minimal section for each range
-        List<Pair<Long,Long>> positions = new ArrayList<>();
-        for (Range<Token> range : Range.normalize(ranges))
-        {
-            assert !range.isWrapAround() || range.right.isMinimum();
-            // truncate the range so it at most covers the sstable
-            AbstractBounds<RowPosition> bounds = range.toRowBounds();
-            RowPosition leftBound = bounds.left.compareTo(first) > 0 ? bounds.left : first.getToken().minKeyBound();
-            RowPosition rightBound = bounds.right.isMinimum() ? last.getToken().maxKeyBound() : bounds.right;
-
-            if (leftBound.compareTo(last) > 0 || rightBound.compareTo(first) < 0)
-                continue;
-
-            long left = getPosition(leftBound, Operator.GT).position;
-            long right = (rightBound.compareTo(last) > 0)
-                         ? uncompressedLength()
-                         : getPosition(rightBound, Operator.GT).position;
-
-            if (left == right)
-                // empty range
-                continue;
-
-            assert left < right : String.format("Range=%s openReason=%s first=%s last=%s left=%d right=%d", range, openReason, first, last, left, right);
-            positions.add(Pair.create(left, right));
-        }
-        return positions;
-    }
-
-    public KeyCacheKey getCacheKey(DecoratedKey key)
-    {
-        return new KeyCacheKey(metadata.ksAndCFName, descriptor, key.getKey());
-    }
-
-    public void cacheKey(DecoratedKey key, RowIndexEntry info)
-    {
-        CachingOptions caching = metadata.getCaching();
-
-        if (!caching.keyCache.isEnabled()
-            || keyCache == null
-            || keyCache.getCapacity() == 0)
-        {
-            return;
-        }
-
-        KeyCacheKey cacheKey = new KeyCacheKey(metadata.ksAndCFName, descriptor, key.getKey());
-        logger.trace("Adding cache entry for {} -> {}", cacheKey, info);
-        keyCache.put(cacheKey, info);
-    }
-
-    public RowIndexEntry getCachedPosition(DecoratedKey key, boolean updateStats)
-    {
-        return getCachedPosition(new KeyCacheKey(metadata.ksAndCFName, descriptor, key.getKey()), updateStats);
-    }
-
-    private RowIndexEntry getCachedPosition(KeyCacheKey unifiedKey, boolean updateStats)
-    {
-        if (keyCache != null && keyCache.getCapacity() > 0 && metadata.getCaching().keyCache.isEnabled()) {
-            if (updateStats)
-            {
-                RowIndexEntry cachedEntry = keyCache.get(unifiedKey);
-                keyCacheRequest.incrementAndGet();
-                if (cachedEntry != null)
-                {
-                    keyCacheHit.incrementAndGet();
-                    bloomFilterTracker.addTruePositive();
-                }
-                return cachedEntry;
-            }
-            else
-            {
-                return keyCache.getInternal(unifiedKey);
-            }
-        }
-        return null;
-    }
-
-    /**
-     * Get position updating key cache and stats.
-     * @see #getPosition(org.apache.cassandra.db.RowPosition, org.apache.cassandra.io.sstable.SSTableReader.Operator, boolean)
-     */
-    public RowIndexEntry getPosition(RowPosition key, Operator op)
-    {
-        return getPosition(key, op, true);
-    }
-
-    /**
-     * @param key The key to apply as the rhs to the given Operator. A 'fake' key is allowed to
-     * allow key selection by token bounds but only if op != * EQ
-     * @param op The Operator defining matching keys: the nearest key to the target matching the operator wins.
-     * @param updateCacheAndStats true if updating stats and cache
-     * @return The index entry corresponding to the key, or null if the key is not present
-     */
-    public RowIndexEntry getPosition(RowPosition key, Operator op, boolean updateCacheAndStats)
-    {
-        return getPosition(key, op, updateCacheAndStats, false);
-    }
-    private RowIndexEntry getPosition(RowPosition key, Operator op, boolean updateCacheAndStats, boolean permitMatchPastLast)
-    {
-        // first, check bloom filter
-        if (op == Operator.EQ)
-        {
-            assert key instanceof DecoratedKey; // EQ only make sense if the key is a valid row key
-            if (!bf.isPresent(((DecoratedKey)key).getKey()))
-            {
-                Tracing.trace("Bloom filter allows skipping sstable {}", descriptor.generation);
-                return null;
-            }
-        }
-
-        // next, the key cache (only make sense for valid row key)
-        if ((op == Operator.EQ || op == Operator.GE) && (key instanceof DecoratedKey))
-        {
-            DecoratedKey decoratedKey = (DecoratedKey)key;
-            KeyCacheKey cacheKey = new KeyCacheKey(metadata.ksAndCFName, descriptor, decoratedKey.getKey());
-            RowIndexEntry cachedPosition = getCachedPosition(cacheKey, updateCacheAndStats);
-            if (cachedPosition != null)
-            {
-                Tracing.trace("Key cache hit for sstable {}", descriptor.generation);
-                return cachedPosition;
-            }
-        }
-
-        // check the smallest and greatest keys in the sstable to see if it can't be present
-        boolean skip = false;
-        if (key.compareTo(first) < 0)
-        {
-            if (op == Operator.EQ)
-                skip = true;
-            else
-                key = first;
-
-            op = Operator.EQ;
-        }
-        else
-        {
-            int l = last.compareTo(key);
-            // l <= 0  => we may be looking past the end of the file; we then narrow our behaviour to:
-            //             1) skipping if strictly greater for GE and EQ;
-            //             2) skipping if equal and searching GT, and we aren't permitting matching past last
-            skip = l <= 0 && (l < 0 || (!permitMatchPastLast && op == Operator.GT));
-        }
-        if (skip)
-        {
-            if (op == Operator.EQ && updateCacheAndStats)
-                bloomFilterTracker.addFalsePositive();
-            Tracing.trace("Check against min and max keys allows skipping sstable {}", descriptor.generation);
-            return null;
-        }
-
-        int binarySearchResult = indexSummary.binarySearch(key);
-        long sampledPosition = getIndexScanPositionFromBinarySearchResult(binarySearchResult, indexSummary);
-        int sampledIndex = getIndexSummaryIndexFromBinarySearchResult(binarySearchResult);
-
-        int effectiveInterval = indexSummary.getEffectiveIndexIntervalAfterIndex(sampledIndex);
-
-        if (ifile == null)
-            return null;
-
-        // scan the on-disk index, starting at the nearest sampled position.
-        // The check against IndexInterval is to be exit the loop in the EQ case when the key looked for is not present
-        // (bloom filter false positive). But note that for non-EQ cases, we might need to check the first key of the
-        // next index position because the searched key can be greater the last key of the index interval checked if it
-        // is lesser than the first key of next interval (and in that case we must return the position of the first key
-        // of the next interval).
-        int i = 0;
-        Iterator<FileDataInput> segments = ifile.iterator(sampledPosition);
-        while (segments.hasNext())
-        {
-            FileDataInput in = segments.next();
-            try
-            {
-                while (!in.isEOF())
-                {
-                    i++;
-
-                    ByteBuffer indexKey = ByteBufferUtil.readWithShortLength(in);
-
-                    boolean opSatisfied; // did we find an appropriate position for the op requested
-                    boolean exactMatch; // is the current position an exact match for the key, suitable for caching
-
-                    // Compare raw keys if possible for performance, otherwise compare decorated keys.
-                    if (op == Operator.EQ && i <= effectiveInterval)
-                    {
-                        opSatisfied = exactMatch = indexKey.equals(((DecoratedKey) key).getKey());
-                    }
-                    else
-                    {
-                        DecoratedKey indexDecoratedKey = partitioner.decorateKey(indexKey);
-                        int comparison = indexDecoratedKey.compareTo(key);
-                        int v = op.apply(comparison);
-                        opSatisfied = (v == 0);
-                        exactMatch = (comparison == 0);
-                        if (v < 0)
-                        {
-                            Tracing.trace("Partition index lookup allows skipping sstable {}", descriptor.generation);
-                            return null;
-                        }
-                    }
-
-                    if (opSatisfied)
-                    {
-                        // read data position from index entry
-                        RowIndexEntry indexEntry = metadata.comparator.rowIndexEntrySerializer().deserialize(in, descriptor.version);
-                        if (exactMatch && updateCacheAndStats)
-                        {
-                            assert key instanceof DecoratedKey; // key can be == to the index key only if it's a true row key
-                            DecoratedKey decoratedKey = (DecoratedKey)key;
-
-                            if (logger.isTraceEnabled())
-                            {
-                                // expensive sanity check!  see CASSANDRA-4687
-                                FileDataInput fdi = dfile.getSegment(indexEntry.position);
-                                DecoratedKey keyInDisk = partitioner.decorateKey(ByteBufferUtil.readWithShortLength(fdi));
-                                if (!keyInDisk.equals(key))
-                                    throw new AssertionError(String.format("%s != %s in %s", keyInDisk, key, fdi.getPath()));
-                                fdi.close();
-                            }
-
-                            // store exact match for the key
-                            cacheKey(decoratedKey, indexEntry);
-                        }
-                        if (op == Operator.EQ && updateCacheAndStats)
-                            bloomFilterTracker.addTruePositive();
-                        Tracing.trace("Partition index with {} entries found for sstable {}", indexEntry.columnsIndex().size(), descriptor.generation);
-                        return indexEntry;
-                    }
-
-                    RowIndexEntry.Serializer.skip(in);
-                }
-            }
-            catch (IOException e)
-            {
-                markSuspect();
-                throw new CorruptSSTableException(e, in.getPath());
-            }
-            finally
-            {
-                FileUtils.closeQuietly(in);
-            }
-        }
-
-        if (op == Operator.EQ && updateCacheAndStats)
-            bloomFilterTracker.addFalsePositive();
-        Tracing.trace("Partition index lookup complete (bloom filter false positive) for sstable {}", descriptor.generation);
-        return null;
-    }
-
-    /**
-     * Finds and returns the first key beyond a given token in this SSTable or null if no such key exists.
-     */
-    public DecoratedKey firstKeyBeyond(RowPosition token)
-    {
-        if (token.compareTo(first) < 0)
-            return first;
-
-        long sampledPosition = getIndexScanPosition(token);
-
-        if (ifile == null)
-            return null;
-
-        Iterator<FileDataInput> segments = ifile.iterator(sampledPosition);
-        while (segments.hasNext())
-        {
-            FileDataInput in = segments.next();
-            try
-            {
-                while (!in.isEOF())
-                {
-                    ByteBuffer indexKey = ByteBufferUtil.readWithShortLength(in);
-                    DecoratedKey indexDecoratedKey = partitioner.decorateKey(indexKey);
-                    if (indexDecoratedKey.compareTo(token) > 0)
-                        return indexDecoratedKey;
-
-                    RowIndexEntry.Serializer.skip(in);
-                }
-            }
-            catch (IOException e)
-            {
-                markSuspect();
-                throw new CorruptSSTableException(e, in.getPath());
-            }
-            finally
-            {
-                FileUtils.closeQuietly(in);
-            }
-        }
-
-        return null;
-    }
-
-    /**
-     * @return The length in bytes of the data for this SSTable. For
-     * compressed files, this is not the same thing as the on disk size (see
-     * onDiskLength())
-     */
-    public long uncompressedLength()
-    {
-        return dfile.length;
-    }
-
-    /**
-     * @return The length in bytes of the on disk size for this SSTable. For
-     * compressed files, this is not the same thing as the data length (see
-     * length())
-     */
-    public long onDiskLength()
-    {
-        return dfile.onDiskLength;
-    }
-
-    /**
-     * Mark the sstable as obsolete, i.e., compacted into newer sstables.
-     *
-     * When calling this function, the caller must ensure that the SSTableReader is not referenced anywhere
-     * except for threads holding a reference.
-     *
-     * @return true if the this is the first time the file was marked obsolete.  Calling this
-     * multiple times is usually buggy (see exceptions in DataTracker.unmarkCompacting and removeOldSSTablesSize).
-     */
-    public boolean markObsolete(DataTracker tracker)
-    {
-        if (logger.isDebugEnabled())
-            logger.debug("Marking {} compacted", getFilename());
-
-        synchronized (tidy.global)
-        {
-            assert !tidy.isReplaced;
-        }
-        if (!tidy.global.isCompacted.getAndSet(true))
-        {
-            tidy.type.markObsolete(this, tracker);
-            return true;
-        }
-        return false;
-    }
-
-    public boolean isMarkedCompacted()
-    {
-        return tidy.global.isCompacted.get();
-    }
-
-    public void markSuspect()
-    {
-        if (logger.isDebugEnabled())
-            logger.debug("Marking {} as a suspect for blacklisting.", getFilename());
-
-        isSuspect.getAndSet(true);
-    }
-
-    public boolean isMarkedSuspect()
-    {
-        return isSuspect.get();
-    }
-
-    /**
-     *
-     * @param dataRange filter to use when reading the columns
-     * @return A Scanner for seeking over the rows of the SSTable.
-     */
-    public ISSTableScanner getScanner(DataRange dataRange)
-    {
-        return SSTableScanner.getScanner(this, dataRange, null);
-    }
-
-    /**
-     * I/O SSTableScanner
-     * @return A Scanner for seeking over the rows of the SSTable.
-     */
-    public ISSTableScanner getScanner()
-    {
-        return getScanner((RateLimiter) null);
-    }
-
-    public ISSTableScanner getScanner(RateLimiter limiter)
-    {
-        return SSTableScanner.getScanner(this, DataRange.allData(partitioner), limiter);
-    }
-
-    /**
-     * Direct I/O SSTableScanner over a defined range of tokens.
-     *
-     * @param range the range of keys to cover
-     * @return A Scanner for seeking over the rows of the SSTable.
-     */
-    public ISSTableScanner getScanner(Range<Token> range, RateLimiter limiter)
-    {
-        if (range == null)
-            return getScanner(limiter);
-        return getScanner(Collections.singletonList(range), limiter);
-    }
-
-   /**
-    * Direct I/O SSTableScanner over a defined collection of ranges of tokens.
-    *
-    * @param ranges the range of keys to cover
-    * @return A Scanner for seeking over the rows of the SSTable.
-    */
-    public ISSTableScanner getScanner(Collection<Range<Token>> ranges, RateLimiter limiter)
-    {
-        return SSTableScanner.getScanner(this, ranges, limiter);
-    }
-
-    public FileDataInput getFileDataInput(long position)
-    {
-        return dfile.getSegment(position);
-    }
-
-    /**
-     * Tests if the sstable contains data newer than the given age param (in localhost currentMilli time).
-     * This works in conjunction with maxDataAge which is an upper bound on the create of data in this sstable.
-     * @param age The age to compare the maxDataAre of this sstable. Measured in millisec since epoc on this host
-     * @return True iff this sstable contains data that's newer than the given age parameter.
-     */
-    public boolean newSince(long age)
-    {
-        return maxDataAge > age;
-    }
-
-    public void createLinks(String snapshotDirectoryPath)
-    {
-        for (Component component : components)
-        {
-            File sourceFile = new File(descriptor.filenameFor(component));
-            File targetLink = new File(snapshotDirectoryPath, sourceFile.getName());
-            FileUtils.createHardLink(sourceFile, targetLink);
-        }
-    }
-
-    public boolean isRepaired()
-    {
-        return sstableMetadata.repairedAt != ActiveRepairService.UNREPAIRED_SSTABLE;
-    }
-
-    /**
-     * TODO: Move someplace reusable
-     */
-    public abstract static class Operator
-    {
-        public static final Operator EQ = new Equals();
-        public static final Operator GE = new GreaterThanOrEqualTo();
-        public static final Operator GT = new GreaterThan();
-
-        /**
-         * @param comparison The result of a call to compare/compareTo, with the desired field on the rhs.
-         * @return less than 0 if the operator cannot match forward, 0 if it matches, greater than 0 if it might match forward.
-         */
-        public abstract int apply(int comparison);
-
-        final static class Equals extends Operator
-        {
-            public int apply(int comparison) { return -comparison; }
-        }
-
-        final static class GreaterThanOrEqualTo extends Operator
-        {
-            public int apply(int comparison) { return comparison >= 0 ? 0 : 1; }
-        }
-
-        final static class GreaterThan extends Operator
-        {
-            public int apply(int comparison) { return comparison > 0 ? 0 : 1; }
-        }
-    }
-
-    public long getBloomFilterFalsePositiveCount()
-    {
-        return bloomFilterTracker.getFalsePositiveCount();
-    }
-
-    public long getRecentBloomFilterFalsePositiveCount()
-    {
-        return bloomFilterTracker.getRecentFalsePositiveCount();
-    }
-
-    public long getBloomFilterTruePositiveCount()
-    {
-        return bloomFilterTracker.getTruePositiveCount();
-    }
-
-    public long getRecentBloomFilterTruePositiveCount()
-    {
-        return bloomFilterTracker.getRecentTruePositiveCount();
-    }
-
-    public InstrumentingCache<KeyCacheKey, RowIndexEntry> getKeyCache()
-    {
-        return keyCache;
-    }
-
-    public EstimatedHistogram getEstimatedRowSize()
-    {
-        return sstableMetadata.estimatedRowSize;
-    }
-
-    public EstimatedHistogram getEstimatedColumnCount()
-    {
-        return sstableMetadata.estimatedColumnCount;
-    }
-
-    public double getEstimatedDroppableTombstoneRatio(int gcBefore)
-    {
-        return sstableMetadata.getEstimatedDroppableTombstoneRatio(gcBefore);
-    }
-
-    public double getDroppableTombstonesBefore(int gcBefore)
-    {
-        return sstableMetadata.getDroppableTombstonesBefore(gcBefore);
-    }
-
-    public double getCompressionRatio()
-    {
-        return sstableMetadata.compressionRatio;
-    }
-
-    public ReplayPosition getReplayPosition()
-    {
-        return sstableMetadata.replayPosition;
-    }
-
-    public long getMinTimestamp()
-    {
-        return sstableMetadata.minTimestamp;
-    }
-
-    public long getMaxTimestamp()
-    {
-        return sstableMetadata.maxTimestamp;
-    }
-
-    public Set<Integer> getAncestors()
-    {
-        try
-        {
-            CompactionMetadata compactionMetadata = (CompactionMetadata) descriptor.getMetadataSerializer().deserialize(descriptor, MetadataType.COMPACTION);
-            if (compactionMetadata != null)
-                return compactionMetadata.ancestors;
-            return Collections.emptySet();
-        }
-        catch (IOException e)
-        {
-            SSTableReader.logOpenException(descriptor, e);
-            return Collections.emptySet();
-        }
-    }
-
-    public int getSSTableLevel()
-    {
-        return sstableMetadata.sstableLevel;
-    }
-
-    /**
-     * Reloads the sstable metadata from disk.
-     *
-     * Called after level is changed on sstable, for example if the sstable is dropped to L0
-     *
-     * Might be possible to remove in future versions
-     *
-     * @throws IOException
-     */
-    public void reloadSSTableMetadata() throws IOException
-    {
-        this.sstableMetadata = (StatsMetadata) descriptor.getMetadataSerializer().deserialize(descriptor, MetadataType.STATS);
-    }
-
-    public StatsMetadata getSSTableMetadata()
-    {
-        return sstableMetadata;
-    }
-
-    public RandomAccessReader openDataReader(RateLimiter limiter)
-    {
-        assert limiter != null;
-        return dfile.createThrottledReader(limiter);
-    }
-
-    public RandomAccessReader openDataReader()
-    {
-        return dfile.createReader();
-    }
-
-    public RandomAccessReader openIndexReader()
-    {
-        if (ifile != null)
-            return ifile.createReader();
-        return null;
-    }
-
-    /**
-     * @param component component to get timestamp.
-     * @return last modified time for given component. 0 if given component does not exist or IO error occurs.
-     */
-    public long getCreationTimeFor(Component component)
-    {
-        return new File(descriptor.filenameFor(component)).lastModified();
-    }
-
-    /**
-     * @return Number of key cache hit
-     */
-    public long getKeyCacheHit()
-    {
-        return keyCacheHit.get();
-    }
-
-    /**
-     * @return Number of key cache request
-     */
-    public long getKeyCacheRequest()
-    {
-        return keyCacheRequest.get();
-    }
-
-    /**
-     * Increment the total row read count and read rate for this SSTable.  This should not be incremented for range
-     * slice queries, row cache hits, or non-query reads, like compaction.
-     */
-    public void incrementReadCount()
-    {
-        if (readMeter != null)
-            readMeter.mark();
-    }
-
-    public static class SizeComparator implements Comparator<SSTableReader>
-    {
-        public int compare(SSTableReader o1, SSTableReader o2)
-        {
-            return Longs.compare(o1.onDiskLength(), o2.onDiskLength());
-        }
-    }
-
-    public Ref<SSTableReader> tryRef()
-    {
-        return selfRef.tryRef();
-    }
-
-    public Ref<SSTableReader> selfRef()
-    {
-        return selfRef;
-    }
-
-    public Ref<SSTableReader> ref()
-    {
-        return selfRef.ref();
-    }
-
-    void setup(boolean trackHotness)
-    {
-        tidy.setup(this, trackHotness);
-        this.readMeter = tidy.global.readMeter;
-    }
-
-    @VisibleForTesting
-    public void overrideReadMeter(RestorableMeter readMeter)
-    {
-        this.readMeter = tidy.global.readMeter = readMeter;
-    }
-
-    /**
-     * One instance per SSTableReader we create. This references the type-shared tidy, which in turn references
-     * the globally shared tidy, i.e.
-     *
-     * InstanceTidier => DescriptorTypeTitdy => GlobalTidy
-     *
-     * We can create many InstanceTidiers (one for every time we reopen an sstable with MOVED_START for example), but there can only be
-     * two DescriptorTypeTidy (FINAL and TEMPLINK) and only one GlobalTidy for one single logical sstable.
-     *
-     * When the InstanceTidier cleansup, it releases its reference to its DescriptorTypeTidy; when all InstanceTidiers
-     * for that type have run, the DescriptorTypeTidy cleansup. DescriptorTypeTidy behaves in the same way towards GlobalTidy.
-     *
-     * For ease, we stash a direct reference to both our type-shared and global tidier
-     */
-    private static final class InstanceTidier implements Tidy
-    {
-        private final Descriptor descriptor;
-        private final CFMetaData metadata;
-        private IFilter bf;
-        private IndexSummary summary;
-
-        private SegmentedFile dfile;
-        private SegmentedFile ifile;
-        private Runnable runOnClose;
-        private boolean isReplaced = false;
-
-        // a reference to our shared per-Descriptor.Type tidy instance, that
-        // we will release when we are ourselves released
-        private Ref<DescriptorTypeTidy> typeRef;
-
-        // a convenience stashing of the shared per-descriptor-type tidy instance itself
-        // and the per-logical-sstable globally shared state that it is linked to
-        private DescriptorTypeTidy type;
-        private GlobalTidy global;
-
-        private boolean setup;
-
-        void setup(SSTableReader reader, boolean trackHotness)
-        {
-            this.setup = true;
-            this.bf = reader.bf;
-            this.summary = reader.indexSummary;
-            this.dfile = reader.dfile;
-            this.ifile = reader.ifile;
-            // get a new reference to the shared descriptor-type tidy
-            this.typeRef = DescriptorTypeTidy.get(reader);
-            this.type = typeRef.get();
-            this.global = type.globalRef.get();
-            if (trackHotness)
-                global.ensureReadMeter();
-        }
-
-        InstanceTidier(Descriptor descriptor, CFMetaData metadata)
-        {
-            this.descriptor = descriptor;
-            this.metadata = metadata;
-        }
-
-        public void tidy()
-        {
-            // don't try to cleanup if the sstablereader was never fully constructed
-            if (!setup)
-                return;
-
-            final ColumnFamilyStore cfs = Schema.instance.getColumnFamilyStoreInstance(metadata.cfId);
-            final OpOrder.Barrier barrier;
-            if (cfs != null)
-            {
-                barrier = cfs.readOrdering.newBarrier();
-                barrier.issue();
-            }
-            else
-                barrier = null;
-
-            ScheduledExecutors.nonPeriodicTasks.execute(new Runnable()
-            {
-                public void run()
-                {
-                    if (barrier != null)
-                        barrier.await();
-                    if (bf != null)
-                        bf.close();
-                    if (summary != null)
-                        summary.close();
-                    if (runOnClose != null)
-                        runOnClose.run();
-                    if (dfile != null)
-                        dfile.close();
-                    if (ifile != null)
-                        ifile.close();
-                    typeRef.release();
-                }
-            });
-        }
-
-        public String name()
-        {
-            return descriptor.toString();
-        }
-
-        void releaseSummary()
-        {
-            summary.close();
-            assert summary.isCleanedUp();
-            summary = null;
-        }
-    }
-
-    /**
-     * One shared between all instances of a given Descriptor.Type.
-     * Performs only two things: the deletion of the sstables for the type,
-     * if necessary; and the shared reference to the globally shared state.
-     *
-     * All InstanceTidiers, on setup(), ask the static get() method for their shared state,
-     * and stash a reference to it to be released when they are. Once all such references are
-     * released, the shared tidy will be performed.
-     */
-    static final class DescriptorTypeTidy implements Tidy
-    {
-        // keyed by REAL descriptor (TMPLINK/FINAL), mapping to the shared DescriptorTypeTidy for that descriptor
-        static final ConcurrentMap<Descriptor, Ref<DescriptorTypeTidy>> lookup = new ConcurrentHashMap<>();
-
-        private final Descriptor desc;
-        private final Ref<GlobalTidy> globalRef;
-        private final Set<Component> components;
-        private long sizeOnDelete;
-        private Counter totalDiskSpaceUsed;
-
-        DescriptorTypeTidy(Descriptor desc, SSTableReader sstable)
-        {
-            this.desc = desc;
-            // get a new reference to the shared global tidy
-            this.globalRef = GlobalTidy.get(sstable);
-            this.components = sstable.components;
-        }
-
-        void markObsolete(SSTableReader instance, DataTracker tracker)
-        {
-            // the tracker is used only to notify listeners of deletion of the sstable;
-            // since deletion of a non-final file is not really deletion of the sstable,
-            // we don't want to notify the listeners in this event
-            if (tracker != null && desc.type == Descriptor.Type.FINAL)
-            {
-                sizeOnDelete = instance.bytesOnDisk();
-                totalDiskSpaceUsed = tracker.cfstore.metric.totalDiskSpaceUsed;
-                tracker.notifyDeleting(instance);
-            }
-        }
-
-        public void tidy()
-        {
-            lookup.remove(desc);
-            boolean isCompacted = globalRef.get().isCompacted.get();
-            globalRef.release();
-            switch (desc.type)
-            {
-                case FINAL:
-                    if (isCompacted)
-                        new SSTableDeletingTask(desc, components, totalDiskSpaceUsed, sizeOnDelete).run();
-                    break;
-                case TEMPLINK:
-                    new SSTableDeletingTask(desc, components, null, 0).run();
-                    break;
-                default:
-                    throw new IllegalStateException();
-            }
-        }
-
-        public String name()
-        {
-            return desc.toString();
-        }
-
-        // get a new reference to the shared DescriptorTypeTidy for this sstable
-        public static Ref<DescriptorTypeTidy> get(SSTableReader sstable)
-        {
-            Descriptor desc = sstable.descriptor;
-            if (sstable.openReason == OpenReason.EARLY)
-                desc = desc.asType(Descriptor.Type.TEMPLINK);
-            Ref<DescriptorTypeTidy> refc = lookup.get(desc);
-            if (refc != null)
-                return refc.ref();
-            final DescriptorTypeTidy tidy = new DescriptorTypeTidy(desc, sstable);
-            refc = new Ref<>(tidy, tidy);
-            Ref<?> ex = lookup.putIfAbsent(desc, refc);
-            assert ex == null;
-            return refc;
-        }
-    }
-
-    /**
-     * One instance per logical sstable. This both tracks shared cleanup and some shared state related
-     * to the sstable's lifecycle. All DescriptorTypeTidy instances, on construction, obtain a reference to us
-     * via our static get(). There should only ever be at most two such references extant at any one time,
-     * since only TMPLINK and FINAL type descriptors should be open as readers. When all files of both
-     * kinds have been released, this shared tidy will be performed.
-     */
-    static final class GlobalTidy implements Tidy
-    {
-        // keyed by FINAL descriptor, mapping to the shared GlobalTidy for that descriptor
-        static final ConcurrentMap<Descriptor, Ref<GlobalTidy>> lookup = new ConcurrentHashMap<>();
-
-        private final Descriptor desc;
-        // the readMeter that is shared between all instances of the sstable, and can be overridden in all of them
-        // at once also, for testing purposes
-        private RestorableMeter readMeter;
-        // the scheduled persistence of the readMeter, that we will cancel once all instances of this logical
-        // sstable have been released
-        private ScheduledFuture readMeterSyncFuture;
-        // shared state managing if the logical sstable has been compacted; this is used in cleanup both here
-        // and in the FINAL type tidier
-        private final AtomicBoolean isCompacted;
-
-        GlobalTidy(final SSTableReader reader)
-        {
-            this.desc = reader.descriptor;
-            this.isCompacted = new AtomicBoolean();
-        }
-
-        void ensureReadMeter()
-        {
-            if (readMeter != null)
-                return;
-
-            // Don't track read rates for tables in the system keyspace and don't bother trying to load or persist
-            // the read meter when in client mode.
-            if (Keyspace.SYSTEM_KS.equals(desc.ksname) || Config.isClientMode())
-            {
-                readMeter = null;
-                readMeterSyncFuture = null;
-                return;
-            }
-
-            readMeter = SystemKeyspace.getSSTableReadMeter(desc.ksname, desc.cfname, desc.generation);
-            // sync the average read rate to system.sstable_activity every five minutes, starting one minute from now
-            readMeterSyncFuture = syncExecutor.scheduleAtFixedRate(new Runnable()
-            {
-                public void run()
-                {
-                    if (!isCompacted.get())
-                    {
-                        meterSyncThrottle.acquire();
-                        SystemKeyspace.persistSSTableReadMeter(desc.ksname, desc.cfname, desc.generation, readMeter);
-                    }
-                }
-            }, 1, 5, TimeUnit.MINUTES);
-        }
-
-        public void tidy()
-        {
-            lookup.remove(desc);
-            if (readMeterSyncFuture != null)
-                readMeterSyncFuture.cancel(true);
-            if (isCompacted.get())
-                SystemKeyspace.clearSSTableReadMeter(desc.ksname, desc.cfname, desc.generation);
-            // don't ideally want to dropPageCache for the file until all instances have been released
-            CLibrary.trySkipCache(desc.filenameFor(Component.DATA), 0, 0);
-            CLibrary.trySkipCache(desc.filenameFor(Component.PRIMARY_INDEX), 0, 0);
-        }
-
-        public String name()
-        {
-            return desc.toString();
-        }
-
-        // get a new reference to the shared GlobalTidy for this sstable
-        public static Ref<GlobalTidy> get(SSTableReader sstable)
-        {
-            Descriptor descriptor = sstable.descriptor;
-            Ref<GlobalTidy> refc = lookup.get(descriptor);
-            if (refc != null)
-                return refc.ref();
-            final GlobalTidy tidy = new GlobalTidy(sstable);
-            refc = new Ref<>(tidy, tidy);
-            Ref<?> ex = lookup.putIfAbsent(descriptor, refc);
-            assert ex == null;
-            return refc;
-        }
-    }
-}
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java
index b56a5dc..c243904 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java
@@ -21,86 +21,76 @@
 import java.util.*;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Throwables;
-import com.google.common.collect.ImmutableList;
 
 import org.apache.cassandra.cache.InstrumentingCache;
 import org.apache.cassandra.cache.KeyCacheKey;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.DataTracker;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.RowIndexEntry;
 import org.apache.cassandra.db.compaction.AbstractCompactedRow;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.utils.CLibrary;
-
-import static org.apache.cassandra.utils.Throwables.merge;
+import org.apache.cassandra.utils.concurrent.Transactional;
 
 /**
  * Wraps one or more writers as output for rewriting one or more readers: every sstable_preemptive_open_interval_in_mb
  * we look in the summary we're collecting for the latest writer for the penultimate key that we know to have been fully
  * flushed to the index file, and then double check that the key is fully present in the flushed data file.
- * Then we move the starts of each reader forwards to that point, replace them in the datatracker, and attach a runnable
+ * Then we move the starts of each reader forwards to that point, replace them in the Tracker, and attach a runnable
  * for on-close (i.e. when all references expire) that drops the page cache prior to that key position
  *
  * hard-links are created for each partially written sstable so that readers opened against them continue to work past
  * the rename of the temporary file, which is deleted once all readers against the hard-link have been closed.
- * If for any reason the writer is rolled over, we immediately rename and fully expose the completed file in the DataTracker.
+ * If for any reason the writer is rolled over, we immediately rename and fully expose the completed file in the Tracker.
  *
  * On abort we restore the original lower bounds to the existing readers and delete any temporary files we had in progress,
  * but leave any hard-links in place for the readers we opened to cleanup when they're finished as we would had we finished
  * successfully.
  */
-public class SSTableRewriter
+public class SSTableRewriter extends Transactional.AbstractTransactional implements Transactional
 {
-    private final DataTracker dataTracker;
+    @VisibleForTesting
+    public static boolean disableEarlyOpeningForTests = false;
+
     private final ColumnFamilyStore cfs;
     private final long preemptiveOpenInterval;
     private final long maxAge;
-    private final List<SSTableReader> finished = new ArrayList<>();
-    private final Set<SSTableReader> rewriting; // the readers we are rewriting (updated as they are replaced)
-    private final Map<Descriptor, DecoratedKey> originalStarts = new HashMap<>(); // the start key for each reader we are rewriting
+    private long repairedAt = -1;
+    // the set of final readers we will expose on commit
+    private final LifecycleTransaction transaction; // the readers we are rewriting (updated as they are replaced)
+    private final List<SSTableReader> preparedForCommit = new ArrayList<>();
     private final Map<Descriptor, Integer> fileDescriptors = new HashMap<>(); // the file descriptors for each reader descriptor we are rewriting
 
-    private SSTableReader currentlyOpenedEarly; // the reader for the most recent (re)opening of the target file
     private long currentlyOpenedEarlyAt; // the position (in MB) in the target file we last (re)opened at
 
-    private final List<SSTableReader> finishedReaders = new ArrayList<>();
-    private final Queue<Finished> finishedEarly = new ArrayDeque<>();
-    // as writers are closed from finishedEarly, their last readers are moved
-    // into discard, so that abort can cleanup after us safely
-    private final List<SSTableReader> discard = new ArrayList<>();
-    private final boolean isOffline; // true for operations that are performed without Cassandra running (prevents updates of DataTracker)
+    private final List<SSTableWriter> writers = new ArrayList<>();
+    private final boolean isOffline; // true for operations that are performed without Cassandra running (prevents updates of Tracker)
 
     private SSTableWriter writer;
     private Map<DecoratedKey, RowIndexEntry> cachedKeys = new HashMap<>();
-    private State state = State.WORKING;
 
-    private static enum State
+    // for testing (TODO: remove when have byteman setup)
+    private boolean throwEarly, throwLate;
+
+    public SSTableRewriter(ColumnFamilyStore cfs, LifecycleTransaction transaction, long maxAge, boolean isOffline)
     {
-        WORKING, FINISHED, ABORTED
+        this(cfs, transaction, maxAge, isOffline, true);
     }
 
-    public SSTableRewriter(ColumnFamilyStore cfs, Set<SSTableReader> rewriting, long maxAge, boolean isOffline)
+    public SSTableRewriter(ColumnFamilyStore cfs, LifecycleTransaction transaction, long maxAge, boolean isOffline, boolean shouldOpenEarly)
     {
-        this(cfs, rewriting, maxAge, isOffline, true);
-    }
-
-    public SSTableRewriter(ColumnFamilyStore cfs, Set<SSTableReader> rewriting, long maxAge, boolean isOffline, boolean shouldOpenEarly)
-    {
-        this(cfs, rewriting, maxAge, isOffline, calculateOpenInterval(shouldOpenEarly));
+        this(cfs, transaction, maxAge, isOffline, calculateOpenInterval(shouldOpenEarly));
     }
 
     @VisibleForTesting
-    public SSTableRewriter(ColumnFamilyStore cfs, Set<SSTableReader> rewriting, long maxAge, boolean isOffline, long preemptiveOpenInterval)
+    public SSTableRewriter(ColumnFamilyStore cfs, LifecycleTransaction transaction, long maxAge, boolean isOffline, long preemptiveOpenInterval)
     {
-        this.rewriting = rewriting;
-        for (SSTableReader sstable : rewriting)
-        {
-            originalStarts.put(sstable.descriptor, sstable.first);
+        this.transaction = transaction;
+        for (SSTableReader sstable : this.transaction.originals())
             fileDescriptors.put(sstable.descriptor, CLibrary.getfd(sstable.getFilename()));
-        }
-        this.dataTracker = cfs.getDataTracker();
         this.cfs = cfs;
         this.maxAge = maxAge;
         this.isOffline = isOffline;
@@ -110,7 +100,7 @@
     private static long calculateOpenInterval(boolean shouldOpenEarly)
     {
         long interval = DatabaseDescriptor.getSSTablePreempiveOpenIntervalInMB() * (1L << 20);
-        if (!shouldOpenEarly || interval < 0)
+        if (disableEarlyOpeningForTests || !shouldOpenEarly || interval < 0)
             interval = Long.MAX_VALUE;
         return interval;
     }
@@ -134,7 +124,7 @@
             else
             {
                 boolean save = false;
-                for (SSTableReader reader : rewriting)
+                for (SSTableReader reader : transaction.originals())
                 {
                     if (reader.getCachedPosition(row.key, false) != null)
                     {
@@ -170,7 +160,7 @@
         {
             if (isOffline)
             {
-                for (SSTableReader reader : rewriting)
+                for (SSTableReader reader : transaction.originals())
                 {
                     RowIndexEntry index = reader.getPosition(key, SSTableReader.Operator.GE);
                     CLibrary.trySkipCache(fileDescriptors.get(reader.descriptor), 0, index == null ? 0 : index.position);
@@ -178,93 +168,34 @@
             }
             else
             {
-                SSTableReader reader = writer.openEarly(maxAge);
+                SSTableReader reader = writer.setMaxDataAge(maxAge).openEarly();
                 if (reader != null)
                 {
-                    replaceEarlyOpenedFile(currentlyOpenedEarly, reader);
-                    currentlyOpenedEarly = reader;
+                    transaction.update(reader, false);
                     currentlyOpenedEarlyAt = writer.getFilePointer();
-                    moveStarts(reader, reader.last, false);
+                    moveStarts(reader, reader.last);
+                    transaction.checkpoint();
                 }
             }
         }
     }
 
-    public void abort()
+    protected Throwable doAbort(Throwable accumulate)
     {
-        switch (state)
-        {
-            case ABORTED:
-                return;
-            case FINISHED:
-                throw new IllegalStateException("Cannot abort - changes have already been committed");
-        }
-        state = State.ABORTED;
-
-        Throwable fail = null;
-        try
-        {
-            moveStarts(null, null, true);
-        }
-        catch (Throwable t)
-        {
-            fail = merge(fail, t);
-        }
-
-        // remove already completed SSTables
-        for (SSTableReader sstable : finished)
-        {
-            try
-            {
-                sstable.markObsolete(null);
-                sstable.selfRef().release();
-            }
-            catch (Throwable t)
-            {
-                fail = merge(fail, t);
-            }
-        }
-
-        if (writer != null)
-            finishedEarly.add(new Finished(writer, currentlyOpenedEarly));
-
         // abort the writers
-        for (Finished finished : finishedEarly)
-        {
-            try
-            {
-                finished.writer.abort();
-            }
-            catch (Throwable t)
-            {
-                fail = merge(fail, t);
-            }
-            try
-            {
-                if (finished.reader != null)
-                {
-                    // if we've already been opened, add ourselves to the discard pile
-                    discard.add(finished.reader);
-                    finished.reader.markObsolete(null);
-                }
-            }
-            catch (Throwable t)
-            {
-                fail = merge(fail, t);
-            }
-        }
+        for (SSTableWriter writer : writers)
+            accumulate = writer.abort(accumulate);
+        // abort the lifecycle transaction
+        accumulate = transaction.abort(accumulate);
+        return accumulate;
+    }
 
-        try
-        {
-            replaceWithFinishedReaders(Collections.<SSTableReader>emptyList());
-        }
-        catch (Throwable t)
-        {
-            fail = merge(fail, t);
-        }
-
-        if (fail != null)
-            throw Throwables.propagate(fail);
+    protected Throwable doCommit(Throwable accumulate)
+    {
+        for (SSTableWriter writer : writers)
+            accumulate = writer.commit(accumulate);
+        accumulate = transaction.commit(accumulate);
+        return accumulate;
     }
 
     /**
@@ -279,62 +210,47 @@
      *
      * @param newReader the rewritten reader that replaces them for this region
      * @param lowerbound if !reset, must be non-null, and marks the exclusive lowerbound of the start for each sstable
-     * @param reset true iff we are restoring earlier starts (increasing the range over which they are valid)
      */
-    private void moveStarts(SSTableReader newReader, DecoratedKey lowerbound, boolean reset)
+    private void moveStarts(SSTableReader newReader, DecoratedKey lowerbound)
     {
         if (isOffline)
             return;
-        List<SSTableReader> toReplace = new ArrayList<>();
-        List<SSTableReader> replaceWith = new ArrayList<>();
+        if (preemptiveOpenInterval == Long.MAX_VALUE)
+            return;
+
         final List<DecoratedKey> invalidateKeys = new ArrayList<>();
-        if (!reset)
-        {
-            newReader.setupKeyCache();
-            invalidateKeys.addAll(cachedKeys.keySet());
-            for (Map.Entry<DecoratedKey, RowIndexEntry> cacheKey : cachedKeys.entrySet())
-                newReader.cacheKey(cacheKey.getKey(), cacheKey.getValue());
-        }
+        invalidateKeys.addAll(cachedKeys.keySet());
+        newReader.setupKeyCache();
+        for (Map.Entry<DecoratedKey, RowIndexEntry> cacheKey : cachedKeys.entrySet())
+            newReader.cacheKey(cacheKey.getKey(), cacheKey.getValue());
 
         cachedKeys = new HashMap<>();
-        for (SSTableReader sstable : ImmutableList.copyOf(rewriting))
+        for (SSTableReader sstable : transaction.originals())
         {
             // we call getCurrentReplacement() to support multiple rewriters operating over the same source readers at once.
             // note: only one such writer should be written to at any moment
-            final SSTableReader latest = dataTracker.getCurrentVersion(sstable);
-            SSTableReader replacement;
-            if (reset)
-            {
-                DecoratedKey newStart = originalStarts.get(sstable.descriptor);
-                replacement = latest.cloneWithNewStart(newStart, null);
-            }
-            else
-            {
-                // skip any sstables that we know to already be shadowed
-                if (latest.openReason == SSTableReader.OpenReason.SHADOWED)
-                    continue;
-                if (latest.first.compareTo(lowerbound) > 0)
-                    continue;
+            final SSTableReader latest = transaction.current(sstable);
 
-                final Runnable runOnClose = new InvalidateKeys(latest, invalidateKeys);
-                if (lowerbound.compareTo(latest.last) >= 0)
+            // skip any sstables that we know to already be shadowed
+            if (latest.first.compareTo(lowerbound) > 0)
+                continue;
+
+            Runnable runOnClose = new InvalidateKeys(latest, invalidateKeys);
+            if (lowerbound.compareTo(latest.last) >= 0)
+            {
+                if (!transaction.isObsolete(latest))
                 {
-                    replacement = latest.cloneAsShadowed(runOnClose);
+                    latest.runOnClose(runOnClose);
+                    transaction.obsolete(latest);
                 }
-                else
-                {
-                    DecoratedKey newStart = latest.firstKeyBeyond(lowerbound);
-                    assert newStart != null;
-                    replacement = latest.cloneWithNewStart(newStart, runOnClose);
-                }
+                continue;
             }
 
-            toReplace.add(latest);
-            replaceWith.add(replacement);
-            rewriting.remove(sstable);
-            rewriting.add(replacement);
+            DecoratedKey newStart = latest.firstKeyBeyond(lowerbound);
+            assert newStart != null;
+            SSTableReader replacement = latest.cloneWithNewStart(newStart, runOnClose);
+            transaction.update(replacement, true);
         }
-        cfs.getDataTracker().replaceWithNewInstances(toReplace, replaceWith);
     }
 
     private static final class InvalidateKeys implements Runnable
@@ -363,173 +279,105 @@
         }
     }
 
-    private void replaceEarlyOpenedFile(SSTableReader toReplace, SSTableReader replaceWith)
-    {
-        if (isOffline)
-            return;
-        Set<SSTableReader> toReplaceSet;
-        if (toReplace != null)
-        {
-            toReplace.setReplacedBy(replaceWith);
-            toReplaceSet = Collections.singleton(toReplace);
-        }
-        else
-        {
-            dataTracker.markCompacting(Collections.singleton(replaceWith), true, isOffline);
-            toReplaceSet = Collections.emptySet();
-        }
-        dataTracker.replaceEarlyOpenedFiles(toReplaceSet, Collections.singleton(replaceWith));
-    }
-
     public void switchWriter(SSTableWriter newWriter)
     {
-        if (writer == null)
+        if (newWriter != null)
+            writers.add(newWriter.setMaxDataAge(maxAge));
+
+        if (writer == null || writer.getFilePointer() == 0)
         {
+            if (writer != null)
+            {
+                writer.abort();
+                writers.remove(writer);
+            }
             writer = newWriter;
             return;
         }
 
-        if (writer.getFilePointer() != 0)
+        SSTableReader reader = null;
+        if (preemptiveOpenInterval != Long.MAX_VALUE)
         {
-            // If early re-open is disabled, simply finalize the writer and store it
-            if (preemptiveOpenInterval == Long.MAX_VALUE)
-            {
-                SSTableReader reader = writer.finish(SSTableWriter.FinishType.NORMAL, maxAge, -1);
-                finishedReaders.add(reader);
-            }
-            else
-            {
-                // we leave it as a tmp file, but we open it and add it to the dataTracker
-                SSTableReader reader = writer.finish(SSTableWriter.FinishType.EARLY, maxAge, -1);
-                replaceEarlyOpenedFile(currentlyOpenedEarly, reader);
-                moveStarts(reader, reader.last, false);
-                finishedEarly.add(new Finished(writer, reader));
-            }
+            // we leave it as a tmp file, but we open it and add it to the Tracker
+            reader = writer.setMaxDataAge(maxAge).openFinalEarly();
+            transaction.update(reader, false);
+            moveStarts(reader, reader.last);
+            transaction.checkpoint();
         }
-        else
-        {
-            writer.abort();
-        }
-        currentlyOpenedEarly = null;
+
         currentlyOpenedEarlyAt = 0;
         writer = newWriter;
     }
 
-    public List<SSTableReader> finish()
+    /**
+     * @param repairedAt the repair time, -1 if we should use the time we supplied when we created
+     *                   the SSTableWriter (and called rewriter.switchWriter(..)), actual time if we want to override the
+     *                   repair time.
+     */
+    public SSTableRewriter setRepairedAt(long repairedAt)
     {
-        return finish(-1);
+        this.repairedAt = repairedAt;
+        return this;
     }
 
     /**
      * Finishes the new file(s)
      *
-     * Creates final files, adds the new files to the dataTracker (via replaceReader).
+     * Creates final files, adds the new files to the Tracker (via replaceReader).
      *
      * We add them to the tracker to be able to get rid of the tmpfiles
      *
      * It is up to the caller to do the compacted sstables replacement
-     * gymnastics (ie, call DataTracker#markCompactedSSTablesReplaced(..))
+     * gymnastics (ie, call Tracker#markCompactedSSTablesReplaced(..))
      *
      *
-     * @param repairedAt the repair time, -1 if we should use the time we supplied when we created
-     *                   the SSTableWriter (and called rewriter.switchWriter(..)), actual time if we want to override the
-     *                   repair time.
      */
-    public List<SSTableReader> finish(long repairedAt)
+    public List<SSTableReader> finish()
     {
-        return finishAndMaybeThrow(repairedAt, false, false);
+        super.finish();
+        return finished();
     }
 
-    @VisibleForTesting
-    void finishAndThrow(boolean throwEarly)
+    // returns, in list form, the
+    public List<SSTableReader> finished()
     {
-        finishAndMaybeThrow(-1, throwEarly, !throwEarly);
+        assert state() == State.COMMITTED || state() == State.READY_TO_COMMIT;
+        return preparedForCommit;
     }
 
-    private List<SSTableReader> finishAndMaybeThrow(long repairedAt, boolean throwEarly, boolean throwLate)
+    protected void doPrepare()
     {
-        switch (state)
-        {
-            case FINISHED: case ABORTED:
-                throw new IllegalStateException("Cannot finish - changes have already been " + state.toString().toLowerCase());
-        }
-
-        List<SSTableReader> newReaders = new ArrayList<>();
         switchWriter(null);
 
         if (throwEarly)
             throw new RuntimeException("exception thrown early in finish, for testing");
 
         // No early open to finalize and replace
-        if (preemptiveOpenInterval == Long.MAX_VALUE)
+        for (SSTableWriter writer : writers)
         {
-            replaceWithFinishedReaders(finishedReaders);
-            if (throwLate)
-                throw new RuntimeException("exception thrown after all sstables finished, for testing");
-            return finishedReaders;
+            assert writer.getFilePointer() > 0;
+            writer.setRepairedAt(repairedAt).setOpenResult(true).prepareToCommit();
+            SSTableReader reader = writer.finished();
+            transaction.update(reader, false);
+            preparedForCommit.add(reader);
         }
-
-        while (!finishedEarly.isEmpty())
-        {
-            Finished f = finishedEarly.peek();
-            if (f.writer.getFilePointer() > 0)
-            {
-                if (f.reader != null)
-                    discard.add(f.reader);
-
-                SSTableReader newReader = f.writer.finish(SSTableWriter.FinishType.FINISH_EARLY, maxAge, repairedAt);
-
-                if (f.reader != null)
-                    f.reader.setReplacedBy(newReader);
-
-                finished.add(newReader);
-                newReaders.add(newReader);
-            }
-            else
-            {
-                f.writer.abort();
-                assert f.reader == null;
-            }
-            finishedEarly.poll();
-        }
+        transaction.checkpoint();
 
         if (throwLate)
             throw new RuntimeException("exception thrown after all sstables finished, for testing");
 
-        replaceWithFinishedReaders(newReaders);
-        state = State.FINISHED;
-        return finished;
+        // TODO: do we always want to avoid obsoleting if offline?
+        if (!isOffline)
+            transaction.obsoleteOriginals();
+
+        transaction.prepareToCommit();
     }
 
-    // cleanup all our temporary readers and swap in our new ones
-    private void replaceWithFinishedReaders(List<SSTableReader> finished)
+    public void throwDuringPrepare(boolean earlyException)
     {
-        if (isOffline)
-        {
-            for (SSTableReader reader : discard)
-            {
-                if (!reader.isReplaced())
-                    reader.markObsolete(null);
-                reader.selfRef().release();
-            }
-        }
+        if (earlyException)
+            throwEarly = true;
         else
-        {
-            dataTracker.replaceEarlyOpenedFiles(discard, finished);
-            dataTracker.unmarkCompacting(discard);
-        }
-        discard.clear();
-    }
-
-    private static final class Finished
-    {
-        final SSTableWriter writer;
-        final SSTableReader reader;
-
-        private Finished(SSTableWriter writer, SSTableReader reader)
-        {
-            this.writer = writer;
-            this.reader = reader;
-        }
+            throwLate = true;
     }
 }
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableScanner.java b/src/java/org/apache/cassandra/io/sstable/SSTableScanner.java
deleted file mode 100644
index ad19f22..0000000
--- a/src/java/org/apache/cassandra/io/sstable/SSTableScanner.java
+++ /dev/null
@@ -1,363 +0,0 @@
-/*
- * 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.io.sstable;
-
-import java.io.IOException;
-import java.util.*;
-
-import com.google.common.collect.AbstractIterator;
-import com.google.common.collect.Ordering;
-import com.google.common.util.concurrent.RateLimiter;
-
-import org.apache.cassandra.db.DataRange;
-import org.apache.cassandra.db.DecoratedKey;
-import org.apache.cassandra.db.RowIndexEntry;
-import org.apache.cassandra.db.RowPosition;
-import org.apache.cassandra.db.columniterator.IColumnIteratorFactory;
-import org.apache.cassandra.db.columniterator.LazyColumnIterator;
-import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
-import org.apache.cassandra.dht.AbstractBounds;
-import org.apache.cassandra.dht.AbstractBounds.Boundary;
-import org.apache.cassandra.dht.Bounds;
-import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.io.util.RandomAccessReader;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.Pair;
-
-import static org.apache.cassandra.dht.AbstractBounds.isEmpty;
-import static org.apache.cassandra.dht.AbstractBounds.maxLeft;
-import static org.apache.cassandra.dht.AbstractBounds.minRight;
-
-public class SSTableScanner implements ISSTableScanner
-{
-    protected final RandomAccessReader dfile;
-    protected final RandomAccessReader ifile;
-    public final SSTableReader sstable;
-
-    private final Iterator<AbstractBounds<RowPosition>> rangeIterator;
-    private AbstractBounds<RowPosition> currentRange;
-
-    private final DataRange dataRange;
-
-    protected Iterator<OnDiskAtomIterator> iterator;
-
-    public static ISSTableScanner getScanner(SSTableReader sstable, DataRange dataRange, RateLimiter limiter)
-    {
-        return new SSTableScanner(sstable, dataRange, limiter);
-    }
-    public static ISSTableScanner getScanner(SSTableReader sstable, Collection<Range<Token>> tokenRanges, RateLimiter limiter)
-    {
-        // We want to avoid allocating a SSTableScanner if the range don't overlap the sstable (#5249)
-        List<Pair<Long, Long>> positions = sstable.getPositionsForRanges(tokenRanges);
-        if (positions.isEmpty())
-            return new EmptySSTableScanner(sstable.getFilename());
-
-        return new SSTableScanner(sstable, tokenRanges, limiter);
-    }
-
-    /**
-     * @param sstable SSTable to scan; must not be null
-     * @param dataRange a single range to scan; must not be null
-     * @param limiter background i/o RateLimiter; may be null
-     */
-    private SSTableScanner(SSTableReader sstable, DataRange dataRange, RateLimiter limiter)
-    {
-        assert sstable != null;
-
-        this.dfile = limiter == null ? sstable.openDataReader() : sstable.openDataReader(limiter);
-        this.ifile = sstable.openIndexReader();
-        this.sstable = sstable;
-        this.dataRange = dataRange;
-
-        List<AbstractBounds<RowPosition>> boundsList = new ArrayList<>(2);
-        addRange(dataRange.keyRange(), boundsList);
-        this.rangeIterator = boundsList.iterator();
-    }
-
-    /**
-     * @param sstable SSTable to scan; must not be null
-     * @param tokenRanges A set of token ranges to scan
-     * @param limiter background i/o RateLimiter; may be null
-     */
-    private SSTableScanner(SSTableReader sstable, Collection<Range<Token>> tokenRanges, RateLimiter limiter)
-    {
-        assert sstable != null;
-
-        this.dfile = limiter == null ? sstable.openDataReader() : sstable.openDataReader(limiter);
-        this.ifile = sstable.openIndexReader();
-        this.sstable = sstable;
-        this.dataRange = null;
-
-        List<AbstractBounds<RowPosition>> boundsList = new ArrayList<>(tokenRanges.size());
-        for (Range<Token> range : Range.normalize(tokenRanges))
-            addRange(range.toRowBounds(), boundsList);
-
-        this.rangeIterator = boundsList.iterator();
-    }
-
-    private void addRange(AbstractBounds<RowPosition> requested, List<AbstractBounds<RowPosition>> boundsList)
-    {
-        if (requested instanceof Range && ((Range)requested).isWrapAround())
-        {
-            if (requested.right.compareTo(sstable.first) >= 0)
-            {
-                // since we wrap, we must contain the whole sstable prior to stopKey()
-                Boundary<RowPosition> left = new Boundary<RowPosition>(sstable.first, true);
-                Boundary<RowPosition> right;
-                right = requested.rightBoundary();
-                right = minRight(right, sstable.last, true);
-                if (!isEmpty(left, right))
-                    boundsList.add(AbstractBounds.bounds(left, right));
-            }
-            if (requested.left.compareTo(sstable.last) <= 0)
-            {
-                // since we wrap, we must contain the whole sstable after dataRange.startKey()
-                Boundary<RowPosition> right = new Boundary<RowPosition>(sstable.last, true);
-                Boundary<RowPosition> left;
-                left = requested.leftBoundary();
-                left = maxLeft(left, sstable.first, true);
-                if (!isEmpty(left, right))
-                    boundsList.add(AbstractBounds.bounds(left, right));
-            }
-        }
-        else
-        {
-            assert requested.left.compareTo(requested.right) <= 0 || requested.right.isMinimum();
-            Boundary<RowPosition> left, right;
-            left = requested.leftBoundary();
-            right = requested.rightBoundary();
-            left = maxLeft(left, sstable.first, true);
-            // apparently isWrapAround() doesn't count Bounds that extend to the limit (min) as wrapping
-            right = requested.right.isMinimum() ? new Boundary<RowPosition>(sstable.last, true)
-                                                    : minRight(right, sstable.last, true);
-            if (!isEmpty(left, right))
-                boundsList.add(AbstractBounds.bounds(left, right));
-        }
-    }
-
-    private void seekToCurrentRangeStart()
-    {
-        long indexPosition = sstable.getIndexScanPosition(currentRange.left);
-        ifile.seek(indexPosition);
-        try
-        {
-
-            while (!ifile.isEOF())
-            {
-                indexPosition = ifile.getFilePointer();
-                DecoratedKey indexDecoratedKey = sstable.partitioner.decorateKey(ByteBufferUtil.readWithShortLength(ifile));
-                if (indexDecoratedKey.compareTo(currentRange.left) > 0 || currentRange.contains(indexDecoratedKey))
-                {
-                    // Found, just read the dataPosition and seek into index and data files
-                    long dataPosition = ifile.readLong();
-                    ifile.seek(indexPosition);
-                    dfile.seek(dataPosition);
-                    break;
-                }
-                else
-                {
-                    RowIndexEntry.Serializer.skip(ifile);
-                }
-            }
-        }
-        catch (IOException e)
-        {
-            sstable.markSuspect();
-            throw new CorruptSSTableException(e, sstable.getFilename());
-        }
-    }
-
-    public void close() throws IOException
-    {
-        FileUtils.close(dfile, ifile);
-    }
-
-    public long getLengthInBytes()
-    {
-        return dfile.length();
-    }
-
-    public long getCurrentPosition()
-    {
-        return dfile.getFilePointer();
-    }
-
-    public String getBackingFiles()
-    {
-        return sstable.toString();
-    }
-
-    public boolean hasNext()
-    {
-        if (iterator == null)
-            iterator = createIterator();
-        return iterator.hasNext();
-    }
-
-    public OnDiskAtomIterator next()
-    {
-        if (iterator == null)
-            iterator = createIterator();
-        return iterator.next();
-    }
-
-    public void remove()
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    private Iterator<OnDiskAtomIterator> createIterator()
-    {
-        return new KeyScanningIterator();
-    }
-
-    protected class KeyScanningIterator extends AbstractIterator<OnDiskAtomIterator>
-    {
-        private DecoratedKey nextKey;
-        private RowIndexEntry nextEntry;
-        private DecoratedKey currentKey;
-        private RowIndexEntry currentEntry;
-
-        protected OnDiskAtomIterator computeNext()
-        {
-            try
-            {
-                if (nextEntry == null)
-                {
-                    do
-                    {
-                        // we're starting the first range or we just passed the end of the previous range
-                        if (!rangeIterator.hasNext())
-                            return endOfData();
-
-                        currentRange = rangeIterator.next();
-                        seekToCurrentRangeStart();
-
-                        if (ifile.isEOF())
-                            return endOfData();
-
-                        currentKey = sstable.partitioner.decorateKey(ByteBufferUtil.readWithShortLength(ifile));
-                        currentEntry = sstable.metadata.comparator.rowIndexEntrySerializer().deserialize(ifile, sstable.descriptor.version);
-                    } while (!currentRange.contains(currentKey));
-                }
-                else
-                {
-                    // we're in the middle of a range
-                    currentKey = nextKey;
-                    currentEntry = nextEntry;
-                }
-
-                long readEnd;
-                if (ifile.isEOF())
-                {
-                    nextEntry = null;
-                    nextKey = null;
-                    readEnd = dfile.length();
-                }
-                else
-                {
-                    // we need the position of the start of the next key, regardless of whether it falls in the current range
-                    nextKey = sstable.partitioner.decorateKey(ByteBufferUtil.readWithShortLength(ifile));
-                    nextEntry = sstable.metadata.comparator.rowIndexEntrySerializer().deserialize(ifile, sstable.descriptor.version);
-                    readEnd = nextEntry.position;
-
-                    if (!currentRange.contains(nextKey))
-                    {
-                        nextKey = null;
-                        nextEntry = null;
-                    }
-                }
-
-                if (dataRange == null || dataRange.selectsFullRowFor(currentKey.getKey()))
-                {
-                    dfile.seek(currentEntry.position);
-                    ByteBufferUtil.readWithShortLength(dfile); // key
-                    long dataSize = readEnd - dfile.getFilePointer();
-                    return new SSTableIdentityIterator(sstable, dfile, currentKey, dataSize);
-                }
-
-                return new LazyColumnIterator(currentKey, new IColumnIteratorFactory()
-                {
-                    public OnDiskAtomIterator create()
-                    {
-                        return dataRange.columnFilter(currentKey.getKey()).getSSTableColumnIterator(sstable, dfile, currentKey, currentEntry);
-                    }
-                });
-
-            }
-            catch (CorruptSSTableException | IOException e)
-            {
-                sstable.markSuspect();
-                throw new CorruptSSTableException(e, sstable.getFilename());
-            }
-        }
-    }
-
-    @Override
-    public String toString()
-    {
-        return getClass().getSimpleName() + "(" +
-               "dfile=" + dfile +
-               " ifile=" + ifile +
-               " sstable=" + sstable +
-               ")";
-    }
-
-    public static class EmptySSTableScanner implements ISSTableScanner
-    {
-        private final String filename;
-
-        public EmptySSTableScanner(String filename)
-        {
-            this.filename = filename;
-        }
-
-        public long getLengthInBytes()
-        {
-            return 0;
-        }
-
-        public long getCurrentPosition()
-        {
-            return 0;
-        }
-
-        public String getBackingFiles()
-        {
-            return filename;
-        }
-
-        public boolean hasNext()
-        {
-            return false;
-        }
-
-        public OnDiskAtomIterator next()
-        {
-            return null;
-        }
-
-        public void close() throws IOException { }
-
-        public void remove() { }
-    }
-
-
-}
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
index 25ec354..534e77b 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
@@ -36,7 +36,7 @@
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.io.compress.CompressionParameters;
-import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 
 /**
@@ -47,7 +47,10 @@
  * created (and the buffer be reseted).
  *
  * @see AbstractSSTableSimpleWriter
+ *
+ * @deprecated this class is depracted in favor of {@link CQLSSTableWriter}.
  */
+@Deprecated
 public class SSTableSimpleUnsortedWriter extends AbstractSSTableSimpleWriter
 {
     private static final Buffer SENTINEL = new Buffer();
@@ -97,8 +100,8 @@
     public SSTableSimpleUnsortedWriter(File directory, CFMetaData metadata, IPartitioner partitioner, long bufferSizeInMB)
     {
         super(directory, metadata, partitioner);
-        this.bufferSize = bufferSizeInMB * 1024L * 1024L;
-        this.diskWriter.start();
+        bufferSize = bufferSizeInMB * 1024L * 1024L;
+        diskWriter.start();
     }
 
     protected void writeRow(DecoratedKey key, ColumnFamily columnFamily) throws IOException
@@ -122,7 +125,7 @@
             replaceColumnFamily();
     }
 
-    protected ColumnFamily getColumnFamily() throws IOException
+    protected ColumnFamily getColumnFamily()
     {
         ColumnFamily previous = buffer.get(currentKey);
         // If the CF already exist in memory, we'll just continue adding to it
@@ -146,7 +149,7 @@
         throw new UnsupportedOperationException();
     }
 
-    protected ColumnFamily createColumnFamily() throws IOException
+    protected ColumnFamily createColumnFamily()
     {
         return ArrayBackedSortedColumns.factory.create(metadata);
     }
@@ -163,6 +166,7 @@
         {
             throw new RuntimeException(e);
         }
+        checkForWriterException();
     }
 
     // This is overridden by CQLSSTableWriter to hold off replacing column family until the next iteration through
@@ -232,32 +236,37 @@
 
         public void run()
         {
-            SSTableWriter writer = null;
-            try
             {
                 while (true)
                 {
-                    Buffer b = writeQueue.take();
-                    if (b == SENTINEL)
-                        return;
-
-                    writer = getWriter();
-                    for (Map.Entry<DecoratedKey, ColumnFamily> entry : b.entrySet())
+                    try
                     {
-                        if (entry.getValue().getColumnCount() > 0)
-                            writer.append(entry.getKey(), entry.getValue());
-                        else if (!entry.getKey().equals(b.getFirstInsertedKey()))
-                            throw new AssertionError("Empty partition");
+                        Buffer b = writeQueue.take();
+                        if (b == SENTINEL)
+                            return;
+
+                        try (SSTableWriter writer = getWriter();)
+                        {
+                            for (Map.Entry<DecoratedKey, ColumnFamily> entry : b.entrySet())
+                            {
+                                if (entry.getValue().getColumnCount() > 0)
+                                    writer.append(entry.getKey(), entry.getValue());
+                                else if (!entry.getKey().equals(b.getFirstInsertedKey()))
+                                    throw new AssertionError("Empty partition");
+                            }
+                            
+                            writer.finish(false);
+                        }
                     }
-                    writer.close();
+                    catch (Throwable e)
+                    {
+                        JVMStabilityInspector.inspectThrowable(e);
+                        // Keep only the first exception
+                        if (exception == null)
+                            exception = e;
+                    }
                 }
-            }
-            catch (Throwable e)
-            {
-                JVMStabilityInspector.inspectThrowable(e);
-                if (writer != null)
-                    writer.abort();
-                exception = e;
+
             }
         }
     }
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
index 23da501..f81e57d 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleWriter.java
@@ -19,11 +19,15 @@
 
 import java.io.File;
 
+import com.google.common.base.Throwables;
+
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.io.FSError;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
 
 /**
  * A SSTable writer that assumes rows are in (partitioner) sorted order.
@@ -35,7 +39,10 @@
  * prefered.
  *
  * @see AbstractSSTableSimpleWriter
+ *
+ * @deprecated this class is depracted in favor of {@link CQLSSTableWriter}.
  */
+@Deprecated
 public class SSTableSimpleWriter extends AbstractSSTableSimpleWriter
 {
     private final SSTableWriter writer;
@@ -69,7 +76,7 @@
     {
         if (currentKey != null)
             writeRow(currentKey, columnFamily);
-        return writer.closeAndOpenReader();
+        return writer.finish(true);
     }
 
     public void close()
@@ -78,12 +85,11 @@
         {
             if (currentKey != null)
                 writeRow(currentKey, columnFamily);
-            writer.close();
+            writer.finish(false);
         }
-        catch (FSError e)
+        catch (Throwable t)
         {
-            writer.abort();
-            throw e;
+            throw Throwables.propagate(writer.abort(t));
         }
     }
 
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
deleted file mode 100644
index 8620f30..0000000
--- a/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
+++ /dev/null
@@ -1,694 +0,0 @@
-/*
- * 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.io.sstable;
-
-import java.io.*;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import com.google.common.collect.Sets;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.db.ArrayBackedSortedColumns;
-import org.apache.cassandra.db.ColumnFamily;
-import org.apache.cassandra.db.ColumnIndex;
-import org.apache.cassandra.db.ColumnSerializer;
-import org.apache.cassandra.db.CounterCell;
-import org.apache.cassandra.db.DecoratedKey;
-import org.apache.cassandra.db.DeletionTime;
-import org.apache.cassandra.db.OnDiskAtom;
-import org.apache.cassandra.db.RangeTombstone;
-import org.apache.cassandra.db.RowIndexEntry;
-import org.apache.cassandra.db.compaction.AbstractCompactedRow;
-import org.apache.cassandra.dht.IPartitioner;
-import org.apache.cassandra.io.FSWriteError;
-import org.apache.cassandra.io.compress.CompressedSequentialWriter;
-import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
-import org.apache.cassandra.io.sstable.metadata.MetadataComponent;
-import org.apache.cassandra.io.sstable.metadata.MetadataType;
-import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
-import org.apache.cassandra.io.util.*;
-import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.FilterFactory;
-import org.apache.cassandra.utils.IFilter;
-import org.apache.cassandra.utils.Pair;
-import org.apache.cassandra.utils.StreamingHistogram;
-
-public class SSTableWriter extends SSTable
-{
-    private static final Logger logger = LoggerFactory.getLogger(SSTableWriter.class);
-
-    // not very random, but the only value that can't be mistaken for a legal column-name length
-    public static final int END_OF_ROW = 0x0000;
-
-    private IndexWriter iwriter;
-    private SegmentedFile.Builder dbuilder;
-    private final SequentialWriter dataFile;
-    private DecoratedKey lastWrittenKey;
-    private FileMark dataMark;
-    private final MetadataCollector sstableMetadataCollector;
-    private final long repairedAt;
-
-    public SSTableWriter(String filename, long keyCount, long repairedAt)
-    {
-        this(filename,
-             keyCount,
-             repairedAt,
-             Schema.instance.getCFMetaData(Descriptor.fromFilename(filename)),
-             StorageService.getPartitioner(),
-             new MetadataCollector(Schema.instance.getCFMetaData(Descriptor.fromFilename(filename)).comparator));
-    }
-
-    private static Set<Component> components(CFMetaData metadata)
-    {
-        Set<Component> components = new HashSet<Component>(Arrays.asList(Component.DATA,
-                                                                         Component.PRIMARY_INDEX,
-                                                                         Component.STATS,
-                                                                         Component.SUMMARY,
-                                                                         Component.TOC,
-                                                                         Component.DIGEST));
-
-        if (metadata.getBloomFilterFpChance() < 1.0)
-            components.add(Component.FILTER);
-
-        if (metadata.compressionParameters().sstableCompressor != null)
-        {
-            components.add(Component.COMPRESSION_INFO);
-        }
-        else
-        {
-            // it would feel safer to actually add this component later in maybeWriteDigest(),
-            // but the components are unmodifiable after construction
-            components.add(Component.CRC);
-        }
-        return components;
-    }
-
-    public SSTableWriter(String filename,
-                         long keyCount,
-                         long repairedAt,
-                         CFMetaData metadata,
-                         IPartitioner partitioner,
-                         MetadataCollector sstableMetadataCollector)
-    {
-        super(Descriptor.fromFilename(filename),
-              components(metadata),
-              metadata,
-              partitioner);
-        this.repairedAt = repairedAt;
-
-        if (compression)
-        {
-            dataFile = SequentialWriter.open(getFilename(),
-                                             descriptor.filenameFor(Component.COMPRESSION_INFO),
-                                             metadata.compressionParameters(),
-                                             sstableMetadataCollector);
-            dbuilder = SegmentedFile.getCompressedBuilder((CompressedSequentialWriter) dataFile);
-        }
-        else
-        {
-            dataFile = SequentialWriter.open(new File(getFilename()), new File(descriptor.filenameFor(Component.CRC)));
-            dbuilder = SegmentedFile.getBuilder(DatabaseDescriptor.getDiskAccessMode());
-        }
-        iwriter = new IndexWriter(keyCount, dataFile);
-
-        this.sstableMetadataCollector = sstableMetadataCollector;
-    }
-
-    public void mark()
-    {
-        dataMark = dataFile.mark();
-        iwriter.mark();
-    }
-
-    public void resetAndTruncate()
-    {
-        dataFile.resetAndTruncate(dataMark);
-        iwriter.resetAndTruncate();
-    }
-
-    /**
-     * Perform sanity checks on @param decoratedKey and @return the position in the data file before any data is written
-     */
-    private long beforeAppend(DecoratedKey decoratedKey)
-    {
-        assert decoratedKey != null : "Keys must not be null"; // empty keys ARE allowed b/c of indexed column values
-        if (lastWrittenKey != null && lastWrittenKey.compareTo(decoratedKey) >= 0)
-            throw new RuntimeException("Last written key " + lastWrittenKey + " >= current key " + decoratedKey + " writing into " + getFilename());
-        return (lastWrittenKey == null) ? 0 : dataFile.getFilePointer();
-    }
-
-    private void afterAppend(DecoratedKey decoratedKey, long dataEnd, RowIndexEntry index)
-    {
-        sstableMetadataCollector.addKey(decoratedKey.getKey());
-        lastWrittenKey = decoratedKey;
-        last = lastWrittenKey;
-        if (first == null)
-            first = lastWrittenKey;
-
-        if (logger.isTraceEnabled())
-            logger.trace("wrote " + decoratedKey + " at " + dataEnd);
-        iwriter.append(decoratedKey, index, dataEnd);
-        dbuilder.addPotentialBoundary(dataEnd);
-    }
-
-    /**
-     * @param row
-     * @return null if the row was compacted away entirely; otherwise, the PK index entry for this row
-     */
-    public RowIndexEntry append(AbstractCompactedRow row)
-    {
-        long startPosition = beforeAppend(row.key);
-        RowIndexEntry entry;
-        try
-        {
-            entry = row.write(startPosition, dataFile.stream);
-            if (entry == null)
-                return null;
-        }
-        catch (IOException e)
-        {
-            throw new FSWriteError(e, dataFile.getPath());
-        }
-        long endPosition = dataFile.getFilePointer();
-        long rowSize = endPosition - startPosition;
-        maybeLogLargePartitionWarning(row.key, rowSize);
-        sstableMetadataCollector.update(rowSize, row.columnStats());
-        afterAppend(row.key, endPosition, entry);
-        return entry;
-    }
-
-    public void append(DecoratedKey decoratedKey, ColumnFamily cf)
-    {
-        if (decoratedKey.getKey().remaining() > FBUtilities.MAX_UNSIGNED_SHORT)
-        {
-            logger.error("Key size {} exceeds maximum of {}, skipping row",
-                         decoratedKey.getKey().remaining(),
-                         FBUtilities.MAX_UNSIGNED_SHORT);
-            return;
-        }
-
-        long startPosition = beforeAppend(decoratedKey);
-        long endPosition;
-        try
-        {
-            RowIndexEntry entry = rawAppend(cf, startPosition, decoratedKey, dataFile.stream);
-            endPosition = dataFile.getFilePointer();
-            afterAppend(decoratedKey, endPosition, entry);
-        }
-        catch (IOException e)
-        {
-            throw new FSWriteError(e, dataFile.getPath());
-        }
-        long rowSize = endPosition - startPosition;
-        maybeLogLargePartitionWarning(decoratedKey, rowSize);
-        sstableMetadataCollector.update(rowSize, cf.getColumnStats());
-    }
-
-    private void maybeLogLargePartitionWarning(DecoratedKey key, long rowSize)
-    {
-        if (rowSize > DatabaseDescriptor.getCompactionLargePartitionWarningThreshold())
-        {
-            String keyString = metadata.getKeyValidator().getString(key.getKey());
-            logger.warn("Compacting large partition {}/{}:{} ({} bytes)", metadata.ksName, metadata.cfName, keyString, rowSize);
-        }
-    }
-
-    public static RowIndexEntry rawAppend(ColumnFamily cf, long startPosition, DecoratedKey key, DataOutputPlus out) throws IOException
-    {
-        assert cf.hasColumns() || cf.isMarkedForDelete();
-
-        ColumnIndex.Builder builder = new ColumnIndex.Builder(cf, key.getKey(), out);
-        ColumnIndex index = builder.build(cf);
-
-        out.writeShort(END_OF_ROW);
-        return RowIndexEntry.create(startPosition, cf.deletionInfo().getTopLevelDeletion(), index);
-    }
-
-    /**
-     * @throws IOException if a read from the DataInput fails
-     * @throws FSWriteError if a write to the dataFile fails
-     */
-    public long appendFromStream(DecoratedKey key, CFMetaData metadata, DataInput in, Descriptor.Version version) throws IOException
-    {
-        long currentPosition = beforeAppend(key);
-
-        ColumnStats.MaxLongTracker maxTimestampTracker = new ColumnStats.MaxLongTracker(Long.MAX_VALUE);
-        ColumnStats.MinLongTracker minTimestampTracker = new ColumnStats.MinLongTracker(Long.MIN_VALUE);
-        ColumnStats.MaxIntTracker maxDeletionTimeTracker = new ColumnStats.MaxIntTracker(Integer.MAX_VALUE);
-        List<ByteBuffer> minColumnNames = Collections.emptyList();
-        List<ByteBuffer> maxColumnNames = Collections.emptyList();
-        StreamingHistogram tombstones = new StreamingHistogram(TOMBSTONE_HISTOGRAM_BIN_SIZE);
-        boolean hasLegacyCounterShards = false;
-
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(metadata);
-        cf.delete(DeletionTime.serializer.deserialize(in));
-
-        ColumnIndex.Builder columnIndexer = new ColumnIndex.Builder(cf, key.getKey(), dataFile.stream);
-
-        if (cf.deletionInfo().getTopLevelDeletion().localDeletionTime < Integer.MAX_VALUE)
-        {
-            tombstones.update(cf.deletionInfo().getTopLevelDeletion().localDeletionTime);
-            maxDeletionTimeTracker.update(cf.deletionInfo().getTopLevelDeletion().localDeletionTime);
-            minTimestampTracker.update(cf.deletionInfo().getTopLevelDeletion().markedForDeleteAt);
-            maxTimestampTracker.update(cf.deletionInfo().getTopLevelDeletion().markedForDeleteAt);
-        }
-
-        Iterator<RangeTombstone> rangeTombstoneIterator = cf.deletionInfo().rangeIterator();
-        while (rangeTombstoneIterator.hasNext())
-        {
-            RangeTombstone rangeTombstone = rangeTombstoneIterator.next();
-            tombstones.update(rangeTombstone.getLocalDeletionTime());
-            minTimestampTracker.update(rangeTombstone.timestamp());
-            maxTimestampTracker.update(rangeTombstone.timestamp());
-            maxDeletionTimeTracker.update(rangeTombstone.getLocalDeletionTime());
-            minColumnNames = ColumnNameHelper.minComponents(minColumnNames, rangeTombstone.min, metadata.comparator);
-            maxColumnNames = ColumnNameHelper.maxComponents(maxColumnNames, rangeTombstone.max, metadata.comparator);
-        }
-
-        Iterator<OnDiskAtom> iter = metadata.getOnDiskIterator(in, ColumnSerializer.Flag.PRESERVE_SIZE, Integer.MIN_VALUE, version);
-        try
-        {
-            while (iter.hasNext())
-            {
-                OnDiskAtom atom = iter.next();
-                if (atom == null)
-                    break;
-
-                if (atom instanceof CounterCell)
-                {
-                    atom = ((CounterCell) atom).markLocalToBeCleared();
-                    hasLegacyCounterShards = hasLegacyCounterShards || ((CounterCell) atom).hasLegacyShards();
-                }
-
-                int deletionTime = atom.getLocalDeletionTime();
-                if (deletionTime < Integer.MAX_VALUE)
-                    tombstones.update(deletionTime);
-                minTimestampTracker.update(atom.timestamp());
-                maxTimestampTracker.update(atom.timestamp());
-                minColumnNames = ColumnNameHelper.minComponents(minColumnNames, atom.name(), metadata.comparator);
-                maxColumnNames = ColumnNameHelper.maxComponents(maxColumnNames, atom.name(), metadata.comparator);
-                maxDeletionTimeTracker.update(atom.getLocalDeletionTime());
-
-                columnIndexer.add(atom); // This write the atom on disk too
-            }
-            columnIndexer.finishAddingAtoms();
-
-            columnIndexer.maybeWriteEmptyRowHeader();
-            dataFile.stream.writeShort(END_OF_ROW);
-        }
-        catch (IOException e)
-        {
-            throw new FSWriteError(e, dataFile.getPath());
-        }
-
-        sstableMetadataCollector.updateMinTimestamp(minTimestampTracker.get())
-                                .updateMaxTimestamp(maxTimestampTracker.get())
-                                .updateMaxLocalDeletionTime(maxDeletionTimeTracker.get())
-                                .addRowSize(dataFile.getFilePointer() - currentPosition)
-                                .addColumnCount(columnIndexer.writtenAtomCount())
-                                .mergeTombstoneHistogram(tombstones)
-                                .updateMinColumnNames(minColumnNames)
-                                .updateMaxColumnNames(maxColumnNames)
-                                .updateHasLegacyCounterShards(hasLegacyCounterShards);
-        afterAppend(key, currentPosition, RowIndexEntry.create(currentPosition, cf.deletionInfo().getTopLevelDeletion(), columnIndexer.build()));
-        return currentPosition;
-    }
-
-    /**
-     * After failure, attempt to close the index writer and data file before deleting all temp components for the sstable
-     */
-    public void abort()
-    {
-        assert descriptor.type.isTemporary;
-        if (iwriter == null && dataFile == null)
-            return;
-
-        if (iwriter != null)
-            iwriter.abort();
-
-        if (dataFile!= null)
-            dataFile.abort();
-
-        Set<Component> components = SSTable.componentsFor(descriptor);
-        try
-        {
-            if (!components.isEmpty())
-                SSTable.delete(descriptor, components);
-        }
-        catch (FSWriteError e)
-        {
-            logger.error(String.format("Failed deleting temp components for %s", descriptor), e);
-            throw e;
-        }
-    }
-
-    // we use this method to ensure any managed data we may have retained references to during the write are no
-    // longer referenced, so that we do not need to enclose the expensive call to closeAndOpenReader() in a transaction
-    public void isolateReferences()
-    {
-        // currently we only maintain references to first/last/lastWrittenKey from the data provided; all other
-        // data retention is done through copying
-        first = getMinimalKey(first);
-        last = lastWrittenKey = getMinimalKey(last);
-    }
-
-    private Descriptor makeTmpLinks()
-    {
-        // create temp links if they don't already exist
-        Descriptor link = descriptor.asType(Descriptor.Type.TEMPLINK);
-        if (!new File(link.filenameFor(Component.PRIMARY_INDEX)).exists())
-        {
-            FileUtils.createHardLink(new File(descriptor.filenameFor(Component.PRIMARY_INDEX)), new File(link.filenameFor(Component.PRIMARY_INDEX)));
-            FileUtils.createHardLink(new File(descriptor.filenameFor(Component.DATA)), new File(link.filenameFor(Component.DATA)));
-        }
-        return link;
-    }
-
-    public SSTableReader openEarly(long maxDataAge)
-    {
-        StatsMetadata sstableMetadata = (StatsMetadata) sstableMetadataCollector.finalizeMetadata(partitioner.getClass().getCanonicalName(),
-                                                  metadata.getBloomFilterFpChance(),
-                                                  repairedAt).get(MetadataType.STATS);
-
-        // find the max (exclusive) readable key
-        IndexSummaryBuilder.ReadableBoundary boundary = iwriter.getMaxReadable();
-        if (boundary == null)
-            return null;
-
-        assert boundary.indexLength > 0 && boundary.dataLength > 0;
-        Descriptor link = makeTmpLinks();
-        // open the reader early, giving it a FINAL descriptor type so that it is indistinguishable for other consumers
-        SegmentedFile ifile = iwriter.builder.complete(link.filenameFor(Component.PRIMARY_INDEX), boundary.indexLength);
-        SegmentedFile dfile = dbuilder.complete(link.filenameFor(Component.DATA), boundary.dataLength);
-        SSTableReader sstable = SSTableReader.internalOpen(descriptor.asType(Descriptor.Type.FINAL),
-                                                           components, metadata,
-                                                           partitioner, ifile,
-                                                           dfile, iwriter.summary.build(partitioner, boundary),
-                                                           iwriter.bf.sharedCopy(), maxDataAge, sstableMetadata, SSTableReader.OpenReason.EARLY);
-
-        // now it's open, find the ACTUAL last readable key (i.e. for which the data file has also been flushed)
-        sstable.first = getMinimalKey(first);
-        sstable.last = getMinimalKey(boundary.lastKey);
-        return sstable;
-    }
-
-    public static enum FinishType
-    {
-        CLOSE(null, true),
-        NORMAL(SSTableReader.OpenReason.NORMAL, true),
-        EARLY(SSTableReader.OpenReason.EARLY, false), // no renaming
-        FINISH_EARLY(SSTableReader.OpenReason.NORMAL, true); // tidy up an EARLY finish
-        final SSTableReader.OpenReason openReason;
-
-        public final boolean isFinal;
-        FinishType(SSTableReader.OpenReason openReason, boolean isFinal)
-        {
-            this.openReason = openReason;
-            this.isFinal = isFinal;
-        }
-    }
-
-    public SSTableReader closeAndOpenReader()
-    {
-        return closeAndOpenReader(System.currentTimeMillis());
-    }
-
-    public SSTableReader closeAndOpenReader(long maxDataAge)
-    {
-        return finish(FinishType.NORMAL, maxDataAge, this.repairedAt);
-    }
-
-    public SSTableReader finish(FinishType finishType, long maxDataAge, long repairedAt)
-    {
-        assert finishType != FinishType.CLOSE;
-        Pair<Descriptor, StatsMetadata> p;
-
-        p = close(finishType, repairedAt < 0 ? this.repairedAt : repairedAt);
-        Descriptor desc = p.left;
-        StatsMetadata metadata = p.right;
-
-        if (finishType == FinishType.EARLY)
-            desc = makeTmpLinks();
-
-        // finalize in-memory state for the reader
-        SegmentedFile ifile = iwriter.builder.complete(desc.filenameFor(Component.PRIMARY_INDEX), finishType.isFinal);
-        SegmentedFile dfile = dbuilder.complete(desc.filenameFor(Component.DATA), finishType.isFinal);
-        SSTableReader sstable = SSTableReader.internalOpen(desc.asType(Descriptor.Type.FINAL),
-                                                           components,
-                                                           this.metadata,
-                                                           partitioner,
-                                                           ifile,
-                                                           dfile,
-                                                           iwriter.summary.build(partitioner),
-                                                           iwriter.bf.sharedCopy(),
-                                                           maxDataAge,
-                                                           metadata,
-                                                           finishType.openReason);
-        sstable.first = getMinimalKey(first);
-        sstable.last = getMinimalKey(last);
-
-        if (finishType.isFinal)
-        {
-            iwriter.bf.close();
-            iwriter.summary.close();
-            // try to save the summaries to disk
-            sstable.saveSummary(iwriter.builder, dbuilder);
-            iwriter = null;
-            dbuilder = null;
-        }
-        return sstable;
-    }
-
-    // Close the writer and return the descriptor to the new sstable and it's metadata
-    public Pair<Descriptor, StatsMetadata> close()
-    {
-        return close(FinishType.CLOSE, this.repairedAt);
-    }
-
-    private Pair<Descriptor, StatsMetadata> close(FinishType type, long repairedAt)
-    {
-        switch (type)
-        {
-            case EARLY: case CLOSE: case NORMAL:
-            iwriter.close();
-            dataFile.close();
-            if (type == FinishType.CLOSE)
-                iwriter.bf.close();
-        }
-
-        // write sstable statistics
-        Map<MetadataType, MetadataComponent> metadataComponents ;
-        metadataComponents = sstableMetadataCollector
-                             .finalizeMetadata(partitioner.getClass().getCanonicalName(),
-                                               metadata.getBloomFilterFpChance(),repairedAt);
-
-        // remove the 'tmp' marker from all components
-        Descriptor descriptor = this.descriptor;
-        if (type.isFinal)
-        {
-            dataFile.writeFullChecksum(descriptor);
-            writeMetadata(descriptor, metadataComponents);
-            // save the table of components
-            SSTable.appendTOC(descriptor, components);
-            descriptor = rename(descriptor, components);
-        }
-
-        return Pair.create(descriptor, (StatsMetadata) metadataComponents.get(MetadataType.STATS));
-    }
-
-    private static void writeMetadata(Descriptor desc, Map<MetadataType, MetadataComponent> components)
-    {
-        SequentialWriter out = SequentialWriter.open(new File(desc.filenameFor(Component.STATS)));
-        try
-        {
-            desc.getMetadataSerializer().serialize(components, out.stream);
-        }
-        catch (IOException e)
-        {
-            throw new FSWriteError(e, out.getPath());
-        }
-        finally
-        {
-            out.close();
-        }
-    }
-
-    static Descriptor rename(Descriptor tmpdesc, Set<Component> components)
-    {
-        Descriptor newdesc = tmpdesc.asType(Descriptor.Type.FINAL);
-        rename(tmpdesc, newdesc, components);
-        return newdesc;
-    }
-
-    public static void rename(Descriptor tmpdesc, Descriptor newdesc, Set<Component> components)
-    {
-        for (Component component : Sets.difference(components, Sets.newHashSet(Component.DATA, Component.SUMMARY)))
-        {
-            FileUtils.renameWithConfirm(tmpdesc.filenameFor(component), newdesc.filenameFor(component));
-        }
-
-        // do -Data last because -Data present should mean the sstable was completely renamed before crash
-        FileUtils.renameWithConfirm(tmpdesc.filenameFor(Component.DATA), newdesc.filenameFor(Component.DATA));
-
-        // rename it without confirmation because summary can be available for loadNewSSTables but not for closeAndOpenReader
-        FileUtils.renameWithOutConfirm(tmpdesc.filenameFor(Component.SUMMARY), newdesc.filenameFor(Component.SUMMARY));
-    }
-
-    public long getFilePointer()
-    {
-        return dataFile.getFilePointer();
-    }
-
-    public long getOnDiskFilePointer()
-    {
-        return dataFile.getOnDiskFilePointer();
-    }
-
-    /**
-     * Encapsulates writing the index and filter for an SSTable. The state of this object is not valid until it has been closed.
-     */
-    class IndexWriter
-    {
-        private final SequentialWriter indexFile;
-        public final SegmentedFile.Builder builder;
-        public final IndexSummaryBuilder summary;
-        public final IFilter bf;
-        private FileMark mark;
-
-        IndexWriter(long keyCount, final SequentialWriter dataFile)
-        {
-            indexFile = SequentialWriter.open(new File(descriptor.filenameFor(Component.PRIMARY_INDEX)));
-            builder = SegmentedFile.getBuilder(DatabaseDescriptor.getIndexAccessMode());
-            summary = new IndexSummaryBuilder(keyCount, metadata.getMinIndexInterval(), Downsampling.BASE_SAMPLING_LEVEL);
-            bf = FilterFactory.getFilter(keyCount, metadata.getBloomFilterFpChance(), true);
-            // register listeners to be alerted when the data files are flushed
-            indexFile.setPostFlushListener(new Runnable()
-            {
-                public void run()
-                {
-                    summary.markIndexSynced(indexFile.getLastFlushOffset());
-                }
-            });
-            dataFile.setPostFlushListener(new Runnable()
-            {
-                public void run()
-                {
-                    summary.markDataSynced(dataFile.getLastFlushOffset());
-                }
-            });
-        }
-
-        // finds the last (-offset) decorated key that can be guaranteed to occur fully in the flushed portion of the index file
-        IndexSummaryBuilder.ReadableBoundary getMaxReadable()
-        {
-            return summary.getLastReadableBoundary();
-        }
-
-        public void append(DecoratedKey key, RowIndexEntry indexEntry, long dataEnd)
-        {
-            bf.add(key.getKey());
-            long indexStart = indexFile.getFilePointer();
-            try
-            {
-                ByteBufferUtil.writeWithShortLength(key.getKey(), indexFile.stream);
-                metadata.comparator.rowIndexEntrySerializer().serialize(indexEntry, indexFile.stream);
-            }
-            catch (IOException e)
-            {
-                throw new FSWriteError(e, indexFile.getPath());
-            }
-            long indexEnd = indexFile.getFilePointer();
-
-            if (logger.isTraceEnabled())
-                logger.trace("wrote index entry: " + indexEntry + " at " + indexStart);
-
-            summary.maybeAddEntry(key, indexStart, indexEnd, dataEnd);
-            builder.addPotentialBoundary(indexStart);
-        }
-
-        public void abort()
-        {
-            summary.close();
-            indexFile.abort();
-            bf.close();
-        }
-
-        /**
-         * Closes the index and bloomfilter, making the public state of this writer valid for consumption.
-         */
-        public void close()
-        {
-            if (components.contains(Component.FILTER))
-            {
-                String path = descriptor.filenameFor(Component.FILTER);
-                try
-                {
-                    // bloom filter
-                    FileOutputStream fos = new FileOutputStream(path);
-                    DataOutputStreamPlus stream = new DataOutputStreamPlus(new BufferedOutputStream(fos));
-                    FilterFactory.serialize(bf, stream);
-                    stream.flush();
-                    fos.getFD().sync();
-                    stream.close();
-                }
-                catch (IOException e)
-                {
-                    throw new FSWriteError(e, path);
-                }
-            }
-
-            // index
-            long position = indexFile.getFilePointer();
-            indexFile.close(); // calls force
-            FileUtils.truncate(indexFile.getPath(), position);
-        }
-
-        public void mark()
-        {
-            mark = indexFile.mark();
-        }
-
-        public void resetAndTruncate()
-        {
-            // we can't un-set the bloom filter addition, but extra keys in there are harmless.
-            // we can't reset dbuilder either, but that is the last thing called in afterappend so
-            // we assume that if that worked then we won't be trying to reset.
-            indexFile.resetAndTruncate(mark);
-        }
-
-        @Override
-        public String toString()
-        {
-            return "IndexWriter(" + descriptor + ")";
-        }
-    }
-}
diff --git a/src/java/org/apache/cassandra/io/sstable/SnapshotDeletingTask.java b/src/java/org/apache/cassandra/io/sstable/SnapshotDeletingTask.java
new file mode 100644
index 0000000..d23c488
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/sstable/SnapshotDeletingTask.java
@@ -0,0 +1,81 @@
+/*
+ * 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.io.sstable;
+
+import java.io.File;
+import java.util.Queue;
+import java.util.concurrent.ConcurrentLinkedQueue;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.concurrent.ScheduledExecutors;
+import org.apache.cassandra.db.WindowsFailedSnapshotTracker;
+import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.io.util.FileUtils;
+
+public class SnapshotDeletingTask implements Runnable
+{
+    private static final Logger logger = LoggerFactory.getLogger(SnapshotDeletingTask.class);
+
+    public final File path;
+    private static final Queue<Runnable> failedTasks = new ConcurrentLinkedQueue<>();
+
+    public static void addFailedSnapshot(File path)
+    {
+        logger.warn("Failed to delete snapshot [{}]. Will retry after further sstable deletions. Folder will be deleted on JVM shutdown or next node restart on crash.", path);
+        WindowsFailedSnapshotTracker.handleFailedSnapshot(path);
+        failedTasks.add(new SnapshotDeletingTask(path));
+    }
+
+    private SnapshotDeletingTask(File path)
+    {
+        this.path = path;
+    }
+
+    public void run()
+    {
+        try
+        {
+            FileUtils.deleteRecursive(path);
+            logger.info("Successfully deleted snapshot {}.", path);
+        }
+        catch (FSWriteError e)
+        {
+            failedTasks.add(this);
+        }
+    }
+
+    /**
+     * Retry all failed deletions.
+     */
+    public static void rescheduleFailedTasks()
+    {
+        Runnable task;
+        while ( null != (task = failedTasks.poll()))
+            ScheduledExecutors.nonPeriodicTasks.submit(task);
+    }
+
+    @VisibleForTesting
+    public static int pendingDeletionCount()
+    {
+        return failedTasks.size();
+    }
+}
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableFormat.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableFormat.java
new file mode 100644
index 0000000..ca003b6
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableFormat.java
@@ -0,0 +1,90 @@
+/*
+ * 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.io.sstable.format;
+
+import com.google.common.base.CharMatcher;
+import com.google.common.collect.ImmutableList;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.ColumnSerializer;
+import org.apache.cassandra.db.OnDiskAtom;
+import org.apache.cassandra.db.RowIndexEntry;
+import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
+import org.apache.cassandra.db.compaction.AbstractCompactedRow;
+import org.apache.cassandra.db.compaction.CompactionController;
+import org.apache.cassandra.io.sstable.format.big.BigFormat;
+import org.apache.cassandra.io.util.FileDataInput;
+
+import java.util.Iterator;
+
+/**
+ * Provides the accessors to data on disk.
+ */
+public interface SSTableFormat
+{
+    static boolean enableSSTableDevelopmentTestMode = Boolean.valueOf(System.getProperty("cassandra.test.sstableformatdevelopment","false"));
+
+
+    Version getLatestVersion();
+    Version getVersion(String version);
+
+    SSTableWriter.Factory getWriterFactory();
+    SSTableReader.Factory getReaderFactory();
+
+    Iterator<OnDiskAtom> getOnDiskIterator(FileDataInput in, ColumnSerializer.Flag flag, int expireBefore, CFMetaData cfm, Version version);
+
+    AbstractCompactedRow getCompactedRowWriter(CompactionController controller, ImmutableList<OnDiskAtomIterator> onDiskAtomIterators);
+
+    RowIndexEntry.IndexSerializer<?> getIndexSerializer(CFMetaData cfm);
+
+    public static enum Type
+    {
+        //Used internally to refer to files with no
+        //format flag in the filename
+        LEGACY("big", BigFormat.instance),
+
+        //The original sstable format
+        BIG("big", BigFormat.instance);
+
+        public final SSTableFormat info;
+        public final String name;
+        private Type(String name, SSTableFormat info)
+        {
+            //Since format comes right after generation
+            //we disallow formats with numeric names
+            assert !CharMatcher.DIGIT.matchesAllOf(name);
+
+            this.name = name;
+            this.info = info;
+        }
+
+        public static Type validate(String name)
+        {
+            for (Type valid : Type.values())
+            {
+                //This is used internally for old sstables
+                if (valid == LEGACY)
+                    continue;
+
+                if (valid.name.equalsIgnoreCase(name))
+                    return valid;
+            }
+
+            throw new IllegalArgumentException("No Type constant " + name);
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
new file mode 100644
index 0000000..ba060d4
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@ -0,0 +1,2328 @@
+/*
+ * 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.io.sstable.format;
+
+import java.io.*;
+import java.nio.ByteBuffer;
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Ordering;
+import com.google.common.primitives.Longs;
+import com.google.common.util.concurrent.RateLimiter;
+
+import com.clearspring.analytics.stream.cardinality.CardinalityMergeException;
+import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus;
+import com.clearspring.analytics.stream.cardinality.ICardinality;
+import com.codahale.metrics.Counter;
+import org.apache.cassandra.cache.CachingOptions;
+import org.apache.cassandra.cache.InstrumentingCache;
+import org.apache.cassandra.cache.KeyCacheKey;
+import org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor;
+import org.apache.cassandra.concurrent.ScheduledExecutors;
+import org.apache.cassandra.config.*;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
+import org.apache.cassandra.db.commitlog.ReplayPosition;
+import org.apache.cassandra.db.composites.CellName;
+import org.apache.cassandra.db.filter.ColumnSlice;
+import org.apache.cassandra.db.index.SecondaryIndex;
+import org.apache.cassandra.db.lifecycle.Tracker;
+import org.apache.cassandra.dht.*;
+import org.apache.cassandra.io.FSError;
+import org.apache.cassandra.io.FSReadError;
+import org.apache.cassandra.io.compress.CompressionMetadata;
+import org.apache.cassandra.io.sstable.*;
+import org.apache.cassandra.io.sstable.metadata.*;
+import org.apache.cassandra.io.util.*;
+import org.apache.cassandra.metrics.RestorableMeter;
+import org.apache.cassandra.metrics.StorageMetrics;
+import org.apache.cassandra.service.ActiveRepairService;
+import org.apache.cassandra.service.CacheService;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.*;
+import org.apache.cassandra.utils.concurrent.OpOrder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.cassandra.utils.concurrent.Ref;
+import org.apache.cassandra.utils.concurrent.SelfRefCounted;
+
+import static org.apache.cassandra.db.Directories.SECONDARY_INDEX_NAME_SEPARATOR;
+
+/**
+ * An SSTableReader can be constructed in a number of places, but typically is either
+ * read from disk at startup, or constructed from a flushed memtable, or after compaction
+ * to replace some existing sstables. However once created, an sstablereader may also be modified.
+ *
+ * A reader's OpenReason describes its current stage in its lifecycle, as follows:
+ *
+ *
+ * <pre> {@code
+ * NORMAL
+ * From:       None        => Reader has been read from disk, either at startup or from a flushed memtable
+ *             EARLY       => Reader is the final result of a compaction
+ *             MOVED_START => Reader WAS being compacted, but this failed and it has been restored to NORMAL status
+ *
+ * EARLY
+ * From:       None        => Reader is a compaction replacement that is either incomplete and has been opened
+ *                            to represent its partial result status, or has been finished but the compaction
+ *                            it is a part of has not yet completed fully
+ *             EARLY       => Same as from None, only it is not the first time it has been
+ *
+ * MOVED_START
+ * From:       NORMAL      => Reader is being compacted. This compaction has not finished, but the compaction result
+ *                            is either partially or fully opened, to either partially or fully replace this reader.
+ *                            This reader's start key has been updated to represent this, so that reads only hit
+ *                            one or the other reader.
+ *
+ * METADATA_CHANGE
+ * From:       NORMAL      => Reader has seen low traffic and the amount of memory available for index summaries is
+ *                            constrained, so its index summary has been downsampled.
+ *         METADATA_CHANGE => Same
+ * } </pre>
+ *
+ * Note that in parallel to this, there are two different Descriptor types; TMPLINK and FINAL; the latter corresponds
+ * to NORMAL state readers and all readers that replace a NORMAL one. TMPLINK is used for EARLY state readers and
+ * no others.
+ *
+ * When a reader is being compacted, if the result is large its replacement may be opened as EARLY before compaction
+ * completes in order to present the result to consumers earlier. In this case the reader will itself be changed to
+ * a MOVED_START state, where its start no longer represents its on-disk minimum key. This is to permit reads to be
+ * directed to only one reader when the two represent the same data. The EARLY file can represent a compaction result
+ * that is either partially complete and still in-progress, or a complete and immutable sstable that is part of a larger
+ * macro compaction action that has not yet fully completed.
+ *
+ * Currently ALL compaction results at least briefly go through an EARLY open state prior to completion, regardless
+ * of if early opening is enabled.
+ *
+ * Since a reader can be created multiple times over the same shared underlying resources, and the exact resources
+ * it shares between each instance differ subtly, we track the lifetime of any underlying resource with its own
+ * reference count, which each instance takes a Ref to. Each instance then tracks references to itself, and once these
+ * all expire it releases its Refs to these underlying resources.
+ *
+ * There is some shared cleanup behaviour needed only once all sstablereaders in a certain stage of their lifecycle
+ * (i.e. EARLY or NORMAL opening), and some that must only occur once all readers of any kind over a single logical
+ * sstable have expired. These are managed by the TypeTidy and GlobalTidy classes at the bottom, and are effectively
+ * managed as another resource each instance tracks its own Ref instance to, to ensure all of these resources are
+ * cleaned up safely and can be debugged otherwise.
+ *
+ * TODO: fill in details about Tracker and lifecycle interactions for tools, and for compaction strategies
+ */
+public abstract class SSTableReader extends SSTable implements SelfRefCounted<SSTableReader>
+{
+    private static final Logger logger = LoggerFactory.getLogger(SSTableReader.class);
+    private static final int ACCURATE_BOUNDARIES_MAGIC_NUMBER = 248923458;
+
+    private static final ScheduledThreadPoolExecutor syncExecutor = new ScheduledThreadPoolExecutor(1);
+    static
+    {
+        // Immediately remove readMeter sync task when cancelled.
+        syncExecutor.setRemoveOnCancelPolicy(true);
+    }
+    private static final RateLimiter meterSyncThrottle = RateLimiter.create(100.0);
+
+    public static final Comparator<SSTableReader> maxTimestampComparator = new Comparator<SSTableReader>()
+    {
+        public int compare(SSTableReader o1, SSTableReader o2)
+        {
+            long ts1 = o1.getMaxTimestamp();
+            long ts2 = o2.getMaxTimestamp();
+            return (ts1 > ts2 ? -1 : (ts1 == ts2 ? 0 : 1));
+        }
+    };
+
+    // it's just an object, which we use regular Object equality on; we introduce a special class just for easy recognition
+    public static final class UniqueIdentifier {}
+
+    public static final Comparator<SSTableReader> sstableComparator = new Comparator<SSTableReader>()
+    {
+        public int compare(SSTableReader o1, SSTableReader o2)
+        {
+            return o1.first.compareTo(o2.first);
+        }
+    };
+
+    public static final Ordering<SSTableReader> sstableOrdering = Ordering.from(sstableComparator);
+
+    /**
+     * maxDataAge is a timestamp in local server time (e.g. System.currentTimeMilli) which represents an upper bound
+     * to the newest piece of data stored in the sstable. In other words, this sstable does not contain items created
+     * later than maxDataAge.
+     *
+     * The field is not serialized to disk, so relying on it for more than what truncate does is not advised.
+     *
+     * When a new sstable is flushed, maxDataAge is set to the time of creation.
+     * When a sstable is created from compaction, maxDataAge is set to max of all merged sstables.
+     *
+     * The age is in milliseconds since epoc and is local to this host.
+     */
+    public final long maxDataAge;
+
+    public enum OpenReason
+    {
+        NORMAL,
+        EARLY,
+        METADATA_CHANGE,
+        MOVED_START
+    }
+
+    public final OpenReason openReason;
+    public final UniqueIdentifier instanceId = new UniqueIdentifier();
+
+    // indexfile and datafile: might be null before a call to load()
+    protected SegmentedFile ifile;
+    protected SegmentedFile dfile;
+    protected IndexSummary indexSummary;
+    protected IFilter bf;
+
+    protected final RowIndexEntry.IndexSerializer rowIndexEntrySerializer;
+
+    protected InstrumentingCache<KeyCacheKey, RowIndexEntry> keyCache;
+
+    protected final BloomFilterTracker bloomFilterTracker = new BloomFilterTracker();
+
+    // technically isCompacted is not necessary since it should never be unreferenced unless it is also compacted,
+    // but it seems like a good extra layer of protection against reference counting bugs to not delete data based on that alone
+    protected final AtomicBoolean isSuspect = new AtomicBoolean(false);
+
+    // not final since we need to be able to change level on a file.
+    protected volatile StatsMetadata sstableMetadata;
+
+    protected final AtomicLong keyCacheHit = new AtomicLong(0);
+    protected final AtomicLong keyCacheRequest = new AtomicLong(0);
+
+    private final InstanceTidier tidy = new InstanceTidier(descriptor, metadata);
+    private final Ref<SSTableReader> selfRef = new Ref<>(this, tidy);
+
+    private RestorableMeter readMeter;
+
+    /**
+     * Calculate approximate key count.
+     * If cardinality estimator is available on all given sstables, then this method use them to estimate
+     * key count.
+     * If not, then this uses index summaries.
+     *
+     * @param sstables SSTables to calculate key count
+     * @return estimated key count
+     */
+    public static long getApproximateKeyCount(Collection<SSTableReader> sstables)
+    {
+        long count = -1;
+
+        // check if cardinality estimator is available for all SSTables
+        boolean cardinalityAvailable = !sstables.isEmpty() && Iterators.all(sstables.iterator(), new Predicate<SSTableReader>()
+        {
+            public boolean apply(SSTableReader sstable)
+            {
+                return sstable.descriptor.version.hasNewStatsFile();
+            }
+        });
+
+        // if it is, load them to estimate key count
+        if (cardinalityAvailable)
+        {
+            boolean failed = false;
+            ICardinality cardinality = null;
+            for (SSTableReader sstable : sstables)
+            {
+                if (sstable.openReason == OpenReason.EARLY)
+                    continue;
+
+                try
+                {
+                    CompactionMetadata metadata = (CompactionMetadata) sstable.descriptor.getMetadataSerializer().deserialize(sstable.descriptor, MetadataType.COMPACTION);
+                    // If we can't load the CompactionMetadata, we are forced to estimate the keys using the index
+                    // summary. (CASSANDRA-10676)
+                    if (metadata == null)
+                    {
+                        logger.warn("Reading cardinality from Statistics.db failed for {}", sstable.getFilename());
+                        failed = true;
+                        break;
+                    }
+
+                    if (cardinality == null)
+                        cardinality = metadata.cardinalityEstimator;
+                    else
+                        cardinality = cardinality.merge(metadata.cardinalityEstimator);
+                }
+                catch (IOException e)
+                {
+                    logger.warn("Reading cardinality from Statistics.db failed.", e);
+                    failed = true;
+                    break;
+                }
+                catch (CardinalityMergeException e)
+                {
+                    logger.warn("Cardinality merge failed.", e);
+                    failed = true;
+                    break;
+                }
+            }
+            if (cardinality != null && !failed)
+                count = cardinality.cardinality();
+        }
+
+        // if something went wrong above or cardinality is not available, calculate using index summary
+        if (count < 0)
+        {
+            for (SSTableReader sstable : sstables)
+                count += sstable.estimatedKeys();
+        }
+        return count;
+    }
+
+    /**
+     * Estimates how much of the keys we would keep if the sstables were compacted together
+     */
+    public static double estimateCompactionGain(Set<SSTableReader> overlapping)
+    {
+        Set<ICardinality> cardinalities = new HashSet<>(overlapping.size());
+        for (SSTableReader sstable : overlapping)
+        {
+            try
+            {
+                ICardinality cardinality = ((CompactionMetadata) sstable.descriptor.getMetadataSerializer().deserialize(sstable.descriptor, MetadataType.COMPACTION)).cardinalityEstimator;
+                if (cardinality != null)
+                    cardinalities.add(cardinality);
+                else
+                    logger.trace("Got a null cardinality estimator in: {}", sstable.getFilename());
+            }
+            catch (IOException e)
+            {
+                logger.warn("Could not read up compaction metadata for {}", sstable, e);
+            }
+        }
+        long totalKeyCountBefore = 0;
+        for (ICardinality cardinality : cardinalities)
+        {
+            totalKeyCountBefore += cardinality.cardinality();
+        }
+        if (totalKeyCountBefore == 0)
+            return 1;
+
+        long totalKeyCountAfter = mergeCardinalities(cardinalities).cardinality();
+        logger.trace("Estimated compaction gain: {}/{}={}", totalKeyCountAfter, totalKeyCountBefore, ((double)totalKeyCountAfter)/totalKeyCountBefore);
+        return ((double)totalKeyCountAfter)/totalKeyCountBefore;
+    }
+
+    private static ICardinality mergeCardinalities(Collection<ICardinality> cardinalities)
+    {
+        ICardinality base = new HyperLogLogPlus(13, 25); // see MetadataCollector.cardinality
+        try
+        {
+            base = base.merge(cardinalities.toArray(new ICardinality[cardinalities.size()]));
+        }
+        catch (CardinalityMergeException e)
+        {
+            logger.warn("Could not merge cardinalities", e);
+        }
+        return base;
+    }
+
+    public static SSTableReader open(Descriptor descriptor) throws IOException
+    {
+        CFMetaData metadata;
+        if (descriptor.cfname.contains(SECONDARY_INDEX_NAME_SEPARATOR))
+        {
+            int i = descriptor.cfname.indexOf(SECONDARY_INDEX_NAME_SEPARATOR);
+            String parentName = descriptor.cfname.substring(0, i);
+            CFMetaData parent = Schema.instance.getCFMetaData(descriptor.ksname, parentName);
+            ColumnDefinition def = parent.getColumnDefinitionForIndex(descriptor.cfname.substring(i + 1));
+            metadata = CFMetaData.newIndexMetadata(parent, def, SecondaryIndex.getIndexComparator(parent, def));
+        }
+        else
+        {
+            metadata = Schema.instance.getCFMetaData(descriptor.ksname, descriptor.cfname);
+        }
+        return open(descriptor, metadata);
+    }
+
+    public static SSTableReader open(Descriptor desc, CFMetaData metadata) throws IOException
+    {
+        IPartitioner p = desc.cfname.contains(SECONDARY_INDEX_NAME_SEPARATOR)
+                ? new LocalPartitioner(metadata.getKeyValidator())
+                : StorageService.getPartitioner();
+        return open(desc, componentsFor(desc), metadata, p);
+    }
+
+    public static SSTableReader open(Descriptor descriptor, Set<Component> components, CFMetaData metadata, IPartitioner partitioner) throws IOException
+    {
+        return open(descriptor, components, metadata, partitioner, true, true);
+    }
+
+    // use only for offline or "Standalone" operations
+    public static SSTableReader openNoValidation(Descriptor descriptor, Set<Component> components, ColumnFamilyStore cfs) throws IOException
+    {
+        return open(descriptor, components, cfs.metadata, cfs.partitioner, false, false); // do not track hotness
+    }
+
+    /**
+     * Open SSTable reader to be used in batch mode(such as sstableloader).
+     *
+     * @param descriptor
+     * @param components
+     * @param metadata
+     * @param partitioner
+     * @return opened SSTableReader
+     * @throws IOException
+     */
+    public static SSTableReader openForBatch(Descriptor descriptor, Set<Component> components, CFMetaData metadata, IPartitioner partitioner) throws IOException
+    {
+        // Minimum components without which we can't do anything
+        assert components.contains(Component.DATA) : "Data component is missing for sstable " + descriptor;
+        assert components.contains(Component.PRIMARY_INDEX) : "Primary index component is missing for sstable " + descriptor;
+
+        Map<MetadataType, MetadataComponent> sstableMetadata = descriptor.getMetadataSerializer().deserialize(descriptor,
+                EnumSet.of(MetadataType.VALIDATION, MetadataType.STATS));
+        ValidationMetadata validationMetadata = (ValidationMetadata) sstableMetadata.get(MetadataType.VALIDATION);
+        StatsMetadata statsMetadata = (StatsMetadata) sstableMetadata.get(MetadataType.STATS);
+
+        // Check if sstable is created using same partitioner.
+        // Partitioner can be null, which indicates older version of sstable or no stats available.
+        // In that case, we skip the check.
+        String partitionerName = partitioner.getClass().getCanonicalName();
+        if (validationMetadata != null && !partitionerName.equals(validationMetadata.partitioner))
+        {
+            logger.error(String.format("Cannot open %s; partitioner %s does not match system partitioner %s.  Note that the default partitioner starting with Cassandra 1.2 is Murmur3Partitioner, so you will need to edit that to match your old partitioner if upgrading.",
+                    descriptor, validationMetadata.partitioner, partitionerName));
+            System.exit(1);
+        }
+
+        logger.debug("Opening {} ({} bytes)", descriptor, new File(descriptor.filenameFor(Component.DATA)).length());
+        SSTableReader sstable = internalOpen(descriptor, components, metadata, partitioner, System.currentTimeMillis(),
+                statsMetadata, OpenReason.NORMAL);
+
+        // special implementation of load to use non-pooled SegmentedFile builders
+        try(SegmentedFile.Builder ibuilder = new BufferedSegmentedFile.Builder();
+            SegmentedFile.Builder dbuilder = sstable.compression
+                ? new CompressedSegmentedFile.Builder(null)
+                : new BufferedSegmentedFile.Builder())
+        {
+            if (!sstable.loadSummary(ibuilder, dbuilder))
+                sstable.buildSummary(false, ibuilder, dbuilder, false, Downsampling.BASE_SAMPLING_LEVEL);
+            sstable.ifile = ibuilder.complete(sstable.descriptor.filenameFor(Component.PRIMARY_INDEX));
+            sstable.dfile = dbuilder.complete(sstable.descriptor.filenameFor(Component.DATA));
+            sstable.bf = FilterFactory.AlwaysPresent;
+            sstable.setup(false);
+            return sstable;
+        }
+    }
+
+    public static SSTableReader open(Descriptor descriptor,
+                                      Set<Component> components,
+                                      CFMetaData metadata,
+                                      IPartitioner partitioner,
+                                      boolean validate,
+                                      boolean trackHotness) throws IOException
+    {
+        // Minimum components without which we can't do anything
+        assert components.contains(Component.DATA) : "Data component is missing for sstable " + descriptor;
+        assert !validate || components.contains(Component.PRIMARY_INDEX) : "Primary index component is missing for sstable " + descriptor;
+
+        Map<MetadataType, MetadataComponent> sstableMetadata = descriptor.getMetadataSerializer().deserialize(descriptor,
+                                                                                                               EnumSet.of(MetadataType.VALIDATION, MetadataType.STATS));
+        ValidationMetadata validationMetadata = (ValidationMetadata) sstableMetadata.get(MetadataType.VALIDATION);
+        StatsMetadata statsMetadata = (StatsMetadata) sstableMetadata.get(MetadataType.STATS);
+
+        // Check if sstable is created using same partitioner.
+        // Partitioner can be null, which indicates older version of sstable or no stats available.
+        // In that case, we skip the check.
+        String partitionerName = partitioner.getClass().getCanonicalName();
+        if (validationMetadata != null && !partitionerName.equals(validationMetadata.partitioner))
+        {
+            logger.error(String.format("Cannot open %s; partitioner %s does not match system partitioner %s.  Note that the default partitioner starting with Cassandra 1.2 is Murmur3Partitioner, so you will need to edit that to match your old partitioner if upgrading.",
+                    descriptor, validationMetadata.partitioner, partitionerName));
+            System.exit(1);
+        }
+
+        logger.debug("Opening {} ({} bytes)", descriptor, new File(descriptor.filenameFor(Component.DATA)).length());
+        SSTableReader sstable = internalOpen(descriptor, components, metadata, partitioner, System.currentTimeMillis(),
+                                             statsMetadata, OpenReason.NORMAL);
+        try
+        {
+            // load index and filter
+            long start = System.nanoTime();
+            sstable.load(validationMetadata);
+            logger.trace("INDEX LOAD TIME for {}: {} ms.", descriptor, TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start));
+
+            sstable.setup(trackHotness);
+            if (validate)
+                sstable.validate();
+
+            if (sstable.getKeyCache() != null)
+                logger.trace("key cache contains {}/{} keys", sstable.getKeyCache().size(), sstable.getKeyCache().getCapacity());
+
+            return sstable;
+        }
+        catch (Throwable t)
+        {
+            sstable.selfRef().release();
+            throw t;
+        }
+    }
+
+    public static void logOpenException(Descriptor descriptor, IOException e)
+    {
+        if (e instanceof FileNotFoundException)
+            logger.error("Missing sstable component in {}; skipped because of {}", descriptor, e.getMessage());
+        else
+            logger.error("Corrupt sstable {}; skipped", descriptor, e);
+    }
+
+    public static Collection<SSTableReader> openAll(Set<Map.Entry<Descriptor, Set<Component>>> entries,
+                                                    final CFMetaData metadata,
+                                                    final IPartitioner partitioner)
+    {
+        final Collection<SSTableReader> sstables = new LinkedBlockingQueue<>();
+
+        ExecutorService executor = DebuggableThreadPoolExecutor.createWithFixedPoolSize("SSTableBatchOpen", FBUtilities.getAvailableProcessors());
+        for (final Map.Entry<Descriptor, Set<Component>> entry : entries)
+        {
+            Runnable runnable = new Runnable()
+            {
+                public void run()
+                {
+                    SSTableReader sstable;
+                    try
+                    {
+                        sstable = open(entry.getKey(), entry.getValue(), metadata, partitioner);
+                    }
+                    catch (CorruptSSTableException ex)
+                    {
+                        FileUtils.handleCorruptSSTable(ex);
+                        logger.error("Corrupt sstable {}; skipping table", entry, ex);
+                        return;
+                    }
+                    catch (FSError ex)
+                    {
+                        FileUtils.handleFSError(ex);
+                        logger.error("Cannot read sstable {}; file system error, skipping table", entry, ex);
+                        return;
+                    }
+                    catch (IOException ex)
+                    {
+                        logger.error("Cannot read sstable {}; other IO error, skipping table", entry, ex);
+                        return;
+                    }
+                    sstables.add(sstable);
+                }
+            };
+            executor.submit(runnable);
+        }
+
+        executor.shutdown();
+        try
+        {
+            executor.awaitTermination(7, TimeUnit.DAYS);
+        }
+        catch (InterruptedException e)
+        {
+            throw new AssertionError(e);
+        }
+
+        return sstables;
+
+    }
+
+    /**
+     * Open a RowIndexedReader which already has its state initialized (by SSTableWriter).
+     */
+    public static SSTableReader internalOpen(Descriptor desc,
+                                      Set<Component> components,
+                                      CFMetaData metadata,
+                                      IPartitioner partitioner,
+                                      SegmentedFile ifile,
+                                      SegmentedFile dfile,
+                                      IndexSummary isummary,
+                                      IFilter bf,
+                                      long maxDataAge,
+                                      StatsMetadata sstableMetadata,
+                                      OpenReason openReason)
+    {
+        assert desc != null && partitioner != null && ifile != null && dfile != null && isummary != null && bf != null && sstableMetadata != null;
+
+        SSTableReader reader = internalOpen(desc, components, metadata, partitioner, maxDataAge, sstableMetadata, openReason);
+
+        reader.bf = bf;
+        reader.ifile = ifile;
+        reader.dfile = dfile;
+        reader.indexSummary = isummary;
+        reader.setup(true);
+
+        return reader;
+    }
+
+
+    private static SSTableReader internalOpen(final Descriptor descriptor,
+                                            Set<Component> components,
+                                            CFMetaData metadata,
+                                            IPartitioner partitioner,
+                                            Long maxDataAge,
+                                            StatsMetadata sstableMetadata,
+                                            OpenReason openReason)
+    {
+        Factory readerFactory = descriptor.getFormat().getReaderFactory();
+
+        return readerFactory.open(descriptor, components, metadata, partitioner, maxDataAge, sstableMetadata, openReason);
+    }
+
+    protected SSTableReader(final Descriptor desc,
+                            Set<Component> components,
+                            CFMetaData metadata,
+                            IPartitioner partitioner,
+                            long maxDataAge,
+                            StatsMetadata sstableMetadata,
+                            OpenReason openReason)
+    {
+        super(desc, components, metadata, partitioner);
+        this.sstableMetadata = sstableMetadata;
+        this.maxDataAge = maxDataAge;
+        this.openReason = openReason;
+        this.rowIndexEntrySerializer = descriptor.version.getSSTableFormat().getIndexSerializer(metadata);
+    }
+
+    public static long getTotalBytes(Iterable<SSTableReader> sstables)
+    {
+        long sum = 0;
+        for (SSTableReader sstable : sstables)
+            sum += sstable.onDiskLength();
+        return sum;
+    }
+
+    public static long getTotalUncompressedBytes(Iterable<SSTableReader> sstables)
+    {
+        long sum = 0;
+        for (SSTableReader sstable : sstables)
+            sum += sstable.uncompressedLength();
+
+        return sum;
+    }
+
+    public boolean equals(Object that)
+    {
+        return that instanceof SSTableReader && ((SSTableReader) that).descriptor.equals(this.descriptor);
+    }
+
+    public int hashCode()
+    {
+        return this.descriptor.hashCode();
+    }
+
+    public String getFilename()
+    {
+        return dfile.path();
+    }
+
+    public void setupKeyCache()
+    {
+        // under normal operation we can do this at any time, but SSTR is also used outside C* proper,
+        // e.g. by BulkLoader, which does not initialize the cache.  As a kludge, we set up the cache
+        // here when we know we're being wired into the rest of the server infrastructure.
+        keyCache = CacheService.instance.keyCache;
+    }
+
+    private void load(ValidationMetadata validation) throws IOException
+    {
+        if (metadata.getBloomFilterFpChance() == 1.0)
+        {
+            // bf is disabled.
+            load(false, true);
+            bf = FilterFactory.AlwaysPresent;
+        }
+        else if (!components.contains(Component.PRIMARY_INDEX))
+        {
+            // avoid any reading of the missing primary index component.
+            // this should only happen during StandaloneScrubber
+            load(false, false);
+        }
+        else if (!components.contains(Component.FILTER) || validation == null)
+        {
+            // bf is enabled, but filter component is missing.
+            load(true, true);
+        }
+        else if (validation.bloomFilterFPChance != metadata.getBloomFilterFpChance())
+        {
+            // bf fp chance in sstable metadata and it has changed since compaction.
+            load(true, true);
+        }
+        else
+        {
+            // bf is enabled and fp chance matches the currently configured value.
+            load(false, true);
+            loadBloomFilter();
+        }
+    }
+
+    /**
+     * Load bloom filter from Filter.db file.
+     *
+     * @throws IOException
+     */
+    private void loadBloomFilter() throws IOException
+    {
+        try (DataInputStream stream = new DataInputStream(new BufferedInputStream(new FileInputStream(descriptor.filenameFor(Component.FILTER)))))
+        {
+            bf = FilterFactory.deserialize(stream, true);
+        }
+    }
+
+    /**
+     * Loads ifile, dfile and indexSummary, and optionally recreates the bloom filter.
+     * @param saveSummaryIfCreated for bulk loading purposes, if the summary was absent and needed to be built, you can
+     *                             avoid persisting it to disk by setting this to false
+     */
+    private void load(boolean recreateBloomFilter, boolean saveSummaryIfCreated) throws IOException
+    {
+        try(SegmentedFile.Builder ibuilder = SegmentedFile.getBuilder(DatabaseDescriptor.getIndexAccessMode(), false);
+            SegmentedFile.Builder dbuilder = SegmentedFile.getBuilder(DatabaseDescriptor.getDiskAccessMode(), compression))
+        {
+            boolean summaryLoaded = loadSummary(ibuilder, dbuilder);
+            boolean builtSummary = false;
+            if (recreateBloomFilter || !summaryLoaded)
+            {
+                buildSummary(recreateBloomFilter, ibuilder, dbuilder, summaryLoaded, Downsampling.BASE_SAMPLING_LEVEL);
+                builtSummary = true;
+            }
+
+            if (components.contains(Component.PRIMARY_INDEX))
+                ifile = ibuilder.complete(descriptor.filenameFor(Component.PRIMARY_INDEX));
+
+            dfile = dbuilder.complete(descriptor.filenameFor(Component.DATA));
+
+            // Check for an index summary that was downsampled even though the serialization format doesn't support
+            // that.  If it was downsampled, rebuild it.  See CASSANDRA-8993 for details.
+        if (!descriptor.version.hasSamplingLevel() && !builtSummary && !validateSummarySamplingLevel() && ifile != null)
+            {
+                indexSummary.close();
+                ifile.close();
+                dfile.close();
+
+                logger.info("Detected erroneously downsampled index summary; will rebuild summary at full sampling");
+                FileUtils.deleteWithConfirm(new File(descriptor.filenameFor(Component.SUMMARY)));
+
+                try(SegmentedFile.Builder ibuilderRebuild = SegmentedFile.getBuilder(DatabaseDescriptor.getIndexAccessMode(), false);
+                    SegmentedFile.Builder dbuilderRebuild = SegmentedFile.getBuilder(DatabaseDescriptor.getDiskAccessMode(), compression))
+                {
+                    buildSummary(false, ibuilderRebuild, dbuilderRebuild, false, Downsampling.BASE_SAMPLING_LEVEL);
+                    ifile = ibuilderRebuild.complete(descriptor.filenameFor(Component.PRIMARY_INDEX));
+                    dfile = dbuilderRebuild.complete(descriptor.filenameFor(Component.DATA));
+                    saveSummary(ibuilderRebuild, dbuilderRebuild);
+                }
+            }
+            else if (saveSummaryIfCreated && builtSummary)
+            {
+                saveSummary(ibuilder, dbuilder);
+            }
+        }
+        catch (Throwable t)
+        { // Because the tidier has not been set-up yet in SSTableReader.open(), we must release the files in case of error
+            if (ifile != null)
+            {
+                ifile.close();
+                ifile = null;
+            }
+
+            if (dfile != null)
+            {
+                dfile.close();
+                dfile = null;
+            }
+
+            if (indexSummary != null)
+            {
+                indexSummary.close();
+                indexSummary = null;
+            }
+
+            throw t;
+        }
+    }
+
+    /**
+     * Build index summary(and optionally bloom filter) by reading through Index.db file.
+     *
+     * @param recreateBloomFilter true if recreate bloom filter
+     * @param ibuilder
+     * @param dbuilder
+     * @param summaryLoaded true if index summary is already loaded and not need to build again
+     * @throws IOException
+     */
+    private void buildSummary(boolean recreateBloomFilter, SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder, boolean summaryLoaded, int samplingLevel) throws IOException
+    {
+         if (!components.contains(Component.PRIMARY_INDEX))
+             return;
+
+        // we read the positions in a BRAF so we don't have to worry about an entry spanning a mmap boundary.
+        try (RandomAccessReader primaryIndex = RandomAccessReader.open(new File(descriptor.filenameFor(Component.PRIMARY_INDEX))))
+        {
+            long indexSize = primaryIndex.length();
+            long histogramCount = sstableMetadata.estimatedRowSize.count();
+            long estimatedKeys = histogramCount > 0 && !sstableMetadata.estimatedRowSize.isOverflowed()
+                    ? histogramCount
+                    : estimateRowsFromIndex(primaryIndex); // statistics is supposed to be optional
+
+            if (recreateBloomFilter)
+                bf = FilterFactory.getFilter(estimatedKeys, metadata.getBloomFilterFpChance(), true);
+
+            try (IndexSummaryBuilder summaryBuilder = summaryLoaded ? null : new IndexSummaryBuilder(estimatedKeys, metadata.getMinIndexInterval(), samplingLevel))
+            {
+                long indexPosition;
+                RowIndexEntry.IndexSerializer rowIndexSerializer = descriptor.getFormat().getIndexSerializer(metadata);
+
+                while ((indexPosition = primaryIndex.getFilePointer()) != indexSize)
+                {
+                    ByteBuffer key = ByteBufferUtil.readWithShortLength(primaryIndex);
+                    RowIndexEntry indexEntry = rowIndexSerializer.deserialize(primaryIndex, descriptor.version);
+                    DecoratedKey decoratedKey = partitioner.decorateKey(key);
+                    if (first == null)
+                        first = decoratedKey;
+                    last = decoratedKey;
+
+                    if (recreateBloomFilter)
+                        bf.add(decoratedKey);
+
+                    // if summary was already read from disk we don't want to re-populate it using primary index
+                    if (!summaryLoaded)
+                    {
+                        summaryBuilder.maybeAddEntry(decoratedKey, indexPosition);
+                        ibuilder.addPotentialBoundary(indexPosition);
+                        dbuilder.addPotentialBoundary(indexEntry.position);
+                    }
+                }
+
+                if (!summaryLoaded)
+                    indexSummary = summaryBuilder.build(partitioner);
+            }
+        }
+
+        first = getMinimalKey(first);
+        last = getMinimalKey(last);
+    }
+
+    /**
+     * Load index summary from Summary.db file if it exists.
+     *
+     * if loaded index summary has different index interval from current value stored in schema,
+     * then Summary.db file will be deleted and this returns false to rebuild summary.
+     *
+     * @param ibuilder
+     * @param dbuilder
+     * @return true if index summary is loaded successfully from Summary.db file.
+     */
+    @SuppressWarnings("resource")
+    public boolean loadSummary(SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder)
+    {
+        File summariesFile = new File(descriptor.filenameFor(Component.SUMMARY));
+        if (!summariesFile.exists())
+            return false;
+
+        DataInputStream iStream = null;
+        try
+        {
+            iStream = new DataInputStream(new FileInputStream(summariesFile));
+            indexSummary = IndexSummary.serializer.deserialize(
+                    iStream, partitioner, descriptor.version.hasSamplingLevel(),
+                    metadata.getMinIndexInterval(), metadata.getMaxIndexInterval());
+            first = partitioner.decorateKey(ByteBufferUtil.readWithLength(iStream));
+            last = partitioner.decorateKey(ByteBufferUtil.readWithLength(iStream));
+            ibuilder.deserializeBounds(iStream);
+            dbuilder.deserializeBounds(iStream);
+
+            boolean checkForRepair = true;
+            try
+            {
+                int v = iStream.readInt();
+                // check for our magic number, indicating this summary has been sampled correctly
+                checkForRepair = v != ACCURATE_BOUNDARIES_MAGIC_NUMBER;
+            }
+            catch (Throwable t) {}
+
+            // fix CASSANDRA-10357 on-the-fly
+            if (checkForRepair && MmappedSegmentedFile.maybeRepair(metadata, descriptor, indexSummary, ibuilder, dbuilder))
+                saveSummary(ibuilder, dbuilder);
+        }
+        catch (IOException e)
+        {
+            if (indexSummary != null)
+                indexSummary.close();
+            logger.trace("Cannot deserialize SSTable Summary File {}: {}", summariesFile.getPath(), e.getMessage());
+            // corrupted; delete it and fall back to creating a new summary
+            FileUtils.closeQuietly(iStream);
+            // delete it and fall back to creating a new summary
+            FileUtils.deleteWithConfirm(summariesFile);
+            return false;
+        }
+        finally
+        {
+            FileUtils.closeQuietly(iStream);
+        }
+
+        return true;
+    }
+
+    /**
+     * Validates that an index summary has full sampling, as expected when the serialization format does not support
+     * persisting the sampling level.
+     * @return true if the summary has full sampling, false otherwise
+     */
+    private boolean validateSummarySamplingLevel()
+    {
+        // We need to check index summary entries against the index to verify that none of them were dropped due to
+        // downsampling.  Downsampling can drop any of the first BASE_SAMPLING_LEVEL entries (repeating that drop pattern
+        // for the remainder of the summary).  Unfortunately, the first entry to be dropped is the entry at
+        // index (BASE_SAMPLING_LEVEL - 1), so we need to check a full set of BASE_SAMPLING_LEVEL entries.
+        if (ifile == null)
+            return false;
+
+        Iterator<FileDataInput> segments = ifile.iterator(0);
+        int i = 0;
+        int summaryEntriesChecked = 0;
+        int expectedIndexInterval = getMinIndexInterval();
+        while (segments.hasNext())
+        {
+            String path = null;
+            try (FileDataInput in = segments.next())
+            {
+                path = in.getPath();
+                while (!in.isEOF())
+                {
+                    ByteBuffer indexKey = ByteBufferUtil.readWithShortLength(in);
+                    if (i % expectedIndexInterval == 0)
+                    {
+                        ByteBuffer summaryKey = ByteBuffer.wrap(indexSummary.getKey(i / expectedIndexInterval));
+                        if (!summaryKey.equals(indexKey))
+                            return false;
+                        summaryEntriesChecked++;
+
+                        if (summaryEntriesChecked == Downsampling.BASE_SAMPLING_LEVEL)
+                            return true;
+                    }
+                    RowIndexEntry.Serializer.skip(in);
+                    i++;
+                }
+            }
+            catch (IOException e)
+            {
+                markSuspect();
+                throw new CorruptSSTableException(e, path);
+            }
+        }
+
+        return true;
+    }
+
+    /**
+     * Save index summary to Summary.db file.
+     *
+     * @param ibuilder
+     * @param dbuilder
+     */
+
+    public void saveSummary(SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder)
+    {
+        saveSummary(this.descriptor, this.first, this.last, ibuilder, dbuilder, indexSummary);
+    }
+
+    private void saveSummary(SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder, IndexSummary newSummary)
+    {
+        saveSummary(this.descriptor, this.first, this.last, ibuilder, dbuilder, newSummary);
+    }
+    /**
+     * Save index summary to Summary.db file.
+     */
+    public static void saveSummary(Descriptor descriptor, DecoratedKey first, DecoratedKey last,
+                                   SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder, IndexSummary summary)
+    {
+        File summariesFile = new File(descriptor.filenameFor(Component.SUMMARY));
+        if (summariesFile.exists())
+            FileUtils.deleteWithConfirm(summariesFile);
+
+        try (DataOutputStreamPlus oStream = new BufferedDataOutputStreamPlus(new FileOutputStream(summariesFile));)
+        {
+            IndexSummary.serializer.serialize(summary, oStream, descriptor.version.hasSamplingLevel());
+            ByteBufferUtil.writeWithLength(first.getKey(), oStream);
+            ByteBufferUtil.writeWithLength(last.getKey(), oStream);
+            ibuilder.serializeBounds(oStream);
+            dbuilder.serializeBounds(oStream);
+            // write a magic number, to indicate this summary has been sampled correctly
+            oStream.writeInt(ACCURATE_BOUNDARIES_MAGIC_NUMBER);
+        }
+        catch (IOException e)
+        {
+            logger.trace("Cannot save SSTable Summary: ", e);
+
+            // corrupted hence delete it and let it load it now.
+            if (summariesFile.exists())
+                FileUtils.deleteWithConfirm(summariesFile);
+        }
+    }
+
+    public void setReplaced()
+    {
+        synchronized (tidy.global)
+        {
+            assert !tidy.isReplaced;
+            tidy.isReplaced = true;
+        }
+    }
+
+    public boolean isReplaced()
+    {
+        synchronized (tidy.global)
+        {
+            return tidy.isReplaced;
+        }
+    }
+
+    // runOnClose must NOT be an anonymous or non-static inner class, nor must it retain a reference chain to this reader
+    public void runOnClose(final Runnable runOnClose)
+    {
+        synchronized (tidy.global)
+        {
+            final Runnable existing = tidy.runOnClose;
+            tidy.runOnClose = AndThen.get(existing, runOnClose);
+        }
+    }
+
+    private static class AndThen implements Runnable
+    {
+        final Runnable runFirst;
+        final Runnable runSecond;
+
+        private AndThen(Runnable runFirst, Runnable runSecond)
+        {
+            this.runFirst = runFirst;
+            this.runSecond = runSecond;
+        }
+
+        public void run()
+        {
+            runFirst.run();
+            runSecond.run();
+        }
+
+        static Runnable get(Runnable runFirst, Runnable runSecond)
+        {
+            if (runFirst == null)
+                return runSecond;
+            return new AndThen(runFirst, runSecond);
+        }
+    }
+
+    /**
+     * Clone this reader with the provided start and open reason, and set the clone as replacement.
+     *
+     * @param newFirst the first key for the replacement (which can be different from the original due to the pre-emptive
+     * opening of compaction results).
+     * @param reason the {@code OpenReason} for the replacement.
+     *
+     * @return the cloned reader. That reader is set as a replacement by the method.
+     */
+    private SSTableReader cloneAndReplace(DecoratedKey newFirst, OpenReason reason)
+    {
+        return cloneAndReplace(newFirst, reason, indexSummary.sharedCopy());
+    }
+
+    /**
+     * Clone this reader with the new values and set the clone as replacement.
+     *
+     * @param newFirst the first key for the replacement (which can be different from the original due to the pre-emptive
+     * opening of compaction results).
+     * @param reason the {@code OpenReason} for the replacement.
+     * @param newSummary the index summary for the replacement.
+     *
+     * @return the cloned reader. That reader is set as a replacement by the method.
+     */
+    private SSTableReader cloneAndReplace(DecoratedKey newFirst, OpenReason reason, IndexSummary newSummary)
+    {
+        SSTableReader replacement = internalOpen(descriptor,
+                                                 components,
+                                                 metadata,
+                                                 partitioner,
+                                                 ifile != null ? ifile.sharedCopy() : null,
+                                                 dfile.sharedCopy(),
+                                                 newSummary,
+                                                 bf.sharedCopy(),
+                                                 maxDataAge,
+                                                 sstableMetadata,
+                                                 reason);
+        replacement.first = newFirst;
+        replacement.last = last;
+        replacement.isSuspect.set(isSuspect.get());
+        return replacement;
+    }
+
+    // runOnClose must NOT be an anonymous or non-static inner class, nor must it retain a reference chain to this reader
+    public SSTableReader cloneWithNewStart(DecoratedKey newStart, final Runnable runOnClose)
+    {
+        synchronized (tidy.global)
+        {
+            assert openReason != OpenReason.EARLY;
+            // TODO: merge with caller's firstKeyBeyond() work,to save time
+            if (newStart.compareTo(first) > 0)
+            {
+                final long dataStart = getPosition(newStart, Operator.EQ).position;
+                final long indexStart = getIndexScanPosition(newStart);
+                this.tidy.runOnClose = new DropPageCache(dfile, dataStart, ifile, indexStart, runOnClose);
+            }
+
+            return cloneAndReplace(newStart, OpenReason.MOVED_START);
+        }
+    }
+
+    private static class DropPageCache implements Runnable
+    {
+        final SegmentedFile dfile;
+        final long dfilePosition;
+        final SegmentedFile ifile;
+        final long ifilePosition;
+        final Runnable andThen;
+
+        private DropPageCache(SegmentedFile dfile, long dfilePosition, SegmentedFile ifile, long ifilePosition, Runnable andThen)
+        {
+            this.dfile = dfile;
+            this.dfilePosition = dfilePosition;
+            this.ifile = ifile;
+            this.ifilePosition = ifilePosition;
+            this.andThen = andThen;
+        }
+
+        public void run()
+        {
+            dfile.dropPageCache(dfilePosition);
+
+            if (ifile != null)
+                ifile.dropPageCache(ifilePosition);
+            andThen.run();
+        }
+    }
+
+    /**
+     * Returns a new SSTableReader with the same properties as this SSTableReader except that a new IndexSummary will
+     * be built at the target samplingLevel.  This (original) SSTableReader instance will be marked as replaced, have
+     * its DeletingTask removed, and have its periodic read-meter sync task cancelled.
+     * @param samplingLevel the desired sampling level for the index summary on the new SSTableReader
+     * @return a new SSTableReader
+     * @throws IOException
+     */
+    @SuppressWarnings("resource")
+    public SSTableReader cloneWithNewSummarySamplingLevel(ColumnFamilyStore parent, int samplingLevel) throws IOException
+    {
+        assert descriptor.version.hasSamplingLevel();
+
+        synchronized (tidy.global)
+        {
+            assert openReason != OpenReason.EARLY;
+
+            int minIndexInterval = metadata.getMinIndexInterval();
+            int maxIndexInterval = metadata.getMaxIndexInterval();
+            double effectiveInterval = indexSummary.getEffectiveIndexInterval();
+
+            IndexSummary newSummary;
+            long oldSize = bytesOnDisk();
+
+            // We have to rebuild the summary from the on-disk primary index in three cases:
+            // 1. The sampling level went up, so we need to read more entries off disk
+            // 2. The min_index_interval changed (in either direction); this changes what entries would be in the summary
+            //    at full sampling (and consequently at any other sampling level)
+            // 3. The max_index_interval was lowered, forcing us to raise the sampling level
+            if (samplingLevel > indexSummary.getSamplingLevel() || indexSummary.getMinIndexInterval() != minIndexInterval || effectiveInterval > maxIndexInterval)
+            {
+                newSummary = buildSummaryAtLevel(samplingLevel);
+            }
+            else if (samplingLevel < indexSummary.getSamplingLevel())
+            {
+                // we can use the existing index summary to make a smaller one
+                newSummary = IndexSummaryBuilder.downsample(indexSummary, samplingLevel, minIndexInterval, partitioner);
+            }
+            else
+            {
+                throw new AssertionError("Attempted to clone SSTableReader with the same index summary sampling level and " +
+                        "no adjustments to min/max_index_interval");
+            }
+
+            //Always save the resampled index
+            try(SegmentedFile.Builder ibuilder = SegmentedFile.getBuilder(DatabaseDescriptor.getIndexAccessMode(), false);
+                SegmentedFile.Builder dbuilder = SegmentedFile.getBuilder(DatabaseDescriptor.getDiskAccessMode(), compression))
+            {
+                for (long boundry : dfile.copyReadableBounds())
+                    dbuilder.addPotentialBoundary(boundry);
+                for (long boundry : ifile.copyReadableBounds())
+                    ibuilder.addPotentialBoundary(boundry);
+
+                saveSummary(ibuilder, dbuilder, newSummary);
+            }
+
+            long newSize = bytesOnDisk();
+            StorageMetrics.load.inc(newSize - oldSize);
+            parent.metric.liveDiskSpaceUsed.inc(newSize - oldSize);
+
+            return cloneAndReplace(first, OpenReason.METADATA_CHANGE, newSummary);
+        }
+    }
+
+    private IndexSummary buildSummaryAtLevel(int newSamplingLevel) throws IOException
+    {
+        // we read the positions in a BRAF so we don't have to worry about an entry spanning a mmap boundary.
+        RandomAccessReader primaryIndex = RandomAccessReader.open(new File(descriptor.filenameFor(Component.PRIMARY_INDEX)));
+        try
+        {
+            long indexSize = primaryIndex.length();
+            try (IndexSummaryBuilder summaryBuilder = new IndexSummaryBuilder(estimatedKeys(), metadata.getMinIndexInterval(), newSamplingLevel))
+            {
+                long indexPosition;
+                while ((indexPosition = primaryIndex.getFilePointer()) != indexSize)
+                {
+                    summaryBuilder.maybeAddEntry(partitioner.decorateKey(ByteBufferUtil.readWithShortLength(primaryIndex)), indexPosition);
+                    RowIndexEntry.Serializer.skip(primaryIndex);
+                }
+
+                return summaryBuilder.build(partitioner);
+            }
+        }
+        finally
+        {
+            FileUtils.closeQuietly(primaryIndex);
+        }
+    }
+
+    public RestorableMeter getReadMeter()
+    {
+        return readMeter;
+    }
+
+    public int getIndexSummarySamplingLevel()
+    {
+        return indexSummary.getSamplingLevel();
+    }
+
+    public long getIndexSummaryOffHeapSize()
+    {
+        return indexSummary.getOffHeapSize();
+    }
+
+    public int getMinIndexInterval()
+    {
+        return indexSummary.getMinIndexInterval();
+    }
+
+    public double getEffectiveIndexInterval()
+    {
+        return indexSummary.getEffectiveIndexInterval();
+    }
+
+    public void releaseSummary()
+    {
+        tidy.releaseSummary();
+        indexSummary = null;
+    }
+
+    private void validate()
+    {
+        if (this.first.compareTo(this.last) > 0)
+        {
+            selfRef().release();
+            throw new IllegalStateException(String.format("SSTable first key %s > last key %s", this.first, this.last));
+        }
+    }
+
+    /**
+     * Gets the position in the index file to start scanning to find the given key (at most indexInterval keys away,
+     * modulo downsampling of the index summary). Always returns a value >= 0
+     */
+    public long getIndexScanPosition(RowPosition key)
+    {
+        if (openReason == OpenReason.MOVED_START && key.compareTo(first) < 0)
+            key = first;
+
+        return getIndexScanPositionFromBinarySearchResult(indexSummary.binarySearch(key), indexSummary);
+    }
+
+    @VisibleForTesting
+    public static long getIndexScanPositionFromBinarySearchResult(int binarySearchResult, IndexSummary referencedIndexSummary)
+    {
+        if (binarySearchResult == -1)
+            return 0;
+        else
+            return referencedIndexSummary.getPosition(getIndexSummaryIndexFromBinarySearchResult(binarySearchResult));
+    }
+
+    public static int getIndexSummaryIndexFromBinarySearchResult(int binarySearchResult)
+    {
+        if (binarySearchResult < 0)
+        {
+            // binary search gives us the first index _greater_ than the key searched for,
+            // i.e., its insertion position
+            int greaterThan = (binarySearchResult + 1) * -1;
+            if (greaterThan == 0)
+                return -1;
+            return greaterThan - 1;
+        }
+        else
+        {
+            return binarySearchResult;
+        }
+    }
+
+    /**
+     * Returns the compression metadata for this sstable.
+     * @throws IllegalStateException if the sstable is not compressed
+     */
+    public CompressionMetadata getCompressionMetadata()
+    {
+        if (!compression)
+            throw new IllegalStateException(this + " is not compressed");
+
+        CompressionMetadata cmd = ((ICompressedFile) dfile).getMetadata();
+
+        //We need the parent cf metadata
+        String cfName = metadata.isSecondaryIndex() ? metadata.getParentColumnFamilyName() : metadata.cfName;
+        cmd.parameters.setLiveMetadata(Schema.instance.getCFMetaData(metadata.ksName, cfName));
+
+        return cmd;
+    }
+
+    /**
+     * Returns the amount of memory in bytes used off heap by the compression meta-data.
+     * @return the amount of memory in bytes used off heap by the compression meta-data
+     */
+    public long getCompressionMetadataOffHeapSize()
+    {
+        if (!compression)
+            return 0;
+
+        return getCompressionMetadata().offHeapSize();
+    }
+
+    /**
+     * For testing purposes only.
+     */
+    public void forceFilterFailures()
+    {
+        bf = FilterFactory.AlwaysPresent;
+    }
+
+    public IFilter getBloomFilter()
+    {
+        return bf;
+    }
+
+    public long getBloomFilterSerializedSize()
+    {
+        return bf.serializedSize();
+    }
+
+    /**
+     * Returns the amount of memory in bytes used off heap by the bloom filter.
+     * @return the amount of memory in bytes used off heap by the bloom filter
+     */
+    public long getBloomFilterOffHeapSize()
+    {
+        return bf.offHeapSize();
+    }
+
+    /**
+     * @return An estimate of the number of keys in this SSTable based on the index summary.
+     */
+    public long estimatedKeys()
+    {
+        return indexSummary.getEstimatedKeyCount();
+    }
+
+    /**
+     * @param ranges
+     * @return An estimate of the number of keys for given ranges in this SSTable.
+     */
+    public long estimatedKeysForRanges(Collection<Range<Token>> ranges)
+    {
+        long sampleKeyCount = 0;
+        List<Pair<Integer, Integer>> sampleIndexes = getSampleIndexesForRanges(indexSummary, ranges);
+        for (Pair<Integer, Integer> sampleIndexRange : sampleIndexes)
+            sampleKeyCount += (sampleIndexRange.right - sampleIndexRange.left + 1);
+
+        // adjust for the current sampling level: (BSL / SL) * index_interval_at_full_sampling
+        long estimatedKeys = sampleKeyCount * ((long) Downsampling.BASE_SAMPLING_LEVEL * indexSummary.getMinIndexInterval()) / indexSummary.getSamplingLevel();
+        return Math.max(1, estimatedKeys);
+    }
+
+    /**
+     * Returns the number of entries in the IndexSummary.  At full sampling, this is approximately 1/INDEX_INTERVALth of
+     * the keys in this SSTable.
+     */
+    public int getIndexSummarySize()
+    {
+        return indexSummary.size();
+    }
+
+    /**
+     * Returns the approximate number of entries the IndexSummary would contain if it were at full sampling.
+     */
+    public int getMaxIndexSummarySize()
+    {
+        return indexSummary.getMaxNumberOfEntries();
+    }
+
+    /**
+     * Returns the key for the index summary entry at `index`.
+     */
+    public byte[] getIndexSummaryKey(int index)
+    {
+        return indexSummary.getKey(index);
+    }
+
+    private static List<Pair<Integer,Integer>> getSampleIndexesForRanges(IndexSummary summary, Collection<Range<Token>> ranges)
+    {
+        // use the index to determine a minimal section for each range
+        List<Pair<Integer,Integer>> positions = new ArrayList<>();
+
+        for (Range<Token> range : Range.normalize(ranges))
+        {
+            RowPosition leftPosition = range.left.maxKeyBound();
+            RowPosition rightPosition = range.right.maxKeyBound();
+
+            int left = summary.binarySearch(leftPosition);
+            if (left < 0)
+                left = (left + 1) * -1;
+            else
+                // left range are start exclusive
+                left = left + 1;
+            if (left == summary.size())
+                // left is past the end of the sampling
+                continue;
+
+            int right = Range.isWrapAround(range.left, range.right)
+                    ? summary.size() - 1
+                    : summary.binarySearch(rightPosition);
+            if (right < 0)
+            {
+                // range are end inclusive so we use the previous index from what binarySearch give us
+                // since that will be the last index we will return
+                right = (right + 1) * -1;
+                if (right == 0)
+                    // Means the first key is already stricly greater that the right bound
+                    continue;
+                right--;
+            }
+
+            if (left > right)
+                // empty range
+                continue;
+            positions.add(Pair.create(left, right));
+        }
+        return positions;
+    }
+
+    public Iterable<DecoratedKey> getKeySamples(final Range<Token> range)
+    {
+        final List<Pair<Integer, Integer>> indexRanges = getSampleIndexesForRanges(indexSummary, Collections.singletonList(range));
+
+        if (indexRanges.isEmpty())
+            return Collections.emptyList();
+
+        return new Iterable<DecoratedKey>()
+        {
+            public Iterator<DecoratedKey> iterator()
+            {
+                return new Iterator<DecoratedKey>()
+                {
+                    private Iterator<Pair<Integer, Integer>> rangeIter = indexRanges.iterator();
+                    private Pair<Integer, Integer> current;
+                    private int idx;
+
+                    public boolean hasNext()
+                    {
+                        if (current == null || idx > current.right)
+                        {
+                            if (rangeIter.hasNext())
+                            {
+                                current = rangeIter.next();
+                                idx = current.left;
+                                return true;
+                            }
+                            return false;
+                        }
+
+                        return true;
+                    }
+
+                    public DecoratedKey next()
+                    {
+                        byte[] bytes = indexSummary.getKey(idx++);
+                        return partitioner.decorateKey(ByteBuffer.wrap(bytes));
+                    }
+
+                    public void remove()
+                    {
+                        throw new UnsupportedOperationException();
+                    }
+                };
+            }
+        };
+    }
+
+    /**
+     * Determine the minimal set of sections that can be extracted from this SSTable to cover the given ranges.
+     * @return A sorted list of (offset,end) pairs that cover the given ranges in the datafile for this SSTable.
+     */
+    public List<Pair<Long,Long>> getPositionsForRanges(Collection<Range<Token>> ranges)
+    {
+        // use the index to determine a minimal section for each range
+        List<Pair<Long,Long>> positions = new ArrayList<>();
+        for (Range<Token> range : Range.normalize(ranges))
+        {
+            assert !range.isWrapAround() || range.right.isMinimum();
+            // truncate the range so it at most covers the sstable
+            AbstractBounds<RowPosition> bounds = Range.makeRowRange(range);
+            RowPosition leftBound = bounds.left.compareTo(first) > 0 ? bounds.left : first.getToken().minKeyBound();
+            RowPosition rightBound = bounds.right.isMinimum() ? last.getToken().maxKeyBound() : bounds.right;
+
+            if (leftBound.compareTo(last) > 0 || rightBound.compareTo(first) < 0)
+                continue;
+
+            long left = getPosition(leftBound, Operator.GT).position;
+            long right = (rightBound.compareTo(last) > 0)
+                         ? uncompressedLength()
+                         : getPosition(rightBound, Operator.GT).position;
+
+            if (left == right)
+                // empty range
+                continue;
+
+            assert left < right : String.format("Range=%s openReason=%s first=%s last=%s left=%d right=%d", range, openReason, first, last, left, right);
+            positions.add(Pair.create(left, right));
+        }
+        return positions;
+    }
+
+    public KeyCacheKey getCacheKey(DecoratedKey key)
+    {
+        return new KeyCacheKey(metadata.ksAndCFName, descriptor, key.getKey());
+    }
+
+    public void cacheKey(DecoratedKey key, RowIndexEntry info)
+    {
+        CachingOptions caching = metadata.getCaching();
+
+        if (!caching.keyCache.isEnabled()
+                || keyCache == null
+                || keyCache.getCapacity() == 0)
+        {
+            return;
+        }
+
+        KeyCacheKey cacheKey = new KeyCacheKey(metadata.ksAndCFName, descriptor, key.getKey());
+        logger.trace("Adding cache entry for {} -> {}", cacheKey, info);
+        keyCache.put(cacheKey, info);
+    }
+
+    public RowIndexEntry getCachedPosition(DecoratedKey key, boolean updateStats)
+    {
+        return getCachedPosition(new KeyCacheKey(metadata.ksAndCFName, descriptor, key.getKey()), updateStats);
+    }
+
+    protected RowIndexEntry getCachedPosition(KeyCacheKey unifiedKey, boolean updateStats)
+    {
+        if (keyCache != null && keyCache.getCapacity() > 0 && metadata.getCaching().keyCache.isEnabled()) {
+            if (updateStats)
+            {
+                RowIndexEntry cachedEntry = keyCache.get(unifiedKey);
+                keyCacheRequest.incrementAndGet();
+                if (cachedEntry != null)
+                {
+                    keyCacheHit.incrementAndGet();
+                    bloomFilterTracker.addTruePositive();
+                }
+                return cachedEntry;
+            }
+            else
+            {
+                return keyCache.getInternal(unifiedKey);
+            }
+        }
+        return null;
+    }
+
+    /**
+     * Get position updating key cache and stats.
+     * @see #getPosition(org.apache.cassandra.db.RowPosition, SSTableReader.Operator, boolean)
+     */
+    public RowIndexEntry getPosition(RowPosition key, Operator op)
+    {
+        return getPosition(key, op, true, false);
+    }
+
+    public RowIndexEntry getPosition(RowPosition key, Operator op, boolean updateCacheAndStats)
+    {
+        return getPosition(key, op, updateCacheAndStats, false);
+    }
+    /**
+     * @param key The key to apply as the rhs to the given Operator. A 'fake' key is allowed to
+     * allow key selection by token bounds but only if op != * EQ
+     * @param op The Operator defining matching keys: the nearest key to the target matching the operator wins.
+     * @param updateCacheAndStats true if updating stats and cache
+     * @return The index entry corresponding to the key, or null if the key is not present
+     */
+    protected abstract RowIndexEntry getPosition(RowPosition key, Operator op, boolean updateCacheAndStats, boolean permitMatchPastLast);
+
+    //Corresponds to a name column
+    public abstract OnDiskAtomIterator iterator(DecoratedKey key, SortedSet<CellName> columns);
+    public abstract OnDiskAtomIterator iterator(FileDataInput file, DecoratedKey key, SortedSet<CellName> columns, RowIndexEntry indexEntry);
+
+    //Corresponds to a slice query
+    public abstract OnDiskAtomIterator iterator(DecoratedKey key, ColumnSlice[] slices, boolean reverse);
+    public abstract OnDiskAtomIterator iterator(FileDataInput file, DecoratedKey key, ColumnSlice[] slices, boolean reversed, RowIndexEntry indexEntry);
+
+    /**
+     * Finds and returns the first key beyond a given token in this SSTable or null if no such key exists.
+     */
+    public DecoratedKey firstKeyBeyond(RowPosition token)
+    {
+        if (token.compareTo(first) < 0)
+            return first;
+
+        long sampledPosition = getIndexScanPosition(token);
+
+        if (ifile == null)
+            return null;
+
+        Iterator<FileDataInput> segments = ifile.iterator(sampledPosition);
+        while (segments.hasNext())
+        {
+            String path = null;
+            try (FileDataInput in = segments.next();)
+            {
+                path = in.getPath();
+                while (!in.isEOF())
+                {
+                    ByteBuffer indexKey = ByteBufferUtil.readWithShortLength(in);
+                    DecoratedKey indexDecoratedKey = partitioner.decorateKey(indexKey);
+                    if (indexDecoratedKey.compareTo(token) > 0)
+                        return indexDecoratedKey;
+
+                    RowIndexEntry.Serializer.skip(in);
+                }
+            }
+            catch (IOException e)
+            {
+                markSuspect();
+                throw new CorruptSSTableException(e, path);
+            }
+        }
+
+        return null;
+    }
+
+    /**
+     * @return The length in bytes of the data for this SSTable. For
+     * compressed files, this is not the same thing as the on disk size (see
+     * onDiskLength())
+     */
+    public long uncompressedLength()
+    {
+        return dfile.length;
+    }
+
+    /**
+     * @return The length in bytes of the on disk size for this SSTable. For
+     * compressed files, this is not the same thing as the data length (see
+     * length())
+     */
+    public long onDiskLength()
+    {
+        return dfile.onDiskLength;
+    }
+
+    /**
+     * Mark the sstable as obsolete, i.e., compacted into newer sstables.
+     *
+     * When calling this function, the caller must ensure that the SSTableReader is not referenced anywhere
+     * except for threads holding a reference.
+     *
+     * @return true if the this is the first time the file was marked obsolete.  Calling this
+     * multiple times is usually buggy (see exceptions in Tracker.unmarkCompacting and removeOldSSTablesSize).
+     */
+    public boolean markObsolete(Tracker tracker)
+    {
+        if (logger.isTraceEnabled())
+            logger.trace("Marking {} compacted", getFilename());
+
+        synchronized (tidy.global)
+        {
+            assert !tidy.isReplaced;
+        }
+        if (!tidy.global.isCompacted.getAndSet(true))
+        {
+            tidy.type.markObsolete(this, tracker);
+            return true;
+        }
+        return false;
+    }
+
+    public boolean isMarkedCompacted()
+    {
+        return tidy.global.isCompacted.get();
+    }
+
+    public void markSuspect()
+    {
+        if (logger.isTraceEnabled())
+            logger.trace("Marking {} as a suspect for blacklisting.", getFilename());
+
+        isSuspect.getAndSet(true);
+    }
+
+    public boolean isMarkedSuspect()
+    {
+        return isSuspect.get();
+    }
+
+
+    /**
+     * I/O SSTableScanner
+     * @return A Scanner for seeking over the rows of the SSTable.
+     */
+    public ISSTableScanner getScanner()
+    {
+        return getScanner((RateLimiter) null);
+    }
+
+    public ISSTableScanner getScanner(RateLimiter limiter)
+    {
+        return getScanner(DataRange.allData(partitioner), limiter);
+    }
+
+    /**
+     *
+     * @param dataRange filter to use when reading the columns
+     * @return A Scanner for seeking over the rows of the SSTable.
+     */
+    public ISSTableScanner getScanner(DataRange dataRange)
+    {
+        return getScanner(dataRange, null);
+    }
+
+    /**
+     * Direct I/O SSTableScanner over a defined range of tokens.
+     *
+     * @param range the range of keys to cover
+     * @return A Scanner for seeking over the rows of the SSTable.
+     */
+    public ISSTableScanner getScanner(Range<Token> range, RateLimiter limiter)
+    {
+        if (range == null)
+            return getScanner(limiter);
+        return getScanner(Collections.singletonList(range), limiter);
+    }
+
+    /**
+     * Direct I/O SSTableScanner over a defined collection of ranges of tokens.
+     *
+     * @param ranges the range of keys to cover
+     * @return A Scanner for seeking over the rows of the SSTable.
+     */
+    public abstract ISSTableScanner getScanner(Collection<Range<Token>> ranges, RateLimiter limiter);
+
+    /**
+     *
+     * @param dataRange filter to use when reading the columns
+     * @return A Scanner for seeking over the rows of the SSTable.
+     */
+    public abstract ISSTableScanner getScanner(DataRange dataRange, RateLimiter limiter);
+
+
+
+    public FileDataInput getFileDataInput(long position)
+    {
+        return dfile.getSegment(position);
+    }
+
+    /**
+     * Tests if the sstable contains data newer than the given age param (in localhost currentMilli time).
+     * This works in conjunction with maxDataAge which is an upper bound on the create of data in this sstable.
+     * @param age The age to compare the maxDataAre of this sstable. Measured in millisec since epoc on this host
+     * @return True iff this sstable contains data that's newer than the given age parameter.
+     */
+    public boolean newSince(long age)
+    {
+        return maxDataAge > age;
+    }
+
+    public void createLinks(String snapshotDirectoryPath)
+    {
+        for (Component component : components)
+        {
+            File sourceFile = new File(descriptor.filenameFor(component));
+            File targetLink = new File(snapshotDirectoryPath, sourceFile.getName());
+            FileUtils.createHardLink(sourceFile, targetLink);
+        }
+    }
+
+    public boolean isRepaired()
+    {
+        return sstableMetadata.repairedAt != ActiveRepairService.UNREPAIRED_SSTABLE;
+    }
+
+    /**
+     * TODO: Move someplace reusable
+     */
+    public abstract static class Operator
+    {
+        public static final Operator EQ = new Equals();
+        public static final Operator GE = new GreaterThanOrEqualTo();
+        public static final Operator GT = new GreaterThan();
+
+        /**
+         * @param comparison The result of a call to compare/compareTo, with the desired field on the rhs.
+         * @return less than 0 if the operator cannot match forward, 0 if it matches, greater than 0 if it might match forward.
+         */
+        public abstract int apply(int comparison);
+
+        final static class Equals extends Operator
+        {
+            public int apply(int comparison) { return -comparison; }
+        }
+
+        final static class GreaterThanOrEqualTo extends Operator
+        {
+            public int apply(int comparison) { return comparison >= 0 ? 0 : 1; }
+        }
+
+        final static class GreaterThan extends Operator
+        {
+            public int apply(int comparison) { return comparison > 0 ? 0 : 1; }
+        }
+    }
+
+    public long getBloomFilterFalsePositiveCount()
+    {
+        return bloomFilterTracker.getFalsePositiveCount();
+    }
+
+    public long getRecentBloomFilterFalsePositiveCount()
+    {
+        return bloomFilterTracker.getRecentFalsePositiveCount();
+    }
+
+    public long getBloomFilterTruePositiveCount()
+    {
+        return bloomFilterTracker.getTruePositiveCount();
+    }
+
+    public long getRecentBloomFilterTruePositiveCount()
+    {
+        return bloomFilterTracker.getRecentTruePositiveCount();
+    }
+
+    public InstrumentingCache<KeyCacheKey, RowIndexEntry> getKeyCache()
+    {
+        return keyCache;
+    }
+
+    public EstimatedHistogram getEstimatedRowSize()
+    {
+        return sstableMetadata.estimatedRowSize;
+    }
+
+    public EstimatedHistogram getEstimatedColumnCount()
+    {
+        return sstableMetadata.estimatedColumnCount;
+    }
+
+    public double getEstimatedDroppableTombstoneRatio(int gcBefore)
+    {
+        return sstableMetadata.getEstimatedDroppableTombstoneRatio(gcBefore);
+    }
+
+    public double getDroppableTombstonesBefore(int gcBefore)
+    {
+        return sstableMetadata.getDroppableTombstonesBefore(gcBefore);
+    }
+
+    public double getCompressionRatio()
+    {
+        return sstableMetadata.compressionRatio;
+    }
+
+    public long getMinTimestamp()
+    {
+        return sstableMetadata.minTimestamp;
+    }
+
+    public long getMaxTimestamp()
+    {
+        return sstableMetadata.maxTimestamp;
+    }
+
+    public Set<Integer> getAncestors()
+    {
+        try
+        {
+            CompactionMetadata compactionMetadata = (CompactionMetadata) descriptor.getMetadataSerializer().deserialize(descriptor, MetadataType.COMPACTION);
+            if (compactionMetadata != null)
+                return compactionMetadata.ancestors;
+            return Collections.emptySet();
+        }
+        catch (IOException e)
+        {
+            SSTableReader.logOpenException(descriptor, e);
+            return Collections.emptySet();
+        }
+    }
+
+    public int getSSTableLevel()
+    {
+        return sstableMetadata.sstableLevel;
+    }
+
+    /**
+     * Reloads the sstable metadata from disk.
+     *
+     * Called after level is changed on sstable, for example if the sstable is dropped to L0
+     *
+     * Might be possible to remove in future versions
+     *
+     * @throws IOException
+     */
+    public void reloadSSTableMetadata() throws IOException
+    {
+        this.sstableMetadata = (StatsMetadata) descriptor.getMetadataSerializer().deserialize(descriptor, MetadataType.STATS);
+    }
+
+    public StatsMetadata getSSTableMetadata()
+    {
+        return sstableMetadata;
+    }
+
+    public RandomAccessReader openDataReader(RateLimiter limiter)
+    {
+        assert limiter != null;
+        return dfile.createThrottledReader(limiter);
+    }
+
+    public RandomAccessReader openDataReader()
+    {
+        return dfile.createReader();
+    }
+
+    public RandomAccessReader openIndexReader()
+    {
+        if (ifile != null)
+            return ifile.createReader();
+        return null;
+    }
+
+    /**
+     * @param component component to get timestamp.
+     * @return last modified time for given component. 0 if given component does not exist or IO error occurs.
+     */
+    public long getCreationTimeFor(Component component)
+    {
+        return new File(descriptor.filenameFor(component)).lastModified();
+    }
+
+    /**
+     * @return Number of key cache hit
+     */
+    public long getKeyCacheHit()
+    {
+        return keyCacheHit.get();
+    }
+
+    /**
+     * @return Number of key cache request
+     */
+    public long getKeyCacheRequest()
+    {
+        return keyCacheRequest.get();
+    }
+
+    /**
+     * Increment the total row read count and read rate for this SSTable.  This should not be incremented for range
+     * slice queries, row cache hits, or non-query reads, like compaction.
+     */
+    public void incrementReadCount()
+    {
+        if (readMeter != null)
+            readMeter.mark();
+    }
+
+    public static class SizeComparator implements Comparator<SSTableReader>
+    {
+        public int compare(SSTableReader o1, SSTableReader o2)
+        {
+            return Longs.compare(o1.onDiskLength(), o2.onDiskLength());
+        }
+    }
+
+    public Ref<SSTableReader> tryRef()
+    {
+        return selfRef.tryRef();
+    }
+
+    public Ref<SSTableReader> selfRef()
+    {
+        return selfRef;
+    }
+
+    public Ref<SSTableReader> ref()
+    {
+        return selfRef.ref();
+    }
+
+    void setup(boolean trackHotness)
+    {
+        tidy.setup(this, trackHotness);
+        this.readMeter = tidy.global.readMeter;
+    }
+
+    @VisibleForTesting
+    public void overrideReadMeter(RestorableMeter readMeter)
+    {
+        this.readMeter = tidy.global.readMeter = readMeter;
+    }
+
+    /**
+     * One instance per SSTableReader we create. This references the type-shared tidy, which in turn references
+     * the globally shared tidy, i.e.
+     *
+     * InstanceTidier => DescriptorTypeTitdy => GlobalTidy
+     *
+     * We can create many InstanceTidiers (one for every time we reopen an sstable with MOVED_START for example), but there can only be
+     * two DescriptorTypeTidy (FINAL and TEMPLINK) and only one GlobalTidy for one single logical sstable.
+     *
+     * When the InstanceTidier cleansup, it releases its reference to its DescriptorTypeTidy; when all InstanceTidiers
+     * for that type have run, the DescriptorTypeTidy cleansup. DescriptorTypeTidy behaves in the same way towards GlobalTidy.
+     *
+     * For ease, we stash a direct reference to both our type-shared and global tidier
+     */
+    private static final class InstanceTidier implements Tidy
+    {
+        private final Descriptor descriptor;
+        private final CFMetaData metadata;
+        private IFilter bf;
+        private IndexSummary summary;
+
+        private SegmentedFile dfile;
+        private SegmentedFile ifile;
+        private Runnable runOnClose;
+        private boolean isReplaced = false;
+
+        // a reference to our shared per-Descriptor.Type tidy instance, that
+        // we will release when we are ourselves released
+        private Ref<DescriptorTypeTidy> typeRef;
+
+        // a convenience stashing of the shared per-descriptor-type tidy instance itself
+        // and the per-logical-sstable globally shared state that it is linked to
+        private DescriptorTypeTidy type;
+        private GlobalTidy global;
+
+        private volatile boolean setup;
+
+        void setup(SSTableReader reader, boolean trackHotness)
+        {
+            this.setup = true;
+            this.bf = reader.bf;
+            this.summary = reader.indexSummary;
+            this.dfile = reader.dfile;
+            this.ifile = reader.ifile;
+            // get a new reference to the shared descriptor-type tidy
+            this.typeRef = DescriptorTypeTidy.get(reader);
+            this.type = typeRef.get();
+            this.global = type.globalRef.get();
+            if (trackHotness)
+                global.ensureReadMeter();
+        }
+
+        InstanceTidier(Descriptor descriptor, CFMetaData metadata)
+        {
+            this.descriptor = descriptor;
+            this.metadata = metadata;
+        }
+
+        public void tidy()
+        {
+            if (logger.isTraceEnabled())
+                logger.trace("Running instance tidier for {} with setup {}", descriptor, setup);
+
+            // don't try to cleanup if the sstablereader was never fully constructed
+            if (!setup)
+                return;
+
+            final ColumnFamilyStore cfs = Schema.instance.getColumnFamilyStoreInstance(metadata.cfId);
+            final OpOrder.Barrier barrier;
+            if (cfs != null)
+            {
+                barrier = cfs.readOrdering.newBarrier();
+                barrier.issue();
+            }
+            else
+                barrier = null;
+
+            ScheduledExecutors.nonPeriodicTasks.execute(new Runnable()
+            {
+                public void run()
+                {
+                    if (logger.isTraceEnabled())
+                        logger.trace("Async instance tidier for {}, before barrier", descriptor);
+
+                    if (barrier != null)
+                        barrier.await();
+
+                    if (logger.isTraceEnabled())
+                        logger.trace("Async instance tidier for {}, after barrier", descriptor);
+
+                    if (bf != null)
+                        bf.close();
+                    if (summary != null)
+                        summary.close();
+                    if (runOnClose != null)
+                        runOnClose.run();
+                    if (dfile != null)
+                        dfile.close();
+                    if (ifile != null)
+                        ifile.close();
+                    typeRef.release();
+
+                    if (logger.isTraceEnabled())
+                        logger.trace("Async instance tidier for {}, completed", descriptor);
+                }
+            });
+        }
+
+        public String name()
+        {
+            return descriptor.toString();
+        }
+
+        void releaseSummary()
+        {
+            summary.close();
+            assert summary.isCleanedUp();
+            summary = null;
+        }
+    }
+
+    /**
+     * One shared between all instances of a given Descriptor.Type.
+     * Performs only two things: the deletion of the sstables for the type,
+     * if necessary; and the shared reference to the globally shared state.
+     *
+     * All InstanceTidiers, on setup(), ask the static get() method for their shared state,
+     * and stash a reference to it to be released when they are. Once all such references are
+     * released, the shared tidy will be performed.
+     */
+    static final class DescriptorTypeTidy implements Tidy
+    {
+        // keyed by REAL descriptor (TMPLINK/FINAL), mapping to the shared DescriptorTypeTidy for that descriptor
+        static final ConcurrentMap<Descriptor, Ref<DescriptorTypeTidy>> lookup = new ConcurrentHashMap<>();
+
+        private final Descriptor desc;
+        private final Ref<GlobalTidy> globalRef;
+        private final Set<Component> components;
+        private long sizeOnDelete;
+        private Counter totalDiskSpaceUsed;
+
+        DescriptorTypeTidy(Descriptor desc, SSTableReader sstable)
+        {
+            this.desc = desc;
+            // get a new reference to the shared global tidy
+            this.globalRef = GlobalTidy.get(sstable);
+            this.components = sstable.components;
+        }
+
+        void markObsolete(SSTableReader instance, Tracker tracker)
+        {
+            // the tracker is used only to notify listeners of deletion of the sstable;
+            // since deletion of a non-final file is not really deletion of the sstable,
+            // we don't want to notify the listeners in this event
+            if (tracker != null && tracker.cfstore != null && desc.type == Descriptor.Type.FINAL)
+            {
+                sizeOnDelete = instance.bytesOnDisk();
+                totalDiskSpaceUsed = tracker.cfstore.metric.totalDiskSpaceUsed;
+                tracker.notifyDeleting(instance);
+            }
+        }
+
+        public void tidy()
+        {
+            lookup.remove(desc);
+            boolean isCompacted = globalRef.get().isCompacted.get();
+            globalRef.release();
+            switch (desc.type)
+            {
+                case FINAL:
+                    if (isCompacted)
+                        new SSTableDeletingTask(desc, components, totalDiskSpaceUsed, sizeOnDelete).run();
+                    break;
+                case TEMPLINK:
+                    new SSTableDeletingTask(desc, components, null, 0).run();
+                    break;
+                default:
+                    throw new IllegalStateException();
+            }
+        }
+
+        public String name()
+        {
+            return desc.toString();
+        }
+
+        // get a new reference to the shared DescriptorTypeTidy for this sstable
+        @SuppressWarnings("resource")
+        public static Ref<DescriptorTypeTidy> get(SSTableReader sstable)
+        {
+            Descriptor desc = sstable.descriptor;
+            if (sstable.openReason == OpenReason.EARLY)
+                desc = desc.asType(Descriptor.Type.TEMPLINK);
+            Ref<DescriptorTypeTidy> refc = lookup.get(desc);
+            if (refc != null)
+                return refc.ref();
+            final DescriptorTypeTidy tidy = new DescriptorTypeTidy(desc, sstable);
+            refc = new Ref<>(tidy, tidy);
+            Ref<?> ex = lookup.putIfAbsent(desc, refc);
+            if (ex != null)
+            {
+                refc.close();
+                throw new AssertionError();
+            }
+            return refc;
+        }
+    }
+
+    /**
+     * One instance per logical sstable. This both tracks shared cleanup and some shared state related
+     * to the sstable's lifecycle. All DescriptorTypeTidy instances, on construction, obtain a reference to us
+     * via our static get(). There should only ever be at most two such references extant at any one time,
+     * since only TMPLINK and FINAL type descriptors should be open as readers. When all files of both
+     * kinds have been released, this shared tidy will be performed.
+     */
+    static final class GlobalTidy implements Tidy
+    {
+        // keyed by FINAL descriptor, mapping to the shared GlobalTidy for that descriptor
+        static final ConcurrentMap<Descriptor, Ref<GlobalTidy>> lookup = new ConcurrentHashMap<>();
+
+        private final Descriptor desc;
+        // the readMeter that is shared between all instances of the sstable, and can be overridden in all of them
+        // at once also, for testing purposes
+        private RestorableMeter readMeter;
+        // the scheduled persistence of the readMeter, that we will cancel once all instances of this logical
+        // sstable have been released
+        private ScheduledFuture readMeterSyncFuture;
+        // shared state managing if the logical sstable has been compacted; this is used in cleanup both here
+        // and in the FINAL type tidier
+        private final AtomicBoolean isCompacted;
+
+        GlobalTidy(final SSTableReader reader)
+        {
+            this.desc = reader.descriptor;
+            this.isCompacted = new AtomicBoolean();
+        }
+
+        void ensureReadMeter()
+        {
+            if (readMeter != null)
+                return;
+
+            // Don't track read rates for tables in the system keyspace and don't bother trying to load or persist
+            // the read meter when in client mode.
+            if (SystemKeyspace.NAME.equals(desc.ksname) || !DatabaseDescriptor.isDaemonInitialized())
+            {
+                readMeter = null;
+                readMeterSyncFuture = null;
+                return;
+            }
+
+            readMeter = SystemKeyspace.getSSTableReadMeter(desc.ksname, desc.cfname, desc.generation);
+            // sync the average read rate to system.sstable_activity every five minutes, starting one minute from now
+            readMeterSyncFuture = syncExecutor.scheduleAtFixedRate(new Runnable()
+            {
+                public void run()
+                {
+                    if (!isCompacted.get())
+                    {
+                        meterSyncThrottle.acquire();
+                        SystemKeyspace.persistSSTableReadMeter(desc.ksname, desc.cfname, desc.generation, readMeter);
+                    }
+                }
+            }, 1, 5, TimeUnit.MINUTES);
+        }
+
+        public void tidy()
+        {
+            lookup.remove(desc);
+            if (readMeterSyncFuture != null)
+            {
+                readMeterSyncFuture.cancel(true);
+                if (isCompacted.get())
+                    SystemKeyspace.clearSSTableReadMeter(desc.ksname, desc.cfname, desc.generation);
+            }
+            // don't ideally want to dropPageCache for the file until all instances have been released
+            CLibrary.trySkipCache(desc.filenameFor(Component.DATA), 0, 0);
+            CLibrary.trySkipCache(desc.filenameFor(Component.PRIMARY_INDEX), 0, 0);
+        }
+
+        public String name()
+        {
+            return desc.toString();
+        }
+
+        // get a new reference to the shared GlobalTidy for this sstable
+        @SuppressWarnings("resource")
+        public static Ref<GlobalTidy> get(SSTableReader sstable)
+        {
+            Descriptor descriptor = sstable.descriptor;
+            Ref<GlobalTidy> refc = lookup.get(descriptor);
+            if (refc != null)
+                return refc.ref();
+            final GlobalTidy tidy = new GlobalTidy(sstable);
+            refc = new Ref<>(tidy, tidy);
+            Ref<?> ex = lookup.putIfAbsent(descriptor, refc);
+            if (ex != null)
+            {
+                refc.close();
+                throw new AssertionError();
+            }
+            return refc;
+        }
+    }
+
+    @VisibleForTesting
+    public static void resetTidying()
+    {
+        GlobalTidy.lookup.clear();
+        DescriptorTypeTidy.lookup.clear();
+    }
+
+    public static abstract class Factory
+    {
+        public abstract SSTableReader open(final Descriptor descriptor,
+                                           Set<Component> components,
+                                           CFMetaData metadata,
+                                           IPartitioner partitioner,
+                                           Long maxDataAge,
+                                           StatsMetadata sstableMetadata,
+                                           OpenReason openReason);
+
+    }
+}
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
new file mode 100644
index 0000000..f99292e
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
@@ -0,0 +1,281 @@
+/*
+ * 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.io.sstable.format;
+
+import com.google.common.collect.Sets;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.db.ColumnFamily;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.RowIndexEntry;
+import org.apache.cassandra.db.compaction.AbstractCompactedRow;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
+import org.apache.cassandra.io.sstable.metadata.MetadataComponent;
+import org.apache.cassandra.io.sstable.metadata.MetadataType;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.concurrent.Transactional;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This is the API all table writers must implement.
+ *
+ * TableWriter.create() is the primary way to create a writer for a particular format.
+ * The format information is part of the Descriptor.
+ */
+public abstract class SSTableWriter extends SSTable implements Transactional
+{
+    protected long repairedAt;
+    protected long maxDataAge = -1;
+    protected final long keyCount;
+    protected final MetadataCollector metadataCollector;
+    protected final RowIndexEntry.IndexSerializer rowIndexEntrySerializer;
+    protected final TransactionalProxy txnProxy = txnProxy();
+
+    protected abstract TransactionalProxy txnProxy();
+
+    // due to lack of multiple inheritance, we use an inner class to proxy our Transactional implementation details
+    protected abstract class TransactionalProxy extends AbstractTransactional
+    {
+        // should be set during doPrepare()
+        protected SSTableReader finalReader;
+        protected boolean openResult;
+    }
+
+    protected SSTableWriter(Descriptor descriptor, long keyCount, long repairedAt, CFMetaData metadata, IPartitioner partitioner, MetadataCollector metadataCollector)
+    {
+        super(descriptor, components(metadata), metadata, partitioner);
+        this.keyCount = keyCount;
+        this.repairedAt = repairedAt;
+        this.metadataCollector = metadataCollector;
+        this.rowIndexEntrySerializer = descriptor.version.getSSTableFormat().getIndexSerializer(metadata);
+    }
+
+    public static SSTableWriter create(Descriptor descriptor, Long keyCount, Long repairedAt, CFMetaData metadata,  IPartitioner partitioner, MetadataCollector metadataCollector)
+    {
+        Factory writerFactory = descriptor.getFormat().getWriterFactory();
+        return writerFactory.open(descriptor, keyCount, repairedAt, metadata, partitioner, metadataCollector);
+    }
+
+    public static SSTableWriter create(Descriptor descriptor, long keyCount, long repairedAt)
+    {
+        return create(descriptor, keyCount, repairedAt, 0);
+    }
+
+    public static SSTableWriter create(Descriptor descriptor, long keyCount, long repairedAt, int sstableLevel)
+    {
+        CFMetaData metadata = Schema.instance.getCFMetaData(descriptor);
+        return create(metadata, descriptor, keyCount, repairedAt, sstableLevel, DatabaseDescriptor.getPartitioner());
+    }
+
+    public static SSTableWriter create(CFMetaData metadata,
+                                       Descriptor descriptor,
+                                       long keyCount,
+                                       long repairedAt,
+                                       int sstableLevel,
+                                       IPartitioner partitioner)
+    {
+        MetadataCollector collector = new MetadataCollector(metadata.comparator).sstableLevel(sstableLevel);
+        return create(descriptor, keyCount, repairedAt, metadata, partitioner, collector);
+    }
+
+    public static SSTableWriter create(String filename, long keyCount, long repairedAt, int sstableLevel)
+    {
+        return create(Descriptor.fromFilename(filename), keyCount, repairedAt, sstableLevel);
+    }
+
+    public static SSTableWriter create(String filename, long keyCount, long repairedAt)
+    {
+        return create(Descriptor.fromFilename(filename), keyCount, repairedAt, 0);
+    }
+
+    private static Set<Component> components(CFMetaData metadata)
+    {
+        Set<Component> components = new HashSet<Component>(Arrays.asList(Component.DATA,
+                Component.PRIMARY_INDEX,
+                Component.STATS,
+                Component.SUMMARY,
+                Component.TOC,
+                Component.DIGEST));
+
+        if (metadata.getBloomFilterFpChance() < 1.0)
+            components.add(Component.FILTER);
+
+        if (metadata.compressionParameters().sstableCompressor != null)
+        {
+            components.add(Component.COMPRESSION_INFO);
+        }
+        else
+        {
+            // it would feel safer to actually add this component later in maybeWriteDigest(),
+            // but the components are unmodifiable after construction
+            components.add(Component.CRC);
+        }
+        return components;
+    }
+
+
+    public abstract void mark();
+
+
+    /**
+     * @param row
+     * @return null if the row was compacted away entirely; otherwise, the PK index entry for this row
+     */
+    public abstract RowIndexEntry append(AbstractCompactedRow row);
+
+    public abstract void append(DecoratedKey decoratedKey, ColumnFamily cf);
+
+    public abstract long appendFromStream(DecoratedKey key, CFMetaData metadata, DataInput in, Version version) throws IOException;
+
+    public abstract long getFilePointer();
+
+    public abstract long getOnDiskFilePointer();
+
+    public abstract void resetAndTruncate();
+
+    public SSTableWriter setRepairedAt(long repairedAt)
+    {
+        if (repairedAt > 0)
+            this.repairedAt = repairedAt;
+        return this;
+    }
+
+    public SSTableWriter setMaxDataAge(long maxDataAge)
+    {
+        this.maxDataAge = maxDataAge;
+        return this;
+    }
+
+    public SSTableWriter setOpenResult(boolean openResult)
+    {
+        txnProxy.openResult = openResult;
+        return this;
+    }
+
+    /**
+     * Open the resultant SSTableReader before it has been fully written
+     */
+    public abstract SSTableReader openEarly();
+
+    /**
+     * Open the resultant SSTableReader once it has been fully written, but before the
+     * _set_ of tables that are being written together as one atomic operation are all ready
+     */
+    public abstract SSTableReader openFinalEarly();
+
+    public SSTableReader finish(long repairedAt, long maxDataAge, boolean openResult)
+    {
+        if (repairedAt > 0)
+            this.repairedAt = repairedAt;
+        this.maxDataAge = maxDataAge;
+        return finish(openResult);
+    }
+
+    public SSTableReader finish(boolean openResult)
+    {
+        txnProxy.openResult = openResult;
+        txnProxy.finish();
+        return finished();
+    }
+
+    /**
+     * Open the resultant SSTableReader once it has been fully written, and all related state
+     * is ready to be finalised including other sstables being written involved in the same operation
+     */
+    public SSTableReader finished()
+    {
+        return txnProxy.finalReader;
+    }
+
+    // finalise our state on disk, including renaming
+    public final void prepareToCommit()
+    {
+        txnProxy.prepareToCommit();
+    }
+
+    public final Throwable commit(Throwable accumulate)
+    {
+        return txnProxy.commit(accumulate);
+    }
+
+    public final Throwable abort(Throwable accumulate)
+    {
+        return txnProxy.abort(accumulate);
+    }
+
+    public final void close()
+    {
+        txnProxy.close();
+    }
+
+    public final void abort()
+    {
+        txnProxy.abort();
+    }
+
+    protected Map<MetadataType, MetadataComponent> finalizeMetadata()
+    {
+        return metadataCollector.finalizeMetadata(partitioner.getClass().getCanonicalName(),
+                                                  metadata.getBloomFilterFpChance(), repairedAt);
+    }
+
+    protected StatsMetadata statsMetadata()
+    {
+        return (StatsMetadata) finalizeMetadata().get(MetadataType.STATS);
+    }
+
+    public static Descriptor rename(Descriptor tmpdesc, Set<Component> components)
+    {
+        Descriptor newdesc = tmpdesc.asType(Descriptor.Type.FINAL);
+        rename(tmpdesc, newdesc, components);
+        return newdesc;
+    }
+
+    public static void rename(Descriptor tmpdesc, Descriptor newdesc, Set<Component> components)
+    {
+        for (Component component : Sets.difference(components, Sets.newHashSet(Component.DATA, Component.SUMMARY)))
+        {
+            FileUtils.renameWithConfirm(tmpdesc.filenameFor(component), newdesc.filenameFor(component));
+        }
+
+        // do -Data last because -Data present should mean the sstable was completely renamed before crash
+        FileUtils.renameWithConfirm(tmpdesc.filenameFor(Component.DATA), newdesc.filenameFor(Component.DATA));
+
+        // rename it without confirmation because summary can be available for loadNewSSTables but not for closeAndOpenReader
+        FileUtils.renameWithOutConfirm(tmpdesc.filenameFor(Component.SUMMARY), newdesc.filenameFor(Component.SUMMARY));
+    }
+
+
+    public static abstract class Factory
+    {
+        public abstract SSTableWriter open(Descriptor descriptor, long keyCount, long repairedAt, CFMetaData metadata, IPartitioner partitioner, MetadataCollector metadataCollector);
+    }
+}
diff --git a/src/java/org/apache/cassandra/io/sstable/format/Version.java b/src/java/org/apache/cassandra/io/sstable/format/Version.java
new file mode 100644
index 0000000..41a83e1
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/sstable/format/Version.java
@@ -0,0 +1,104 @@
+/*
+ * 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.io.sstable.format;
+
+/**
+ * A set of feature flags associated with a SSTable format
+ *
+ * versions are denoted as [major][minor].  Minor versions must be forward-compatible:
+ * new fields are allowed in e.g. the metadata component, but fields can't be removed
+ * or have their size changed.
+ *
+ * Minor versions were introduced with version "hb" for Cassandra 1.0.3; prior to that,
+ * we always incremented the major version.
+ *
+ */
+public abstract class Version
+{
+    protected final String version;
+    protected final SSTableFormat format;
+    protected Version(SSTableFormat format, String version)
+    {
+        this.format = format;
+        this.version = version;
+    }
+
+    public abstract boolean isLatestVersion();
+
+    public abstract boolean hasSamplingLevel();
+
+    public abstract boolean hasNewStatsFile();
+
+    public abstract boolean hasAllAdlerChecksums();
+
+    public abstract boolean hasRepairedAt();
+
+    public abstract boolean tracksLegacyCounterShards();
+
+    public abstract boolean hasNewFileName();
+
+    public abstract boolean hasCommitLogLowerBound();
+
+    public String getVersion()
+    {
+        return version;
+    }
+
+    public SSTableFormat getSSTableFormat()
+    {
+        return format;
+    }
+
+    /**
+     * @param ver SSTable version
+     * @return True if the given version string matches the format.
+     * @see #version
+     */
+    public static boolean validate(String ver)
+    {
+        return ver != null && ver.matches("[a-z]+");
+    }
+
+    abstract public boolean isCompatible();
+
+    @Override
+    public String toString()
+    {
+        return version;
+    }
+
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        Version version1 = (Version) o;
+
+        if (version != null ? !version.equals(version1.version) : version1.version != null) return false;
+
+        return true;
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return version != null ? version.hashCode() : 0;
+    }
+}
diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java
new file mode 100644
index 0000000..9244bbb
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java
@@ -0,0 +1,216 @@
+/*
+ * 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.io.sstable.format.big;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.AbstractCell;
+import org.apache.cassandra.db.ColumnSerializer;
+import org.apache.cassandra.db.OnDiskAtom;
+import org.apache.cassandra.db.RowIndexEntry;
+import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
+import org.apache.cassandra.db.compaction.AbstractCompactedRow;
+import org.apache.cassandra.db.compaction.CompactionController;
+import org.apache.cassandra.db.compaction.LazilyCompactedRow;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.IndexHelper;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.io.util.FileDataInput;
+
+import java.util.Iterator;
+import java.util.Set;
+
+/**
+ * Legacy bigtable format
+ */
+public class BigFormat implements SSTableFormat
+{
+    public static final BigFormat instance = new BigFormat();
+    public static final BigVersion latestVersion = new BigVersion(BigVersion.current_version);
+    private static final SSTableReader.Factory readerFactory = new ReaderFactory();
+    private static final SSTableWriter.Factory writerFactory = new WriterFactory();
+
+    private BigFormat()
+    {
+
+    }
+
+    @Override
+    public Version getLatestVersion()
+    {
+        return latestVersion;
+    }
+
+    @Override
+    public Version getVersion(String version)
+    {
+        return new BigVersion(version);
+    }
+
+    @Override
+    public SSTableWriter.Factory getWriterFactory()
+    {
+        return writerFactory;
+    }
+
+    @Override
+    public SSTableReader.Factory getReaderFactory()
+    {
+        return readerFactory;
+    }
+
+    @Override
+    public Iterator<OnDiskAtom> getOnDiskIterator(FileDataInput in, ColumnSerializer.Flag flag, int expireBefore, CFMetaData cfm, Version version)
+    {
+        return AbstractCell.onDiskIterator(in, flag, expireBefore, version, cfm.comparator);
+    }
+
+    @Override
+    public AbstractCompactedRow getCompactedRowWriter(CompactionController controller, ImmutableList<OnDiskAtomIterator> onDiskAtomIterators)
+    {
+        return new LazilyCompactedRow(controller, onDiskAtomIterators);
+    }
+
+    @Override
+    public RowIndexEntry.IndexSerializer getIndexSerializer(CFMetaData cfMetaData)
+    {
+        return new RowIndexEntry.Serializer(new IndexHelper.IndexInfo.Serializer(cfMetaData.comparator));
+    }
+
+    static class WriterFactory extends SSTableWriter.Factory
+    {
+        @Override
+        public SSTableWriter open(Descriptor descriptor, long keyCount, long repairedAt, CFMetaData metadata, IPartitioner partitioner, MetadataCollector metadataCollector)
+        {
+            return new BigTableWriter(descriptor, keyCount, repairedAt, metadata, partitioner, metadataCollector);
+        }
+    }
+
+    static class ReaderFactory extends SSTableReader.Factory
+    {
+        @Override
+        public SSTableReader open(Descriptor descriptor, Set<Component> components, CFMetaData metadata, IPartitioner partitioner, Long maxDataAge, StatsMetadata sstableMetadata, SSTableReader.OpenReason openReason)
+        {
+            return new BigTableReader(descriptor, components, metadata, partitioner, maxDataAge, sstableMetadata, openReason);
+        }
+    }
+
+    // versions are denoted as [major][minor].  Minor versions must be forward-compatible:
+    // new fields are allowed in e.g. the metadata component, but fields can't be removed
+    // or have their size changed.
+    //
+    // Minor versions were introduced with version "hb" for Cassandra 1.0.3; prior to that,
+    // we always incremented the major version.
+    static class BigVersion extends Version
+    {
+        public static final String current_version = "lb";
+        public static final String earliest_supported_version = "jb";
+
+        // jb (2.0.1): switch from crc32 to adler32 for compression checksums
+        //             checksum the compressed data
+        // ka (2.1.0): new Statistics.db file format
+        //             index summaries can be downsampled and the sampling level is persisted
+        //             switch uncompressed checksums to adler32
+        //             tracks presense of legacy (local and remote) counter shards
+        // la (2.2.0): new file name format
+        // lb (2.2.7): commit log lower bound included
+
+        private final boolean isLatestVersion;
+        private final boolean hasSamplingLevel;
+        private final boolean newStatsFile;
+        private final boolean hasAllAdlerChecksums;
+        private final boolean hasRepairedAt;
+        private final boolean tracksLegacyCounterShards;
+        private final boolean newFileName;
+        private final boolean hasCommitLogLowerBound;
+
+        public BigVersion(String version)
+        {
+            super(instance,version);
+
+            isLatestVersion = version.compareTo(current_version) == 0;
+            hasSamplingLevel = version.compareTo("ka") >= 0;
+            newStatsFile = version.compareTo("ka") >= 0;
+            hasAllAdlerChecksums = version.compareTo("ka") >= 0;
+            hasRepairedAt = version.compareTo("ka") >= 0;
+            tracksLegacyCounterShards = version.compareTo("ka") >= 0;
+            newFileName = version.compareTo("la") >= 0;
+            hasCommitLogLowerBound = version.compareTo("lb") >= 0;
+        }
+
+        @Override
+        public boolean isLatestVersion()
+        {
+            return isLatestVersion;
+        }
+
+        @Override
+        public boolean hasSamplingLevel()
+        {
+            return hasSamplingLevel;
+        }
+
+        @Override
+        public boolean hasNewStatsFile()
+        {
+            return newStatsFile;
+        }
+
+        @Override
+        public boolean hasAllAdlerChecksums()
+        {
+            return hasAllAdlerChecksums;
+        }
+
+        @Override
+        public boolean hasRepairedAt()
+        {
+            return hasRepairedAt;
+        }
+
+        @Override
+        public boolean tracksLegacyCounterShards()
+        {
+            return tracksLegacyCounterShards;
+        }
+
+        @Override
+        public boolean hasNewFileName()
+        {
+            return newFileName;
+        }
+
+        public boolean hasCommitLogLowerBound()
+        {
+            return hasCommitLogLowerBound;
+        }
+
+        @Override
+        public boolean isCompatible()
+        {
+            return version.compareTo(earliest_supported_version) >= 0 && version.charAt(0) <= current_version.charAt(0);
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableReader.java
new file mode 100644
index 0000000..5c51fbb
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableReader.java
@@ -0,0 +1,261 @@
+/*
+ * 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.io.sstable.format.big;
+
+import com.google.common.util.concurrent.RateLimiter;
+import org.apache.cassandra.cache.KeyCacheKey;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.RowIndexEntry;
+import org.apache.cassandra.db.RowPosition;
+import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
+import org.apache.cassandra.db.composites.CellName;
+import org.apache.cassandra.db.filter.ColumnSlice;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.tracing.Tracing;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.*;
+
+/**
+ * SSTableReaders are open()ed by Keyspace.onStart; after that they are created by SSTableWriter.renameAndOpen.
+ * Do not re-call open() on existing SSTable files; use the references kept by ColumnFamilyStore post-start instead.
+ */
+public class BigTableReader extends SSTableReader
+{
+    private static final Logger logger = LoggerFactory.getLogger(BigTableReader.class);
+
+    BigTableReader(Descriptor desc, Set<Component> components, CFMetaData metadata, IPartitioner partitioner, Long maxDataAge, StatsMetadata sstableMetadata, OpenReason openReason)
+    {
+        super(desc, components, metadata, partitioner, maxDataAge, sstableMetadata, openReason);
+    }
+
+    public OnDiskAtomIterator iterator(DecoratedKey key, SortedSet<CellName> columns)
+    {
+        return new SSTableNamesIterator(this, key, columns);
+    }
+
+    public OnDiskAtomIterator iterator(FileDataInput input, DecoratedKey key, SortedSet<CellName> columns, RowIndexEntry indexEntry )
+    {
+        return new SSTableNamesIterator(this, input, key, columns, indexEntry);
+    }
+
+    public OnDiskAtomIterator iterator(DecoratedKey key, ColumnSlice[] slices, boolean reverse)
+    {
+        return new SSTableSliceIterator(this, key, slices, reverse);
+    }
+
+    public OnDiskAtomIterator iterator(FileDataInput input, DecoratedKey key, ColumnSlice[] slices, boolean reverse, RowIndexEntry indexEntry)
+    {
+        return new SSTableSliceIterator(this, input, key, slices, reverse, indexEntry);
+    }
+    /**
+     *
+     * @param dataRange filter to use when reading the columns
+     * @return A Scanner for seeking over the rows of the SSTable.
+     */
+    public ISSTableScanner getScanner(DataRange dataRange, RateLimiter limiter)
+    {
+        return BigTableScanner.getScanner(this, dataRange, limiter);
+    }
+
+
+    /**
+     * Direct I/O SSTableScanner over a defined collection of ranges of tokens.
+     *
+     * @param ranges the range of keys to cover
+     * @return A Scanner for seeking over the rows of the SSTable.
+     */
+    public ISSTableScanner getScanner(Collection<Range<Token>> ranges, RateLimiter limiter)
+    {
+        return BigTableScanner.getScanner(this, ranges, limiter);
+    }
+
+
+    /**
+     * @param key The key to apply as the rhs to the given Operator. A 'fake' key is allowed to
+     * allow key selection by token bounds but only if op != * EQ
+     * @param op The Operator defining matching keys: the nearest key to the target matching the operator wins.
+     * @param updateCacheAndStats true if updating stats and cache
+     * @return The index entry corresponding to the key, or null if the key is not present
+     */
+    protected RowIndexEntry getPosition(RowPosition key, Operator op, boolean updateCacheAndStats, boolean permitMatchPastLast)
+    {
+        if (op == Operator.EQ)
+        {
+            assert key instanceof DecoratedKey; // EQ only make sense if the key is a valid row key
+            if (!bf.isPresent((DecoratedKey)key))
+            {
+                Tracing.trace("Bloom filter allows skipping sstable {}", descriptor.generation);
+                return null;
+            }
+        }
+
+        // next, the key cache (only make sense for valid row key)
+        if ((op == Operator.EQ || op == Operator.GE) && (key instanceof DecoratedKey))
+        {
+            DecoratedKey decoratedKey = (DecoratedKey)key;
+            KeyCacheKey cacheKey = new KeyCacheKey(metadata.ksAndCFName, descriptor, decoratedKey.getKey());
+            RowIndexEntry cachedPosition = getCachedPosition(cacheKey, updateCacheAndStats);
+            if (cachedPosition != null)
+            {
+                Tracing.trace("Key cache hit for sstable {}", descriptor.generation);
+                return cachedPosition;
+            }
+        }
+
+        // check the smallest and greatest keys in the sstable to see if it can't be present
+        boolean skip = false;
+        if (key.compareTo(first) < 0)
+        {
+            if (op == Operator.EQ)
+                skip = true;
+            else
+                key = first;
+
+            op = Operator.EQ;
+        }
+        else
+        {
+            int l = last.compareTo(key);
+            // l <= 0  => we may be looking past the end of the file; we then narrow our behaviour to:
+            //             1) skipping if strictly greater for GE and EQ;
+            //             2) skipping if equal and searching GT, and we aren't permitting matching past last
+            skip = l <= 0 && (l < 0 || (!permitMatchPastLast && op == Operator.GT));
+        }
+        if (skip)
+        {
+            if (op == Operator.EQ && updateCacheAndStats)
+                bloomFilterTracker.addFalsePositive();
+            Tracing.trace("Check against min and max keys allows skipping sstable {}", descriptor.generation);
+            return null;
+        }
+
+        int binarySearchResult = indexSummary.binarySearch(key);
+        long sampledPosition = getIndexScanPositionFromBinarySearchResult(binarySearchResult, indexSummary);
+        int sampledIndex = getIndexSummaryIndexFromBinarySearchResult(binarySearchResult);
+
+        int effectiveInterval = indexSummary.getEffectiveIndexIntervalAfterIndex(sampledIndex);
+
+        if (ifile == null)
+            return null;
+
+        // scan the on-disk index, starting at the nearest sampled position.
+        // The check against IndexInterval is to be exit the loop in the EQ case when the key looked for is not present
+        // (bloom filter false positive). But note that for non-EQ cases, we might need to check the first key of the
+        // next index position because the searched key can be greater the last key of the index interval checked if it
+        // is lesser than the first key of next interval (and in that case we must return the position of the first key
+        // of the next interval).
+        int i = 0;
+        Iterator<FileDataInput> segments = ifile.iterator(sampledPosition);
+        while (segments.hasNext())
+        {
+            String path = null;
+            try (FileDataInput in = segments.next())
+            {
+                path = in.getPath();
+                while (!in.isEOF())
+                {
+                    i++;
+
+                    ByteBuffer indexKey = ByteBufferUtil.readWithShortLength(in);
+
+                    boolean opSatisfied; // did we find an appropriate position for the op requested
+                    boolean exactMatch; // is the current position an exact match for the key, suitable for caching
+
+                    // Compare raw keys if possible for performance, otherwise compare decorated keys.
+                    if (op == Operator.EQ && i <= effectiveInterval)
+                    {
+                        opSatisfied = exactMatch = indexKey.equals(((DecoratedKey) key).getKey());
+                    }
+                    else
+                    {
+                        DecoratedKey indexDecoratedKey = partitioner.decorateKey(indexKey);
+                        int comparison = indexDecoratedKey.compareTo(key);
+                        int v = op.apply(comparison);
+                        opSatisfied = (v == 0);
+                        exactMatch = (comparison == 0);
+                        if (v < 0)
+                        {
+                            Tracing.trace("Partition index lookup allows skipping sstable {}", descriptor.generation);
+                            return null;
+                        }
+                    }
+
+                    if (opSatisfied)
+                    {
+                        // read data position from index entry
+                        RowIndexEntry indexEntry = rowIndexEntrySerializer.deserialize(in, descriptor.version);
+                        if (exactMatch && updateCacheAndStats)
+                        {
+                            assert key instanceof DecoratedKey; // key can be == to the index key only if it's a true row key
+                            DecoratedKey decoratedKey = (DecoratedKey)key;
+
+                            if (logger.isTraceEnabled())
+                            {
+                                // expensive sanity check!  see CASSANDRA-4687
+                                try (FileDataInput fdi = dfile.getSegment(indexEntry.position))
+                                {
+                                    DecoratedKey keyInDisk = partitioner.decorateKey(ByteBufferUtil.readWithShortLength(fdi));
+                                    if (!keyInDisk.equals(key))
+                                        throw new AssertionError(String.format("%s != %s in %s", keyInDisk, key, fdi.getPath()));
+                                }
+                            }
+
+                            // store exact match for the key
+                            cacheKey(decoratedKey, indexEntry);
+                        }
+                        if (op == Operator.EQ && updateCacheAndStats)
+                            bloomFilterTracker.addTruePositive();
+                        Tracing.trace("Partition index with {} entries found for sstable {}", indexEntry.columnsIndex().size(), descriptor.generation);
+                        return indexEntry;
+                    }
+
+                    RowIndexEntry.Serializer.skip(in);
+                }
+            }
+            catch (IOException e)
+            {
+                markSuspect();
+                throw new CorruptSSTableException(e, path);
+            }
+        }
+
+        if (op == SSTableReader.Operator.EQ && updateCacheAndStats)
+            bloomFilterTracker.addFalsePositive();
+        Tracing.trace("Partition index lookup complete (bloom filter false positive) for sstable {}", descriptor.generation);
+        return null;
+    }
+
+
+}
diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableScanner.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableScanner.java
new file mode 100644
index 0000000..d477152
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableScanner.java
@@ -0,0 +1,369 @@
+/*
+ * 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.io.sstable.format.big;
+
+import java.io.IOException;
+import java.util.*;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import com.google.common.collect.AbstractIterator;
+import com.google.common.collect.Ordering;
+import com.google.common.util.concurrent.RateLimiter;
+
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.RowIndexEntry;
+import org.apache.cassandra.db.RowPosition;
+import org.apache.cassandra.db.columniterator.IColumnIteratorFactory;
+import org.apache.cassandra.db.columniterator.LazyColumnIterator;
+import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.AbstractBounds.Boundary;
+import org.apache.cassandra.dht.Bounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.SSTableIdentityIterator;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+
+import static org.apache.cassandra.dht.AbstractBounds.isEmpty;
+import static org.apache.cassandra.dht.AbstractBounds.maxLeft;
+import static org.apache.cassandra.dht.AbstractBounds.minRight;
+
+public class BigTableScanner implements ISSTableScanner
+{
+    private AtomicBoolean isClosed = new AtomicBoolean(false);
+    protected final RandomAccessReader dfile;
+    protected final RandomAccessReader ifile;
+    public final SSTableReader sstable;
+
+    private final Iterator<AbstractBounds<RowPosition>> rangeIterator;
+    private AbstractBounds<RowPosition> currentRange;
+
+    private final DataRange dataRange;
+    private final RowIndexEntry.IndexSerializer rowIndexEntrySerializer;
+
+    protected Iterator<OnDiskAtomIterator> iterator;
+
+    public static ISSTableScanner getScanner(SSTableReader sstable, DataRange dataRange, RateLimiter limiter)
+    {
+        return new BigTableScanner(sstable, dataRange, limiter);
+    }
+    public static ISSTableScanner getScanner(SSTableReader sstable, Collection<Range<Token>> tokenRanges, RateLimiter limiter)
+    {
+        // We want to avoid allocating a SSTableScanner if the range don't overlap the sstable (#5249)
+        List<Pair<Long, Long>> positions = sstable.getPositionsForRanges(tokenRanges);
+        if (positions.isEmpty())
+            return new EmptySSTableScanner(sstable.getFilename());
+
+        return new BigTableScanner(sstable, tokenRanges, limiter);
+    }
+
+    /**
+     * @param sstable SSTable to scan; must not be null
+     * @param dataRange a single range to scan; must not be null
+     * @param limiter background i/o RateLimiter; may be null
+     */
+    private BigTableScanner(SSTableReader sstable, DataRange dataRange, RateLimiter limiter)
+    {
+        assert sstable != null;
+
+        this.dfile = limiter == null ? sstable.openDataReader() : sstable.openDataReader(limiter);
+        this.ifile = sstable.openIndexReader();
+        this.sstable = sstable;
+        this.dataRange = dataRange;
+        this.rowIndexEntrySerializer = sstable.descriptor.version.getSSTableFormat().getIndexSerializer(sstable.metadata);
+
+        List<AbstractBounds<RowPosition>> boundsList = new ArrayList<>(2);
+        addRange(dataRange.keyRange(), boundsList);
+        this.rangeIterator = boundsList.iterator();
+    }
+
+    /**
+     * @param sstable SSTable to scan; must not be null
+     * @param tokenRanges A set of token ranges to scan
+     * @param limiter background i/o RateLimiter; may be null
+     */
+    private BigTableScanner(SSTableReader sstable, Collection<Range<Token>> tokenRanges, RateLimiter limiter)
+    {
+        assert sstable != null;
+
+        this.dfile = limiter == null ? sstable.openDataReader() : sstable.openDataReader(limiter);
+        this.ifile = sstable.openIndexReader();
+        this.sstable = sstable;
+        this.dataRange = null;
+        this.rowIndexEntrySerializer = sstable.descriptor.version.getSSTableFormat().getIndexSerializer(sstable.metadata);
+
+        List<AbstractBounds<RowPosition>> boundsList = new ArrayList<>(tokenRanges.size());
+        for (Range<Token> range : Range.normalize(tokenRanges))
+            addRange(Range.makeRowRange(range), boundsList);
+
+        this.rangeIterator = boundsList.iterator();
+    }
+
+    private void addRange(AbstractBounds<RowPosition> requested, List<AbstractBounds<RowPosition>> boundsList)
+    {
+        if (requested instanceof Range && ((Range)requested).isWrapAround())
+        {
+            if (requested.right.compareTo(sstable.first) >= 0)
+            {
+                // since we wrap, we must contain the whole sstable prior to stopKey()
+                Boundary<RowPosition> left = new Boundary<RowPosition>(sstable.first, true);
+                Boundary<RowPosition> right;
+                right = requested.rightBoundary();
+                right = minRight(right, sstable.last, true);
+                if (!isEmpty(left, right))
+                    boundsList.add(AbstractBounds.bounds(left, right));
+            }
+            if (requested.left.compareTo(sstable.last) <= 0)
+            {
+                // since we wrap, we must contain the whole sstable after dataRange.startKey()
+                Boundary<RowPosition> right = new Boundary<RowPosition>(sstable.last, true);
+                Boundary<RowPosition> left;
+                left = requested.leftBoundary();
+                left = maxLeft(left, sstable.first, true);
+                if (!isEmpty(left, right))
+                    boundsList.add(AbstractBounds.bounds(left, right));
+            }
+        }
+        else
+        {
+            assert requested.left.compareTo(requested.right) <= 0 || requested.right.isMinimum();
+            Boundary<RowPosition> left, right;
+            left = requested.leftBoundary();
+            right = requested.rightBoundary();
+            left = maxLeft(left, sstable.first, true);
+            // apparently isWrapAround() doesn't count Bounds that extend to the limit (min) as wrapping
+            right = requested.right.isMinimum() ? new Boundary<RowPosition>(sstable.last, true)
+                                                    : minRight(right, sstable.last, true);
+            if (!isEmpty(left, right))
+                boundsList.add(AbstractBounds.bounds(left, right));
+        }
+    }
+
+    private void seekToCurrentRangeStart()
+    {
+        long indexPosition = sstable.getIndexScanPosition(currentRange.left);
+        ifile.seek(indexPosition);
+        try
+        {
+
+            while (!ifile.isEOF())
+            {
+                indexPosition = ifile.getFilePointer();
+                DecoratedKey indexDecoratedKey = sstable.partitioner.decorateKey(ByteBufferUtil.readWithShortLength(ifile));
+                if (indexDecoratedKey.compareTo(currentRange.left) > 0 || currentRange.contains(indexDecoratedKey))
+                {
+                    // Found, just read the dataPosition and seek into index and data files
+                    long dataPosition = ifile.readLong();
+                    ifile.seek(indexPosition);
+                    dfile.seek(dataPosition);
+                    break;
+                }
+                else
+                {
+                    RowIndexEntry.Serializer.skip(ifile);
+                }
+            }
+        }
+        catch (IOException e)
+        {
+            sstable.markSuspect();
+            throw new CorruptSSTableException(e, sstable.getFilename());
+        }
+    }
+
+    public void close() throws IOException
+    {
+        if (isClosed.compareAndSet(false, true))
+            FileUtils.close(dfile, ifile);
+    }
+
+    public long getLengthInBytes()
+    {
+        return dfile.length();
+    }
+
+    public long getCurrentPosition()
+    {
+        return dfile.getFilePointer();
+    }
+
+    public String getBackingFiles()
+    {
+        return sstable.toString();
+    }
+
+    public boolean hasNext()
+    {
+        if (iterator == null)
+            iterator = createIterator();
+        return iterator.hasNext();
+    }
+
+    public OnDiskAtomIterator next()
+    {
+        if (iterator == null)
+            iterator = createIterator();
+        return iterator.next();
+    }
+
+    public void remove()
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    private Iterator<OnDiskAtomIterator> createIterator()
+    {
+        return new KeyScanningIterator();
+    }
+
+    protected class KeyScanningIterator extends AbstractIterator<OnDiskAtomIterator>
+    {
+        private DecoratedKey nextKey;
+        private RowIndexEntry nextEntry;
+        private DecoratedKey currentKey;
+        private RowIndexEntry currentEntry;
+
+        protected OnDiskAtomIterator computeNext()
+        {
+            try
+            {
+                if (nextEntry == null)
+                {
+                    do
+                    {
+                        // we're starting the first range or we just passed the end of the previous range
+                        if (!rangeIterator.hasNext())
+                            return endOfData();
+
+                        currentRange = rangeIterator.next();
+                        seekToCurrentRangeStart();
+
+                        if (ifile.isEOF())
+                            return endOfData();
+
+                        currentKey = sstable.partitioner.decorateKey(ByteBufferUtil.readWithShortLength(ifile));
+                        currentEntry = rowIndexEntrySerializer.deserialize(ifile, sstable.descriptor.version);
+                    } while (!currentRange.contains(currentKey));
+                }
+                else
+                {
+                    // we're in the middle of a range
+                    currentKey = nextKey;
+                    currentEntry = nextEntry;
+                }
+
+                if (ifile.isEOF())
+                {
+                    nextEntry = null;
+                    nextKey = null;
+                }
+                else
+                {
+                    // we need the position of the start of the next key, regardless of whether it falls in the current range
+                    nextKey = sstable.partitioner.decorateKey(ByteBufferUtil.readWithShortLength(ifile));
+                    nextEntry = rowIndexEntrySerializer.deserialize(ifile, sstable.descriptor.version);
+
+                    if (!currentRange.contains(nextKey))
+                    {
+                        nextKey = null;
+                        nextEntry = null;
+                    }
+                }
+
+                if (dataRange == null || dataRange.selectsFullRowFor(currentKey.getKey()))
+                {
+                    dfile.seek(currentEntry.position + currentEntry.headerOffset());
+                    ByteBufferUtil.readWithShortLength(dfile); // key
+                    return new SSTableIdentityIterator(sstable, dfile, currentKey);
+                }
+
+                return new LazyColumnIterator(currentKey, new IColumnIteratorFactory()
+                {
+                    public OnDiskAtomIterator create()
+                    {
+                        return dataRange.columnFilter(currentKey.getKey()).getSSTableColumnIterator(sstable, dfile, currentKey, currentEntry);
+                    }
+                });
+
+            }
+            catch (CorruptSSTableException | IOException e)
+            {
+                sstable.markSuspect();
+                throw new CorruptSSTableException(e, sstable.getFilename());
+            }
+        }
+    }
+
+    @Override
+    public String toString()
+    {
+        return getClass().getSimpleName() + "(" +
+               "dfile=" + dfile +
+               " ifile=" + ifile +
+               " sstable=" + sstable +
+               ")";
+    }
+
+    public static class EmptySSTableScanner implements ISSTableScanner
+    {
+        private final String filename;
+
+        public EmptySSTableScanner(String filename)
+        {
+            this.filename = filename;
+        }
+
+        public long getLengthInBytes()
+        {
+            return 0;
+        }
+
+        public long getCurrentPosition()
+        {
+            return 0;
+        }
+
+        public String getBackingFiles()
+        {
+            return filename;
+        }
+
+        public boolean hasNext()
+        {
+            return false;
+        }
+
+        public OnDiskAtomIterator next()
+        {
+            return null;
+        }
+
+        public void close() throws IOException { }
+
+        public void remove() { }
+    }
+
+
+}
diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
new file mode 100644
index 0000000..3a01f87
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
@@ -0,0 +1,588 @@
+/*
+ * 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.io.sstable.format.big;
+
+import java.io.*;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.io.sstable.*;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.compaction.AbstractCompactedRow;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.io.compress.CompressedSequentialWriter;
+import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
+import org.apache.cassandra.io.sstable.metadata.MetadataComponent;
+import org.apache.cassandra.io.sstable.metadata.MetadataType;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.io.util.*;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.FilterFactory;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.StreamingHistogram;
+import org.apache.cassandra.utils.concurrent.Transactional;
+
+import static org.apache.cassandra.utils.Throwables.merge;
+import org.apache.cassandra.utils.SyncUtil;
+
+public class BigTableWriter extends SSTableWriter
+{
+    private static final Logger logger = LoggerFactory.getLogger(BigTableWriter.class);
+
+    // not very random, but the only value that can't be mistaken for a legal column-name length
+    public static final int END_OF_ROW = 0x0000;
+
+    private final IndexWriter iwriter;
+    private SegmentedFile.Builder dbuilder;
+    private final SequentialWriter dataFile;
+    private DecoratedKey lastWrittenKey;
+    private FileMark dataMark;
+
+    BigTableWriter(Descriptor descriptor, Long keyCount, Long repairedAt, CFMetaData metadata, IPartitioner partitioner, MetadataCollector metadataCollector)
+    {
+        super(descriptor, keyCount, repairedAt, metadata, partitioner, metadataCollector);
+
+        if (compression)
+        {
+            dataFile = SequentialWriter.open(getFilename(),
+                                             descriptor.filenameFor(Component.COMPRESSION_INFO),
+                                             metadata.compressionParameters(),
+                                             metadataCollector);
+            dbuilder = SegmentedFile.getCompressedBuilder((CompressedSequentialWriter) dataFile);
+        }
+        else
+        {
+            dataFile = SequentialWriter.open(new File(getFilename()), new File(descriptor.filenameFor(Component.CRC)));
+            dbuilder = SegmentedFile.getBuilder(DatabaseDescriptor.getDiskAccessMode(), false);
+        }
+        iwriter = new IndexWriter(keyCount, dataFile);
+    }
+
+    public void mark()
+    {
+        dataMark = dataFile.mark();
+        iwriter.mark();
+    }
+
+    public void resetAndTruncate()
+    {
+        dataFile.resetAndTruncate(dataMark);
+        iwriter.resetAndTruncate();
+    }
+
+    /**
+     * Perform sanity checks on @param decoratedKey and @return the position in the data file before any data is written
+     */
+    private long beforeAppend(DecoratedKey decoratedKey)
+    {
+        assert decoratedKey != null : "Keys must not be null"; // empty keys ARE allowed b/c of indexed column values
+        if (lastWrittenKey != null && lastWrittenKey.compareTo(decoratedKey) >= 0)
+            throw new RuntimeException("Last written key " + lastWrittenKey + " >= current key " + decoratedKey + " writing into " + getFilename());
+        return (lastWrittenKey == null) ? 0 : dataFile.getFilePointer();
+    }
+
+    private void afterAppend(DecoratedKey decoratedKey, long dataEnd, RowIndexEntry index) throws IOException
+    {
+        metadataCollector.addKey(decoratedKey.getKey());
+        lastWrittenKey = decoratedKey;
+        last = lastWrittenKey;
+        if (first == null)
+            first = lastWrittenKey;
+
+        if (logger.isTraceEnabled())
+            logger.trace("wrote {} at {}", decoratedKey, dataEnd);
+        iwriter.append(decoratedKey, index, dataEnd);
+        dbuilder.addPotentialBoundary(dataEnd);
+    }
+
+    /**
+     * @param row
+     * @return null if the row was compacted away entirely; otherwise, the PK index entry for this row
+     */
+    public RowIndexEntry append(AbstractCompactedRow row)
+    {
+        long startPosition = beforeAppend(row.key);
+        RowIndexEntry entry;
+        try
+        {
+            entry = row.write(startPosition, dataFile);
+            if (entry == null)
+                return null;
+            long endPosition = dataFile.getFilePointer();
+            long rowSize = endPosition - startPosition;
+            maybeLogLargePartitionWarning(row.key, rowSize);
+            metadataCollector.update(rowSize, row.columnStats());
+            afterAppend(row.key, endPosition, entry);
+            return entry;
+        }
+        catch (IOException e)
+        {
+            throw new FSWriteError(e, dataFile.getPath());
+        }
+    }
+
+    public void append(DecoratedKey decoratedKey, ColumnFamily cf)
+    {
+        if (decoratedKey.getKey().remaining() > FBUtilities.MAX_UNSIGNED_SHORT)
+        {
+            logger.error("Key size {} exceeds maximum of {}, skipping row",
+                         decoratedKey.getKey().remaining(),
+                         FBUtilities.MAX_UNSIGNED_SHORT);
+            return;
+        }
+
+        long startPosition = beforeAppend(decoratedKey);
+        long endPosition;
+        try
+        {
+            RowIndexEntry entry = rawAppend(cf, startPosition, decoratedKey, dataFile.stream);
+            endPosition = dataFile.getFilePointer();
+            afterAppend(decoratedKey, endPosition, entry);
+        }
+        catch (IOException e)
+        {
+            throw new FSWriteError(e, dataFile.getPath());
+        }
+        long rowSize = endPosition - startPosition;
+        maybeLogLargePartitionWarning(decoratedKey, rowSize);
+        metadataCollector.update(endPosition - startPosition, cf.getColumnStats());
+    }
+
+    private void maybeLogLargePartitionWarning(DecoratedKey key, long rowSize)
+    {
+        if (rowSize > DatabaseDescriptor.getCompactionLargePartitionWarningThreshold())
+        {
+            String keyString = metadata.getKeyValidator().getString(key.getKey());
+            logger.warn("Writing large partition {}/{}:{} ({} bytes)", metadata.ksName, metadata.cfName, keyString, rowSize);
+        }
+    }
+
+    private static RowIndexEntry rawAppend(ColumnFamily cf, long startPosition, DecoratedKey key, DataOutputPlus out) throws IOException
+    {
+        assert cf.hasColumns() || cf.isMarkedForDelete();
+
+        ColumnIndex.Builder builder = new ColumnIndex.Builder(cf, key.getKey(), out);
+        ColumnIndex index = builder.build(cf);
+
+        out.writeShort(END_OF_ROW);
+        return RowIndexEntry.create(startPosition, cf.deletionInfo().getTopLevelDeletion(), index);
+    }
+
+    /**
+     * @throws IOException if a read from the DataInput fails
+     * @throws FSWriteError if a write to the dataFile fails
+     */
+    public long appendFromStream(DecoratedKey key, CFMetaData metadata, DataInput in, Version version) throws IOException
+    {
+        long currentPosition = beforeAppend(key);
+
+        ColumnStats.MaxLongTracker maxTimestampTracker = new ColumnStats.MaxLongTracker(Long.MAX_VALUE);
+        ColumnStats.MinLongTracker minTimestampTracker = new ColumnStats.MinLongTracker(Long.MIN_VALUE);
+        ColumnStats.MaxIntTracker maxDeletionTimeTracker = new ColumnStats.MaxIntTracker(Integer.MAX_VALUE);
+        List<ByteBuffer> minColumnNames = Collections.emptyList();
+        List<ByteBuffer> maxColumnNames = Collections.emptyList();
+        StreamingHistogram tombstones = new StreamingHistogram(TOMBSTONE_HISTOGRAM_BIN_SIZE);
+        boolean hasLegacyCounterShards = false;
+
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(metadata);
+        cf.delete(DeletionTime.serializer.deserialize(in));
+
+        ColumnIndex.Builder columnIndexer = new ColumnIndex.Builder(cf, key.getKey(), dataFile.stream);
+
+        if (cf.deletionInfo().getTopLevelDeletion().localDeletionTime < Integer.MAX_VALUE)
+        {
+            tombstones.update(cf.deletionInfo().getTopLevelDeletion().localDeletionTime);
+            maxDeletionTimeTracker.update(cf.deletionInfo().getTopLevelDeletion().localDeletionTime);
+            minTimestampTracker.update(cf.deletionInfo().getTopLevelDeletion().markedForDeleteAt);
+            maxTimestampTracker.update(cf.deletionInfo().getTopLevelDeletion().markedForDeleteAt);
+        }
+
+        Iterator<RangeTombstone> rangeTombstoneIterator = cf.deletionInfo().rangeIterator();
+        while (rangeTombstoneIterator.hasNext())
+        {
+            RangeTombstone rangeTombstone = rangeTombstoneIterator.next();
+            tombstones.update(rangeTombstone.getLocalDeletionTime());
+            minTimestampTracker.update(rangeTombstone.timestamp());
+            maxTimestampTracker.update(rangeTombstone.timestamp());
+            maxDeletionTimeTracker.update(rangeTombstone.getLocalDeletionTime());
+            minColumnNames = ColumnNameHelper.minComponents(minColumnNames, rangeTombstone.min, metadata.comparator);
+            maxColumnNames = ColumnNameHelper.maxComponents(maxColumnNames, rangeTombstone.max, metadata.comparator);
+        }
+
+        Iterator<OnDiskAtom> iter = AbstractCell.onDiskIterator(in, ColumnSerializer.Flag.PRESERVE_SIZE, Integer.MIN_VALUE, version, metadata.comparator);
+        try
+        {
+            while (iter.hasNext())
+            {
+                OnDiskAtom atom = iter.next();
+                if (atom == null)
+                    break;
+
+                if (atom instanceof CounterCell)
+                {
+                    atom = ((CounterCell) atom).markLocalToBeCleared();
+                    hasLegacyCounterShards = hasLegacyCounterShards || ((CounterCell) atom).hasLegacyShards();
+                }
+
+                int deletionTime = atom.getLocalDeletionTime();
+                if (deletionTime < Integer.MAX_VALUE)
+                    tombstones.update(deletionTime);
+                minTimestampTracker.update(atom.timestamp());
+                maxTimestampTracker.update(atom.timestamp());
+                minColumnNames = ColumnNameHelper.minComponents(minColumnNames, atom.name(), metadata.comparator);
+                maxColumnNames = ColumnNameHelper.maxComponents(maxColumnNames, atom.name(), metadata.comparator);
+                maxDeletionTimeTracker.update(atom.getLocalDeletionTime());
+
+                columnIndexer.add(atom); // This write the atom on disk too
+            }
+            columnIndexer.finishAddingAtoms();
+
+            columnIndexer.maybeWriteEmptyRowHeader();
+            dataFile.stream.writeShort(END_OF_ROW);
+        }
+        catch (IOException e)
+        {
+            throw new FSWriteError(e, dataFile.getPath());
+        }
+
+        metadataCollector.updateMinTimestamp(minTimestampTracker.get())
+                         .updateMaxTimestamp(maxTimestampTracker.get())
+                         .updateMaxLocalDeletionTime(maxDeletionTimeTracker.get())
+                         .addRowSize(dataFile.getFilePointer() - currentPosition)
+                         .addColumnCount(columnIndexer.writtenAtomCount())
+                         .mergeTombstoneHistogram(tombstones)
+                         .updateMinColumnNames(minColumnNames)
+                         .updateMaxColumnNames(maxColumnNames)
+                         .updateHasLegacyCounterShards(hasLegacyCounterShards);
+
+        afterAppend(key, currentPosition, RowIndexEntry.create(currentPosition, cf.deletionInfo().getTopLevelDeletion(), columnIndexer.build()));
+        return currentPosition;
+    }
+
+    private Descriptor makeTmpLinks()
+    {
+        // create temp links if they don't already exist
+        Descriptor link = descriptor.asType(Descriptor.Type.TEMPLINK);
+        if (!new File(link.filenameFor(Component.PRIMARY_INDEX)).exists())
+        {
+            FileUtils.createHardLink(new File(descriptor.filenameFor(Component.PRIMARY_INDEX)), new File(link.filenameFor(Component.PRIMARY_INDEX)));
+            FileUtils.createHardLink(new File(descriptor.filenameFor(Component.DATA)), new File(link.filenameFor(Component.DATA)));
+        }
+        return link;
+    }
+
+    @SuppressWarnings("resource")
+    public SSTableReader openEarly()
+    {
+        // find the max (exclusive) readable key
+        IndexSummaryBuilder.ReadableBoundary boundary = iwriter.getMaxReadable();
+        if (boundary == null)
+            return null;
+
+        StatsMetadata stats = statsMetadata();
+        assert boundary.indexLength > 0 && boundary.dataLength > 0;
+        Descriptor link = makeTmpLinks();
+        // open the reader early, giving it a FINAL descriptor type so that it is indistinguishable for other consumers
+        SegmentedFile ifile = iwriter.builder.complete(link.filenameFor(Component.PRIMARY_INDEX), boundary.indexLength);
+        SegmentedFile dfile = dbuilder.complete(link.filenameFor(Component.DATA), boundary.dataLength);
+        SSTableReader sstable = SSTableReader.internalOpen(descriptor.asType(Descriptor.Type.FINAL),
+                                                           components, metadata,
+                                                           partitioner, ifile,
+                                                           dfile, iwriter.summary.build(partitioner, boundary),
+                                                           iwriter.bf.sharedCopy(), maxDataAge, stats, SSTableReader.OpenReason.EARLY);
+
+        // now it's open, find the ACTUAL last readable key (i.e. for which the data file has also been flushed)
+        sstable.first = getMinimalKey(first);
+        sstable.last = getMinimalKey(boundary.lastKey);
+        return sstable;
+    }
+
+    public SSTableReader openFinalEarly()
+    {
+        // we must ensure the data is completely flushed to disk
+        dataFile.sync();
+        iwriter.indexFile.sync();
+        return openFinal(makeTmpLinks(), SSTableReader.OpenReason.EARLY);
+    }
+
+    @SuppressWarnings("resource")
+    private SSTableReader openFinal(Descriptor desc, SSTableReader.OpenReason openReason)
+    {
+        if (maxDataAge < 0)
+            maxDataAge = System.currentTimeMillis();
+
+        StatsMetadata stats = statsMetadata();
+        // finalize in-memory state for the reader
+        SegmentedFile ifile = iwriter.builder.complete(desc.filenameFor(Component.PRIMARY_INDEX));
+        SegmentedFile dfile = dbuilder.complete(desc.filenameFor(Component.DATA));
+        SSTableReader sstable = SSTableReader.internalOpen(desc.asType(Descriptor.Type.FINAL),
+                                                           components,
+                                                           this.metadata,
+                                                           partitioner,
+                                                           ifile,
+                                                           dfile,
+                                                           iwriter.summary.build(partitioner),
+                                                           iwriter.bf.sharedCopy(),
+                                                           maxDataAge,
+                                                           stats,
+                                                           openReason);
+        sstable.first = getMinimalKey(first);
+        sstable.last = getMinimalKey(last);
+        return sstable;
+    }
+
+    protected SSTableWriter.TransactionalProxy txnProxy()
+    {
+        return new TransactionalProxy();
+    }
+
+    class TransactionalProxy extends SSTableWriter.TransactionalProxy
+    {
+        // finalise our state on disk, including renaming
+        protected void doPrepare()
+        {
+            iwriter.prepareToCommit();
+
+            // write sstable statistics
+            dataFile.setDescriptor(descriptor).prepareToCommit();
+            writeMetadata(descriptor, finalizeMetadata());
+
+            // save the table of components
+            SSTable.appendTOC(descriptor, components);
+
+            // rename to final
+            rename(descriptor, components);
+
+            if (openResult)
+                finalReader = openFinal(descriptor.asType(Descriptor.Type.FINAL), SSTableReader.OpenReason.NORMAL);
+        }
+
+        protected Throwable doCommit(Throwable accumulate)
+        {
+            accumulate = dataFile.commit(accumulate);
+            accumulate = iwriter.commit(accumulate);
+            return accumulate;
+        }
+
+        @Override
+        protected Throwable doPreCleanup(Throwable accumulate)
+        {
+            accumulate = dbuilder.close(accumulate);
+            return accumulate;
+        }
+
+        protected Throwable doAbort(Throwable accumulate)
+        {
+            accumulate = iwriter.abort(accumulate);
+            accumulate = dataFile.abort(accumulate);
+
+            accumulate = delete(descriptor, accumulate);
+            if (!openResult)
+                accumulate = delete(descriptor.asType(Descriptor.Type.FINAL), accumulate);
+            return accumulate;
+        }
+
+        private Throwable delete(Descriptor desc, Throwable accumulate)
+        {
+            try
+            {
+                Set<Component> components = SSTable.discoverComponentsFor(desc);
+                if (!components.isEmpty())
+                    SSTable.delete(desc, components);
+            }
+            catch (Throwable t)
+            {
+                logger.error(String.format("Failed deleting temp components for %s", descriptor), t);
+                accumulate = merge(accumulate, t);
+            }
+            return accumulate;
+        }
+    }
+
+    private static void writeMetadata(Descriptor desc, Map<MetadataType, MetadataComponent> components)
+    {
+        File file = new File(desc.filenameFor(Component.STATS));
+        try (SequentialWriter out = SequentialWriter.open(file);)
+        {
+            desc.getMetadataSerializer().serialize(components, desc.version, out.stream);
+            out.setDescriptor(desc).finish();
+        }
+        catch (IOException e)
+        {
+            throw new FSWriteError(e, file.getPath());
+        }
+    }
+
+    public long getFilePointer()
+    {
+        return dataFile.getFilePointer();
+    }
+
+    public long getOnDiskFilePointer()
+    {
+        return dataFile.getOnDiskFilePointer();
+    }
+
+    /**
+     * Encapsulates writing the index and filter for an SSTable. The state of this object is not valid until it has been closed.
+     */
+    class IndexWriter extends AbstractTransactional implements Transactional
+    {
+        private final SequentialWriter indexFile;
+        public final SegmentedFile.Builder builder;
+        public final IndexSummaryBuilder summary;
+        public final IFilter bf;
+        private FileMark mark;
+
+        IndexWriter(long keyCount, final SequentialWriter dataFile)
+        {
+            indexFile = SequentialWriter.open(new File(descriptor.filenameFor(Component.PRIMARY_INDEX)));
+            builder = SegmentedFile.getBuilder(DatabaseDescriptor.getIndexAccessMode(), false);
+            summary = new IndexSummaryBuilder(keyCount, metadata.getMinIndexInterval(), Downsampling.BASE_SAMPLING_LEVEL);
+            bf = FilterFactory.getFilter(keyCount, metadata.getBloomFilterFpChance(), true);
+            // register listeners to be alerted when the data files are flushed
+            indexFile.setPostFlushListener(new Runnable()
+            {
+                public void run()
+                {
+                    summary.markIndexSynced(indexFile.getLastFlushOffset());
+                }
+            });
+            dataFile.setPostFlushListener(new Runnable()
+            {
+                public void run()
+                {
+                    summary.markDataSynced(dataFile.getLastFlushOffset());
+                }
+            });
+        }
+
+        // finds the last (-offset) decorated key that can be guaranteed to occur fully in the flushed portion of the index file
+        IndexSummaryBuilder.ReadableBoundary getMaxReadable()
+        {
+            return summary.getLastReadableBoundary();
+        }
+
+        public void append(DecoratedKey key, RowIndexEntry indexEntry, long dataEnd) throws IOException
+        {
+            bf.add(key);
+            long indexStart = indexFile.getFilePointer();
+            try
+            {
+                ByteBufferUtil.writeWithShortLength(key.getKey(), indexFile.stream);
+                rowIndexEntrySerializer.serialize(indexEntry, indexFile.stream);
+            }
+            catch (IOException e)
+            {
+                throw new FSWriteError(e, indexFile.getPath());
+            }
+            long indexEnd = indexFile.getFilePointer();
+
+            if (logger.isTraceEnabled())
+                logger.trace("wrote index entry: {} at {}", indexEntry, indexStart);
+
+            summary.maybeAddEntry(key, indexStart, indexEnd, dataEnd);
+            builder.addPotentialBoundary(indexStart);
+        }
+
+        /**
+         * Closes the index and bloomfilter, making the public state of this writer valid for consumption.
+         */
+        void flushBf()
+        {
+            if (components.contains(Component.FILTER))
+            {
+                String path = descriptor.filenameFor(Component.FILTER);
+                try (FileOutputStream fos = new FileOutputStream(path);
+                     DataOutputStreamPlus stream = new BufferedDataOutputStreamPlus(fos))
+                {
+                    // bloom filter
+                    FilterFactory.serialize(bf, stream);
+                    stream.flush();
+                    SyncUtil.sync(fos);
+                }
+                catch (IOException e)
+                {
+                    throw new FSWriteError(e, path);
+                }
+            }
+        }
+
+        public void mark()
+        {
+            mark = indexFile.mark();
+        }
+
+        public void resetAndTruncate()
+        {
+            // we can't un-set the bloom filter addition, but extra keys in there are harmless.
+            // we can't reset dbuilder either, but that is the last thing called in afterappend so
+            // we assume that if that worked then we won't be trying to reset.
+            indexFile.resetAndTruncate(mark);
+        }
+
+        protected void doPrepare()
+        {
+            flushBf();
+
+            // truncate index file
+            long position = iwriter.indexFile.getFilePointer();
+            iwriter.indexFile.setDescriptor(descriptor).prepareToCommit();
+            FileUtils.truncate(iwriter.indexFile.getPath(), position);
+
+            // save summary
+            summary.prepareToCommit();
+            try (IndexSummary summary = iwriter.summary.build(partitioner))
+            {
+                SSTableReader.saveSummary(descriptor, first, last, iwriter.builder, dbuilder, summary);
+            }
+        }
+
+        protected Throwable doCommit(Throwable accumulate)
+        {
+            return indexFile.commit(accumulate);
+        }
+
+        protected Throwable doAbort(Throwable accumulate)
+        {
+            return indexFile.abort(accumulate);
+        }
+
+        @Override
+        protected Throwable doPreCleanup(Throwable accumulate)
+        {
+            accumulate = summary.close(accumulate);
+            accumulate = bf.close(accumulate);
+            accumulate = builder.close(accumulate);
+            return accumulate;
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/IndexedSliceReader.java b/src/java/org/apache/cassandra/io/sstable/format/big/IndexedSliceReader.java
new file mode 100644
index 0000000..6db9c3d
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/sstable/format/big/IndexedSliceReader.java
@@ -0,0 +1,542 @@
+/*
+ * 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.io.sstable.format.big;
+
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.Deque;
+import java.util.List;
+
+import com.google.common.collect.AbstractIterator;
+
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
+import org.apache.cassandra.db.composites.CellNameType;
+import org.apache.cassandra.db.composites.Composite;
+import org.apache.cassandra.db.filter.ColumnSlice;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.IndexHelper;
+import org.apache.cassandra.io.sstable.IndexHelper.IndexInfo;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileMark;
+import org.apache.cassandra.tracing.Tracing;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+/**
+ * This is a reader that finds the block for a starting column and returns blocks before/after it for each next call.
+ * This function assumes that the CF is sorted by name and exploits the name index.
+ */
+class IndexedSliceReader extends AbstractIterator<OnDiskAtom> implements OnDiskAtomIterator
+{
+    private final ColumnFamily emptyColumnFamily;
+
+    private final SSTableReader sstable;
+    private final List<IndexHelper.IndexInfo> indexes;
+    private final FileDataInput originalInput;
+    private FileDataInput file;
+    private final boolean reversed;
+    private final ColumnSlice[] slices;
+    private final BlockFetcher fetcher;
+    private final Deque<OnDiskAtom> blockColumns = new ArrayDeque<OnDiskAtom>();
+    private final CellNameType comparator;
+
+    // Holds range tombstone in reverse queries. See addColumn()
+    private final Deque<OnDiskAtom> rangeTombstonesReversed;
+
+    /**
+     * This slice reader assumes that slices are sorted correctly, e.g. that for forward lookup slices are in
+     * lexicographic order of start elements and that for reverse lookup they are in reverse lexicographic order of
+     * finish (reverse start) elements. i.e. forward: [a,b],[d,e],[g,h] reverse: [h,g],[e,d],[b,a]. This reader also
+     * assumes that validation has been performed in terms of intervals (no overlapping intervals).
+     */
+    IndexedSliceReader(SSTableReader sstable, RowIndexEntry indexEntry, FileDataInput input, ColumnSlice[] slices, boolean reversed)
+    {
+        Tracing.trace("Seeking to partition indexed section in data file");
+        this.sstable = sstable;
+        this.originalInput = input;
+        this.reversed = reversed;
+        this.slices = slices;
+        this.comparator = sstable.metadata.comparator;
+        this.rangeTombstonesReversed = reversed ? new ArrayDeque<OnDiskAtom>() : null;
+
+        try
+        {
+            this.indexes = indexEntry.columnsIndex();
+            emptyColumnFamily = ArrayBackedSortedColumns.factory.create(sstable.metadata);
+            if (indexes.isEmpty())
+            {
+                setToRowStart(indexEntry, input);
+                emptyColumnFamily.delete(DeletionTime.serializer.deserialize(file));
+                fetcher = new SimpleBlockFetcher();
+            }
+            else
+            {
+                emptyColumnFamily.delete(indexEntry.deletionTime());
+                fetcher = new IndexedBlockFetcher(indexEntry.position);
+            }
+        }
+        catch (IOException e)
+        {
+            sstable.markSuspect();
+            throw new CorruptSSTableException(e, file.getPath());
+        }
+    }
+
+    /**
+     * Sets the seek position to the start of the row for column scanning.
+     */
+    private void setToRowStart(RowIndexEntry rowEntry, FileDataInput in) throws IOException
+    {
+        if (in == null)
+        {
+            this.file = sstable.getFileDataInput(rowEntry.position);
+        }
+        else
+        {
+            this.file = in;
+            in.seek(rowEntry.position);
+        }
+        sstable.partitioner.decorateKey(ByteBufferUtil.readWithShortLength(file));
+    }
+
+    public ColumnFamily getColumnFamily()
+    {
+        return emptyColumnFamily;
+    }
+
+    public DecoratedKey getKey()
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    protected OnDiskAtom computeNext()
+    {
+        while (true)
+        {
+            if (reversed)
+            {
+                // Return all tombstone for the block first (see addColumn() below)
+                OnDiskAtom column = rangeTombstonesReversed.poll();
+                if (column != null)
+                    return column;
+            }
+
+            OnDiskAtom column = blockColumns.poll();
+            if (column == null)
+            {
+                if (!fetcher.fetchMoreData())
+                    return endOfData();
+            }
+            else
+            {
+                return column;
+            }
+        }
+    }
+
+    public void close() throws IOException
+    {
+        if (originalInput == null && file != null)
+            file.close();
+    }
+
+    protected void addColumn(OnDiskAtom col)
+    {
+        if (reversed)
+        {
+            /*
+             * We put range tomstone markers at the beginning of the range they delete. But for reversed queries,
+             * the caller still need to know about a RangeTombstone before it sees any column that it covers.
+             * To make that simple, we keep said tombstones separate and return them all before any column for
+             * a given block.
+             */
+            if (col instanceof RangeTombstone)
+                rangeTombstonesReversed.addFirst(col);
+            else
+                blockColumns.addFirst(col);
+        }
+        else
+        {
+            blockColumns.addLast(col);
+        }
+    }
+
+    private abstract class BlockFetcher
+    {
+        protected int currentSliceIdx;
+
+        protected BlockFetcher(int sliceIdx)
+        {
+            this.currentSliceIdx = sliceIdx;
+        }
+
+        /*
+         * Return the smallest key selected by the current ColumnSlice.
+         */
+        protected Composite currentStart()
+        {
+            return reversed ? slices[currentSliceIdx].finish : slices[currentSliceIdx].start;
+        }
+
+        /*
+         * Return the biggest key selected by the current ColumnSlice.
+         */
+        protected Composite currentFinish()
+        {
+            return reversed ? slices[currentSliceIdx].start : slices[currentSliceIdx].finish;
+        }
+
+        protected abstract boolean setNextSlice();
+
+        protected abstract boolean fetchMoreData();
+
+        protected boolean isColumnBeforeSliceStart(OnDiskAtom column)
+        {
+            return isBeforeSliceStart(column.name());
+        }
+
+        protected boolean isBeforeSliceStart(Composite name)
+        {
+            Composite start = currentStart();
+            return !start.isEmpty() && comparator.compare(name, start) < 0;
+        }
+
+        protected boolean isColumnBeforeSliceFinish(OnDiskAtom column)
+        {
+            Composite finish = currentFinish();
+            return finish.isEmpty() || comparator.compare(column.name(), finish) <= 0;
+        }
+
+        protected boolean isAfterSliceFinish(Composite name)
+        {
+            Composite finish = currentFinish();
+            return !finish.isEmpty() && comparator.compare(name, finish) > 0;
+        }
+    }
+
+    private class IndexedBlockFetcher extends BlockFetcher
+    {
+        // where this row starts
+        private final long columnsStart;
+
+        // the index entry for the next block to deserialize
+        private int nextIndexIdx = -1;
+
+        // index of the last block we've read from disk;
+        private int lastDeserializedBlock = -1;
+
+        // For reversed, keep columns at the beginning of the last deserialized block that
+        // may still match a slice
+        private final Deque<OnDiskAtom> prefetched;
+
+        public IndexedBlockFetcher(long columnsStart)
+        {
+            super(-1);
+            this.columnsStart = columnsStart;
+            this.prefetched = reversed ? new ArrayDeque<OnDiskAtom>() : null;
+            setNextSlice();
+        }
+
+        protected boolean setNextSlice()
+        {
+            while (++currentSliceIdx < slices.length)
+            {
+                nextIndexIdx = IndexHelper.indexFor(slices[currentSliceIdx].start, indexes, comparator, reversed, nextIndexIdx);
+                if (nextIndexIdx < 0 || nextIndexIdx >= indexes.size())
+                    // no index block for that slice
+                    continue;
+
+                // Check if we can exclude this slice entirely from the index
+                IndexInfo info = indexes.get(nextIndexIdx);
+                if (reversed)
+                {
+                    if (!isBeforeSliceStart(info.lastName))
+                        return true;
+                }
+                else
+                {
+                    if (!isAfterSliceFinish(info.firstName))
+                        return true;
+                }
+            }
+            nextIndexIdx = -1;
+            return false;
+        }
+
+        protected boolean hasMoreSlice()
+        {
+            return currentSliceIdx < slices.length;
+        }
+
+        protected boolean fetchMoreData()
+        {
+            if (!hasMoreSlice())
+                return false;
+
+            // If we read blocks in reversed disk order, we may have columns from the previous block to handle.
+            // Note that prefetched keeps columns in reversed disk order.
+            // Also note that Range Tombstone handling is a bit tricky, because we may run into range tombstones
+            // that cover a slice *after* we've move to the previous slice. To keep it simple, we simply include
+            // every RT in prefetched: it's only slightly inefficient to do so and there is only so much RT that
+            // can be mistakenly added this way.
+            if (reversed && !prefetched.isEmpty())
+            {
+                // Avoids some comparison when we know it's not useful
+                boolean inSlice = false;
+
+                OnDiskAtom prefetchedCol;
+                while ((prefetchedCol = prefetched.peek()) != null)
+                {
+                    // col is before slice, we update the slice
+                    if (isColumnBeforeSliceStart(prefetchedCol))
+                    {
+                        inSlice = false;
+
+                        // As explained above, we add RT unconditionally
+                        if (prefetchedCol instanceof RangeTombstone)
+                        {
+                            blockColumns.addLast(prefetched.poll());
+                            continue;
+                        }
+
+                        // Otherwise, we either move to the next slice. If we have no more slice, then
+                        // simply unwind prefetched entirely and add all RT.
+                        if (!setNextSlice())
+                        {
+                            while ((prefetchedCol = prefetched.poll()) != null)
+                                if (prefetchedCol instanceof RangeTombstone)
+                                    blockColumns.addLast(prefetchedCol);
+                            break;
+                        }
+
+                    }
+                    // col is within slice, all columns
+                    // (we go in reverse, so as soon as we are in a slice, no need to check
+                    // we're after the slice until we change slice)
+                    else if (inSlice || isColumnBeforeSliceFinish(prefetchedCol))
+                    {
+                        blockColumns.addLast(prefetched.poll());
+                        inSlice = true;
+                    }
+                    // if col is after slice, ignore
+                    else
+                    {
+                        prefetched.poll();
+                    }
+                }
+
+                if (!blockColumns.isEmpty())
+                    return true;
+                else if (!hasMoreSlice())
+                    return false;
+            }
+            try
+            {
+                return getNextBlock();
+            }
+            catch (IOException e)
+            {
+                throw new CorruptSSTableException(e, file.getPath());
+            }
+        }
+
+        private boolean getNextBlock() throws IOException
+        {
+            if (lastDeserializedBlock == nextIndexIdx)
+            {
+                if (reversed)
+                    nextIndexIdx--;
+                else
+                    nextIndexIdx++;
+            }
+            lastDeserializedBlock = nextIndexIdx;
+
+            // Are we done?
+            if (lastDeserializedBlock < 0 || lastDeserializedBlock >= indexes.size())
+                return false;
+
+            IndexInfo currentIndex = indexes.get(lastDeserializedBlock);
+
+            /* seek to the correct offset to the data, and calculate the data size */
+            long positionToSeek = columnsStart + currentIndex.offset;
+
+            // With new promoted indexes, our first seek in the data file will happen at that point.
+            if (file == null)
+                file = originalInput == null ? sstable.getFileDataInput(positionToSeek) : originalInput;
+
+            AtomDeserializer deserializer = emptyColumnFamily.metadata().getOnDiskDeserializer(file, sstable.descriptor.version);
+
+            file.seek(positionToSeek);
+            FileMark mark = file.mark();
+
+            // We remenber when we are whithin a slice to avoid some comparison
+            boolean inSlice = false;
+
+            // scan from index start
+            while (file.bytesPastMark(mark) < currentIndex.width || deserializer.hasUnprocessed())
+            {
+                // col is before slice
+                // (If in slice, don't bother checking that until we change slice)
+                Composite start = currentStart();
+                if (!inSlice && !start.isEmpty() && deserializer.compareNextTo(start) < 0)
+                {
+                    // If it's a rangeTombstone, then we need to read it and include it unless it's end
+                    // stops before our slice start.
+                    if (deserializer.nextIsRangeTombstone())
+                    {
+                        RangeTombstone rt = (RangeTombstone)deserializer.readNext();
+                        if (comparator.compare(rt.max, start) >= 0)
+                            addColumn(rt);
+                        continue;
+                    }
+
+                    if (reversed)
+                    {
+                        // the next slice select columns that are before the current one, so it may
+                        // match this column, so keep it around.
+                        prefetched.addFirst(deserializer.readNext());
+                    }
+                    else
+                    {
+                        deserializer.skipNext();
+                    }
+                }
+                // col is within slice
+                else
+                {
+                    Composite finish = currentFinish();
+                    if (finish.isEmpty() || deserializer.compareNextTo(finish) <= 0)
+                    {
+                        inSlice = true;
+                        addColumn(deserializer.readNext());
+                    }
+                    // col is after slice.
+                    else
+                    {
+                        // When reading forward, if we hit a column that sorts after the current slice, it means we're done with this slice.
+                        // For reversed, this may either mean that we're done with the current slice, or that we need to read the previous
+                        // index block. However, we can be sure that we are in the first case though (the current slice is done) if the first
+                        // columns of the block were not part of the current slice, i.e. if we have columns in prefetched.
+                        if (reversed && prefetched.isEmpty())
+                            break;
+
+                        if (!setNextSlice())
+                            break;
+
+                        inSlice = false;
+
+                        // The next index block now corresponds to the first block that may have columns for the newly set slice.
+                        // So if it's different from the current block, we're done with this block. And in that case, we know
+                        // that our prefetched columns won't match.
+                        if (nextIndexIdx != lastDeserializedBlock)
+                        {
+                            if (reversed)
+                                prefetched.clear();
+                            break;
+                        }
+
+                        // Even if the next slice may have column in this blocks, if we're reversed, those columns have been
+                        // prefetched and we're done with that block
+                        if (reversed)
+                            break;
+
+                        // otherwise, we will deal with that column at the next iteration
+                    }
+                }
+            }
+            return true;
+        }
+    }
+
+    private class SimpleBlockFetcher extends BlockFetcher
+    {
+        public SimpleBlockFetcher() throws IOException
+        {
+            // Since we have to deserialize in order and will read all slices might as well reverse the slices and
+            // behave as if it was not reversed
+            super(reversed ? slices.length - 1 : 0);
+
+            // We remenber when we are whithin a slice to avoid some comparison
+            boolean inSlice = false;
+
+            AtomDeserializer deserializer = emptyColumnFamily.metadata().getOnDiskDeserializer(file, sstable.descriptor.version);
+            while (deserializer.hasNext())
+            {
+                // col is before slice
+                // (If in slice, don't bother checking that until we change slice)
+                Composite start = currentStart();
+                if (!inSlice && !start.isEmpty() && deserializer.compareNextTo(start) < 0)
+                {
+                    // If it's a rangeTombstone, then we need to read it and include it unless it's end
+                    // stops before our slice start. Otherwise, we can skip it.
+                    if (deserializer.nextIsRangeTombstone())
+                    {
+                        RangeTombstone rt = (RangeTombstone)deserializer.readNext();
+                        if (comparator.compare(rt.max, start) >= 0)
+                            addColumn(rt);
+                    }
+                    else
+                    {
+                        deserializer.skipNext();
+                    }
+                    continue;
+                }
+
+                // col is within slice
+                Composite finish = currentFinish();
+                if (finish.isEmpty() || deserializer.compareNextTo(finish) <= 0)
+                {
+                    inSlice = true;
+                    addColumn(deserializer.readNext());
+                }
+                // col is after slice. more slices?
+                else
+                {
+                    inSlice = false;
+                    if (!setNextSlice())
+                        break;
+                }
+            }
+        }
+
+        protected boolean setNextSlice()
+        {
+            if (reversed)
+            {
+                if (currentSliceIdx <= 0)
+                    return false;
+
+                currentSliceIdx--;
+            }
+            else
+            {
+                if (currentSliceIdx >= slices.length - 1)
+                    return false;
+
+                currentSliceIdx++;
+            }
+            return true;
+        }
+
+        protected boolean fetchMoreData()
+        {
+            return false;
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/SSTableNamesIterator.java b/src/java/org/apache/cassandra/io/sstable/format/big/SSTableNamesIterator.java
new file mode 100644
index 0000000..b8910c7
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/sstable/format/big/SSTableNamesIterator.java
@@ -0,0 +1,264 @@
+/*
+ * 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.io.sstable.format.big;
+
+import java.io.IOException;
+import java.util.*;
+
+import com.google.common.collect.AbstractIterator;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
+import org.apache.cassandra.db.composites.CellName;
+import org.apache.cassandra.db.composites.CellNameType;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.IndexHelper;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileMark;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+class SSTableNamesIterator extends AbstractIterator<OnDiskAtom> implements OnDiskAtomIterator
+{
+    private ColumnFamily cf;
+    private final SSTableReader sstable;
+    private FileDataInput fileToClose;
+    private Iterator<OnDiskAtom> iter;
+    public final SortedSet<CellName> columns;
+    public final DecoratedKey key;
+
+    public SSTableNamesIterator(SSTableReader sstable, DecoratedKey key, SortedSet<CellName> columns)
+    {
+        assert columns != null;
+        this.sstable = sstable;
+        this.columns = columns;
+        this.key = key;
+
+        RowIndexEntry indexEntry = sstable.getPosition(key, SSTableReader.Operator.EQ);
+        if (indexEntry == null)
+            return;
+
+        try
+        {
+            read(sstable, null, indexEntry);
+        }
+        catch (IOException e)
+        {
+            sstable.markSuspect();
+            throw new CorruptSSTableException(e, sstable.getFilename());
+        }
+        finally
+        {
+            if (fileToClose != null)
+                FileUtils.closeQuietly(fileToClose);
+        }
+    }
+
+    public SSTableNamesIterator(SSTableReader sstable, FileDataInput file, DecoratedKey key, SortedSet<CellName> columns, RowIndexEntry indexEntry)
+    {
+        assert columns != null;
+        this.sstable = sstable;
+        this.columns = columns;
+        this.key = key;
+
+        try
+        {
+            read(sstable, file, indexEntry);
+        }
+        catch (IOException e)
+        {
+            sstable.markSuspect();
+            throw new CorruptSSTableException(e, sstable.getFilename());
+        }
+    }
+
+    private FileDataInput createFileDataInput(long position)
+    {
+        fileToClose = sstable.getFileDataInput(position);
+        return fileToClose;
+    }
+
+    @SuppressWarnings("resource")
+    private void read(SSTableReader sstable, FileDataInput file, RowIndexEntry indexEntry)
+    throws IOException
+    {
+        List<IndexHelper.IndexInfo> indexList;
+
+        // If the entry is not indexed or the index is not promoted, read from the row start
+        if (!indexEntry.isIndexed())
+        {
+            if (file == null)
+                file = createFileDataInput(indexEntry.position);
+            else
+                file.seek(indexEntry.position);
+
+            DecoratedKey keyInDisk = sstable.partitioner.decorateKey(ByteBufferUtil.readWithShortLength(file));
+            assert keyInDisk.equals(key) : String.format("%s != %s in %s", keyInDisk, key, file.getPath());
+        }
+
+        indexList = indexEntry.columnsIndex();
+
+        if (!indexEntry.isIndexed())
+        {
+            ColumnFamilySerializer serializer = ColumnFamily.serializer;
+            try
+            {
+                cf = ArrayBackedSortedColumns.factory.create(sstable.metadata);
+                cf.delete(DeletionTime.serializer.deserialize(file));
+            }
+            catch (Exception e)
+            {
+                throw new IOException(serializer + " failed to deserialize " + sstable.getColumnFamilyName() + " with " + sstable.metadata + " from " + file, e);
+            }
+        }
+        else
+        {
+            cf = ArrayBackedSortedColumns.factory.create(sstable.metadata);
+            cf.delete(indexEntry.deletionTime());
+        }
+
+        List<OnDiskAtom> result = new ArrayList<OnDiskAtom>();
+        if (indexList.isEmpty())
+        {
+            readSimpleColumns(file, columns, result);
+        }
+        else
+        {
+            readIndexedColumns(sstable.metadata, file, columns, indexList, indexEntry.position, result);
+        }
+
+        // create an iterator view of the columns we read
+        iter = result.iterator();
+    }
+
+    private void readSimpleColumns(FileDataInput file, SortedSet<CellName> columnNames, List<OnDiskAtom> result)
+    {
+        Iterator<OnDiskAtom> atomIterator = cf.metadata().getOnDiskIterator(file, sstable.descriptor.version);
+        int n = 0;
+        while (atomIterator.hasNext())
+        {
+            OnDiskAtom column = atomIterator.next();
+            if (column instanceof Cell)
+            {
+                if (columnNames.contains(column.name()))
+                {
+                    result.add(column);
+                    if (++n >= columns.size())
+                        break;
+                }
+            }
+            else
+            {
+                result.add(column);
+            }
+        }
+    }
+
+    @SuppressWarnings("resource")
+    private void readIndexedColumns(CFMetaData metadata,
+                                    FileDataInput file,
+                                    SortedSet<CellName> columnNames,
+                                    List<IndexHelper.IndexInfo> indexList,
+                                    long basePosition,
+                                    List<OnDiskAtom> result)
+    throws IOException
+    {
+        /* get the various column ranges we have to read */
+        CellNameType comparator = metadata.comparator;
+        List<IndexHelper.IndexInfo> ranges = new ArrayList<IndexHelper.IndexInfo>();
+        int lastIndexIdx = -1;
+        for (CellName name : columnNames)
+        {
+            int index = IndexHelper.indexFor(name, indexList, comparator, false, lastIndexIdx);
+            if (index < 0 || index == indexList.size())
+                continue;
+            IndexHelper.IndexInfo indexInfo = indexList.get(index);
+            // Check the index block does contain the column names and that we haven't inserted this block yet.
+            if (comparator.compare(name, indexInfo.firstName) < 0 || index == lastIndexIdx)
+                continue;
+
+            ranges.add(indexInfo);
+            lastIndexIdx = index;
+        }
+
+        if (ranges.isEmpty())
+            return;
+
+        Iterator<CellName> toFetch = columnNames.iterator();
+        CellName nextToFetch = toFetch.next();
+        for (IndexHelper.IndexInfo indexInfo : ranges)
+        {
+            long positionToSeek = basePosition + indexInfo.offset;
+
+            // With new promoted indexes, our first seek in the data file will happen at that point.
+            if (file == null)
+                file = createFileDataInput(positionToSeek);
+
+            AtomDeserializer deserializer = cf.metadata().getOnDiskDeserializer(file, sstable.descriptor.version);
+            file.seek(positionToSeek);
+            FileMark mark = file.mark();
+            while (file.bytesPastMark(mark) < indexInfo.width && nextToFetch != null)
+            {
+                int cmp = deserializer.compareNextTo(nextToFetch);
+                if (cmp < 0)
+                {
+                    // If it's a rangeTombstone, then we need to read it and include
+                    // it if it includes our target. Otherwise, we can skip it.
+                    if (deserializer.nextIsRangeTombstone())
+                    {
+                        RangeTombstone rt = (RangeTombstone)deserializer.readNext();
+                        if (comparator.compare(rt.max, nextToFetch) >= 0)
+                            result.add(rt);
+                    }
+                    else
+                    {
+                        deserializer.skipNext();
+                    }
+                }
+                else if (cmp == 0)
+                {
+                    nextToFetch = toFetch.hasNext() ? toFetch.next() : null;
+                    result.add(deserializer.readNext());
+                }
+                else
+                    nextToFetch = toFetch.hasNext() ? toFetch.next() : null;
+            }
+        }
+    }
+
+    public DecoratedKey getKey()
+    {
+        return key;
+    }
+
+    public ColumnFamily getColumnFamily()
+    {
+        return cf;
+    }
+
+    protected OnDiskAtom computeNext()
+    {
+        if (iter == null || !iter.hasNext())
+            return endOfData();
+        return iter.next();
+    }
+
+    public void close() throws IOException { }
+}
diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/SSTableSliceIterator.java b/src/java/org/apache/cassandra/io/sstable/format/big/SSTableSliceIterator.java
new file mode 100644
index 0000000..07d867d
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/sstable/format/big/SSTableSliceIterator.java
@@ -0,0 +1,102 @@
+/*
+ * 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.io.sstable.format.big;
+
+import java.io.IOException;
+
+import org.apache.cassandra.db.ColumnFamily;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.OnDiskAtom;
+import org.apache.cassandra.db.RowIndexEntry;
+import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
+import org.apache.cassandra.db.filter.ColumnSlice;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.FileDataInput;
+
+/**
+ *  A Cell Iterator over SSTable
+ */
+class SSTableSliceIterator implements OnDiskAtomIterator
+{
+    private final OnDiskAtomIterator reader;
+    private final DecoratedKey key;
+
+    public SSTableSliceIterator(SSTableReader sstable, DecoratedKey key, ColumnSlice[] slices, boolean reversed)
+    {
+        this.key = key;
+        RowIndexEntry indexEntry = sstable.getPosition(key, SSTableReader.Operator.EQ);
+        this.reader = indexEntry == null ? null : createReader(sstable, indexEntry, null, slices, reversed);
+    }
+
+    /**
+     * An iterator for a slice within an SSTable
+     * @param sstable Keyspace for the CFS we are reading from
+     * @param file Optional parameter that input is read from.  If null is passed, this class creates an appropriate one automatically.
+     * If this class creates, it will close the underlying file when #close() is called.
+     * If a caller passes a non-null argument, this class will NOT close the underlying file when the iterator is closed (i.e. the caller is responsible for closing the file)
+     * In all cases the caller should explicitly #close() this iterator.
+     * @param key The key the requested slice resides under
+     * @param slices the column slices
+     * @param reversed Results are returned in reverse order iff reversed is true.
+     * @param indexEntry position of the row
+     */
+    public SSTableSliceIterator(SSTableReader sstable, FileDataInput file, DecoratedKey key, ColumnSlice[] slices, boolean reversed, RowIndexEntry indexEntry)
+    {
+        this.key = key;
+        reader = createReader(sstable, indexEntry, file, slices, reversed);
+    }
+
+    private static OnDiskAtomIterator createReader(SSTableReader sstable, RowIndexEntry indexEntry, FileDataInput file, ColumnSlice[] slices, boolean reversed)
+    {
+        return slices.length == 1 && slices[0].start.isEmpty() && !reversed
+             ? new SimpleSliceReader(sstable, indexEntry, file, slices[0].finish)
+             : new IndexedSliceReader(sstable, indexEntry, file, slices, reversed);
+    }
+
+    public DecoratedKey getKey()
+    {
+        return key;
+    }
+
+    public ColumnFamily getColumnFamily()
+    {
+        return reader == null ? null : reader.getColumnFamily();
+    }
+
+    public boolean hasNext()
+    {
+        return reader != null && reader.hasNext();
+    }
+
+    public OnDiskAtom next()
+    {
+        return reader.next();
+    }
+
+    public void remove()
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    public void close() throws IOException
+    {
+        if (reader != null)
+            reader.close();
+    }
+
+}
diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/SimpleSliceReader.java b/src/java/org/apache/cassandra/io/sstable/format/big/SimpleSliceReader.java
new file mode 100644
index 0000000..9fec303
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/sstable/format/big/SimpleSliceReader.java
@@ -0,0 +1,108 @@
+/*
+ * 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.io.sstable.format.big;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import com.google.common.collect.AbstractIterator;
+import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.composites.CellNameType;
+import org.apache.cassandra.db.composites.Composite;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.tracing.Tracing;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+class SimpleSliceReader extends AbstractIterator<OnDiskAtom> implements OnDiskAtomIterator
+{
+    private static final Logger logger = LoggerFactory.getLogger(SimpleSliceReader.class);
+
+    private final FileDataInput file;
+    private final boolean needsClosing;
+    private final Composite finishColumn;
+    private final CellNameType comparator;
+    private final ColumnFamily emptyColumnFamily;
+    private final Iterator<OnDiskAtom> atomIterator;
+
+    SimpleSliceReader(SSTableReader sstable, RowIndexEntry indexEntry, FileDataInput input, Composite finishColumn)
+    {
+        Tracing.trace("Seeking to partition beginning in data file");
+        this.finishColumn = finishColumn;
+        this.comparator = sstable.metadata.comparator;
+        try
+        {
+            if (input == null)
+            {
+                this.file = sstable.getFileDataInput(indexEntry.position);
+                this.needsClosing = true;
+            }
+            else
+            {
+                this.file = input;
+                input.seek(indexEntry.position);
+                this.needsClosing = false;
+            }
+
+            // Skip key and data size
+            ByteBufferUtil.skipShortLength(file);
+
+            emptyColumnFamily = ArrayBackedSortedColumns.factory.create(sstable.metadata);
+            emptyColumnFamily.delete(DeletionTime.serializer.deserialize(file));
+            atomIterator = emptyColumnFamily.metadata().getOnDiskIterator(file, sstable.descriptor.version);
+        }
+        catch (IOException e)
+        {
+            sstable.markSuspect();
+            throw new CorruptSSTableException(e, sstable.getFilename());
+        }
+    }
+
+    protected OnDiskAtom computeNext()
+    {
+        if (!atomIterator.hasNext())
+            return endOfData();
+
+        OnDiskAtom column = atomIterator.next();
+        if (!finishColumn.isEmpty() && comparator.compare(column.name(), finishColumn) > 0)
+            return endOfData();
+
+        return column;
+    }
+
+    public ColumnFamily getColumnFamily()
+    {
+        return emptyColumnFamily;
+    }
+
+    public void close() throws IOException
+    {
+        if (needsClosing)
+            file.close();
+    }
+
+    public DecoratedKey getKey()
+    {
+        throw new UnsupportedOperationException();
+    }
+}
diff --git a/src/java/org/apache/cassandra/io/sstable/metadata/CompactionMetadata.java b/src/java/org/apache/cassandra/io/sstable/metadata/CompactionMetadata.java
index f801dac..c8e6ee8 100644
--- a/src/java/org/apache/cassandra/io/sstable/metadata/CompactionMetadata.java
+++ b/src/java/org/apache/cassandra/io/sstable/metadata/CompactionMetadata.java
@@ -26,7 +26,7 @@
 import com.clearspring.analytics.stream.cardinality.ICardinality;
 
 import org.apache.cassandra.db.TypeSizes;
-import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.format.Version;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
@@ -72,7 +72,7 @@
 
     public static class CompactionMetadataSerializer implements IMetadataComponentSerializer<CompactionMetadata>
     {
-        public int serializedSize(CompactionMetadata component) throws IOException
+        public int serializedSize(CompactionMetadata component, Version version) throws IOException
         {
             int size = 0;
             size += TypeSizes.NATIVE.sizeof(component.ancestors.size());
@@ -83,7 +83,7 @@
             return size;
         }
 
-        public void serialize(CompactionMetadata component, DataOutputPlus out) throws IOException
+        public void serialize(CompactionMetadata component, Version version, DataOutputPlus out) throws IOException
         {
             out.writeInt(component.ancestors.size());
             for (int g : component.ancestors)
@@ -91,7 +91,7 @@
             ByteBufferUtil.writeWithLength(component.cardinalityEstimator.getBytes(), out);
         }
 
-        public CompactionMetadata deserialize(Descriptor.Version version, DataInput in) throws IOException
+        public CompactionMetadata deserialize(Version version, DataInput in) throws IOException
         {
             int nbAncestors = in.readInt();
             Set<Integer> ancestors = new HashSet<>(nbAncestors);
diff --git a/src/java/org/apache/cassandra/io/sstable/metadata/IMetadataComponentSerializer.java b/src/java/org/apache/cassandra/io/sstable/metadata/IMetadataComponentSerializer.java
index 49ae378..e3d867f 100644
--- a/src/java/org/apache/cassandra/io/sstable/metadata/IMetadataComponentSerializer.java
+++ b/src/java/org/apache/cassandra/io/sstable/metadata/IMetadataComponentSerializer.java
@@ -20,7 +20,7 @@
 import java.io.DataInput;
 import java.io.IOException;
 
-import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.format.Version;
 import org.apache.cassandra.io.util.DataOutputPlus;
 
 /**
@@ -35,7 +35,7 @@
      * @return serialized size of this component
      * @throws IOException
      */
-    int serializedSize(T component) throws IOException;
+    int serializedSize(T component, Version version) throws IOException;
 
     /**
      * Serialize metadata component to given output.
@@ -45,7 +45,7 @@
      * @param out  serialize destination
      * @throws IOException
      */
-    void serialize(T component, DataOutputPlus out) throws IOException;
+    void serialize(T component, Version version, DataOutputPlus out) throws IOException;
 
     /**
      * Deserialize metadata component from given input.
@@ -55,5 +55,5 @@
      * @return Deserialized component
      * @throws IOException
      */
-    T deserialize(Descriptor.Version version, DataInput in) throws IOException;
+    T deserialize(Version version, DataInput in) throws IOException;
 }
diff --git a/src/java/org/apache/cassandra/io/sstable/metadata/IMetadataSerializer.java b/src/java/org/apache/cassandra/io/sstable/metadata/IMetadataSerializer.java
index df577df..a7d23f4 100644
--- a/src/java/org/apache/cassandra/io/sstable/metadata/IMetadataSerializer.java
+++ b/src/java/org/apache/cassandra/io/sstable/metadata/IMetadataSerializer.java
@@ -22,6 +22,7 @@
 import java.util.Map;
 
 import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.format.Version;
 import org.apache.cassandra.io.util.DataOutputPlus;
 
 /**
@@ -37,7 +38,7 @@
      * @param out
      * @throws IOException
      */
-    void serialize(Map<MetadataType, MetadataComponent> components, DataOutputPlus out) throws IOException;
+    void serialize(Map<MetadataType, MetadataComponent> components, Version version, DataOutputPlus out) throws IOException;
 
     /**
      * Deserialize specified metadata components from given descriptor.
diff --git a/src/java/org/apache/cassandra/io/sstable/metadata/LegacyMetadataSerializer.java b/src/java/org/apache/cassandra/io/sstable/metadata/LegacyMetadataSerializer.java
index 4bd060e..bfeb930 100644
--- a/src/java/org/apache/cassandra/io/sstable/metadata/LegacyMetadataSerializer.java
+++ b/src/java/org/apache/cassandra/io/sstable/metadata/LegacyMetadataSerializer.java
@@ -26,6 +26,7 @@
 import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.format.Version;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -42,7 +43,7 @@
      * Legacy serialization is only used for SSTable level reset.
      */
     @Override
-    public void serialize(Map<MetadataType, MetadataComponent> components, DataOutputPlus out) throws IOException
+    public void serialize(Map<MetadataType, MetadataComponent> components, Version version, DataOutputPlus out) throws IOException
     {
         ValidationMetadata validation = (ValidationMetadata) components.get(MetadataType.VALIDATION);
         StatsMetadata stats = (StatsMetadata) components.get(MetadataType.STATS);
@@ -52,7 +53,7 @@
 
         EstimatedHistogram.serializer.serialize(stats.estimatedRowSize, out);
         EstimatedHistogram.serializer.serialize(stats.estimatedColumnCount, out);
-        ReplayPosition.serializer.serialize(stats.replayPosition, out);
+        ReplayPosition.serializer.serialize(stats.commitLogUpperBound, out);
         out.writeLong(stats.minTimestamp);
         out.writeLong(stats.maxTimestamp);
         out.writeInt(stats.maxLocalDeletionTime);
@@ -70,6 +71,8 @@
         out.writeInt(stats.maxColumnNames.size());
         for (ByteBuffer columnName : stats.maxColumnNames)
             ByteBufferUtil.writeWithShortLength(columnName, out);
+        if (version.hasCommitLogLowerBound())
+            ReplayPosition.serializer.serialize(stats.commitLogLowerBound, out);
     }
 
     /**
@@ -91,7 +94,8 @@
             {
                 EstimatedHistogram rowSizes = EstimatedHistogram.serializer.deserialize(in);
                 EstimatedHistogram columnCounts = EstimatedHistogram.serializer.deserialize(in);
-                ReplayPosition replayPosition = ReplayPosition.serializer.deserialize(in);
+                ReplayPosition commitLogLowerBound = ReplayPosition.NONE;
+                ReplayPosition commitLogUpperBound = ReplayPosition.serializer.deserialize(in);
                 long minTimestamp = in.readLong();
                 long maxTimestamp = in.readLong();
                 int maxLocalDeletionTime = in.readInt();
@@ -116,6 +120,8 @@
                 List<ByteBuffer> maxColumnNames = new ArrayList<>(colCount);
                 for (int i = 0; i < colCount; i++)
                     maxColumnNames.add(ByteBufferUtil.readWithShortLength(in));
+                if (descriptor.version.hasCommitLogLowerBound())
+                    commitLogLowerBound = ReplayPosition.serializer.deserialize(in);
 
                 if (types.contains(MetadataType.VALIDATION))
                     components.put(MetadataType.VALIDATION,
@@ -124,7 +130,8 @@
                     components.put(MetadataType.STATS,
                                    new StatsMetadata(rowSizes,
                                                      columnCounts,
-                                                     replayPosition,
+                                                     commitLogLowerBound,
+                                                     commitLogUpperBound,
                                                      minTimestamp,
                                                      maxTimestamp,
                                                      maxLocalDeletionTime,
diff --git a/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java b/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
index 7ba2895..579ff7a 100644
--- a/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
+++ b/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
@@ -28,6 +28,7 @@
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Maps;
+import com.google.common.collect.Ordering;
 
 import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus;
 import com.clearspring.analytics.stream.cardinality.ICardinality;
@@ -37,7 +38,7 @@
 import org.apache.cassandra.io.sstable.ColumnStats;
 import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.SSTable;
-import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.utils.EstimatedHistogram;
 import org.apache.cassandra.utils.MurmurHash;
@@ -69,6 +70,7 @@
         return new StatsMetadata(defaultRowSizeHistogram(),
                                  defaultColumnCountHistogram(),
                                  ReplayPosition.NONE,
+                                 ReplayPosition.NONE,
                                  Long.MIN_VALUE,
                                  Long.MAX_VALUE,
                                  Integer.MAX_VALUE,
@@ -83,7 +85,8 @@
 
     protected EstimatedHistogram estimatedRowSize = defaultRowSizeHistogram();
     protected EstimatedHistogram estimatedColumnCount = defaultColumnCountHistogram();
-    protected ReplayPosition replayPosition = ReplayPosition.NONE;
+    protected ReplayPosition commitLogLowerBound = ReplayPosition.NONE;
+    protected ReplayPosition commitLogUpperBound = ReplayPosition.NONE;
     protected long minTimestamp = Long.MAX_VALUE;
     protected long maxTimestamp = Long.MIN_VALUE;
     protected int maxLocalDeletionTime = Integer.MIN_VALUE;
@@ -109,23 +112,47 @@
         this.columnNameComparator = columnNameComparator;
     }
 
-    public MetadataCollector(Collection<SSTableReader> sstables, CellNameType columnNameComparator, int level)
+    public MetadataCollector(Iterable<SSTableReader> sstables, CellNameType columnNameComparator, int level, boolean skipAncestors)
     {
         this(columnNameComparator);
 
-        replayPosition(ReplayPosition.getReplayPosition(sstables));
+        ReplayPosition min = null, max = null;
+        for (SSTableReader sstable : sstables)
+        {
+            if (min == null)
+            {
+                min = sstable.getSSTableMetadata().commitLogLowerBound;
+                max = sstable.getSSTableMetadata().commitLogUpperBound;
+            }
+            else
+            {
+                min = Ordering.natural().min(min, sstable.getSSTableMetadata().commitLogLowerBound);
+                max = Ordering.natural().max(max, sstable.getSSTableMetadata().commitLogUpperBound);
+            }
+        }
+
+        commitLogLowerBound(min);
+        commitLogUpperBound(max);
         sstableLevel(level);
         // Get the max timestamp of the precompacted sstables
         // and adds generation of live ancestors
-        for (SSTableReader sstable : sstables)
+        if (!skipAncestors)
         {
-            addAncestor(sstable.descriptor.generation);
-            for (Integer i : sstable.getAncestors())
-                if (new File(sstable.descriptor.withGeneration(i).filenameFor(Component.DATA)).exists())
-                    addAncestor(i);
+            for (SSTableReader sstable : sstables)
+            {
+                addAncestor(sstable.descriptor.generation);
+                for (Integer i : sstable.getAncestors())
+                    if (new File(sstable.descriptor.withGeneration(i).filenameFor(Component.DATA)).exists())
+                        addAncestor(i);
+            }
         }
     }
 
+    public MetadataCollector(Iterable<SSTableReader> sstables, CellNameType columnNameComparator, int level)
+    {
+        this(sstables, columnNameComparator, level, false);
+    }
+
     public MetadataCollector addKey(ByteBuffer key)
     {
         long hashed = MurmurHash.hash2_64(key, key.position(), key.remaining(), 0);
@@ -191,9 +218,15 @@
         return this;
     }
 
-    public MetadataCollector replayPosition(ReplayPosition replayPosition)
+    public MetadataCollector commitLogLowerBound(ReplayPosition commitLogLowerBound)
     {
-        this.replayPosition = replayPosition;
+        this.commitLogLowerBound = commitLogLowerBound;
+        return this;
+    }
+
+    public MetadataCollector commitLogUpperBound(ReplayPosition commitLogUpperBound)
+    {
+        this.commitLogUpperBound = commitLogUpperBound;
         return this;
     }
 
@@ -249,7 +282,8 @@
         components.put(MetadataType.VALIDATION, new ValidationMetadata(partitioner, bloomFilterFPChance));
         components.put(MetadataType.STATS, new StatsMetadata(estimatedRowSize,
                                                              estimatedColumnCount,
-                                                             replayPosition,
+                                                             commitLogLowerBound,
+                                                             commitLogUpperBound,
                                                              minTimestamp,
                                                              maxTimestamp,
                                                              maxLocalDeletionTime,
diff --git a/src/java/org/apache/cassandra/io/sstable/metadata/MetadataSerializer.java b/src/java/org/apache/cassandra/io/sstable/metadata/MetadataSerializer.java
index 0dcd981..ca7fe82 100644
--- a/src/java/org/apache/cassandra/io/sstable/metadata/MetadataSerializer.java
+++ b/src/java/org/apache/cassandra/io/sstable/metadata/MetadataSerializer.java
@@ -22,15 +22,17 @@
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
 import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.format.Version;
 import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.io.util.DataOutputStreamAndChannel;
+import org.apache.cassandra.io.util.DataOutputStreamPlus;
 import org.apache.cassandra.io.util.FileDataInput;
 import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.BufferedDataOutputStreamPlus;
 import org.apache.cassandra.io.util.RandomAccessReader;
 import org.apache.cassandra.utils.FBUtilities;
 
@@ -48,7 +50,7 @@
 {
     private static final Logger logger = LoggerFactory.getLogger(MetadataSerializer.class);
 
-    public void serialize(Map<MetadataType, MetadataComponent> components, DataOutputPlus out) throws IOException
+    public void serialize(Map<MetadataType, MetadataComponent> components, Version version, DataOutputPlus out) throws IOException
     {
         // sort components by type
         List<MetadataComponent> sortedComponents = Lists.newArrayList(components.values());
@@ -65,23 +67,23 @@
             out.writeInt(type.ordinal());
             // serialize position
             out.writeInt(lastPosition);
-            lastPosition += type.serializer.serializedSize(component);
+            lastPosition += type.serializer.serializedSize(component, version);
         }
         // serialize components
         for (MetadataComponent component : sortedComponents)
         {
-            component.getType().serializer.serialize(component, out);
+            component.getType().serializer.serialize(component, version, out);
         }
     }
 
     public Map<MetadataType, MetadataComponent> deserialize(Descriptor descriptor, EnumSet<MetadataType> types) throws IOException
     {
         Map<MetadataType, MetadataComponent> components;
-        logger.debug("Load metadata for {}", descriptor);
+        logger.trace("Load metadata for {}", descriptor);
         File statsFile = new File(descriptor.filenameFor(Component.STATS));
         if (!statsFile.exists())
         {
-            logger.debug("No sstable stats for {}", descriptor);
+            logger.trace("No sstable stats for {}", descriptor);
             components = Maps.newHashMap();
             components.put(MetadataType.STATS, MetadataCollector.defaultStatsMetadata());
         }
@@ -107,16 +109,18 @@
         int numComponents = in.readInt();
         // read toc
         Map<MetadataType, Integer> toc = new HashMap<>(numComponents);
+        MetadataType[] values = MetadataType.values();
         for (int i = 0; i < numComponents; i++)
         {
-            toc.put(MetadataType.values()[in.readInt()], in.readInt());
+            toc.put(values[in.readInt()], in.readInt());
         }
         for (MetadataType type : types)
         {
             MetadataComponent component = null;
-            if (toc.containsKey(type))
+            Integer offset = toc.get(type);
+            if (offset != null)
             {
-                in.seek(toc.get(type));
+                in.seek(offset);
                 component = type.serializer.deserialize(descriptor.version, in);
             }
             components.put(type, component);
@@ -126,7 +130,7 @@
 
     public void mutateLevel(Descriptor descriptor, int newLevel) throws IOException
     {
-        logger.debug("Mutating {} to level {}", descriptor.filenameFor(Component.STATS), newLevel);
+        logger.trace("Mutating {} to level {}", descriptor.filenameFor(Component.STATS), newLevel);
         Map<MetadataType, MetadataComponent> currentComponents = deserialize(descriptor, EnumSet.allOf(MetadataType.class));
         StatsMetadata stats = (StatsMetadata) currentComponents.remove(MetadataType.STATS);
         // mutate level
@@ -136,7 +140,7 @@
 
     public void mutateRepairedAt(Descriptor descriptor, long newRepairedAt) throws IOException
     {
-        logger.debug("Mutating {} to repairedAt time {}", descriptor.filenameFor(Component.STATS), newRepairedAt);
+        logger.trace("Mutating {} to repairedAt time {}", descriptor.filenameFor(Component.STATS), newRepairedAt);
         Map<MetadataType, MetadataComponent> currentComponents = deserialize(descriptor, EnumSet.allOf(MetadataType.class));
         StatsMetadata stats = (StatsMetadata) currentComponents.remove(MetadataType.STATS);
         // mutate level
@@ -148,9 +152,9 @@
     {
         Descriptor tmpDescriptor = descriptor.asType(Descriptor.Type.TEMP);
 
-        try (DataOutputStreamAndChannel out = new DataOutputStreamAndChannel(new FileOutputStream(tmpDescriptor.filenameFor(Component.STATS))))
+        try (DataOutputStreamPlus out = new BufferedDataOutputStreamPlus(new FileOutputStream(tmpDescriptor.filenameFor(Component.STATS))))
         {
-            serialize(currentComponents, out);
+            serialize(currentComponents, descriptor.version, out);
             out.flush();
         }
         // we cant move a file on top of another file in windows:
diff --git a/src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java b/src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java
index a557b88..3d48e34 100644
--- a/src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java
+++ b/src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java
@@ -23,12 +23,12 @@
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.cassandra.io.sstable.format.Version;
 import org.apache.commons.lang3.builder.EqualsBuilder;
 import org.apache.commons.lang3.builder.HashCodeBuilder;
 
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.db.commitlog.ReplayPosition;
-import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.EstimatedHistogram;
@@ -43,7 +43,8 @@
 
     public final EstimatedHistogram estimatedRowSize;
     public final EstimatedHistogram estimatedColumnCount;
-    public final ReplayPosition replayPosition;
+    public final ReplayPosition commitLogLowerBound;
+    public final ReplayPosition commitLogUpperBound;
     public final long minTimestamp;
     public final long maxTimestamp;
     public final int maxLocalDeletionTime;
@@ -57,7 +58,8 @@
 
     public StatsMetadata(EstimatedHistogram estimatedRowSize,
                          EstimatedHistogram estimatedColumnCount,
-                         ReplayPosition replayPosition,
+                         ReplayPosition commitLogLowerBound,
+                         ReplayPosition commitLogUpperBound,
                          long minTimestamp,
                          long maxTimestamp,
                          int maxLocalDeletionTime,
@@ -71,7 +73,8 @@
     {
         this.estimatedRowSize = estimatedRowSize;
         this.estimatedColumnCount = estimatedColumnCount;
-        this.replayPosition = replayPosition;
+        this.commitLogLowerBound = commitLogLowerBound;
+        this.commitLogUpperBound = commitLogUpperBound;
         this.minTimestamp = minTimestamp;
         this.maxTimestamp = maxTimestamp;
         this.maxLocalDeletionTime = maxLocalDeletionTime;
@@ -117,7 +120,8 @@
     {
         return new StatsMetadata(estimatedRowSize,
                                  estimatedColumnCount,
-                                 replayPosition,
+                                 commitLogLowerBound,
+                                 commitLogUpperBound,
                                  minTimestamp,
                                  maxTimestamp,
                                  maxLocalDeletionTime,
@@ -134,7 +138,8 @@
     {
         return new StatsMetadata(estimatedRowSize,
                                  estimatedColumnCount,
-                                 replayPosition,
+                                 commitLogLowerBound,
+                                 commitLogUpperBound,
                                  minTimestamp,
                                  maxTimestamp,
                                  maxLocalDeletionTime,
@@ -157,7 +162,8 @@
         return new EqualsBuilder()
                        .append(estimatedRowSize, that.estimatedRowSize)
                        .append(estimatedColumnCount, that.estimatedColumnCount)
-                       .append(replayPosition, that.replayPosition)
+                       .append(commitLogLowerBound, that.commitLogLowerBound)
+                       .append(commitLogUpperBound, that.commitLogUpperBound)
                        .append(minTimestamp, that.minTimestamp)
                        .append(maxTimestamp, that.maxTimestamp)
                        .append(maxLocalDeletionTime, that.maxLocalDeletionTime)
@@ -177,7 +183,8 @@
         return new HashCodeBuilder()
                        .append(estimatedRowSize)
                        .append(estimatedColumnCount)
-                       .append(replayPosition)
+                       .append(commitLogLowerBound)
+                       .append(commitLogUpperBound)
                        .append(minTimestamp)
                        .append(maxTimestamp)
                        .append(maxLocalDeletionTime)
@@ -193,12 +200,12 @@
 
     public static class StatsMetadataSerializer implements IMetadataComponentSerializer<StatsMetadata>
     {
-        public int serializedSize(StatsMetadata component) throws IOException
+        public int serializedSize(StatsMetadata component, Version version) throws IOException
         {
             int size = 0;
             size += EstimatedHistogram.serializer.serializedSize(component.estimatedRowSize, TypeSizes.NATIVE);
             size += EstimatedHistogram.serializer.serializedSize(component.estimatedColumnCount, TypeSizes.NATIVE);
-            size += ReplayPosition.serializer.serializedSize(component.replayPosition, TypeSizes.NATIVE);
+            size += ReplayPosition.serializer.serializedSize(component.commitLogUpperBound, TypeSizes.NATIVE);
             size += 8 + 8 + 4 + 8 + 8; // mix/max timestamp(long), maxLocalDeletionTime(int), compressionRatio(double), repairedAt (long)
             size += StreamingHistogram.serializer.serializedSize(component.estimatedTombstoneDropTime, TypeSizes.NATIVE);
             size += TypeSizes.NATIVE.sizeof(component.sstableLevel);
@@ -211,14 +218,16 @@
             for (ByteBuffer columnName : component.maxColumnNames)
                 size += 2 + columnName.remaining(); // with short length
             size += TypeSizes.NATIVE.sizeof(component.hasLegacyCounterShards);
+            if (version.hasCommitLogLowerBound())
+                size += ReplayPosition.serializer.serializedSize(component.commitLogLowerBound, TypeSizes.NATIVE);
             return size;
         }
 
-        public void serialize(StatsMetadata component, DataOutputPlus out) throws IOException
+        public void serialize(StatsMetadata component, Version version, DataOutputPlus out) throws IOException
         {
             EstimatedHistogram.serializer.serialize(component.estimatedRowSize, out);
             EstimatedHistogram.serializer.serialize(component.estimatedColumnCount, out);
-            ReplayPosition.serializer.serialize(component.replayPosition, out);
+            ReplayPosition.serializer.serialize(component.commitLogUpperBound, out);
             out.writeLong(component.minTimestamp);
             out.writeLong(component.maxTimestamp);
             out.writeInt(component.maxLocalDeletionTime);
@@ -233,13 +242,16 @@
             for (ByteBuffer columnName : component.maxColumnNames)
                 ByteBufferUtil.writeWithShortLength(columnName, out);
             out.writeBoolean(component.hasLegacyCounterShards);
+            if (version.hasCommitLogLowerBound())
+                ReplayPosition.serializer.serialize(component.commitLogLowerBound, out);
         }
 
-        public StatsMetadata deserialize(Descriptor.Version version, DataInput in) throws IOException
+        public StatsMetadata deserialize(Version version, DataInput in) throws IOException
         {
             EstimatedHistogram rowSizes = EstimatedHistogram.serializer.deserialize(in);
             EstimatedHistogram columnCounts = EstimatedHistogram.serializer.deserialize(in);
-            ReplayPosition replayPosition = ReplayPosition.serializer.deserialize(in);
+            ReplayPosition commitLogLowerBound = ReplayPosition.NONE, commitLogUpperBound;
+            commitLogUpperBound = ReplayPosition.serializer.deserialize(in);
             long minTimestamp = in.readLong();
             long maxTimestamp = in.readLong();
             int maxLocalDeletionTime = in.readInt();
@@ -247,7 +259,7 @@
             StreamingHistogram tombstoneHistogram = StreamingHistogram.serializer.deserialize(in);
             int sstableLevel = in.readInt();
             long repairedAt = 0;
-            if (version.hasRepairedAt)
+            if (version.hasRepairedAt())
                 repairedAt = in.readLong();
 
             int colCount = in.readInt();
@@ -261,12 +273,15 @@
                 maxColumnNames.add(ByteBufferUtil.readWithShortLength(in));
 
             boolean hasLegacyCounterShards = true;
-            if (version.tracksLegacyCounterShards)
+            if (version.tracksLegacyCounterShards())
                 hasLegacyCounterShards = in.readBoolean();
 
+            if (version.hasCommitLogLowerBound())
+                commitLogLowerBound = ReplayPosition.serializer.deserialize(in);
             return new StatsMetadata(rowSizes,
                                      columnCounts,
-                                     replayPosition,
+                                     commitLogLowerBound,
+                                     commitLogUpperBound,
                                      minTimestamp,
                                      maxTimestamp,
                                      maxLocalDeletionTime,
diff --git a/src/java/org/apache/cassandra/io/sstable/metadata/ValidationMetadata.java b/src/java/org/apache/cassandra/io/sstable/metadata/ValidationMetadata.java
index e00c55c..4ca078b 100644
--- a/src/java/org/apache/cassandra/io/sstable/metadata/ValidationMetadata.java
+++ b/src/java/org/apache/cassandra/io/sstable/metadata/ValidationMetadata.java
@@ -21,7 +21,7 @@
 import java.io.IOException;
 
 import org.apache.cassandra.db.TypeSizes;
-import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.format.Version;
 import org.apache.cassandra.io.util.DataOutputPlus;
 
 /**
@@ -71,18 +71,18 @@
 
     public static class ValidationMetadataSerializer implements IMetadataComponentSerializer<ValidationMetadata>
     {
-        public int serializedSize(ValidationMetadata component) throws IOException
+        public int serializedSize(ValidationMetadata component, Version version) throws IOException
         {
             return TypeSizes.NATIVE.sizeof(component.partitioner) + 8;
         }
 
-        public void serialize(ValidationMetadata component, DataOutputPlus out) throws IOException
+        public void serialize(ValidationMetadata component, Version version, DataOutputPlus out) throws IOException
         {
             out.writeUTF(component.partitioner);
             out.writeDouble(component.bloomFilterFPChance);
         }
 
-        public ValidationMetadata deserialize(Descriptor.Version version, DataInput in) throws IOException
+        public ValidationMetadata deserialize(Version version, DataInput in) throws IOException
         {
 
             return new ValidationMetadata(in.readUTF(), in.readDouble());
diff --git a/src/java/org/apache/cassandra/io/util/AbstractDataInput.java b/src/java/org/apache/cassandra/io/util/AbstractDataInput.java
index ff8b6b2..588540d 100644
--- a/src/java/org/apache/cassandra/io/util/AbstractDataInput.java
+++ b/src/java/org/apache/cassandra/io/util/AbstractDataInput.java
@@ -21,12 +21,20 @@
 
 public abstract class AbstractDataInput extends InputStream implements DataInput
 {
-    protected abstract void seekInternal(int position);
-    protected abstract int getPosition();
+    public abstract void seek(long position) throws IOException;
+    public abstract long getPosition();
+    public abstract long getPositionLimit();
 
-    /*
-     !! DataInput methods below are copied from the implementation in Apache Harmony RandomAccessFile.
-     */
+    public int skipBytes(int n) throws IOException
+    {
+        if (n <= 0)
+            return 0;
+        long oldPosition = getPosition();
+        seek(Math.min(getPositionLimit(), oldPosition + n));
+        long skipped = getPosition() - oldPosition;
+        assert skipped >= 0 && skipped <= n;
+        return (int) skipped;
+    }
 
     /**
      * Reads a boolean from the current position in this file. Blocks until one
@@ -214,7 +222,7 @@
     public final String readLine() throws IOException {
         StringBuilder line = new StringBuilder(80); // Typical line length
         boolean foundTerminator = false;
-        int unreadPosition = 0;
+        long unreadPosition = -1;
         while (true) {
             int nextByte = read();
             switch (nextByte) {
@@ -222,7 +230,7 @@
                     return line.length() != 0 ? line.toString() : null;
                 case (byte) '\r':
                     if (foundTerminator) {
-                        seekInternal(unreadPosition);
+                        seek(unreadPosition);
                         return line.toString();
                     }
                     foundTerminator = true;
@@ -233,7 +241,7 @@
                     return line.toString();
                 default:
                     if (foundTerminator) {
-                        seekInternal(unreadPosition);
+                        seek(unreadPosition);
                         return line.toString();
                     }
                     line.append((char) nextByte);
diff --git a/src/java/org/apache/cassandra/io/util/AbstractDataOutput.java b/src/java/org/apache/cassandra/io/util/AbstractDataOutput.java
deleted file mode 100644
index 8f4bed8..0000000
--- a/src/java/org/apache/cassandra/io/util/AbstractDataOutput.java
+++ /dev/null
@@ -1,329 +0,0 @@
-/*
- * 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.io.util;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.io.UTFDataFormatException;
-import java.nio.ByteBuffer;
-
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-public abstract class AbstractDataOutput extends OutputStream implements DataOutputPlus
-{
-    /*
-    !! DataOutput methods below are copied from the implementation in Apache Harmony RandomAccessFile.
-    */
-
-    /**
-     * Writes the entire contents of the byte array <code>buffer</code> to
-     * this RandomAccessFile starting at the current file pointer.
-     * 
-     * @param buffer
-     *            the buffer to be written.
-     * 
-     * @throws IOException
-     *             If an error occurs trying to write to this RandomAccessFile.
-     */
-    public void write(byte[] buffer) throws IOException {
-        write(buffer, 0, buffer.length);
-    }
-
-    /**
-     * Writes <code>count</code> bytes from the byte array <code>buffer</code>
-     * starting at <code>offset</code> to this RandomAccessFile starting at
-     * the current file pointer..
-     * 
-     * @param buffer
-     *            the bytes to be written
-     * @param offset
-     *            offset in buffer to get bytes
-     * @param count
-     *            number of bytes in buffer to write
-     * 
-     * @throws IOException
-     *             If an error occurs attempting to write to this
-     *             RandomAccessFile.
-     * @throws IndexOutOfBoundsException
-     *             If offset or count are outside of bounds.
-     */
-    public abstract void write(byte[] buffer, int offset, int count) throws IOException;
-
-    /**
-     * Writes the specified byte <code>oneByte</code> to this RandomAccessFile
-     * starting at the current file pointer. Only the low order byte of
-     * <code>oneByte</code> is written.
-     * 
-     * @param oneByte
-     *            the byte to be written
-     * 
-     * @throws IOException
-     *             If an error occurs attempting to write to this
-     *             RandomAccessFile.
-     */
-    public abstract void write(int oneByte) throws IOException;
-
-    /**
-     * Writes a boolean to this output stream.
-     * 
-     * @param val
-     *            the boolean value to write to the OutputStream
-     * 
-     * @throws IOException
-     *             If an error occurs attempting to write to this
-     *             DataOutputStream.
-     */
-    public final void writeBoolean(boolean val) throws IOException {
-        write(val ? 1 : 0);
-    }
-
-    /**
-     * Writes a 8-bit byte to this output stream.
-     * 
-     * @param val
-     *            the byte value to write to the OutputStream
-     * 
-     * @throws java.io.IOException
-     *             If an error occurs attempting to write to this
-     *             DataOutputStream.
-     */
-    public final void writeByte(int val) throws IOException {
-        write(val & 0xFF);
-    }
-
-    /**
-     * Writes the low order 8-bit bytes from a String to this output stream.
-     * 
-     * @param str
-     *            the String containing the bytes to write to the OutputStream
-     * 
-     * @throws IOException
-     *             If an error occurs attempting to write to this
-     *             DataOutputStream.
-     */
-    public final void writeBytes(String str) throws IOException {
-        byte bytes[] = new byte[str.length()];
-        for (int index = 0; index < str.length(); index++) {
-            bytes[index] = (byte) (str.charAt(index) & 0xFF);
-        }
-        write(bytes);
-    }
-
-    /**
-     * Writes the specified 16-bit character to the OutputStream. Only the lower
-     * 2 bytes are written with the higher of the 2 bytes written first. This
-     * represents the Unicode value of val.
-     * 
-     * @param val
-     *            the character to be written
-     * 
-     * @throws IOException
-     *             If an error occurs attempting to write to this
-     *             DataOutputStream.
-     */
-    public final void writeChar(int val) throws IOException {
-        write((val >>> 8) & 0xFF);
-        write((val >>> 0) & 0xFF);
-    }
-
-    /**
-     * Writes the specified 16-bit characters contained in str to the
-     * OutputStream. Only the lower 2 bytes of each character are written with
-     * the higher of the 2 bytes written first. This represents the Unicode
-     * value of each character in str.
-     * 
-     * @param str
-     *            the String whose characters are to be written.
-     * 
-     * @throws IOException
-     *             If an error occurs attempting to write to this
-     *             DataOutputStream.
-     */
-    public final void writeChars(String str) throws IOException {
-        byte newBytes[] = new byte[str.length() * 2];
-        for (int index = 0; index < str.length(); index++) {
-            int newIndex = index == 0 ? index : index * 2;
-            newBytes[newIndex] = (byte) ((str.charAt(index) >> 8) & 0xFF);
-            newBytes[newIndex + 1] = (byte) (str.charAt(index) & 0xFF);
-        }
-        write(newBytes);
-    }
-
-    /**
-     * Writes a 64-bit double to this output stream. The resulting output is the
-     * 8 bytes resulting from calling Double.doubleToLongBits().
-     * 
-     * @param val
-     *            the double to be written.
-     * 
-     * @throws IOException
-     *             If an error occurs attempting to write to this
-     *             DataOutputStream.
-     */
-    public final void writeDouble(double val) throws IOException {
-        writeLong(Double.doubleToLongBits(val));
-    }
-
-    /**
-     * Writes a 32-bit float to this output stream. The resulting output is the
-     * 4 bytes resulting from calling Float.floatToIntBits().
-     * 
-     * @param val
-     *            the float to be written.
-     * 
-     * @throws IOException
-     *             If an error occurs attempting to write to this
-     *             DataOutputStream.
-     */
-    public final void writeFloat(float val) throws IOException {
-        writeInt(Float.floatToIntBits(val));
-    }
-
-    /**
-     * Writes a 32-bit int to this output stream. The resulting output is the 4
-     * bytes, highest order first, of val.
-     * 
-     * @param val
-     *            the int to be written.
-     * 
-     * @throws IOException
-     *             If an error occurs attempting to write to this
-     *             DataOutputStream.
-     */
-    public void writeInt(int val) throws IOException {
-        write((val >>> 24) & 0xFF);
-        write((val >>> 16) & 0xFF);
-        write((val >>>  8) & 0xFF);
-        write((val >>> 0) & 0xFF);
-    }
-
-    /**
-     * Writes a 64-bit long to this output stream. The resulting output is the 8
-     * bytes, highest order first, of val.
-     * 
-     * @param val
-     *            the long to be written.
-     * 
-     * @throws IOException
-     *             If an error occurs attempting to write to this
-     *             DataOutputStream.
-     */
-    public void writeLong(long val) throws IOException {
-        write((int)(val >>> 56) & 0xFF);
-        write((int)(val >>> 48) & 0xFF);
-        write((int)(val >>> 40) & 0xFF);
-        write((int)(val >>> 32) & 0xFF);
-        write((int)(val >>> 24) & 0xFF);
-        write((int)(val >>> 16) & 0xFF);
-        write((int)(val >>>  8) & 0xFF);
-        write((int) (val >>> 0) & 0xFF);
-    }
-
-    /**
-     * Writes the specified 16-bit short to the OutputStream. Only the lower 2
-     * bytes are written with the higher of the 2 bytes written first.
-     * 
-     * @param val
-     *            the short to be written
-     * 
-     * @throws IOException
-     *             If an error occurs attempting to write to this
-     *             DataOutputStream.
-     */
-    public void writeShort(int val) throws IOException {
-        writeChar(val);
-    }
-
-    /**
-     * Writes the specified String out in UTF format.
-     * 
-     * @param str
-     *            the String to be written in UTF format.
-     * 
-     * @throws IOException
-     *             If an error occurs attempting to write to this
-     *             DataOutputStream.
-     */
-    public final void writeUTF(String str) throws IOException {
-        int utfCount = 0, length = str.length();
-        for (int i = 0; i < length; i++) {
-            int charValue = str.charAt(i);
-            if (charValue > 0 && charValue <= 127) {
-                utfCount++;
-            } else if (charValue <= 2047) {
-                utfCount += 2;
-            } else {
-                utfCount += 3;
-            }
-        }
-        if (utfCount > 65535) {
-            throw new UTFDataFormatException(); //$NON-NLS-1$
-        }
-        byte utfBytes[] = new byte[utfCount + 2];
-        int utfIndex = 2;
-        for (int i = 0; i < length; i++) {
-            int charValue = str.charAt(i);
-            if (charValue > 0 && charValue <= 127) {
-                utfBytes[utfIndex++] = (byte) charValue;
-            } else if (charValue <= 2047) {
-                utfBytes[utfIndex++] = (byte) (0xc0 | (0x1f & (charValue >> 6)));
-                utfBytes[utfIndex++] = (byte) (0x80 | (0x3f & charValue));
-            } else {
-                utfBytes[utfIndex++] = (byte) (0xe0 | (0x0f & (charValue >> 12)));
-                utfBytes[utfIndex++] = (byte) (0x80 | (0x3f & (charValue >> 6)));
-                utfBytes[utfIndex++] = (byte) (0x80 | (0x3f & charValue));
-            }
-        }
-        utfBytes[0] = (byte) (utfCount >> 8);
-        utfBytes[1] = (byte) utfCount;
-        write(utfBytes);
-    }
-
-    private byte[] buf;
-    public synchronized void write(ByteBuffer buffer) throws IOException
-    {
-        int len = buffer.remaining();
-        if (len < 16)
-        {
-            int offset = buffer.position();
-            for (int i = 0 ; i < len ; i++)
-                write(buffer.get(i + offset));
-            return;
-        }
-
-        byte[] buf = this.buf;
-        if (buf == null)
-            this.buf = buf = new byte[256];
-
-        int offset = 0;
-        while (len > 0)
-        {
-            int sublen = Math.min(buf.length, len);
-            ByteBufferUtil.arrayCopy(buffer, buffer.position() + offset, buf, 0, sublen);
-            write(buf, 0, sublen);
-            offset += sublen;
-            len -= sublen;
-        }
-    }
-
-    public void write(Memory memory, long offset, long length) throws IOException
-    {
-        for (ByteBuffer buffer : memory.asByteBuffers(offset, length))
-            write(buffer);
-    }
-}
diff --git a/src/java/org/apache/cassandra/io/util/BufferedDataOutputStreamPlus.java b/src/java/org/apache/cassandra/io/util/BufferedDataOutputStreamPlus.java
new file mode 100644
index 0000000..d55db47
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/util/BufferedDataOutputStreamPlus.java
@@ -0,0 +1,305 @@
+/*
+ * 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.io.util;
+
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.nio.channels.WritableByteChannel;
+
+import com.google.common.base.Function;
+import com.google.common.base.Preconditions;
+
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.utils.memory.MemoryUtil;
+
+
+/**
+ * An implementation of the DataOutputStreamPlus interface using a ByteBuffer to stage writes
+ * before flushing them to an underlying channel.
+ *
+ * This class is completely thread unsafe.
+ */
+public class BufferedDataOutputStreamPlus extends DataOutputStreamPlus
+{
+    private static final int DEFAULT_BUFFER_SIZE = Integer.getInteger(Config.PROPERTY_PREFIX + "nio_data_output_stream_plus_buffer_size", 1024 * 32);
+
+    ByteBuffer buffer;
+
+    public BufferedDataOutputStreamPlus(RandomAccessFile ras)
+    {
+        this(ras.getChannel());
+    }
+
+    public BufferedDataOutputStreamPlus(RandomAccessFile ras, int bufferSize)
+    {
+        this(ras.getChannel(), bufferSize);
+    }
+
+    public BufferedDataOutputStreamPlus(FileOutputStream fos)
+    {
+        this(fos.getChannel());
+    }
+
+    public BufferedDataOutputStreamPlus(FileOutputStream fos, int bufferSize)
+    {
+        this(fos.getChannel(), bufferSize);
+    }
+
+    public BufferedDataOutputStreamPlus(WritableByteChannel wbc)
+    {
+        this(wbc, DEFAULT_BUFFER_SIZE);
+    }
+
+    public BufferedDataOutputStreamPlus(WritableByteChannel wbc, int bufferSize)
+    {
+        this(wbc, ByteBuffer.allocateDirect(bufferSize));
+        Preconditions.checkNotNull(wbc);
+        Preconditions.checkArgument(bufferSize >= 8, "Buffer size must be large enough to accommodate a long/double");
+    }
+
+    protected BufferedDataOutputStreamPlus(WritableByteChannel channel, ByteBuffer buffer)
+    {
+        super(channel);
+        this.buffer = buffer;
+    }
+
+    protected BufferedDataOutputStreamPlus(ByteBuffer buffer)
+    {
+        super();
+        this.buffer = buffer;
+    }
+
+    @Override
+    public void write(byte[] b) throws IOException
+    {
+        write(b, 0, b.length);
+    }
+
+    @Override
+    public void write(byte[] b, int off, int len) throws IOException
+    {
+        if (b == null)
+            throw new NullPointerException();
+
+        // avoid int overflow
+        if (off < 0 || off > b.length || len < 0
+            || len > b.length - off)
+            throw new IndexOutOfBoundsException();
+
+        if (len == 0)
+            return;
+
+        int copied = 0;
+        while (copied < len)
+        {
+            if (buffer.hasRemaining())
+            {
+                int toCopy = Math.min(len - copied, buffer.remaining());
+                buffer.put(b, off + copied, toCopy);
+                copied += toCopy;
+            }
+            else
+            {
+                doFlush(len - copied);
+            }
+        }
+    }
+
+    // ByteBuffer to use for defensive copies
+    private final ByteBuffer hollowBuffer = MemoryUtil.getHollowDirectByteBuffer();
+
+    /*
+     * Makes a defensive copy of the incoming ByteBuffer and don't modify the position or limit
+     * even temporarily so it is thread-safe WRT to the incoming buffer
+     * (non-Javadoc)
+     * @see org.apache.cassandra.io.util.DataOutputPlus#write(java.nio.ByteBuffer)
+     */
+    @Override
+    public void write(ByteBuffer toWrite) throws IOException
+    {
+        if (toWrite.hasArray())
+        {
+            write(toWrite.array(), toWrite.arrayOffset() + toWrite.position(), toWrite.remaining());
+        }
+        else
+        {
+            assert toWrite.isDirect();
+            int toWriteRemaining = toWrite.remaining();
+            if (toWriteRemaining > buffer.remaining())
+            {
+                doFlush(toWriteRemaining);
+                MemoryUtil.duplicateDirectByteBuffer(toWrite, hollowBuffer);
+                if (toWriteRemaining > buffer.remaining())
+                {
+                    while (hollowBuffer.hasRemaining())
+                        channel.write(hollowBuffer);
+                }
+                else
+                {
+                    buffer.put(hollowBuffer);
+                }
+            }
+            else
+            {
+                MemoryUtil.duplicateDirectByteBuffer(toWrite, hollowBuffer);
+                buffer.put(hollowBuffer);
+            }
+        }
+    }
+
+
+    @Override
+    public void write(int b) throws IOException
+    {
+        ensureRemaining(1);
+        buffer.put((byte) (b & 0xFF));
+    }
+
+    @Override
+    public void writeBoolean(boolean v) throws IOException
+    {
+        ensureRemaining(1);
+        buffer.put(v ? (byte)1 : (byte)0);
+    }
+
+    @Override
+    public void writeByte(int v) throws IOException
+    {
+        write(v);
+    }
+
+    @Override
+    public void writeShort(int v) throws IOException
+    {
+        ensureRemaining(2);
+        buffer.putShort((short) v);
+    }
+
+    @Override
+    public void writeChar(int v) throws IOException
+    {
+        ensureRemaining(2);
+        buffer.putChar((char) v);
+    }
+
+    @Override
+    public void writeInt(int v) throws IOException
+    {
+        ensureRemaining(4);
+        buffer.putInt(v);
+    }
+
+    @Override
+    public void writeLong(long v) throws IOException
+    {
+        ensureRemaining(8);
+        buffer.putLong(v);
+    }
+
+    @Override
+    public void writeFloat(float v) throws IOException
+    {
+        ensureRemaining(4);
+        buffer.putFloat(v);
+    }
+
+    @Override
+    public void writeDouble(double v) throws IOException
+    {
+        ensureRemaining(8);
+        buffer.putDouble(v);
+    }
+
+    @Override
+    public void writeBytes(String s) throws IOException
+    {
+        for (int index = 0; index < s.length(); index++)
+            writeByte(s.charAt(index));
+    }
+
+    @Override
+    public void writeChars(String s) throws IOException
+    {
+        for (int index = 0; index < s.length(); index++)
+            writeChar(s.charAt(index));
+    }
+
+    @Override
+    public void writeUTF(String s) throws IOException
+    {
+        UnbufferedDataOutputStreamPlus.writeUTF(s, this);
+    }
+
+    @Override
+    public void write(Memory memory, long offset, long length) throws IOException
+    {
+        for (ByteBuffer buffer : memory.asByteBuffers(offset, length))
+            write(buffer);
+    }
+
+    /*
+     * Count is the number of bytes remaining to write ignoring already remaining capacity
+     */
+    protected void doFlush(int count) throws IOException
+    {
+        buffer.flip();
+
+        while (buffer.hasRemaining())
+            channel.write(buffer);
+
+        buffer.clear();
+    }
+
+    @Override
+    public void flush() throws IOException
+    {
+        doFlush(0);
+    }
+
+    @Override
+    public void close() throws IOException
+    {
+        doFlush(0);
+        channel.close();
+        FileUtils.clean(buffer);
+        buffer = null;
+    }
+
+    protected void ensureRemaining(int minimum) throws IOException
+    {
+        if (buffer.remaining() < minimum)
+            doFlush(minimum);
+    }
+
+    @Override
+    public <R> R applyToChannel(Function<WritableByteChannel, R> f) throws IOException
+    {
+        //Don't allow writes to the underlying channel while data is buffered
+        flush();
+        return f.apply(channel);
+    }
+
+    public BufferedDataOutputStreamPlus order(ByteOrder order)
+    {
+        this.buffer.order(order);
+        return this;
+    }
+}
diff --git a/src/java/org/apache/cassandra/io/util/BufferedPoolingSegmentedFile.java b/src/java/org/apache/cassandra/io/util/BufferedPoolingSegmentedFile.java
index f04a1fb..b623e54 100644
--- a/src/java/org/apache/cassandra/io/util/BufferedPoolingSegmentedFile.java
+++ b/src/java/org/apache/cassandra/io/util/BufferedPoolingSegmentedFile.java
@@ -17,15 +17,11 @@
 */
 package org.apache.cassandra.io.util;
 
-import java.io.File;
-
-import org.apache.cassandra.io.sstable.SSTableWriter;
-
 public class BufferedPoolingSegmentedFile extends PoolingSegmentedFile
 {
-    public BufferedPoolingSegmentedFile(String path, long length)
+    public BufferedPoolingSegmentedFile(ChannelProxy channel, long length)
     {
-        super(new Cleanup(path), path, length);
+        super(new Cleanup(channel), channel, length);
     }
 
     private BufferedPoolingSegmentedFile(BufferedPoolingSegmentedFile copy)
@@ -45,11 +41,10 @@
             // only one segment in a standard-io file
         }
 
-        public SegmentedFile complete(String path, long overrideLength, boolean isFinal)
+        public SegmentedFile complete(ChannelProxy channel, long overrideLength)
         {
-            assert !isFinal || overrideLength <= 0;
-            long length = overrideLength > 0 ? overrideLength : new File(path).length();
-            return new BufferedPoolingSegmentedFile(path, length);
+            long length = overrideLength > 0 ? overrideLength : channel.size();
+            return new BufferedPoolingSegmentedFile(channel, length);
         }
     }
 }
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/io/util/BufferedSegmentedFile.java b/src/java/org/apache/cassandra/io/util/BufferedSegmentedFile.java
index 1a1d208..2c59def 100644
--- a/src/java/org/apache/cassandra/io/util/BufferedSegmentedFile.java
+++ b/src/java/org/apache/cassandra/io/util/BufferedSegmentedFile.java
@@ -17,16 +17,11 @@
  */
 package org.apache.cassandra.io.util;
 
-import java.io.File;
-
-import org.apache.cassandra.io.sstable.SSTableWriter;
-import org.apache.cassandra.utils.concurrent.SharedCloseable;
-
 public class BufferedSegmentedFile extends SegmentedFile
 {
-    public BufferedSegmentedFile(String path, long length)
+    public BufferedSegmentedFile(ChannelProxy channel, long length)
     {
-        super(new Cleanup(path), path, length);
+        super(new Cleanup(channel), channel, length);
     }
 
     private BufferedSegmentedFile(BufferedSegmentedFile copy)
@@ -36,13 +31,13 @@
 
     private static class Cleanup extends SegmentedFile.Cleanup
     {
-        protected Cleanup(String path)
+        protected Cleanup(ChannelProxy channel)
         {
-            super(path);
+            super(channel);
         }
-        public void tidy() throws Exception
+        public void tidy()
         {
-
+            super.tidy();
         }
     }
 
@@ -53,17 +48,16 @@
             // only one segment in a standard-io file
         }
 
-        public SegmentedFile complete(String path, long overrideLength, boolean isFinal)
+        public SegmentedFile complete(ChannelProxy channel, long overrideLength)
         {
-            assert !isFinal || overrideLength <= 0;
-            long length = overrideLength > 0 ? overrideLength : new File(path).length();
-            return new BufferedSegmentedFile(path, length);
+            long length = overrideLength > 0 ? overrideLength : channel.size();
+            return new BufferedSegmentedFile(channel, length);
         }
     }
 
     public FileDataInput getSegment(long position)
     {
-        RandomAccessReader reader = RandomAccessReader.open(new File(path));
+        RandomAccessReader reader = RandomAccessReader.open(channel);
         reader.seek(position);
         return reader;
     }
diff --git a/src/java/org/apache/cassandra/io/util/ByteBufferDataInput.java b/src/java/org/apache/cassandra/io/util/ByteBufferDataInput.java
new file mode 100644
index 0000000..bf926e9
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/util/ByteBufferDataInput.java
@@ -0,0 +1,171 @@
+/*
+ * 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.io.util;
+
+import java.io.*;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class ByteBufferDataInput extends AbstractDataInput implements FileDataInput, DataInput
+{
+    private final ByteBuffer buffer;
+    private final String filename;
+    private final long segmentOffset;
+    private int position;
+
+    public ByteBufferDataInput(ByteBuffer buffer, String filename, long segmentOffset, int position)
+    {
+        assert buffer != null;
+        this.buffer = buffer;
+        this.filename = filename;
+        this.segmentOffset = segmentOffset;
+        this.position = position;
+    }
+
+    // Only use when we know the seek in within the mapped segment. Throws an
+    // IOException otherwise.
+    public void seek(long pos) throws IOException
+    {
+        long inSegmentPos = pos - segmentOffset;
+        if (inSegmentPos < 0 || inSegmentPos > buffer.capacity())
+            throw new IOException(String.format("Seek position %d is not within mmap segment (seg offs: %d, length: %d)", pos, segmentOffset, buffer.capacity()));
+
+        position = (int) inSegmentPos;
+    }
+
+    public long getFilePointer()
+    {
+        return segmentOffset + position;
+    }
+
+    public long getPosition()
+    {
+        return segmentOffset + position;
+    }
+
+    public long getPositionLimit()
+    {
+        return segmentOffset + buffer.capacity();
+    }
+
+    @Override
+    public boolean markSupported()
+    {
+        return false;
+    }
+
+    public void reset(FileMark mark) throws IOException
+    {
+        assert mark instanceof MappedFileDataInputMark;
+        position = ((MappedFileDataInputMark) mark).position;
+    }
+
+    public FileMark mark()
+    {
+        return new MappedFileDataInputMark(position);
+    }
+
+    public long bytesPastMark(FileMark mark)
+    {
+        assert mark instanceof MappedFileDataInputMark;
+        assert position >= ((MappedFileDataInputMark) mark).position;
+        return position - ((MappedFileDataInputMark) mark).position;
+    }
+
+    public boolean isEOF() throws IOException
+    {
+        return position == buffer.capacity();
+    }
+
+    public long bytesRemaining() throws IOException
+    {
+        return buffer.capacity() - position;
+    }
+
+    public String getPath()
+    {
+        return filename;
+    }
+
+    public int read() throws IOException
+    {
+        if (isEOF())
+            return -1;
+        return buffer.get(position++) & 0xFF;
+    }
+
+    /**
+     * Does the same thing as <code>readFully</code> do but without copying data (thread safe)
+     * @param length length of the bytes to read
+     * @return buffer with portion of file content
+     * @throws IOException on any fail of I/O operation
+     */
+    public ByteBuffer readBytes(int length) throws IOException
+    {
+        int remaining = buffer.remaining() - position;
+        if (length > remaining)
+            throw new IOException(String.format("mmap segment underflow; remaining is %d but %d requested",
+                                                remaining, length));
+
+        if (length == 0)
+            return ByteBufferUtil.EMPTY_BYTE_BUFFER;
+
+        ByteBuffer bytes = buffer.duplicate();
+        bytes.position(buffer.position() + position).limit(buffer.position() + position + length);
+        position += length;
+
+        // we have to copy the data in case we unreference the underlying sstable.  See CASSANDRA-3179
+        ByteBuffer clone = ByteBuffer.allocate(bytes.remaining());
+        clone.put(bytes);
+        clone.flip();
+        return clone;
+    }
+
+    @Override
+    public final void readFully(byte[] bytes) throws IOException
+    {
+        ByteBufferUtil.arrayCopy(buffer, buffer.position() + position, bytes, 0, bytes.length);
+        position += bytes.length;
+    }
+
+    @Override
+    public final void readFully(byte[] bytes, int offset, int count) throws IOException
+    {
+        ByteBufferUtil.arrayCopy(buffer, buffer.position() + position, bytes, offset, count);
+        position += count;
+    }
+
+    private static class MappedFileDataInputMark implements FileMark
+    {
+        int position;
+
+        MappedFileDataInputMark(int position)
+        {
+            this.position = position;
+        }
+    }
+
+    @Override
+    public String toString() {
+        return getClass().getSimpleName() + "(" +
+               "filename='" + filename + "'" +
+               ", position=" + position +
+               ")";
+    }
+}
diff --git a/src/java/org/apache/cassandra/io/util/ChannelProxy.java b/src/java/org/apache/cassandra/io/util/ChannelProxy.java
new file mode 100644
index 0000000..79954a5
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/util/ChannelProxy.java
@@ -0,0 +1,182 @@
+/*
+ * 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.io.util;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.MappedByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.channels.WritableByteChannel;
+import java.nio.file.StandardOpenOption;
+
+import org.apache.cassandra.io.FSReadError;
+import org.apache.cassandra.utils.CLibrary;
+import org.apache.cassandra.utils.concurrent.RefCounted;
+import org.apache.cassandra.utils.concurrent.SharedCloseableImpl;
+
+/**
+ * A proxy of a FileChannel that:
+ *
+ * - implements reference counting
+ * - exports only thread safe FileChannel operations
+ * - wraps IO exceptions into runtime exceptions
+ *
+ * Tested by RandomAccessReaderTest.
+ */
+public final class ChannelProxy extends SharedCloseableImpl
+{
+    private final String filePath;
+    private final FileChannel channel;
+
+    public static FileChannel openChannel(File file)
+    {
+        try
+        {
+            return FileChannel.open(file.toPath(), StandardOpenOption.READ);
+        }
+        catch (IOException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public ChannelProxy(String path)
+    {
+        this (new File(path));
+    }
+
+    public ChannelProxy(File file)
+    {
+        this(file.getAbsolutePath(), openChannel(file));
+    }
+
+    public ChannelProxy(String filePath, FileChannel channel)
+    {
+        super(new Cleanup(filePath, channel));
+
+        this.filePath = filePath;
+        this.channel = channel;
+    }
+
+    public ChannelProxy(ChannelProxy copy)
+    {
+        super(copy);
+
+        this.filePath = copy.filePath;
+        this.channel = copy.channel;
+    }
+
+    private final static class Cleanup implements RefCounted.Tidy
+    {
+        final String filePath;
+        final FileChannel channel;
+
+        protected Cleanup(String filePath, FileChannel channel)
+        {
+            this.filePath = filePath;
+            this.channel = channel;
+        }
+
+        public String name()
+        {
+            return filePath;
+        }
+
+        public void tidy()
+        {
+            try
+            {
+                channel.close();
+            }
+            catch (IOException e)
+            {
+                throw new FSReadError(e, filePath);
+            }
+        }
+    }
+
+    public ChannelProxy sharedCopy()
+    {
+        return new ChannelProxy(this);
+    }
+
+    public String filePath()
+    {
+        return filePath;
+    }
+
+    public int read(ByteBuffer buffer, long position)
+    {
+        try
+        {
+            return channel.read(buffer, position);
+        }
+        catch (IOException e)
+        {
+            throw new FSReadError(e, filePath);
+        }
+    }
+
+    public long transferTo(long position, long count, WritableByteChannel target)
+    {
+        try
+        {
+            return channel.transferTo(position, count, target);
+        }
+        catch (IOException e)
+        {
+            throw new FSReadError(e, filePath);
+        }
+    }
+
+    public MappedByteBuffer map(FileChannel.MapMode mode, long position, long size)
+    {
+        try
+        {
+            return channel.map(mode, position, size);
+        }
+        catch (IOException e)
+        {
+            throw new FSReadError(e, filePath);
+        }
+    }
+
+    public long size()
+    {
+        try
+        {
+            return channel.size();
+        }
+        catch (IOException e)
+        {
+            throw new FSReadError(e, filePath);
+        }
+    }
+
+    public int getFileDescriptor()
+    {
+        return CLibrary.getfd(channel);
+    }
+
+    @Override
+    public String toString()
+    {
+        return filePath();
+    }
+}
diff --git a/src/java/org/apache/cassandra/io/util/ChecksummedRandomAccessReader.java b/src/java/org/apache/cassandra/io/util/ChecksummedRandomAccessReader.java
new file mode 100644
index 0000000..9015b61
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/util/ChecksummedRandomAccessReader.java
@@ -0,0 +1,129 @@
+/*
+ * 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.io.util;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.zip.Adler32;
+
+import org.apache.cassandra.io.compress.BufferType;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class ChecksummedRandomAccessReader extends RandomAccessReader
+{
+    @SuppressWarnings("serial")
+    public static class CorruptFileException extends RuntimeException
+    {
+        public final File file;
+
+        public CorruptFileException(Exception cause, File file)
+        {
+            super(cause);
+            this.file = file;
+        }
+    }
+
+    private final DataIntegrityMetadata.ChecksumValidator validator;
+    private final File file;
+
+    protected ChecksummedRandomAccessReader(File file, ChannelProxy channel, DataIntegrityMetadata.ChecksumValidator validator)
+    {
+        super(channel, validator.chunkSize, -1, BufferType.ON_HEAP, null);
+        this.validator = validator;
+        this.file = file;
+    }
+
+    @SuppressWarnings("resource")
+    public static ChecksummedRandomAccessReader open(File file, File crcFile) throws IOException
+    {
+        try (ChannelProxy channel = new ChannelProxy(file))
+        {
+            RandomAccessReader crcReader = RandomAccessReader.open(crcFile);
+            boolean closeCrcReader = true;
+            try
+            {
+                DataIntegrityMetadata.ChecksumValidator validator =
+                        new DataIntegrityMetadata.ChecksumValidator(new Adler32(), crcReader, file.getPath());
+                closeCrcReader = false;
+                boolean closeValidator = true;
+                try
+                {
+                    ChecksummedRandomAccessReader retval = new ChecksummedRandomAccessReader(file, channel, validator);
+                    closeValidator = false;
+                    return retval;
+                }
+                finally
+                {
+                    if (closeValidator)
+                        validator.close();
+                }
+            }
+            finally
+            {
+                if (closeCrcReader)
+                    crcReader.close();
+            }
+        }
+    }
+
+    @Override
+    protected void reBuffer()
+    {
+        long desiredPosition = current();
+        // align with buffer size, as checksums were computed in chunks of buffer size each.
+        bufferOffset = (desiredPosition / buffer.capacity()) * buffer.capacity();
+
+        buffer.clear();
+
+        long position = bufferOffset;
+        while (buffer.hasRemaining())
+        {
+            int n = channel.read(buffer, position);
+            if (n < 0)
+                break;
+            position += n;
+        }
+
+        buffer.flip();
+
+        try
+        {
+            validator.validate(ByteBufferUtil.getArray(buffer), 0, buffer.remaining());
+        }
+        catch (IOException e)
+        {
+            throw new CorruptFileException(e, file);
+        }
+
+        buffer.position((int) (desiredPosition - bufferOffset));
+    }
+
+    @Override
+    public void seek(long newPosition)
+    {
+        validator.seek(newPosition);
+        super.seek(newPosition);
+    }
+
+    @Override
+    public void close()
+    {
+        super.close();
+        validator.close();
+    }
+}
diff --git a/src/java/org/apache/cassandra/io/util/ChecksummedSequentialWriter.java b/src/java/org/apache/cassandra/io/util/ChecksummedSequentialWriter.java
index 526347b..d5e6be9 100644
--- a/src/java/org/apache/cassandra/io/util/ChecksummedSequentialWriter.java
+++ b/src/java/org/apache/cassandra/io/util/ChecksummedSequentialWriter.java
@@ -18,8 +18,9 @@
 package org.apache.cassandra.io.util;
 
 import java.io.File;
+import java.nio.ByteBuffer;
 
-import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.compress.BufferType;
 
 public class ChecksummedSequentialWriter extends SequentialWriter
 {
@@ -28,32 +29,51 @@
 
     public ChecksummedSequentialWriter(File file, int bufferSize, File crcPath)
     {
-        super(file, bufferSize);
-        crcWriter = new SequentialWriter(crcPath, 8 * 1024);
+        super(file, bufferSize, BufferType.ON_HEAP);
+        crcWriter = new SequentialWriter(crcPath, 8 * 1024, BufferType.ON_HEAP);
         crcMetadata = new DataIntegrityMetadata.ChecksumWriter(crcWriter.stream);
-        crcMetadata.writeChunkSize(buffer.length);
+        crcMetadata.writeChunkSize(buffer.capacity());
     }
 
     protected void flushData()
     {
         super.flushData();
-        crcMetadata.append(buffer, 0, validBufferBytes, false);
+        ByteBuffer toAppend = buffer.duplicate();
+        toAppend.position(0);
+        toAppend.limit(buffer.position());
+        crcMetadata.appendDirect(toAppend, false);
     }
 
-    public void writeFullChecksum(Descriptor descriptor)
+    protected class TransactionalProxy extends SequentialWriter.TransactionalProxy
     {
-        crcMetadata.writeFullChecksum(descriptor);
+        @Override
+        protected Throwable doCommit(Throwable accumulate)
+        {
+            return crcWriter.commit(accumulate);
+        }
+
+        @Override
+        protected Throwable doAbort(Throwable accumulate)
+        {
+            return super.doAbort(crcWriter.abort(accumulate));
+        }
+
+        @Override
+        protected void doPrepare()
+        {
+            syncInternal();
+            if (descriptor != null)
+                crcMetadata.writeFullChecksum(descriptor);
+            crcWriter.setDescriptor(descriptor).prepareToCommit();
+            // we must cleanup our file handles during prepareCommit for Windows compatibility as we cannot rename an open file;
+            // TODO: once we stop file renaming, remove this for clarity
+            releaseFileHandle();
+        }
     }
 
-    public void close()
+    @Override
+    protected SequentialWriter.TransactionalProxy txnProxy()
     {
-        super.close();
-        crcWriter.close();
-    }
-
-    public void abort()
-    {
-        super.abort();
-        crcWriter.abort();
+        return new TransactionalProxy();
     }
 }
diff --git a/src/java/org/apache/cassandra/io/util/CompressedPoolingSegmentedFile.java b/src/java/org/apache/cassandra/io/util/CompressedPoolingSegmentedFile.java
index 3b71ddd..fdc4f61 100644
--- a/src/java/org/apache/cassandra/io/util/CompressedPoolingSegmentedFile.java
+++ b/src/java/org/apache/cassandra/io/util/CompressedPoolingSegmentedFile.java
@@ -17,6 +17,10 @@
 */
 package org.apache.cassandra.io.util;
 
+import java.nio.MappedByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.TreeMap;
+
 import com.google.common.util.concurrent.RateLimiter;
 
 import org.apache.cassandra.io.compress.CompressedRandomAccessReader;
@@ -27,31 +31,56 @@
 public class CompressedPoolingSegmentedFile extends PoolingSegmentedFile implements ICompressedFile
 {
     public final CompressionMetadata metadata;
+    private final TreeMap<Long, MappedByteBuffer> chunkSegments;
 
-    public CompressedPoolingSegmentedFile(String path, CompressionMetadata metadata)
+    public CompressedPoolingSegmentedFile(ChannelProxy channel, CompressionMetadata metadata)
     {
-        super(new Cleanup(path, metadata), path, metadata.dataLength, metadata.compressedFileLength);
+        this(channel, metadata, CompressedSegmentedFile.createMappedSegments(channel, metadata));
+    }
+
+    private CompressedPoolingSegmentedFile(ChannelProxy channel, CompressionMetadata metadata, TreeMap<Long, MappedByteBuffer> chunkSegments)
+    {
+        super(new Cleanup(channel, metadata, chunkSegments), channel, metadata.dataLength, metadata.compressedFileLength);
         this.metadata = metadata;
+        this.chunkSegments = chunkSegments;
     }
 
     private CompressedPoolingSegmentedFile(CompressedPoolingSegmentedFile copy)
     {
         super(copy);
         this.metadata = copy.metadata;
+        this.chunkSegments = copy.chunkSegments;
+    }
+
+    public ChannelProxy channel()
+    {
+        return channel;
+    }
+
+    public TreeMap<Long, MappedByteBuffer> chunkSegments()
+    {
+        return chunkSegments;
     }
 
     protected static final class Cleanup extends PoolingSegmentedFile.Cleanup
     {
         final CompressionMetadata metadata;
-        protected Cleanup(String path, CompressionMetadata metadata)
+        final TreeMap<Long, MappedByteBuffer> chunkSegments;
+        protected Cleanup(ChannelProxy channel, CompressionMetadata metadata, TreeMap<Long, MappedByteBuffer> chunkSegments)
         {
-            super(path);
+            super(channel);
             this.metadata = metadata;
+            this.chunkSegments = chunkSegments;
         }
-        public void tidy() throws Exception
+        public void tidy()
         {
             super.tidy();
             metadata.close();
+            if (chunkSegments != null)
+            {
+                for (MappedByteBuffer segment : chunkSegments.values())
+                    FileUtils.clean(segment);
+            }
         }
     }
 
@@ -67,9 +96,9 @@
             // only one segment in a standard-io file
         }
 
-        public SegmentedFile complete(String path, long overrideLength, boolean isFinal)
+        public SegmentedFile complete(ChannelProxy channel, long overrideLength)
         {
-            return new CompressedPoolingSegmentedFile(path, metadata(path, overrideLength, isFinal));
+            return new CompressedPoolingSegmentedFile(channel, metadata(channel.filePath(), overrideLength));
         }
     }
 
@@ -82,17 +111,17 @@
 
     public RandomAccessReader createReader()
     {
-        return CompressedRandomAccessReader.open(path, metadata, null);
+        return CompressedRandomAccessReader.open(this);
     }
 
     public RandomAccessReader createThrottledReader(RateLimiter limiter)
     {
-        return CompressedThrottledReader.open(path, metadata, limiter);
+        return CompressedThrottledReader.open(this, limiter);
     }
 
     protected RandomAccessReader createPooledReader()
     {
-        return CompressedRandomAccessReader.open(path, metadata, this);
+        return CompressedRandomAccessReader.open(this);
     }
 
     public CompressionMetadata getMetadata()
diff --git a/src/java/org/apache/cassandra/io/util/CompressedSegmentedFile.java b/src/java/org/apache/cassandra/io/util/CompressedSegmentedFile.java
index 830fc4b..ceff7ba 100644
--- a/src/java/org/apache/cassandra/io/util/CompressedSegmentedFile.java
+++ b/src/java/org/apache/cassandra/io/util/CompressedSegmentedFile.java
@@ -17,41 +17,104 @@
  */
 package org.apache.cassandra.io.util;
 
+import java.nio.MappedByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.TreeMap;
+
 import com.google.common.util.concurrent.RateLimiter;
 
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.io.compress.CompressedRandomAccessReader;
 import org.apache.cassandra.io.compress.CompressedSequentialWriter;
 import org.apache.cassandra.io.compress.CompressedThrottledReader;
 import org.apache.cassandra.io.compress.CompressionMetadata;
-import org.apache.cassandra.io.sstable.SSTableWriter;
 
 public class CompressedSegmentedFile extends SegmentedFile implements ICompressedFile
 {
     public final CompressionMetadata metadata;
+    private static final boolean useMmap = DatabaseDescriptor.getDiskAccessMode() == Config.DiskAccessMode.mmap;
+    private static int MAX_SEGMENT_SIZE = Integer.MAX_VALUE;
+    private final TreeMap<Long, MappedByteBuffer> chunkSegments;
 
-    public CompressedSegmentedFile(String path, CompressionMetadata metadata)
+    public CompressedSegmentedFile(ChannelProxy channel, CompressionMetadata metadata)
     {
-        super(new Cleanup(path, metadata), path, metadata.dataLength, metadata.compressedFileLength);
+        this(channel, metadata, createMappedSegments(channel, metadata));
+    }
+
+    public CompressedSegmentedFile(ChannelProxy channel, CompressionMetadata metadata, TreeMap<Long, MappedByteBuffer> chunkSegments)
+    {
+        super(new Cleanup(channel, metadata, chunkSegments), channel, metadata.dataLength, metadata.compressedFileLength);
         this.metadata = metadata;
+        this.chunkSegments = chunkSegments;
     }
 
     private CompressedSegmentedFile(CompressedSegmentedFile copy)
     {
         super(copy);
         this.metadata = copy.metadata;
+        this.chunkSegments = copy.chunkSegments;
+    }
+
+    public ChannelProxy channel()
+    {
+        return channel;
+    }
+
+    public TreeMap<Long, MappedByteBuffer> chunkSegments()
+    {
+        return chunkSegments;
+    }
+
+    static TreeMap<Long, MappedByteBuffer> createMappedSegments(ChannelProxy channel, CompressionMetadata metadata)
+    {
+        if (!useMmap)
+            return null;
+        TreeMap<Long, MappedByteBuffer> chunkSegments = new TreeMap<>();
+        long offset = 0;
+        long lastSegmentOffset = 0;
+        long segmentSize = 0;
+
+        while (offset < metadata.dataLength)
+        {
+            CompressionMetadata.Chunk chunk = metadata.chunkFor(offset);
+
+            //Reached a new mmap boundary
+            if (segmentSize + chunk.length + 4 > MAX_SEGMENT_SIZE)
+            {
+                chunkSegments.put(lastSegmentOffset, channel.map(FileChannel.MapMode.READ_ONLY, lastSegmentOffset, segmentSize));
+                lastSegmentOffset += segmentSize;
+                segmentSize = 0;
+            }
+
+            segmentSize += chunk.length + 4; //checksum
+            offset += metadata.chunkLength();
+        }
+
+        if (segmentSize > 0)
+            chunkSegments.put(lastSegmentOffset, channel.map(FileChannel.MapMode.READ_ONLY, lastSegmentOffset, segmentSize));
+        return chunkSegments;
     }
 
     private static final class Cleanup extends SegmentedFile.Cleanup
     {
         final CompressionMetadata metadata;
-        protected Cleanup(String path, CompressionMetadata metadata)
+        final TreeMap<Long, MappedByteBuffer> chunkSegments;
+        protected Cleanup(ChannelProxy channel, CompressionMetadata metadata, TreeMap<Long, MappedByteBuffer> chunkSegments)
         {
-            super(path);
+            super(channel);
             this.metadata = metadata;
+            this.chunkSegments = chunkSegments;
         }
-        public void tidy() throws Exception
+        public void tidy()
         {
+            super.tidy();
             metadata.close();
+            if (chunkSegments != null)
+            {
+                for (MappedByteBuffer segment : chunkSegments.values())
+                    FileUtils.clean(segment);
+            }
         }
     }
 
@@ -73,18 +136,17 @@
             // only one segment in a standard-io file
         }
 
-        protected CompressionMetadata metadata(String path, long overrideLength, boolean isFinal)
+        protected CompressionMetadata metadata(String path, long overrideLength)
         {
             if (writer == null)
                 return CompressionMetadata.create(path);
 
-            return writer.open(overrideLength, isFinal);
+            return writer.open(overrideLength);
         }
 
-        public SegmentedFile complete(String path, long overrideLength, boolean isFinal)
+        public SegmentedFile complete(ChannelProxy channel, long overrideLength)
         {
-            assert !isFinal || overrideLength <= 0;
-            return new CompressedSegmentedFile(path, metadata(path, overrideLength, isFinal));
+            return new CompressedSegmentedFile(channel, metadata(channel.filePath(), overrideLength));
         }
     }
 
@@ -97,12 +159,12 @@
 
     public RandomAccessReader createReader()
     {
-        return CompressedRandomAccessReader.open(path, metadata);
+        return CompressedRandomAccessReader.open(this);
     }
 
     public RandomAccessReader createThrottledReader(RateLimiter limiter)
     {
-        return CompressedThrottledReader.open(path, metadata, limiter);
+        return CompressedThrottledReader.open(this, limiter);
     }
 
     public CompressionMetadata getMetadata()
diff --git a/src/java/org/apache/cassandra/io/util/DataIntegrityMetadata.java b/src/java/org/apache/cassandra/io/util/DataIntegrityMetadata.java
index ac924ec..d44bd1c 100644
--- a/src/java/org/apache/cassandra/io/util/DataIntegrityMetadata.java
+++ b/src/java/org/apache/cassandra/io/util/DataIntegrityMetadata.java
@@ -26,6 +26,7 @@
 import java.nio.ByteBuffer;
 import java.nio.file.Files;
 import java.util.zip.Adler32;
+import java.util.zip.CheckedInputStream;
 import java.util.zip.Checksum;
 
 import com.google.common.base.Charsets;
@@ -33,7 +34,8 @@
 import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.utils.PureJavaCrc32;
+import org.apache.cassandra.utils.CRC32Factory;
+import org.apache.cassandra.utils.FBUtilities;
 
 public class DataIntegrityMetadata
 {
@@ -46,14 +48,20 @@
     {
         private final Checksum checksum;
         private final RandomAccessReader reader;
-        private final Descriptor descriptor;
         public final int chunkSize;
+        private final String dataFilename;
 
         public ChecksumValidator(Descriptor descriptor) throws IOException
         {
-            this.descriptor = descriptor;
-            checksum = descriptor.version.hasAllAdlerChecksums ? new Adler32() : new PureJavaCrc32();
-            reader = RandomAccessReader.open(new File(descriptor.filenameFor(Component.CRC)));
+            this(descriptor.version.hasAllAdlerChecksums() ? new Adler32() : CRC32Factory.instance.create(),
+                 RandomAccessReader.open(new File(descriptor.filenameFor(Component.CRC))),
+                 descriptor.filenameFor(Component.DATA));
+        }
+
+        public ChecksumValidator(Checksum checksum, RandomAccessReader reader, String dataFilename) throws IOException {
+            this.checksum = checksum;
+            this.reader = reader;
+            this.dataFilename = dataFilename;
             chunkSize = reader.readInt();
         }
 
@@ -76,7 +84,7 @@
             checksum.reset();
             int actual = reader.readInt();
             if (current != actual)
-                throw new IOException("Corrupted SSTable : " + descriptor.filenameFor(Component.DATA));
+                throw new IOException("Corrupted File : " + dataFilename);
         }
 
         public void close()
@@ -85,11 +93,62 @@
         }
     }
 
+    public static FileDigestValidator fileDigestValidator(Descriptor desc) throws IOException
+    {
+        return new FileDigestValidator(desc);
+    }
+
+    public static class FileDigestValidator implements Closeable
+    {
+        private final Checksum checksum;
+        private final RandomAccessReader digestReader;
+        private final RandomAccessReader dataReader;
+        private final Descriptor descriptor;
+        private long storedDigestValue;
+
+        public FileDigestValidator(Descriptor descriptor) throws IOException
+        {
+            this.descriptor = descriptor;
+            checksum = descriptor.version.hasAllAdlerChecksums() ? new Adler32() : CRC32Factory.instance.create();
+            digestReader = RandomAccessReader.open(new File(descriptor.filenameFor(Component.DIGEST)));
+            dataReader = RandomAccessReader.open(new File(descriptor.filenameFor(Component.DATA)));
+            try
+            {
+                storedDigestValue = Long.parseLong(digestReader.readLine());
+            }
+            catch (Exception e)
+            {
+                // Attempting to create a FileDigestValidator without a DIGEST file will fail
+                throw new IOException("Corrupted SSTable : " + descriptor.filenameFor(Component.DATA));
+            }
+
+        }
+
+        // Validate the entire file
+        public void validate() throws IOException
+        {
+            CheckedInputStream checkedInputStream = new CheckedInputStream(dataReader, checksum);
+            byte[] chunk = new byte[64 * 1024];
+
+            while( checkedInputStream.read(chunk) > 0 ) { }
+            long calculatedDigestValue = checkedInputStream.getChecksum().getValue();
+            if (storedDigestValue != calculatedDigestValue) {
+                throw new IOException("Corrupted SSTable : " + descriptor.filenameFor(Component.DATA));
+            }
+        }
+
+        public void close()
+        {
+            this.digestReader.close();
+        }
+    }
+
+
     public static class ChecksumWriter
     {
-        private final Checksum incrementalChecksum = new Adler32();
+        private final Adler32 incrementalChecksum = new Adler32();
         private final DataOutput incrementalOut;
-        private final Checksum fullChecksum = new Adler32();
+        private final Adler32 fullChecksum = new Adler32();
 
         public ChecksumWriter(DataOutput incrementalOut)
         {
@@ -115,25 +174,28 @@
 
         // CompressedSequentialWriters serialize the partial checksums inline with the compressed data chunks they
         // corroborate, whereas ChecksummedSequentialWriters serialize them to a different file.
-        public void append(byte[] buffer, int start, int end, boolean checksumIncrementalResult)
+        public void appendDirect(ByteBuffer bb, boolean checksumIncrementalResult)
         {
             try
             {
-                int incrementalChecksumValue;
 
-                incrementalChecksum.update(buffer, start, end);
-                incrementalChecksumValue = (int) incrementalChecksum.getValue();
+                ByteBuffer toAppend = bb.duplicate();
+                toAppend.mark();
+                FBUtilities.directCheckSum(incrementalChecksum, toAppend);
+                toAppend.reset();
+
+                int incrementalChecksumValue = (int) incrementalChecksum.getValue();
                 incrementalOut.writeInt(incrementalChecksumValue);
-                incrementalChecksum.reset();
 
-                fullChecksum.update(buffer, start, end);
-
+                FBUtilities.directCheckSum(fullChecksum, toAppend);
                 if (checksumIncrementalResult)
                 {
                     ByteBuffer byteBuffer = ByteBuffer.allocate(4);
                     byteBuffer.putInt(incrementalChecksumValue);
                     fullChecksum.update(byteBuffer.array(), 0, byteBuffer.array().length);
                 }
+                incrementalChecksum.reset();
+
             }
             catch (IOException e)
             {
@@ -144,20 +206,14 @@
         public void writeFullChecksum(Descriptor descriptor)
         {
             File outFile = new File(descriptor.filenameFor(Component.DIGEST));
-            BufferedWriter out = null;
-            try
+            try (BufferedWriter out =Files.newBufferedWriter(outFile.toPath(), Charsets.UTF_8))
             {
-                out = Files.newBufferedWriter(outFile.toPath(), Charsets.UTF_8);
                 out.write(String.valueOf(fullChecksum.getValue()));
             }
             catch (IOException e)
             {
                 throw new FSWriteError(e, outFile);
             }
-            finally
-            {
-                FileUtils.closeQuietly(out);
-            }
         }
     }
 }
diff --git a/src/java/org/apache/cassandra/io/util/DataOutputBuffer.java b/src/java/org/apache/cassandra/io/util/DataOutputBuffer.java
index 7577567..6ea6d97 100644
--- a/src/java/org/apache/cassandra/io/util/DataOutputBuffer.java
+++ b/src/java/org/apache/cassandra/io/util/DataOutputBuffer.java
@@ -19,8 +19,12 @@
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.util.Arrays;
+import java.nio.channels.WritableByteChannel;
 
+import org.apache.cassandra.config.Config;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
 
 /**
  * An implementation of the DataOutputStream interface using a FastByteArrayOutputStream and exposing
@@ -28,8 +32,13 @@
  *
  * This class is completely thread unsafe.
  */
-public final class DataOutputBuffer extends DataOutputStreamPlus
+public class DataOutputBuffer extends BufferedDataOutputStreamPlus
 {
+    /*
+     * Threshold at which resizing transitions from doubling to increasing by 50%
+     */
+    private static final long DOUBLING_THRESHOLD = Long.getLong(Config.PROPERTY_PREFIX + "DOB_DOUBLING_THRESHOLD_MB", 64);
+
     public DataOutputBuffer()
     {
         this(128);
@@ -37,65 +46,143 @@
 
     public DataOutputBuffer(int size)
     {
-        super(new FastByteArrayOutputStream(size));
+        super(ByteBuffer.allocate(size));
+    }
+
+    protected DataOutputBuffer(ByteBuffer buffer)
+    {
+        super(buffer);
     }
 
     @Override
-    public void write(int b)
+    public void flush() throws IOException
     {
-        try
+        throw new UnsupportedOperationException();
+    }
+
+    //The actual value observed in Hotspot is only -2
+    //ByteArrayOutputStream uses -8
+    @VisibleForTesting
+    static final int MAX_ARRAY_SIZE = Integer.MAX_VALUE - 8;
+
+    @VisibleForTesting
+    static int saturatedArraySizeCast(long size)
+    {
+        Preconditions.checkArgument(size >= 0);
+        return (int)Math.min(MAX_ARRAY_SIZE, size);
+    }
+
+    @VisibleForTesting
+    static int checkedArraySizeCast(long size)
+    {
+        Preconditions.checkArgument(size >= 0);
+        Preconditions.checkArgument(size <= MAX_ARRAY_SIZE);
+        return (int)size;
+    }
+
+    @Override
+    protected void doFlush(int count) throws IOException
+    {
+        reallocate(count);
+    }
+
+    //Hack for test, make it possible to override checking the buffer capacity
+    @VisibleForTesting
+    long capacity()
+    {
+        return buffer.capacity();
+    }
+
+    @VisibleForTesting
+    long validateReallocation(long newSize)
+    {
+        int saturatedSize = saturatedArraySizeCast(newSize);
+        if (saturatedSize <= capacity())
+            throw new RuntimeException();
+        return saturatedSize;
+    }
+
+    @VisibleForTesting
+    long calculateNewSize(long count)
+    {
+        long capacity = capacity();
+        //Both sides of this max expression need to use long arithmetic to avoid integer overflow
+        //count and capacity are longs so that ensures it right now.
+        long newSize = capacity + count;
+
+        //For large buffers don't double, increase by 50%
+        if (capacity > 1024L * 1024L * DOUBLING_THRESHOLD)
+            newSize = Math.max((capacity * 3L) / 2L, newSize);
+        else
+            newSize = Math.max(capacity * 2L, newSize);
+
+        return validateReallocation(newSize);
+    }
+
+    protected void reallocate(long count)
+    {
+        if (count <= 0)
+            return;
+        ByteBuffer newBuffer = ByteBuffer.allocate(checkedArraySizeCast(calculateNewSize(count)));
+        buffer.flip();
+        newBuffer.put(buffer);
+        buffer = newBuffer;
+    }
+
+    @Override
+    protected WritableByteChannel newDefaultChannel()
+    {
+        return new GrowingChannel();
+    }
+
+    @VisibleForTesting
+    final class GrowingChannel implements WritableByteChannel
+    {
+        public int write(ByteBuffer src) throws IOException
         {
-            super.write(b);
+            int count = src.remaining();
+            reallocate(count);
+            buffer.put(src);
+            return count;
         }
-        catch (IOException e)
+
+        public boolean isOpen()
         {
-            throw new AssertionError(e); // FBOS does not throw IOE
+            return true;
+        }
+
+        public void close() throws IOException
+        {
         }
     }
 
     @Override
-    public void write(byte[] b, int off, int len)
+    public void close()
     {
-        try
-        {
-            super.write(b, off, len);
-        }
-        catch (IOException e)
-        {
-            throw new AssertionError(e); // FBOS does not throw IOE
-        }
     }
 
-    public void write(ByteBuffer buffer) throws IOException
+    public ByteBuffer buffer()
     {
-        ((FastByteArrayOutputStream) out).write(buffer);
+        ByteBuffer result = buffer.duplicate();
+        result.flip();
+        return result;
     }
 
-    /**
-     * Returns the current contents of the buffer. Data is only valid to
-     * {@link #getLength()}.
-     */
     public byte[] getData()
     {
-        return ((FastByteArrayOutputStream) out).buf;
+        return buffer.array();
+    }
+
+    public int getLength()
+    {
+        return buffer.position();
     }
 
     public byte[] toByteArray()
     {
-        FastByteArrayOutputStream out = (FastByteArrayOutputStream) this.out;
-        return Arrays.copyOfRange(out.buf, 0, out.count);
-
-    }
-
-    public ByteBuffer asByteBuffer()
-    {
-        FastByteArrayOutputStream out = (FastByteArrayOutputStream) this.out;
-        return ByteBuffer.wrap(out.buf, 0, out.count);
-    }
-
-    /** Returns the length of the valid data currently in the buffer. */
-    public int getLength()
-    {
-        return ((FastByteArrayOutputStream) out).count;
+        ByteBuffer buffer = buffer();
+        byte[] result = new byte[buffer.remaining()];
+        buffer.get(result);
+        return result;
     }
 }
diff --git a/src/java/org/apache/cassandra/io/util/DataOutputBufferFixed.java b/src/java/org/apache/cassandra/io/util/DataOutputBufferFixed.java
new file mode 100644
index 0000000..c815c9e
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/util/DataOutputBufferFixed.java
@@ -0,0 +1,65 @@
+/*
+ * 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.io.util;
+
+import java.io.IOException;
+import java.nio.BufferOverflowException;
+import java.nio.ByteBuffer;
+
+
+/**
+ * An implementation of the DataOutputStream interface using a FastByteArrayOutputStream and exposing
+ * its buffer so copies can be avoided. This version does not expand if it runs out of capacity and
+ * throws BufferOverflowException instead.
+ *
+ * This class is completely thread unsafe.
+ */
+public class DataOutputBufferFixed extends DataOutputBuffer
+{
+    public DataOutputBufferFixed()
+    {
+        this(128);
+    }
+
+    public DataOutputBufferFixed(int size)
+    {
+        super(ByteBuffer.allocate(size));
+    }
+
+    public DataOutputBufferFixed(ByteBuffer buffer)
+    {
+        super(buffer);
+    }
+
+    @Override
+    protected void doFlush(int count) throws IOException
+    {
+        throw new BufferOverflowException();
+    }
+
+    /*
+     * Not currently reachable (all paths hit doFLush first), but in the spirit of things this should throw
+     * if it is called.
+     * @see org.apache.cassandra.io.util.DataOutputBuffer#reallocate(long)
+     */
+    @Override
+    protected void reallocate(long newSize)
+    {
+        throw new BufferOverflowException();
+    }
+}
diff --git a/src/java/org/apache/cassandra/io/util/DataOutputByteBuffer.java b/src/java/org/apache/cassandra/io/util/DataOutputByteBuffer.java
deleted file mode 100644
index b40d30e..0000000
--- a/src/java/org/apache/cassandra/io/util/DataOutputByteBuffer.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * 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.io.util;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-
-/**
- * An implementation of the DataOutputStream interface using a FastByteArrayOutputStream and exposing
- * its buffer so copies can be avoided.
- *
- * This class is completely thread unsafe.
- */
-public final class DataOutputByteBuffer extends AbstractDataOutput
-{
-
-    final ByteBuffer buffer;
-    public DataOutputByteBuffer(ByteBuffer buffer)
-    {
-        this.buffer = buffer;
-    }
-
-    @Override
-    public void write(int b)
-    {
-        buffer.put((byte) b);
-    }
-
-    @Override
-    public void write(byte[] b, int off, int len)
-    {
-        buffer.put(b, off, len);
-    }
-
-    public void write(ByteBuffer buffer) throws IOException
-    {
-        int len = buffer.remaining();
-        ByteBufferUtil.arrayCopy(buffer, buffer.position(), this.buffer, this.buffer.position(), len);
-        this.buffer.position(this.buffer.position() + len);
-    }
-}
diff --git a/src/java/org/apache/cassandra/io/util/DataOutputPlus.java b/src/java/org/apache/cassandra/io/util/DataOutputPlus.java
index c2901e1..f63c1e5 100644
--- a/src/java/org/apache/cassandra/io/util/DataOutputPlus.java
+++ b/src/java/org/apache/cassandra/io/util/DataOutputPlus.java
@@ -20,12 +20,24 @@
 import java.io.DataOutput;
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.nio.channels.WritableByteChannel;
 
+import com.google.common.base.Function;
+
+/**
+ * Extension to DataOutput that provides for writing ByteBuffer and Memory, potentially with an efficient
+ * implementation that is zero copy or at least has reduced bounds checking overhead.
+ */
 public interface DataOutputPlus extends DataOutput
 {
-
     // write the buffer without modifying its position
     void write(ByteBuffer buffer) throws IOException;
 
     void write(Memory memory, long offset, long length) throws IOException;
+
+    /**
+     * Safe way to operate against the underlying channel. Impossible to stash a reference to the channel
+     * and forget to flush
+     */
+    <R> R applyToChannel(Function<WritableByteChannel, R> c) throws IOException;
 }
diff --git a/src/java/org/apache/cassandra/io/util/DataOutputStreamAndChannel.java b/src/java/org/apache/cassandra/io/util/DataOutputStreamAndChannel.java
deleted file mode 100644
index 30cf38b..0000000
--- a/src/java/org/apache/cassandra/io/util/DataOutputStreamAndChannel.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * 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.io.util;
-
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.ByteBuffer;
-import java.nio.channels.Channels;
-import java.nio.channels.WritableByteChannel;
-
-public class DataOutputStreamAndChannel extends DataOutputStreamPlus
-{
-    private final WritableByteChannel channel;
-    public DataOutputStreamAndChannel(OutputStream os, WritableByteChannel channel)
-    {
-        super(os);
-        this.channel = channel;
-    }
-    public DataOutputStreamAndChannel(WritableByteChannel channel)
-    {
-        this(Channels.newOutputStream(channel), channel);
-    }
-    public DataOutputStreamAndChannel(FileOutputStream fos)
-    {
-        this(fos, fos.getChannel());
-    }
-
-    public void write(ByteBuffer buffer) throws IOException
-    {
-        buffer = buffer.duplicate();
-        while (buffer.remaining() > 0)
-            channel.write(buffer);
-    }
-
-    public WritableByteChannel getChannel()
-    {
-        return channel;
-    }
-}
diff --git a/src/java/org/apache/cassandra/io/util/DataOutputStreamPlus.java b/src/java/org/apache/cassandra/io/util/DataOutputStreamPlus.java
index 6f2e21a..a846384 100644
--- a/src/java/org/apache/cassandra/io/util/DataOutputStreamPlus.java
+++ b/src/java/org/apache/cassandra/io/util/DataOutputStreamPlus.java
@@ -20,38 +20,116 @@
 import java.io.IOException;
 import java.io.OutputStream;
 import java.nio.ByteBuffer;
+import java.nio.channels.WritableByteChannel;
 
+import org.apache.cassandra.config.Config;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 /**
- * When possible use {@link DataOutputStreamAndChannel} instead of this class, as it will
- * be more efficient. This class is only for situations where it cannot be used
+ * Abstract base class for DataOutputStreams that accept writes from ByteBuffer or Memory and also provide
+ * access to the underlying WritableByteChannel associated with their output stream.
+ *
+ * If no channel is provided by derived classes then a wrapper channel is provided.
  */
-public class DataOutputStreamPlus extends AbstractDataOutput implements DataOutputPlus
+public abstract class DataOutputStreamPlus extends OutputStream implements DataOutputPlus
 {
-    protected final OutputStream out;
-    public DataOutputStreamPlus(OutputStream out)
+    //Dummy wrapper channel for derived implementations that don't have a channel
+    protected final WritableByteChannel channel;
+
+    protected DataOutputStreamPlus()
     {
-        this.out = out;
+        this.channel = newDefaultChannel();
     }
 
-    public void write(byte[] buffer, int offset, int count) throws IOException
+    protected DataOutputStreamPlus(WritableByteChannel channel)
     {
-        out.write(buffer, offset, count);
+        this.channel = channel;
     }
 
-    public void write(int oneByte) throws IOException
+    private static int MAX_BUFFER_SIZE =
+            Integer.getInteger(Config.PROPERTY_PREFIX + "data_output_stream_plus_temp_buffer_size", 8192);
+
+    /*
+     * Factored out into separate method to create more flexibility around inlining
+     */
+    protected static byte[] retrieveTemporaryBuffer(int minSize)
     {
-        out.write(oneByte);
+        byte[] bytes = tempBuffer.get();
+        if (bytes.length < Math.min(minSize, MAX_BUFFER_SIZE))
+        {
+            // increase in powers of 2, to avoid wasted repeat allocations
+            bytes = new byte[Math.min(MAX_BUFFER_SIZE, 2 * Integer.highestOneBit(minSize))];
+            tempBuffer.set(bytes);
+        }
+        return bytes;
     }
 
-    public void close() throws IOException
+    private static final ThreadLocal<byte[]> tempBuffer = new ThreadLocal<byte[]>()
     {
-        out.close();
+        @Override
+        public byte[] initialValue()
+        {
+            return new byte[16];
+        }
+    };
+
+    // Derived classes can override and *construct* a real channel, if it is not possible to provide one to the constructor
+    protected WritableByteChannel newDefaultChannel()
+    {
+        return new WritableByteChannel()
+        {
+
+            @Override
+            public boolean isOpen()
+            {
+                return true;
+            }
+
+            @Override
+            public void close() throws IOException
+            {
+            }
+
+            @Override
+            public int write(ByteBuffer src) throws IOException
+            {
+                int toWrite = src.remaining();
+
+                if (src.hasArray())
+                {
+                    DataOutputStreamPlus.this.write(src.array(), src.arrayOffset() + src.position(), src.remaining());
+                    src.position(src.limit());
+                    return toWrite;
+                }
+
+                if (toWrite < 16)
+                {
+                    int offset = src.position();
+                    for (int i = 0 ; i < toWrite ; i++)
+                        DataOutputStreamPlus.this.write(src.get(i + offset));
+                    src.position(src.limit());
+                    return toWrite;
+                }
+
+                byte[] buf = retrieveTemporaryBuffer(toWrite);
+
+                int totalWritten = 0;
+                while (totalWritten < toWrite)
+                {
+                    int toWriteThisTime = Math.min(buf.length, toWrite - totalWritten);
+
+                    ByteBufferUtil.arrayCopy(src, src.position() + totalWritten, buf, 0, toWriteThisTime);
+
+                    DataOutputStreamPlus.this.write(buf, 0, toWriteThisTime);
+
+                    totalWritten += toWriteThisTime;
+                }
+
+                src.position(src.limit());
+                return totalWritten;
+            }
+
+        };
     }
 
-    public void flush() throws IOException
-    {
-        out.flush();
-    }
 }
diff --git a/src/java/org/apache/cassandra/io/util/DiskAwareRunnable.java b/src/java/org/apache/cassandra/io/util/DiskAwareRunnable.java
deleted file mode 100644
index 1a15d6f..0000000
--- a/src/java/org/apache/cassandra/io/util/DiskAwareRunnable.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * 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.io.util;
-
-import java.io.IOException;
-
-import org.apache.cassandra.db.Directories;
-import org.apache.cassandra.io.FSWriteError;
-import org.apache.cassandra.utils.WrappedRunnable;
-
-public abstract class DiskAwareRunnable extends WrappedRunnable
-{
-    protected Directories.DataDirectory getWriteDirectory(long writeSize)
-    {
-        Directories.DataDirectory directory = getDirectories().getWriteableLocation(writeSize);
-        if (directory == null)
-            throw new FSWriteError(new IOException("Insufficient disk space to write " + writeSize + " bytes"), "");
-
-        return directory;
-    }
-
-    /**
-     * Get sstable directories for the CF.
-     * @return Directories instance for the CF.
-     */
-    protected abstract Directories getDirectories();
-}
diff --git a/src/java/org/apache/cassandra/io/util/FastByteArrayOutputStream.java b/src/java/org/apache/cassandra/io/util/FastByteArrayOutputStream.java
deleted file mode 100644
index c831508..0000000
--- a/src/java/org/apache/cassandra/io/util/FastByteArrayOutputStream.java
+++ /dev/null
@@ -1,266 +0,0 @@
-/*
- * 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.io.util;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.io.UnsupportedEncodingException;
-import java.nio.ByteBuffer;
-
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-/*
- * This file has been modified from Apache Harmony's ByteArrayOutputStream
- * implementation. The synchronized methods of the original have been
- * replaced by non-synchronized methods. This makes certain operations
- * much FASTer, but also *not thread-safe*.
- *
- * This file remains formatted the same as the Apache Harmony original to
- * make patching easier if any bug fixes are made to the Harmony version.
- */
-
-/**
- * A specialized {@link OutputStream} for class for writing content to an
- * (internal) byte array. As bytes are written to this stream, the byte array
- * may be expanded to hold more bytes. When the writing is considered to be
- * finished, a copy of the byte array can be requested from the class.
- *
- * @see ByteArrayOutputStream
- */
-public class FastByteArrayOutputStream extends OutputStream {
-    /**
-     * The byte array containing the bytes written.
-     */
-    protected byte[] buf;
-
-    /**
-     * The number of bytes written.
-     */
-    protected int count;
-
-    /**
-     * Constructs a new ByteArrayOutputStream with a default size of 32 bytes.
-     * If more than 32 bytes are written to this instance, the underlying byte
-     * array will expand.
-     */
-    public FastByteArrayOutputStream() {
-        buf = new byte[32];
-    }
-
-    /**
-     * Constructs a new {@code ByteArrayOutputStream} with a default size of
-     * {@code size} bytes. If more than {@code size} bytes are written to this
-     * instance, the underlying byte array will expand.
-     *
-     * @param size
-     *            initial size for the underlying byte array, must be
-     *            non-negative.
-     * @throws IllegalArgumentException
-     *             if {@code size} < 0.
-     */
-    public FastByteArrayOutputStream(int size) {
-        if (size >= 0) {
-            buf = new byte[size];
-        } else {
-            throw new IllegalArgumentException();
-        }
-    }
-
-    /**
-     * Closes this stream. This releases system resources used for this stream.
-     *
-     * @throws IOException
-     *             if an error occurs while attempting to close this stream.
-     */
-    @Override
-    public void close() throws IOException {
-        /**
-         * Although the spec claims "A closed stream cannot perform output
-         * operations and cannot be reopened.", this implementation must do
-         * nothing.
-         */
-        super.close();
-    }
-
-    private void expand(int i) {
-        /* Can the buffer handle @i more bytes, if not expand it */
-        if (count + i <= buf.length) {
-            return;
-        }
-
-        long expectedExtent = (count + i) * 2L; //long to deal with possible int overflow
-        int newSize = (int) Math.min(Integer.MAX_VALUE - 8, expectedExtent); // MAX_ARRAY_SIZE
-        byte[] newbuf = new byte[newSize];
-        System.arraycopy(buf, 0, newbuf, 0, count);
-        buf = newbuf;
-    }
-
-    /**
-     * Resets this stream to the beginning of the underlying byte array. All
-     * subsequent writes will overwrite any bytes previously stored in this
-     * stream.
-     */
-    public void reset() {
-        count = 0;
-    }
-
-    /**
-     * Returns the total number of bytes written to this stream so far.
-     *
-     * @return the number of bytes written to this stream.
-     */
-    public int size() {
-        return count;
-    }
-
-    /**
-     * Returns the contents of this ByteArrayOutputStream as a byte array. Any
-     * changes made to the receiver after returning will not be reflected in the
-     * byte array returned to the caller.
-     *
-     * @return this stream's current contents as a byte array.
-     */
-    public byte[] toByteArray() {
-        byte[] newArray = new byte[count];
-        System.arraycopy(buf, 0, newArray, 0, count);
-        return newArray;
-    }
-
-    /**
-     * Returns the contents of this ByteArrayOutputStream as a string. Any
-     * changes made to the receiver after returning will not be reflected in the
-     * string returned to the caller.
-     *
-     * @return this stream's current contents as a string.
-     */
-
-    @Override
-    public String toString() {
-        return new String(buf, 0, count);
-    }
-
-    /**
-     * Returns the contents of this ByteArrayOutputStream as a string. Each byte
-     * {@code b} in this stream is converted to a character {@code c} using the
-     * following function:
-     * {@code c == (char)(((hibyte & 0xff) << 8) | (b & 0xff))}. This method is
-     * deprecated and either {@link #toString()} or {@link #toString(String)}
-     * should be used.
-     *
-     * @param hibyte
-     *            the high byte of each resulting Unicode character.
-     * @return this stream's current contents as a string with the high byte set
-     *         to {@code hibyte}.
-     * @deprecated Use {@link #toString()}.
-     */
-    @Deprecated
-    public String toString(int hibyte) {
-        char[] newBuf = new char[size()];
-        for (int i = 0; i < newBuf.length; i++) {
-            newBuf[i] = (char) (((hibyte & 0xff) << 8) | (buf[i] & 0xff));
-        }
-        return new String(newBuf);
-    }
-
-    /**
-     * Returns the contents of this ByteArrayOutputStream as a string converted
-     * according to the encoding declared in {@code enc}.
-     *
-     * @param enc
-     *            a string representing the encoding to use when translating
-     *            this stream to a string.
-     * @return this stream's current contents as an encoded string.
-     * @throws UnsupportedEncodingException
-     *             if the provided encoding is not supported.
-     */
-    public String toString(String enc) throws UnsupportedEncodingException {
-        return new String(buf, 0, count, enc);
-    }
-
-    /**
-     * Writes {@code count} bytes from the byte array {@code buffer} starting at
-     * offset {@code index} to this stream.
-     *
-     * @param buffer
-     *            the buffer to be written.
-     * @param offset
-     *            the initial position in {@code buffer} to retrieve bytes.
-     * @param len
-     *            the number of bytes of {@code buffer} to write.
-     * @throws NullPointerException
-     *             if {@code buffer} is {@code null}.
-     * @throws IndexOutOfBoundsException
-     *             if {@code offset < 0} or {@code len < 0}, or if
-     *             {@code offset + len} is greater than the length of
-     *             {@code buffer}.
-     */
-    @Override
-    public void write(byte[] buffer, int offset, int len) {
-        // avoid int overflow
-        if (offset < 0 || offset > buffer.length || len < 0
-                || len > buffer.length - offset
-                || this.count + len < 0) {
-            throw new IndexOutOfBoundsException();
-        }
-        if (len == 0) {
-            return;
-        }
-
-        /* Expand if necessary */
-        expand(len);
-        System.arraycopy(buffer, offset, buf, this.count, len);
-        this.count += len;
-    }
-
-    public void write(ByteBuffer buffer)
-    {
-        int len = buffer.remaining();
-        expand(len);
-        ByteBufferUtil.arrayCopy(buffer, buffer.position(), buf, this.count, len);
-        this.count += len;
-    }
-
-    /**
-     * Writes the specified byte {@code oneByte} to the OutputStream. Only the
-     * low order byte of {@code oneByte} is written.
-     *
-     * @param oneByte
-     *            the byte to be written.
-     */
-    @Override
-    public void write(int oneByte) {
-        if (count == buf.length) {
-            expand(1);
-        }
-        buf[count++] = (byte) oneByte;
-    }
-
-    /**
-     * Takes the contents of this stream and writes it to the output stream
-     * {@code out}.
-     *
-     * @param out
-     *            an OutputStream on which to write the contents of this stream.
-     * @throws IOException
-     *             if an error occurs while writing to {@code out}.
-     */
-    public void writeTo(OutputStream out) throws IOException {
-        out.write(buf, 0, count);
-    }
-}
diff --git a/src/java/org/apache/cassandra/io/util/FileUtils.java b/src/java/org/apache/cassandra/io/util/FileUtils.java
index f69ed01..8d122dd 100644
--- a/src/java/org/apache/cassandra/io/util/FileUtils.java
+++ b/src/java/org/apache/cassandra/io/util/FileUtils.java
@@ -17,23 +17,15 @@
  */
 package org.apache.cassandra.io.util;
 
-import java.io.Closeable;
-import java.io.DataInput;
-import java.io.EOFException;
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.RandomAccessFile;
+import java.io.*;
 import java.nio.ByteBuffer;
-import java.nio.MappedByteBuffer;
-import java.nio.file.AtomicMoveNotSupportedException;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.StandardCopyOption;
+import java.nio.channels.FileChannel;
+import java.nio.file.*;
 import java.text.DecimalFormat;
 import java.util.Arrays;
 import java.util.concurrent.atomic.AtomicReference;
 
+import org.apache.cassandra.config.Config;
 import sun.nio.ch.DirectBuffer;
 
 import org.slf4j.Logger;
@@ -47,6 +39,9 @@
 import org.apache.cassandra.io.sstable.CorruptSSTableException;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 
+import static org.apache.cassandra.utils.Throwables.maybeFail;
+import static org.apache.cassandra.utils.Throwables.merge;
+
 public final class FileUtils
 {
     private static final Logger logger = LoggerFactory.getLogger(FileUtils.class);
@@ -115,6 +110,34 @@
         return createTempFile(prefix, suffix, new File(System.getProperty("java.io.tmpdir")));
     }
 
+    public static Throwable deleteWithConfirm(String filePath, boolean expect, Throwable accumulate)
+    {
+        return deleteWithConfirm(new File(filePath), expect, accumulate);
+    }
+
+    public static Throwable deleteWithConfirm(File file, boolean expect, Throwable accumulate)
+    {
+        boolean exists = file.exists();
+        assert exists || !expect : "attempted to delete non-existing file " + file.getName();
+        try
+        {
+            if (exists)
+                Files.delete(file.toPath());
+        }
+        catch (Throwable t)
+        {
+            try
+            {
+                throw new FSWriteError(t, file);
+            }
+            catch (Throwable t2)
+            {
+                accumulate = merge(accumulate, t2);
+            }
+        }
+        return accumulate;
+    }
+
     public static void deleteWithConfirm(String file)
     {
         deleteWithConfirm(new File(file));
@@ -122,17 +145,7 @@
 
     public static void deleteWithConfirm(File file)
     {
-        assert file.exists() : "attempted to delete non-existing file " + file.getName();
-        if (logger.isDebugEnabled())
-            logger.debug("Deleting {}", file.getName());
-        try
-        {
-            Files.delete(file.toPath());
-        }
-        catch (IOException e)
-        {
-            throw new FSWriteError(e, file);
-        }
+        maybeFail(deleteWithConfirm(file, true, null));
     }
 
     public static void renameWithOutConfirm(String from, String to)
@@ -155,9 +168,9 @@
 
     public static void renameWithConfirm(File from, File to)
     {
-        assert from.exists();
-        if (logger.isDebugEnabled())
-            logger.debug((String.format("Renaming %s to %s", from.getPath(), to.getPath())));
+        assert from.exists() : from + " should exist";
+        if (logger.isTraceEnabled())
+            logger.trace((String.format("Renaming %s to %s", from.getPath(), to.getPath())));
         // this is not FSWE because usually when we see it it's because we didn't close the file before renaming it,
         // and Windows is picky about that.
         try
@@ -184,35 +197,20 @@
         }
         catch (AtomicMoveNotSupportedException e)
         {
-            logger.debug("Could not do an atomic move", e);
+            logger.trace("Could not do an atomic move", e);
             Files.move(from, to, StandardCopyOption.REPLACE_EXISTING);
         }
 
     }
     public static void truncate(String path, long size)
     {
-        RandomAccessFile file;
-
-        try
+        try(FileChannel channel = FileChannel.open(Paths.get(path), StandardOpenOption.READ, StandardOpenOption.WRITE))
         {
-            file = new RandomAccessFile(path, "rw");
-        }
-        catch (FileNotFoundException e)
-        {
-            throw new RuntimeException(e);
-        }
-
-        try
-        {
-            file.getChannel().truncate(size);
+            channel.truncate(size);
         }
         catch (IOException e)
         {
-            throw new FSWriteError(e, path);
-        }
-        finally
-        {
-            closeQuietly(file);
+            throw new RuntimeException(e);
         }
     }
 
@@ -283,9 +281,10 @@
         return canCleanDirectBuffers;
     }
 
-    public static void clean(MappedByteBuffer buffer)
+    public static void clean(ByteBuffer buffer)
     {
-        ((DirectBuffer) buffer).cleaner().clean();
+        if (isCleanerAvailable() && buffer.isDirect())
+            ((DirectBuffer)buffer).cleaner().clean();
     }
 
     public static void createDirectory(String directory)
@@ -380,6 +379,23 @@
         deleteWithConfirm(dir);
     }
 
+    /**
+     * Schedules deletion of all file and subdirectories under "dir" on JVM shutdown.
+     * @param dir Directory to be deleted
+     */
+    public static void deleteRecursiveOnExit(File dir)
+    {
+        if (dir.isDirectory())
+        {
+            String[] children = dir.list();
+            for (String child : children)
+                deleteRecursiveOnExit(new File(dir, child));
+        }
+
+        logger.trace("Scheduling deferred deletion of file: " + dir);
+        dir.deleteOnExit();
+    }
+
     public static void skipBytesFully(DataInput in, int bytes) throws IOException
     {
         int n = 0;
@@ -423,6 +439,41 @@
         return length;
     }
 
+    public static void copyTo(DataInput in, OutputStream out, int length) throws IOException
+    {
+        byte[] buffer = new byte[64 * 1024];
+        int copiedBytes = 0;
+
+        while (copiedBytes + buffer.length < length)
+        {
+            in.readFully(buffer);
+            out.write(buffer);
+            copiedBytes += buffer.length;
+        }
+
+        if (copiedBytes < length)
+        {
+            int left = length - copiedBytes;
+            in.readFully(buffer, 0, left);
+            out.write(buffer, 0, left);
+        }
+    }
+
+    public static boolean isSubDirectory(File parent, File child) throws IOException
+    {
+        parent = parent.getCanonicalFile();
+        child = child.getCanonicalFile();
+
+        File toCheck = child;
+        while (toCheck != null)
+        {
+            if (parent.equals(toCheck))
+                return true;
+            toCheck = toCheck.getParentFile();
+        }
+        return false;
+    }
+
     public static void setFSErrorHandler(FSErrorHandler handler)
     {
         fsErrorHandler.getAndSet(handler);
diff --git a/src/java/org/apache/cassandra/io/util/IAllocator.java b/src/java/org/apache/cassandra/io/util/IAllocator.java
deleted file mode 100644
index d633617..0000000
--- a/src/java/org/apache/cassandra/io/util/IAllocator.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/*
- * 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.io.util;
-
-public interface IAllocator
-{
-    long allocate(long size);
-    void free(long peer);
-}
diff --git a/src/java/org/apache/cassandra/io/util/ICompressedFile.java b/src/java/org/apache/cassandra/io/util/ICompressedFile.java
index 3ca7718..ce7b22c 100644
--- a/src/java/org/apache/cassandra/io/util/ICompressedFile.java
+++ b/src/java/org/apache/cassandra/io/util/ICompressedFile.java
@@ -17,9 +17,14 @@
  */
 package org.apache.cassandra.io.util;
 
+import java.nio.MappedByteBuffer;
+import java.util.TreeMap;
+
 import org.apache.cassandra.io.compress.CompressionMetadata;
 
 public interface ICompressedFile
 {
+    public ChannelProxy channel();
     public CompressionMetadata getMetadata();
+    public TreeMap<Long, MappedByteBuffer> chunkSegments();
 }
diff --git a/src/java/org/apache/cassandra/io/util/JEMallocAllocator.java b/src/java/org/apache/cassandra/io/util/JEMallocAllocator.java
deleted file mode 100644
index 07d19fe..0000000
--- a/src/java/org/apache/cassandra/io/util/JEMallocAllocator.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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.io.util;
-
-import com.sun.jna.Library;
-import com.sun.jna.Native;
-
-public class JEMallocAllocator implements IAllocator
-{
-    public interface JEMLibrary extends Library
-    {
-        long malloc(long size);
-
-        void free(long pointer);
-    }
-
-    private final JEMLibrary library;
-    
-    public JEMallocAllocator()
-    {
-        library = (JEMLibrary) Native.loadLibrary("jemalloc", JEMLibrary.class);
-    }
-
-    public long allocate(long size)
-    {
-        return library.malloc(size);
-    }
-
-    public void free(long peer)
-    {
-        library.free(peer);
-    }
-}
diff --git a/src/java/org/apache/cassandra/io/util/MappedFileDataInput.java b/src/java/org/apache/cassandra/io/util/MappedFileDataInput.java
deleted file mode 100644
index d056240..0000000
--- a/src/java/org/apache/cassandra/io/util/MappedFileDataInput.java
+++ /dev/null
@@ -1,184 +0,0 @@
-/*
- * 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.io.util;
-
-import java.io.*;
-import java.nio.ByteBuffer;
-import java.nio.MappedByteBuffer;
-import java.nio.channels.FileChannel;
-
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-public class MappedFileDataInput extends AbstractDataInput implements FileDataInput
-{
-    private final MappedByteBuffer buffer;
-    private final String filename;
-    private final long segmentOffset;
-    private int position;
-
-    public MappedFileDataInput(MappedByteBuffer buffer, String filename, long segmentOffset, int position)
-    {
-        assert buffer != null;
-        this.buffer = buffer;
-        this.filename = filename;
-        this.segmentOffset = segmentOffset;
-        this.position = position;
-    }
-
-    // don't make this public, this is only for seeking WITHIN the current mapped segment
-    protected void seekInternal(int pos)
-    {
-        position = pos;
-    }
-
-    // Only use when we know the seek in within the mapped segment. Throws an
-    // IOException otherwise.
-    public void seek(long pos) throws IOException
-    {
-        long inSegmentPos = pos - segmentOffset;
-        if (inSegmentPos < 0 || inSegmentPos > buffer.capacity())
-            throw new IOException(String.format("Seek position %d is not within mmap segment (seg offs: %d, length: %d)", pos, segmentOffset, buffer.capacity()));
-
-        seekInternal((int) inSegmentPos);
-    }
-
-    public long getFilePointer()
-    {
-        return segmentOffset + (long)position;
-    }
-
-    protected int getPosition()
-    {
-        return position;
-    }
-
-    @Override
-    public boolean markSupported()
-    {
-        return false;
-    }
-
-    public void reset(FileMark mark) throws IOException
-    {
-        assert mark instanceof MappedFileDataInputMark;
-        seekInternal(((MappedFileDataInputMark) mark).position);
-    }
-
-    public FileMark mark()
-    {
-        return new MappedFileDataInputMark(position);
-    }
-
-    public long bytesPastMark(FileMark mark)
-    {
-        assert mark instanceof MappedFileDataInputMark;
-        assert position >= ((MappedFileDataInputMark) mark).position;
-        return position - ((MappedFileDataInputMark) mark).position;
-    }
-
-    public boolean isEOF() throws IOException
-    {
-        return position == buffer.capacity();
-    }
-
-    public long bytesRemaining() throws IOException
-    {
-        return buffer.capacity() - position;
-    }
-
-    public String getPath()
-    {
-        return filename;
-    }
-
-    public int read() throws IOException
-    {
-        if (isEOF())
-            return -1;
-        return buffer.get(position++) & 0xFF;
-    }
-
-    /**
-     * Does the same thing as <code>readFully</code> do but without copying data (thread safe)
-     * @param length length of the bytes to read
-     * @return buffer with portion of file content
-     * @throws IOException on any fail of I/O operation
-     */
-    public ByteBuffer readBytes(int length) throws IOException
-    {
-        int remaining = buffer.remaining() - position;
-        if (length > remaining)
-            throw new IOException(String.format("mmap segment underflow; remaining is %d but %d requested",
-                                                remaining, length));
-
-        if (length == 0)
-            return ByteBufferUtil.EMPTY_BYTE_BUFFER;
-
-        ByteBuffer bytes = buffer.duplicate();
-        bytes.position(buffer.position() + position).limit(buffer.position() + position + length);
-        position += length;
-
-        // we have to copy the data in case we unreference the underlying sstable.  See CASSANDRA-3179
-        ByteBuffer clone = ByteBuffer.allocate(bytes.remaining());
-        clone.put(bytes);
-        clone.flip();
-        return clone;
-    }
-
-    @Override
-    public final void readFully(byte[] bytes) throws IOException
-    {
-        ByteBufferUtil.arrayCopy(buffer, buffer.position() + position, bytes, 0, bytes.length);
-        position += bytes.length;
-    }
-
-    @Override
-    public final void readFully(byte[] buffer, int offset, int count) throws IOException
-    {
-        throw new UnsupportedOperationException("use readBytes instead");
-    }
-
-    public int skipBytes(int n) throws IOException
-    {
-        assert n >= 0 : "skipping negative bytes is illegal: " + n;
-        if (n == 0)
-            return 0;
-        int oldPosition = position;
-        assert ((long)oldPosition) + n <= Integer.MAX_VALUE;
-        position = Math.min(buffer.capacity(), position + n);
-        return position - oldPosition;
-    }
-
-    private static class MappedFileDataInputMark implements FileMark
-    {
-        int position;
-
-        MappedFileDataInputMark(int position)
-        {
-            this.position = position;
-        }
-    }
-
-    @Override
-    public String toString() {
-        return getClass().getSimpleName() + "(" +
-               "filename='" + filename + "'" +
-               ", position=" + position +
-               ")";
-    }
-}
diff --git a/src/java/org/apache/cassandra/io/util/Memory.java b/src/java/org/apache/cassandra/io/util/Memory.java
index bbe6049..07d3ca3 100644
--- a/src/java/org/apache/cassandra/io/util/Memory.java
+++ b/src/java/org/apache/cassandra/io/util/Memory.java
@@ -17,13 +17,11 @@
  */
 package org.apache.cassandra.io.util;
 
-import java.io.Closeable;
+import java.lang.reflect.Field;
 import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
 
-import com.sun.jna.Native;
 import net.nicoulaj.compilecommand.annotations.Inline;
-import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.utils.FastByteOperations;
 import org.apache.cassandra.utils.concurrent.Ref;
 import org.apache.cassandra.utils.memory.MemoryUtil;
@@ -35,13 +33,28 @@
  */
 public class Memory implements AutoCloseable
 {
-    private static final Unsafe unsafe = NativeAllocator.unsafe;
-    static final IAllocator allocator = DatabaseDescriptor.getoffHeapMemoryAllocator();
+    private static final Unsafe unsafe;
+    static
+    {
+        try
+        {
+            Field field = sun.misc.Unsafe.class.getDeclaredField("theUnsafe");
+            field.setAccessible(true);
+            unsafe = (sun.misc.Unsafe) field.get(null);
+        }
+        catch (Exception e)
+        {
+            throw new AssertionError(e);
+        }
+    }
+
     private static final long BYTE_ARRAY_BASE_OFFSET = unsafe.arrayBaseOffset(byte[].class);
 
     private static final boolean bigEndian = ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN);
     private static final boolean unaligned;
 
+    public static final ByteBuffer[] NO_BYTE_BUFFERS = new ByteBuffer[0];
+
     static
     {
         String arch = System.getProperty("os.arch");
@@ -58,7 +71,7 @@
         if (bytes <= 0)
             throw new AssertionError();
         size = bytes;
-        peer = allocator.allocate(size);
+        peer = MemoryUtil.allocate(size);
         // we permit a 0 peer iff size is zero, since such an allocation makes no sense, and an allocator would be
         // justified in returning a null pointer (and permitted to do so: http://www.cplusplus.com/reference/cstdlib/malloc)
         if (peer == 0)
@@ -169,7 +182,7 @@
 
     public void setShort(long offset, short l)
     {
-        checkBounds(offset, offset + 4);
+        checkBounds(offset, offset + 2);
         if (unaligned)
         {
             unsafe.putShort(peer + offset, l);
@@ -232,9 +245,7 @@
         else if (count == 0)
             return;
 
-        long end = memoryOffset + count;
-        checkBounds(memoryOffset, end);
-
+        checkBounds(memoryOffset, memoryOffset + count);
         unsafe.copyMemory(buffer, BYTE_ARRAY_BASE_OFFSET + bufferOffset, null, peer + memoryOffset, count);
     }
 
@@ -344,7 +355,7 @@
 
     public void free()
     {
-        if (peer != 0) allocator.free(peer);
+        if (peer != 0) MemoryUtil.free(peer);
         else assert size == 0;
         peer = 0;
     }
@@ -375,8 +386,9 @@
 
     public ByteBuffer[] asByteBuffers(long offset, long length)
     {
+        checkBounds(offset, offset + length);
         if (size() == 0)
-            return new ByteBuffer[0];
+            return NO_BYTE_BUFFERS;
 
         ByteBuffer[] result = new ByteBuffer[(int) (length / Integer.MAX_VALUE) + 1];
         int size = (int) (size() / result.length);
@@ -390,6 +402,19 @@
         return result;
     }
 
+    public ByteBuffer asByteBuffer(long offset, int length)
+    {
+        checkBounds(offset, offset + length);
+        return MemoryUtil.getByteBuffer(peer + offset, length);
+    }
+
+    // MUST provide a buffer created via MemoryUtil.getHollowDirectByteBuffer()
+    public void setByteBuffer(ByteBuffer buffer, long offset, int length)
+    {
+        checkBounds(offset, offset + length);
+        MemoryUtil.setByteBuffer(buffer, peer + offset, length);
+    }
+
     public String toString()
     {
         return toString(peer, size);
diff --git a/src/java/org/apache/cassandra/io/util/MemoryInputStream.java b/src/java/org/apache/cassandra/io/util/MemoryInputStream.java
index eee030a..45261e0 100644
--- a/src/java/org/apache/cassandra/io/util/MemoryInputStream.java
+++ b/src/java/org/apache/cassandra/io/util/MemoryInputStream.java
@@ -17,9 +17,10 @@
  */
 package org.apache.cassandra.io.util;
 
+import java.io.DataInput;
 import java.io.IOException;
 
-public class MemoryInputStream extends AbstractDataInput
+public class MemoryInputStream extends AbstractDataInput implements DataInput
 {
     private final Memory mem;
     private int position = 0;
@@ -40,20 +41,24 @@
         position += count;
     }
 
-    protected void seekInternal(int pos)
+    public void seek(long pos)
     {
-        position = pos;
+        position = (int) pos;
     }
 
-    protected int getPosition()
+    public long getPosition()
     {
         return position;
     }
 
-    public int skipBytes(int n) throws IOException
+    public long getPositionLimit()
     {
-        seekInternal(getPosition() + n);
-        return position;
+        return mem.size();
+    }
+
+    protected long length()
+    {
+        return mem.size();
     }
 
     public void close()
diff --git a/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java b/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
index 1109d37..70ac77a 100644
--- a/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
+++ b/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java
@@ -30,7 +30,6 @@
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.RowIndexEntry;
-import org.apache.cassandra.io.FSReadError;
 import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.sstable.IndexSummary;
@@ -51,9 +50,9 @@
      */
     private final Segment[] segments;
 
-    public MmappedSegmentedFile(String path, long length, Segment[] segments)
+    public MmappedSegmentedFile(ChannelProxy channel, long length, Segment[] segments)
     {
-        super(new Cleanup(path, segments), path, length);
+        super(new Cleanup(channel, segments), channel, length);
         this.segments = segments;
     }
 
@@ -73,10 +72,10 @@
      */
     private Segment floor(long position)
     {
-        assert 0 <= position && position < length: String.format("%d >= %d in %s", position, length, path);
+        assert 0 <= position && position < length: String.format("%d >= %d in %s", position, length, path());
         Segment seg = new Segment(position, null);
         int idx = Arrays.binarySearch(segments, seg);
-        assert idx != -1 : String.format("Bad position %d for segments %s in %s", position, Arrays.toString(segments), path);
+        assert idx != -1 : String.format("Bad position %d for segments %s in %s", position, Arrays.toString(segments), path());
         if (idx < 0)
             // round down to entry at insertion point
             idx = -(idx + 2);
@@ -92,28 +91,40 @@
         if (segment.right != null)
         {
             // segment is mmap'd
-            return new MappedFileDataInput(segment.right, path, segment.left, (int) (position - segment.left));
+            return new ByteBufferDataInput(segment.right, path(), segment.left, (int) (position - segment.left));
         }
 
         // we can have single cells or partitions larger than 2Gb, which is our maximum addressable range in a single segment;
         // in this case we open as a normal random access reader
         // FIXME: brafs are unbounded, so this segment will cover the rest of the file, rather than just the row
-        RandomAccessReader file = RandomAccessReader.open(new File(path));
+        RandomAccessReader file = RandomAccessReader.open(channel);
         file.seek(position);
         return file;
     }
 
+    @Override
+    public long[] copyReadableBounds()
+    {
+        long[] bounds  = new long[segments.length + 1];
+        for (int i = 0; i < segments.length; i++)
+            bounds[i] = segments[i].left;
+        bounds[segments.length] = length;
+        return bounds;
+    }
+
     private static final class Cleanup extends SegmentedFile.Cleanup
     {
         final Segment[] segments;
-        protected Cleanup(String path, Segment[] segments)
+        protected Cleanup(ChannelProxy channel, Segment[] segments)
         {
-            super(path);
+            super(channel);
             this.segments = segments;
         }
 
         public void tidy()
         {
+            super.tidy();
+
             if (!FileUtils.isCleanerAvailable())
                 return;
 
@@ -130,7 +141,7 @@
                         continue;
                     FileUtils.clean(segment.right);
                 }
-                logger.debug("All segments have been unmapped successfully");
+                logger.trace("All segments have been unmapped successfully");
             }
             catch (Exception e)
             {
@@ -164,6 +175,7 @@
         if (dbuilder instanceof Builder)
             ((Builder) dbuilder).boundaries.clear();
 
+        RowIndexEntry.IndexSerializer rowIndexEntrySerializer = descriptor.version.getSSTableFormat().getIndexSerializer(metadata);
         try (RandomAccessFile raf = new RandomAccessFile(descriptor.filenameFor(Component.PRIMARY_INDEX), "r");)
         {
             long iprev = 0, dprev = 0;
@@ -173,7 +185,7 @@
                 long icur = indexSummary.getPosition(i);
                 raf.seek(icur);
                 ByteBufferUtil.readWithShortLength(raf);
-                RowIndexEntry rie = metadata.comparator.rowIndexEntrySerializer().deserialize(raf, descriptor.version);
+                RowIndexEntry rie = rowIndexEntrySerializer.deserialize(raf, descriptor.version);
                 long dcur = rie.position;
 
                 // if these positions are small enough to map out a segment from the prior version (i.e. less than 2Gb),
@@ -190,7 +202,7 @@
                         ibuilder.addPotentialBoundary(raf.getFilePointer());
                         // then read the RIE, and add its data file position as a boundary for the data file
                         ByteBufferUtil.readWithShortLength(raf);
-                        rie = metadata.comparator.rowIndexEntrySerializer().deserialize(raf, descriptor.version);
+                        rie = rowIndexEntrySerializer.deserialize(raf, descriptor.version);
                         dbuilder.addPotentialBoundary(rie.position);
                     }
                 }
@@ -357,52 +369,27 @@
             boundaries.addCandidate(boundary);
         }
 
-        public SegmentedFile complete(String path, long overrideLength, boolean isFinal)
+        public SegmentedFile complete(ChannelProxy channel, long overrideLength)
         {
-            assert !isFinal || overrideLength <= 0;
-            long length = overrideLength > 0 ? overrideLength : new File(path).length();
+            long length = overrideLength > 0 ? overrideLength : channel.size();
             // create the segments
-            return new MmappedSegmentedFile(path, length, createSegments(path, length, isFinal));
-        }
 
-        private Segment[] createSegments(String path, long length, boolean isFinal)
-        {
-            RandomAccessFile raf;
-            try
-            {
-                raf = new RandomAccessFile(path, "r");
-            }
-            catch (IOException e)
-            {
-                throw new RuntimeException(e);
-            }
-
-            long[] boundaries = this.boundaries.finish(length, isFinal);
+            long[] boundaries = this.boundaries.finish(length, overrideLength <= 0);
 
             int segcount = boundaries.length - 1;
             Segment[] segments = new Segment[segcount];
 
-            try
+            for (int i = 0; i < segcount; i++)
             {
-                for (int i = 0; i < segcount; i++)
-                {
-                    long start = boundaries[i];
-                    long size = boundaries[i + 1] - start;
-                    MappedByteBuffer segment = size <= MAX_SEGMENT_SIZE
-                                               ? raf.getChannel().map(FileChannel.MapMode.READ_ONLY, start, size)
-                                               : null;
-                    segments[i] = new Segment(start, segment);
-                }
+                long start = boundaries[i];
+                long size = boundaries[i + 1] - start;
+                MappedByteBuffer segment = size <= MAX_SEGMENT_SIZE
+                                           ? channel.map(FileChannel.MapMode.READ_ONLY, start, size)
+                                           : null;
+                segments[i] = new Segment(start, segment);
             }
-            catch (IOException e)
-            {
-                throw new FSReadError(e, path);
-            }
-            finally
-            {
-                FileUtils.closeQuietly(raf);
-            }
-            return segments;
+
+            return new MmappedSegmentedFile(channel, length, segments);
         }
 
         @Override
diff --git a/src/java/org/apache/cassandra/io/util/NIODataInputStream.java b/src/java/org/apache/cassandra/io/util/NIODataInputStream.java
new file mode 100644
index 0000000..ebeb8ba
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/util/NIODataInputStream.java
@@ -0,0 +1,314 @@
+/*
+ * 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.io.util;
+
+import java.io.Closeable;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.SeekableByteChannel;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Rough equivalent of BufferedInputStream and DataInputStream wrapping the input stream of a File or Socket
+ * Created to work around the fact that when BIS + DIS delegate to NIO for socket IO they will allocate large
+ * thread local direct byte buffers when a large array is used to read.
+ *
+ * There may also be some performance improvement due to using a DBB as the underlying buffer for IO and the removal
+ * of some indirection and delegation when it comes to reading out individual values, but that is not the goal.
+ *
+ * Closing NIODataInputStream will invoke close on the ReadableByteChannel provided at construction.
+ *
+ * NIODataInputStream is not thread safe.
+ */
+public class NIODataInputStream extends InputStream implements DataInput, Closeable
+{
+    private final ReadableByteChannel rbc;
+    private ByteBuffer buf;
+
+
+    public NIODataInputStream(ReadableByteChannel rbc, int bufferSize)
+    {
+        Preconditions.checkNotNull(rbc);
+        Preconditions.checkArgument(bufferSize >= 8, "Buffer size must be large enough to accomadate a long/double");
+        this.rbc = rbc;
+        buf = ByteBuffer.allocateDirect(bufferSize);
+        buf.position(0);
+        buf.limit(0);
+    }
+
+    @Override
+    public void readFully(byte[] b) throws IOException
+    {
+        readFully(b, 0, b.length);
+    }
+
+
+    @Override
+    public void readFully(byte[] b, int off, int len) throws IOException
+    {
+        int copied = 0;
+        while (copied < len)
+        {
+            int read = read(b, off + copied, len - copied);
+            if (read < 0)
+                throw new EOFException();
+            copied += read;
+        }
+    }
+
+    @Override
+    public int read(byte b[], int off, int len) throws IOException {
+        if (b == null)
+            throw new NullPointerException();
+
+        // avoid int overflow
+        if (off < 0 || off > b.length || len < 0
+                || len > b.length - off)
+            throw new IndexOutOfBoundsException();
+
+        if (len == 0)
+            return 0;
+
+        int copied = 0;
+        while (copied < len)
+        {
+            if (buf.hasRemaining())
+            {
+                int toCopy = Math.min(len - copied, buf.remaining());
+                buf.get(b, off + copied, toCopy);
+                copied += toCopy;
+            }
+            else
+            {
+                int read = readNext();
+                if (read < 0 && copied == 0) return -1;
+                if (read <= 0) return copied;
+            }
+        }
+
+        return copied;
+    }
+
+    /*
+     * Refill the buffer, preserving any unread bytes remaining in the buffer
+     */
+    private int readNext() throws IOException
+    {
+        Preconditions.checkState(buf.remaining() != buf.capacity());
+        assert(buf.remaining() < 8);
+
+        /*
+         * If there is data already at the start of the buffer, move the position to the end
+         * If there is data but not at the start, move it to the start
+         * Otherwise move the position to 0 so writes start at the beginning of the buffer
+         *
+         * We go to the trouble of shuffling the bytes remaining for cases where the buffer isn't fully drained
+         * while retrieving a multi-byte value while the position is in the middle.
+         */
+        if (buf.position() == 0 && buf.hasRemaining())
+        {
+            buf.position(buf.limit());
+        }
+        else if (buf.hasRemaining())
+        {
+            ByteBuffer dup = buf.duplicate();
+            buf.clear();
+            buf.put(dup);
+        }
+        else
+        {
+            buf.position(0);
+        }
+
+        buf.limit(buf.capacity());
+
+        int read = 0;
+        while ((read = rbc.read(buf)) == 0) {}
+
+        buf.flip();
+
+        return read;
+    }
+
+    /*
+     * Read at least minimum bytes and throw EOF if that fails
+     */
+    private void readMinimum(int minimum) throws IOException
+    {
+        assert(buf.remaining() < 8);
+        while (buf.remaining() < minimum)
+        {
+            int read = readNext();
+            if (read == -1)
+            {
+                //DataInputStream consumes the bytes even if it doesn't get the entire value, match the behavior here
+                buf.position(0);
+                buf.limit(0);
+                throw new EOFException();
+            }
+        }
+    }
+
+    /*
+     * Ensure the buffer contains the minimum number of readable bytes
+     */
+    private void prepareReadPrimitive(int minimum) throws IOException
+    {
+        if (buf.remaining() < minimum) readMinimum(minimum);
+    }
+
+    @Override
+    public int skipBytes(int n) throws IOException
+    {
+        int skipped = 0;
+
+        while (skipped < n)
+        {
+            int skippedThisTime = (int)skip(n - skipped);
+            if (skippedThisTime <= 0) break;
+            skipped += skippedThisTime;
+        }
+
+        return skipped;
+    }
+
+    @Override
+    public boolean readBoolean() throws IOException
+    {
+        prepareReadPrimitive(1);
+        return buf.get() != 0;
+    }
+
+    @Override
+    public byte readByte() throws IOException
+    {
+        prepareReadPrimitive(1);
+        return buf.get();
+    }
+
+    @Override
+    public int readUnsignedByte() throws IOException
+    {
+        prepareReadPrimitive(1);
+        return buf.get() & 0xff;
+    }
+
+    @Override
+    public short readShort() throws IOException
+    {
+        prepareReadPrimitive(2);
+        return buf.getShort();
+    }
+
+    @Override
+    public int readUnsignedShort() throws IOException
+    {
+        return readShort() & 0xFFFF;
+    }
+
+    @Override
+    public char readChar() throws IOException
+    {
+        prepareReadPrimitive(2);
+        return buf.getChar();
+    }
+
+    @Override
+    public int readInt() throws IOException
+    {
+        prepareReadPrimitive(4);
+        return buf.getInt();
+    }
+
+    @Override
+    public long readLong() throws IOException
+    {
+        prepareReadPrimitive(8);
+        return buf.getLong();
+    }
+
+    @Override
+    public float readFloat() throws IOException
+    {
+        prepareReadPrimitive(4);
+        return buf.getFloat();
+    }
+
+    @Override
+    public double readDouble() throws IOException
+    {
+        prepareReadPrimitive(8);
+        return buf.getDouble();
+    }
+
+    @Override
+    public String readLine() throws IOException
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public String readUTF() throws IOException
+    {
+        return DataInputStream.readUTF(this);
+    }
+
+    @Override
+    public void close() throws IOException
+    {
+        rbc.close();
+        FileUtils.clean(buf);
+        buf = null;
+    }
+
+    @Override
+    public int read() throws IOException
+    {
+        return readUnsignedByte();
+    }
+
+    @Override
+    public int available() throws IOException
+    {
+        if (rbc instanceof SeekableByteChannel)
+        {
+            SeekableByteChannel sbc = (SeekableByteChannel)rbc;
+            long remainder = Math.max(0, sbc.size() - sbc.position());
+            return (remainder > Integer.MAX_VALUE) ? Integer.MAX_VALUE : (int)(remainder + buf.remaining());
+        }
+        return buf.remaining();
+    }
+
+    @Override
+    public void reset() throws IOException
+    {
+        throw new IOException("mark/reset not supported");
+    }
+
+    @Override
+    public boolean markSupported()
+    {
+        return false;
+    }
+}
diff --git a/src/java/org/apache/cassandra/io/util/NativeAllocator.java b/src/java/org/apache/cassandra/io/util/NativeAllocator.java
deleted file mode 100644
index da0b362..0000000
--- a/src/java/org/apache/cassandra/io/util/NativeAllocator.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.io.util;
-
-import java.lang.reflect.Field;
-
-import sun.misc.Unsafe;
-
-public class NativeAllocator implements IAllocator
-{
-    static final Unsafe unsafe;
-    static
-    {
-        try
-        {
-            Field field = sun.misc.Unsafe.class.getDeclaredField("theUnsafe");
-            field.setAccessible(true);
-            unsafe = (sun.misc.Unsafe) field.get(null);
-        }
-        catch (Exception e)
-        {
-            throw new AssertionError(e);
-        }
-    }
-
-    public long allocate(long size)
-    {
-        return unsafe.allocateMemory(size);
-    }
-
-    public void free(long peer)
-    {
-        unsafe.freeMemory(peer);
-    }
-
-}
diff --git a/src/java/org/apache/cassandra/io/util/PoolingSegmentedFile.java b/src/java/org/apache/cassandra/io/util/PoolingSegmentedFile.java
index 4ab98af..a5fa20b 100644
--- a/src/java/org/apache/cassandra/io/util/PoolingSegmentedFile.java
+++ b/src/java/org/apache/cassandra/io/util/PoolingSegmentedFile.java
@@ -17,21 +17,19 @@
  */
 package org.apache.cassandra.io.util;
 
-import java.io.File;
-
 import org.apache.cassandra.service.FileCacheService;
 
 public abstract class PoolingSegmentedFile extends SegmentedFile
 {
     final FileCacheService.CacheKey cacheKey;
-    protected PoolingSegmentedFile(Cleanup cleanup, String path, long length)
+    protected PoolingSegmentedFile(Cleanup cleanup, ChannelProxy channel, long length)
     {
-        this(cleanup, path, length, length);
+        this(cleanup, channel, length, length);
     }
 
-    protected PoolingSegmentedFile(Cleanup cleanup, String path, long length, long onDiskLength)
+    protected PoolingSegmentedFile(Cleanup cleanup, ChannelProxy channel, long length, long onDiskLength)
     {
-        super(cleanup, path, length, onDiskLength);
+        super(cleanup, channel, length, onDiskLength);
         cacheKey = cleanup.cacheKey;
     }
 
@@ -44,16 +42,19 @@
     protected static class Cleanup extends SegmentedFile.Cleanup
     {
         final FileCacheService.CacheKey cacheKey = new FileCacheService.CacheKey();
-        protected Cleanup(String path)
+        protected Cleanup(ChannelProxy channel)
         {
-            super(path);
+            super(channel);
         }
-        public void tidy() throws Exception
+        public void tidy()
         {
-            FileCacheService.instance.invalidate(cacheKey, path);
+            super.tidy();
+
+            FileCacheService.instance.invalidate(cacheKey, channel.filePath());
         }
     }
 
+    @SuppressWarnings("resource")
     public FileDataInput getSegment(long position)
     {
         RandomAccessReader reader = FileCacheService.instance.get(cacheKey);
@@ -67,7 +68,7 @@
 
     protected RandomAccessReader createPooledReader()
     {
-        return RandomAccessReader.open(new File(path), length, this);
+        return RandomAccessReader.open(channel, length, this);
     }
 
     public void recycle(RandomAccessReader reader)
diff --git a/src/java/org/apache/cassandra/io/util/RandomAccessReader.java b/src/java/org/apache/cassandra/io/util/RandomAccessReader.java
index d15fe46..0265be5 100644
--- a/src/java/org/apache/cassandra/io/util/RandomAccessReader.java
+++ b/src/java/org/apache/cassandra/io/util/RandomAccessReader.java
@@ -19,36 +19,27 @@
 
 import java.io.*;
 import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
 
 import com.google.common.annotations.VisibleForTesting;
 
 import org.apache.cassandra.io.FSReadError;
+import org.apache.cassandra.io.compress.BufferType;
+import org.apache.cassandra.utils.ByteBufferUtil;
 
-public class RandomAccessReader extends RandomAccessFile implements FileDataInput
+public class RandomAccessReader extends AbstractDataInput implements FileDataInput
 {
-    public static final long CACHE_FLUSH_INTERVAL_IN_BYTES = (long) Math.pow(2, 27); // 128mb
-
     // default buffer size, 64Kb
     public static final int DEFAULT_BUFFER_SIZE = 65536;
     public static final int BUFFER_SIZE = Integer.getInteger("cassandra.rar_buffer_size", DEFAULT_BUFFER_SIZE);
 
-    // absolute filesystem path to the file
-    private final String filePath;
-
     // buffer which will cache file blocks
-    protected byte[] buffer;
+    protected ByteBuffer buffer;
 
-    // `current` as current position in file
     // `bufferOffset` is the offset of the beginning of the buffer
     // `markedPointer` folds the offset of the last file mark
-    protected long bufferOffset, current = 0, markedPointer;
-    // `validBufferBytes` is the number of bytes in the buffer that are actually valid;
-    //  this will be LESS than buffer capacity if buffer is not full!
-    protected int validBufferBytes = 0;
+    protected long bufferOffset, markedPointer;
 
-    // channel liked with the file, used to retrieve data and force updates.
-    protected final FileChannel channel;
+    protected final ChannelProxy channel;
 
     // this can be overridden at construction to a value shorter than the true length of the file;
     // if so, it acts as an imposed limit on reads, rather than a convenience property
@@ -56,80 +47,74 @@
 
     protected final PoolingSegmentedFile owner;
 
-    protected RandomAccessReader(File file, int bufferSize, PoolingSegmentedFile owner) throws FileNotFoundException
+    protected RandomAccessReader(ChannelProxy channel, int bufferSize, long overrideLength, BufferType bufferType, PoolingSegmentedFile owner)
     {
-        this(file, bufferSize, -1, owner);
-    }
-    protected RandomAccessReader(File file, int bufferSize, long overrideLength, PoolingSegmentedFile owner) throws FileNotFoundException
-    {
-        super(file, "r");
-
+        this.channel = channel.sharedCopy();
         this.owner = owner;
 
-        channel = super.getChannel();
-        filePath = file.getAbsolutePath();
-
         // allocating required size of the buffer
         if (bufferSize <= 0)
             throw new IllegalArgumentException("bufferSize must be positive");
 
-        buffer = new byte[bufferSize];
-
         // we can cache file length in read-only mode
-        long fileLength = overrideLength;
-        if (fileLength <= 0)
-        {
-            try
-            {
-                fileLength = channel.size();
-            }
-            catch (IOException e)
-            {
-                throw new FSReadError(e, filePath);
-            }
-        }
+        fileLength = overrideLength <= 0 ? channel.size() : overrideLength;
 
-        this.fileLength = fileLength;
-        validBufferBytes = -1; // that will trigger reBuffer() on demand by read/seek operations
+        buffer = allocateBuffer(bufferSize, bufferType);
+        buffer.limit(0);
     }
 
-    public static RandomAccessReader open(File file, long overrideSize, PoolingSegmentedFile owner)
+    protected ByteBuffer allocateBuffer(int bufferSize, BufferType bufferType)
     {
-        return open(file, BUFFER_SIZE, overrideSize, owner);
+        int size = (int) Math.min(fileLength, bufferSize);
+        return bufferType.allocate(size);
+    }
+
+    public static RandomAccessReader open(ChannelProxy channel, long overrideSize, PoolingSegmentedFile owner)
+    {
+        return open(channel, BUFFER_SIZE, overrideSize, owner);
     }
 
     public static RandomAccessReader open(File file)
     {
-        return open(file, -1L);
+        try (ChannelProxy channel = new ChannelProxy(file))
+        {
+            return open(channel);
+        }
     }
 
-    public static RandomAccessReader open(File file, long overrideSize)
+    public static RandomAccessReader open(ChannelProxy channel)
     {
-        return open(file, BUFFER_SIZE, overrideSize, null);
+        return open(channel, -1L);
+    }
+
+    public static RandomAccessReader open(ChannelProxy channel, long overrideSize)
+    {
+        return open(channel, BUFFER_SIZE, overrideSize, null);
     }
 
     @VisibleForTesting
-    static RandomAccessReader open(File file, int bufferSize, PoolingSegmentedFile owner)
+    static RandomAccessReader open(ChannelProxy channel, int bufferSize, PoolingSegmentedFile owner)
     {
-        return open(file, bufferSize, -1L, owner);
+        return open(channel, bufferSize, -1L, owner);
     }
 
-    private static RandomAccessReader open(File file, int bufferSize, long overrideSize, PoolingSegmentedFile owner)
+    private static RandomAccessReader open(ChannelProxy channel, int bufferSize, long overrideSize, PoolingSegmentedFile owner)
     {
-        try
-        {
-            return new RandomAccessReader(file, bufferSize, overrideSize, owner);
-        }
-        catch (FileNotFoundException e)
-        {
-            throw new RuntimeException(e);
-        }
+        return new RandomAccessReader(channel, bufferSize, overrideSize, BufferType.ON_HEAP, owner);
     }
 
     @VisibleForTesting
     static RandomAccessReader open(SequentialWriter writer)
     {
-        return open(new File(writer.getPath()), BUFFER_SIZE, null);
+        try (ChannelProxy channel = new ChannelProxy(writer.getPath()))
+        {
+            return open(channel, BUFFER_SIZE, null);
+        }
+    }
+
+    public ChannelProxy getChannel()
+    {
+        return channel;
     }
 
     /**
@@ -137,55 +122,47 @@
      */
     protected void reBuffer()
     {
-        resetBuffer();
+        bufferOffset += buffer.position();
+        buffer.clear();
+        assert bufferOffset < fileLength;
 
-        try
+        long position = bufferOffset;
+        long limit = bufferOffset;
+        while (buffer.hasRemaining() && limit < fileLength)
         {
-            int read = buffer.length;
-            if (bufferOffset + read > fileLength)
-            {
-                if (bufferOffset >= fileLength)
-                    return;
-                read = (int) (fileLength - bufferOffset);
-            }
-
-            channel.position(bufferOffset); // setting channel position
-
-            int offset = 0;
-            while (read > 0)
-            {
-                int n = super.read(buffer, offset, read);
-                if (n < 0)
-                    throw new IllegalStateException();
-                read -= n;
-                offset += n;
-            }
-
-            validBufferBytes = offset;
+            int n = channel.read(buffer, position);
+            if (n < 0)
+                break;
+            position += n;
+            limit = bufferOffset + buffer.position();
         }
-        catch (IOException e)
-        {
-            throw new FSReadError(e, filePath);
-        }
+        if (limit > fileLength)
+            buffer.position((int)(fileLength - bufferOffset));
+        buffer.flip();
     }
 
     @Override
     public long getFilePointer()
     {
-        return current;
+        return current();
+    }
+
+    protected long current()
+    {
+        return bufferOffset + (buffer == null ? 0 : buffer.position());
     }
 
     public String getPath()
     {
-        return filePath;
+        return channel.filePath();
     }
 
     public int getTotalBufferSize()
     {
         //This may NPE so we make a ref
         //https://issues.apache.org/jira/browse/CASSANDRA-7756
-        byte[] ref = buffer;
-        return ref != null ? ref.length : 0;
+        ByteBuffer ref = buffer;
+        return ref != null ? ref.capacity() : 0;
     }
 
     public void reset()
@@ -195,14 +172,14 @@
 
     public long bytesPastMark()
     {
-        long bytes = current - markedPointer;
+        long bytes = current() - markedPointer;
         assert bytes >= 0;
         return bytes;
     }
 
     public FileMark mark()
     {
-        markedPointer = current;
+        markedPointer = current();
         return new BufferedRandomAccessFileMark(markedPointer);
     }
 
@@ -215,7 +192,7 @@
     public long bytesPastMark(FileMark mark)
     {
         assert mark instanceof BufferedRandomAccessFileMark;
-        long bytes = current - ((BufferedRandomAccessFileMark) mark).pointer;
+        long bytes = current() - ((BufferedRandomAccessFileMark) mark).pointer;
         assert bytes >= 0;
         return bytes;
     }
@@ -233,17 +210,6 @@
         return length() - getFilePointer();
     }
 
-    protected int bufferCursor()
-    {
-        return (int) (current - bufferOffset);
-    }
-
-    protected void resetBuffer()
-    {
-        bufferOffset = current;
-        validBufferBytes = 0;
-    }
-
     @Override
     public void close()
     {
@@ -264,22 +230,21 @@
 
     public void deallocate()
     {
-        buffer = null; // makes sure we don't use this after it's ostensibly closed
+        //make idempotent
+        if (buffer == null)
+            return;
 
-        try
-        {
-            super.close();
-        }
-        catch (IOException e)
-        {
-            throw new FSReadError(e, filePath);
-        }
+        bufferOffset += buffer.position();
+        FileUtils.clean(buffer);
+
+        buffer = null; // makes sure we don't use this after it's ostensibly closed
+        channel.close();
     }
 
     @Override
     public String toString()
     {
-        return getClass().getSimpleName() + "(" + "filePath='" + filePath + "')";
+        return getClass().getSimpleName() + "(" + "filePath='" + channel + "')";
     }
 
     /**
@@ -301,17 +266,28 @@
         if (newPosition < 0)
             throw new IllegalArgumentException("new position should not be negative");
 
-        if (newPosition > length()) // it is save to call length() in read-only mode
-            throw new IllegalArgumentException(String.format("unable to seek to position %d in %s (%d bytes) in read-only mode",
+        if (newPosition >= length()) // it is save to call length() in read-only mode
+        {
+            if (newPosition > length())
+                throw new IllegalArgumentException(String.format("Unable to seek to position %d in %s (%d bytes) in read-only mode",
                                                              newPosition, getPath(), length()));
+            buffer.limit(0);
+            bufferOffset = newPosition;
+            return;
+        }
 
-        current = newPosition;
-
-        if (newPosition > (bufferOffset + validBufferBytes) || newPosition < bufferOffset)
-            reBuffer();
+        if (newPosition >= bufferOffset && newPosition < bufferOffset + buffer.limit())
+        {
+            buffer.position((int) (newPosition - bufferOffset));
+            return;
+        }
+        // Set current location to newPosition and clear buffer so reBuffer calculates from newPosition
+        bufferOffset = newPosition;
+        buffer.clear();
+        reBuffer();
+        assert current() == newPosition;
     }
 
-    @Override
     // -1 will be returned if there is nothing to read; higher-level methods like readInt
     // or readFully (from RandomAccessFile) will throw EOFException but this should not
     public int read()
@@ -319,15 +295,15 @@
         if (buffer == null)
             throw new AssertionError("Attempted to read from closed RAR");
 
-        if (isEOF())
-            return -1; // required by RandomAccessFile
+        if (!buffer.hasRemaining())
+        {
+            if (isEOF())
+                return -1; // required by RandomAccessFile
 
-        if (current >= bufferOffset + buffer.length || validBufferBytes == -1)
             reBuffer();
+        }
 
-        assert current >= bufferOffset && current < bufferOffset + validBufferBytes;
-
-        return ((int) buffer[(int) (current++ - bufferOffset)]) & 0xff;
+        return (int)buffer.get() & 0xff;
     }
 
     @Override
@@ -347,50 +323,49 @@
         if (length == 0)
             return 0;
 
-        if (isEOF())
-            return -1;
+        if (!buffer.hasRemaining())
+        {
+            if (isEOF())
+                return -1;
 
-        if (current >= bufferOffset + buffer.length || validBufferBytes == -1)
             reBuffer();
+        }
 
-        assert current >= bufferOffset && current < bufferOffset + validBufferBytes
-                : String.format("File (%s), current offset %d, buffer offset %d, buffer limit %d",
-                                getPath(),
-                                current,
-                                bufferOffset,
-                                validBufferBytes);
-
-        int toCopy = Math.min(length, validBufferBytes - bufferCursor());
-
-        System.arraycopy(buffer, bufferCursor(), buff, offset, toCopy);
-        current += toCopy;
-
+        int toCopy = Math.min(length, buffer.remaining());
+        buffer.get(buff, offset, toCopy);
         return toCopy;
     }
 
     public ByteBuffer readBytes(int length) throws EOFException
     {
         assert length >= 0 : "buffer length should not be negative: " + length;
-
-        byte[] buff = new byte[length];
-
         try
         {
-            readFully(buff); // reading data buffer
+            ByteBuffer result = ByteBuffer.allocate(length);
+            while (result.hasRemaining())
+            {
+                if (!buffer.hasRemaining())
+                {
+                    if (isEOF())
+                        throw new EOFException();
+
+                    reBuffer();
+                }
+                ByteBufferUtil.put(buffer, result);
+            }
+            result.flip();
+            return result;
         }
         catch (EOFException e)
         {
             throw e;
         }
-        catch (IOException e)
+        catch (Exception e)
         {
-            throw new FSReadError(e, filePath);
+            throw new FSReadError(e, channel.toString());
         }
-
-        return ByteBuffer.wrap(buff);
     }
 
-    @Override
     public long length()
     {
         return fileLength;
@@ -398,24 +373,11 @@
 
     public long getPosition()
     {
-        return current;
+        return bufferOffset + buffer.position();
     }
 
-    @Override
-    public void write(int value)
+    public long getPositionLimit()
     {
-        throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public void write(byte[] buffer)
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public void write(byte[] buffer, int offset, int length)
-    {
-        throw new UnsupportedOperationException();
+        return length();
     }
 }
diff --git a/src/java/org/apache/cassandra/io/util/SafeMemory.java b/src/java/org/apache/cassandra/io/util/SafeMemory.java
index 45419a6..ad11472 100644
--- a/src/java/org/apache/cassandra/io/util/SafeMemory.java
+++ b/src/java/org/apache/cassandra/io/util/SafeMemory.java
@@ -22,6 +22,7 @@
 import org.apache.cassandra.utils.concurrent.Ref;
 import org.apache.cassandra.utils.concurrent.RefCounted;
 import org.apache.cassandra.utils.concurrent.SharedCloseable;
+import org.apache.cassandra.utils.memory.MemoryUtil;
 
 public class SafeMemory extends Memory implements SharedCloseable
 {
@@ -61,6 +62,11 @@
         peer = 0;
     }
 
+    public Throwable close(Throwable accumulate)
+    {
+        return ref.ensureReleased(accumulate);
+    }
+
     public SafeMemory copy(long newSize)
     {
         SafeMemory copy = new SafeMemory(newSize);
@@ -82,7 +88,7 @@
         {
             /** see {@link Memory#Memory(long)} re: null pointers*/
             if (peer != 0)
-                Memory.allocator.free(peer);
+                MemoryUtil.free(peer);
         }
 
         public String name()
diff --git a/src/java/org/apache/cassandra/io/util/SafeMemoryWriter.java b/src/java/org/apache/cassandra/io/util/SafeMemoryWriter.java
index 1998cc6..24eb93c 100644
--- a/src/java/org/apache/cassandra/io/util/SafeMemoryWriter.java
+++ b/src/java/org/apache/cassandra/io/util/SafeMemoryWriter.java
@@ -18,119 +18,96 @@
 */
 package org.apache.cassandra.io.util;
 
-import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
 
-public class SafeMemoryWriter extends AbstractDataOutput implements DataOutputPlus
+public class SafeMemoryWriter extends DataOutputBuffer
 {
-    private ByteOrder order = ByteOrder.BIG_ENDIAN;
-    private SafeMemory buffer;
-    private long length;
+    private SafeMemory memory;
 
+    @SuppressWarnings("resource")
     public SafeMemoryWriter(long initialCapacity)
     {
-        buffer = new SafeMemory(initialCapacity);
+        this(new SafeMemory(initialCapacity));
     }
 
-    public void write(byte[] buffer, int offset, int count)
+    private SafeMemoryWriter(SafeMemory memory)
     {
-        long newLength = ensureCapacity(count);
-        this.buffer.setBytes(this.length, buffer, offset, count);
-        this.length = newLength;
-    }
-
-    public void write(int oneByte)
-    {
-        long newLength = ensureCapacity(1);
-        buffer.setByte(length++, (byte) oneByte);
-        length = newLength;
-    }
-
-    public void writeShort(int val) throws IOException
-    {
-        if (order != ByteOrder.nativeOrder())
-            val = Short.reverseBytes((short) val);
-        long newLength = ensureCapacity(2);
-        buffer.setShort(length, (short) val);
-        length = newLength;
-    }
-
-    public void writeInt(int val)
-    {
-        if (order != ByteOrder.nativeOrder())
-            val = Integer.reverseBytes(val);
-        long newLength = ensureCapacity(4);
-        buffer.setInt(length, val);
-        length = newLength;
-    }
-
-    public void writeLong(long val)
-    {
-        if (order != ByteOrder.nativeOrder())
-            val = Long.reverseBytes(val);
-        long newLength = ensureCapacity(8);
-        buffer.setLong(length, val);
-        length = newLength;
-    }
-
-    public void write(ByteBuffer buffer)
-    {
-        long newLength = ensureCapacity(buffer.remaining());
-        this.buffer.setBytes(length, buffer);
-        length = newLength;
-    }
-
-    public void write(Memory memory)
-    {
-        long newLength = ensureCapacity(memory.size());
-        buffer.put(length, memory, 0, memory.size());
-        length = newLength;
-    }
-
-    private long ensureCapacity(long size)
-    {
-        long newLength = this.length + size;
-        if (newLength > buffer.size())
-            setCapacity(Math.max(newLength, buffer.size() + (buffer.size() / 2)));
-        return newLength;
+        super(tailBuffer(memory).order(ByteOrder.BIG_ENDIAN));
+        this.memory = memory;
     }
 
     public SafeMemory currentBuffer()
     {
-        return buffer;
+        return memory;
     }
 
-    public void setCapacity(long newCapacity)
+    @Override
+    protected void reallocate(long count)
     {
+        long newCapacity = calculateNewSize(count);
         if (newCapacity != capacity())
         {
-            SafeMemory oldBuffer = buffer;
-            buffer = this.buffer.copy(newCapacity);
+            long position = length();
+            ByteOrder order = buffer.order();
+
+            SafeMemory oldBuffer = memory;
+            memory = this.memory.copy(newCapacity);
+            buffer = tailBuffer(memory);
+
+            int newPosition = (int) (position - tailOffset(memory));
+            buffer.position(newPosition);
+            buffer.order(order);
+
             oldBuffer.free();
         }
     }
 
+    public void setCapacity(long newCapacity)
+    {
+        reallocate(newCapacity);
+    }
+
     public void close()
     {
-        buffer.close();
+        memory.close();
+    }
+
+    public Throwable close(Throwable accumulate)
+    {
+        return memory.close(accumulate);
     }
 
     public long length()
     {
-        return length;
+        return tailOffset(memory) +  buffer.position();
     }
 
     public long capacity()
     {
-        return buffer.size();
+        return memory.size();
     }
 
-    // TODO: consider hoisting this into DataOutputPlus, since most implementations can copy with this gracefully
-    // this would simplify IndexSummary.IndexSummarySerializer.serialize()
-    public SafeMemoryWriter withByteOrder(ByteOrder order)
+    @Override
+    public SafeMemoryWriter order(ByteOrder order)
     {
-        this.order = order;
+        super.order(order);
         return this;
     }
+
+    @Override
+    public long validateReallocation(long newSize)
+    {
+        return newSize;
+    }
+
+    private static long tailOffset(Memory memory)
+    {
+        return Math.max(0, memory.size - Integer.MAX_VALUE);
+    }
+
+    private static ByteBuffer tailBuffer(Memory memory)
+    {
+        return memory.asByteBuffer(tailOffset(memory), (int) Math.min(memory.size, Integer.MAX_VALUE));
+    }
 }
diff --git a/src/java/org/apache/cassandra/io/util/SegmentedFile.java b/src/java/org/apache/cassandra/io/util/SegmentedFile.java
index 23454bc..cb331de 100644
--- a/src/java/org/apache/cassandra/io/util/SegmentedFile.java
+++ b/src/java/org/apache/cassandra/io/util/SegmentedFile.java
@@ -26,6 +26,7 @@
 import java.util.List;
 import java.util.NoSuchElementException;
 
+import com.google.common.base.Throwables;
 import com.google.common.util.concurrent.RateLimiter;
 
 import org.apache.cassandra.config.Config;
@@ -37,6 +38,8 @@
 import org.apache.cassandra.utils.concurrent.RefCounted;
 import org.apache.cassandra.utils.concurrent.SharedCloseableImpl;
 
+import static org.apache.cassandra.utils.Throwables.maybeFail;
+
 /**
  * Abstracts a read-only file that has been split into segments, each of which can be represented by an independent
  * FileDataInput. Allows for iteration over the FileDataInputs, or random access to the FileDataInput for a given
@@ -48,7 +51,7 @@
  */
 public abstract class SegmentedFile extends SharedCloseableImpl
 {
-    public final String path;
+    public final ChannelProxy channel;
     public final long length;
 
     // This differs from length for compressed files (but we still need length for
@@ -58,15 +61,15 @@
     /**
      * Use getBuilder to get a Builder to construct a SegmentedFile.
      */
-    SegmentedFile(Cleanup cleanup, String path, long length)
+    SegmentedFile(Cleanup cleanup, ChannelProxy channel, long length)
     {
-        this(cleanup, path, length, length);
+        this(cleanup, channel, length, length);
     }
 
-    protected SegmentedFile(Cleanup cleanup, String path, long length, long onDiskLength)
+    protected SegmentedFile(Cleanup cleanup, ChannelProxy channel, long length, long onDiskLength)
     {
         super(cleanup);
-        this.path = new File(path).getAbsolutePath();
+        this.channel = channel;
         this.length = length;
         this.onDiskLength = onDiskLength;
     }
@@ -74,22 +77,32 @@
     public SegmentedFile(SegmentedFile copy)
     {
         super(copy);
-        path = copy.path;
+        channel = copy.channel;
         length = copy.length;
         onDiskLength = copy.onDiskLength;
     }
 
+    public String path()
+    {
+        return channel.filePath();
+    }
+
     protected static abstract class Cleanup implements RefCounted.Tidy
     {
-        final String path;
-        protected Cleanup(String path)
+        final ChannelProxy channel;
+        protected Cleanup(ChannelProxy channel)
         {
-            this.path = path;
+            this.channel = channel;
         }
 
         public String name()
         {
-            return path;
+            return channel.filePath();
+        }
+
+        public void tidy()
+        {
+            channel.close();
         }
     }
 
@@ -97,13 +110,13 @@
 
     public RandomAccessReader createReader()
     {
-        return RandomAccessReader.open(new File(path), length);
+        return RandomAccessReader.open(channel, length);
     }
 
     public RandomAccessReader createThrottledReader(RateLimiter limiter)
     {
         assert limiter != null;
-        return ThrottledReader.open(new File(path), length, limiter);
+        return ThrottledReader.open(channel, length, limiter);
     }
 
     public FileDataInput getSegment(long position)
@@ -115,22 +128,17 @@
 
     public void dropPageCache(long before)
     {
-        CLibrary.trySkipCache(path, 0, before);
+        CLibrary.trySkipCache(channel.getFileDescriptor(), 0, before);
     }
 
     /**
      * @return A SegmentedFile.Builder.
      */
-    public static Builder getBuilder(Config.DiskAccessMode mode)
+    public static Builder getBuilder(Config.DiskAccessMode mode, boolean compressed)
     {
-        return mode == Config.DiskAccessMode.mmap
-               ? new MmappedSegmentedFile.Builder()
-               : new BufferedPoolingSegmentedFile.Builder();
-    }
-
-    public static Builder getCompressedBuilder()
-    {
-        return getCompressedBuilder(null);
+        return compressed ? new CompressedPoolingSegmentedFile.Builder(null)
+                          : mode == Config.DiskAccessMode.mmap ? new MmappedSegmentedFile.Builder()
+                                                               : new BufferedPoolingSegmentedFile.Builder();
     }
 
     public static Builder getCompressedBuilder(CompressedSequentialWriter writer)
@@ -147,10 +155,22 @@
     }
 
     /**
+     * Retrieve the readable bounds if any so they can be cloned into other files such
+     * as when downsampling an index summary. Readable bounds are in between record locations in a file
+     * that are good positions for mapping the file such that records don't cross mappings.
+     */
+    public long[] copyReadableBounds()
+    {
+        return new long[0];
+    }
+
+    /**
      * Collects potential segmentation points in an underlying file, and builds a SegmentedFile to represent it.
      */
-    public static abstract class Builder
+    public static abstract class Builder implements AutoCloseable
     {
+        private ChannelProxy channel;
+
         /**
          * Adds a position that would be a safe place for a segment boundary in the file. For a block/row based file
          * format, safe boundaries are block/row edges.
@@ -160,23 +180,28 @@
 
         /**
          * Called after all potential boundaries have been added to apply this Builder to a concrete file on disk.
-         * @param path The file on disk.
+         * @param channel The channel to the file on disk.
          */
-        protected abstract SegmentedFile complete(String path, long overrideLength, boolean isFinal);
+        protected abstract SegmentedFile complete(ChannelProxy channel, long overrideLength);
 
         public SegmentedFile complete(String path)
         {
-            return complete(path, -1, true);
+            return complete(path, -1L);
         }
 
-        public SegmentedFile complete(String path, boolean isFinal)
-        {
-            return complete(path, -1, isFinal);
-        }
-
+        @SuppressWarnings("resource")
         public SegmentedFile complete(String path, long overrideLength)
         {
-            return complete(path, overrideLength, false);
+            ChannelProxy channelCopy = getChannel(path);
+            try
+            {
+                return complete(channelCopy, overrideLength);
+            }
+            catch (Throwable t)
+            {
+                channelCopy.close();
+                throw t;
+            }
         }
 
         public void serializeBounds(DataOutput out) throws IOException
@@ -189,6 +214,32 @@
             if (!in.readUTF().equals(DatabaseDescriptor.getDiskAccessMode().name()))
                 throw new IOException("Cannot deserialize SSTable Summary component because the DiskAccessMode was changed!");
         }
+
+        public Throwable close(Throwable accumulate)
+        {
+            if (channel != null)
+                return channel.close(accumulate);
+            return accumulate;
+        }
+
+        public void close()
+        {
+            maybeFail(close(null));
+        }
+
+        private ChannelProxy getChannel(String path)
+        {
+            if (channel != null)
+            {
+                if (channel.filePath().equals(path))
+                    return channel.sharedCopy();
+                else
+                    channel.close();
+            }
+
+            channel = new ChannelProxy(path);
+            return channel.sharedCopy();
+        }
     }
 
     static final class Segment extends Pair<Long, MappedByteBuffer> implements Comparable<Segment>
@@ -234,7 +285,7 @@
             }
             catch (IOException e)
             {
-                throw new FSReadError(e, path);
+                throw new FSReadError(e, path());
             }
             return segment;
         }
@@ -244,7 +295,7 @@
 
     @Override
     public String toString() {
-        return getClass().getSimpleName() + "(path='" + path + "'" +
+        return getClass().getSimpleName() + "(path='" + path() + "'" +
                ", length=" + length +
                ")";
 }
diff --git a/src/java/org/apache/cassandra/io/util/SequentialWriter.java b/src/java/org/apache/cassandra/io/util/SequentialWriter.java
index 40f3e9d..0c39469 100644
--- a/src/java/org/apache/cassandra/io/util/SequentialWriter.java
+++ b/src/java/org/apache/cassandra/io/util/SequentialWriter.java
@@ -20,45 +20,45 @@
 import java.io.*;
 import java.nio.ByteBuffer;
 import java.nio.channels.ClosedChannelException;
+import java.nio.channels.FileChannel;
 import java.nio.channels.WritableByteChannel;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import java.nio.file.StandardOpenOption;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.io.FSReadError;
 import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.io.compress.BufferType;
 import org.apache.cassandra.io.compress.CompressedSequentialWriter;
 import org.apache.cassandra.io.compress.CompressionParameters;
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
-import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.CLibrary;
+import org.apache.cassandra.utils.concurrent.Transactional;
+
+import static org.apache.cassandra.utils.Throwables.merge;
+import org.apache.cassandra.utils.SyncUtil;
 
 /**
  * Adds buffering, mark, and fsyncing to OutputStream.  We always fsync on close; we may also
  * fsync incrementally if Config.trickle_fsync is enabled.
  */
-public class SequentialWriter extends OutputStream implements WritableByteChannel
+public class SequentialWriter extends OutputStream implements WritableByteChannel, Transactional
 {
-    private static final Logger logger = LoggerFactory.getLogger(SequentialWriter.class);
-
     // isDirty - true if this.buffer contains any un-synced bytes
     protected boolean isDirty = false, syncNeeded = false;
 
     // absolute path to the given file
     private final String filePath;
 
-    protected byte[] buffer;
-    private final int fd;
+    protected ByteBuffer buffer;
     private int directoryFD;
     // directory should be synced only after first file sync, in other words, only once per file
     private boolean directorySynced = false;
 
-    protected long current = 0, bufferOffset;
-    protected int validBufferBytes;
+    // Offset for start of buffer relative to underlying file
+    protected long bufferOffset;
 
-    protected final RandomAccessFile out;
+    protected final FileChannel channel;
 
     // whether to do trickling fsync() to avoid sudden bursts of dirty buffer flushing by kernel causing read
     // latency spikes
@@ -71,44 +71,88 @@
 
     protected Runnable runPostFlush;
 
-    public SequentialWriter(File file, int bufferSize)
+    private final TransactionalProxy txnProxy = txnProxy();
+    private boolean finishOnClose;
+    protected Descriptor descriptor;
+
+    // due to lack of multiple-inheritance, we proxy our transactional implementation
+    protected class TransactionalProxy extends AbstractTransactional
+    {
+        @Override
+        protected Throwable doPreCleanup(Throwable accumulate)
+        {
+            if (directoryFD >= 0)
+            {
+                try { CLibrary.tryCloseFD(directoryFD); }
+                catch (Throwable t) { accumulate = merge(accumulate, t); }
+                directoryFD = -1;
+            }
+
+            // close is idempotent
+            try { channel.close(); }
+            catch (Throwable t) { accumulate = merge(accumulate, t); }
+
+            if (buffer != null)
+            {
+                try { FileUtils.clean(buffer); }
+                catch (Throwable t) { accumulate = merge(accumulate, t); }
+                buffer = null;
+            }
+
+            return accumulate;
+        }
+
+        protected void doPrepare()
+        {
+            syncInternal();
+            // we must cleanup our file handles during prepareCommit for Windows compatibility as we cannot rename an open file;
+            // TODO: once we stop file renaming, remove this for clarity
+            releaseFileHandle();
+        }
+
+        protected Throwable doCommit(Throwable accumulate)
+        {
+            return accumulate;
+        }
+
+        protected Throwable doAbort(Throwable accumulate)
+        {
+            return FileUtils.deleteWithConfirm(filePath, false, accumulate);
+        }
+    }
+
+    public SequentialWriter(File file, int bufferSize, BufferType bufferType)
     {
         try
         {
-            out = new RandomAccessFile(file, "rw");
+            if (file.exists())
+                channel = FileChannel.open(file.toPath(), StandardOpenOption.READ, StandardOpenOption.WRITE);
+            else
+                channel = FileChannel.open(file.toPath(), StandardOpenOption.READ, StandardOpenOption.WRITE, StandardOpenOption.CREATE);
         }
-        catch (FileNotFoundException e)
+        catch (IOException e)
         {
             throw new RuntimeException(e);
         }
 
         filePath = file.getAbsolutePath();
 
-        buffer = new byte[bufferSize];
+        // Allow children to allocate buffer as direct (snappy compression) if necessary
+        buffer = bufferType.allocate(bufferSize);
+
         this.trickleFsync = DatabaseDescriptor.getTrickleFsync();
         this.trickleFsyncByteInterval = DatabaseDescriptor.getTrickleFsyncIntervalInKb() * 1024;
 
-        try
-        {
-            fd = CLibrary.getfd(out.getFD());
-        }
-        catch (IOException e)
-        {
-            throw new RuntimeException(e); // shouldn't happen
-        }
-
         directoryFD = CLibrary.tryOpenDirectory(file.getParent());
-        stream = new DataOutputStreamAndChannel(this, this);
+        stream = new WrappedDataOutputStreamPlus(this, this);
     }
 
+    /**
+     * Open a heap-based, non-compressed SequentialWriter
+     */
     public static SequentialWriter open(File file)
     {
-        return open(file, RandomAccessReader.DEFAULT_BUFFER_SIZE);
-    }
-
-    public static SequentialWriter open(File file, int bufferSize)
-    {
-        return new SequentialWriter(file, bufferSize);
+        return new SequentialWriter(file, RandomAccessReader.DEFAULT_BUFFER_SIZE, BufferType.ON_HEAP);
     }
 
     public static ChecksummedSequentialWriter open(File file, File crcPath)
@@ -124,37 +168,51 @@
         return new CompressedSequentialWriter(new File(dataFilePath), offsetsPath, parameters, sstableMetadataCollector);
     }
 
+    public SequentialWriter finishOnClose()
+    {
+        finishOnClose = true;
+        return this;
+    }
+
     public void write(int value) throws ClosedChannelException
     {
-        if (current >= bufferOffset + buffer.length)
-            reBuffer();
-
-        assert current < bufferOffset + buffer.length
-                : String.format("File (%s) offset %d, buffer offset %d.", getPath(), current, bufferOffset);
-
-        buffer[bufferCursor()] = (byte) value;
-
-        validBufferBytes += 1;
-        current += 1;
-        isDirty = true;
-        syncNeeded = true;
-    }
-
-    public void write(byte[] buffer) throws ClosedChannelException
-    {
-        write(buffer, 0, buffer.length);
-    }
-
-    public void write(byte[] data, int offset, int length) throws ClosedChannelException
-    {
         if (buffer == null)
             throw new ClosedChannelException();
 
-        while (length > 0)
+        if (!buffer.hasRemaining())
         {
-            int n = writeAtMost(data, offset, length);
-            offset += n;
-            length -= n;
+            reBuffer();
+        }
+
+        buffer.put((byte) value);
+
+        isDirty = true;
+        syncNeeded = true;
+    }
+
+    public void write(byte[] buffer) throws IOException
+    {
+        write(buffer, 0, buffer.length);
+    }
+
+    public void write(byte[] data, int offset, int length) throws IOException
+    {
+        if (buffer == null)
+            throw new ClosedChannelException();
+
+        int position = offset;
+        int remaining = length;
+        while (remaining > 0)
+        {
+            if (!buffer.hasRemaining())
+                reBuffer();
+
+            int toCopy = Math.min(remaining, buffer.remaining());
+            buffer.put(data, position, toCopy);
+
+            remaining -= toCopy;
+            position += toCopy;
+
             isDirty = true;
             syncNeeded = true;
         }
@@ -166,75 +224,23 @@
             throw new ClosedChannelException();
 
         int length = src.remaining();
-        int offset = src.position();
-        while (length > 0)
+        int finalLimit = src.limit();
+        while (src.hasRemaining())
         {
-            int n = writeAtMost(src, offset, length);
-            offset += n;
-            length -= n;
+            if (!buffer.hasRemaining())
+                reBuffer();
+
+            if (buffer.remaining() < src.remaining())
+                src.limit(src.position() + buffer.remaining());
+            buffer.put(src);
+            src.limit(finalLimit);
+
             isDirty = true;
             syncNeeded = true;
         }
-        src.position(offset);
         return length;
     }
 
-    /*
-     * Write at most "length" bytes from "data" starting at position "offset", and
-     * return the number of bytes written. caller is responsible for setting
-     * isDirty.
-     */
-    private int writeAtMost(ByteBuffer data, int offset, int length)
-    {
-        if (current >= bufferOffset + buffer.length)
-            reBuffer();
-
-        assert current < bufferOffset + buffer.length
-        : String.format("File (%s) offset %d, buffer offset %d.", getPath(), current, bufferOffset);
-
-
-        int toCopy = Math.min(length, buffer.length - bufferCursor());
-
-        // copy bytes from external buffer
-        ByteBufferUtil.arrayCopy(data, offset, buffer, bufferCursor(), toCopy);
-
-        assert current <= bufferOffset + buffer.length
-        : String.format("File (%s) offset %d, buffer offset %d.", getPath(), current, bufferOffset);
-
-        validBufferBytes = Math.max(validBufferBytes, bufferCursor() + toCopy);
-        current += toCopy;
-
-        return toCopy;
-    }
-
-    /*
-     * Write at most "length" bytes from "data" starting at position "offset", and
-     * return the number of bytes written. caller is responsible for setting
-     * isDirty.
-     */
-    private int writeAtMost(byte[] data, int offset, int length)
-    {
-        if (current >= bufferOffset + buffer.length)
-            reBuffer();
-
-        assert current < bufferOffset + buffer.length
-                : String.format("File (%s) offset %d, buffer offset %d.", getPath(), current, bufferOffset);
-
-
-        int toCopy = Math.min(length, buffer.length - bufferCursor());
-
-        // copy bytes from external buffer
-        System.arraycopy(data, offset, buffer, bufferCursor(), toCopy);
-
-        assert current <= bufferOffset + buffer.length
-                : String.format("File (%s) offset %d, buffer offset %d.", getPath(), current, bufferOffset);
-
-        validBufferBytes = Math.max(validBufferBytes, bufferCursor() + toCopy);
-        current += toCopy;
-
-        return toCopy;
-    }
-
     /**
      * Synchronize file contents with disk.
      */
@@ -247,7 +253,7 @@
     {
         try
         {
-            out.getFD().sync();
+            SyncUtil.force(channel, false);
         }
         catch (IOException e)
         {
@@ -264,7 +270,7 @@
 
             if (!directorySynced)
             {
-                CLibrary.trySync(directoryFD);
+                SyncUtil.trySync(directoryFD);
                 directorySynced = true;
             }
 
@@ -291,7 +297,7 @@
 
             if (trickleFsync)
             {
-                bytesSinceTrickleFsync += validBufferBytes;
+                bytesSinceTrickleFsync += buffer.position();
                 if (bytesSinceTrickleFsync >= trickleFsyncByteInterval)
                 {
                     syncDataOnlyInternal();
@@ -320,8 +326,9 @@
     {
         try
         {
-            out.write(buffer, 0, validBufferBytes);
-            lastFlushOffset += validBufferBytes;
+            buffer.flip();
+            channel.write(buffer);
+            lastFlushOffset += buffer.position();
         }
         catch (IOException e)
         {
@@ -333,15 +340,17 @@
 
     public long getFilePointer()
     {
-        return current;
+        return current();
     }
 
     /**
-     * Return the current file pointer of the underlying on-disk file.
+     * Returns the current file pointer of the underlying on-disk file.
      * Note that since write works by buffering data, the value of this will increase by buffer
      * size and not every write to the writer will modify this value.
      * Furthermore, for compressed files, this value refers to compressed data, while the
      * writer getFilePointer() refers to uncompressedFile
+     *
+     * @return the current file pointer
      */
     public long getOnDiskFilePointer()
     {
@@ -352,7 +361,7 @@
     {
         try
         {
-            return Math.max(Math.max(current, out.length()), bufferOffset + validBufferBytes);
+            return Math.max(current(), channel.size());
         }
         catch (IOException e)
         {
@@ -373,44 +382,48 @@
 
     protected void resetBuffer()
     {
-        bufferOffset = current;
-        validBufferBytes = 0;
+        bufferOffset = current();
+        buffer.clear();
     }
 
-    private int bufferCursor()
+    protected long current()
     {
-        return (int) (current - bufferOffset);
+        return bufferOffset + (buffer == null ? 0 : buffer.position());
     }
 
     public FileMark mark()
     {
-        return new BufferedFileWriterMark(current);
+        return new BufferedFileWriterMark(current());
     }
 
+    /**
+     * Drops all buffered data that's past the limits of our new file mark + buffer capacity, or syncs and truncates
+     * the underlying file to the marked position
+     */
     public void resetAndTruncate(FileMark mark)
     {
         assert mark instanceof BufferedFileWriterMark;
 
-        long previous = current;
-        current = ((BufferedFileWriterMark) mark).pointer;
+        long previous = current();
+        long truncateTarget = ((BufferedFileWriterMark) mark).pointer;
 
-        if (previous - current <= validBufferBytes) // current buffer
+        // If we're resetting to a point within our buffered data, just adjust our buffered position to drop bytes to
+        // the right of the desired mark.
+        if (previous - truncateTarget <= buffer.position())
         {
-            validBufferBytes = validBufferBytes - ((int) (previous - current));
+            buffer.position(buffer.position() - ((int) (previous - truncateTarget)));
             return;
         }
 
-        // synchronize current buffer with disk
-        // because we don't want any data loss
+        // synchronize current buffer with disk - we don't want any data loss
         syncInternal();
 
         // truncate file to given position
-        truncate(current);
+        truncate(truncateTarget);
 
-        // reset channel position
         try
         {
-            out.seek(current);
+            channel.position(truncateTarget);
         }
         catch (IOException e)
         {
@@ -429,7 +442,7 @@
     {
         try
         {
-            out.getChannel().truncate(toSize);
+            channel.truncate(toSize);
         }
         catch (IOException e)
         {
@@ -439,52 +452,59 @@
 
     public boolean isOpen()
     {
-        return out.getChannel().isOpen();
+        return channel.isOpen();
+    }
+
+    public SequentialWriter setDescriptor(Descriptor descriptor)
+    {
+        this.descriptor = descriptor;
+        return this;
+    }
+
+    public final void prepareToCommit()
+    {
+        txnProxy.prepareToCommit();
+    }
+
+    public final Throwable commit(Throwable accumulate)
+    {
+        return txnProxy.commit(accumulate);
+    }
+
+    public final Throwable abort(Throwable accumulate)
+    {
+        return txnProxy.abort(accumulate);
     }
 
     @Override
-    public void close()
+    public final void close()
     {
-        if (buffer == null)
-            return; // already closed
-
-        syncInternal();
-
-        buffer = null;
-
-        cleanup(true);
-    }
-
-    public void abort()
-    {
-        cleanup(false);
-    }
-
-    private void cleanup(boolean throwExceptions)
-    {
-        if (directoryFD >= 0)
-        {
-            try { CLibrary.tryCloseFD(directoryFD); }
-            catch (Throwable t) { handle(t, throwExceptions); }
-            directoryFD = -1;
-        }
-
-        // close is idempotent
-        try { out.close(); }
-        catch (Throwable t) { handle(t, throwExceptions); }
-    }
-
-    private void handle(Throwable t, boolean throwExceptions)
-    {
-        if (!throwExceptions)
-            logger.warn("Suppressing exception thrown while aborting writer", t);
+        if (finishOnClose)
+            txnProxy.finish();
         else
-            throw new FSWriteError(t, getPath());
+            txnProxy.close();
     }
 
-    // hack to make life easier for subclasses
-    public void writeFullChecksum(Descriptor descriptor)
+    public final void finish()
     {
+        txnProxy.finish();
+    }
+
+    protected TransactionalProxy txnProxy()
+    {
+        return new TransactionalProxy();
+    }
+
+    public void releaseFileHandle()
+    {
+        try
+        {
+            channel.close();
+        }
+        catch (IOException e)
+        {
+            throw new FSWriteError(e, filePath);
+        }
     }
 
     /**
diff --git a/src/java/org/apache/cassandra/io/util/ThrottledReader.java b/src/java/org/apache/cassandra/io/util/ThrottledReader.java
index c4845c5..f725984 100644
--- a/src/java/org/apache/cassandra/io/util/ThrottledReader.java
+++ b/src/java/org/apache/cassandra/io/util/ThrottledReader.java
@@ -21,32 +21,33 @@
  */
 
 
-import java.io.File;
 import java.io.FileNotFoundException;
 
 import com.google.common.util.concurrent.RateLimiter;
 
+import org.apache.cassandra.io.compress.BufferType;
+
 public class ThrottledReader extends RandomAccessReader
 {
     private final RateLimiter limiter;
 
-    protected ThrottledReader(File file, long overrideLength, RateLimiter limiter) throws FileNotFoundException
+    protected ThrottledReader(ChannelProxy channel, long overrideLength, RateLimiter limiter) throws FileNotFoundException
     {
-        super(file, RandomAccessReader.DEFAULT_BUFFER_SIZE, overrideLength, null);
+        super(channel, RandomAccessReader.DEFAULT_BUFFER_SIZE, overrideLength, BufferType.ON_HEAP, null);
         this.limiter = limiter;
     }
 
     protected void reBuffer()
     {
-        limiter.acquire(buffer.length);
+        limiter.acquire(buffer.capacity());
         super.reBuffer();
     }
 
-    public static ThrottledReader open(File file, long overrideLength, RateLimiter limiter)
+    public static ThrottledReader open(ChannelProxy channel, long overrideLength, RateLimiter limiter)
     {
         try
         {
-            return new ThrottledReader(file, overrideLength, limiter);
+            return new ThrottledReader(channel, overrideLength, limiter);
         }
         catch (FileNotFoundException e)
         {
diff --git a/src/java/org/apache/cassandra/io/util/UnbufferedDataOutputStreamPlus.java b/src/java/org/apache/cassandra/io/util/UnbufferedDataOutputStreamPlus.java
new file mode 100644
index 0000000..10aefa6
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/util/UnbufferedDataOutputStreamPlus.java
@@ -0,0 +1,385 @@
+/*
+ * 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.io.util;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.UTFDataFormatException;
+import java.nio.ByteBuffer;
+import java.nio.channels.WritableByteChannel;
+
+import org.apache.cassandra.utils.memory.MemoryUtil;
+
+import com.google.common.base.Function;
+
+/**
+ * Base class for DataOutput implementations that does not have an optimized implementations of Plus methods
+ * and does no buffering.
+ * <p>
+ * Unlike BufferedDataOutputStreamPlus this is capable of operating as an unbuffered output stream.
+ * Currently necessary because SequentialWriter implements its own buffering along with mark/reset/truncate.
+ * </p>
+ */
+public abstract class UnbufferedDataOutputStreamPlus extends DataOutputStreamPlus
+{
+    private static final byte[] zeroBytes = new byte[2];
+
+    protected UnbufferedDataOutputStreamPlus()
+    {
+        super();
+    }
+
+    protected UnbufferedDataOutputStreamPlus(WritableByteChannel channel)
+    {
+        super(channel);
+    }
+
+    /*
+    !! DataOutput methods below are copied from the implementation in Apache Harmony RandomAccessFile.
+    */
+
+    /**
+     * Writes the entire contents of the byte array <code>buffer</code> to
+     * this RandomAccessFile starting at the current file pointer.
+     *
+     * @param buffer the buffer to be written.
+     * @throws IOException If an error occurs trying to write to this RandomAccessFile.
+     */
+    public void write(byte[] buffer) throws IOException
+    {
+        write(buffer, 0, buffer.length);
+    }
+
+    /**
+     * Writes <code>count</code> bytes from the byte array <code>buffer</code>
+     * starting at <code>offset</code> to this RandomAccessFile starting at
+     * the current file pointer..
+     *
+     * @param buffer the bytes to be written
+     * @param offset offset in buffer to get bytes
+     * @param count  number of bytes in buffer to write
+     * @throws IOException               If an error occurs attempting to write to this
+     *                                   RandomAccessFile.
+     * @throws IndexOutOfBoundsException If offset or count are outside of bounds.
+     */
+    public abstract void write(byte[] buffer, int offset, int count) throws IOException;
+
+    /**
+     * Writes the specified byte <code>oneByte</code> to this RandomAccessFile
+     * starting at the current file pointer. Only the low order byte of
+     * <code>oneByte</code> is written.
+     *
+     * @param oneByte the byte to be written
+     * @throws IOException If an error occurs attempting to write to this
+     *                     RandomAccessFile.
+     */
+    public abstract void write(int oneByte) throws IOException;
+
+    /**
+     * Writes a boolean to this output stream.
+     *
+     * @param val the boolean value to write to the OutputStream
+     * @throws IOException If an error occurs attempting to write to this
+     *                     DataOutputStream.
+     */
+    public final void writeBoolean(boolean val) throws IOException
+    {
+        write(val ? 1 : 0);
+    }
+
+    /**
+     * Writes a 8-bit byte to this output stream.
+     *
+     * @param val the byte value to write to the OutputStream
+     * @throws java.io.IOException If an error occurs attempting to write to this
+     *                             DataOutputStream.
+     */
+    public final void writeByte(int val) throws IOException
+    {
+        write(val & 0xFF);
+    }
+
+    /**
+     * Writes the low order 8-bit bytes from a String to this output stream.
+     *
+     * @param str the String containing the bytes to write to the OutputStream
+     * @throws IOException If an error occurs attempting to write to this
+     *                     DataOutputStream.
+     */
+    public final void writeBytes(String str) throws IOException
+    {
+        byte bytes[] = new byte[str.length()];
+        for (int index = 0; index < str.length(); index++)
+        {
+            bytes[index] = (byte) (str.charAt(index) & 0xFF);
+        }
+        write(bytes);
+    }
+
+    /**
+     * Writes the specified 16-bit character to the OutputStream. Only the lower
+     * 2 bytes are written with the higher of the 2 bytes written first. This
+     * represents the Unicode value of val.
+     *
+     * @param val the character to be written
+     * @throws IOException If an error occurs attempting to write to this
+     *                     DataOutputStream.
+     */
+    public final void writeChar(int val) throws IOException
+    {
+        write((val >>> 8) & 0xFF);
+        write((val >>> 0) & 0xFF);
+    }
+
+    /**
+     * Writes the specified 16-bit characters contained in str to the
+     * OutputStream. Only the lower 2 bytes of each character are written with
+     * the higher of the 2 bytes written first. This represents the Unicode
+     * value of each character in str.
+     *
+     * @param str the String whose characters are to be written.
+     * @throws IOException If an error occurs attempting to write to this
+     *                     DataOutputStream.
+     */
+    public final void writeChars(String str) throws IOException
+    {
+        byte newBytes[] = new byte[str.length() * 2];
+        for (int index = 0; index < str.length(); index++)
+        {
+            int newIndex = index == 0 ? index : index * 2;
+            newBytes[newIndex] = (byte) ((str.charAt(index) >> 8) & 0xFF);
+            newBytes[newIndex + 1] = (byte) (str.charAt(index) & 0xFF);
+        }
+        write(newBytes);
+    }
+
+    /**
+     * Writes a 64-bit double to this output stream. The resulting output is the
+     * 8 bytes resulting from calling Double.doubleToLongBits().
+     *
+     * @param val the double to be written.
+     * @throws IOException If an error occurs attempting to write to this
+     *                     DataOutputStream.
+     */
+    public final void writeDouble(double val) throws IOException
+    {
+        writeLong(Double.doubleToLongBits(val));
+    }
+
+    /**
+     * Writes a 32-bit float to this output stream. The resulting output is the
+     * 4 bytes resulting from calling Float.floatToIntBits().
+     *
+     * @param val the float to be written.
+     * @throws IOException If an error occurs attempting to write to this
+     *                     DataOutputStream.
+     */
+    public final void writeFloat(float val) throws IOException
+    {
+        writeInt(Float.floatToIntBits(val));
+    }
+
+    /**
+     * Writes a 32-bit int to this output stream. The resulting output is the 4
+     * bytes, highest order first, of val.
+     *
+     * @param val the int to be written.
+     * @throws IOException If an error occurs attempting to write to this
+     *                     DataOutputStream.
+     */
+    public void writeInt(int val) throws IOException
+    {
+        write((val >>> 24) & 0xFF);
+        write((val >>> 16) & 0xFF);
+        write((val >>> 8) & 0xFF);
+        write((val >>> 0) & 0xFF);
+    }
+
+    /**
+     * Writes a 64-bit long to this output stream. The resulting output is the 8
+     * bytes, highest order first, of val.
+     *
+     * @param val the long to be written.
+     * @throws IOException If an error occurs attempting to write to this
+     *                     DataOutputStream.
+     */
+    public void writeLong(long val) throws IOException
+    {
+        write((int) (val >>> 56) & 0xFF);
+        write((int) (val >>> 48) & 0xFF);
+        write((int) (val >>> 40) & 0xFF);
+        write((int) (val >>> 32) & 0xFF);
+        write((int) (val >>> 24) & 0xFF);
+        write((int) (val >>> 16) & 0xFF);
+        write((int) (val >>> 8) & 0xFF);
+        write((int) (val >>> 0) & 0xFF);
+    }
+
+    /**
+     * Writes the specified 16-bit short to the OutputStream. Only the lower 2
+     * bytes are written with the higher of the 2 bytes written first.
+     *
+     * @param val the short to be written
+     * @throws IOException If an error occurs attempting to write to this
+     *                     DataOutputStream.
+     */
+    public void writeShort(int val) throws IOException
+    {
+        writeChar(val);
+    }
+
+    /**
+     * Writes the specified String out in UTF format to the provided DataOutput
+     *
+     * @param str the String to be written in UTF format.
+     * @param out the DataOutput to write the UTF encoded string to
+     * @throws IOException If an error occurs attempting to write to this
+     *                     DataOutputStream.
+     */
+    public static void writeUTF(String str, DataOutput out) throws IOException
+    {
+        int length = str.length();
+        if (length == 0)
+        {
+            out.write(zeroBytes);
+            return;
+        }
+
+        int utfCount = 0;
+        int maxSize = 2;
+        for (int i = 0 ; i < length ; i++)
+        {
+            int ch = str.charAt(i);
+            if ((ch > 0) & (ch <= 127))
+                utfCount += 1;
+            else if (ch <= 2047)
+                utfCount += 2;
+            else
+                utfCount += maxSize = 3;
+        }
+
+        if (utfCount > 65535)
+            throw new UTFDataFormatException(); //$NON-NLS-1$
+
+        byte[] utfBytes = retrieveTemporaryBuffer(utfCount + 2);
+
+        int bufferLength = utfBytes.length;
+        if (utfCount == length)
+        {
+            utfBytes[0] = (byte) (utfCount >> 8);
+            utfBytes[1] = (byte) utfCount;
+            int firstIndex = 2;
+            for (int offset = 0 ; offset < length ; offset += bufferLength)
+            {
+                int runLength = Math.min(bufferLength - firstIndex, length - offset) + firstIndex;
+                offset -= firstIndex;
+                for (int i = firstIndex ; i < runLength; i++)
+                    utfBytes[i] = (byte) str.charAt(offset + i);
+                out.write(utfBytes, 0, runLength);
+                firstIndex = 0;
+            }
+        }
+        else
+        {
+            int utfIndex = 2;
+            int offset = 0;
+            utfBytes[0] = (byte) (utfCount >> 8);
+            utfBytes[1] = (byte) utfCount;
+
+            while (length > 0)
+            {
+                int charRunLength = (utfBytes.length - utfIndex) / maxSize;
+                if (charRunLength < 128 && charRunLength < length)
+                {
+                    out.write(utfBytes, 0, utfIndex);
+                    utfIndex = 0;
+                }
+                if (charRunLength > length)
+                    charRunLength = length;
+
+                for (int i = 0 ; i < charRunLength ; i++)
+                {
+                    char ch = str.charAt(offset + i);
+                    if ((ch > 0) & (ch <= 127))
+                    {
+                        utfBytes[utfIndex++] = (byte) ch;
+                    }
+                    else if (ch <= 2047)
+                    {
+                        utfBytes[utfIndex++] = (byte) (0xc0 | (0x1f & (ch >> 6)));
+                        utfBytes[utfIndex++] = (byte) (0x80 | (0x3f & ch));
+                    }
+                    else
+                    {
+                        utfBytes[utfIndex++] = (byte) (0xe0 | (0x0f & (ch >> 12)));
+                        utfBytes[utfIndex++] = (byte) (0x80 | (0x3f & (ch >> 6)));
+                        utfBytes[utfIndex++] = (byte) (0x80 | (0x3f & ch));
+                    }
+                }
+
+                offset += charRunLength;
+                length -= charRunLength;
+            }
+
+            out.write(utfBytes, 0, utfIndex);
+        }
+    }
+
+    /**
+     * Writes the specified String out in UTF format.
+     *
+     * @param str the String to be written in UTF format.
+     * @throws IOException If an error occurs attempting to write to this
+     *                     DataOutputStream.
+     */
+    public final void writeUTF(String str) throws IOException
+    {
+        writeUTF(str, this);
+    }
+
+    // ByteBuffer to use for defensive copies
+    private final ByteBuffer hollowBufferD = MemoryUtil.getHollowDirectByteBuffer();
+
+    @Override
+    public void write(ByteBuffer buf) throws IOException
+    {
+        if (buf.hasArray())
+        {
+            write(buf.array(), buf.arrayOffset() + buf.position(), buf.remaining());
+        }
+        else
+        {
+            assert buf.isDirect();
+            MemoryUtil.duplicateDirectByteBuffer(buf, hollowBufferD);
+            while (hollowBufferD.hasRemaining())
+                channel.write(hollowBufferD);
+        }
+    }
+
+    public void write(Memory memory, long offset, long length) throws IOException
+    {
+        for (ByteBuffer buffer : memory.asByteBuffers(offset, length))
+            write(buffer);
+    }
+
+    @Override
+    public <R> R applyToChannel(Function<WritableByteChannel, R> f) throws IOException
+    {
+        return f.apply(channel);
+    }
+}
diff --git a/src/java/org/apache/cassandra/io/util/WrappedDataOutputStreamPlus.java b/src/java/org/apache/cassandra/io/util/WrappedDataOutputStreamPlus.java
new file mode 100644
index 0000000..d8c8f0c
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/util/WrappedDataOutputStreamPlus.java
@@ -0,0 +1,68 @@
+/*
+ * 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.io.util;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.channels.WritableByteChannel;
+
+/**
+ * When possible use {@link WrappedDataOutputStreamPlus} instead of this class, as it will
+ * be more efficient when using Plus methods. This class is only for situations where it cannot be used.
+ *
+ * The channel provided by this class is just a wrapper around the output stream.
+ */
+public class WrappedDataOutputStreamPlus extends UnbufferedDataOutputStreamPlus
+{
+    protected final OutputStream out;
+    public WrappedDataOutputStreamPlus(OutputStream out)
+    {
+        super();
+        this.out = out;
+    }
+
+    public WrappedDataOutputStreamPlus(OutputStream out, WritableByteChannel channel)
+    {
+        super(channel);
+        this.out = out;
+    }
+
+    @Override
+    public void write(byte[] buffer, int offset, int count) throws IOException
+    {
+        out.write(buffer, offset, count);
+    }
+
+    @Override
+    public void write(int oneByte) throws IOException
+    {
+        out.write(oneByte);
+    }
+
+    @Override
+    public void close() throws IOException
+    {
+        out.close();
+    }
+
+    @Override
+    public void flush() throws IOException
+    {
+        out.flush();
+    }
+}
diff --git a/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java b/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
index 2178e17..c90c6a1 100644
--- a/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
+++ b/src/java/org/apache/cassandra/locator/AbstractReplicationStrategy.java
@@ -84,7 +84,7 @@
             {
                 if (lastVersion > lastInvalidatedVersion)
                 {
-                    logger.debug("clearing cached endpoints");
+                    logger.trace("clearing cached endpoints");
                     cachedEndpoints.clear();
                     lastInvalidatedVersion = lastVersion;
                 }
@@ -128,18 +128,22 @@
      */
     public abstract List<InetAddress> calculateNaturalEndpoints(Token searchToken, TokenMetadata tokenMetadata);
 
-    public AbstractWriteResponseHandler getWriteResponseHandler(Collection<InetAddress> naturalEndpoints, Collection<InetAddress> pendingEndpoints, ConsistencyLevel consistency_level, Runnable callback, WriteType writeType)
+    public <T> AbstractWriteResponseHandler<T> getWriteResponseHandler(Collection<InetAddress> naturalEndpoints,
+                                                                Collection<InetAddress> pendingEndpoints,
+                                                                ConsistencyLevel consistency_level,
+                                                                Runnable callback,
+                                                                WriteType writeType)
     {
         if (consistency_level.isDatacenterLocal())
         {
             // block for in this context will be localnodes block.
-            return new DatacenterWriteResponseHandler(naturalEndpoints, pendingEndpoints, consistency_level, getKeyspace(), callback, writeType);
+            return new DatacenterWriteResponseHandler<T>(naturalEndpoints, pendingEndpoints, consistency_level, getKeyspace(), callback, writeType);
         }
         else if (consistency_level == ConsistencyLevel.EACH_QUORUM && (this instanceof NetworkTopologyStrategy))
         {
-            return new DatacenterSyncWriteResponseHandler(naturalEndpoints, pendingEndpoints, consistency_level, getKeyspace(), callback, writeType);
+            return new DatacenterSyncWriteResponseHandler<T>(naturalEndpoints, pendingEndpoints, consistency_level, getKeyspace(), callback, writeType);
         }
-        return new WriteResponseHandler(naturalEndpoints, pendingEndpoints, consistency_level, getKeyspace(), callback, writeType);
+        return new WriteResponseHandler<T>(naturalEndpoints, pendingEndpoints, consistency_level, getKeyspace(), callback, writeType);
     }
 
     private Keyspace getKeyspace()
@@ -257,28 +261,20 @@
                                                                         IEndpointSnitch snitch,
                                                                         Map<String, String> strategyOptions)
     {
+        AbstractReplicationStrategy strategy = createInternal(keyspaceName, strategyClass, tokenMetadata, snitch, strategyOptions);
+
+        // Because we used to not properly validate unrecognized options, we only log a warning if we find one.
         try
         {
-            AbstractReplicationStrategy strategy = createInternal(keyspaceName, strategyClass, tokenMetadata, snitch, strategyOptions);
-
-            // Because we used to not properly validate unrecognized options, we only log a warning if we find one.
-            try
-            {
-                strategy.validateExpectedOptions();
-            }
-            catch (ConfigurationException e)
-            {
-                logger.warn("Ignoring {}", e.getMessage());
-            }
-
-            strategy.validateOptions();
-            return strategy;
+            strategy.validateExpectedOptions();
         }
         catch (ConfigurationException e)
         {
-            // If that happens at this point, there is nothing we can do about it.
-            throw new RuntimeException(e);
+            logger.warn("Ignoring {}", e.getMessage());
         }
+
+        strategy.validateOptions();
+        return strategy;
     }
 
     public static void validateReplicationStrategy(String keyspaceName,
diff --git a/src/java/org/apache/cassandra/locator/CloudstackSnitch.java b/src/java/org/apache/cassandra/locator/CloudstackSnitch.java
index 7f72ae2..88c62e9 100644
--- a/src/java/org/apache/cassandra/locator/CloudstackSnitch.java
+++ b/src/java/org/apache/cassandra/locator/CloudstackSnitch.java
@@ -44,9 +44,11 @@
 import org.apache.cassandra.utils.JVMStabilityInspector;
 
 /**
+ * {@code
  * A snitch that assumes a Cloudstack Zone follows the typical convention
  * <country>-<location>-<availability zone> and uses the country/location
  * tuple as a datacenter and the availability zone as a rack
+ * }
  */
 
 public class CloudstackSnitch extends AbstractNetworkTopologySnitch
@@ -170,17 +172,14 @@
         throw new ConfigurationException("No valid DHCP lease file could be found.");
     }
 
-    String csEndpointFromLease(File lease) throws ConfigurationException, IOException
+    String csEndpointFromLease(File lease) throws ConfigurationException
     {
-        BufferedReader reader = null;
-
-        String line = null;
+        String line;
         String endpoint = null;
         Pattern identifierPattern = Pattern.compile("^[ \t]*option dhcp-server-identifier (.*);$");
 
-        try 
+        try (BufferedReader reader = new BufferedReader(new FileReader(lease)))
         {
-            reader = new BufferedReader(new FileReader(lease));
             
             while ((line = reader.readLine()) != null) 
             {
@@ -192,14 +191,10 @@
                     break;
                 }
             }
-        } 
+        }
         catch (Exception e)  
         {
             throw new ConfigurationException("CloudstackSnitch cannot access lease file.");
-        } 
-        finally
-        {
-        	FileUtils.closeQuietly(reader);
         }
 
         if (endpoint == null) 
diff --git a/src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java b/src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java
index c84ca27..9c0c57e 100644
--- a/src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java
+++ b/src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java
@@ -24,6 +24,8 @@
 import java.util.*;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.TimeUnit;
+
+import com.codahale.metrics.ExponentiallyDecayingReservoir;
 import javax.management.MBeanServer;
 import javax.management.ObjectName;
 
@@ -33,7 +35,6 @@
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.FBUtilities;
 
-import com.yammer.metrics.stats.ExponentiallyDecayingSample;
 
 /**
  * A dynamic snitch that sorts endpoints by latency with an adapted phi failure detector
@@ -56,8 +57,8 @@
     private String mbeanName;
     private boolean registered = false;
 
-    private volatile HashMap<InetAddress, Double> scores = new HashMap<InetAddress, Double>();
-    private final ConcurrentHashMap<InetAddress, ExponentiallyDecayingSample> samples = new ConcurrentHashMap<InetAddress, ExponentiallyDecayingSample>();
+    private volatile HashMap<InetAddress, Double> scores = new HashMap<>();
+    private final ConcurrentHashMap<InetAddress, ExponentiallyDecayingReservoir> samples = new ConcurrentHashMap<>();
 
     public final IEndpointSnitch subsnitch;
 
@@ -184,7 +185,7 @@
         {
             Double score = scores.get(inet);
             if (score == null)
-                return;
+                continue;
             subsnitchOrderedScores.add(score);
         }
 
@@ -214,13 +215,11 @@
         if (scored1 == null)
         {
             scored1 = 0.0;
-            receiveTiming(a1, 0);
         }
 
         if (scored2 == null)
         {
             scored2 = 0.0;
-            receiveTiming(a2, 0);
         }
 
         if (scored1.equals(scored2))
@@ -241,10 +240,10 @@
 
     public void receiveTiming(InetAddress host, long latency) // this is cheap
     {
-        ExponentiallyDecayingSample sample = samples.get(host);
+        ExponentiallyDecayingReservoir sample = samples.get(host);
         if (sample == null)
         {
-            ExponentiallyDecayingSample maybeNewSample = new ExponentiallyDecayingSample(WINDOW_SIZE, ALPHA);
+            ExponentiallyDecayingReservoir maybeNewSample = new ExponentiallyDecayingReservoir(WINDOW_SIZE, ALPHA);
             sample = samples.putIfAbsent(host, maybeNewSample);
             if (sample == null)
                 sample = maybeNewSample;
@@ -269,14 +268,14 @@
         // We're going to weight the latency for each host against the worst one we see, to
         // arrive at sort of a 'badness percentage' for them. First, find the worst for each:
         HashMap<InetAddress, Double> newScores = new HashMap<>();
-        for (Map.Entry<InetAddress, ExponentiallyDecayingSample> entry : samples.entrySet())
+        for (Map.Entry<InetAddress, ExponentiallyDecayingReservoir> entry : samples.entrySet())
         {
             double mean = entry.getValue().getSnapshot().getMedian();
             if (mean > maxLatency)
                 maxLatency = mean;
         }
         // now make another pass to do the weighting based on the maximums we found before
-        for (Map.Entry<InetAddress, ExponentiallyDecayingSample> entry: samples.entrySet())
+        for (Map.Entry<InetAddress, ExponentiallyDecayingReservoir> entry: samples.entrySet())
         {
             double score = entry.getValue().getSnapshot().getMedian() / maxLatency;
             // finally, add the severity without any weighting, since hosts scale this relative to their own load and the size of the task causing the severity.
@@ -291,8 +290,7 @@
 
     private void reset()
     {
-        for (ExponentiallyDecayingSample sample : samples.values())
-            sample.clear();
+       samples.clear();
     }
 
     public Map<InetAddress, Double> getScores()
@@ -322,7 +320,7 @@
     {
         InetAddress host = InetAddress.getByName(hostname);
         ArrayList<Double> timings = new ArrayList<Double>();
-        ExponentiallyDecayingSample sample = samples.get(host);
+        ExponentiallyDecayingReservoir sample = samples.get(host);
         if (sample != null)
         {
             for (double time: sample.getSnapshot().getValues())
diff --git a/src/java/org/apache/cassandra/locator/Ec2MultiRegionSnitch.java b/src/java/org/apache/cassandra/locator/Ec2MultiRegionSnitch.java
index 511cbb5..b32ca84 100644
--- a/src/java/org/apache/cassandra/locator/Ec2MultiRegionSnitch.java
+++ b/src/java/org/apache/cassandra/locator/Ec2MultiRegionSnitch.java
@@ -41,20 +41,24 @@
 {
     private static final String PUBLIC_IP_QUERY_URL = "http://169.254.169.254/latest/meta-data/public-ipv4";
     private static final String PRIVATE_IP_QUERY_URL = "http://169.254.169.254/latest/meta-data/local-ipv4";
-    private final InetAddress localPublicAddress;
     private final String localPrivateAddress;
 
     public Ec2MultiRegionSnitch() throws IOException, ConfigurationException
     {
         super();
-        localPublicAddress = InetAddress.getByName(awsApiCall(PUBLIC_IP_QUERY_URL));
+        InetAddress localPublicAddress = InetAddress.getByName(awsApiCall(PUBLIC_IP_QUERY_URL));
         logger.info("EC2Snitch using publicIP as identifier: {}", localPublicAddress);
         localPrivateAddress = awsApiCall(PRIVATE_IP_QUERY_URL);
         // use the Public IP to broadcast Address to other nodes.
         DatabaseDescriptor.setBroadcastAddress(localPublicAddress);
-        DatabaseDescriptor.setBroadcastRpcAddress(localPublicAddress);
+        if (DatabaseDescriptor.getBroadcastRpcAddress() == null)
+        {
+            logger.info("broadcast_rpc_address unset, broadcasting public IP as rpc_address: {}", localPublicAddress);
+            DatabaseDescriptor.setBroadcastRpcAddress(localPublicAddress);
+        }
     }
 
+    @Override
     public void gossiperStarting()
     {
         super.gossiperStarting();
diff --git a/src/java/org/apache/cassandra/locator/EndpointSnitchInfo.java b/src/java/org/apache/cassandra/locator/EndpointSnitchInfo.java
index c2f88bb..bbfabb6 100644
--- a/src/java/org/apache/cassandra/locator/EndpointSnitchInfo.java
+++ b/src/java/org/apache/cassandra/locator/EndpointSnitchInfo.java
@@ -25,6 +25,7 @@
 import javax.management.ObjectName;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.utils.FBUtilities;
 
 public class EndpointSnitchInfo implements EndpointSnitchInfoMBean
 {
@@ -51,6 +52,16 @@
         return DatabaseDescriptor.getEndpointSnitch().getRack(InetAddress.getByName(host));
     }
 
+    public String getDatacenter()
+    {
+        return DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddress());
+    }
+
+    public String getRack()
+    {
+        return DatabaseDescriptor.getEndpointSnitch().getRack(FBUtilities.getBroadcastAddress());
+    }
+
     public String getSnitchName()
     {
         return DatabaseDescriptor.getEndpointSnitch().getClass().getName();
diff --git a/src/java/org/apache/cassandra/locator/EndpointSnitchInfoMBean.java b/src/java/org/apache/cassandra/locator/EndpointSnitchInfoMBean.java
index 84d2499..6de5022 100644
--- a/src/java/org/apache/cassandra/locator/EndpointSnitchInfoMBean.java
+++ b/src/java/org/apache/cassandra/locator/EndpointSnitchInfoMBean.java
@@ -38,6 +38,15 @@
      */
     public String getDatacenter(String host) throws UnknownHostException;
 
+    /**
+     * Provides the Rack name depending on the respective snitch used for this node
+     */
+    public String getRack();
+
+    /**
+     * Provides the Datacenter name depending on the respective snitch used for this node
+     */
+    public String getDatacenter();
 
     /**
      * Provides the snitch name of the cluster
diff --git a/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java b/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
index a792aa8..7c8d95e 100644
--- a/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
+++ b/src/java/org/apache/cassandra/locator/NetworkTopologyStrategy.java
@@ -32,15 +32,17 @@
 import com.google.common.collect.Multimap;
 
 /**
+ * <p>
  * This Replication Strategy takes a property file that gives the intended
  * replication factor in each datacenter.  The sum total of the datacenter
  * replication factor values should be equal to the keyspace replication
  * factor.
- * <p/>
+ * </p>
+ * <p>
  * So for example, if the keyspace replication factor is 6, the
  * datacenter replication factors could be 3, 2, and 1 - so 3 replicas in
  * one datacenter, 2 in another, and 1 in another - totalling 6.
- * <p/>
+ * </p>
  * This class also caches the Endpoints and invalidates the cache if there is a
  * change in the number of tokens.
  */
@@ -69,7 +71,7 @@
         }
 
         datacenters = Collections.unmodifiableMap(newDatacenters);
-        logger.debug("Configured datacenter replicas are {}", FBUtilities.toString(datacenters));
+        logger.trace("Configured datacenter replicas are {}", FBUtilities.toString(datacenters));
     }
 
     /**
@@ -79,13 +81,12 @@
     public List<InetAddress> calculateNaturalEndpoints(Token searchToken, TokenMetadata tokenMetadata)
     {
         // we want to preserve insertion order so that the first added endpoint becomes primary
-        Set<InetAddress> replicas = new LinkedHashSet<InetAddress>();
+        Set<InetAddress> replicas = new LinkedHashSet<>();
         // replicas we have found in each DC
-        Map<String, Set<InetAddress>> dcReplicas = new HashMap<String, Set<InetAddress>>(datacenters.size())
-        {{
-            for (Map.Entry<String, Integer> dc : datacenters.entrySet())
-                put(dc.getKey(), new HashSet<InetAddress>(dc.getValue()));
-        }};
+        Map<String, Set<InetAddress>> dcReplicas = new HashMap<>(datacenters.size());
+        for (Map.Entry<String, Integer> dc : datacenters.entrySet())
+            dcReplicas.put(dc.getKey(), new HashSet<InetAddress>(dc.getValue()));
+
         Topology topology = tokenMetadata.getTopology();
         // all endpoints in each DC, so we can check when we have exhausted all the members of a DC
         Multimap<String, InetAddress> allEndpoints = topology.getDatacenterEndpoints();
@@ -94,18 +95,16 @@
         assert !allEndpoints.isEmpty() && !racks.isEmpty() : "not aware of any cluster members";
 
         // tracks the racks we have already placed replicas in
-        Map<String, Set<String>> seenRacks = new HashMap<String, Set<String>>(datacenters.size())
-        {{
-            for (Map.Entry<String, Integer> dc : datacenters.entrySet())
-                put(dc.getKey(), new HashSet<String>());
-        }};
+        Map<String, Set<String>> seenRacks = new HashMap<>(datacenters.size());
+        for (Map.Entry<String, Integer> dc : datacenters.entrySet())
+            seenRacks.put(dc.getKey(), new HashSet<String>());
+
         // tracks the endpoints that we skipped over while looking for unique racks
         // when we relax the rack uniqueness we can append this to the current result so we don't have to wind back the iterator
-        Map<String, Set<InetAddress>> skippedDcEndpoints = new HashMap<String, Set<InetAddress>>(datacenters.size())
-        {{
-            for (Map.Entry<String, Integer> dc : datacenters.entrySet())
-                put(dc.getKey(), new LinkedHashSet<InetAddress>());
-        }};
+        Map<String, Set<InetAddress>> skippedDcEndpoints = new HashMap<>(datacenters.size());
+        for (Map.Entry<String, Integer> dc : datacenters.entrySet())
+            skippedDcEndpoints.put(dc.getKey(), new LinkedHashSet<InetAddress>());
+
         Iterator<Token> tokenIter = TokenMetadata.ringIterator(tokenMetadata.sortedTokens(), searchToken, false);
         while (tokenIter.hasNext() && !hasSufficientReplicas(dcReplicas, allEndpoints))
         {
diff --git a/src/java/org/apache/cassandra/locator/PendingRangeMaps.java b/src/java/org/apache/cassandra/locator/PendingRangeMaps.java
new file mode 100644
index 0000000..cfeccc4
--- /dev/null
+++ b/src/java/org/apache/cassandra/locator/PendingRangeMaps.java
@@ -0,0 +1,229 @@
+/*
+ *
+ * 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.locator;
+
+import com.google.common.collect.Iterators;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.InetAddress;
+import java.util.*;
+
+public class PendingRangeMaps implements Iterable<Map.Entry<Range<Token>, List<InetAddress>>>
+{
+    private static final Logger logger = LoggerFactory.getLogger(PendingRangeMaps.class);
+
+    /**
+     * We have for NavigableMap to be able to search for ranges containing a token efficiently.
+     *
+     * First two are for non-wrap-around ranges, and the last two are for wrap-around ranges.
+     */
+    // ascendingMap will sort the ranges by the ascending order of right token
+    final NavigableMap<Range<Token>, List<InetAddress>> ascendingMap;
+    /**
+     * sorting end ascending, if ends are same, sorting begin descending, so that token (end, end) will
+     * come before (begin, end] with the same end, and (begin, end) will be selected in the tailMap.
+     */
+    static final Comparator<Range<Token>> ascendingComparator = new Comparator<Range<Token>>()
+        {
+            @Override
+            public int compare(Range<Token> o1, Range<Token> o2)
+            {
+                int res = o1.right.compareTo(o2.right);
+                if (res != 0)
+                    return res;
+
+                return o2.left.compareTo(o1.left);
+            }
+        };
+
+    // ascendingMap will sort the ranges by the descending order of left token
+    final NavigableMap<Range<Token>, List<InetAddress>> descendingMap;
+    /**
+     * sorting begin descending, if begins are same, sorting end descending, so that token (begin, begin) will
+     * come after (begin, end] with the same begin, and (begin, end) won't be selected in the tailMap.
+     */
+    static final Comparator<Range<Token>> descendingComparator = new Comparator<Range<Token>>()
+        {
+            @Override
+            public int compare(Range<Token> o1, Range<Token> o2)
+            {
+                int res = o2.left.compareTo(o1.left);
+                if (res != 0)
+                    return res;
+
+                // if left tokens are same, sort by the descending of the right tokens.
+                return o2.right.compareTo(o1.right);
+            }
+        };
+
+    // these two maps are for warp around ranges.
+    final NavigableMap<Range<Token>, List<InetAddress>> ascendingMapForWrapAround;
+    /**
+     * for wrap around range (begin, end], which begin > end.
+     * Sorting end ascending, if ends are same, sorting begin ascending,
+     * so that token (end, end) will come before (begin, end] with the same end, and (begin, end] will be selected in
+     * the tailMap.
+     */
+    static final Comparator<Range<Token>> ascendingComparatorForWrapAround = new Comparator<Range<Token>>()
+    {
+        @Override
+        public int compare(Range<Token> o1, Range<Token> o2)
+        {
+            int res = o1.right.compareTo(o2.right);
+            if (res != 0)
+                return res;
+
+            return o1.left.compareTo(o2.left);
+        }
+    };
+
+    final NavigableMap<Range<Token>, List<InetAddress>> descendingMapForWrapAround;
+    /**
+     * for wrap around ranges, which begin > end.
+     * Sorting end ascending, so that token (begin, begin) will come after (begin, end] with the same begin,
+     * and (begin, end) won't be selected in the tailMap.
+     */
+    static final Comparator<Range<Token>> descendingComparatorForWrapAround = new Comparator<Range<Token>>()
+    {
+        @Override
+        public int compare(Range<Token> o1, Range<Token> o2)
+        {
+            int res = o2.left.compareTo(o1.left);
+            if (res != 0)
+                return res;
+            return o1.right.compareTo(o2.right);
+        }
+    };
+
+    public PendingRangeMaps()
+    {
+        this.ascendingMap = new TreeMap<Range<Token>, List<InetAddress>>(ascendingComparator);
+        this.descendingMap = new TreeMap<Range<Token>, List<InetAddress>>(descendingComparator);
+        this.ascendingMapForWrapAround = new TreeMap<Range<Token>, List<InetAddress>>(ascendingComparatorForWrapAround);
+        this.descendingMapForWrapAround = new TreeMap<Range<Token>, List<InetAddress>>(descendingComparatorForWrapAround);
+    }
+
+    static final void addToMap(Range<Token> range,
+                               InetAddress address,
+                               NavigableMap<Range<Token>, List<InetAddress>> ascendingMap,
+                               NavigableMap<Range<Token>, List<InetAddress>> descendingMap)
+    {
+        List<InetAddress> addresses = ascendingMap.get(range);
+        if (addresses == null)
+        {
+            addresses = new ArrayList<InetAddress>(1);
+            ascendingMap.put(range, addresses);
+            descendingMap.put(range, addresses);
+        }
+        addresses.add(address);
+    }
+
+    public void addPendingRange(Range<Token> range, InetAddress address)
+    {
+        if (Range.isWrapAround(range.left, range.right))
+        {
+            addToMap(range, address, ascendingMapForWrapAround, descendingMapForWrapAround);
+        }
+        else
+        {
+            addToMap(range, address, ascendingMap, descendingMap);
+        }
+    }
+
+    static final void addIntersections(Set<InetAddress> endpointsToAdd,
+                                       NavigableMap<Range<Token>, List<InetAddress>> smallerMap,
+                                       NavigableMap<Range<Token>, List<InetAddress>> biggerMap)
+    {
+        // find the intersection of two sets
+        for (Range<Token> range : smallerMap.keySet())
+        {
+            List<InetAddress> addresses = biggerMap.get(range);
+            if (addresses != null)
+            {
+                endpointsToAdd.addAll(addresses);
+            }
+        }
+    }
+
+    public Collection<InetAddress> pendingEndpointsFor(Token token)
+    {
+        Set<InetAddress> endpoints = new HashSet<>();
+
+        Range searchRange = new Range(token, token);
+
+        // search for non-wrap-around maps
+        NavigableMap<Range<Token>, List<InetAddress>> ascendingTailMap = ascendingMap.tailMap(searchRange, true);
+        NavigableMap<Range<Token>, List<InetAddress>> descendingTailMap = descendingMap.tailMap(searchRange, false);
+
+        // add intersections of two maps
+        if (ascendingTailMap.size() < descendingTailMap.size())
+        {
+            addIntersections(endpoints, ascendingTailMap, descendingTailMap);
+        }
+        else
+        {
+            addIntersections(endpoints, descendingTailMap, ascendingTailMap);
+        }
+
+        // search for wrap-around sets
+        ascendingTailMap = ascendingMapForWrapAround.tailMap(searchRange, true);
+        descendingTailMap = descendingMapForWrapAround.tailMap(searchRange, false);
+
+        // add them since they are all necessary.
+        for (Map.Entry<Range<Token>, List<InetAddress>> entry : ascendingTailMap.entrySet())
+        {
+            endpoints.addAll(entry.getValue());
+        }
+        for (Map.Entry<Range<Token>, List<InetAddress>> entry : descendingTailMap.entrySet())
+        {
+            endpoints.addAll(entry.getValue());
+        }
+
+        return endpoints;
+    }
+
+    public String printPendingRanges()
+    {
+        StringBuilder sb = new StringBuilder();
+
+        for (Map.Entry<Range<Token>, List<InetAddress>> entry : this)
+        {
+            Range<Token> range = entry.getKey();
+
+            for (InetAddress address : entry.getValue())
+            {
+                sb.append(address).append(':').append(range);
+                sb.append(System.getProperty("line.separator"));
+            }
+        }
+
+        return sb.toString();
+    }
+
+    @Override
+    public Iterator<Map.Entry<Range<Token>, List<InetAddress>>> iterator()
+    {
+        return Iterators.concat(ascendingMap.entrySet().iterator(), ascendingMapForWrapAround.entrySet().iterator());
+    }
+}
diff --git a/src/java/org/apache/cassandra/locator/PropertyFileSnitch.java b/src/java/org/apache/cassandra/locator/PropertyFileSnitch.java
index dc9a3b8..8cc6549 100644
--- a/src/java/org/apache/cassandra/locator/PropertyFileSnitch.java
+++ b/src/java/org/apache/cassandra/locator/PropertyFileSnitch.java
@@ -31,7 +31,6 @@
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.ResourceWatcher;
@@ -39,8 +38,9 @@
 import org.apache.commons.lang3.StringUtils;
 
 /**
+ * <p>
  * Used to determine if two IP's are in the same datacenter or on the same rack.
- * <p/>
+ * </p>
  * Based on a properties file in the following format:
  *
  * 10.0.0.13=DC1:RAC2
@@ -106,7 +106,7 @@
         String[] value = endpointMap.get(endpoint);
         if (value == null)
         {
-            logger.debug("Could not find end point information for {}, will use default", endpoint);
+            logger.trace("Could not find end point information for {}, will use default", endpoint);
             return defaultDCRack;
         }
         return value;
@@ -144,20 +144,14 @@
         String[] reloadedDefaultDCRack = null;
 
         Properties properties = new Properties();
-        InputStream stream = null;
-        try
+        try (InputStream stream = getClass().getClassLoader().getResourceAsStream(SNITCH_PROPERTIES_FILENAME))
         {
-            stream = getClass().getClassLoader().getResourceAsStream(SNITCH_PROPERTIES_FILENAME);
             properties.load(stream);
         }
         catch (Exception e)
         {
             throw new ConfigurationException("Unable to read " + SNITCH_PROPERTIES_FILENAME, e);
         }
-        finally
-        {
-            FileUtils.closeQuietly(stream);
-        }
 
         for (Map.Entry<Object, Object> entry : properties.entrySet())
         {
@@ -192,20 +186,27 @@
                 reloadedMap.put(host, token);
             }
         }
-        if (reloadedDefaultDCRack == null && !reloadedMap.containsKey(FBUtilities.getBroadcastAddress()))
+        InetAddress broadcastAddress = FBUtilities.getBroadcastAddress();
+        String[] localInfo = reloadedMap.get(broadcastAddress);
+        if (reloadedDefaultDCRack == null && localInfo == null)
             throw new ConfigurationException(String.format("Snitch definitions at %s do not define a location for " +
                                                            "this node's broadcast address %s, nor does it provides a default",
-                                                           SNITCH_PROPERTIES_FILENAME, FBUtilities.getBroadcastAddress()));
+                                                           SNITCH_PROPERTIES_FILENAME, broadcastAddress));
+        // OutboundTcpConnectionPool.getEndpoint() converts our broadcast address to local,
+        // make sure we can be found at that as well.
+        InetAddress localAddress = FBUtilities.getLocalAddress();
+        if (!localAddress.equals(broadcastAddress) && !reloadedMap.containsKey(localAddress))
+            reloadedMap.put(localAddress, localInfo);
 
         if (isUpdate && !livenessCheck(reloadedMap, reloadedDefaultDCRack))
             return;
 
-        if (logger.isDebugEnabled())
+        if (logger.isTraceEnabled())
         {
             StringBuilder sb = new StringBuilder();
             for (Map.Entry<InetAddress, String[]> entry : reloadedMap.entrySet())
                 sb.append(entry.getKey()).append(':').append(Arrays.toString(entry.getValue())).append(", ");
-            logger.debug("Loaded network topology from property file: {}", StringUtils.removeEnd(sb.toString(), ", "));
+            logger.trace("Loaded network topology from property file: {}", StringUtils.removeEnd(sb.toString(), ", "));
         }
 
 
diff --git a/src/java/org/apache/cassandra/locator/SimpleSeedProvider.java b/src/java/org/apache/cassandra/locator/SimpleSeedProvider.java
index 6f36cd0..665261d 100644
--- a/src/java/org/apache/cassandra/locator/SimpleSeedProvider.java
+++ b/src/java/org/apache/cassandra/locator/SimpleSeedProvider.java
@@ -17,9 +17,7 @@
  */
 package org.apache.cassandra.locator;
 
-import java.io.InputStream;
 import java.net.InetAddress;
-import java.net.URL;
 import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -28,12 +26,8 @@
 
 import org.apache.cassandra.config.Config;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.SeedProviderDef;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.yaml.snakeyaml.Loader;
-import org.yaml.snakeyaml.TypeDescription;
-import org.yaml.snakeyaml.Yaml;
 
 public class SimpleSeedProvider implements SeedProvider
 {
diff --git a/src/java/org/apache/cassandra/locator/TokenMetadata.java b/src/java/org/apache/cassandra/locator/TokenMetadata.java
index 0427227..aafd7f9 100644
--- a/src/java/org/apache/cassandra/locator/TokenMetadata.java
+++ b/src/java/org/apache/cassandra/locator/TokenMetadata.java
@@ -26,6 +26,7 @@
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
+import com.google.common.base.Optional;
 import com.google.common.collect.*;
 import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
@@ -79,10 +80,13 @@
     // means we can detect and reject the addition of multiple nodes at the same token
     // before one becomes part of the ring.
     private final BiMultiValMap<Token, InetAddress> bootstrapTokens = new BiMultiValMap<>();
+
+    private final BiMap<InetAddress, InetAddress> replacementToOriginal = HashBiMap.create();
+
     // (don't need to record Token here since it's still part of tokenToEndpointMap until it's done leaving)
     private final Set<InetAddress> leavingEndpoints = new HashSet<>();
     // this is a cache of the calculation from {tokenToEndpointMap, bootstrapTokens, leavingEndpoints}
-    private final ConcurrentMap<String, Multimap<Range<Token>, InetAddress>> pendingRanges = new ConcurrentHashMap<>();
+    private final ConcurrentMap<String, PendingRangeMaps> pendingRanges = new ConcurrentHashMap<String, PendingRangeMaps>();
 
     // nodes which are migrating to the new tokens in the ring
     private final Set<Pair<Token, InetAddress>> movingEndpoints = new HashSet<>();
@@ -185,6 +189,7 @@
                 tokenToEndpointMap.removeValue(endpoint);
                 topology.addEndpoint(endpoint);
                 leavingEndpoints.remove(endpoint);
+                replacementToOriginal.remove(endpoint);
                 removeFromMoving(endpoint); // also removing this endpoint from moving
 
                 for (Token token : tokens)
@@ -297,13 +302,17 @@
 
     public void addBootstrapTokens(Collection<Token> tokens, InetAddress endpoint)
     {
+        addBootstrapTokens(tokens, endpoint, null);
+    }
+
+    private void addBootstrapTokens(Collection<Token> tokens, InetAddress endpoint, InetAddress original)
+    {
         assert tokens != null && !tokens.isEmpty();
         assert endpoint != null;
 
         lock.writeLock().lock();
         try
         {
-
             InetAddress oldEndpoint;
 
             for (Token token : tokens)
@@ -313,7 +322,7 @@
                     throw new RuntimeException("Bootstrap Token collision between " + oldEndpoint + " and " + endpoint + " (token " + token);
 
                 oldEndpoint = tokenToEndpointMap.get(token);
-                if (oldEndpoint != null && !oldEndpoint.equals(endpoint))
+                if (oldEndpoint != null && !oldEndpoint.equals(endpoint) && !oldEndpoint.equals(original))
                     throw new RuntimeException("Bootstrap Token collision between " + oldEndpoint + " and " + endpoint + " (token " + token);
             }
 
@@ -328,6 +337,43 @@
         }
     }
 
+    public void addReplaceTokens(Collection<Token> replacingTokens, InetAddress newNode, InetAddress oldNode)
+    {
+        assert replacingTokens != null && !replacingTokens.isEmpty();
+        assert newNode != null && oldNode != null;
+
+        lock.writeLock().lock();
+        try
+        {
+            Collection<Token> oldNodeTokens = tokenToEndpointMap.inverse().get(oldNode);
+            if (!replacingTokens.containsAll(oldNodeTokens) || !oldNodeTokens.containsAll(replacingTokens))
+            {
+                throw new RuntimeException(String.format("Node %s is trying to replace node %s with tokens %s with a " +
+                                                         "different set of tokens %s.", newNode, oldNode, oldNodeTokens,
+                                                         replacingTokens));
+            }
+
+            logger.debug("Replacing {} with {}", newNode, oldNode);
+            replacementToOriginal.put(newNode, oldNode);
+
+            addBootstrapTokens(replacingTokens, newNode, oldNode);
+        }
+        finally
+        {
+            lock.writeLock().unlock();
+        }
+    }
+
+    public Optional<InetAddress> getReplacementNode(InetAddress endpoint)
+    {
+        return Optional.fromNullable(replacementToOriginal.inverse().get(endpoint));
+    }
+
+    public Optional<InetAddress> getReplacingNode(InetAddress endpoint)
+    {
+        return Optional.fromNullable((replacementToOriginal.get(endpoint)));
+    }
+
     public void removeBootstrapTokens(Collection<Token> tokens)
     {
         assert tokens != null && !tokens.isEmpty();
@@ -391,6 +437,10 @@
             tokenToEndpointMap.removeValue(endpoint);
             topology.removeEndpoint(endpoint);
             leavingEndpoints.remove(endpoint);
+            if (replacementToOriginal.remove(endpoint) != null)
+            {
+                logger.debug("Node {} failed during replace.", endpoint);
+            }
             endpointToHostIdMap.remove(endpoint);
             sortedTokens = sortTokens();
             invalidateCachedRings();
@@ -599,12 +649,7 @@
         lock.readLock().lock();
         try
         {
-            TokenMetadata allLeftMetadata = cloneOnlyTokenMap();
-
-            for (InetAddress endpoint : leavingEndpoints)
-                allLeftMetadata.removeEndpoint(endpoint);
-
-            return allLeftMetadata;
+            return removeEndpoints(cloneOnlyTokenMap(), leavingEndpoints);
         }
         finally
         {
@@ -612,6 +657,14 @@
         }
     }
 
+    private static TokenMetadata removeEndpoints(TokenMetadata allLeftMetadata, Set<InetAddress> leavingEndpoints)
+    {
+        for (InetAddress endpoint : leavingEndpoints)
+            allLeftMetadata.removeEndpoint(endpoint);
+
+        return allLeftMetadata;
+    }
+
     /**
      * Create a copy of TokenMetadata with tokenToEndpointMap reflecting situation after all
      * current leave, and move operations have finished.
@@ -673,23 +726,30 @@
         return sortedTokens;
     }
 
-    private Multimap<Range<Token>, InetAddress> getPendingRangesMM(String keyspaceName)
+    public Multimap<Range<Token>, InetAddress> getPendingRangesMM(String keyspaceName)
     {
-        Multimap<Range<Token>, InetAddress> map = pendingRanges.get(keyspaceName);
-        if (map == null)
+        Multimap<Range<Token>, InetAddress> map = HashMultimap.create();
+        PendingRangeMaps pendingRangeMaps = this.pendingRanges.get(keyspaceName);
+
+        if (pendingRangeMaps != null)
         {
-            map = HashMultimap.create();
-            Multimap<Range<Token>, InetAddress> priorMap = pendingRanges.putIfAbsent(keyspaceName, map);
-            if (priorMap != null)
-                map = priorMap;
+            for (Map.Entry<Range<Token>, List<InetAddress>> entry : pendingRangeMaps)
+            {
+                Range<Token> range = entry.getKey();
+                for (InetAddress address : entry.getValue())
+                {
+                    map.put(range, address);
+                }
+            }
         }
+
         return map;
     }
 
     /** a mutable map may be returned but caller should not modify it */
-    public Map<Range<Token>, Collection<InetAddress>> getPendingRanges(String keyspaceName)
+    public PendingRangeMaps getPendingRanges(String keyspaceName)
     {
-        return getPendingRangesMM(keyspaceName).asMap();
+        return this.pendingRanges.get(keyspaceName);
     }
 
     public List<Range<Token>> getPendingRanges(String keyspaceName, InetAddress endpoint)
@@ -705,9 +765,179 @@
         return ranges;
     }
 
-    public void setPendingRanges(String keyspaceName, Multimap<Range<Token>, InetAddress> rangeMap)
+     /**
+     * Calculate pending ranges according to bootsrapping and leaving nodes. Reasoning is:
+     *
+     * (1) When in doubt, it is better to write too much to a node than too little. That is, if
+     * there are multiple nodes moving, calculate the biggest ranges a node could have. Cleaning
+     * up unneeded data afterwards is better than missing writes during movement.
+     * (2) When a node leaves, ranges for other nodes can only grow (a node might get additional
+     * ranges, but it will not lose any of its current ranges as a result of a leave). Therefore
+     * we will first remove _all_ leaving tokens for the sake of calculation and then check what
+     * ranges would go where if all nodes are to leave. This way we get the biggest possible
+     * ranges with regard current leave operations, covering all subsets of possible final range
+     * values.
+     * (3) When a node bootstraps, ranges of other nodes can only get smaller. Without doing
+     * complex calculations to see if multiple bootstraps overlap, we simply base calculations
+     * on the same token ring used before (reflecting situation after all leave operations have
+     * completed). Bootstrapping nodes will be added and removed one by one to that metadata and
+     * checked what their ranges would be. This will give us the biggest possible ranges the
+     * node could have. It might be that other bootstraps make our actual final ranges smaller,
+     * but it does not matter as we can clean up the data afterwards.
+     *
+     * NOTE: This is heavy and ineffective operation. This will be done only once when a node
+     * changes state in the cluster, so it should be manageable.
+     */
+    public void calculatePendingRanges(AbstractReplicationStrategy strategy, String keyspaceName)
     {
-        pendingRanges.put(keyspaceName, rangeMap);
+        // avoid race between both branches - do not use a lock here as this will block any other unrelated operations!
+        synchronized (pendingRanges)
+        {
+            if (bootstrapTokens.isEmpty() && leavingEndpoints.isEmpty() && movingEndpoints.isEmpty())
+            {
+                if (logger.isTraceEnabled())
+                    logger.trace("No bootstrapping, leaving or moving nodes -> empty pending ranges for {}", keyspaceName);
+
+                pendingRanges.put(keyspaceName, new PendingRangeMaps());
+            }
+            else
+            {
+                if (logger.isDebugEnabled())
+                    logger.debug("Starting pending range calculation for {}", keyspaceName);
+
+                long startedAt = System.currentTimeMillis();
+
+                // create clone of current state
+                BiMultiValMap<Token, InetAddress> bootstrapTokens = new BiMultiValMap<>();
+                Set<InetAddress> leavingEndpoints = new HashSet<>();
+                Set<Pair<Token, InetAddress>> movingEndpoints = new HashSet<>();
+                TokenMetadata metadata;
+
+                lock.readLock().lock();
+                try
+                {
+                    bootstrapTokens.putAll(this.bootstrapTokens);
+                    leavingEndpoints.addAll(this.leavingEndpoints);
+                    movingEndpoints.addAll(this.movingEndpoints);
+                    metadata = this.cloneOnlyTokenMap();
+                }
+                finally
+                {
+                    lock.readLock().unlock();
+                }
+
+                pendingRanges.put(keyspaceName, calculatePendingRanges(strategy, metadata, bootstrapTokens,
+                                                                       leavingEndpoints, movingEndpoints));
+                long took = System.currentTimeMillis() - startedAt;
+
+                if (logger.isDebugEnabled())
+                    logger.debug("Pending range calculation for {} completed (took: {}ms)", keyspaceName, took);
+                if (logger.isTraceEnabled())
+                    logger.trace("Calculated pending ranges for {}:\n{}", keyspaceName, (pendingRanges.isEmpty() ? "<empty>" : printPendingRanges()));
+            }
+        }
+    }
+
+    /**
+     * @see TokenMetadata#calculatePendingRanges(AbstractReplicationStrategy, String)
+     */
+    private static PendingRangeMaps calculatePendingRanges(AbstractReplicationStrategy strategy,
+                                                           TokenMetadata metadata,
+                                                           BiMultiValMap<Token, InetAddress> bootstrapTokens,
+                                                           Set<InetAddress> leavingEndpoints,
+                                                           Set<Pair<Token, InetAddress>> movingEndpoints)
+    {
+        PendingRangeMaps newPendingRanges = new PendingRangeMaps();
+
+        Multimap<InetAddress, Range<Token>> addressRanges = strategy.getAddressRanges(metadata);
+
+        // Copy of metadata reflecting the situation after all leave operations are finished.
+        TokenMetadata allLeftMetadata = removeEndpoints(metadata.cloneOnlyTokenMap(), leavingEndpoints);
+
+        // get all ranges that will be affected by leaving nodes
+        Set<Range<Token>> affectedRanges = new HashSet<Range<Token>>();
+        for (InetAddress endpoint : leavingEndpoints)
+            affectedRanges.addAll(addressRanges.get(endpoint));
+
+        // for each of those ranges, find what new nodes will be responsible for the range when
+        // all leaving nodes are gone.
+        for (Range<Token> range : affectedRanges)
+        {
+            Set<InetAddress> currentEndpoints = ImmutableSet.copyOf(strategy.calculateNaturalEndpoints(range.right, metadata));
+            Set<InetAddress> newEndpoints = ImmutableSet.copyOf(strategy.calculateNaturalEndpoints(range.right, allLeftMetadata));
+            for (InetAddress address : Sets.difference(newEndpoints, currentEndpoints))
+            {
+                newPendingRanges.addPendingRange(range, address);
+            }
+        }
+
+        // At this stage newPendingRanges has been updated according to leave operations. We can
+        // now continue the calculation by checking bootstrapping nodes.
+
+        // For each of the bootstrapping nodes, simply add and remove them one by one to
+        // allLeftMetadata and check in between what their ranges would be.
+        Multimap<InetAddress, Token> bootstrapAddresses = bootstrapTokens.inverse();
+        for (InetAddress endpoint : bootstrapAddresses.keySet())
+        {
+            Collection<Token> tokens = bootstrapAddresses.get(endpoint);
+
+            allLeftMetadata.updateNormalTokens(tokens, endpoint);
+            for (Range<Token> range : strategy.getAddressRanges(allLeftMetadata).get(endpoint))
+            {
+                newPendingRanges.addPendingRange(range, endpoint);
+            }
+            allLeftMetadata.removeEndpoint(endpoint);
+        }
+
+        // At this stage newPendingRanges has been updated according to leaving and bootstrapping nodes.
+        // We can now finish the calculation by checking moving nodes.
+
+        // For each of the moving nodes, we do the same thing we did for bootstrapping:
+        // simply add and remove them one by one to allLeftMetadata and check in between what their ranges would be.
+        for (Pair<Token, InetAddress> moving : movingEndpoints)
+        {
+            //Calculate all the ranges which will could be affected. This will include the ranges before and after the move.
+            Set<Range<Token>> moveAffectedRanges = new HashSet<>();
+            InetAddress endpoint = moving.right; // address of the moving node
+            //Add ranges before the move
+            for (Range<Token> range : strategy.getAddressRanges(allLeftMetadata).get(endpoint))
+            {
+                moveAffectedRanges.add(range);
+            }
+
+            allLeftMetadata.updateNormalToken(moving.left, endpoint);
+            //Add ranges after the move
+            for (Range<Token> range : strategy.getAddressRanges(allLeftMetadata).get(endpoint))
+            {
+                moveAffectedRanges.add(range);
+            }
+
+            for(Range<Token> range : moveAffectedRanges)
+            {
+                Set<InetAddress> currentEndpoints = ImmutableSet.copyOf(strategy.calculateNaturalEndpoints(range.right, metadata));
+                Set<InetAddress> newEndpoints = ImmutableSet.copyOf(strategy.calculateNaturalEndpoints(range.right, allLeftMetadata));
+                Set<InetAddress> difference = Sets.difference(newEndpoints, currentEndpoints);
+                for(final InetAddress address : difference)
+                {
+                    Collection<Range<Token>> newRanges = strategy.getAddressRanges(allLeftMetadata).get(address);
+                    Collection<Range<Token>> oldRanges = strategy.getAddressRanges(metadata).get(address);
+                    //We want to get rid of any ranges which the node is currently getting.
+                    newRanges.removeAll(oldRanges);
+
+                    for(Range<Token> newRange : newRanges)
+                    {
+                        for(Range<Token> pendingRange : newRange.subtractAll(oldRanges))
+                        {
+                            newPendingRanges.addPendingRange(pendingRange, address);
+                        }
+                    }
+                }
+            }
+
+            allLeftMetadata.removeEndpoint(endpoint);
+        }
+
+        return newPendingRanges;
     }
 
     public Token getPredecessor(Token token)
@@ -850,12 +1080,14 @@
         lock.writeLock().lock();
         try
         {
-            bootstrapTokens.clear();
             tokenToEndpointMap.clear();
-            topology.clear();
+            endpointToHostIdMap.clear();
+            bootstrapTokens.clear();
             leavingEndpoints.clear();
             pendingRanges.clear();
-            endpointToHostIdMap.clear();
+            movingEndpoints.clear();
+            sortedTokens.clear();
+            topology.clear();
             invalidateCachedRings();
         }
         finally
@@ -922,17 +1154,13 @@
         return sb.toString();
     }
 
-    public String printPendingRanges()
+    private String printPendingRanges()
     {
         StringBuilder sb = new StringBuilder();
 
-        for (Map.Entry<String, Multimap<Range<Token>, InetAddress>> entry : pendingRanges.entrySet())
+        for (PendingRangeMaps pendingRangeMaps : pendingRanges.values())
         {
-            for (Map.Entry<Range<Token>, InetAddress> rmap : entry.getValue().entries())
-            {
-                sb.append(rmap.getValue()).append(':').append(rmap.getKey());
-                sb.append(System.getProperty("line.separator"));
-            }
+            sb.append(pendingRangeMaps.printPendingRanges());
         }
 
         return sb.toString();
@@ -940,18 +1168,11 @@
 
     public Collection<InetAddress> pendingEndpointsFor(Token token, String keyspaceName)
     {
-        Map<Range<Token>, Collection<InetAddress>> ranges = getPendingRanges(keyspaceName);
-        if (ranges.isEmpty())
+        PendingRangeMaps pendingRangeMaps = this.pendingRanges.get(keyspaceName);
+        if (pendingRangeMaps == null)
             return Collections.emptyList();
 
-        Set<InetAddress> endpoints = new HashSet<>();
-        for (Map.Entry<Range<Token>, Collection<InetAddress>> entry : ranges.entrySet())
-        {
-            if (entry.getKey().contains(token))
-                endpoints.addAll(entry.getValue());
-        }
-
-        return endpoints;
+        return pendingRangeMaps.pendingEndpointsFor(token);
     }
 
     /**
diff --git a/src/java/org/apache/cassandra/locator/YamlFileNetworkTopologySnitch.java b/src/java/org/apache/cassandra/locator/YamlFileNetworkTopologySnitch.java
deleted file mode 100644
index 0279e9f..0000000
--- a/src/java/org/apache/cassandra/locator/YamlFileNetworkTopologySnitch.java
+++ /dev/null
@@ -1,473 +0,0 @@
-/*
- * 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.locator;
-
-import java.io.InputStream;
-import java.net.InetAddress;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.gms.ApplicationState;
-import org.apache.cassandra.gms.EndpointState;
-import org.apache.cassandra.gms.Gossiper;
-import org.apache.cassandra.gms.VersionedValue;
-import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.ResourceWatcher;
-import org.apache.cassandra.utils.WrappedRunnable;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.yaml.snakeyaml.TypeDescription;
-import org.yaml.snakeyaml.Yaml;
-import org.yaml.snakeyaml.constructor.Constructor;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Objects;
-import com.google.common.collect.Sets;
-import com.google.common.net.InetAddresses;
-
-/**
- * Network topology snitch that reads its configuration from a YAML file.
- * <p>
- * This snitch supports connections over preferred addresses, such as a data-center-local address, based on the
- * reconnection trick used in {@link Ec2MultiRegionSnitch}. The configuration file, {@code cassandra-topology.yaml}, is
- * checked periodically for updates.
- * </p>
- */
-public class YamlFileNetworkTopologySnitch
-        extends AbstractNetworkTopologySnitch
-{
-
-    private static final Logger logger = LoggerFactory.getLogger(YamlFileNetworkTopologySnitch.class);
-    
-    /**
-     * How often to check the topology configuration file, in milliseconds; defaults to one minute.
-     */
-    private static final int CHECK_PERIOD_IN_MS = 5 * 1000;
-
-    /** Default name for the topology configuration file. */
-    static final String DEFAULT_TOPOLOGY_CONFIG_FILENAME = "cassandra-topology.yaml";
-
-    /** Node data map, keyed by broadcast address. */
-    private volatile Map<InetAddress, NodeData> nodeDataMap;
-
-    /** Node data for this node. */
-    private volatile NodeData localNodeData;
-
-    /** Node data to fall back to when there is no match. */
-    private volatile NodeData defaultNodeData;
-
-    /** Name of the topology configuration file. */
-    private final String topologyConfigFilename;
-
-    /** True if the gossiper has been initialized. */
-    private volatile boolean gossiperInitialized = false;
-
-    /**
-     * Constructor.
-     *
-     * @throws ConfigurationException
-     *             on failure
-     */
-    public YamlFileNetworkTopologySnitch() throws ConfigurationException
-    {
-        this(DEFAULT_TOPOLOGY_CONFIG_FILENAME);
-    }
-
-    /**
-     * Constructor.
-     *
-     * @param topologyConfigFilename
-     *            name of the topology configuration file
-     * @throws ConfigurationException
-     *             on failure
-     */
-    YamlFileNetworkTopologySnitch(final String topologyConfigFilename)
-            throws ConfigurationException
-    {
-        logger.warn("YamlFileNetworkTopologySnitch is deprecated; switch to GossipingPropertyFileSnitch instead");
-        this.topologyConfigFilename = topologyConfigFilename;
-        loadTopologyConfiguration(false);
-
-        try
-        {
-            /*
-             * Check if the topology configuration file is a plain file.
-             */
-            FBUtilities.resourceToFile(topologyConfigFilename);
-
-            final Runnable runnable = new WrappedRunnable()
-            {
-                /**
-                 * Loads the topology.
-                 */
-                protected void runMayThrow() throws ConfigurationException
-                {
-                    loadTopologyConfiguration(true);
-                }
-            };
-            ResourceWatcher.watch(topologyConfigFilename, runnable,
-                    CHECK_PERIOD_IN_MS);
-        }
-        catch (final ConfigurationException e)
-        {
-            logger.debug(
-                    "{} found, but does not look like a plain file. Will not watch it for changes",
-                    topologyConfigFilename);
-        }
-    }
-
-    /**
-     * Returns the name of the rack for the endpoint, or {@code UNKNOWN} if not known.
-     *
-     * @return the name of the data center for the endpoint, or {@code UNKNOWN} if not known
-     */
-    @Override
-    public String getRack(final InetAddress endpoint)
-    {
-        final NodeData nodeData = nodeDataMap.get(endpoint);
-        return nodeData != null ? nodeData.rack : defaultNodeData.rack;
-    }
-
-    /**
-     * Returns the name of the data center for the endpoint, or {@code UNKNOWN} if not known.
-     *
-     * @return the name of the data center for the endpoint, or {@code UNKNOWN} if not known
-     */
-    @Override
-    public String getDatacenter(final InetAddress endpoint)
-    {
-        final NodeData nodeData = nodeDataMap.get(endpoint);
-        return nodeData != null ? nodeData.datacenter
-                : defaultNodeData.datacenter;
-    }
-
-    /**
-     * Root object type for the YAML topology configuration.
-     */
-    public static class TopologyConfig
-    {
-        public List<Datacenter> topology;
-        public String default_dc_name = "UNKNOWN";
-        public String default_rack_name = "UNKNOWN";
-    }
-
-    /**
-     * Data center object type for the YAML topology configuration.
-     */
-    public static class Datacenter
-    {
-        public String dc_name;
-        public List<Rack> racks = Collections.emptyList();
-    }
-
-    /**
-     * Rack object type for the YAML topology configuration.
-     */
-    public static class Rack
-    {
-        public String rack_name;
-        public List<Node> nodes = Collections.emptyList();
-    }
-
-    /**
-     * Node object type for the YAML topology configuration.
-     */
-    public static class Node
-    {
-        public String broadcast_address;
-        public String dc_local_address;
-    }
-
-    public TopologyConfig readConfig() throws ConfigurationException
-    {
-        final TypeDescription topologyConfigTypeDescription = new TypeDescription(TopologyConfig.class);
-        topologyConfigTypeDescription.putListPropertyType("topology", Datacenter.class);
-
-        final TypeDescription topologyTypeDescription = new TypeDescription(Datacenter.class);
-        topologyTypeDescription.putListPropertyType("racks", Rack.class);
-
-        final TypeDescription rackTypeDescription = new TypeDescription(Rack.class);
-        rackTypeDescription.putListPropertyType("nodes", Node.class);
-
-        final Constructor configConstructor = new Constructor(TopologyConfig.class);
-        configConstructor.addTypeDescription(topologyConfigTypeDescription);
-        configConstructor.addTypeDescription(topologyTypeDescription);
-        configConstructor.addTypeDescription(rackTypeDescription);
-
-        final InputStream configFileInputStream = getClass().getClassLoader()
-                                                            .getResourceAsStream(topologyConfigFilename);
-        if (configFileInputStream == null)
-        {
-            throw new ConfigurationException(
-                                            "Could not read topology config file "
-                                            + topologyConfigFilename);
-        }
-        Yaml yaml;
-        TopologyConfig topologyConfig;
-        try
-        {
-            yaml = new Yaml(configConstructor);
-            topologyConfig = (TopologyConfig) yaml.load(configFileInputStream);
-        }
-        finally
-        {
-            FileUtils.closeQuietly(configFileInputStream);
-        }
-
-        return topologyConfig;
-    }
-
-    /**
-     * Loads the topology configuration file.
-     *
-     * @throws ConfigurationException
-     *             on failure
-     */
-    private synchronized void loadTopologyConfiguration(boolean isUpdate)
-            throws ConfigurationException
-    {
-        logger.debug("Loading topology configuration from {}",
-                     topologyConfigFilename);
-
-        loadTopologyConfiguration(isUpdate, readConfig());
-    }
-
-    @VisibleForTesting
-    synchronized void loadTopologyConfiguration(boolean isUpdate, TopologyConfig topologyConfig)
-            throws ConfigurationException
-    {
-        final Map<InetAddress, NodeData> nodeDataMap = new HashMap<>();
-
-        if (topologyConfig.topology == null)
-        {
-            throw new ConfigurationException(
-                    "Topology configuration file is missing the topology section");
-        }
-
-        for (final Datacenter datacenter : topologyConfig.topology)
-        {
-            if (datacenter.dc_name == null)
-            {
-                throw new ConfigurationException(
-                        "Topology configuration file is missing a data center name for some data center");
-            }
-
-            for (final Rack rack : datacenter.racks)
-            {
-                if (rack.rack_name == null)
-                {
-                    throw new ConfigurationException(
-                            String.format(
-                                    "Topology configuration file is missing a rack name for some rack under data center '%s'",
-                                    datacenter.dc_name));
-                }
-
-                for (final Node node : rack.nodes)
-                {
-                    if (node.broadcast_address == null)
-                    {
-                        throw new ConfigurationException(
-                                String.format(
-                                        "Topology configuration file is missing a broadcast address for some node under data center '%s' rack '%s'",
-                                        datacenter.dc_name, rack.rack_name));
-                    }
-
-                    final InetAddress endpoint = InetAddresses
-                            .forString(node.broadcast_address);
-                    final InetAddress dcLocalAddress = node.dc_local_address == null ? null
-                            : InetAddresses.forString(node.dc_local_address);
-
-                    final NodeData nodeData = new NodeData();
-                    nodeData.datacenter = datacenter.dc_name;
-                    nodeData.rack = rack.rack_name;
-                    nodeData.dcLocalAddress = dcLocalAddress;
-
-                    if (nodeDataMap.put(endpoint, nodeData) != null)
-                    {
-                        throw new ConfigurationException(
-                                String.format(
-                                        "IP address '%s' appears more than once in the topology configuration file",
-                                        endpoint));
-                    }
-
-                    if (dcLocalAddress != null
-                            && nodeDataMap.put(dcLocalAddress, nodeData) != null)
-                    {
-                        throw new ConfigurationException(
-                                String.format(
-                                        "IP address '%s' appears more than once in the topology configuration file",
-                                        dcLocalAddress));
-                    }
-                }
-            }
-        }
-
-        final NodeData localNodeData = nodeDataMap.get(FBUtilities
-                .getBroadcastAddress());
-        if (localNodeData == null)
-        {
-            throw new ConfigurationException(
-                    "Topology configuration missing information for the local node");
-        }
-
-        final NodeData defaultNodeData = new NodeData();
-
-        if (topologyConfig.default_dc_name == null)
-        {
-            throw new ConfigurationException(
-                    "default_dc_name must be specified");
-        }
-        if (topologyConfig.default_rack_name == null)
-        {
-            throw new ConfigurationException(
-                    "default_rack_name must be specified");
-        }
-
-        defaultNodeData.datacenter = topologyConfig.default_dc_name;
-        defaultNodeData.rack = topologyConfig.default_rack_name;
-
-        if (isUpdate && !livenessCheck(nodeDataMap, defaultNodeData))
-            return;
-
-        // YAML configuration looks good; now make the changes
-
-        this.nodeDataMap = nodeDataMap;
-        this.localNodeData = localNodeData;
-        this.defaultNodeData = defaultNodeData;
-        maybeSetApplicationState();
-
-        if (logger.isDebugEnabled())
-        {
-            logger.debug(
-                    "Built topology map from config file: localNodeData={}, nodeDataMap={}",
-                    localNodeData, nodeDataMap);
-        }
-
-        if (gossiperInitialized)
-        {
-            StorageService.instance.gossipSnitchInfo();
-        }
-
-        if (isUpdate && StorageService.instance != null)
-            StorageService.instance.updateTopology();
-    }
-
-    /**
-     * We cannot update rack or data-center for a live node, see CASSANDRA-10243.
-     *
-     * @param reloadedMap - the new map of hosts to NodeData
-     * @param reloadedDefaultData - the default NodeData
-     * @return true if we can continue updating (no live host had dc or rack updated)
-     */
-    private boolean livenessCheck(Map<InetAddress, NodeData> reloadedMap, NodeData reloadedDefaultData)
-    {
-        // If the default has changed we must check all live hosts but hopefully we will find a live
-        // host quickly and interrupt the loop. Otherwise we only check the live hosts that were either
-        // in the old set or in the new set
-        Set<InetAddress> hosts = NodeData.isSameDcRack(defaultNodeData, reloadedDefaultData)
-                                 ? Sets.intersection(StorageService.instance.getLiveRingMembers(), // same default
-                                                     Sets.union(nodeDataMap.keySet(), reloadedMap.keySet()))
-                                 : StorageService.instance.getLiveRingMembers(); // default updated
-
-        for (InetAddress host : hosts)
-        {
-            NodeData origValue = nodeDataMap.containsKey(host) ? nodeDataMap.get(host) : defaultNodeData;
-            NodeData updateValue = reloadedMap.containsKey(host) ? reloadedMap.get(host) : reloadedDefaultData;
-
-            if (!NodeData.isSameDcRack(origValue, updateValue))
-            {
-                logger.error("Cannot update data center or rack from {} to {} for live host {}, property file NOT RELOADED",
-                             new String[] { origValue.datacenter, origValue.rack }, // same format as error in PropertyFileSnitch,
-                             new String[] { updateValue.datacenter, updateValue.rack },
-                             host);
-                return false;
-            }
-        }
-
-        return true;
-    }
-
-    /**
-     * be careful about just blindly updating ApplicationState.INTERNAL_IP everytime we read the yaml file,
-     * as that can cause connections to get unnecessarily reset (via IESCS.onChange()).
-     */
-    private void maybeSetApplicationState()
-    {
-        if (localNodeData.dcLocalAddress == null)
-            return;
-        final EndpointState es = Gossiper.instance.getEndpointStateForEndpoint(FBUtilities.getBroadcastAddress());
-        if (es == null)
-            return;
-        final VersionedValue vv = es.getApplicationState(ApplicationState.INTERNAL_IP);
-        if ((vv != null && !vv.value.equals(localNodeData.dcLocalAddress.getHostAddress()))
-            || vv == null)
-        {
-            Gossiper.instance.addLocalApplicationState(ApplicationState.INTERNAL_IP,
-                StorageService.instance.valueFactory.internalIP(localNodeData.dcLocalAddress.getHostAddress()));
-        }
-    }
-
-    /**
-     * Topology data for a node.
-     */
-    private static final class NodeData
-    {
-        /** Data center name. */
-        public String datacenter;
-        /** Rack name. */
-        public String rack;
-        /** Data-center-local address. */
-        public InetAddress dcLocalAddress;
-
-        /**
-         * Returns a simple key-value string representation of this node's data.
-         *
-         * @return a simple key-value string representation of this node's data
-         */
-        public String toString()
-        {
-            return Objects.toStringHelper(this).add("datacenter", datacenter)
-                    .add("rack", rack).add("dcLocalAddress", dcLocalAddress)
-                    .toString();
-        }
-
-        public static boolean isSameDcRack(NodeData a, NodeData b)
-        {
-            return a == b ||
-                   (a != null && Objects.equal(a.datacenter, b.datacenter) && Objects.equal(a.rack, b.rack));
-        }
-    }
-
-    /**
-     * Called in preparation for the initiation of the gossip loop.
-     */
-    @Override
-    public synchronized void gossiperStarting()
-    {
-        gossiperInitialized = true;
-        StorageService.instance.gossipSnitchInfo();
-        Gossiper.instance.register(new ReconnectableSnitchHelper(this, localNodeData.datacenter, true));
-    }
-
-}
diff --git a/src/java/org/apache/cassandra/metrics/CASClientRequestMetrics.java b/src/java/org/apache/cassandra/metrics/CASClientRequestMetrics.java
index 3210d45..4e64cff 100644
--- a/src/java/org/apache/cassandra/metrics/CASClientRequestMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/CASClientRequestMetrics.java
@@ -18,12 +18,14 @@
  */
 package org.apache.cassandra.metrics;
 
-import com.yammer.metrics.*;
-import com.yammer.metrics.core.*;
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.Histogram;
+
+import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
+
 
 public class CASClientRequestMetrics extends ClientRequestMetrics
 {
-
     public final Histogram contention;
     /* Used only for write  */
     public final Counter conditionNotMet;
@@ -32,16 +34,16 @@
 
     public CASClientRequestMetrics(String scope) {
         super(scope);
-        contention = Metrics.newHistogram(factory.createMetricName("ContentionHistogram"), true);
-        conditionNotMet =  Metrics.newCounter(factory.createMetricName("ConditionNotMet"));
-        unfinishedCommit =  Metrics.newCounter(factory.createMetricName("UnfinishedCommit"));
+        contention = Metrics.histogram(factory.createMetricName("ContentionHistogram"), false);
+        conditionNotMet =  Metrics.counter(factory.createMetricName("ConditionNotMet"));
+        unfinishedCommit =  Metrics.counter(factory.createMetricName("UnfinishedCommit"));
     }
 
     public void release()
     {
         super.release();
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("ContentionHistogram"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("ConditionNotMet"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("UnfinishedCommit"));
+        Metrics.remove(factory.createMetricName("ContentionHistogram"));
+        Metrics.remove(factory.createMetricName("ConditionNotMet"));
+        Metrics.remove(factory.createMetricName("UnfinishedCommit"));
     }
 }
diff --git a/src/java/org/apache/cassandra/metrics/CQLMetrics.java b/src/java/org/apache/cassandra/metrics/CQLMetrics.java
index a7076dd..1020e92 100644
--- a/src/java/org/apache/cassandra/metrics/CQLMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/CQLMetrics.java
@@ -17,11 +17,12 @@
  */
 package org.apache.cassandra.metrics;
 
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.RatioGauge;
 import org.apache.cassandra.cql3.QueryProcessor;
-import com.yammer.metrics.Metrics;
-import com.yammer.metrics.core.Counter;
-import com.yammer.metrics.core.Gauge;
-import com.yammer.metrics.util.RatioGauge;
+
+import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
 
 public class CQLMetrics
 {
@@ -36,27 +37,32 @@
 
     public CQLMetrics()
     {
-        regularStatementsExecuted = Metrics.newCounter(factory.createMetricName("RegularStatementsExecuted"));
-        preparedStatementsExecuted = Metrics.newCounter(factory.createMetricName("PreparedStatementsExecuted"));
-        preparedStatementsEvicted = Metrics.newCounter(factory.createMetricName("PreparedStatementsEvicted"));
+        regularStatementsExecuted = Metrics.counter(factory.createMetricName("RegularStatementsExecuted"));
+        preparedStatementsExecuted = Metrics.counter(factory.createMetricName("PreparedStatementsExecuted"));
+        preparedStatementsEvicted = Metrics.counter(factory.createMetricName("PreparedStatementsEvicted"));
 
-        preparedStatementsCount = Metrics.newGauge(factory.createMetricName("PreparedStatementsCount"), new Gauge<Integer>()
+        preparedStatementsCount = Metrics.register(factory.createMetricName("PreparedStatementsCount"), new Gauge<Integer>()
         {
-            public Integer value()
+            public Integer getValue()
             {
                 return QueryProcessor.preparedStatementsCount();
             }
         });
-        preparedStatementsRatio = Metrics.newGauge(factory.createMetricName("PreparedStatementsRatio"), new RatioGauge()
+        preparedStatementsRatio = Metrics.register(factory.createMetricName("PreparedStatementsRatio"), new RatioGauge()
         {
+            public Ratio getRatio()
+            {
+                return Ratio.of(getNumerator(), getDenominator());
+            }
+
             public double getNumerator()
             {
-                return preparedStatementsExecuted.count();
+                return preparedStatementsExecuted.getCount();
             }
 
             public double getDenominator()
             {
-                return regularStatementsExecuted.count() + preparedStatementsExecuted.count();
+                return regularStatementsExecuted.getCount() + preparedStatementsExecuted.getCount();
             }
         });
     }
diff --git a/src/java/org/apache/cassandra/metrics/CacheMetrics.java b/src/java/org/apache/cassandra/metrics/CacheMetrics.java
index 9d5d783..8b00e1c 100644
--- a/src/java/org/apache/cassandra/metrics/CacheMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/CacheMetrics.java
@@ -17,16 +17,15 @@
  */
 package org.apache.cassandra.metrics;
 
-import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 
-import com.yammer.metrics.Metrics;
-import com.yammer.metrics.core.Gauge;
-import com.yammer.metrics.core.Meter;
-import com.yammer.metrics.util.RatioGauge;
-
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Meter;
+import com.codahale.metrics.RatioGauge;
 import org.apache.cassandra.cache.ICache;
 
+import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
+
 /**
  * Metrics for {@code ICache}.
  */
@@ -45,9 +44,6 @@
     /** Total number of cache entries */
     public final Gauge<Integer> entries;
 
-    private final AtomicLong lastRequests = new AtomicLong(0);
-    private final AtomicLong lastHits = new AtomicLong(0);
-
     /**
      * Create metrics for given cache.
      *
@@ -58,57 +54,36 @@
     {
         MetricNameFactory factory = new DefaultNameFactory("Cache", type);
 
-        capacity = Metrics.newGauge(factory.createMetricName("Capacity"), new Gauge<Long>()
+        capacity = Metrics.register(factory.createMetricName("Capacity"), new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 return cache.capacity();
             }
         });
-        hits = Metrics.newMeter(factory.createMetricName("Hits"), "hits", TimeUnit.SECONDS);
-        requests = Metrics.newMeter(factory.createMetricName("Requests"), "requests", TimeUnit.SECONDS);
-        hitRate = Metrics.newGauge(factory.createMetricName("HitRate"), new RatioGauge()
+        hits = Metrics.meter(factory.createMetricName("Hits"));
+        requests = Metrics.meter(factory.createMetricName("Requests"));
+        hitRate = Metrics.register(factory.createMetricName("HitRate"), new RatioGauge()
         {
-            protected double getNumerator()
+            @Override
+            public Ratio getRatio()
             {
-                return hits.count();
-            }
-
-            protected double getDenominator()
-            {
-                return requests.count();
+                return Ratio.of(hits.getCount(), requests.getCount());
             }
         });
-        size = Metrics.newGauge(factory.createMetricName("Size"), new Gauge<Long>()
+        size = Metrics.register(factory.createMetricName("Size"), new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 return cache.weightedSize();
             }
         });
-        entries = Metrics.newGauge(factory.createMetricName("Entries"), new Gauge<Integer>()
+        entries = Metrics.register(factory.createMetricName("Entries"), new Gauge<Integer>()
         {
-            public Integer value()
+            public Integer getValue()
             {
                 return cache.size();
             }
         });
     }
-
-    // for backward compatibility
-    @Deprecated
-    public double getRecentHitRate()
-    {
-        long r = requests.count();
-        long h = hits.count();
-        try
-        {
-            return ((double)(h - lastHits.get())) / (r - lastRequests.get());
-        }
-        finally
-        {
-            lastRequests.set(r);
-            lastHits.set(h);
-        }
-    }
 }
diff --git a/src/java/org/apache/cassandra/metrics/CassandraMetricsRegistry.java b/src/java/org/apache/cassandra/metrics/CassandraMetricsRegistry.java
new file mode 100644
index 0000000..8e5671b
--- /dev/null
+++ b/src/java/org/apache/cassandra/metrics/CassandraMetricsRegistry.java
@@ -0,0 +1,776 @@
+/*
+ * 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.metrics;
+
+import java.lang.management.ManagementFactory;
+import java.lang.reflect.Method;
+import java.util.Locale;
+import java.util.concurrent.TimeUnit;
+
+import com.codahale.metrics.*;
+import javax.management.*;
+
+/**
+ * Makes integrating 3.0 metrics API with 2.0.
+ * <p>
+ * The 3.0 API comes with poor JMX integration
+ * </p>
+ */
+public class CassandraMetricsRegistry extends MetricRegistry
+{
+    public static final CassandraMetricsRegistry Metrics = new CassandraMetricsRegistry();
+
+    private final MBeanServer mBeanServer = ManagementFactory.getPlatformMBeanServer();
+
+    private CassandraMetricsRegistry()
+    {
+        super();
+    }
+
+    public Counter counter(MetricName name)
+    {
+        Counter counter = counter(name.getMetricName());
+        registerMBean(counter, name.getMBeanName());
+
+        return counter;
+    }
+
+    public Meter meter(MetricName name)
+    {
+        Meter meter = meter(name.getMetricName());
+        registerMBean(meter, name.getMBeanName());
+
+        return meter;
+    }
+
+    public Histogram histogram(MetricName name, boolean considerZeroes)
+    {
+        Histogram histogram = register(name, new ClearableHistogram(new DecayingEstimatedHistogramReservoir(considerZeroes)));
+        registerMBean(histogram, name.getMBeanName());
+
+        return histogram;
+    }
+
+    public Timer timer(MetricName name)
+    {
+        Timer timer = register(name, new Timer(new DecayingEstimatedHistogramReservoir()));
+        registerMBean(timer, name.getMBeanName());
+
+        return timer;
+    }
+
+    public <T extends Metric> T register(MetricName name, T metric)
+    {
+        try
+        {
+            register(name.getMetricName(), metric);
+            registerMBean(metric, name.getMBeanName());
+            return metric;
+        }
+        catch (IllegalArgumentException e)
+        {
+            Metric existing = Metrics.getMetrics().get(name.getMetricName());
+            return (T)existing;
+        }
+    }
+
+    public boolean remove(MetricName name)
+    {
+        boolean removed = remove(name.getMetricName());
+
+        try
+        {
+            mBeanServer.unregisterMBean(name.getMBeanName());
+        } catch (Exception ignore) {}
+
+        return removed;
+    }
+
+    public void registerMBean(Metric metric, ObjectName name)
+    {
+        AbstractBean mbean;
+
+        if (metric instanceof Gauge)
+        {
+            mbean = new JmxGauge((Gauge<?>) metric, name);
+        } else if (metric instanceof Counter)
+        {
+            mbean = new JmxCounter((Counter) metric, name);
+        } else if (metric instanceof Histogram)
+        {
+            mbean = new JmxHistogram((Histogram) metric, name);
+        } else if (metric instanceof Meter)
+        {
+            mbean = new JmxMeter((Meter) metric, name, TimeUnit.SECONDS);
+        } else if (metric instanceof Timer)
+        {
+            mbean = new JmxTimer((Timer) metric, name, TimeUnit.SECONDS, TimeUnit.MICROSECONDS);
+        } else
+        {
+            throw new IllegalArgumentException("Unknown metric type: " + metric.getClass());
+        }
+
+        try
+        {
+            mBeanServer.registerMBean(mbean, name);
+        } catch (Exception ignored) {}
+    }
+
+    public interface MetricMBean
+    {
+        ObjectName objectName();
+    }
+
+    private abstract static class AbstractBean implements MetricMBean
+    {
+        private final ObjectName objectName;
+
+        AbstractBean(ObjectName objectName)
+        {
+            this.objectName = objectName;
+        }
+
+        @Override
+        public ObjectName objectName()
+        {
+            return objectName;
+        }
+    }
+
+
+    public interface JmxGaugeMBean extends MetricMBean
+    {
+        Object getValue();
+    }
+
+    private static class JmxGauge extends AbstractBean implements JmxGaugeMBean
+    {
+        private final Gauge<?> metric;
+
+        private JmxGauge(Gauge<?> metric, ObjectName objectName)
+        {
+            super(objectName);
+            this.metric = metric;
+        }
+
+        @Override
+        public Object getValue()
+        {
+            return metric.getValue();
+        }
+    }
+
+    public interface JmxHistogramMBean extends MetricMBean
+    {
+        long getCount();
+
+        long getMin();
+
+        long getMax();
+
+        double getMean();
+
+        double getStdDev();
+
+        double get50thPercentile();
+
+        double get75thPercentile();
+
+        double get95thPercentile();
+
+        double get98thPercentile();
+
+        double get99thPercentile();
+
+        double get999thPercentile();
+
+        long[] values();
+    }
+
+    private static class JmxHistogram extends AbstractBean implements JmxHistogramMBean
+    {
+        private final Histogram metric;
+
+        private JmxHistogram(Histogram metric, ObjectName objectName)
+        {
+            super(objectName);
+            this.metric = metric;
+        }
+
+        @Override
+        public double get50thPercentile()
+        {
+            return metric.getSnapshot().getMedian();
+        }
+
+        @Override
+        public long getCount()
+        {
+            return metric.getCount();
+        }
+
+        @Override
+        public long getMin()
+        {
+            return metric.getSnapshot().getMin();
+        }
+
+        @Override
+        public long getMax()
+        {
+            return metric.getSnapshot().getMax();
+        }
+
+        @Override
+        public double getMean()
+        {
+            return metric.getSnapshot().getMean();
+        }
+
+        @Override
+        public double getStdDev()
+        {
+            return metric.getSnapshot().getStdDev();
+        }
+
+        @Override
+        public double get75thPercentile()
+        {
+            return metric.getSnapshot().get75thPercentile();
+        }
+
+        @Override
+        public double get95thPercentile()
+        {
+            return metric.getSnapshot().get95thPercentile();
+        }
+
+        @Override
+        public double get98thPercentile()
+        {
+            return metric.getSnapshot().get98thPercentile();
+        }
+
+        @Override
+        public double get99thPercentile()
+        {
+            return metric.getSnapshot().get99thPercentile();
+        }
+
+        @Override
+        public double get999thPercentile()
+        {
+            return metric.getSnapshot().get999thPercentile();
+        }
+
+        @Override
+        public long[] values()
+        {
+            return metric.getSnapshot().getValues();
+        }
+    }
+
+    public interface JmxCounterMBean extends MetricMBean
+    {
+        long getCount();
+    }
+
+    private static class JmxCounter extends AbstractBean implements JmxCounterMBean
+    {
+        private final Counter metric;
+
+        private JmxCounter(Counter metric, ObjectName objectName)
+        {
+            super(objectName);
+            this.metric = metric;
+        }
+
+        @Override
+        public long getCount()
+        {
+            return metric.getCount();
+        }
+    }
+
+    public interface JmxMeterMBean extends MetricMBean
+    {
+        long getCount();
+
+        double getMeanRate();
+
+        double getOneMinuteRate();
+
+        double getFiveMinuteRate();
+
+        double getFifteenMinuteRate();
+
+        String getRateUnit();
+    }
+
+    private static class JmxMeter extends AbstractBean implements JmxMeterMBean
+    {
+        private final Metered metric;
+        private final double rateFactor;
+        private final String rateUnit;
+
+        private JmxMeter(Metered metric, ObjectName objectName, TimeUnit rateUnit)
+        {
+            super(objectName);
+            this.metric = metric;
+            this.rateFactor = rateUnit.toSeconds(1);
+            this.rateUnit = "events/" + calculateRateUnit(rateUnit);
+        }
+
+        @Override
+        public long getCount()
+        {
+            return metric.getCount();
+        }
+
+        @Override
+        public double getMeanRate()
+        {
+            return metric.getMeanRate() * rateFactor;
+        }
+
+        @Override
+        public double getOneMinuteRate()
+        {
+            return metric.getOneMinuteRate() * rateFactor;
+        }
+
+        @Override
+        public double getFiveMinuteRate()
+        {
+            return metric.getFiveMinuteRate() * rateFactor;
+        }
+
+        @Override
+        public double getFifteenMinuteRate()
+        {
+            return metric.getFifteenMinuteRate() * rateFactor;
+        }
+
+        @Override
+        public String getRateUnit()
+        {
+            return rateUnit;
+        }
+
+        private String calculateRateUnit(TimeUnit unit)
+        {
+            final String s = unit.toString().toLowerCase(Locale.US);
+            return s.substring(0, s.length() - 1);
+        }
+    }
+
+    public interface JmxTimerMBean extends JmxMeterMBean
+    {
+        double getMin();
+
+        double getMax();
+
+        double getMean();
+
+        double getStdDev();
+
+        double get50thPercentile();
+
+        double get75thPercentile();
+
+        double get95thPercentile();
+
+        double get98thPercentile();
+
+        double get99thPercentile();
+
+        double get999thPercentile();
+
+        long[] values();
+
+        String getDurationUnit();
+    }
+
+    static class JmxTimer extends JmxMeter implements JmxTimerMBean
+    {
+        private final Timer metric;
+        private final double durationFactor;
+        private final String durationUnit;
+
+        private JmxTimer(Timer metric,
+                         ObjectName objectName,
+                         TimeUnit rateUnit,
+                         TimeUnit durationUnit)
+        {
+            super(metric, objectName, rateUnit);
+            this.metric = metric;
+            this.durationFactor = 1.0 / durationUnit.toNanos(1);
+            this.durationUnit = durationUnit.toString().toLowerCase(Locale.US);
+        }
+
+        @Override
+        public double get50thPercentile()
+        {
+            return metric.getSnapshot().getMedian() * durationFactor;
+        }
+
+        @Override
+        public double getMin()
+        {
+            return metric.getSnapshot().getMin() * durationFactor;
+        }
+
+        @Override
+        public double getMax()
+        {
+            return metric.getSnapshot().getMax() * durationFactor;
+        }
+
+        @Override
+        public double getMean()
+        {
+            return metric.getSnapshot().getMean() * durationFactor;
+        }
+
+        @Override
+        public double getStdDev()
+        {
+            return metric.getSnapshot().getStdDev() * durationFactor;
+        }
+
+        @Override
+        public double get75thPercentile()
+        {
+            return metric.getSnapshot().get75thPercentile() * durationFactor;
+        }
+
+        @Override
+        public double get95thPercentile()
+        {
+            return metric.getSnapshot().get95thPercentile() * durationFactor;
+        }
+
+        @Override
+        public double get98thPercentile()
+        {
+            return metric.getSnapshot().get98thPercentile() * durationFactor;
+        }
+
+        @Override
+        public double get99thPercentile()
+        {
+            return metric.getSnapshot().get99thPercentile() * durationFactor;
+        }
+
+        @Override
+        public double get999thPercentile()
+        {
+            return metric.getSnapshot().get999thPercentile() * durationFactor;
+        }
+
+        @Override
+        public long[] values()
+        {
+            return metric.getSnapshot().getValues();
+        }
+
+        @Override
+        public String getDurationUnit()
+        {
+            return durationUnit;
+        }
+    }
+
+    /**
+     * A value class encapsulating a metric's owning class and name.
+     */
+    public static class MetricName implements Comparable<MetricName>
+    {
+        private final String group;
+        private final String type;
+        private final String name;
+        private final String scope;
+        private final String mBeanName;
+
+        /**
+         * Creates a new {@link MetricName} without a scope.
+         *
+         * @param klass the {@link Class} to which the {@link Metric} belongs
+         * @param name  the name of the {@link Metric}
+         */
+        public MetricName(Class<?> klass, String name)
+        {
+            this(klass, name, null);
+        }
+
+        /**
+         * Creates a new {@link MetricName} without a scope.
+         *
+         * @param group the group to which the {@link Metric} belongs
+         * @param type  the type to which the {@link Metric} belongs
+         * @param name  the name of the {@link Metric}
+         */
+        public MetricName(String group, String type, String name)
+        {
+            this(group, type, name, null);
+        }
+
+        /**
+         * Creates a new {@link MetricName} without a scope.
+         *
+         * @param klass the {@link Class} to which the {@link Metric} belongs
+         * @param name  the name of the {@link Metric}
+         * @param scope the scope of the {@link Metric}
+         */
+        public MetricName(Class<?> klass, String name, String scope)
+        {
+            this(klass.getPackage() == null ? "" : klass.getPackage().getName(),
+                    klass.getSimpleName().replaceAll("\\$$", ""),
+                    name,
+                    scope);
+        }
+
+        /**
+         * Creates a new {@link MetricName} without a scope.
+         *
+         * @param group the group to which the {@link Metric} belongs
+         * @param type  the type to which the {@link Metric} belongs
+         * @param name  the name of the {@link Metric}
+         * @param scope the scope of the {@link Metric}
+         */
+        public MetricName(String group, String type, String name, String scope)
+        {
+            this(group, type, name, scope, createMBeanName(group, type, name, scope));
+        }
+
+        /**
+         * Creates a new {@link MetricName} without a scope.
+         *
+         * @param group     the group to which the {@link Metric} belongs
+         * @param type      the type to which the {@link Metric} belongs
+         * @param name      the name of the {@link Metric}
+         * @param scope     the scope of the {@link Metric}
+         * @param mBeanName the 'ObjectName', represented as a string, to use when registering the
+         *                  MBean.
+         */
+        public MetricName(String group, String type, String name, String scope, String mBeanName)
+        {
+            if (group == null || type == null)
+            {
+                throw new IllegalArgumentException("Both group and type need to be specified");
+            }
+            if (name == null)
+            {
+                throw new IllegalArgumentException("Name needs to be specified");
+            }
+            this.group = group;
+            this.type = type;
+            this.name = name;
+            this.scope = scope;
+            this.mBeanName = mBeanName;
+        }
+
+        /**
+         * Returns the group to which the {@link Metric} belongs. For class-based metrics, this will be
+         * the package name of the {@link Class} to which the {@link Metric} belongs.
+         *
+         * @return the group to which the {@link Metric} belongs
+         */
+        public String getGroup()
+        {
+            return group;
+        }
+
+        /**
+         * Returns the type to which the {@link Metric} belongs. For class-based metrics, this will be
+         * the simple class name of the {@link Class} to which the {@link Metric} belongs.
+         *
+         * @return the type to which the {@link Metric} belongs
+         */
+        public String getType()
+        {
+            return type;
+        }
+
+        /**
+         * Returns the name of the {@link Metric}.
+         *
+         * @return the name of the {@link Metric}
+         */
+        public String getName()
+        {
+            return name;
+        }
+
+        public String getMetricName()
+        {
+            return MetricRegistry.name(group, type, name, scope);
+        }
+
+        /**
+         * Returns the scope of the {@link Metric}.
+         *
+         * @return the scope of the {@link Metric}
+         */
+        public String getScope()
+        {
+            return scope;
+        }
+
+        /**
+         * Returns {@code true} if the {@link Metric} has a scope, {@code false} otherwise.
+         *
+         * @return {@code true} if the {@link Metric} has a scope
+         */
+        public boolean hasScope()
+        {
+            return scope != null;
+        }
+
+        /**
+         * Returns the MBean name for the {@link Metric} identified by this metric name.
+         *
+         * @return the MBean name
+         */
+        public ObjectName getMBeanName()
+        {
+
+            String mname = mBeanName;
+
+            if (mname == null)
+                mname = getMetricName();
+
+            try
+            {
+
+                return new ObjectName(mname);
+            } catch (MalformedObjectNameException e)
+            {
+                try
+                {
+                    return new ObjectName(ObjectName.quote(mname));
+                } catch (MalformedObjectNameException e1)
+                {
+                    throw new RuntimeException(e1);
+                }
+            }
+        }
+
+        @Override
+        public boolean equals(Object o)
+        {
+            if (this == o)
+            {
+                return true;
+            }
+            if (o == null || getClass() != o.getClass())
+            {
+                return false;
+            }
+            final MetricName that = (MetricName) o;
+            return mBeanName.equals(that.mBeanName);
+        }
+
+        @Override
+        public int hashCode()
+        {
+            return mBeanName.hashCode();
+        }
+
+        @Override
+        public String toString()
+        {
+            return mBeanName;
+        }
+
+        @Override
+        public int compareTo(MetricName o)
+        {
+            return mBeanName.compareTo(o.mBeanName);
+        }
+
+        private static String createMBeanName(String group, String type, String name, String scope)
+        {
+            final StringBuilder nameBuilder = new StringBuilder();
+            nameBuilder.append(ObjectName.quote(group));
+            nameBuilder.append(":type=");
+            nameBuilder.append(ObjectName.quote(type));
+            if (scope != null)
+            {
+                nameBuilder.append(",scope=");
+                nameBuilder.append(ObjectName.quote(scope));
+            }
+            if (name.length() > 0)
+            {
+                nameBuilder.append(",name=");
+                nameBuilder.append(ObjectName.quote(name));
+            }
+            return nameBuilder.toString();
+        }
+
+        /**
+         * If the group is empty, use the package name of the given class. Otherwise use group
+         *
+         * @param group The group to use by default
+         * @param klass The class being tracked
+         * @return a group for the metric
+         */
+        public static String chooseGroup(String group, Class<?> klass)
+        {
+            if (group == null || group.isEmpty())
+            {
+                group = klass.getPackage() == null ? "" : klass.getPackage().getName();
+            }
+            return group;
+        }
+
+        /**
+         * If the type is empty, use the simple name of the given class. Otherwise use type
+         *
+         * @param type  The type to use by default
+         * @param klass The class being tracked
+         * @return a type for the metric
+         */
+        public static String chooseType(String type, Class<?> klass)
+        {
+            if (type == null || type.isEmpty())
+            {
+                type = klass.getSimpleName().replaceAll("\\$$", "");
+            }
+            return type;
+        }
+
+        /**
+         * If name is empty, use the name of the given method. Otherwise use name
+         *
+         * @param name   The name to use by default
+         * @param method The method being tracked
+         * @return a name for the metric
+         */
+        public static String chooseName(String name, Method method)
+        {
+            if (name == null || name.isEmpty())
+            {
+                name = method.getName();
+            }
+            return name;
+        }
+    }
+}
+
+
diff --git a/src/java/org/apache/cassandra/metrics/ClearableHistogram.java b/src/java/org/apache/cassandra/metrics/ClearableHistogram.java
new file mode 100644
index 0000000..4a081d8
--- /dev/null
+++ b/src/java/org/apache/cassandra/metrics/ClearableHistogram.java
@@ -0,0 +1,48 @@
+/*
+ * 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.metrics;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import com.codahale.metrics.Histogram;
+
+/**
+ * Adds ability to reset a histogram
+ */
+public class ClearableHistogram extends Histogram
+{
+    private final DecayingEstimatedHistogramReservoir reservoirRef;
+
+    /**
+     * Creates a new {@link com.codahale.metrics.Histogram} with the given reservoir.
+     *
+     * @param reservoir the reservoir to create a histogram from
+     */
+    public ClearableHistogram(DecayingEstimatedHistogramReservoir reservoir)
+    {
+        super(reservoir);
+
+        this.reservoirRef = reservoir;
+    }
+
+    @VisibleForTesting
+    public void clear()
+    {
+        reservoirRef.clear();
+    }
+}
diff --git a/src/java/org/apache/cassandra/metrics/ClientMetrics.java b/src/java/org/apache/cassandra/metrics/ClientMetrics.java
index 59bc94d..4a384eb 100644
--- a/src/java/org/apache/cassandra/metrics/ClientMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/ClientMetrics.java
@@ -20,8 +20,10 @@
 
 import java.util.concurrent.Callable;
 
-import com.yammer.metrics.Metrics;
-import com.yammer.metrics.core.Gauge;
+import com.codahale.metrics.Gauge;
+
+import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
+
 
 public class ClientMetrics
 {
@@ -35,15 +37,14 @@
 
     public void addCounter(String name, final Callable<Integer> provider)
     {
-        Metrics.newGauge(factory.createMetricName(name), new Gauge<Integer>()
+        Metrics.register(factory.createMetricName(name), new Gauge<Integer>()
         {
-            public Integer value()
+            public Integer getValue()
             {
                 try
                 {
                     return provider.call();
-                }
-                catch (Exception e)
+                } catch (Exception e)
                 {
                     throw new RuntimeException(e);
                 }
diff --git a/src/java/org/apache/cassandra/metrics/ClientRequestMetrics.java b/src/java/org/apache/cassandra/metrics/ClientRequestMetrics.java
index 1ac3482..e3a6970 100644
--- a/src/java/org/apache/cassandra/metrics/ClientRequestMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/ClientRequestMetrics.java
@@ -20,34 +20,32 @@
  */
 package org.apache.cassandra.metrics;
 
-import java.util.concurrent.TimeUnit;
 
-import com.yammer.metrics.Metrics;
-import com.yammer.metrics.core.Counter;
-import com.yammer.metrics.core.Meter;
+import com.codahale.metrics.Meter;
+
+import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
+
 
 public class ClientRequestMetrics extends LatencyMetrics
 {
-    @Deprecated public static final Counter readTimeouts = Metrics.newCounter(DefaultNameFactory.createMetricName("ClientRequestMetrics", "ReadTimeouts", null));
-    @Deprecated public static final Counter writeTimeouts = Metrics.newCounter(DefaultNameFactory.createMetricName("ClientRequestMetrics", "WriteTimeouts", null));
-    @Deprecated public static final Counter readUnavailables = Metrics.newCounter(DefaultNameFactory.createMetricName("ClientRequestMetrics", "ReadUnavailables", null));
-    @Deprecated public static final Counter writeUnavailables = Metrics.newCounter(DefaultNameFactory.createMetricName("ClientRequestMetrics", "WriteUnavailables", null));
-
     public final Meter timeouts;
     public final Meter unavailables;
+    public final Meter failures;
 
     public ClientRequestMetrics(String scope)
     {
         super("ClientRequest", scope);
 
-        timeouts = Metrics.newMeter(factory.createMetricName("Timeouts"), "timeouts", TimeUnit.SECONDS);
-        unavailables = Metrics.newMeter(factory.createMetricName("Unavailables"), "unavailables", TimeUnit.SECONDS);
+        timeouts = Metrics.meter(factory.createMetricName("Timeouts"));
+        unavailables = Metrics.meter(factory.createMetricName("Unavailables"));
+        failures = Metrics.meter(factory.createMetricName("Failures"));
     }
 
     public void release()
     {
         super.release();
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("Timeouts"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("Unavailables"));
+        Metrics.remove(factory.createMetricName("Timeouts"));
+        Metrics.remove(factory.createMetricName("Unavailables"));
+        Metrics.remove(factory.createMetricName("Failures"));
     }
 }
diff --git a/src/java/org/apache/cassandra/metrics/ColumnFamilyMetrics.java b/src/java/org/apache/cassandra/metrics/ColumnFamilyMetrics.java
index c4a2aa2..40ed2e4 100644
--- a/src/java/org/apache/cassandra/metrics/ColumnFamilyMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/ColumnFamilyMetrics.java
@@ -20,28 +20,29 @@
 import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.TimeUnit;
 
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import com.codahale.metrics.*;
+import com.codahale.metrics.Timer;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.Memtable;
-import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 import org.apache.cassandra.utils.EstimatedHistogram;
 import org.apache.cassandra.utils.TopKSampler;
 
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-import com.yammer.metrics.Metrics;
-import com.yammer.metrics.core.*;
-import com.yammer.metrics.core.Timer;
-import com.yammer.metrics.util.RatioGauge;
+import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
+
 
 /**
  * Metrics for {@link ColumnFamilyStore}.
  */
 public class ColumnFamilyMetrics
 {
+
     /** Total amount of data stored in the memtable that resides on-heap, including column related overhead and overwritten rows. */
     public final Gauge<Long> memtableOnHeapSize;
     /** Total amount of data stored in the memtable that resides off-heap, including column related overhead and overwritten rows. */
@@ -133,17 +134,13 @@
     public final Timer coordinatorScanLatency;
 
     /** Time spent waiting for free memtable space, either on- or off-heap */
-    public final Timer waitingOnFreeMemtableSpace;
+    public final Histogram waitingOnFreeMemtableSpace;
 
     private final MetricNameFactory factory;
-    private static final MetricNameFactory globalNameFactory = new AllColumnFamilyMetricNameFactory();;
+    private static final MetricNameFactory globalNameFactory = new AllColumnFamilyMetricNameFactory();
 
     public final Counter speculativeRetries;
 
-    // for backward compatibility
-    @Deprecated public final EstimatedHistogram sstablesPerRead = new EstimatedHistogram(35);
-    @Deprecated public final EstimatedHistogram recentSSTablesPerRead = new EstimatedHistogram(35);
-    
     public final static LatencyMetrics globalReadLatency = new LatencyMetrics(globalNameFactory, "Read");
     public final static LatencyMetrics globalWriteLatency = new LatencyMetrics(globalNameFactory, "Write");
     public final static LatencyMetrics globalRangeLatency = new LatencyMetrics(globalNameFactory, "Range");
@@ -216,66 +213,66 @@
 
         memtableColumnsCount = createColumnFamilyGauge("MemtableColumnsCount", new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
-                return cfs.getDataTracker().getView().getCurrentMemtable().getOperations();
+                return cfs.getTracker().getView().getCurrentMemtable().getOperations();
             }
         });
         memtableOnHeapSize = createColumnFamilyGauge("MemtableOnHeapSize", new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
-                return cfs.getDataTracker().getView().getCurrentMemtable().getAllocator().onHeap().owns();
+                return cfs.getTracker().getView().getCurrentMemtable().getAllocator().onHeap().owns();
             }
         });
         memtableOffHeapSize = createColumnFamilyGauge("MemtableOffHeapSize", new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
-                return cfs.getDataTracker().getView().getCurrentMemtable().getAllocator().offHeap().owns();
+                return cfs.getTracker().getView().getCurrentMemtable().getAllocator().offHeap().owns();
             }
         });
         memtableLiveDataSize = createColumnFamilyGauge("MemtableLiveDataSize", new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
-                return cfs.getDataTracker().getView().getCurrentMemtable().getLiveDataSize();
+                return cfs.getTracker().getView().getCurrentMemtable().getLiveDataSize();
             }
         });
         allMemtablesOnHeapSize = createColumnFamilyGauge("AllMemtablesHeapSize", new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 long size = 0;
                 for (ColumnFamilyStore cfs2 : cfs.concatWithIndexes())
-                    size += cfs2.getDataTracker().getView().getCurrentMemtable().getAllocator().onHeap().owns();
+                    size += cfs2.getTracker().getView().getCurrentMemtable().getAllocator().onHeap().owns();
                 return size;
             }
         });
         allMemtablesOffHeapSize = createColumnFamilyGauge("AllMemtablesOffHeapSize", new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 long size = 0;
                 for (ColumnFamilyStore cfs2 : cfs.concatWithIndexes())
-                    size += cfs2.getDataTracker().getView().getCurrentMemtable().getAllocator().offHeap().owns();
+                    size += cfs2.getTracker().getView().getCurrentMemtable().getAllocator().offHeap().owns();
                 return size;
             }
         });
         allMemtablesLiveDataSize = createColumnFamilyGauge("AllMemtablesLiveDataSize", new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 long size = 0;
                 for (ColumnFamilyStore cfs2 : cfs.concatWithIndexes())
-                    size += cfs2.getDataTracker().getView().getCurrentMemtable().getLiveDataSize();
+                    size += cfs2.getTracker().getView().getCurrentMemtable().getLiveDataSize();
                 return size;
             }
         });
         memtableSwitchCount = createColumnFamilyCounter("MemtableSwitchCount");
-        estimatedRowSizeHistogram = Metrics.newGauge(factory.createMetricName("EstimatedRowSizeHistogram"), new Gauge<long[]>()
+        estimatedRowSizeHistogram = Metrics.register(factory.createMetricName("EstimatedRowSizeHistogram"), new Gauge<long[]>()
         {
-            public long[] value()
+            public long[] getValue()
             {
                 return combineHistograms(cfs.getSSTables(), new GetHistogram()
                 {
@@ -286,19 +283,19 @@
                 });
             }
         });
-        estimatedRowCount = Metrics.newGauge(factory.createMetricName("EstimatedRowCount"), new Gauge<Long>()
+        estimatedRowCount = Metrics.register(factory.createMetricName("EstimatedRowCount"), new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 long memtablePartitions = 0;
-                for (Memtable memtable : cfs.getDataTracker().getView().getAllMemtables())
+                for (Memtable memtable : cfs.getTracker().getView().getAllMemtables())
                     memtablePartitions += memtable.partitionCount();
                 return SSTableReader.getApproximateKeyCount(cfs.getSSTables()) + memtablePartitions;
             }
         });
-        estimatedColumnCountHistogram = Metrics.newGauge(factory.createMetricName("EstimatedColumnCountHistogram"), new Gauge<long[]>()
+        estimatedColumnCountHistogram = Metrics.register(factory.createMetricName("EstimatedColumnCountHistogram"), new Gauge<long[]>()
         {
-            public long[] value()
+            public long[] getValue()
             {
                 return combineHistograms(cfs.getSSTables(), new GetHistogram()
                 {
@@ -309,10 +306,10 @@
                 });
             }
         });
-        sstablesPerReadHistogram = createColumnFamilyHistogram("SSTablesPerReadHistogram", cfs.keyspace.metric.sstablesPerReadHistogram);
+        sstablesPerReadHistogram = createColumnFamilyHistogram("SSTablesPerReadHistogram", cfs.keyspace.metric.sstablesPerReadHistogram, true);
         compressionRatio = createColumnFamilyGauge("CompressionRatio", new Gauge<Double>()
         {
-            public Double value()
+            public Double getValue()
             {
                 double sum = 0;
                 int total = 0;
@@ -328,7 +325,7 @@
             }
         }, new Gauge<Double>() // global gauge
         {
-            public Double value()
+            public Double getValue()
             {
                 double sum = 0;
                 int total = 0;
@@ -352,23 +349,23 @@
         pendingFlushes = createColumnFamilyCounter("PendingFlushes");
         pendingCompactions = createColumnFamilyGauge("PendingCompactions", new Gauge<Integer>()
         {
-            public Integer value()
+            public Integer getValue()
             {
                 return cfs.getCompactionStrategy().getEstimatedRemainingTasks();
             }
         });
         liveSSTableCount = createColumnFamilyGauge("LiveSSTableCount", new Gauge<Integer>()
         {
-            public Integer value()
+            public Integer getValue()
             {
-                return cfs.getDataTracker().getSSTables().size();
+                return cfs.getTracker().getSSTables().size();
             }
         });
         liveDiskSpaceUsed = createColumnFamilyCounter("LiveDiskSpaceUsed");
         totalDiskSpaceUsed = createColumnFamilyCounter("TotalDiskSpaceUsed");
         minRowSize = createColumnFamilyGauge("MinRowSize", new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 long min = 0;
                 for (SSTableReader sstable : cfs.getSSTables())
@@ -380,19 +377,19 @@
             }
         }, new Gauge<Long>() // global gauge
         {
-            public Long value()
+            public Long getValue()
             {
                 long min = Long.MAX_VALUE;
                 for (Metric cfGauge : allColumnFamilyMetrics.get("MinRowSize"))
                 {
-                    min = Math.min(min, ((Gauge<? extends Number>) cfGauge).value().longValue());
+                    min = Math.min(min, ((Gauge<? extends Number>) cfGauge).getValue().longValue());
                 }
                 return min;
             }
         });
         maxRowSize = createColumnFamilyGauge("MaxRowSize", new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 long max = 0;
                 for (SSTableReader sstable : cfs.getSSTables())
@@ -404,19 +401,19 @@
             }
         }, new Gauge<Long>() // global gauge
         {
-            public Long value()
+            public Long getValue()
             {
                 long max = 0;
                 for (Metric cfGauge : allColumnFamilyMetrics.get("MaxRowSize"))
                 {
-                    max = Math.max(max, ((Gauge<? extends Number>) cfGauge).value().longValue());
+                    max = Math.max(max, ((Gauge<? extends Number>) cfGauge).getValue().longValue());
                 }
                 return max;
             }
         });
         meanRowSize = createColumnFamilyGauge("MeanRowSize", new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 long sum = 0;
                 long count = 0;
@@ -430,7 +427,7 @@
             }
         }, new Gauge<Long>() // global gauge
         {
-            public Long value()
+            public Long getValue()
             {
                 long sum = 0;
                 long count = 0;
@@ -448,7 +445,7 @@
         });
         bloomFilterFalsePositives = createColumnFamilyGauge("BloomFilterFalsePositives", new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 long count = 0L;
                 for (SSTableReader sstable: cfs.getSSTables())
@@ -458,7 +455,7 @@
         });
         recentBloomFilterFalsePositives = createColumnFamilyGauge("RecentBloomFilterFalsePositives", new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 long count = 0L;
                 for (SSTableReader sstable : cfs.getSSTables())
@@ -468,7 +465,7 @@
         });
         bloomFilterFalseRatio = createColumnFamilyGauge("BloomFilterFalseRatio", new Gauge<Double>()
         {
-            public Double value()
+            public Double getValue()
             {
                 long falseCount = 0L;
                 long trueCount = 0L;
@@ -483,7 +480,7 @@
             }
         }, new Gauge<Double>() // global gauge
         {
-            public Double value()
+            public Double getValue()
             {
                 long falseCount = 0L;
                 long trueCount = 0L;
@@ -502,7 +499,7 @@
         });
         recentBloomFilterFalseRatio = createColumnFamilyGauge("RecentBloomFilterFalseRatio", new Gauge<Double>()
         {
-            public Double value()
+            public Double getValue()
             {
                 long falseCount = 0L;
                 long trueCount = 0L;
@@ -517,7 +514,7 @@
             }
         }, new Gauge<Double>() // global gauge
         {
-            public Double value()
+            public Double getValue()
             {
                 long falseCount = 0L;
                 long trueCount = 0L;
@@ -536,7 +533,7 @@
         });
         bloomFilterDiskSpaceUsed = createColumnFamilyGauge("BloomFilterDiskSpaceUsed", new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 long total = 0;
                 for (SSTableReader sst : cfs.getSSTables())
@@ -546,7 +543,7 @@
         });
         bloomFilterOffHeapMemoryUsed = createColumnFamilyGauge("BloomFilterOffHeapMemoryUsed", new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 long total = 0;
                 for (SSTableReader sst : cfs.getSSTables())
@@ -556,7 +553,7 @@
         });
         indexSummaryOffHeapMemoryUsed = createColumnFamilyGauge("IndexSummaryOffHeapMemoryUsed", new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 long total = 0;
                 for (SSTableReader sst : cfs.getSSTables())
@@ -566,7 +563,7 @@
         });
         compressionMetadataOffHeapMemoryUsed = createColumnFamilyGauge("CompressionMetadataOffHeapMemoryUsed", new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 long total = 0;
                 for (SSTableReader sst : cfs.getSSTables())
@@ -575,8 +572,14 @@
             }
         });
         speculativeRetries = createColumnFamilyCounter("SpeculativeRetries");
-        keyCacheHitRate = Metrics.newGauge(factory.createMetricName("KeyCacheHitRate"), new RatioGauge()
+        keyCacheHitRate = Metrics.register(factory.createMetricName("KeyCacheHitRate"), new RatioGauge()
         {
+            @Override
+            public Ratio getRatio()
+            {
+                return Ratio.of(getNumerator(), getDenominator());
+            }
+
             protected double getNumerator()
             {
                 long hits = 0L;
@@ -593,16 +596,16 @@
                 return Math.max(requests, 1); // to avoid NaN.
             }
         });
-        tombstoneScannedHistogram = createColumnFamilyHistogram("TombstoneScannedHistogram", cfs.keyspace.metric.tombstoneScannedHistogram);
-        liveScannedHistogram = createColumnFamilyHistogram("LiveScannedHistogram", cfs.keyspace.metric.liveScannedHistogram);
-        colUpdateTimeDeltaHistogram = createColumnFamilyHistogram("ColUpdateTimeDeltaHistogram", cfs.keyspace.metric.colUpdateTimeDeltaHistogram);
-        coordinatorReadLatency = Metrics.newTimer(factory.createMetricName("CoordinatorReadLatency"), TimeUnit.MICROSECONDS, TimeUnit.SECONDS);
-        coordinatorScanLatency = Metrics.newTimer(factory.createMetricName("CoordinatorScanLatency"), TimeUnit.MICROSECONDS, TimeUnit.SECONDS);
-        waitingOnFreeMemtableSpace = Metrics.newTimer(factory.createMetricName("WaitingOnFreeMemtableSpace"), TimeUnit.MICROSECONDS, TimeUnit.SECONDS);
+        tombstoneScannedHistogram = createColumnFamilyHistogram("TombstoneScannedHistogram", cfs.keyspace.metric.tombstoneScannedHistogram, false);
+        liveScannedHistogram = createColumnFamilyHistogram("LiveScannedHistogram", cfs.keyspace.metric.liveScannedHistogram, false);
+        colUpdateTimeDeltaHistogram = createColumnFamilyHistogram("ColUpdateTimeDeltaHistogram", cfs.keyspace.metric.colUpdateTimeDeltaHistogram, false);
+        coordinatorReadLatency = Metrics.timer(factory.createMetricName("CoordinatorReadLatency"));
+        coordinatorScanLatency = Metrics.timer(factory.createMetricName("CoordinatorScanLatency"));
+        waitingOnFreeMemtableSpace = Metrics.histogram(factory.createMetricName("WaitingOnFreeMemtableSpace"), false);
 
         trueSnapshotsSize = createColumnFamilyGauge("SnapshotsSize", new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 return cfs.trueSnapshotsSize();
             }
@@ -619,8 +622,6 @@
     public void updateSSTableIterated(int count)
     {
         sstablesPerReadHistogram.update(count);
-        recentSSTablesPerRead.add(count);
-        sstablesPerRead.add(count);
     }
 
     /**
@@ -630,19 +631,19 @@
     {
         for(String name : all)
         {
-            allColumnFamilyMetrics.get(name).remove(Metrics.defaultRegistry().allMetrics().get(factory.createMetricName(name)));
-            Metrics.defaultRegistry().removeMetric(factory.createMetricName(name));
+            allColumnFamilyMetrics.get(name).remove(Metrics.getMetrics().get(factory.createMetricName(name).getMetricName()));
+            Metrics.remove(factory.createMetricName(name));
         }
         readLatency.release();
         writeLatency.release();
         rangeLatency.release();
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("EstimatedRowSizeHistogram"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("EstimatedRowCount"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("EstimatedColumnCountHistogram"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("KeyCacheHitRate"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("CoordinatorReadLatency"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("CoordinatorScanLatency"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("WaitingOnFreeMemtableSpace"));
+        Metrics.remove(factory.createMetricName("EstimatedRowSizeHistogram"));
+        Metrics.remove(factory.createMetricName("EstimatedRowCount"));
+        Metrics.remove(factory.createMetricName("EstimatedColumnCountHistogram"));
+        Metrics.remove(factory.createMetricName("KeyCacheHitRate"));
+        Metrics.remove(factory.createMetricName("CoordinatorReadLatency"));
+        Metrics.remove(factory.createMetricName("CoordinatorScanLatency"));
+        Metrics.remove(factory.createMetricName("WaitingOnFreeMemtableSpace"));
     }
 
 
@@ -654,12 +655,12 @@
     {
         return createColumnFamilyGauge(name, gauge, new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 long total = 0;
                 for (Metric cfGauge : allColumnFamilyMetrics.get(name))
                 {
-                    total = total + ((Gauge<? extends Number>) cfGauge).value().longValue();
+                    total = total + ((Gauge<? extends Number>) cfGauge).getValue().longValue();
                 }
                 return total;
             }
@@ -672,10 +673,10 @@
      */
     protected <G,T> Gauge<T> createColumnFamilyGauge(String name, Gauge<T> gauge, Gauge<G> globalGauge)
     {
-        Gauge<T> cfGauge = Metrics.newGauge(factory.createMetricName(name), gauge);
+        Gauge<T> cfGauge = Metrics.register(factory.createMetricName(name), gauge);
         if (register(name, cfGauge))
         {
-            Metrics.newGauge(globalNameFactory.createMetricName(name), globalGauge);
+            Metrics.register(globalNameFactory.createMetricName(name), globalGauge);
         }
         return cfGauge;
     }
@@ -686,17 +687,17 @@
      */
     protected Counter createColumnFamilyCounter(final String name)
     {
-        Counter cfCounter = Metrics.newCounter(factory.createMetricName(name));
+        Counter cfCounter = Metrics.counter(factory.createMetricName(name));
         if (register(name, cfCounter))
         {
-            Metrics.newGauge(globalNameFactory.createMetricName(name), new Gauge<Long>()
+            Metrics.register(globalNameFactory.createMetricName(name), new Gauge<Long>()
             {
-                public Long value()
+                public Long getValue()
                 {
                     long total = 0;
                     for (Metric cfGauge : allColumnFamilyMetrics.get(name))
                     {
-                        total += ((Counter) cfGauge).count();
+                        total += ((Counter) cfGauge).getCount();
                     }
                     return total;
                 }
@@ -709,13 +710,13 @@
      * Create a histogram-like interface that will register both a CF, keyspace and global level
      * histogram and forward any updates to both
      */
-    protected ColumnFamilyHistogram createColumnFamilyHistogram(String name, Histogram keyspaceHistogram) 
+    protected ColumnFamilyHistogram createColumnFamilyHistogram(String name, Histogram keyspaceHistogram, boolean considerZeroes)
     {
-        Histogram cfHistogram = Metrics.newHistogram(factory.createMetricName(name), true);  
+        Histogram cfHistogram = Metrics.histogram(factory.createMetricName(name), considerZeroes);
         register(name, cfHistogram);
-        return new ColumnFamilyHistogram(cfHistogram, keyspaceHistogram, Metrics.newHistogram(globalNameFactory.createMetricName(name), true));
+        return new ColumnFamilyHistogram(cfHistogram, keyspaceHistogram, Metrics.histogram(globalNameFactory.createMetricName(name), considerZeroes));
     }
-    
+
     /**
      * Registers a metric to be removed when unloading CF.
      * @return true if first time metric with that name has been registered
@@ -728,17 +729,17 @@
         return ret;
     }
     
-    public class ColumnFamilyHistogram
+    public static class ColumnFamilyHistogram
     {
         public final Histogram[] all;
         public final Histogram cf;
-        private ColumnFamilyHistogram(Histogram cf, Histogram keyspace, Histogram global) 
+        private ColumnFamilyHistogram(Histogram cf, Histogram keyspace, Histogram global)
         {
             this.cf = cf;
             this.all = new Histogram[]{cf, keyspace, global};
         }
-        
-        public void update(long i) 
+
+        public void update(long i)
         {
             for(Histogram histo : all)
             {
@@ -747,7 +748,7 @@
         }
     }
     
-    class ColumnFamilyMetricNameFactory implements MetricNameFactory
+    static class ColumnFamilyMetricNameFactory implements MetricNameFactory
     {
         private final String keyspaceName;
         private final String columnFamilyName;
@@ -760,7 +761,7 @@
             isIndex = cfs.isIndex();
         }
 
-        public MetricName createMetricName(String metricName)
+        public CassandraMetricsRegistry.MetricName createMetricName(String metricName)
         {
             String groupName = ColumnFamilyMetrics.class.getPackage().getName();
             String type = isIndex ? "IndexColumnFamily" : "ColumnFamily";
@@ -772,20 +773,20 @@
             mbeanName.append(",scope=").append(columnFamilyName);
             mbeanName.append(",name=").append(metricName);
 
-            return new MetricName(groupName, type, metricName, keyspaceName + "." + columnFamilyName, mbeanName.toString());
+            return new CassandraMetricsRegistry.MetricName(groupName, type, metricName, keyspaceName + "." + columnFamilyName, mbeanName.toString());
         }
     }
     
     static class AllColumnFamilyMetricNameFactory implements MetricNameFactory
     {
-        public MetricName createMetricName(String metricName)
+        public CassandraMetricsRegistry.MetricName createMetricName(String metricName)
         {
             String groupName = ColumnFamilyMetrics.class.getPackage().getName(); 
             StringBuilder mbeanName = new StringBuilder();
             mbeanName.append(groupName).append(":");
             mbeanName.append("type=ColumnFamily");
             mbeanName.append(",name=").append(metricName);
-            return new MetricName(groupName, "ColumnFamily", metricName, "all", mbeanName.toString());
+            return new CassandraMetricsRegistry.MetricName(groupName, "ColumnFamily", metricName, "all", mbeanName.toString());
         }
     }
 
diff --git a/src/java/org/apache/cassandra/metrics/CommitLogMetrics.java b/src/java/org/apache/cassandra/metrics/CommitLogMetrics.java
index e9c0719..1da6ed0 100644
--- a/src/java/org/apache/cassandra/metrics/CommitLogMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/CommitLogMetrics.java
@@ -17,14 +17,13 @@
  */
 package org.apache.cassandra.metrics;
 
-import com.yammer.metrics.Metrics;
-import com.yammer.metrics.core.Gauge;
 
-import com.yammer.metrics.core.Timer;
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Timer;
 import org.apache.cassandra.db.commitlog.AbstractCommitLogService;
 import org.apache.cassandra.db.commitlog.CommitLogSegmentManager;
 
-import java.util.concurrent.TimeUnit;
+import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
 
 /**
  * Metrics for commit log
@@ -34,40 +33,44 @@
     public static final MetricNameFactory factory = new DefaultNameFactory("CommitLog");
 
     /** Number of completed tasks */
-    public final Gauge<Long> completedTasks;
+    public Gauge<Long> completedTasks;
     /** Number of pending tasks */
-    public final Gauge<Long> pendingTasks;
+    public Gauge<Long> pendingTasks;
     /** Current size used by all the commit log segments */
-    public final Gauge<Long> totalCommitLogSize;
+    public Gauge<Long> totalCommitLogSize;
     /** Time spent waiting for a CLS to be allocated - under normal conditions this should be zero */
     public final Timer waitingOnSegmentAllocation;
     /** The time spent waiting on CL sync; for Periodic this is only occurs when the sync is lagging its sync interval */
     public final Timer waitingOnCommit;
-
-    public CommitLogMetrics(final AbstractCommitLogService service, final CommitLogSegmentManager allocator)
+    
+    public CommitLogMetrics()
     {
-        completedTasks = Metrics.newGauge(factory.createMetricName("CompletedTasks"), new Gauge<Long>()
+        waitingOnSegmentAllocation = Metrics.timer(factory.createMetricName("WaitingOnSegmentAllocation"));
+        waitingOnCommit = Metrics.timer(factory.createMetricName("WaitingOnCommit"));
+    }
+
+    public void attach(final AbstractCommitLogService service, final CommitLogSegmentManager allocator)
+    {
+        completedTasks = Metrics.register(factory.createMetricName("CompletedTasks"), new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 return service.getCompletedTasks();
             }
         });
-        pendingTasks = Metrics.newGauge(factory.createMetricName("PendingTasks"), new Gauge<Long>()
+        pendingTasks = Metrics.register(factory.createMetricName("PendingTasks"), new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 return service.getPendingTasks();
             }
         });
-        totalCommitLogSize = Metrics.newGauge(factory.createMetricName("TotalCommitLogSize"), new Gauge<Long>()
+        totalCommitLogSize = Metrics.register(factory.createMetricName("TotalCommitLogSize"), new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
-                return allocator.bytesUsed();
+                return allocator.onDiskSize();
             }
         });
-        waitingOnSegmentAllocation = Metrics.newTimer(factory.createMetricName("WaitingOnSegmentAllocation"), TimeUnit.MICROSECONDS, TimeUnit.SECONDS);
-        waitingOnCommit = Metrics.newTimer(factory.createMetricName("WaitingOnCommit"), TimeUnit.MICROSECONDS, TimeUnit.SECONDS);
     }
 }
diff --git a/src/java/org/apache/cassandra/metrics/CompactionMetrics.java b/src/java/org/apache/cassandra/metrics/CompactionMetrics.java
index f7a99e1..eb00728 100644
--- a/src/java/org/apache/cassandra/metrics/CompactionMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/CompactionMetrics.java
@@ -19,12 +19,10 @@
 
 import java.util.*;
 import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
 
-import com.yammer.metrics.Metrics;
-import com.yammer.metrics.core.Counter;
-import com.yammer.metrics.core.Gauge;
-import com.yammer.metrics.core.Meter;
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Meter;
 
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.ColumnFamilyStore;
@@ -32,6 +30,8 @@
 import org.apache.cassandra.db.compaction.CompactionInfo;
 import org.apache.cassandra.db.compaction.CompactionManager;
 
+import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
+
 /**
  * Metrics for compaction.
  */
@@ -53,9 +53,9 @@
 
     public CompactionMetrics(final ThreadPoolExecutor... collectors)
     {
-        pendingTasks = Metrics.newGauge(factory.createMetricName("PendingTasks"), new Gauge<Integer>()
+        pendingTasks = Metrics.register(factory.createMetricName("PendingTasks"), new Gauge<Integer>()
         {
-            public Integer value()
+            public Integer getValue()
             {
                 int n = 0;
                 // add estimate number of compactions need to be done
@@ -68,9 +68,9 @@
                 return n + compactions.size();
             }
         });
-        completedTasks = Metrics.newGauge(factory.createMetricName("CompletedTasks"), new Gauge<Long>()
+        completedTasks = Metrics.register(factory.createMetricName("CompletedTasks"), new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 long completedTasks = 0;
                 for (ThreadPoolExecutor collector : collectors)
@@ -78,8 +78,8 @@
                 return completedTasks;
             }
         });
-        totalCompactionsCompleted = Metrics.newMeter(factory.createMetricName("TotalCompactionsCompleted"), "compaction completed", TimeUnit.SECONDS);
-        bytesCompacted = Metrics.newCounter(factory.createMetricName("BytesCompacted"));
+        totalCompactionsCompleted = Metrics.meter(factory.createMetricName("TotalCompactionsCompleted"));
+        bytesCompacted = Metrics.counter(factory.createMetricName("BytesCompacted"));
     }
 
     public void beginCompaction(CompactionInfo.Holder ci)
diff --git a/src/java/org/apache/cassandra/metrics/ConnectionMetrics.java b/src/java/org/apache/cassandra/metrics/ConnectionMetrics.java
index 6ab2204..f01c06d 100644
--- a/src/java/org/apache/cassandra/metrics/ConnectionMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/ConnectionMetrics.java
@@ -18,11 +18,12 @@
 package org.apache.cassandra.metrics;
 
 import java.net.InetAddress;
-import java.util.concurrent.TimeUnit;
 
-import com.yammer.metrics.Metrics;
-import com.yammer.metrics.core.Gauge;
-import com.yammer.metrics.core.Meter;
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Meter;
+
+import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
+
 
 import org.apache.cassandra.net.OutboundTcpConnectionPool;
 
@@ -34,27 +35,33 @@
     public static final String TYPE_NAME = "Connection";
 
     /** Total number of timeouts happened on this node */
-    public static final Meter totalTimeouts = Metrics.newMeter(DefaultNameFactory.createMetricName(TYPE_NAME, "TotalTimeouts", null), "total timeouts", TimeUnit.SECONDS);
-    private static long recentTimeouts;
+    public static final Meter totalTimeouts = Metrics.meter(DefaultNameFactory.createMetricName(TYPE_NAME, "TotalTimeouts", null));
 
     public final String address;
-    /** Pending tasks for Command(Mutations, Read etc) TCP Connections */
-    public final Gauge<Integer> commandPendingTasks;
-    /** Completed tasks for Command(Mutations, Read etc) TCP Connections */
-    public final Gauge<Long> commandCompletedTasks;
-    /** Dropped tasks for Command(Mutations, Read etc) TCP Connections */
-    public final Gauge<Long> commandDroppedTasks;
-    /** Pending tasks for Response(GOSSIP & RESPONSE) TCP Connections */
-    public final Gauge<Integer> responsePendingTasks;
-    /** Completed tasks for Response(GOSSIP & RESPONSE) TCP Connections */
-    public final Gauge<Long> responseCompletedTasks;
+    /** Pending tasks for large message TCP Connections */
+    public final Gauge<Integer> largeMessagePendingTasks;
+    /** Completed tasks for large message TCP Connections */
+    public final Gauge<Long> largeMessageCompletedTasks;
+    /** Dropped tasks for large message TCP Connections */
+    public final Gauge<Long> largeMessageDroppedTasks;
+    /** Pending tasks for small message TCP Connections */
+    public final Gauge<Integer> smallMessagePendingTasks;
+    /** Completed tasks for small message TCP Connections */
+    public final Gauge<Long> smallMessageCompletedTasks;
+    /** Dropped tasks for small message TCP Connections */
+    public final Gauge<Long> smallMessageDroppedTasks;
+    /** Pending tasks for gossip message TCP Connections */
+    public final Gauge<Integer> gossipMessagePendingTasks;
+    /** Completed tasks for gossip message TCP Connections */
+    public final Gauge<Long> gossipMessageCompletedTasks;
+    /** Dropped tasks for gossip message TCP Connections */
+    public final Gauge<Long> gossipMessageDroppedTasks;
+
     /** Number of timeouts for specific IP */
     public final Meter timeouts;
 
     private final MetricNameFactory factory;
 
-    private long recentTimeoutCount;
-
     /**
      * Create metrics for given connection pool.
      *
@@ -68,69 +75,83 @@
 
         factory = new DefaultNameFactory("Connection", address);
 
-        commandPendingTasks = Metrics.newGauge(factory.createMetricName("CommandPendingTasks"), new Gauge<Integer>()
+        largeMessagePendingTasks = Metrics.register(factory.createMetricName("LargeMessagePendingTasks"), new Gauge<Integer>()
         {
-            public Integer value()
+            public Integer getValue()
             {
-                return connectionPool.cmdCon.getPendingMessages();
+                return connectionPool.largeMessages.getPendingMessages();
             }
         });
-        commandCompletedTasks = Metrics.newGauge(factory.createMetricName("CommandCompletedTasks"), new Gauge<Long>()
+        largeMessageCompletedTasks = Metrics.register(factory.createMetricName("LargeMessageCompletedTasks"), new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
-                return connectionPool.cmdCon.getCompletedMesssages();
+                return connectionPool.largeMessages.getCompletedMesssages();
             }
         });
-        commandDroppedTasks = Metrics.newGauge(factory.createMetricName("CommandDroppedTasks"), new Gauge<Long>()
+        largeMessageDroppedTasks = Metrics.register(factory.createMetricName("LargeMessageDroppedTasks"), new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
-                return connectionPool.cmdCon.getDroppedMessages();
+                return connectionPool.largeMessages.getDroppedMessages();
             }
         });
-        responsePendingTasks = Metrics.newGauge(factory.createMetricName("ResponsePendingTasks"), new Gauge<Integer>()
+        smallMessagePendingTasks = Metrics.register(factory.createMetricName("SmallMessagePendingTasks"), new Gauge<Integer>()
         {
-            public Integer value()
+            public Integer getValue()
             {
-                return connectionPool.ackCon.getPendingMessages();
+                return connectionPool.smallMessages.getPendingMessages();
             }
         });
-        responseCompletedTasks = Metrics.newGauge(factory.createMetricName("ResponseCompletedTasks"), new Gauge<Long>()
+        smallMessageCompletedTasks = Metrics.register(factory.createMetricName("SmallMessageCompletedTasks"), new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
-                return connectionPool.ackCon.getCompletedMesssages();
+                return connectionPool.smallMessages.getCompletedMesssages();
             }
         });
-        timeouts = Metrics.newMeter(factory.createMetricName("Timeouts"), "timeouts", TimeUnit.SECONDS);
+        smallMessageDroppedTasks = Metrics.register(factory.createMetricName("SmallMessageDroppedTasks"), new Gauge<Long>()
+        {
+            public Long getValue()
+            {
+                return connectionPool.smallMessages.getDroppedMessages();
+            }
+        });
+        gossipMessagePendingTasks = Metrics.register(factory.createMetricName("GossipMessagePendingTasks"), new Gauge<Integer>()
+        {
+            public Integer getValue()
+            {
+                return connectionPool.gossipMessages.getPendingMessages();
+            }
+        });
+        gossipMessageCompletedTasks = Metrics.register(factory.createMetricName("GossipMessageCompletedTasks"), new Gauge<Long>()
+        {
+            public Long getValue()
+            {
+                return connectionPool.gossipMessages.getCompletedMesssages();
+            }
+        });
+        gossipMessageDroppedTasks = Metrics.register(factory.createMetricName("GossipMessageDroppedTasks"), new Gauge<Long>()
+        {
+            public Long getValue()
+            {
+                return connectionPool.gossipMessages.getDroppedMessages();
+            }
+        });
+        timeouts = Metrics.meter(factory.createMetricName("Timeouts"));
     }
 
     public void release()
     {
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("CommandPendingTasks"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("CommandCompletedTasks"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("CommandDroppedTasks"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("ResponsePendingTasks"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("ResponseCompletedTasks"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("Timeouts"));
-    }
-
-    @Deprecated
-    public static long getRecentTotalTimeout()
-    {
-        long total = totalTimeouts.count();
-        long recent = total - recentTimeouts;
-        recentTimeouts = total;
-        return recent;
-    }
-
-    @Deprecated
-    public long getRecentTimeout()
-    {
-        long timeoutCount = timeouts.count();
-        long recent = timeoutCount - recentTimeoutCount;
-        recentTimeoutCount = timeoutCount;
-        return recent;
+        Metrics.remove(factory.createMetricName("LargeMessagePendingTasks"));
+        Metrics.remove(factory.createMetricName("LargeMessageCompletedTasks"));
+        Metrics.remove(factory.createMetricName("LargeMessageDroppedTasks"));
+        Metrics.remove(factory.createMetricName("SmallMessagePendingTasks"));
+        Metrics.remove(factory.createMetricName("SmallMessageCompletedTasks"));
+        Metrics.remove(factory.createMetricName("SmallMessageDroppedTasks"));
+        Metrics.remove(factory.createMetricName("GossipMessagePendingTasks"));
+        Metrics.remove(factory.createMetricName("GossipMessageCompletedTasks"));
+        Metrics.remove(factory.createMetricName("GossipMessageDroppedTasks"));
+        Metrics.remove(factory.createMetricName("Timeouts"));
     }
 }
diff --git a/src/java/org/apache/cassandra/metrics/DecayingEstimatedHistogramReservoir.java b/src/java/org/apache/cassandra/metrics/DecayingEstimatedHistogramReservoir.java
new file mode 100644
index 0000000..2f8bdf8
--- /dev/null
+++ b/src/java/org/apache/cassandra/metrics/DecayingEstimatedHistogramReservoir.java
@@ -0,0 +1,550 @@
+/*
+ * 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.metrics;
+
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.PrintWriter;
+import java.nio.charset.Charset;
+import java.util.Arrays;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicLongArray;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import com.codahale.metrics.Clock;
+import com.codahale.metrics.Reservoir;
+import com.codahale.metrics.Snapshot;
+import org.apache.cassandra.utils.EstimatedHistogram;
+
+/**
+ * A decaying histogram reservoir where values collected during each minute will be twice as significant as the values
+ * collected in the previous minute. Measured values are collected in variable sized buckets, using small buckets in the
+ * lower range and larger buckets in the upper range. Use this histogram when you want to know if the distribution of
+ * the underlying data stream has changed recently and you want high resolution on values in the lower range.
+ *
+ * The histogram use forward decay [1] to make recent values more significant. The forward decay factor will be doubled
+ * every minute (half-life time set to 60 seconds) [2]. The forward decay landmark is reset every 30 minutes (or at
+ * first read/update after 30 minutes). During landmark reset, updates and reads in the reservoir will be blocked in a
+ * fashion similar to the one used in the metrics library [3]. The 30 minute rescale interval is used based on the
+ * assumption that in an extreme case we would have to collect a metric 1M times for a single bucket each second. By the
+ * end of the 30:th minute all collected values will roughly add up to 1.000.000 * 60 * pow(2, 30) which can be
+ * represented with 56 bits giving us some head room in a signed 64 bit long.
+ *
+ * Internally two reservoirs are maintained, one with decay and one without decay. All public getters in a {@Snapshot}
+ * will expose the decay functionality with the exception of the {@link Snapshot#getValues()} which will return values
+ * from the reservoir without decay. This makes it possible for the caller to maintain precise deltas in an interval of
+ * its choise.
+ *
+ * The bucket size starts at 1 and grows by 1.2 each time (rounding and removing duplicates). It goes from 1 to around
+ * 18T by default (creating 164+1 buckets), which will give a timing resolution from microseconds to roughly 210 days,
+ * with less precision as the numbers get larger.
+ *
+ * The series of values to which the counts in `decayingBuckets` correspond:
+ * 1, 2, 3, 4, 5, 6, 7, 8, 10, 12, 14, 17, 20, 24, 29, 35, 42, 50, 60, 72 etc.
+ * Thus, a `decayingBuckets` of [0, 0, 1, 10] would mean we had seen 1 value of 3 and 10 values of 4.
+ *
+ * Each bucket represents values from (previous bucket offset, current offset].
+ *
+ * [1]: http://dimacs.rutgers.edu/~graham/pubs/papers/fwddecay.pdf
+ * [2]: https://en.wikipedia.org/wiki/Half-life
+ * [3]: https://github.com/dropwizard/metrics/blob/v3.1.2/metrics-core/src/main/java/com/codahale/metrics/ExponentiallyDecayingReservoir.java
+ */
+public class DecayingEstimatedHistogramReservoir implements Reservoir
+{
+    /**
+     * The default number of decayingBuckets. Use this bucket count to reduce memory allocation for bucket offsets.
+     */
+    public static final int DEFAULT_BUCKET_COUNT = 164;
+    public static final boolean DEFAULT_ZERO_CONSIDERATION = false;
+
+    // The offsets used with a default sized bucket array without a separate bucket for zero values.
+    public static final long[] DEFAULT_WITHOUT_ZERO_BUCKET_OFFSETS = EstimatedHistogram.newOffsets(DEFAULT_BUCKET_COUNT, false);
+
+    // The offsets used with a default sized bucket array with a separate bucket for zero values.
+    public static final long[] DEFAULT_WITH_ZERO_BUCKET_OFFSETS = EstimatedHistogram.newOffsets(DEFAULT_BUCKET_COUNT, true);
+
+    // Represents the bucket offset as created by {@link EstimatedHistogram#newOffsets()}
+    private final long[] bucketOffsets;
+
+    // decayingBuckets and buckets are one element longer than bucketOffsets -- the last element is values greater than the last offset
+    private final AtomicLongArray decayingBuckets;
+    private final AtomicLongArray buckets;
+
+    public static final long HALF_TIME_IN_S = 60L;
+    public static final double MEAN_LIFETIME_IN_S = HALF_TIME_IN_S / Math.log(2.0);
+    public static final long LANDMARK_RESET_INTERVAL_IN_MS = 30L * 60L * 1000L;
+
+    private final AtomicBoolean rescaling = new AtomicBoolean(false);
+    private volatile long decayLandmark;
+
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    // Wrapper around System.nanoTime() to simplify unit testing.
+    private final Clock clock;
+
+
+    /**
+     * Construct a decaying histogram with default number of buckets and without considering zeroes.
+     */
+    public DecayingEstimatedHistogramReservoir()
+    {
+        this(DEFAULT_ZERO_CONSIDERATION, DEFAULT_BUCKET_COUNT, Clock.defaultClock());
+    }
+
+    /**
+     * Construct a decaying histogram with default number of buckets.
+     *
+     * @param considerZeroes when true, 0-value measurements in a separate bucket, otherwise they will be collected in
+     *                       same bucket as 1-value measurements
+     */
+    public DecayingEstimatedHistogramReservoir(boolean considerZeroes)
+    {
+        this(considerZeroes, DEFAULT_BUCKET_COUNT, Clock.defaultClock());
+    }
+
+    /**
+     * Construct a decaying histogram.
+     *
+     * @param considerZeroes when true, 0-value measurements in a separate bucket, otherwise they will be collected in
+     *                       same bucket as 1-value measurements
+     * @param bucketCount number of buckets used to collect measured values
+     */
+    public DecayingEstimatedHistogramReservoir(boolean considerZeroes, int bucketCount)
+    {
+        this(considerZeroes, bucketCount, Clock.defaultClock());
+    }
+
+    @VisibleForTesting
+    DecayingEstimatedHistogramReservoir(boolean considerZeroes, int bucketCount, Clock clock)
+    {
+        if (bucketCount == DEFAULT_BUCKET_COUNT)
+        {
+            if (considerZeroes == true)
+            {
+                bucketOffsets = DEFAULT_WITH_ZERO_BUCKET_OFFSETS;
+            }
+            else
+            {
+                bucketOffsets = DEFAULT_WITHOUT_ZERO_BUCKET_OFFSETS;
+            }
+        }
+        else
+        {
+            bucketOffsets = EstimatedHistogram.newOffsets(bucketCount, considerZeroes);
+        }
+        decayingBuckets = new AtomicLongArray(bucketOffsets.length + 1);
+        buckets = new AtomicLongArray(bucketOffsets.length + 1);
+        this.clock = clock;
+        decayLandmark = clock.getTime();
+    }
+
+    /**
+     * Increments the count of the bucket closest to n, rounding UP.
+     *
+     * @param value the data point to add to the histogram
+     */
+    public void update(long value)
+    {
+        long now = clock.getTime();
+        rescaleIfNeeded(now);
+
+        int index = Arrays.binarySearch(bucketOffsets, value);
+        if (index < 0)
+        {
+            // inexact match, take the first bucket higher than n
+            index = -index - 1;
+        }
+        // else exact match; we're good
+
+        lockForRegularUsage();
+
+        try
+        {
+            decayingBuckets.getAndAdd(index, Math.round(forwardDecayWeight(now)));
+        }
+        finally
+        {
+            unlockForRegularUsage();
+        }
+
+        buckets.getAndIncrement(index);
+    }
+
+    private double forwardDecayWeight(long now)
+    {
+        return Math.exp(((now - decayLandmark) / 1000L) / MEAN_LIFETIME_IN_S);
+    }
+
+    /**
+     * Return the number of buckets where recorded values are stored.
+     *
+     * This method does not return the number of recorded values as suggested by the {@link Reservoir} interface.
+     *
+     * @return the number of buckets
+     */
+    public int size()
+    {
+        return decayingBuckets.length();
+    }
+
+    /**
+     * Returns a snapshot of the decaying values in this reservoir.
+     *
+     * Non-decaying reservoir will not be included in the snapshot.
+     *
+     * @return the snapshot
+     */
+    public Snapshot getSnapshot()
+    {
+        rescaleIfNeeded();
+
+        lockForRegularUsage();
+
+        try
+        {
+            return new EstimatedHistogramReservoirSnapshot(this);
+        }
+        finally
+        {
+            unlockForRegularUsage();
+        }
+    }
+
+    /**
+     * @return true if this histogram has overflowed -- that is, a value larger than our largest bucket could bound was added
+     */
+    @VisibleForTesting
+    boolean isOverflowed()
+    {
+        return decayingBuckets.get(decayingBuckets.length() - 1) > 0;
+    }
+
+    private void rescaleIfNeeded()
+    {
+        rescaleIfNeeded(clock.getTime());
+    }
+
+    private void rescaleIfNeeded(long now)
+    {
+        if (needRescale(now))
+        {
+            if (rescaling.compareAndSet(false, true))
+            {
+                try
+                {
+                    rescale(now);
+                }
+                finally
+                {
+                    rescaling.set(false);
+                }
+            }
+        }
+    }
+
+    private void rescale(long now)
+    {
+        // Check again to make sure that another thread didn't complete rescale already
+        if (needRescale(now))
+        {
+            lockForRescale();
+
+            try
+            {
+                final double rescaleFactor = forwardDecayWeight(now);
+                decayLandmark = now;
+
+                final int bucketCount = decayingBuckets.length();
+                for (int i = 0; i < bucketCount; i++)
+                {
+                    long newValue = Math.round((decayingBuckets.get(i) / rescaleFactor));
+                    decayingBuckets.set(i, newValue);
+                }
+            }
+            finally
+            {
+                unlockForRescale();
+            }
+        }
+    }
+
+    private boolean needRescale(long now)
+    {
+        return (now - decayLandmark) > LANDMARK_RESET_INTERVAL_IN_MS;
+    }
+
+    @VisibleForTesting
+    public void clear()
+    {
+        lockForRescale();
+
+        try
+        {
+            final int bucketCount = decayingBuckets.length();
+            for (int i = 0; i < bucketCount; i++)
+            {
+                decayingBuckets.set(i, 0L);
+                buckets.set(i, 0L);
+            }
+        }
+        finally
+        {
+            unlockForRescale();
+        }
+    }
+
+    private void lockForRegularUsage()
+    {
+        this.lock.readLock().lock();
+    }
+
+    private void unlockForRegularUsage()
+    {
+        this.lock.readLock().unlock();
+    }
+
+    private void lockForRescale()
+    {
+        this.lock.writeLock().lock();
+    }
+
+    private void unlockForRescale()
+    {
+        this.lock.writeLock().unlock();
+    }
+
+
+    private static final Charset UTF_8 = Charset.forName("UTF-8");
+
+    /**
+     * Represents a snapshot of the decaying histogram.
+     *
+     * The decaying buckets are copied into a snapshot array to give a consistent view for all getters. However, the
+     * copy is made without a write-lock and so other threads may change the buckets while the array is copied,
+     * probably causign a slight skew up in the quantiles and mean values.
+     *
+     * The decaying buckets will be used for quantile calculations and mean values, but the non decaying buckets will be
+     * exposed for calls to {@link Snapshot#getValues()}.
+     */
+    private class EstimatedHistogramReservoirSnapshot extends Snapshot
+    {
+        private final long[] decayingBuckets;
+
+        public EstimatedHistogramReservoirSnapshot(DecayingEstimatedHistogramReservoir reservoir)
+        {
+            final int length = reservoir.decayingBuckets.length();
+            final double rescaleFactor = forwardDecayWeight(clock.getTime());
+
+            this.decayingBuckets = new long[length];
+
+            for (int i = 0; i < length; i++)
+                this.decayingBuckets[i] = Math.round(reservoir.decayingBuckets.get(i) / rescaleFactor);
+        }
+
+        /**
+         * Get the estimated value at the specified quantile in the distribution.
+         *
+         * @param quantile the quantile specified as a value between 0.0 (zero) and 1.0 (one)
+         * @return estimated value at given quantile
+         * @throws IllegalStateException in case the histogram overflowed
+         */
+        public double getValue(double quantile)
+        {
+            assert quantile >= 0 && quantile <= 1.0;
+
+            final int lastBucket = decayingBuckets.length - 1;
+
+            if (decayingBuckets[lastBucket] > 0)
+                throw new IllegalStateException("Unable to compute when histogram overflowed");
+
+            final long qcount = (long) Math.ceil(count() * quantile);
+            if (qcount == 0)
+                return 0;
+
+            long elements = 0;
+            for (int i = 0; i < lastBucket; i++)
+            {
+                elements += decayingBuckets[i];
+                if (elements >= qcount)
+                    return bucketOffsets[i];
+            }
+            return 0;
+        }
+
+        /**
+         * Will return a snapshot of the non-decaying buckets.
+         *
+         * The values returned will not be consistent with the quantile and mean values. The caller must be aware of the
+         * offsets created by {@link EstimatedHistogram#getBucketOffsets()} to make use of the values returned.
+         *
+         * @return a snapshot of the non-decaying buckets.
+         */
+        public long[] getValues()
+        {
+            final int length = buckets.length();
+
+            long[] values = new long[length];
+
+            for (int i = 0; i < length; i++)
+                values[i] = buckets.get(i);
+
+            return values;
+        }
+
+        /**
+         * Return the number of buckets where recorded values are stored.
+         *
+         * This method does not return the number of recorded values as suggested by the {@link Snapshot} interface.
+         *
+         * @return the number of buckets
+         */
+        public int size()
+        {
+            return decayingBuckets.length;
+        }
+
+        /**
+         * Return the number of registered values taking forward decay into account.
+         *
+         * @return the sum of all bucket values
+         */
+        private long count()
+        {
+            long sum = 0L;
+            for (int i = 0; i < decayingBuckets.length; i++)
+                sum += decayingBuckets[i];
+            return sum;
+        }
+
+        /**
+         * Get the estimated max-value that could have been added to this reservoir.
+         *
+         * As values are collected in variable sized buckets, the actual max value recored in the reservoir may be less
+         * than the value returned.
+         *
+         * @return the largest value that could have been added to this reservoir, or Long.MAX_VALUE if the reservoir
+         * overflowed
+         */
+        public long getMax()
+        {
+            final int lastBucket = decayingBuckets.length - 1;
+
+            if (decayingBuckets[lastBucket] > 0)
+                return Long.MAX_VALUE;
+
+            for (int i = lastBucket - 1; i >= 0; i--)
+            {
+                if (decayingBuckets[i] > 0)
+                    return bucketOffsets[i];
+            }
+            return 0;
+        }
+
+        /**
+         * Get the estimated mean value in the distribution.
+         *
+         * @return the mean histogram value (average of bucket offsets, weighted by count)
+         * @throws IllegalStateException if any values were greater than the largest bucket threshold
+         */
+        public double getMean()
+        {
+            final int lastBucket = decayingBuckets.length - 1;
+
+            if (decayingBuckets[lastBucket] > 0)
+                throw new IllegalStateException("Unable to compute when histogram overflowed");
+
+            long elements = 0;
+            long sum = 0;
+            for (int i = 0; i < lastBucket; i++)
+            {
+                long bCount = decayingBuckets[i];
+                elements += bCount;
+                sum += bCount * bucketOffsets[i];
+            }
+
+            return (double) sum / elements;
+        }
+
+        /**
+         * Get the estimated min-value that could have been added to this reservoir.
+         *
+         * As values are collected in variable sized buckets, the actual min value recored in the reservoir may be
+         * higher than the value returned.
+         *
+         * @return the smallest value that could have been added to this reservoir
+         */
+        public long getMin()
+        {
+            for (int i = 0; i < decayingBuckets.length; i++)
+            {
+                if (decayingBuckets[i] > 0)
+                    return i == 0 ? 0 : 1 + bucketOffsets[i - 1];
+            }
+            return 0;
+        }
+
+        /**
+         * Get the estimated standard deviation of the values added to this reservoir.
+         *
+         * As values are collected in variable sized buckets, the actual deviation may be more or less than the value
+         * returned.
+         *
+         * @return an estimate of the standard deviation
+         */
+        public double getStdDev()
+        {
+            final int lastBucket = decayingBuckets.length - 1;
+
+            if (decayingBuckets[lastBucket] > 0)
+                throw new IllegalStateException("Unable to compute when histogram overflowed");
+
+            final long count = count();
+
+            if(count <= 1) {
+                return 0.0D;
+            } else {
+                double mean = this.getMean();
+                double sum = 0.0D;
+
+                for(int i = 0; i < lastBucket; ++i) {
+                    long value = bucketOffsets[i];
+                    double diff = (double)value - mean;
+                    sum += diff * diff * decayingBuckets[i];
+                }
+
+                return Math.sqrt(sum / (double)(count - 1));
+            }
+        }
+
+        public void dump(OutputStream output)
+        {
+            try (PrintWriter out = new PrintWriter(new OutputStreamWriter(output, UTF_8)))
+            {
+                int length = decayingBuckets.length;
+
+                for(int i = 0; i < length; ++i) {
+                    out.printf("%d%n", decayingBuckets[i]);
+                }
+            }
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/metrics/DefaultNameFactory.java b/src/java/org/apache/cassandra/metrics/DefaultNameFactory.java
index f7f23ee..db51906 100644
--- a/src/java/org/apache/cassandra/metrics/DefaultNameFactory.java
+++ b/src/java/org/apache/cassandra/metrics/DefaultNameFactory.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.metrics;
 
-import com.yammer.metrics.core.MetricName;
 
 /**
  * MetricNameFactory that generates default MetricName of metrics.
@@ -40,14 +39,14 @@
         this.scope = scope;
     }
 
-    public MetricName createMetricName(String metricName)
+    public CassandraMetricsRegistry.MetricName createMetricName(String metricName)
     {
         return createMetricName(type, metricName, scope);
     }
 
-    public static MetricName createMetricName(String type, String metricName, String scope)
+    public static CassandraMetricsRegistry.MetricName createMetricName(String type, String metricName, String scope)
     {
-        return new MetricName(GROUP_NAME, type, metricName, scope, createDefaultMBeanName(type, metricName, scope));
+        return new CassandraMetricsRegistry.MetricName(GROUP_NAME, type, metricName, scope, createDefaultMBeanName(type, metricName, scope));
     }
 
     protected static String createDefaultMBeanName(String type, String name, String scope)
diff --git a/src/java/org/apache/cassandra/metrics/DroppedMessageMetrics.java b/src/java/org/apache/cassandra/metrics/DroppedMessageMetrics.java
index f94ea04..58c80fb 100644
--- a/src/java/org/apache/cassandra/metrics/DroppedMessageMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/DroppedMessageMetrics.java
@@ -17,13 +17,11 @@
  */
 package org.apache.cassandra.metrics;
 
-import java.util.concurrent.TimeUnit;
-
-import com.yammer.metrics.Metrics;
-import com.yammer.metrics.core.Meter;
-
+import com.codahale.metrics.Meter;
 import org.apache.cassandra.net.MessagingService;
 
+import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
+
 /**
  * Metrics for dropped messages by verb.
  */
@@ -32,20 +30,9 @@
     /** Number of dropped messages */
     public final Meter dropped;
 
-    private long lastDropped = 0;
-
     public DroppedMessageMetrics(MessagingService.Verb verb)
     {
         MetricNameFactory factory = new DefaultNameFactory("DroppedMessage", verb.toString());
-        dropped = Metrics.newMeter(factory.createMetricName("Dropped"), "dropped", TimeUnit.SECONDS);
-    }
-
-    @Deprecated
-    public int getRecentlyDropped()
-    {
-        long currentDropped = dropped.count();
-        long recentlyDropped = currentDropped - lastDropped;
-        lastDropped = currentDropped;
-        return (int)recentlyDropped;
+        dropped = Metrics.meter(factory.createMetricName("Dropped"));
     }
 }
diff --git a/src/java/org/apache/cassandra/metrics/FileCacheMetrics.java b/src/java/org/apache/cassandra/metrics/FileCacheMetrics.java
index 42444f3..c240c03 100644
--- a/src/java/org/apache/cassandra/metrics/FileCacheMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/FileCacheMetrics.java
@@ -17,15 +17,14 @@
  */
 package org.apache.cassandra.metrics;
 
-import java.util.concurrent.TimeUnit;
-
-import com.yammer.metrics.Metrics;
-import com.yammer.metrics.core.Gauge;
-import com.yammer.metrics.core.Meter;
-import com.yammer.metrics.core.MetricName;
-import com.yammer.metrics.util.RatioGauge;
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Meter;
+import com.codahale.metrics.RatioGauge;
 import org.apache.cassandra.service.FileCacheService;
 
+import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
+
+
 public class FileCacheMetrics
 {
     private static final MetricNameFactory factory = new DefaultNameFactory("FileCache");
@@ -41,23 +40,19 @@
 
     public FileCacheMetrics()
     {
-        hits = Metrics.newMeter(factory.createMetricName("Hits"), "hits", TimeUnit.SECONDS);
-        requests = Metrics.newMeter(factory.createMetricName("Requests"), "requests", TimeUnit.SECONDS);
-        hitRate = Metrics.newGauge(factory.createMetricName("HitRate"), new RatioGauge()
+        hits = Metrics.meter(factory.createMetricName("Hits"));
+        requests = Metrics.meter(factory.createMetricName("Requests"));
+        hitRate = Metrics.register(factory.createMetricName("HitRate"), new RatioGauge()
         {
-            protected double getNumerator()
+            @Override
+            public Ratio getRatio()
             {
-                return hits.count();
-            }
-
-            protected double getDenominator()
-            {
-                return requests.count();
+                return Ratio.of(hits.getCount(), requests.getCount());
             }
         });
-        size = Metrics.newGauge(factory.createMetricName("Size"), new Gauge<Long>()
+        size = Metrics.register(factory.createMetricName("Size"), new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 return FileCacheService.instance.sizeInBytes();
             }
diff --git a/src/java/org/apache/cassandra/metrics/HintedHandoffMetrics.java b/src/java/org/apache/cassandra/metrics/HintedHandoffMetrics.java
index ea81a9a..e44279a 100644
--- a/src/java/org/apache/cassandra/metrics/HintedHandoffMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/HintedHandoffMetrics.java
@@ -20,6 +20,7 @@
 import java.net.InetAddress;
 import java.util.Map.Entry;
 
+import com.codahale.metrics.Counter;
 import org.apache.cassandra.db.HintedHandOffManager;
 import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.utils.UUIDGen;
@@ -29,8 +30,8 @@
 import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
-import com.yammer.metrics.Metrics;
-import com.yammer.metrics.core.Counter;
+
+import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
 
 /**
  * Metrics for {@link HintedHandOffManager}.
@@ -39,7 +40,7 @@
 {
     private static final Logger logger = LoggerFactory.getLogger(HintedHandoffMetrics.class);
 
-    private final MetricNameFactory factory = new DefaultNameFactory("HintedHandOffManager");
+    private static final MetricNameFactory factory = new DefaultNameFactory("HintedHandOffManager");
 
     /** Total number of hints which are not stored, This is not a cache. */
     private final LoadingCache<InetAddress, DifferencingCounter> notStored = CacheBuilder.newBuilder().build(new CacheLoader<InetAddress, DifferencingCounter>()
@@ -55,7 +56,7 @@
     {
         public Counter load(InetAddress address)
         {
-            return Metrics.newCounter(factory.createMetricName("Hints_created-" + address.getHostAddress().replace(':', '.')));
+            return Metrics.counter(factory.createMetricName("Hints_created-" + address.getHostAddress().replace(':', '.')));
         }
     });
 
@@ -81,19 +82,19 @@
         }
     }
 
-    public class DifferencingCounter
+    public static class DifferencingCounter
     {
         private final Counter meter;
         private long reported = 0;
 
         public DifferencingCounter(InetAddress address)
         {
-            this.meter = Metrics.newCounter(factory.createMetricName("Hints_not_stored-" + address.getHostAddress().replace(':', '.')));
+            this.meter = Metrics.counter(factory.createMetricName("Hints_not_stored-" + address.getHostAddress().replace(':', '.')));
         }
 
         public long difference()
         {
-            long current = meter.count();
+            long current = meter.getCount();
             long difference = current - reported;
             this.reported = current;
             return difference;
@@ -101,7 +102,7 @@
 
         public long count()
         {
-            return meter.count();
+            return meter.getCount();
         }
 
         public void mark()
diff --git a/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java b/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java
index 8a55c3b..369f323 100644
--- a/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/KeyspaceMetrics.java
@@ -19,13 +19,17 @@
 
 import java.util.Set;
 
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Histogram;
+import com.codahale.metrics.MetricRegistry;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
-import com.yammer.metrics.Metrics;
-import com.yammer.metrics.core.*;
+
+import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
+
 
 /**
  * Metrics for {@link ColumnFamilyStore}.
@@ -104,112 +108,112 @@
         {
             public Long getValue(ColumnFamilyMetrics metric)
             {
-                return metric.memtableColumnsCount.value();
+                return metric.memtableColumnsCount.getValue();
             }
         });
         memtableLiveDataSize = createKeyspaceGauge("MemtableLiveDataSize", new MetricValue()
         {
             public Long getValue(ColumnFamilyMetrics metric)
             {
-                return metric.memtableLiveDataSize.value();
+                return metric.memtableLiveDataSize.getValue();
             }
         }); 
         memtableOnHeapDataSize = createKeyspaceGauge("MemtableOnHeapDataSize", new MetricValue()
         {
             public Long getValue(ColumnFamilyMetrics metric)
             {
-                return metric.memtableOnHeapSize.value();
+                return metric.memtableOnHeapSize.getValue();
             }
         });
         memtableOffHeapDataSize = createKeyspaceGauge("MemtableOffHeapDataSize", new MetricValue()
         {
             public Long getValue(ColumnFamilyMetrics metric)
             {
-                return metric.memtableOffHeapSize.value();
+                return metric.memtableOffHeapSize.getValue();
             }
         });
         allMemtablesLiveDataSize = createKeyspaceGauge("AllMemtablesLiveDataSize", new MetricValue()
         {
             public Long getValue(ColumnFamilyMetrics metric)
             {
-                return metric.allMemtablesLiveDataSize.value();
+                return metric.allMemtablesLiveDataSize.getValue();
             }
         });
         allMemtablesOnHeapDataSize = createKeyspaceGauge("AllMemtablesOnHeapDataSize", new MetricValue()
         {
             public Long getValue(ColumnFamilyMetrics metric)
             {
-                return metric.allMemtablesOnHeapSize.value();
+                return metric.allMemtablesOnHeapSize.getValue();
             }
         });
         allMemtablesOffHeapDataSize = createKeyspaceGauge("AllMemtablesOffHeapDataSize", new MetricValue()
         {
             public Long getValue(ColumnFamilyMetrics metric)
             {
-                return metric.allMemtablesOffHeapSize.value();
+                return metric.allMemtablesOffHeapSize.getValue();
             }
         });
         memtableSwitchCount = createKeyspaceGauge("MemtableSwitchCount", new MetricValue()
         {
             public Long getValue(ColumnFamilyMetrics metric)
             {
-                return metric.memtableSwitchCount.count();
+                return metric.memtableSwitchCount.getCount();
             }
         });
         pendingCompactions = createKeyspaceGauge("PendingCompactions", new MetricValue()
         {
             public Long getValue(ColumnFamilyMetrics metric)
             {
-                return (long) metric.pendingCompactions.value();
+                return (long) metric.pendingCompactions.getValue();
             }
         });
         pendingFlushes = createKeyspaceGauge("PendingFlushes", new MetricValue()
         {
             public Long getValue(ColumnFamilyMetrics metric)
             {
-                return (long) metric.pendingFlushes.count();
+                return (long) metric.pendingFlushes.getCount();
             }
         });
         liveDiskSpaceUsed = createKeyspaceGauge("LiveDiskSpaceUsed", new MetricValue()
         {
             public Long getValue(ColumnFamilyMetrics metric)
             {
-                return metric.liveDiskSpaceUsed.count();
+                return metric.liveDiskSpaceUsed.getCount();
             }
         });
         totalDiskSpaceUsed = createKeyspaceGauge("TotalDiskSpaceUsed", new MetricValue()
         {
             public Long getValue(ColumnFamilyMetrics metric)
             {
-                return metric.totalDiskSpaceUsed.count();
+                return metric.totalDiskSpaceUsed.getCount();
             }
         });
         bloomFilterDiskSpaceUsed = createKeyspaceGauge("BloomFilterDiskSpaceUsed", new MetricValue()
         {
             public Long getValue(ColumnFamilyMetrics metric)
             {
-                return metric.bloomFilterDiskSpaceUsed.value();
+                return metric.bloomFilterDiskSpaceUsed.getValue();
             }
         });
         bloomFilterOffHeapMemoryUsed = createKeyspaceGauge("BloomFilterOffHeapMemoryUsed", new MetricValue()
         {
             public Long getValue(ColumnFamilyMetrics metric)
             {
-                return metric.bloomFilterOffHeapMemoryUsed.value();
+                return metric.bloomFilterOffHeapMemoryUsed.getValue();
             }
         });
         indexSummaryOffHeapMemoryUsed = createKeyspaceGauge("IndexSummaryOffHeapMemoryUsed", new MetricValue()
         {
             public Long getValue(ColumnFamilyMetrics metric)
             {
-                return metric.indexSummaryOffHeapMemoryUsed.value();
+                return metric.indexSummaryOffHeapMemoryUsed.getValue();
             }
         });
         compressionMetadataOffHeapMemoryUsed = createKeyspaceGauge("CompressionMetadataOffHeapMemoryUsed", new MetricValue()
         {
             public Long getValue(ColumnFamilyMetrics metric)
             {
-                return metric.compressionMetadataOffHeapMemoryUsed.value();
+                return metric.compressionMetadataOffHeapMemoryUsed.getValue();
             }
         });
         // latency metrics for ColumnFamilyMetrics to update
@@ -217,10 +221,10 @@
         writeLatency = new LatencyMetrics(factory, "Write");
         rangeLatency = new LatencyMetrics(factory, "Range");
         // create histograms for ColumnFamilyMetrics to replicate updates to
-        sstablesPerReadHistogram = Metrics.newHistogram(factory.createMetricName("SSTablesPerReadHistogram"), true);
-        tombstoneScannedHistogram = Metrics.newHistogram(factory.createMetricName("TombstoneScannedHistogram"), true);
-        liveScannedHistogram = Metrics.newHistogram(factory.createMetricName("LiveScannedHistogram"), true);
-        colUpdateTimeDeltaHistogram = Metrics.newHistogram(factory.createMetricName("ColUpdateTimeDeltaHistogram"), true);
+        sstablesPerReadHistogram = Metrics.histogram(factory.createMetricName("SSTablesPerReadHistogram"), true);
+        tombstoneScannedHistogram = Metrics.histogram(factory.createMetricName("TombstoneScannedHistogram"), false);
+        liveScannedHistogram = Metrics.histogram(factory.createMetricName("LiveScannedHistogram"), false);
+        colUpdateTimeDeltaHistogram = Metrics.histogram(factory.createMetricName("ColUpdateTimeDeltaHistogram"), false);
         // add manually since histograms do not use createKeyspaceGauge method
         allMetrics.addAll(Lists.newArrayList("SSTablesPerReadHistogram", "TombstoneScannedHistogram", "LiveScannedHistogram"));
 
@@ -236,7 +240,7 @@
     {
         for(String name : allMetrics) 
         {
-            Metrics.defaultRegistry().removeMetric(factory.createMetricName(name));
+            Metrics.remove(factory.createMetricName(name));
         }
         // latency metrics contain multiple metrics internally and need to be released manually
         readLatency.release();
@@ -251,7 +255,7 @@
     {
         /**
          * get value of a metric
-         * @param columnfamilymetrics of a column family in this keyspace
+         * @param metric of a column family in this keyspace
          * @return current value of a metric
          */
         public Long getValue(ColumnFamilyMetrics metric);
@@ -260,15 +264,15 @@
     /**
      * Creates a gauge that will sum the current value of a metric for all column families in this keyspace
      * @param name
-     * @param MetricValue 
+     * @param extractor
      * @return Gauge&gt;Long> that computes sum of MetricValue.getValue()
      */
-    private <T extends Number> Gauge<Long> createKeyspaceGauge(String name, final MetricValue extractor)
+    private Gauge<Long> createKeyspaceGauge(String name, final MetricValue extractor)
     {
         allMetrics.add(name);
-        return Metrics.newGauge(factory.createMetricName(name), new Gauge<Long>()
+        return Metrics.register(factory.createMetricName(name), new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 long sum = 0;
                 for (ColumnFamilyStore cf : keyspace.getColumnFamilyStores())
@@ -280,7 +284,7 @@
         });
     }
 
-    class KeyspaceMetricNameFactory implements MetricNameFactory
+    static class KeyspaceMetricNameFactory implements MetricNameFactory
     {
         private final String keyspaceName;
 
@@ -289,7 +293,7 @@
             this.keyspaceName = ks.getName();
         }
 
-        public MetricName createMetricName(String metricName)
+        public CassandraMetricsRegistry.MetricName createMetricName(String metricName)
         {
             String groupName = ColumnFamilyMetrics.class.getPackage().getName();
 
@@ -299,7 +303,7 @@
             mbeanName.append(",keyspace=").append(keyspaceName);
             mbeanName.append(",name=").append(metricName);
 
-            return new MetricName(groupName, "keyspace", metricName, keyspaceName, mbeanName.toString());
+            return new CassandraMetricsRegistry.MetricName(groupName, "keyspace", metricName, keyspaceName, mbeanName.toString());
         }
     }
 }
diff --git a/src/java/org/apache/cassandra/metrics/LatencyMetrics.java b/src/java/org/apache/cassandra/metrics/LatencyMetrics.java
index 28256fe..a2eef68 100644
--- a/src/java/org/apache/cassandra/metrics/LatencyMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/LatencyMetrics.java
@@ -20,13 +20,14 @@
 import java.util.List;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.cassandra.utils.EstimatedHistogram;
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.Timer;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
-import com.yammer.metrics.Metrics;
-import com.yammer.metrics.core.Counter;
-import com.yammer.metrics.core.Timer;
+
+import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
+
 
 /**
  * Metrics about latencies
@@ -44,11 +45,6 @@
     protected final MetricNameFactory factory;
     protected final String namePrefix;
 
-    @Deprecated public final EstimatedHistogram totalLatencyHistogram = new EstimatedHistogram();
-    @Deprecated public final EstimatedHistogram recentLatencyHistogram = new EstimatedHistogram();
-    protected long lastLatency;
-    protected long lastOpCount;
-
     /**
      * Create LatencyMetrics with given group, type, and scope. Name prefix for each metric will be empty.
      *
@@ -83,8 +79,8 @@
         this.factory = factory;
         this.namePrefix = namePrefix;
 
-        latency = Metrics.newTimer(factory.createMetricName(namePrefix + "Latency"), TimeUnit.MICROSECONDS, TimeUnit.SECONDS);
-        totalLatency = Metrics.newCounter(factory.createMetricName(namePrefix + "TotalLatency"));
+        latency = Metrics.timer(factory.createMetricName(namePrefix + "Latency"));
+        totalLatency = Metrics.counter(factory.createMetricName(namePrefix + "TotalLatency"));
     }
     
     /**
@@ -93,7 +89,7 @@
      *
      * @param factory MetricName factory to use
      * @param namePrefix Prefix to append to each metric name
-     * @param parents... any amount of parents to replicate updates to
+     * @param parents any amount of parents to replicate updates to
      */
     public LatencyMetrics(MetricNameFactory factory, String namePrefix, LatencyMetrics ... parents)
     {
@@ -107,8 +103,6 @@
         // convert to microseconds. 1 millionth
         latency.update(nanos, TimeUnit.NANOSECONDS);
         totalLatency.inc(nanos / 1000);
-        totalLatencyHistogram.add(nanos / 1000);
-        recentLatencyHistogram.add(nanos / 1000);
         for(LatencyMetrics parent : parents)
         {
             parent.addNano(nanos);
@@ -117,25 +111,7 @@
 
     public void release()
     {
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName(namePrefix + "Latency"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName(namePrefix + "TotalLatency"));
-    }
-
-    @Deprecated
-    public synchronized double getRecentLatency()
-    {
-        long ops = latency.count();
-        long n = totalLatency.count();
-        if (ops == lastOpCount)
-            return 0;
-        try
-        {
-            return ((double) n - lastLatency) / (ops - lastOpCount);
-        }
-        finally
-        {
-            lastLatency = n;
-            lastOpCount = ops;
-        }
+        Metrics.remove(factory.createMetricName(namePrefix + "Latency"));
+        Metrics.remove(factory.createMetricName(namePrefix + "TotalLatency"));
     }
 }
diff --git a/src/java/org/apache/cassandra/metrics/MetricNameFactory.java b/src/java/org/apache/cassandra/metrics/MetricNameFactory.java
index 78f8b14..4e8f2b4 100644
--- a/src/java/org/apache/cassandra/metrics/MetricNameFactory.java
+++ b/src/java/org/apache/cassandra/metrics/MetricNameFactory.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.metrics;
 
-import com.yammer.metrics.core.MetricName;
 
 public interface MetricNameFactory
 {
@@ -27,5 +26,5 @@
      * @param metricName part of qualified name.
      * @return new String with given metric name.
      */
-    MetricName createMetricName(String metricName);
+    CassandraMetricsRegistry.MetricName createMetricName(String metricName);
 }
diff --git a/src/java/org/apache/cassandra/metrics/ReadRepairMetrics.java b/src/java/org/apache/cassandra/metrics/ReadRepairMetrics.java
index f9b1a40..9ee1c60 100644
--- a/src/java/org/apache/cassandra/metrics/ReadRepairMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/ReadRepairMetrics.java
@@ -17,10 +17,9 @@
  */
 package org.apache.cassandra.metrics;
 
-import java.util.concurrent.TimeUnit;
+import com.codahale.metrics.Meter;
 
-import com.yammer.metrics.Metrics;
-import com.yammer.metrics.core.Meter;
+import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
 
 /**
  * Metrics related to Read Repair.
@@ -29,10 +28,7 @@
 {
     private static final MetricNameFactory factory = new DefaultNameFactory("ReadRepair");
 
-    public static final Meter repairedBlocking =
-            Metrics.newMeter(factory.createMetricName("RepairedBlocking"), "RepairedBlocking", TimeUnit.SECONDS);
-    public static final Meter repairedBackground =
-            Metrics.newMeter(factory.createMetricName("RepairedBackground"), "RepairedBackground", TimeUnit.SECONDS);
-    public static final Meter attempted = 
-            Metrics.newMeter(factory.createMetricName("Attempted"), "Attempted", TimeUnit.SECONDS);
+    public static final Meter repairedBlocking = Metrics.meter(factory.createMetricName("RepairedBlocking"));
+    public static final Meter repairedBackground = Metrics.meter(factory.createMetricName("RepairedBackground"));
+    public static final Meter attempted = Metrics.meter(factory.createMetricName("Attempted"));
 }
diff --git a/src/java/org/apache/cassandra/metrics/RestorableMeter.java b/src/java/org/apache/cassandra/metrics/RestorableMeter.java
index 4df4665..9e2ed92 100644
--- a/src/java/org/apache/cassandra/metrics/RestorableMeter.java
+++ b/src/java/org/apache/cassandra/metrics/RestorableMeter.java
@@ -18,12 +18,12 @@
  */
 package org.apache.cassandra.metrics;
 
-import com.yammer.metrics.core.Clock;
 
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 
 import static java.lang.Math.exp;
+import com.codahale.metrics.Clock;
 
 /**
  * A meter metric which measures mean throughput as well as fifteen-minute and two-hour
@@ -52,7 +52,7 @@
     public RestorableMeter() {
         this.m15Rate = new RestorableEWMA(TimeUnit.MINUTES.toSeconds(15));
         this.m120Rate = new RestorableEWMA(TimeUnit.MINUTES.toSeconds(120));
-        this.startTime = this.clock.tick();
+        this.startTime = this.clock.getTick();
         this.lastTick = new AtomicLong(startTime);
     }
 
@@ -64,7 +64,7 @@
     public RestorableMeter(double lastM15Rate, double lastM120Rate) {
         this.m15Rate = new RestorableEWMA(lastM15Rate, TimeUnit.MINUTES.toSeconds(15));
         this.m120Rate = new RestorableEWMA(lastM120Rate, TimeUnit.MINUTES.toSeconds(120));
-        this.startTime = this.clock.tick();
+        this.startTime = this.clock.getTick();
         this.lastTick = new AtomicLong(startTime);
     }
 
@@ -73,7 +73,7 @@
      */
     private void tickIfNecessary() {
         final long oldTick = lastTick.get();
-        final long newTick = clock.tick();
+        final long newTick = clock.getTick();
         final long age = newTick - oldTick;
         if (age > TICK_INTERVAL) {
             final long newIntervalStartTick = newTick - age % TICK_INTERVAL;
@@ -139,12 +139,12 @@
         if (count() == 0) {
             return 0.0;
         } else {
-            final long elapsed = (clock.tick() - startTime);
+            final long elapsed = (clock.getTick() - startTime);
             return (count() / (double) elapsed) * NANOS_PER_SECOND;
         }
     }
 
-    class RestorableEWMA {
+    static class RestorableEWMA {
         private volatile boolean initialized = false;
         private volatile double rate = 0.0; // average rate in terms of events per nanosecond
 
@@ -158,7 +158,7 @@
          */
         public RestorableEWMA(long windowInSeconds) {
             this.alpha = 1 - exp((-TICK_INTERVAL / NANOS_PER_SECOND) / windowInSeconds);
-            this.interval = (double) TICK_INTERVAL;
+            this.interval = TICK_INTERVAL;
         }
 
         /**
diff --git a/src/java/org/apache/cassandra/metrics/SEPMetrics.java b/src/java/org/apache/cassandra/metrics/SEPMetrics.java
index 58fe9c5..35f02b4 100644
--- a/src/java/org/apache/cassandra/metrics/SEPMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/SEPMetrics.java
@@ -17,22 +17,24 @@
  */
 package org.apache.cassandra.metrics;
 
-import com.yammer.metrics.Metrics;
-import com.yammer.metrics.core.Gauge;
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.Gauge;
 
 import org.apache.cassandra.concurrent.SEPExecutor;
 
+import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
+
 public class SEPMetrics
 {
     /** Number of active tasks. */
     public final Gauge<Integer> activeTasks;
     /** Number of tasks that had blocked before being accepted (or rejected). */
-    public final Gauge<Integer> totalBlocked;
+    public final Counter totalBlocked;
     /**
      * Number of tasks currently blocked, waiting to be accepted by
      * the executor (because all threads are busy and the backing queue is full).
      */
-    public final Gauge<Long> currentBlocked;
+    public final Counter currentBlocked;
     /** Number of completed tasks. */
     public final Gauge<Long> completedTasks;
     /** Number of tasks waiting to be executed. */
@@ -52,44 +54,33 @@
     public SEPMetrics(final SEPExecutor executor, String path, String poolName)
     {
         this.factory = new ThreadPoolMetricNameFactory("ThreadPools", path, poolName);
-        activeTasks = Metrics.newGauge(factory.createMetricName("ActiveTasks"), new Gauge<Integer>()
+        activeTasks = Metrics.register(factory.createMetricName("ActiveTasks"), new Gauge<Integer>()
         {
-            public Integer value()
+            public Integer getValue()
             {
                 return executor.getActiveCount();
             }
         });
-        pendingTasks = Metrics.newGauge(factory.createMetricName("PendingTasks"), new Gauge<Long>()
+        pendingTasks = Metrics.register(factory.createMetricName("PendingTasks"), new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 return executor.getPendingTasks();
             }
         });
-        totalBlocked = Metrics.newGauge(factory.createMetricName("TotalBlockedTasks"), new Gauge<Integer>()
+        totalBlocked = Metrics.counter(factory.createMetricName("TotalBlockedTasks"));
+        currentBlocked = Metrics.counter(factory.createMetricName("CurrentlyBlockedTasks"));
+
+        completedTasks = Metrics.register(factory.createMetricName("CompletedTasks"), new Gauge<Long>()
         {
-            public Integer value()
-            {
-                return executor.getTotalBlockedTasks();
-            }
-        });
-        currentBlocked = Metrics.newGauge(factory.createMetricName("CurrentlyBlockedTasks"), new Gauge<Long>()
-        {
-            public Long value()
-            {
-                return (long) executor.getCurrentlyBlockedTasks();
-            }
-        });
-        completedTasks = Metrics.newGauge(factory.createMetricName("CompletedTasks"), new Gauge<Long>()
-        {
-            public Long value()
+            public Long getValue()
             {
                 return executor.getCompletedTasks();
             }
         });
-        maxPoolSize =  Metrics.newGauge(factory.createMetricName("MaxPoolSize"), new Gauge<Integer>()
+        maxPoolSize =  Metrics.register(factory.createMetricName("MaxPoolSize"), new Gauge<Integer>()
         {
-            public Integer value()
+            public Integer getValue()
             {
                 return executor.maxWorkers;
             }
@@ -98,11 +89,11 @@
 
     public void release()
     {
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("ActiveTasks"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("PendingTasks"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("CompletedTasks"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("TotalBlockedTasks"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("CurrentlyBlockedTasks"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("MaxPoolSize"));
+        Metrics.remove(factory.createMetricName("ActiveTasks"));
+        Metrics.remove(factory.createMetricName("PendingTasks"));
+        Metrics.remove(factory.createMetricName("CompletedTasks"));
+        Metrics.remove(factory.createMetricName("TotalBlockedTasks"));
+        Metrics.remove(factory.createMetricName("CurrentlyBlockedTasks"));
+        Metrics.remove(factory.createMetricName("MaxPoolSize"));
     }
 }
diff --git a/src/java/org/apache/cassandra/metrics/StorageMetrics.java b/src/java/org/apache/cassandra/metrics/StorageMetrics.java
index 8ee90bb..12196f7 100644
--- a/src/java/org/apache/cassandra/metrics/StorageMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/StorageMetrics.java
@@ -17,8 +17,9 @@
  */
 package org.apache.cassandra.metrics;
 
-import com.yammer.metrics.Metrics;
-import com.yammer.metrics.core.Counter;
+import com.codahale.metrics.Counter;
+
+import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
 
 /**
  * Metrics related to Storage.
@@ -27,8 +28,8 @@
 {
     private static final MetricNameFactory factory = new DefaultNameFactory("Storage");
 
-    public static final Counter load = Metrics.newCounter(factory.createMetricName("Load"));
-    public static final Counter exceptions = Metrics.newCounter(factory.createMetricName("Exceptions"));
-    public static final Counter totalHintsInProgress  = Metrics.newCounter(factory.createMetricName("TotalHintsInProgress"));
-    public static final Counter totalHints = Metrics.newCounter(factory.createMetricName("TotalHints"));
+    public static final Counter load = Metrics.counter(factory.createMetricName("Load"));
+    public static final Counter exceptions = Metrics.counter(factory.createMetricName("Exceptions"));
+    public static final Counter totalHintsInProgress  = Metrics.counter(factory.createMetricName("TotalHintsInProgress"));
+    public static final Counter totalHints = Metrics.counter(factory.createMetricName("TotalHints"));
 }
diff --git a/src/java/org/apache/cassandra/metrics/StreamingMetrics.java b/src/java/org/apache/cassandra/metrics/StreamingMetrics.java
index 4c57812..72e9b23 100644
--- a/src/java/org/apache/cassandra/metrics/StreamingMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/StreamingMetrics.java
@@ -20,10 +20,12 @@
 import java.net.InetAddress;
 import java.util.concurrent.ConcurrentMap;
 
-import com.yammer.metrics.Metrics;
-import com.yammer.metrics.core.Counter;
+
+import com.codahale.metrics.Counter;
 import org.cliffc.high_scale_lib.NonBlockingHashMap;
 
+import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
+
 /**
  * Metrics for streaming.
  */
@@ -33,9 +35,9 @@
 
     private static final ConcurrentMap<InetAddress, StreamingMetrics> instances = new NonBlockingHashMap<InetAddress, StreamingMetrics>();
 
-    public static final Counter activeStreamsOutbound = Metrics.newCounter(DefaultNameFactory.createMetricName(TYPE_NAME, "ActiveOutboundStreams", null));
-    public static final Counter totalIncomingBytes = Metrics.newCounter(DefaultNameFactory.createMetricName(TYPE_NAME, "TotalIncomingBytes", null));
-    public static final Counter totalOutgoingBytes = Metrics.newCounter(DefaultNameFactory.createMetricName(TYPE_NAME, "TotalOutgoingBytes", null));
+    public static final Counter activeStreamsOutbound = Metrics.counter(DefaultNameFactory.createMetricName(TYPE_NAME, "ActiveOutboundStreams", null));
+    public static final Counter totalIncomingBytes = Metrics.counter(DefaultNameFactory.createMetricName(TYPE_NAME, "TotalIncomingBytes", null));
+    public static final Counter totalOutgoingBytes = Metrics.counter(DefaultNameFactory.createMetricName(TYPE_NAME, "TotalOutgoingBytes", null));
     public final Counter incomingBytes;
     public final Counter outgoingBytes;
 
@@ -53,7 +55,7 @@
     public StreamingMetrics(final InetAddress peer)
     {
         MetricNameFactory factory = new DefaultNameFactory("Streaming", peer.getHostAddress().replace(':', '.'));
-        incomingBytes = Metrics.newCounter(factory.createMetricName("IncomingBytes"));
-        outgoingBytes= Metrics.newCounter(factory.createMetricName("OutgoingBytes"));
+        incomingBytes = Metrics.counter(factory.createMetricName("IncomingBytes"));
+        outgoingBytes= Metrics.counter(factory.createMetricName("OutgoingBytes"));
     }
 }
diff --git a/src/java/org/apache/cassandra/metrics/ThreadPoolMetricNameFactory.java b/src/java/org/apache/cassandra/metrics/ThreadPoolMetricNameFactory.java
index 4afc4d3..7810108 100644
--- a/src/java/org/apache/cassandra/metrics/ThreadPoolMetricNameFactory.java
+++ b/src/java/org/apache/cassandra/metrics/ThreadPoolMetricNameFactory.java
@@ -17,8 +17,6 @@
  */
 package org.apache.cassandra.metrics;
 
-import com.yammer.metrics.core.MetricName;
-
 class ThreadPoolMetricNameFactory implements MetricNameFactory
 {
     private final String type;
@@ -32,7 +30,7 @@
         this.poolName = poolName;
     }
 
-    public MetricName createMetricName(String metricName)
+    public CassandraMetricsRegistry.MetricName createMetricName(String metricName)
     {
         String groupName = ThreadPoolMetrics.class.getPackage().getName();
         StringBuilder mbeanName = new StringBuilder();
@@ -42,6 +40,6 @@
         mbeanName.append(",scope=").append(poolName);
         mbeanName.append(",name=").append(metricName);
 
-        return new MetricName(groupName, type, metricName, path + "." + poolName, mbeanName.toString());
+        return new CassandraMetricsRegistry.MetricName(groupName, type, metricName, path + "." + poolName, mbeanName.toString());
     }
 }
diff --git a/src/java/org/apache/cassandra/metrics/ThreadPoolMetrics.java b/src/java/org/apache/cassandra/metrics/ThreadPoolMetrics.java
index a5e6daf..268e878 100644
--- a/src/java/org/apache/cassandra/metrics/ThreadPoolMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/ThreadPoolMetrics.java
@@ -17,10 +17,24 @@
  */
 package org.apache.cassandra.metrics;
 
+import java.io.IOException;
+import java.util.Set;
 import java.util.concurrent.ThreadPoolExecutor;
 
-import com.yammer.metrics.Metrics;
-import com.yammer.metrics.core.*;
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.JmxReporter;
+
+import javax.management.JMX;
+import javax.management.MBeanServerConnection;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimap;
+
+import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
+
 
 /**
  * Metrics for {@link ThreadPoolExecutor}.
@@ -56,32 +70,32 @@
     {
         this.factory = new ThreadPoolMetricNameFactory("ThreadPools", path, poolName);
 
-        activeTasks = Metrics.newGauge(factory.createMetricName("ActiveTasks"), new Gauge<Integer>()
+        activeTasks = Metrics.register(factory.createMetricName("ActiveTasks"), new Gauge<Integer>()
         {
-            public Integer value()
+            public Integer getValue()
             {
                 return executor.getActiveCount();
             }
         });
-        totalBlocked = Metrics.newCounter(factory.createMetricName("TotalBlockedTasks"));
-        currentBlocked = Metrics.newCounter(factory.createMetricName("CurrentlyBlockedTasks"));
-        completedTasks = Metrics.newGauge(factory.createMetricName("CompletedTasks"), new Gauge<Long>()
+        totalBlocked = Metrics.counter(factory.createMetricName("TotalBlockedTasks"));
+        currentBlocked = Metrics.counter(factory.createMetricName("CurrentlyBlockedTasks"));
+        completedTasks = Metrics.register(factory.createMetricName("CompletedTasks"), new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 return executor.getCompletedTaskCount();
             }
         });
-        pendingTasks = Metrics.newGauge(factory.createMetricName("PendingTasks"), new Gauge<Long>()
+        pendingTasks = Metrics.register(factory.createMetricName("PendingTasks"), new Gauge<Long>()
         {
-            public Long value()
+            public Long getValue()
             {
                 return executor.getTaskCount() - executor.getCompletedTaskCount();
             }
         });
-        maxPoolSize =  Metrics.newGauge(factory.createMetricName("MaxPoolSize"), new Gauge<Integer>()
+        maxPoolSize = Metrics.register(factory.createMetricName("MaxPoolSize"), new Gauge<Integer>()
         {
-            public Integer value()
+            public Integer getValue()
             {
                 return executor.getMaximumPoolSize();
             }
@@ -90,11 +104,67 @@
 
     public void release()
     {
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("ActiveTasks"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("PendingTasks"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("CompletedTasks"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("TotalBlockedTasks"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("CurrentlyBlockedTasks"));
-        Metrics.defaultRegistry().removeMetric(factory.createMetricName("MaxPoolSize"));
+        Metrics.remove(factory.createMetricName("ActiveTasks"));
+        Metrics.remove(factory.createMetricName("PendingTasks"));
+        Metrics.remove(factory.createMetricName("CompletedTasks"));
+        Metrics.remove(factory.createMetricName("TotalBlockedTasks"));
+        Metrics.remove(factory.createMetricName("CurrentlyBlockedTasks"));
+        Metrics.remove(factory.createMetricName("MaxPoolSize"));
     }
+
+    public static Object getJmxMetric(MBeanServerConnection mbeanServerConn, String jmxPath, String poolName, String metricName)
+    {
+        String name = String.format("org.apache.cassandra.metrics:type=ThreadPools,path=%s,scope=%s,name=%s", jmxPath, poolName, metricName);
+
+        try
+        {
+            ObjectName oName = new ObjectName(name);
+            if (!mbeanServerConn.isRegistered(oName))
+            {
+                return "N/A";
+            }
+
+            switch (metricName)
+            {
+                case "ActiveTasks":
+                case "PendingTasks":
+                case "CompletedTasks":
+                    return JMX.newMBeanProxy(mbeanServerConn, oName, JmxReporter.JmxGaugeMBean.class).getValue();
+                case "TotalBlockedTasks":
+                case "CurrentlyBlockedTasks":
+                    return JMX.newMBeanProxy(mbeanServerConn, oName, JmxReporter.JmxCounterMBean.class).getCount();
+                default:
+                    throw new AssertionError("Unknown metric name " + metricName);
+            }
+        }
+        catch (Exception e)
+        {
+            throw new RuntimeException("Error reading: " + name, e);
+        }
+    }
+
+    public static Multimap<String, String> getJmxThreadPools(MBeanServerConnection mbeanServerConn)
+    {
+        try
+        {
+            Multimap<String, String> threadPools = HashMultimap.create();
+            Set<ObjectName> threadPoolObjectNames = mbeanServerConn.queryNames(new ObjectName("org.apache.cassandra.metrics:type=ThreadPools,*"),
+                                                                               null);
+            for (ObjectName oName : threadPoolObjectNames)
+            {
+                threadPools.put(oName.getKeyProperty("path"), oName.getKeyProperty("scope"));
+            }
+
+            return threadPools;
+        }
+        catch (MalformedObjectNameException e)
+        {
+            throw new RuntimeException("Bad query to JMX server: ", e);
+        }
+        catch (IOException e)
+        {
+            throw new RuntimeException("Error getting threadpool names from JMX", e);
+        }
+    }
+
 }
diff --git a/src/java/org/apache/cassandra/net/CallbackInfo.java b/src/java/org/apache/cassandra/net/CallbackInfo.java
index b61210c..ea000ae 100644
--- a/src/java/org/apache/cassandra/net/CallbackInfo.java
+++ b/src/java/org/apache/cassandra/net/CallbackInfo.java
@@ -33,17 +33,13 @@
     protected final IVersionedSerializer<?> serializer;
     private final boolean failureCallback;
 
-    public CallbackInfo(InetAddress target, IAsyncCallback callback, IVersionedSerializer<?> serializer)
-    {
-        this(target, callback, serializer, false);
-    }
-
     /**
      * Create CallbackInfo without sent message
      *
      * @param target target to send message
      * @param callback
      * @param serializer serializer to deserialize response message
+     * @param failureCallback True when we have a callback to handle failures
      */
     public CallbackInfo(InetAddress target, IAsyncCallback callback, IVersionedSerializer<?> serializer, boolean failureCallback)
     {
diff --git a/src/java/org/apache/cassandra/net/IMessageSink.java b/src/java/org/apache/cassandra/net/IMessageSink.java
new file mode 100644
index 0000000..5150901
--- /dev/null
+++ b/src/java/org/apache/cassandra/net/IMessageSink.java
@@ -0,0 +1,37 @@
+/*
+ * 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.net;
+
+import java.net.InetAddress;
+
+public interface IMessageSink
+{
+    /**
+     * Allow or drop an outgoing message
+     *
+     * @return true if the message is allowed, false if it should be dropped
+     */
+    boolean allowOutgoingMessage(MessageOut message, int id, InetAddress to);
+
+    /**
+     * Allow or drop an incoming message
+     *
+     * @return true if the message is allowed, false if it should be dropped
+     */
+    boolean allowIncomingMessage(MessageIn message, int id);
+}
diff --git a/src/java/org/apache/cassandra/net/IVerbHandler.java b/src/java/org/apache/cassandra/net/IVerbHandler.java
index 7f835c0..574f30f 100644
--- a/src/java/org/apache/cassandra/net/IVerbHandler.java
+++ b/src/java/org/apache/cassandra/net/IVerbHandler.java
@@ -17,6 +17,8 @@
  */
 package org.apache.cassandra.net;
 
+import java.io.IOException;
+
 /**
  * IVerbHandler provides the method that all verb handlers need to implement.
  * The concrete implementation of this interface would provide the functionality
@@ -34,5 +36,5 @@
      * @param message - incoming message that needs handling.
      * @param id
      */
-    public void doVerb(MessageIn<T> message, int id);
+    public void doVerb(MessageIn<T> message, int id) throws IOException;
 }
diff --git a/src/java/org/apache/cassandra/net/IncomingStreamingConnection.java b/src/java/org/apache/cassandra/net/IncomingStreamingConnection.java
index bfe92f9..f7dc240 100644
--- a/src/java/org/apache/cassandra/net/IncomingStreamingConnection.java
+++ b/src/java/org/apache/cassandra/net/IncomingStreamingConnection.java
@@ -56,9 +56,10 @@
     {
         try
         {
-            // streaming connections are per-session and have a fixed version.  we can't do anything with a wrong-version stream connection, so drop it.
+            // streaming connections are per-session and have a fixed version.
+            // we can't do anything with a wrong-version stream connection, so drop it.
             if (version != StreamMessage.CURRENT_VERSION)
-                throw new IOException(String.format("Received stream using protocol version %d (my version %d). Terminating connection", version, MessagingService.current_version));
+                throw new IOException(String.format("Received stream using protocol version %d (my version %d). Terminating connection", version, StreamMessage.CURRENT_VERSION));
 
             DataInput input = new DataInputStream(socket.getInputStream());
             StreamInitMessage init = StreamInitMessage.serializer.deserialize(input, version);
@@ -71,11 +72,11 @@
             // The receiving side distinguish two connections by looking at StreamInitMessage#isForOutgoing.
             // Note: we cannot use the same socket for incoming and outgoing streams because we want to
             // parallelize said streams and the socket is blocking, so we might deadlock.
-            StreamResultFuture.initReceivingSide(init.sessionIndex, init.planId, init.description, init.from, this, init.isForOutgoing, version);
+            StreamResultFuture.initReceivingSide(init.sessionIndex, init.planId, init.description, init.from, this, init.isForOutgoing, version, init.keepSSTableLevel, init.isIncremental);
         }
-        catch (IOException e)
+        catch (Throwable t)
         {
-            logger.debug("IOException reading from socket; closing", e);
+            logger.error("Error while reading from socket from {}.", socket.getRemoteSocketAddress(), t);
             close();
         }
     }
diff --git a/src/java/org/apache/cassandra/net/IncomingTcpConnection.java b/src/java/org/apache/cassandra/net/IncomingTcpConnection.java
index 198976b..6ac9cd1 100644
--- a/src/java/org/apache/cassandra/net/IncomingTcpConnection.java
+++ b/src/java/org/apache/cassandra/net/IncomingTcpConnection.java
@@ -21,6 +21,8 @@
 import java.net.InetAddress;
 import java.net.Socket;
 import java.net.SocketException;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
 import java.util.zip.Checksum;
 import java.util.Set;
 
@@ -31,12 +33,13 @@
 import net.jpountz.lz4.LZ4FastDecompressor;
 import net.jpountz.lz4.LZ4Factory;
 import net.jpountz.xxhash.XXHashFactory;
-import org.xerial.snappy.SnappyInputStream;
 
 import org.apache.cassandra.config.Config;
+import org.xerial.snappy.SnappyInputStream;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.UnknownColumnFamilyException;
 import org.apache.cassandra.gms.Gossiper;
+import org.apache.cassandra.io.util.NIODataInputStream;
 
 public class IncomingTcpConnection extends Thread implements Closeable
 {
@@ -98,14 +101,14 @@
         }
         catch (IOException e)
         {
-            logger.debug("IOException reading from socket; closing", e);
+            logger.trace("IOException reading from socket; closing", e);
         }
         finally
         {
             close();
         }
     }
-    
+
     @Override
     public void close()
     {
@@ -120,7 +123,7 @@
         }
         catch (IOException e)
         {
-            logger.debug("Error closing socket", e);
+            logger.trace("Error closing socket", e);
         }
         finally
         {
@@ -136,18 +139,18 @@
         // to connect with, the other node will disconnect
         out.writeInt(MessagingService.current_version);
         out.flush();
-        DataInputStream in = new DataInputStream(socket.getInputStream());
+        DataInput in = new DataInputStream(socket.getInputStream());
         int maxVersion = in.readInt();
         // outbound side will reconnect if necessary to upgrade version
         assert version <= MessagingService.current_version;
         from = CompactEndpointSerializationHelper.deserialize(in);
         // record the (true) version of the endpoint
         MessagingService.instance().setVersion(from, maxVersion);
-        logger.debug("Set version for {} to {} (will use {})", from, maxVersion, MessagingService.instance().getVersion(from));
+        logger.trace("Set version for {} to {} (will use {})", from, maxVersion, MessagingService.instance().getVersion(from));
 
         if (compressed)
         {
-            logger.debug("Upgrading incoming connection to be compressed");
+            logger.trace("Upgrading incoming connection to be compressed");
             if (version < MessagingService.VERSION_21)
             {
                 in = new DataInputStream(new SnappyInputStream(socket.getInputStream()));
@@ -163,7 +166,9 @@
         }
         else
         {
-            in = new DataInputStream(new BufferedInputStream(socket.getInputStream(), BUFFER_SIZE));
+            @SuppressWarnings("resource")
+            ReadableByteChannel channel = socket.getChannel();
+            in = new NIODataInputStream(channel != null ? channel : Channels.newChannel(socket.getInputStream()), BUFFER_SIZE);
         }
 
         while (true)
@@ -173,7 +178,7 @@
         }
     }
 
-    private InetAddress receiveMessage(DataInputStream input, int version) throws IOException
+    private InetAddress receiveMessage(DataInput input, int version) throws IOException
     {
         int id;
         if (version < MessagingService.VERSION_20)
@@ -204,7 +209,7 @@
         }
         else
         {
-            logger.debug("Received connection from newer protocol version {}. Ignoring message", version);
+            logger.trace("Received connection from newer protocol version {}. Ignoring message", version);
         }
         return message.from;
     }
diff --git a/src/java/org/apache/cassandra/net/MessageDeliveryTask.java b/src/java/org/apache/cassandra/net/MessageDeliveryTask.java
index a16f53f..4211f5a 100644
--- a/src/java/org/apache/cassandra/net/MessageDeliveryTask.java
+++ b/src/java/org/apache/cassandra/net/MessageDeliveryTask.java
@@ -17,11 +17,14 @@
  */
 package org.apache.cassandra.net;
 
+import java.io.IOException;
 import java.util.EnumSet;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.db.filter.TombstoneOverwhelmingException;
+import org.apache.cassandra.db.index.IndexNotAvailableException;
 import org.apache.cassandra.gms.Gossiper;
 
 public class MessageDeliveryTask implements Runnable
@@ -55,7 +58,7 @@
         IVerbHandler verbHandler = MessagingService.instance().getVerbHandler(verb);
         if (verbHandler == null)
         {
-            logger.debug("Unknown verb {}", verb);
+            logger.trace("Unknown verb {}", verb);
             return;
         }
 
@@ -63,22 +66,37 @@
         {
             verbHandler.doVerb(message, id);
         }
+        catch (IOException ioe)
+        {
+            handleFailure(ioe);
+            throw new RuntimeException(ioe);
+        }
+        catch (TombstoneOverwhelmingException | IndexNotAvailableException e)
+        {
+            handleFailure(e);
+            logger.error(e.getMessage());
+        }
         catch (Throwable t)
         {
-            if (message.doCallbackOnFailure())
-            {
-                MessageOut response = new MessageOut(MessagingService.Verb.INTERNAL_RESPONSE)
-                                                    .withParameter(MessagingService.FAILURE_RESPONSE_PARAM, MessagingService.ONE_BYTE);
-                MessagingService.instance().sendReply(response, id, message.from);
-            }
-
+            handleFailure(t);
             throw t;
         }
+
         if (GOSSIP_VERBS.contains(message.verb))
             Gossiper.instance.setLastProcessedMessageAt(constructionTime);
     }
 
+    private void handleFailure(Throwable t)
+    {
+        if (message.doCallbackOnFailure())
+        {
+            MessageOut response = new MessageOut(MessagingService.Verb.INTERNAL_RESPONSE)
+                                                .withParameter(MessagingService.FAILURE_RESPONSE_PARAM, MessagingService.ONE_BYTE);
+            MessagingService.instance().sendReply(response, id, message.from);
+        }
+    }
+
     private static final EnumSet<MessagingService.Verb> GOSSIP_VERBS = EnumSet.of(MessagingService.Verb.GOSSIP_DIGEST_ACK,
-                                                             MessagingService.Verb.GOSSIP_DIGEST_ACK2,
-                                                             MessagingService.Verb.GOSSIP_DIGEST_SYN);
+                                                                                  MessagingService.Verb.GOSSIP_DIGEST_ACK2,
+                                                                                  MessagingService.Verb.GOSSIP_DIGEST_SYN);
 }
diff --git a/src/java/org/apache/cassandra/net/MessageOut.java b/src/java/org/apache/cassandra/net/MessageOut.java
index c6577be..1e291c2 100644
--- a/src/java/org/apache/cassandra/net/MessageOut.java
+++ b/src/java/org/apache/cassandra/net/MessageOut.java
@@ -18,7 +18,6 @@
 
 package org.apache.cassandra.net;
 
-import java.io.DataOutputStream;
 import java.io.IOException;
 import java.net.InetAddress;
 import java.util.Collections;
@@ -37,6 +36,7 @@
 import org.apache.cassandra.utils.UUIDGen;
 
 import static org.apache.cassandra.tracing.Tracing.TRACE_HEADER;
+import static org.apache.cassandra.tracing.Tracing.TRACE_TYPE;
 import static org.apache.cassandra.tracing.Tracing.isTracing;
 
 public class MessageOut<T>
@@ -46,6 +46,8 @@
     public final T payload;
     public final IVersionedSerializer<T> serializer;
     public final Map<String, byte[]> parameters;
+    private long payloadSize = -1;
+    private int payloadSizeVersion = -1;
 
     // we do support messages that just consist of a verb
     public MessageOut(MessagingService.Verb verb)
@@ -58,8 +60,10 @@
         this(verb,
              payload,
              serializer,
-             isTracing() ? ImmutableMap.of(TRACE_HEADER, UUIDGen.decompose(Tracing.instance.getSessionId()))
-                         : Collections.<String, byte[]>emptyMap());
+             isTracing()
+                 ? ImmutableMap.of(TRACE_HEADER, UUIDGen.decompose(Tracing.instance.getSessionId()),
+                                   TRACE_TYPE, new byte[] { Tracing.TraceType.serialize(Tracing.instance.getTraceType()) })
+                 : Collections.<String, byte[]>emptyMap());
     }
 
     private MessageOut(MessagingService.Verb verb, T payload, IVersionedSerializer<T> serializer, Map<String, byte[]> parameters)
@@ -114,7 +118,7 @@
             out.write(entry.getValue());
         }
 
-        long longSize = payload == null ? 0 : serializer.serializedSize(payload, version);
+        long longSize = payloadSize(version);
         assert longSize <= Integer.MAX_VALUE; // larger values are supported in sstables but not messages
         out.writeInt((int) longSize);
         if (payload != null)
@@ -134,10 +138,38 @@
             size += entry.getValue().length;
         }
 
-        long longSize = payload == null ? 0 : serializer.serializedSize(payload, version);
+        long longSize = payloadSize(version);
         assert longSize <= Integer.MAX_VALUE; // larger values are supported in sstables but not messages
         size += TypeSizes.NATIVE.sizeof((int) longSize);
         size += longSize;
         return size;
     }
+
+    /**
+     * Calculate the size of the payload of this message for the specified protocol version
+     * and memoize the result for the specified protocol version. Memoization only covers the protocol
+     * version of the first invocation.
+     *
+     * It is not safe to call payloadSize concurrently from multiple threads unless it has already been invoked
+     * once from a single thread and there is a happens before relationship between that invocation and other
+     * threads concurrently invoking payloadSize.
+     *
+     * For instance it would be safe to invokePayload size to make a decision in the thread that created the message
+     * and then hand it off to other threads via a thread-safe queue, volatile write, or synchronized/ReentrantLock.
+     * @param version Protocol version to use when calculating payload size
+     * @return Size of the payload of this message in bytes
+     */
+    public long payloadSize(int version)
+    {
+        if (payloadSize == -1)
+        {
+            payloadSize = payload == null ? 0 : serializer.serializedSize(payload, version);
+            payloadSizeVersion = version;
+        }
+        else if (payloadSizeVersion != version)
+        {
+            return payload == null ? 0 : serializer.serializedSize(payload, version);
+        }
+        return payloadSize;
+    }
 }
diff --git a/src/java/org/apache/cassandra/net/MessagingService.java b/src/java/org/apache/cassandra/net/MessagingService.java
index ac8ad79..f0e2fbf 100644
--- a/src/java/org/apache/cassandra/net/MessagingService.java
+++ b/src/java/org/apache/cassandra/net/MessagingService.java
@@ -25,11 +25,13 @@
 import java.nio.channels.ServerSocketChannel;
 import java.util.*;
 import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CopyOnWriteArraySet;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import javax.management.MBeanServer;
 import javax.management.ObjectName;
+import javax.net.ssl.SSLHandshakeException;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Function;
@@ -37,17 +39,22 @@
 import com.google.common.collect.Sets;
 
 import org.cliffc.high_scale_lib.NonBlockingHashMap;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.concurrent.ExecutorLocal;
+import org.apache.cassandra.concurrent.ExecutorLocals;
 import org.apache.cassandra.concurrent.ScheduledExecutors;
 import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.concurrent.StageManager;
-import org.apache.cassandra.concurrent.TracingAwareExecutorService;
+import org.apache.cassandra.concurrent.LocalAwareExecutorService;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions;
 import org.apache.cassandra.db.*;
+import org.apache.cassandra.dht.AbstractBounds;
 import org.apache.cassandra.dht.BootStrapper;
+import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.gms.EchoMessage;
 import org.apache.cassandra.gms.GossipDigestAck;
@@ -59,7 +66,6 @@
 import org.apache.cassandra.locator.ILatencySubscriber;
 import org.apache.cassandra.metrics.ConnectionMetrics;
 import org.apache.cassandra.metrics.DroppedMessageMetrics;
-import org.apache.cassandra.sink.SinkManager;
 import org.apache.cassandra.repair.messages.RepairMessage;
 import org.apache.cassandra.security.SSLFactory;
 import org.apache.cassandra.service.*;
@@ -78,7 +84,8 @@
     public static final int VERSION_12 = 6;
     public static final int VERSION_20 = 7;
     public static final int VERSION_21 = 8;
-    public static final int current_version = VERSION_21;
+    public static final int VERSION_22 = 9;
+    public static final int current_version = VERSION_22;
 
     public static final String FAILURE_CALLBACK_PARAM = "CAL_BAC";
     public static final byte[] ONE_BYTE = new byte[1];
@@ -89,7 +96,7 @@
      */
     public static final int PROTOCOL_MAGIC = 0xCA552DFA;
 
-    private boolean allNodesAtLeast21 = true;
+    private boolean allNodesAtLeast22 = true;
 
     /* All verb handler identifiers */
     public enum Verb
@@ -286,8 +293,7 @@
      * all correspond to client requests or something triggered by them; we don't want to
      * drop internal messages like bootstrap or repair notifications.
      */
-    public static final EnumSet<Verb> DROPPABLE_VERBS = EnumSet.of(Verb.BINARY,
-                                                                   Verb._TRACE,
+    public static final EnumSet<Verb> DROPPABLE_VERBS = EnumSet.of(Verb._TRACE,
                                                                    Verb.MUTATION,
                                                                    Verb.COUNTER_MUTATION,
                                                                    Verb.READ_REPAIR,
@@ -319,6 +325,19 @@
     // protocol versions of the other nodes in the cluster
     private final ConcurrentMap<InetAddress, Integer> versions = new NonBlockingHashMap<InetAddress, Integer>();
 
+    // message sinks are a testing hook
+    private final Set<IMessageSink> messageSinks = new CopyOnWriteArraySet<>();
+
+    public void addMessageSink(IMessageSink sink)
+    {
+        messageSinks.add(sink);
+    }
+
+    public void clearMessageSinks()
+    {
+        messageSinks.clear();
+    }
+
     private static class MSHandle
     {
         public static final MessagingService instance = new MessagingService(false);
@@ -426,27 +445,38 @@
      */
     public void convict(InetAddress ep)
     {
-        logger.debug("Resetting pool for {}", ep);
+        logger.trace("Resetting pool for {}", ep);
         getConnectionPool(ep).reset();
     }
 
+    public void listen()
+    {
+        callbacks.reset(); // hack to allow tests to stop/restart MS
+        listen(FBUtilities.getLocalAddress());
+        if (DatabaseDescriptor.shouldListenOnBroadcastAddress()
+            && !FBUtilities.getLocalAddress().equals(FBUtilities.getBroadcastAddress()))
+        {
+            listen(FBUtilities.getBroadcastAddress());
+        }
+        listenGate.signalAll();
+    }
+
     /**
      * Listen on the specified port.
      *
      * @param localEp InetAddress whose port to listen on.
      */
-    public void listen(InetAddress localEp) throws ConfigurationException
+    private void listen(InetAddress localEp) throws ConfigurationException
     {
-        callbacks.reset(); // hack to allow tests to stop/restart MS
         for (ServerSocket ss : getServerSockets(localEp))
         {
             SocketThread th = new SocketThread(ss, "ACCEPT-" + localEp);
             th.start();
             socketThreads.add(th);
         }
-        listenGate.signalAll();
     }
 
+    @SuppressWarnings("resource")
     private List<ServerSocket> getServerSockets(InetAddress localEp) throws ConfigurationException
     {
         final List<ServerSocket> ss = new ArrayList<ServerSocket>(2);
@@ -482,6 +512,7 @@
             }
             catch (SocketException e)
             {
+                FileUtils.closeQuietly(socket);
                 throw new ConfigurationException("Insufficient permissions to setReuseAddress", e);
             }
             InetSocketAddress address = new InetSocketAddress(localEp, DatabaseDescriptor.getStoragePort());
@@ -491,6 +522,7 @@
             }
             catch (BindException e)
             {
+                FileUtils.closeQuietly(socket);
                 if (e.getMessage().contains("in use"))
                     throw new ConfigurationException(address + " is in use by another process.  Change listen_address:storage_port in cassandra.yaml to values that do not conflict with other services");
                 else if (e.getMessage().contains("Cannot assign requested address"))
@@ -501,9 +533,12 @@
             }
             catch (IOException e)
             {
+                FileUtils.closeQuietly(socket);
                 throw new RuntimeException(e);
             }
-            logger.info("Starting Messaging Service on port {}", DatabaseDescriptor.getStoragePort());
+            String nic = FBUtilities.getNetworkInterface(localEp);
+            logger.info("Starting Messaging Service on {}:{}{}", localEp, DatabaseDescriptor.getStoragePort(),
+                        nic == null? "" : String.format(" (%s)", nic));
             ss.add(socket);
         }
         return ss;
@@ -517,7 +552,7 @@
         }
         catch (InterruptedException ie)
         {
-            logger.debug("await interrupted");
+            logger.trace("await interrupted");
         }
     }
 
@@ -550,7 +585,7 @@
         cp.waitForStarted();
         return cp;
     }
-    
+
 
     public OutboundTcpConnection getConnection(InetAddress to, MessageOut msg)
     {
@@ -662,11 +697,11 @@
      */
     public int sendRR(MessageOut<?> message,
                       InetAddress to,
-                      AbstractWriteResponseHandler handler,
+                      AbstractWriteResponseHandler<?> handler,
                       boolean allowHints)
     {
         int id = addCallback(handler, message, to, message.getTimeout(), handler.consistencyLevel, allowHints);
-        sendOneWay(message, id, to);
+        sendOneWay(message.withParameter(FAILURE_CALLBACK_PARAM, ONE_BYTE), id, to);
         return id;
     }
 
@@ -690,23 +725,21 @@
     public void sendOneWay(MessageOut message, int id, InetAddress to)
     {
         if (logger.isTraceEnabled())
-            logger.trace(FBUtilities.getBroadcastAddress() + " sending " + message.verb + " to " + id + "@" + to);
+            logger.trace("{} sending {} to {}@{}", FBUtilities.getBroadcastAddress(), message.verb, id, to);
 
         if (to.equals(FBUtilities.getBroadcastAddress()))
             logger.trace("Message-to-self {} going over MessagingService", message);
 
         // message sinks are a testing hook
-        MessageOut processedMessage = SinkManager.processOutboundMessage(message, id, to);
-        if (processedMessage == null)
-        {
-            return;
-        }
+        for (IMessageSink ms : messageSinks)
+            if (!ms.allowOutgoingMessage(message, id, to))
+                return;
 
         // get pooled connection (really, connection queue)
-        OutboundTcpConnection connection = getConnection(to, processedMessage);
+        OutboundTcpConnection connection = getConnection(to, message);
 
         // write it
-        connection.enqueue(processedMessage, id);
+        connection.enqueue(message, id);
     }
 
     public <T> AsyncOneResponse<T> sendRR(MessageOut message, InetAddress to)
@@ -736,7 +769,8 @@
         assert !StageManager.getStage(Stage.MUTATION).isShutdown();
 
         // the important part
-        callbacks.shutdownBlocking();
+        if (!callbacks.shutdownBlocking())
+            logger.warn("Failed to wait for messaging service callbacks shutdown");
 
         // attempt to humor tests that try to stop and restart MS
         try
@@ -764,19 +798,16 @@
         if (state != null)
             state.trace("{} message received from {}", message.verb, message.from);
 
-        Verb verb = message.verb;
-        message = SinkManager.processInboundMessage(message, id);
-        if (message == null)
-        {
-            incrementRejectedMessages(verb);
-            return;
-        }
+        // message sinks are a testing hook
+        for (IMessageSink ms : messageSinks)
+            if (!ms.allowIncomingMessage(message, id))
+                return;
 
         Runnable runnable = new MessageDeliveryTask(message, id, timestamp, isCrossNodeTimestamp);
-        TracingAwareExecutorService stage = StageManager.getStage(message.getMessageType());
+        LocalAwareExecutorService stage = StageManager.getStage(message.getMessageType());
         assert stage != null : "No stage for message type " + message.verb;
 
-        stage.execute(runnable, state);
+        stage.execute(runnable, ExecutorLocals.create(state));
     }
 
     public void setCallbackForTests(int messageId, CallbackInfo callback)
@@ -813,9 +844,9 @@
         return packed >>> (start + 1) - count & ~(-1 << count);
     }
 
-    public boolean areAllNodesAtLeast21()
+    public boolean areAllNodesAtLeast22()
     {
-        return allNodesAtLeast21;
+        return allNodesAtLeast22;
     }
 
     /**
@@ -823,37 +854,39 @@
      */
     public int setVersion(InetAddress endpoint, int version)
     {
-        logger.debug("Setting version {} for {}", version, endpoint);
-        if (version < VERSION_21)
-            allNodesAtLeast21 = false;
+        logger.trace("Setting version {} for {}", version, endpoint);
+
+        if (version < VERSION_22)
+            allNodesAtLeast22 = false;
+
         Integer v = versions.put(endpoint, version);
 
-        // if the version was increased to 2.0 or later, see if all nodes are >= 2.0 now
-        if (v != null && v < VERSION_21 && version >= VERSION_21)
-            refreshAllNodesAtLeast21();
+        // if the version was increased to 2.2 or later, see if all nodes are >= 2.2 now
+        if (v != null && v < VERSION_22 && version >= VERSION_22)
+            refreshAllNodesAtLeast22();
 
         return v == null ? version : v;
     }
 
     public void resetVersion(InetAddress endpoint)
     {
-        logger.debug("Resetting version for {}", endpoint);
+        logger.trace("Resetting version for {}", endpoint);
         Integer removed = versions.remove(endpoint);
-        if (removed != null && removed <= VERSION_21)
-            refreshAllNodesAtLeast21();
+        if (removed != null && removed <= VERSION_22)
+            refreshAllNodesAtLeast22();
     }
 
-    private void refreshAllNodesAtLeast21()
+    private void refreshAllNodesAtLeast22()
     {
         for (Integer version: versions.values())
         {
-            if (version < VERSION_21)
+            if (version < VERSION_22)
             {
-                allNodesAtLeast21 = false;
+                allNodesAtLeast22 = false;
                 return;
             }
         }
-        allNodesAtLeast21 = true;
+        allNodesAtLeast22 = true;
     }
 
     public int getVersion(InetAddress endpoint)
@@ -887,7 +920,6 @@
         return versions.containsKey(endpoint);
     }
 
-
     public void incrementDroppedMessages(Verb verb)
     {
         incrementDroppedMessages(verb, false);
@@ -899,20 +931,6 @@
         incrementDroppedMessages(droppedMessagesMap.get(verb), isCrossNodeTimeout);
     }
 
-    /**
-     * Same as incrementDroppedMessages(), but allows non-droppable verbs. Called for IMessageSink-caused message drops.
-     */
-    private void incrementRejectedMessages(Verb verb)
-    {
-        DroppedMessages droppedMessages = droppedMessagesMap.get(verb);
-        if (droppedMessages == null)
-        {
-            droppedMessages = new DroppedMessages(verb);
-            droppedMessagesMap.put(verb, droppedMessages);
-        }
-        incrementDroppedMessages(droppedMessagesMap.get(verb), false);
-    }
-
     private void incrementDroppedMessages(DroppedMessages droppedMessages, boolean isCrossNodeTimeout)
     {
         droppedMessages.metrics.dropped.mark();
@@ -968,6 +986,7 @@
             this.server = server;
         }
 
+        @SuppressWarnings("resource")
         public void run()
         {
             while (!server.isClosed())
@@ -978,7 +997,7 @@
                     socket = server.accept();
                     if (!authenticate(socket))
                     {
-                        logger.debug("remote failed to authenticate");
+                        logger.trace("remote failed to authenticate");
                         socket.close();
                         continue;
                     }
@@ -991,7 +1010,7 @@
                     int header = in.readInt();
                     boolean isStream = MessagingService.getBits(header, 3, 1) == 1;
                     int version = MessagingService.getBits(header, 15, 8);
-                    logger.debug("Connection version {} from {}", version, socket.getInetAddress());
+                    logger.trace("Connection version {} from {}", version, socket.getInetAddress());
                     socket.setSoTimeout(0);
 
                     Thread thread = isStream
@@ -1003,17 +1022,22 @@
                 catch (AsynchronousCloseException e)
                 {
                     // this happens when another thread calls close().
-                    logger.debug("Asynchronous close seen by server thread");
+                    logger.trace("Asynchronous close seen by server thread");
                     break;
                 }
                 catch (ClosedChannelException e)
                 {
-                    logger.debug("MessagingService server thread already closed");
+                    logger.trace("MessagingService server thread already closed");
                     break;
                 }
-                catch (IOException e)
+                catch (SSLHandshakeException e)
                 {
-                    logger.debug("Error reading the socket " + socket, e);
+                    logger.error("SSL handshake error for inbound connection from " + socket, e);
+                    FileUtils.closeQuietly(socket);
+                }
+                catch (Throwable t)
+                {
+                    logger.trace("Error reading the socket {}", socket, t);
                     FileUtils.closeQuietly(socket);
                 }
             }
@@ -1023,6 +1047,7 @@
         void close() throws IOException
         {
             logger.trace("Closing accept() thread");
+
             try
             {
                 server.close();
@@ -1032,7 +1057,7 @@
                 // see https://issues.apache.org/jira/browse/CASSANDRA-8220
                 handleIOException(e);
             }
-            for (Closeable connection : connections) 
+            for (Closeable connection : connections)
             {
                 connection.close();
             }
@@ -1052,81 +1077,101 @@
             throw e;
     }
 
-    public Map<String, Integer> getCommandPendingTasks()
+    public Map<String, Integer> getLargeMessagePendingTasks()
     {
-        Map<String, Integer> pendingTasks = new HashMap<String, Integer>();
+        Map<String, Integer> pendingTasks = new HashMap<String, Integer>(connectionManagers.size());
         for (Map.Entry<InetAddress, OutboundTcpConnectionPool> entry : connectionManagers.entrySet())
-            pendingTasks.put(entry.getKey().getHostAddress(), entry.getValue().cmdCon.getPendingMessages());
+            pendingTasks.put(entry.getKey().getHostAddress(), entry.getValue().largeMessages.getPendingMessages());
         return pendingTasks;
     }
 
-    public int getCommandPendingTasks(InetAddress address)
+    public int getLargeMessagePendingTasks(InetAddress address)
     {
         OutboundTcpConnectionPool connection = connectionManagers.get(address);
-        return connection == null ? 0 : connection.cmdCon.getPendingMessages();
+        return connection == null ? 0 : connection.largeMessages.getPendingMessages();
     }
 
-    public Map<String, Long> getCommandCompletedTasks()
+    public Map<String, Long> getLargeMessageCompletedTasks()
     {
-        Map<String, Long> completedTasks = new HashMap<String, Long>();
+        Map<String, Long> completedTasks = new HashMap<String, Long>(connectionManagers.size());
         for (Map.Entry<InetAddress, OutboundTcpConnectionPool> entry : connectionManagers.entrySet())
-            completedTasks.put(entry.getKey().getHostAddress(), entry.getValue().cmdCon.getCompletedMesssages());
+            completedTasks.put(entry.getKey().getHostAddress(), entry.getValue().largeMessages.getCompletedMesssages());
         return completedTasks;
     }
 
-    public Map<String, Long> getCommandDroppedTasks()
+    public Map<String, Long> getLargeMessageDroppedTasks()
     {
-        Map<String, Long> droppedTasks = new HashMap<String, Long>();
+        Map<String, Long> droppedTasks = new HashMap<String, Long>(connectionManagers.size());
         for (Map.Entry<InetAddress, OutboundTcpConnectionPool> entry : connectionManagers.entrySet())
-            droppedTasks.put(entry.getKey().getHostAddress(), entry.getValue().cmdCon.getDroppedMessages());
+            droppedTasks.put(entry.getKey().getHostAddress(), entry.getValue().largeMessages.getDroppedMessages());
         return droppedTasks;
     }
 
-    public Map<String, Integer> getResponsePendingTasks()
+    public Map<String, Integer> getSmallMessagePendingTasks()
     {
-        Map<String, Integer> pendingTasks = new HashMap<String, Integer>();
+        Map<String, Integer> pendingTasks = new HashMap<String, Integer>(connectionManagers.size());
         for (Map.Entry<InetAddress, OutboundTcpConnectionPool> entry : connectionManagers.entrySet())
-            pendingTasks.put(entry.getKey().getHostAddress(), entry.getValue().ackCon.getPendingMessages());
+            pendingTasks.put(entry.getKey().getHostAddress(), entry.getValue().smallMessages.getPendingMessages());
         return pendingTasks;
     }
 
-    public Map<String, Long> getResponseCompletedTasks()
+    public Map<String, Long> getSmallMessageCompletedTasks()
     {
-        Map<String, Long> completedTasks = new HashMap<String, Long>();
+        Map<String, Long> completedTasks = new HashMap<String, Long>(connectionManagers.size());
         for (Map.Entry<InetAddress, OutboundTcpConnectionPool> entry : connectionManagers.entrySet())
-            completedTasks.put(entry.getKey().getHostAddress(), entry.getValue().ackCon.getCompletedMesssages());
+            completedTasks.put(entry.getKey().getHostAddress(), entry.getValue().smallMessages.getCompletedMesssages());
         return completedTasks;
     }
 
+    public Map<String, Long> getSmallMessageDroppedTasks()
+    {
+        Map<String, Long> droppedTasks = new HashMap<String, Long>(connectionManagers.size());
+        for (Map.Entry<InetAddress, OutboundTcpConnectionPool> entry : connectionManagers.entrySet())
+            droppedTasks.put(entry.getKey().getHostAddress(), entry.getValue().smallMessages.getDroppedMessages());
+        return droppedTasks;
+    }
+
+    public Map<String, Integer> getGossipMessagePendingTasks()
+    {
+        Map<String, Integer> pendingTasks = new HashMap<String, Integer>(connectionManagers.size());
+        for (Map.Entry<InetAddress, OutboundTcpConnectionPool> entry : connectionManagers.entrySet())
+            pendingTasks.put(entry.getKey().getHostAddress(), entry.getValue().gossipMessages.getPendingMessages());
+        return pendingTasks;
+    }
+
+    public Map<String, Long> getGossipMessageCompletedTasks()
+    {
+        Map<String, Long> completedTasks = new HashMap<String, Long>(connectionManagers.size());
+        for (Map.Entry<InetAddress, OutboundTcpConnectionPool> entry : connectionManagers.entrySet())
+            completedTasks.put(entry.getKey().getHostAddress(), entry.getValue().gossipMessages.getCompletedMesssages());
+        return completedTasks;
+    }
+
+    public Map<String, Long> getGossipMessageDroppedTasks()
+    {
+        Map<String, Long> droppedTasks = new HashMap<String, Long>(connectionManagers.size());
+        for (Map.Entry<InetAddress, OutboundTcpConnectionPool> entry : connectionManagers.entrySet())
+            droppedTasks.put(entry.getKey().getHostAddress(), entry.getValue().gossipMessages.getDroppedMessages());
+        return droppedTasks;
+    }
+
     public Map<String, Integer> getDroppedMessages()
     {
-        Map<String, Integer> map = new HashMap<String, Integer>();
+        Map<String, Integer> map = new HashMap<>(droppedMessagesMap.size());
         for (Map.Entry<Verb, DroppedMessages> entry : droppedMessagesMap.entrySet())
-            map.put(entry.getKey().toString(), (int) entry.getValue().metrics.dropped.count());
+            map.put(entry.getKey().toString(), (int) entry.getValue().metrics.dropped.getCount());
         return map;
     }
 
-    public Map<String, Integer> getRecentlyDroppedMessages()
-    {
-        Map<String, Integer> map = new HashMap<String, Integer>();
-        for (Map.Entry<Verb, DroppedMessages> entry : droppedMessagesMap.entrySet())
-            map.put(entry.getKey().toString(), entry.getValue().metrics.getRecentlyDropped());
-        return map;
-    }
 
     public long getTotalTimeouts()
     {
-        return ConnectionMetrics.totalTimeouts.count();
-    }
-
-    public long getRecentTotalTimouts()
-    {
-        return ConnectionMetrics.getRecentTotalTimeout();
+        return ConnectionMetrics.totalTimeouts.getCount();
     }
 
     public Map<String, Long> getTimeoutsPerHost()
     {
-        Map<String, Long> result = new HashMap<String, Long>();
+        Map<String, Long> result = new HashMap<String, Long>(connectionManagers.size());
         for (Map.Entry<InetAddress, OutboundTcpConnectionPool> entry: connectionManagers.entrySet())
         {
             String ip = entry.getKey().getHostAddress();
@@ -1136,16 +1181,15 @@
         return result;
     }
 
-    public Map<String, Long> getRecentTimeoutsPerHost()
+    public static IPartitioner globalPartitioner()
     {
-        Map<String, Long> result = new HashMap<String, Long>();
-        for (Map.Entry<InetAddress, OutboundTcpConnectionPool> entry: connectionManagers.entrySet())
-        {
-            String ip = entry.getKey().getHostAddress();
-            long recent = entry.getValue().getRecentTimeouts();
-            result.put(ip, recent);
-        }
-        return result;
+        return DatabaseDescriptor.getPartitioner();
+    }
+
+    public static void validatePartitioner(AbstractBounds<?> bounds)
+    {
+        if (globalPartitioner() != bounds.left.getPartitioner())
+            throw new AssertionError();
     }
 
     @VisibleForTesting
diff --git a/src/java/org/apache/cassandra/net/MessagingServiceMBean.java b/src/java/org/apache/cassandra/net/MessagingServiceMBean.java
index ff39e23..3bcb0d5 100644
--- a/src/java/org/apache/cassandra/net/MessagingServiceMBean.java
+++ b/src/java/org/apache/cassandra/net/MessagingServiceMBean.java
@@ -29,29 +29,49 @@
 public interface MessagingServiceMBean
 {
     /**
-     * Pending tasks for Command(Mutations, Read etc) TCP Connections
+     * Pending tasks for large message TCP Connections
      */
-    public Map<String, Integer> getCommandPendingTasks();
+    public Map<String, Integer> getLargeMessagePendingTasks();
 
     /**
-     * Completed tasks for Command(Mutations, Read etc) TCP Connections
+     * Completed tasks for large message) TCP Connections
      */
-    public Map<String, Long> getCommandCompletedTasks();
+    public Map<String, Long> getLargeMessageCompletedTasks();
 
     /**
-     * Dropped tasks for Command(Mutations, Read etc) TCP Connections
+     * Dropped tasks for large message TCP Connections
      */
-    public Map<String, Long> getCommandDroppedTasks();
+    public Map<String, Long> getLargeMessageDroppedTasks();
 
     /**
-     * Pending tasks for Response(GOSSIP & RESPONSE) TCP Connections
+     * Pending tasks for small message TCP Connections
      */
-    public Map<String, Integer> getResponsePendingTasks();
+    public Map<String, Integer> getSmallMessagePendingTasks();
 
     /**
-     * Completed tasks for Response(GOSSIP & RESPONSE) TCP Connections
+     * Completed tasks for small message TCP Connections
      */
-    public Map<String, Long> getResponseCompletedTasks();
+    public Map<String, Long> getSmallMessageCompletedTasks();
+
+    /**
+     * Dropped tasks for small message TCP Connections
+     */
+    public Map<String, Long> getSmallMessageDroppedTasks();
+
+    /**
+     * Pending tasks for gossip message TCP Connections
+     */
+    public Map<String, Integer> getGossipMessagePendingTasks();
+
+    /**
+     * Completed tasks for gossip message TCP Connections
+     */
+    public Map<String, Long> getGossipMessageCompletedTasks();
+
+    /**
+     * Dropped tasks for gossip message TCP Connections
+     */
+    public Map<String, Long> getGossipMessageDroppedTasks();
 
     /**
      * dropped message counts for server lifetime
@@ -59,11 +79,6 @@
     public Map<String, Integer> getDroppedMessages();
 
     /**
-     * dropped message counts since last called
-     */
-    public Map<String, Integer> getRecentlyDroppedMessages();
-
-    /**
      * Total number of timeouts happened on this node
      */
     public long getTotalTimeouts();
@@ -73,15 +88,5 @@
      */
     public Map<String, Long> getTimeoutsPerHost();
 
-    /**
-     * Number of timeouts since last check.
-     */
-    public long getRecentTotalTimouts();
-
-    /**
-     * Number of timeouts since last check per host.
-     */
-    public Map<String, Long> getRecentTimeoutsPerHost();
-
     public int getVersion(String address) throws UnknownHostException;
 }
diff --git a/src/java/org/apache/cassandra/net/OutboundTcpConnection.java b/src/java/org/apache/cassandra/net/OutboundTcpConnection.java
index ff2d929..8baac75 100644
--- a/src/java/org/apache/cassandra/net/OutboundTcpConnection.java
+++ b/src/java/org/apache/cassandra/net/OutboundTcpConnection.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.net;
 
-import java.io.BufferedOutputStream;
 import java.io.DataInputStream;
 import java.io.DataOutput;
 import java.io.IOException;
@@ -25,6 +24,8 @@
 import java.net.Socket;
 import java.net.SocketException;
 import java.nio.ByteBuffer;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
 import java.util.*;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.CountDownLatch;
@@ -34,6 +35,8 @@
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.zip.Checksum;
 
+import javax.net.ssl.SSLHandshakeException;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -43,6 +46,8 @@
 import net.jpountz.xxhash.XXHashFactory;
 
 import org.apache.cassandra.io.util.DataOutputStreamPlus;
+import org.apache.cassandra.io.util.BufferedDataOutputStreamPlus;
+import org.apache.cassandra.io.util.WrappedDataOutputStreamPlus;
 import org.apache.cassandra.tracing.TraceState;
 import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.utils.CoalescingStrategies;
@@ -76,6 +81,8 @@
     private static final String BUFFER_SIZE_PROPERTY = PREFIX + "otc_buffer_size";
     private static final int BUFFER_SIZE = Integer.getInteger(BUFFER_SIZE_PROPERTY, 1024 * 64);
 
+    public static final int MAX_COALESCED_MESSAGES = 128;
+
     private static CoalescingStrategy newCoalescingStrategy(String displayName)
     {
         return CoalescingStrategies.newCoalescingStrategy(DatabaseDescriptor.getOtcCoalescingStrategy(),
@@ -94,7 +101,7 @@
         case "MOVINGAVERAGE":
         case "FIXED":
         case "DISABLED":
-            logger.info("OutboundTcpConnection using coalescing strategy " + strategy);
+            logger.info("OutboundTcpConnection using coalescing strategy {}", strategy);
             break;
             default:
                 //Check that it can be loaded
@@ -103,7 +110,7 @@
 
         int coalescingWindow = DatabaseDescriptor.getOtcCoalescingWindow();
         if (coalescingWindow != Config.otc_coalescing_window_us_default)
-            logger.info("OutboundTcpConnection coalescing window set to " + coalescingWindow + "μs");
+            logger.info("OutboundTcpConnection coalescing window set to {}μs", coalescingWindow);
 
         if (coalescingWindow < 0)
             throw new ExceptionInInitializerError(
@@ -129,7 +136,7 @@
     private volatile long completed;
     private final AtomicLong dropped = new AtomicLong();
     private volatile int currentMsgBufferCount = 0;
-    private int targetVersion;
+    private int targetVersion = MessagingService.current_version;
 
     public OutboundTcpConnection(OutboundTcpConnectionPool pool)
     {
@@ -181,7 +188,7 @@
 
     public void run()
     {
-        final int drainedMessageSize = 128;
+        final int drainedMessageSize = MAX_COALESCED_MESSAGES;
         // keeping list (batch) size small for now; that way we don't have an unbounded array (that we never resize)
         final List<QueuedMessage> drainedMessages = new ArrayList<>(drainedMessageSize);
 
@@ -276,7 +283,9 @@
                 // session may have already finished; see CASSANDRA-5668
                 if (state == null)
                 {
-                    TraceState.trace(ByteBuffer.wrap(sessionBytes), message, -1);
+                    byte[] traceTypeBytes = qm.message.parameters.get(Tracing.TRACE_TYPE);
+                    Tracing.TraceType traceType = traceTypeBytes == null ? Tracing.TraceType.QUERY : Tracing.TraceType.deserialize(traceTypeBytes[0]);
+                    TraceState.mutateWithTracing(ByteBuffer.wrap(sessionBytes), message, -1, traceType.getTTL());
                 }
                 else
                 {
@@ -297,10 +306,10 @@
         {
             JVMStabilityInspector.inspectThrowable(e);
             disconnect();
-            if (e instanceof IOException)
+            if (e instanceof IOException || e.getCause() instanceof IOException)
             {
-                if (logger.isDebugEnabled())
-                    logger.debug("error writing to {}", poolReference.endPoint(), e);
+                if (logger.isTraceEnabled())
+                    logger.trace("error writing to {}", poolReference.endPoint(), e);
 
                 // if the message was important, such as a repair acknowledgement, put it back on the queue
                 // to retry after re-connecting.  See CASSANDRA-5393
@@ -374,10 +383,11 @@
         }
     }
 
+    @SuppressWarnings("resource")
     private boolean connect()
     {
-        if (logger.isDebugEnabled())
-            logger.debug("attempting to connect to {}", poolReference.endPoint());
+        if (logger.isTraceEnabled())
+            logger.trace("attempting to connect to {}", poolReference.endPoint());
 
         long start = System.nanoTime();
         long timeout = TimeUnit.MILLISECONDS.toNanos(DatabaseDescriptor.getRpcTimeout());
@@ -407,7 +417,10 @@
                         logger.warn("Failed to set send buffer size on internode socket.", se);
                     }
                 }
-                out = new DataOutputStreamPlus(new BufferedOutputStream(socket.getOutputStream(), BUFFER_SIZE));
+
+                // SocketChannel may be null when using SSL
+                WritableByteChannel ch = socket.getChannel();
+                out = new BufferedDataOutputStreamPlus(ch != null ? ch : Channels.newChannel(socket.getOutputStream()), BUFFER_SIZE);
 
                 out.writeInt(MessagingService.PROTOCOL_MAGIC);
                 writeHeader(out, targetVersion, shouldCompressConnection());
@@ -420,9 +433,7 @@
                     // no version is returned, so disconnect an try again: we will either get
                     // a different target version (targetVersion < MessagingService.VERSION_12)
                     // or if the same version the handshake will finally succeed
-                    logger.debug("Target max version is {}; no version information yet, will retry", maxTargetVersion);
-                    if (DatabaseDescriptor.getSeeds().contains(poolReference.endPoint()))
-                        logger.warn("Seed gossip version is {}; will not connect with that version", maxTargetVersion);
+                    logger.trace("Target max version is {}; no version information yet, will retry", maxTargetVersion);
                     disconnect();
                     continue;
                 }
@@ -433,9 +444,25 @@
 
                 if (targetVersion > maxTargetVersion)
                 {
-                    logger.debug("Target max version is {}; will reconnect with that version", maxTargetVersion);
-                    disconnect();
-                    return false;
+                    logger.trace("Target max version is {}; will reconnect with that version", maxTargetVersion);
+                    try
+                    {
+                        if (DatabaseDescriptor.getSeeds().contains(poolReference.endPoint()))
+                            logger.warn("Seed gossip version is {}; will not connect with that version", maxTargetVersion);
+                    }
+                    catch (Throwable e)
+                    {
+                        // If invalid yaml has been added to the config since startup, getSeeds() will throw an AssertionError
+                        // Additionally, third party seed providers may throw exceptions if network is flakey
+                        // Regardless of what's thrown, we must catch it, disconnect, and try again
+                        JVMStabilityInspector.inspectThrowable(e);
+                        logger.warn("Configuration error prevented outbound connection: {}", e.getLocalizedMessage());
+                    }
+                    finally
+                    {
+                        disconnect();
+                        return false;
+                    }
                 }
 
                 if (targetVersion < maxTargetVersion && targetVersion < MessagingService.current_version)
@@ -454,14 +481,14 @@
                     if (targetVersion < MessagingService.VERSION_21)
                     {
                         // Snappy is buffered, so no need for extra buffering output stream
-                        out = new DataOutputStreamPlus(new SnappyOutputStream(socket.getOutputStream()));
+                        out = new WrappedDataOutputStreamPlus(new SnappyOutputStream(socket.getOutputStream()));
                     }
                     else
                     {
                         // TODO: custom LZ4 OS that supports BB write methods
                         LZ4Compressor compressor = LZ4Factory.fastestInstance().fastCompressor();
                         Checksum checksum = XXHashFactory.fastestInstance().newStreamingHash32(LZ4_HASH_SEED).asChecksum();
-                        out = new DataOutputStreamPlus(new LZ4BlockOutputStream(socket.getOutputStream(),
+                        out = new WrappedDataOutputStreamPlus(new LZ4BlockOutputStream(socket.getOutputStream(),
                                                                             1 << 14,  // 16k block size
                                                                             compressor,
                                                                             checksum,
@@ -471,6 +498,13 @@
 
                 return true;
             }
+            catch (SSLHandshakeException e)
+            {
+                logger.error("SSL handshake error for outbound connection to " + socket, e);
+                socket = null;
+                // SSL errors won't be recoverable within timeout period so we'll just abort
+                return false;
+            }
             catch (IOException e)
             {
                 socket = null;
diff --git a/src/java/org/apache/cassandra/net/OutboundTcpConnectionPool.java b/src/java/org/apache/cassandra/net/OutboundTcpConnectionPool.java
index 400fd35..2b92036 100644
--- a/src/java/org/apache/cassandra/net/OutboundTcpConnectionPool.java
+++ b/src/java/org/apache/cassandra/net/OutboundTcpConnectionPool.java
@@ -36,11 +36,16 @@
 
 public class OutboundTcpConnectionPool
 {
+    public static final long LARGE_MESSAGE_THRESHOLD =
+            Long.getLong(Config.PROPERTY_PREFIX + "otcp_large_message_threshold", 1024 * 64);
+
     // pointer for the real Address.
     private final InetAddress id;
     private final CountDownLatch started;
-    public final OutboundTcpConnection cmdCon;
-    public final OutboundTcpConnection ackCon;
+    public final OutboundTcpConnection smallMessages;
+    public final OutboundTcpConnection largeMessages;
+    public final OutboundTcpConnection gossipMessages;
+
     // pointer to the reset Address.
     private InetAddress resetEndpoint;
     private ConnectionMetrics metrics;
@@ -51,8 +56,9 @@
         resetEndpoint = SystemKeyspace.getPreferredIP(remoteEp);
         started = new CountDownLatch(1);
 
-        cmdCon = new OutboundTcpConnection(this);
-        ackCon = new OutboundTcpConnection(this);
+        smallMessages = new OutboundTcpConnection(this);
+        largeMessages = new OutboundTcpConnection(this);
+        gossipMessages = new OutboundTcpConnection(this);
     }
 
     /**
@@ -61,21 +67,22 @@
      */
     OutboundTcpConnection getConnection(MessageOut msg)
     {
-        Stage stage = msg.getStage();
-        return stage == Stage.REQUEST_RESPONSE || stage == Stage.INTERNAL_RESPONSE || stage == Stage.GOSSIP
-               ? ackCon
-               : cmdCon;
+        if (Stage.GOSSIP == msg.getStage())
+            return gossipMessages;
+        return msg.payloadSize(smallMessages.getTargetVersion()) > LARGE_MESSAGE_THRESHOLD
+               ? largeMessages
+               : smallMessages;
     }
 
     void reset()
     {
-        for (OutboundTcpConnection conn : new OutboundTcpConnection[] { cmdCon, ackCon })
+        for (OutboundTcpConnection conn : new OutboundTcpConnection[] { smallMessages, largeMessages, gossipMessages })
             conn.closeSocket(false);
     }
 
     public void resetToNewerVersion(int version)
     {
-        for (OutboundTcpConnection conn : new OutboundTcpConnection[] { cmdCon, ackCon })
+        for (OutboundTcpConnection conn : new OutboundTcpConnection[] { smallMessages, largeMessages, gossipMessages })
         {
             if (version > conn.getTargetVersion())
                 conn.softCloseSocket();
@@ -91,7 +98,7 @@
     {
         SystemKeyspace.updatePreferredIP(id, remoteEP);
         resetEndpoint = remoteEP;
-        for (OutboundTcpConnection conn : new OutboundTcpConnection[] { cmdCon, ackCon })
+        for (OutboundTcpConnection conn : new OutboundTcpConnection[] { smallMessages, largeMessages, gossipMessages })
             conn.softCloseSocket();
 
         // release previous metrics and create new one with reset address
@@ -101,13 +108,9 @@
 
     public long getTimeouts()
     {
-       return metrics.timeouts.count();
+       return metrics.timeouts.getCount();
     }
 
-    public long getRecentTimeouts()
-    {
-        return metrics.getRecentTimeout();
-    }
 
     public void incrementTimeout()
     {
@@ -119,12 +122,14 @@
         return newSocket(endPoint());
     }
 
+    // Closing the socket will close the underlying channel.
+    @SuppressWarnings("resource")
     public static Socket newSocket(InetAddress endpoint) throws IOException
     {
         // zero means 'bind on any available port.'
         if (isEncryptedChannel(endpoint))
         {
-            if (Config.getOutboundBindAny())
+            if (DatabaseDescriptor.getOutboundBindAny())
                 return SSLFactory.getSocket(DatabaseDescriptor.getServerEncryptionOptions(), endpoint, DatabaseDescriptor.getSSLStoragePort());
             else
                 return SSLFactory.getSocket(DatabaseDescriptor.getServerEncryptionOptions(), endpoint, DatabaseDescriptor.getSSLStoragePort(), FBUtilities.getLocalAddress(), 0);
@@ -132,7 +137,7 @@
         else
         {
             SocketChannel channel = SocketChannel.open();
-            if (!Config.getOutboundBindAny())
+            if (!DatabaseDescriptor.getOutboundBindAny())
                 channel.bind(new InetSocketAddress(FBUtilities.getLocalAddress(), 0));
             channel.connect(new InetSocketAddress(endpoint, DatabaseDescriptor.getStoragePort()));
             return channel.socket();
@@ -168,17 +173,18 @@
         }
         return true;
     }
-    
+
     public void start()
     {
-        cmdCon.start();
-        ackCon.start();
+        smallMessages.start();
+        largeMessages.start();
+        gossipMessages.start();
 
         metrics = new ConnectionMetrics(id, this);
-        
+
         started.countDown();
     }
-    
+
     public void waitForStarted()
     {
         if (started.getCount() == 0)
@@ -202,11 +208,13 @@
     public void close()
     {
         // these null guards are simply for tests
-        if (ackCon != null)
-            ackCon.closeSocket(true);
-        if (cmdCon != null)
-            cmdCon.closeSocket(true);
-        
+        if (largeMessages != null)
+            largeMessages.closeSocket(true);
+        if (smallMessages != null)
+            smallMessages.closeSocket(true);
+        if (gossipMessages != null)
+            gossipMessages.closeSocket(true);
+
         metrics.release();
     }
 }
diff --git a/src/java/org/apache/cassandra/net/ResponseVerbHandler.java b/src/java/org/apache/cassandra/net/ResponseVerbHandler.java
index 0ec91c6..28ed365 100644
--- a/src/java/org/apache/cassandra/net/ResponseVerbHandler.java
+++ b/src/java/org/apache/cassandra/net/ResponseVerbHandler.java
@@ -19,7 +19,6 @@
 
 import java.util.concurrent.TimeUnit;
 
-import org.apache.cassandra.db.IMutation;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -36,7 +35,7 @@
         if (callbackInfo == null)
         {
             String msg = "Callback already removed for {} (from {})";
-            logger.debug(msg, id, message.from);
+            logger.trace(msg, id, message.from);
             Tracing.trace(msg, id, message.from);
             return;
         }
diff --git a/src/java/org/apache/cassandra/net/WriteCallbackInfo.java b/src/java/org/apache/cassandra/net/WriteCallbackInfo.java
index d86df5f..c1fb98d 100644
--- a/src/java/org/apache/cassandra/net/WriteCallbackInfo.java
+++ b/src/java/org/apache/cassandra/net/WriteCallbackInfo.java
@@ -40,7 +40,7 @@
                              ConsistencyLevel consistencyLevel,
                              boolean allowHints)
     {
-        super(target, callback, serializer);
+        super(target, callback, serializer, true);
         assert message != null;
         this.sentMessage = message;
         this.consistencyLevel = consistencyLevel;
diff --git a/src/java/org/apache/cassandra/notifications/SSTableAddedNotification.java b/src/java/org/apache/cassandra/notifications/SSTableAddedNotification.java
index e4aff96..15230ea 100644
--- a/src/java/org/apache/cassandra/notifications/SSTableAddedNotification.java
+++ b/src/java/org/apache/cassandra/notifications/SSTableAddedNotification.java
@@ -17,7 +17,7 @@
  */
 package org.apache.cassandra.notifications;
 
-import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 
 public class SSTableAddedNotification implements INotification
 {
diff --git a/src/java/org/apache/cassandra/notifications/SSTableCompactingNotification.java b/src/java/org/apache/cassandra/notifications/SSTableCompactingNotification.java
deleted file mode 100644
index 6eddf3f..0000000
--- a/src/java/org/apache/cassandra/notifications/SSTableCompactingNotification.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * 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.notifications;
-
-import org.apache.cassandra.io.sstable.SSTableReader;
-
-public class SSTableCompactingNotification implements INotification
-{
-    public final Iterable<SSTableReader> sstables;
-    public final boolean compacting;
-
-    public SSTableCompactingNotification(Iterable<SSTableReader> sstables, boolean compacting)
-    {
-        this.sstables = sstables;
-        this.compacting = compacting;
-    }
-
-    public String toString()
-    {
-        return "SSTableCompactingNotification{" +
-               "sstables=" + sstables +
-               ", compacting=" + compacting +
-               '}';
-    }
-}
diff --git a/src/java/org/apache/cassandra/notifications/SSTableDeletingNotification.java b/src/java/org/apache/cassandra/notifications/SSTableDeletingNotification.java
index 8b0f597..dcaa3b5 100644
--- a/src/java/org/apache/cassandra/notifications/SSTableDeletingNotification.java
+++ b/src/java/org/apache/cassandra/notifications/SSTableDeletingNotification.java
@@ -17,7 +17,7 @@
  */
 package org.apache.cassandra.notifications;
 
-import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 
 /**
  * Fired right before removing an SSTable.
diff --git a/src/java/org/apache/cassandra/notifications/SSTableListChangedNotification.java b/src/java/org/apache/cassandra/notifications/SSTableListChangedNotification.java
index c36583c..7ca574b 100644
--- a/src/java/org/apache/cassandra/notifications/SSTableListChangedNotification.java
+++ b/src/java/org/apache/cassandra/notifications/SSTableListChangedNotification.java
@@ -19,9 +19,8 @@
 
 import java.util.Collection;
 
-import org.apache.cassandra.io.sstable.SSTableReader;
-
 import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 
 public class SSTableListChangedNotification implements INotification
 {
diff --git a/src/java/org/apache/cassandra/notifications/SSTableRepairStatusChanged.java b/src/java/org/apache/cassandra/notifications/SSTableRepairStatusChanged.java
index a473a43..d1398bc 100644
--- a/src/java/org/apache/cassandra/notifications/SSTableRepairStatusChanged.java
+++ b/src/java/org/apache/cassandra/notifications/SSTableRepairStatusChanged.java
@@ -20,7 +20,7 @@
 
 import java.util.Collection;
 
-import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 
 public class SSTableRepairStatusChanged implements INotification
 {
diff --git a/src/java/org/apache/cassandra/repair/AnticompactionTask.java b/src/java/org/apache/cassandra/repair/AnticompactionTask.java
index 8b68fd3..02e18a7 100644
--- a/src/java/org/apache/cassandra/repair/AnticompactionTask.java
+++ b/src/java/org/apache/cassandra/repair/AnticompactionTask.java
@@ -19,74 +19,101 @@
 
 import java.io.IOException;
 import java.net.InetAddress;
+import java.util.Collection;
 import java.util.UUID;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import com.google.common.util.concurrent.AbstractFuture;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.SystemKeyspace;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.gms.ApplicationState;
+import org.apache.cassandra.gms.EndpointState;
 import org.apache.cassandra.gms.FailureDetector;
+import org.apache.cassandra.gms.IEndpointStateChangeSubscriber;
+import org.apache.cassandra.gms.IFailureDetectionEventListener;
+import org.apache.cassandra.gms.VersionedValue;
 import org.apache.cassandra.net.IAsyncCallbackWithFailure;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.repair.messages.AnticompactionRequest;
-import org.apache.cassandra.repair.messages.CleanupMessage;
-import org.apache.cassandra.utils.SemanticVersion;
+import org.apache.cassandra.utils.CassandraVersion;
 
-public class AnticompactionTask extends AbstractFuture<InetAddress> implements Runnable
+public class AnticompactionTask extends AbstractFuture<InetAddress> implements Runnable, IEndpointStateChangeSubscriber,
+                                                                               IFailureDetectionEventListener
 {
     /*
      * Version that anticompaction response is not supported up to.
      * If Cassandra version is more than this, we need to wait for anticompaction response.
      */
-    private static final SemanticVersion VERSION_CHECKER = new SemanticVersion("2.1.5");
+    private static final CassandraVersion VERSION_CHECKER = new CassandraVersion("2.1.5");
+    private static Logger logger = LoggerFactory.getLogger(AnticompactionTask.class);
 
     private final UUID parentSession;
     private final InetAddress neighbor;
-    private final boolean doAnticompaction;
+    private final Collection<Range<Token>> successfulRanges;
+    private final AtomicBoolean isFinished = new AtomicBoolean(false);
 
-    public AnticompactionTask(UUID parentSession, InetAddress neighbor, boolean doAnticompaction)
+    public AnticompactionTask(UUID parentSession, InetAddress neighbor, Collection<Range<Token>> successfulRanges)
     {
         this.parentSession = parentSession;
         this.neighbor = neighbor;
-        this.doAnticompaction = doAnticompaction;
+        this.successfulRanges = successfulRanges;
     }
 
     public void run()
     {
         if (FailureDetector.instance.isAlive(neighbor))
         {
-            AnticompactionRequest acr = new AnticompactionRequest(parentSession);
-            SemanticVersion peerVersion = SystemKeyspace.getReleaseVersion(neighbor);
+            AnticompactionRequest acr = new AnticompactionRequest(parentSession, successfulRanges);
+            CassandraVersion peerVersion = SystemKeyspace.getReleaseVersion(neighbor);
             if (peerVersion != null && peerVersion.compareTo(VERSION_CHECKER) > 0)
             {
-                if (doAnticompaction)
-                {
-                    MessagingService.instance().sendRR(acr.createMessage(), neighbor, new AnticompactionCallback(this), TimeUnit.DAYS.toMillis(1), true);
-                }
-                else
-                {
-                    // we need to clean up parent session
-                    MessagingService.instance().sendRR(new CleanupMessage(parentSession).createMessage(), neighbor, new AnticompactionCallback(this), TimeUnit.DAYS.toMillis(1), true);
-                }
+                MessagingService.instance().sendRR(acr.createMessage(), neighbor, new AnticompactionCallback(this), TimeUnit.DAYS.toMillis(1), true);
             }
             else
             {
-                MessagingService.instance().sendOneWay(acr.createMessage(), neighbor);
                 // immediately return after sending request
-                set(neighbor);
+                MessagingService.instance().sendOneWay(acr.createMessage(), neighbor);
+                maybeSetResult(neighbor);
             }
         }
         else
         {
-            setException(new IOException(neighbor + " is down"));
+            maybeSetException(new IOException(neighbor + " is down"));
         }
     }
 
+    private boolean maybeSetException(Throwable t)
+    {
+        if (isFinished.compareAndSet(false, true))
+        {
+            setException(t);
+            return true;
+        }
+        return false;
+    }
+
+    private boolean maybeSetResult(InetAddress o)
+    {
+        if (isFinished.compareAndSet(false, true))
+        {
+            set(o);
+            return true;
+        }
+        return false;
+    }
+
     /**
      * Callback for antitcompaction request. Run on INTERNAL_RESPONSE stage.
      */
-    public static class AnticompactionCallback implements IAsyncCallbackWithFailure
+    public class AnticompactionCallback implements IAsyncCallbackWithFailure
     {
         final AnticompactionTask task;
 
@@ -97,7 +124,7 @@
 
         public void response(MessageIn msg)
         {
-            task.set(msg.from);
+            maybeSetResult(msg.from);
         }
 
         public boolean isLatencyForSnitch()
@@ -107,7 +134,40 @@
 
         public void onFailure(InetAddress from)
         {
-            task.setException(new RuntimeException("Anticompaction failed or timed out in " + from));
+            maybeSetException(new RuntimeException("Anticompaction failed or timed out in " + from));
+        }
+    }
+
+    public void onJoin(InetAddress endpoint, EndpointState epState) {}
+    public void beforeChange(InetAddress endpoint, EndpointState currentState, ApplicationState newStateKey, VersionedValue newValue) {}
+    public void onChange(InetAddress endpoint, ApplicationState state, VersionedValue value) {}
+    public void onAlive(InetAddress endpoint, EndpointState state) {}
+    public void onDead(InetAddress endpoint, EndpointState state) {}
+
+    public void onRemove(InetAddress endpoint)
+    {
+        convict(endpoint, Double.MAX_VALUE);
+    }
+
+    public void onRestart(InetAddress endpoint, EndpointState epState)
+    {
+        convict(endpoint, Double.MAX_VALUE);
+    }
+
+    public void convict(InetAddress endpoint, double phi)
+    {
+        if (!neighbor.equals(endpoint))
+            return;
+
+        // We want a higher confidence in the failure detection than usual because failing a repair wrongly has a high cost.
+        if (phi < 2 * DatabaseDescriptor.getPhiConvictThreshold())
+            return;
+
+        Exception exception = new IOException(String.format("Endpoint %s died during anti-compaction.", endpoint));
+        if (maybeSetException(exception))
+        {
+            // Though unlikely, it is possible to arrive here multiple time and we want to avoid print an error message twice
+            logger.error("[repair #{}] Endpoint {} died during anti-compaction", endpoint, parentSession, exception);
         }
     }
 }
diff --git a/src/java/org/apache/cassandra/repair/DatacenterAwareRequestCoordinator.java b/src/java/org/apache/cassandra/repair/DatacenterAwareRequestCoordinator.java
deleted file mode 100644
index ab3e03e..0000000
--- a/src/java/org/apache/cassandra/repair/DatacenterAwareRequestCoordinator.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * 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.repair;
-
-import org.apache.cassandra.config.DatabaseDescriptor;
-
-import java.net.InetAddress;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.Map;
-import java.util.Queue;
-
-public class DatacenterAwareRequestCoordinator implements IRequestCoordinator<InetAddress>
-{
-    private Map<String, Queue<InetAddress>> requestsByDatacenter = new HashMap<>();
-    private int remaining = 0;
-    private final IRequestProcessor<InetAddress> processor;
-
-    protected DatacenterAwareRequestCoordinator(IRequestProcessor<InetAddress> processor)
-    {
-        this.processor = processor;
-    }
-
-    public void add(InetAddress request)
-    {
-        String dc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(request);
-        Queue<InetAddress> queue = requestsByDatacenter.get(dc);
-        if (queue == null)
-        {
-            queue = new LinkedList<>();
-            requestsByDatacenter.put(dc, queue);
-        }
-        queue.add(request);
-        remaining++;
-    }
-
-    public void start()
-    {
-        for (Queue<InetAddress> requests : requestsByDatacenter.values())
-        {
-            if (!requests.isEmpty())
-              processor.process(requests.peek());
-        }
-    }
-
-    // Returns how many request remains
-    public int completed(InetAddress request)
-    {
-        String dc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(request);
-        Queue<InetAddress> requests = requestsByDatacenter.get(dc);
-        assert requests != null;
-        assert request.equals(requests.peek());
-        requests.poll();
-        if (!requests.isEmpty())
-            processor.process(requests.peek());
-        return --remaining;
-    }
-}
diff --git a/src/java/org/apache/cassandra/repair/Differencer.java b/src/java/org/apache/cassandra/repair/Differencer.java
deleted file mode 100644
index 214d2c9..0000000
--- a/src/java/org/apache/cassandra/repair/Differencer.java
+++ /dev/null
@@ -1,136 +0,0 @@
-/*
- * 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.repair;
-
-import java.net.InetAddress;
-import java.util.ArrayList;
-import java.util.List;
-
-import com.google.common.base.Objects;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.repair.messages.SyncComplete;
-import org.apache.cassandra.repair.messages.SyncRequest;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.MerkleTree;
-
-/**
- * Runs on the node that initiated a request to compare two trees, and launch repairs for disagreeing ranges.
- */
-public class Differencer implements Runnable
-{
-    private static Logger logger = LoggerFactory.getLogger(Differencer.class);
-
-    private final RepairJobDesc desc;
-    public final TreeResponse r1;
-    public final TreeResponse r2;
-    public final List<Range<Token>> differences = new ArrayList<>();
-
-    public Differencer(RepairJobDesc desc, TreeResponse r1, TreeResponse r2)
-    {
-        this.desc = desc;
-        this.r1 = r1;
-        this.r2 = r2;
-    }
-
-    /**
-     * Compares our trees, and triggers repairs for any ranges that mismatch.
-     */
-    public void run()
-    {
-        // compare trees, and collect differences
-        differences.addAll(MerkleTree.difference(r1.tree, r2.tree));
-
-        // choose a repair method based on the significance of the difference
-        String format = String.format("[repair #%s] Endpoints %s and %s %%s for %s", desc.sessionId, r1.endpoint, r2.endpoint, desc.columnFamily);
-        if (differences.isEmpty())
-        {
-            logger.info(String.format(format, "are consistent"));
-            // send back sync complete message
-            MessagingService.instance().sendOneWay(new SyncComplete(desc, r1.endpoint, r2.endpoint, true).createMessage(), FBUtilities.getLocalAddress());
-            return;
-        }
-
-        // non-0 difference: perform streaming repair
-        logger.info(String.format(format, "have " + differences.size() + " range(s) out of sync"));
-        performStreamingRepair();
-    }
-
-    /**
-     * Starts sending/receiving our list of differences to/from the remote endpoint: creates a callback
-     * that will be called out of band once the streams complete.
-     */
-    void performStreamingRepair()
-    {
-        InetAddress local = FBUtilities.getBroadcastAddress();
-        // We can take anyone of the node as source or destination, however if one is localhost, we put at source to avoid a forwarding
-        InetAddress src = r2.endpoint.equals(local) ? r2.endpoint : r1.endpoint;
-        InetAddress dst = r2.endpoint.equals(local) ? r1.endpoint : r2.endpoint;
-
-        SyncRequest request = new SyncRequest(desc, local, src, dst, differences);
-        StreamingRepairTask task = new StreamingRepairTask(desc, request);
-        task.run();
-    }
-
-
-    /**
-     * In order to remove completed Differencer, equality is computed only from {@code desc} and
-     * endpoint part of two TreeResponses.
-     */
-    @Override
-    public boolean equals(Object o)
-    {
-        if (this == o) return true;
-        if (o == null || getClass() != o.getClass()) return false;
-
-        Differencer that = (Differencer) o;
-        if (!desc.equals(that.desc)) return false;
-        return minEndpoint().equals(that.minEndpoint()) && maxEndpoint().equals(that.maxEndpoint());
-    }
-
-    @Override
-    public int hashCode()
-    {
-        return Objects.hashCode(desc, minEndpoint(), maxEndpoint());
-    }
-
-    // For equals and hashcode, we don't want to take the endpoint order into account.
-    // So we just order endpoint deterministically to simplify this
-    private InetAddress minEndpoint()
-    {
-        return FBUtilities.compareUnsigned(r1.endpoint.getAddress(), r2.endpoint.getAddress()) < 0
-             ? r1.endpoint
-             : r2.endpoint;
-    }
-
-    private InetAddress maxEndpoint()
-    {
-        return FBUtilities.compareUnsigned(r1.endpoint.getAddress(), r2.endpoint.getAddress()) < 0
-             ? r2.endpoint
-             : r1.endpoint;
-    }
-
-    public String toString()
-    {
-        return "#<Differencer " + r1.endpoint + "<->" + r2.endpoint + ":" + desc.columnFamily + "@" + desc.range + ">";
-    }
-}
diff --git a/src/java/org/apache/cassandra/repair/IRepairJobEventListener.java b/src/java/org/apache/cassandra/repair/IRepairJobEventListener.java
deleted file mode 100644
index 778c09d..0000000
--- a/src/java/org/apache/cassandra/repair/IRepairJobEventListener.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * 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.repair;
-
-/**
- * Implemented by the RepairSession to accept callbacks from sequential snapshot creation failure.
- */
-
-public interface IRepairJobEventListener
-{
-    /**
-     * Signal that there was a failure during the snapshot creation process.
-     *
-     */
-    public void failedSnapshot();
-}
diff --git a/src/java/org/apache/cassandra/repair/IRequestCoordinator.java b/src/java/org/apache/cassandra/repair/IRequestCoordinator.java
deleted file mode 100644
index db2ffe3..0000000
--- a/src/java/org/apache/cassandra/repair/IRequestCoordinator.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * 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.repair;
-
-public interface IRequestCoordinator<R>
-{
-    void add(R request);
-
-    void start();
-
-    // Returns how many request remains
-    int completed(R request);
-}
diff --git a/src/java/org/apache/cassandra/repair/IRequestProcessor.java b/src/java/org/apache/cassandra/repair/IRequestProcessor.java
deleted file mode 100644
index c7b49d7..0000000
--- a/src/java/org/apache/cassandra/repair/IRequestProcessor.java
+++ /dev/null
@@ -1,23 +0,0 @@
-/*
- * 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.repair;
-
-public interface IRequestProcessor<R>
-{
-    void process(R request);
-}
diff --git a/src/java/org/apache/cassandra/repair/LocalSyncTask.java b/src/java/org/apache/cassandra/repair/LocalSyncTask.java
new file mode 100644
index 0000000..daace01
--- /dev/null
+++ b/src/java/org/apache/cassandra/repair/LocalSyncTask.java
@@ -0,0 +1,123 @@
+/*
+ * 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.repair;
+
+import java.net.InetAddress;
+import java.util.List;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.SystemKeyspace;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.service.ActiveRepairService;
+import org.apache.cassandra.streaming.ProgressInfo;
+import org.apache.cassandra.streaming.StreamEvent;
+import org.apache.cassandra.streaming.StreamEventHandler;
+import org.apache.cassandra.streaming.StreamPlan;
+import org.apache.cassandra.streaming.StreamState;
+import org.apache.cassandra.tracing.TraceState;
+import org.apache.cassandra.tracing.Tracing;
+import org.apache.cassandra.utils.FBUtilities;
+
+/**
+ * LocalSyncTask performs streaming between local(coordinator) node and remote replica.
+ */
+public class LocalSyncTask extends SyncTask implements StreamEventHandler
+{
+    private final TraceState state = Tracing.instance.get();
+
+    private static final Logger logger = LoggerFactory.getLogger(LocalSyncTask.class);
+
+    private final long repairedAt;
+
+    public LocalSyncTask(RepairJobDesc desc, TreeResponse r1, TreeResponse r2, long repairedAt)
+    {
+        super(desc, r1, r2);
+        this.repairedAt = repairedAt;
+    }
+
+    /**
+     * Starts sending/receiving our list of differences to/from the remote endpoint: creates a callback
+     * that will be called out of band once the streams complete.
+     */
+    protected void startSync(List<Range<Token>> differences)
+    {
+        InetAddress local = FBUtilities.getBroadcastAddress();
+        // We can take anyone of the node as source or destination, however if one is localhost, we put at source to avoid a forwarding
+        InetAddress dst = r2.endpoint.equals(local) ? r1.endpoint : r2.endpoint;
+        InetAddress preferred = SystemKeyspace.getPreferredIP(dst);
+
+        String message = String.format("Performing streaming repair of %d ranges with %s", differences.size(), dst);
+        logger.info("[repair #{}] {}", desc.sessionId, message);
+        boolean isIncremental = false;
+        if (desc.parentSessionId != null)
+        {
+            ActiveRepairService.ParentRepairSession prs = ActiveRepairService.instance.getParentRepairSession(desc.parentSessionId);
+            isIncremental = prs.isIncremental;
+        }
+        Tracing.traceRepair(message);
+        new StreamPlan("Repair", repairedAt, 1, false, isIncremental).listeners(this)
+                                            .flushBeforeTransfer(true)
+                                            // request ranges from the remote node
+                                            .requestRanges(dst, preferred, desc.keyspace, differences, desc.columnFamily)
+                                            // send ranges to the remote node
+                                            .transferRanges(dst, preferred, desc.keyspace, differences, desc.columnFamily)
+                                            .execute();
+    }
+
+    public void handleStreamEvent(StreamEvent event)
+    {
+        if (state == null)
+            return;
+        switch (event.eventType)
+        {
+            case STREAM_PREPARED:
+                StreamEvent.SessionPreparedEvent spe = (StreamEvent.SessionPreparedEvent) event;
+                state.trace("Streaming session with {} prepared", spe.session.peer);
+                break;
+            case STREAM_COMPLETE:
+                StreamEvent.SessionCompleteEvent sce = (StreamEvent.SessionCompleteEvent) event;
+                state.trace("Streaming session with {} {}", sce.peer, sce.success ? "completed successfully" : "failed");
+                break;
+            case FILE_PROGRESS:
+                ProgressInfo pi = ((StreamEvent.ProgressEvent) event).progress;
+                state.trace("{}/{} bytes ({}%) {} idx:{}{}",
+                            new Object[] { pi.currentBytes,
+                                           pi.totalBytes,
+                                           pi.currentBytes * 100 / pi.totalBytes,
+                                           pi.direction == ProgressInfo.Direction.OUT ? "sent to" : "received from",
+                                           pi.sessionIndex,
+                                           pi.peer });
+        }
+    }
+
+    public void onSuccess(StreamState result)
+    {
+        String message = String.format("Sync complete using session %s between %s and %s on %s", desc.sessionId, r1.endpoint, r2.endpoint, desc.columnFamily);
+        logger.info("[repair #{}] {}", desc.sessionId, message);
+        Tracing.traceRepair(message);
+        set(stat);
+    }
+
+    public void onFailure(Throwable t)
+    {
+        setException(t);
+    }
+}
diff --git a/src/java/org/apache/cassandra/repair/ParallelRequestCoordinator.java b/src/java/org/apache/cassandra/repair/ParallelRequestCoordinator.java
deleted file mode 100644
index 839bb43..0000000
--- a/src/java/org/apache/cassandra/repair/ParallelRequestCoordinator.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * 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.repair;
-
-import java.util.HashSet;
-import java.util.Set;
-
-public class ParallelRequestCoordinator<R> implements IRequestCoordinator<R>
-{
-    private final Set<R> requests = new HashSet<>();
-    private final IRequestProcessor<R> processor;
-
-    public ParallelRequestCoordinator(IRequestProcessor<R> processor)
-    {
-        this.processor = processor;
-    }
-
-    @Override
-    public void add(R request) { requests.add(request); }
-
-    @Override
-    public void start()
-    {
-        for (R request : requests)
-            processor.process(request);
-    }
-
-    @Override
-    public int completed(R request)
-    {
-        requests.remove(request);
-        return requests.size();
-    }
-}
diff --git a/src/java/org/apache/cassandra/repair/RemoteSyncTask.java b/src/java/org/apache/cassandra/repair/RemoteSyncTask.java
new file mode 100644
index 0000000..ededc40
--- /dev/null
+++ b/src/java/org/apache/cassandra/repair/RemoteSyncTask.java
@@ -0,0 +1,70 @@
+/*
+ * 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.repair;
+
+import java.net.InetAddress;
+import java.util.List;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.exceptions.RepairException;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.repair.messages.SyncRequest;
+import org.apache.cassandra.tracing.Tracing;
+import org.apache.cassandra.utils.FBUtilities;
+
+/**
+ * RemoteSyncTask sends {@link SyncRequest} to remote(non-coordinator) node
+ * to repair(stream) data with other replica.
+ *
+ * When RemoteSyncTask receives SyncComplete from remote node, task completes.
+ */
+public class RemoteSyncTask extends SyncTask
+{
+    private static final Logger logger = LoggerFactory.getLogger(RemoteSyncTask.class);
+
+    public RemoteSyncTask(RepairJobDesc desc, TreeResponse r1, TreeResponse r2)
+    {
+        super(desc, r1, r2);
+    }
+
+    protected void startSync(List<Range<Token>> differences)
+    {
+        InetAddress local = FBUtilities.getBroadcastAddress();
+        SyncRequest request = new SyncRequest(desc, local, r1.endpoint, r2.endpoint, differences);
+        String message = String.format("Forwarding streaming repair of %d ranges to %s (to be streamed with %s)", request.ranges.size(), request.src, request.dst);
+        logger.info("[repair #{}] {}", desc.sessionId, message);
+        Tracing.traceRepair(message);
+        MessagingService.instance().sendOneWay(request.createMessage(), request.src);
+    }
+
+    public void syncComplete(boolean success)
+    {
+        if (success)
+        {
+            set(stat);
+        }
+        else
+        {
+            setException(new RepairException(desc, String.format("Sync failed between %s and %s", r1.endpoint, r2.endpoint)));
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/repair/RepairFuture.java b/src/java/org/apache/cassandra/repair/RepairFuture.java
deleted file mode 100644
index 127d873..0000000
--- a/src/java/org/apache/cassandra/repair/RepairFuture.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * 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.repair;
-
-import java.util.concurrent.FutureTask;
-
-public class RepairFuture extends FutureTask<Void>
-{
-    public final RepairSession session;
-
-    public RepairFuture(RepairSession session)
-    {
-        super(session, null);
-        this.session = session;
-    }
-}
diff --git a/src/java/org/apache/cassandra/repair/RepairJob.java b/src/java/org/apache/cassandra/repair/RepairJob.java
index 20d5d97..a92233b 100644
--- a/src/java/org/apache/cassandra/repair/RepairJob.java
+++ b/src/java/org/apache/cassandra/repair/RepairJob.java
@@ -19,110 +19,68 @@
 
 import java.net.InetAddress;
 import java.util.*;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.locks.Condition;
 
 import com.google.common.util.concurrent.*;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.repair.messages.ValidationRequest;
+import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.MerkleTree;
-import org.apache.cassandra.utils.concurrent.SimpleCondition;
+import org.apache.cassandra.utils.Pair;
 
 /**
  * RepairJob runs repair on given ColumnFamily.
  */
-public class RepairJob
+public class RepairJob extends AbstractFuture<RepairResult> implements Runnable
 {
     private static Logger logger = LoggerFactory.getLogger(RepairJob.class);
 
-    public final RepairJobDesc desc;
+    private final RepairSession session;
+    private final RepairJobDesc desc;
     private final RepairParallelism parallelismDegree;
-    // first we send tree requests. this tracks the endpoints remaining to hear from
-    private final IRequestCoordinator<InetAddress> treeRequests;
-    // tree responses are then tracked here
-    private final List<TreeResponse> trees = new ArrayList<>();
-    // once all responses are received, each tree is compared with each other, and differencer tasks
-    // are submitted. the job is done when all differencers are complete.
+    private final long repairedAt;
     private final ListeningExecutorService taskExecutor;
-    private final Condition requestsSent = new SimpleCondition();
-    private int gcBefore = -1;
-
-    private volatile boolean failed = false;
-    /* Count down as sync completes */
-    private AtomicInteger waitForSync;
-
-    private final IRepairJobEventListener listener;
 
     /**
      * Create repair job to run on specific columnfamily
+     *
+     * @param session RepairSession that this RepairJob belongs
+     * @param columnFamily name of the ColumnFamily to repair
+     * @param parallelismDegree how to run repair job in parallel
+     * @param repairedAt when the repair occurred (millis)
+     * @param taskExecutor Executor to run various repair tasks
      */
-    public RepairJob(IRepairJobEventListener listener,
-                     UUID parentSessionId,
-                     UUID sessionId,
-                     String keyspace,
+    public RepairJob(RepairSession session,
                      String columnFamily,
-                     Range<Token> range,
                      RepairParallelism parallelismDegree,
+                     long repairedAt,
                      ListeningExecutorService taskExecutor)
     {
-        this.listener = listener;
-        this.desc = new RepairJobDesc(parentSessionId, sessionId, keyspace, columnFamily, range);
-        this.parallelismDegree = parallelismDegree;
+        this.session = session;
+        this.desc = new RepairJobDesc(session.parentRepairSession, session.getId(), session.keyspace, columnFamily, session.getRange());
+        this.repairedAt = repairedAt;
         this.taskExecutor = taskExecutor;
-
-        IRequestProcessor<InetAddress> processor = new IRequestProcessor<InetAddress>()
-        {
-            @Override
-            public void process(InetAddress endpoint)
-            {
-                ValidationRequest request = new ValidationRequest(desc, gcBefore);
-                MessagingService.instance().sendOneWay(request.createMessage(), endpoint);
-            }
-        };
-
-        switch (parallelismDegree)
-        {
-            case SEQUENTIAL:
-                this.treeRequests = new SequentialRequestCoordinator<>(processor);
-                break;
-            case PARALLEL:
-                this.treeRequests = new ParallelRequestCoordinator<>(processor);
-                break;
-            case DATACENTER_AWARE:
-                this.treeRequests = new DatacenterAwareRequestCoordinator(processor);
-                break;
-            default:
-                throw new AssertionError("Unknown degree of parallelism specified");
-        }
+        this.parallelismDegree = parallelismDegree;
     }
 
     /**
-     * @return true if this job failed
+     * Runs repair job.
+     *
+     * This sets up necessary task and runs them on given {@code taskExecutor}.
+     * After submitting all tasks, waits until validation with replica completes.
      */
-    public boolean isFailed()
+    public void run()
     {
-        return failed;
-    }
-
-    /**
-     * Send merkle tree request to every involved neighbor.
-     */
-    public void sendTreeRequests(Collection<InetAddress> endpoints)
-    {
-        // send requests to all nodes
-        List<InetAddress> allEndpoints = new ArrayList<>(endpoints);
+        List<InetAddress> allEndpoints = new ArrayList<>(session.endpoints);
         allEndpoints.add(FBUtilities.getBroadcastAddress());
 
+        ListenableFuture<List<TreeResponse>> validations;
         // Create a snapshot at all nodes unless we're using pure parallel repairs
         if (parallelismDegree != RepairParallelism.PARALLEL)
         {
+            // Request snapshot to all replica
             List<ListenableFuture<InetAddress>> snapshotTasks = new ArrayList<>(allEndpoints.size());
             for (InetAddress endpoint : allEndpoints)
             {
@@ -130,102 +88,205 @@
                 snapshotTasks.add(snapshotTask);
                 taskExecutor.execute(snapshotTask);
             }
+            // When all snapshot complete, send validation requests
             ListenableFuture<List<InetAddress>> allSnapshotTasks = Futures.allAsList(snapshotTasks);
-            // Execute send tree request after all snapshot complete
-            Futures.addCallback(allSnapshotTasks, new FutureCallback<List<InetAddress>>()
+            validations = Futures.transform(allSnapshotTasks, new AsyncFunction<List<InetAddress>, List<TreeResponse>>()
             {
-                public void onSuccess(List<InetAddress> endpoints)
+                public ListenableFuture<List<TreeResponse>> apply(List<InetAddress> endpoints) throws Exception
                 {
-                    sendTreeRequestsInternal(endpoints);
-                }
-
-                public void onFailure(Throwable throwable)
-                {
-                    // TODO need to propagate error to RepairSession
-                    logger.error("Error occurred during snapshot phase", throwable);
-                    listener.failedSnapshot();
-                    failed = true;
+                    if (parallelismDegree == RepairParallelism.SEQUENTIAL)
+                        return sendSequentialValidationRequest(endpoints);
+                    else
+                        return sendDCAwareValidationRequest(endpoints);
                 }
             }, taskExecutor);
         }
         else
         {
-            sendTreeRequestsInternal(allEndpoints);
-        }
-    }
-
-    private void sendTreeRequestsInternal(Collection<InetAddress> endpoints)
-    {
-        this.gcBefore = Keyspace.open(desc.keyspace).getColumnFamilyStore(desc.columnFamily).gcBefore(System.currentTimeMillis());
-        for (InetAddress endpoint : endpoints)
-            treeRequests.add(endpoint);
-
-        logger.info(String.format("[repair #%s] requesting merkle trees for %s (to %s)", desc.sessionId, desc.columnFamily, endpoints));
-        treeRequests.start();
-        requestsSent.signalAll();
-    }
-
-    /**
-     * Add a new received tree and return the number of remaining tree to
-     * be received for the job to be complete.
-     *
-     * Callers may assume exactly one addTree call will result in zero remaining endpoints.
-     *
-     * @param endpoint address of the endpoint that sent response
-     * @param tree sent Merkle tree or null if validation failed on endpoint
-     * @return the number of responses waiting to receive
-     */
-    public synchronized int addTree(InetAddress endpoint, MerkleTree tree)
-    {
-        // Wait for all request to have been performed (see #3400)
-        try
-        {
-            requestsSent.await();
-        }
-        catch (InterruptedException e)
-        {
-            throw new AssertionError("Interrupted while waiting for requests to be sent");
+            // If not sequential, just send validation request to all replica
+            validations = sendValidationRequest(allEndpoints);
         }
 
-        if (tree == null)
-            failed = true;
-        else
-            trees.add(new TreeResponse(endpoint, tree));
-        return treeRequests.completed(endpoint);
-    }
-
-    /**
-     * Submit differencers for running.
-     * All tree *must* have been received before this is called.
-     */
-    public void submitDifferencers()
-    {
-        assert !failed;
-        List<Differencer> differencers = new ArrayList<>();
-        // We need to difference all trees one against another
-        for (int i = 0; i < trees.size() - 1; ++i)
+        // When all validations complete, submit sync tasks
+        ListenableFuture<List<SyncStat>> syncResults = Futures.transform(validations, new AsyncFunction<List<TreeResponse>, List<SyncStat>>()
         {
-            TreeResponse r1 = trees.get(i);
-            for (int j = i + 1; j < trees.size(); ++j)
+            public ListenableFuture<List<SyncStat>> apply(List<TreeResponse> trees) throws Exception
             {
-                TreeResponse r2 = trees.get(j);
-                Differencer differencer = new Differencer(desc, r1, r2);
-                differencers.add(differencer);
-                logger.debug("Queueing comparison {}", differencer);
-            }
-        }
-        waitForSync = new AtomicInteger(differencers.size());
-        for (Differencer differencer : differencers)
-            taskExecutor.submit(differencer);
+                InetAddress local = FBUtilities.getLocalAddress();
 
-        trees.clear(); // allows gc to do its thing
+                List<SyncTask> syncTasks = new ArrayList<>();
+                // We need to difference all trees one against another
+                for (int i = 0; i < trees.size() - 1; ++i)
+                {
+                    TreeResponse r1 = trees.get(i);
+                    for (int j = i + 1; j < trees.size(); ++j)
+                    {
+                        TreeResponse r2 = trees.get(j);
+                        SyncTask task;
+                        if (r1.endpoint.equals(local) || r2.endpoint.equals(local))
+                        {
+                            task = new LocalSyncTask(desc, r1, r2, repairedAt);
+                        }
+                        else
+                        {
+                            task = new RemoteSyncTask(desc, r1, r2);
+                            // RemoteSyncTask expects SyncComplete message sent back.
+                            // Register task to RepairSession to receive response.
+                            session.waitForSync(Pair.create(desc, new NodePair(r1.endpoint, r2.endpoint)), (RemoteSyncTask) task);
+                        }
+                        syncTasks.add(task);
+                        taskExecutor.submit(task);
+                    }
+                }
+                return Futures.allAsList(syncTasks);
+            }
+        }, taskExecutor);
+
+        // When all sync complete, set the final result
+        Futures.addCallback(syncResults, new FutureCallback<List<SyncStat>>()
+        {
+            public void onSuccess(List<SyncStat> stats)
+            {
+                logger.info(String.format("[repair #%s] %s is fully synced", session.getId(), desc.columnFamily));
+                SystemDistributedKeyspace.successfulRepairJob(session.getId(), desc.keyspace, desc.columnFamily);
+                set(new RepairResult(desc, stats));
+            }
+
+            /**
+             * Snapshot, validation and sync failures are all handled here
+             */
+            public void onFailure(Throwable t)
+            {
+                logger.warn(String.format("[repair #%s] %s sync failed", session.getId(), desc.columnFamily));
+                SystemDistributedKeyspace.failedRepairJob(session.getId(), desc.keyspace, desc.columnFamily, t);
+                setException(t);
+            }
+        }, taskExecutor);
+
+        // Wait for validation to complete
+        Futures.getUnchecked(validations);
     }
 
     /**
-     * @return true if the given node pair was the last remaining
+     * Creates {@link ValidationTask} and submit them to task executor in parallel.
+     *
+     * @param endpoints Endpoint addresses to send validation request
+     * @return Future that can get all {@link TreeResponse} from replica, if all validation succeed.
      */
-    boolean completedSynchronization()
+    private ListenableFuture<List<TreeResponse>> sendValidationRequest(Collection<InetAddress> endpoints)
     {
-        return waitForSync.decrementAndGet() == 0;
+        String message = String.format("Requesting merkle trees for %s (to %s)", desc.columnFamily, endpoints);
+        logger.info("[repair #{}] {}", desc.sessionId, message);
+        Tracing.traceRepair(message);
+        int gcBefore = Keyspace.open(desc.keyspace).getColumnFamilyStore(desc.columnFamily).gcBefore(System.currentTimeMillis());
+        List<ListenableFuture<TreeResponse>> tasks = new ArrayList<>(endpoints.size());
+        for (InetAddress endpoint : endpoints)
+        {
+            ValidationTask task = new ValidationTask(desc, endpoint, gcBefore);
+            tasks.add(task);
+            session.waitForValidation(Pair.create(desc, endpoint), task);
+            taskExecutor.execute(task);
+        }
+        return Futures.allAsList(tasks);
+    }
+
+    /**
+     * Creates {@link ValidationTask} and submit them to task executor so that tasks run sequentially.
+     */
+    private ListenableFuture<List<TreeResponse>> sendSequentialValidationRequest(Collection<InetAddress> endpoints)
+    {
+        String message = String.format("Requesting merkle trees for %s (to %s)", desc.columnFamily, endpoints);
+        logger.info("[repair #{}] {}", desc.sessionId, message);
+        Tracing.traceRepair(message);
+        int gcBefore = Keyspace.open(desc.keyspace).getColumnFamilyStore(desc.columnFamily).gcBefore(System.currentTimeMillis());
+        List<ListenableFuture<TreeResponse>> tasks = new ArrayList<>(endpoints.size());
+
+        Queue<InetAddress> requests = new LinkedList<>(endpoints);
+        InetAddress address = requests.poll();
+        ValidationTask firstTask = new ValidationTask(desc, address, gcBefore);
+        logger.info("Validating {}", address);
+        session.waitForValidation(Pair.create(desc, address), firstTask);
+        tasks.add(firstTask);
+        ValidationTask currentTask = firstTask;
+        while (requests.size() > 0)
+        {
+            final InetAddress nextAddress = requests.poll();
+            final ValidationTask nextTask = new ValidationTask(desc, nextAddress, gcBefore);
+            tasks.add(nextTask);
+            Futures.addCallback(currentTask, new FutureCallback<TreeResponse>()
+            {
+                public void onSuccess(TreeResponse result)
+                {
+                    logger.info("Validating {}", nextAddress);
+                    session.waitForValidation(Pair.create(desc, nextAddress), nextTask);
+                    taskExecutor.execute(nextTask);
+                }
+
+                // failure is handled at root of job chain
+                public void onFailure(Throwable t) {}
+            });
+            currentTask = nextTask;
+        }
+        // start running tasks
+        taskExecutor.execute(firstTask);
+        return Futures.allAsList(tasks);
+    }
+
+    /**
+     * Creates {@link ValidationTask} and submit them to task executor so that tasks run sequentially within each dc.
+     */
+    private ListenableFuture<List<TreeResponse>> sendDCAwareValidationRequest(Collection<InetAddress> endpoints)
+    {
+        String message = String.format("Requesting merkle trees for %s (to %s)", desc.columnFamily, endpoints);
+        logger.info("[repair #{}] {}", desc.sessionId, message);
+        Tracing.traceRepair(message);
+        int gcBefore = Keyspace.open(desc.keyspace).getColumnFamilyStore(desc.columnFamily).gcBefore(System.currentTimeMillis());
+        List<ListenableFuture<TreeResponse>> tasks = new ArrayList<>(endpoints.size());
+
+        Map<String, Queue<InetAddress>> requestsByDatacenter = new HashMap<>();
+        for (InetAddress endpoint : endpoints)
+        {
+            String dc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(endpoint);
+            Queue<InetAddress> queue = requestsByDatacenter.get(dc);
+            if (queue == null)
+            {
+                queue = new LinkedList<>();
+                requestsByDatacenter.put(dc, queue);
+            }
+            queue.add(endpoint);
+        }
+
+        for (Map.Entry<String, Queue<InetAddress>> entry : requestsByDatacenter.entrySet())
+        {
+            Queue<InetAddress> requests = entry.getValue();
+            InetAddress address = requests.poll();
+            ValidationTask firstTask = new ValidationTask(desc, address, gcBefore);
+            logger.info("Validating {}", address);
+            session.waitForValidation(Pair.create(desc, address), firstTask);
+            tasks.add(firstTask);
+            ValidationTask currentTask = firstTask;
+            while (requests.size() > 0)
+            {
+                final InetAddress nextAddress = requests.poll();
+                final ValidationTask nextTask = new ValidationTask(desc, nextAddress, gcBefore);
+                tasks.add(nextTask);
+                Futures.addCallback(currentTask, new FutureCallback<TreeResponse>()
+                {
+                    public void onSuccess(TreeResponse result)
+                    {
+                        logger.info("Validating {}", nextAddress);
+                        session.waitForValidation(Pair.create(desc, nextAddress), nextTask);
+                        taskExecutor.execute(nextTask);
+                    }
+
+                    // failure is handled at root of job chain
+                    public void onFailure(Throwable t) {}
+                });
+                currentTask = nextTask;
+            }
+            // start running tasks
+            taskExecutor.execute(firstTask);
+        }
+        return Futures.allAsList(tasks);
     }
 }
diff --git a/src/java/org/apache/cassandra/repair/RepairJobDesc.java b/src/java/org/apache/cassandra/repair/RepairJobDesc.java
index 5ce5969..8382136 100644
--- a/src/java/org/apache/cassandra/repair/RepairJobDesc.java
+++ b/src/java/org/apache/cassandra/repair/RepairJobDesc.java
@@ -100,7 +100,8 @@
             UUIDSerializer.serializer.serialize(desc.sessionId, out, version);
             out.writeUTF(desc.keyspace);
             out.writeUTF(desc.columnFamily);
-            AbstractBounds.serializer.serialize(desc.range, out, version);
+            MessagingService.validatePartitioner(desc.range);
+            AbstractBounds.tokenSerializer.serialize(desc.range, out, version);
         }
 
         public RepairJobDesc deserialize(DataInput in, int version) throws IOException
@@ -114,7 +115,7 @@
             UUID sessionId = UUIDSerializer.serializer.deserialize(in, version);
             String keyspace = in.readUTF();
             String columnFamily = in.readUTF();
-            Range<Token> range = (Range<Token>)AbstractBounds.serializer.deserialize(in, version);
+            Range<Token> range = (Range<Token>)AbstractBounds.tokenSerializer.deserialize(in, MessagingService.globalPartitioner(), version);
             return new RepairJobDesc(parentSessionId, sessionId, keyspace, columnFamily, range);
         }
 
@@ -130,7 +131,7 @@
             size += UUIDSerializer.serializer.serializedSize(desc.sessionId, version);
             size += TypeSizes.NATIVE.sizeof(desc.keyspace);
             size += TypeSizes.NATIVE.sizeof(desc.columnFamily);
-            size += AbstractBounds.serializer.serializedSize(desc.range, version);
+            size += AbstractBounds.tokenSerializer.serializedSize(desc.range, version);
             return size;
         }
     }
diff --git a/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java b/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java
index 7debc93..1701e9a 100644
--- a/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java
+++ b/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java
@@ -17,33 +17,31 @@
  */
 package org.apache.cassandra.repair;
 
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.UUID;
+import java.net.InetAddress;
+import java.util.*;
 
 import com.google.common.base.Predicate;
+import com.google.common.collect.Sets;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.MoreExecutors;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.dht.Bounds;
 import org.apache.cassandra.dht.LocalPartitioner;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.net.IVerbHandler;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.repair.messages.*;
 import org.apache.cassandra.service.ActiveRepairService;
-import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.CassandraVersion;
 
 /**
  * Handles all repair related message.
@@ -61,43 +59,79 @@
         {
             switch (message.payload.messageType)
             {
+                case PREPARE_GLOBAL_MESSAGE:
                 case PREPARE_MESSAGE:
                     PrepareMessage prepareMessage = (PrepareMessage) message.payload;
+                    logger.debug("Preparing, {}", prepareMessage);
                     List<ColumnFamilyStore> columnFamilyStores = new ArrayList<>(prepareMessage.cfIds.size());
                     for (UUID cfId : prepareMessage.cfIds)
                     {
-                        Pair<String, String> kscf = Schema.instance.getCF(cfId);
-                        ColumnFamilyStore columnFamilyStore = Keyspace.open(kscf.left).getColumnFamilyStore(kscf.right);
+                        ColumnFamilyStore columnFamilyStore = ColumnFamilyStore.getIfExists(cfId);
+                        if (columnFamilyStore == null)
+                        {
+                            logErrorAndSendFailureResponse(String.format("Table with id %s was dropped during prepare phase of repair",
+                                                                         cfId.toString()), message.from, id);
+                            return;
+                        }
                         columnFamilyStores.add(columnFamilyStore);
                     }
+                    CassandraVersion peerVersion = SystemKeyspace.getReleaseVersion(message.from);
+                    // note that we default isGlobal to true since old version always default to true:
+                    boolean isGlobal = peerVersion == null ||
+                                       peerVersion.compareTo(ActiveRepairService.SUPPORTS_GLOBAL_PREPARE_FLAG_VERSION) < 0 ||
+                                       message.payload.messageType.equals(RepairMessage.Type.PREPARE_GLOBAL_MESSAGE);
+                    logger.debug("Received prepare message: global message = {}, peerVersion = {},", message.payload.messageType.equals(RepairMessage.Type.PREPARE_GLOBAL_MESSAGE), peerVersion);
                     ActiveRepairService.instance.registerParentRepairSession(prepareMessage.parentRepairSession,
                                                                              message.from,
                                                                              columnFamilyStores,
-                                                                             prepareMessage.ranges);
+                                                                             prepareMessage.ranges,
+                                                                             prepareMessage.isIncremental,
+                                                                             isGlobal);
                     MessagingService.instance().sendReply(new MessageOut(MessagingService.Verb.INTERNAL_RESPONSE), id, message.from);
                     break;
 
                 case SNAPSHOT:
-                    ColumnFamilyStore cfs = Keyspace.open(desc.keyspace).getColumnFamilyStore(desc.columnFamily);
-                    final Range<Token> repairingRange = desc.range;
-                    cfs.snapshot(desc.sessionId.toString(), new Predicate<SSTableReader>()
+                    logger.debug("Snapshotting {}", desc);
+                    final ColumnFamilyStore cfs = ColumnFamilyStore.getIfExists(desc.keyspace, desc.columnFamily);
+                    if (cfs == null)
                     {
-                        public boolean apply(SSTableReader sstable)
+                        logErrorAndSendFailureResponse(String.format("Table %s.%s was dropped during snapshot phase of repair",
+                                                                     desc.keyspace, desc.columnFamily), message.from, id);
+                        return;
+                    }
+                    ActiveRepairService.ParentRepairSession prs = ActiveRepairService.instance.getParentRepairSession(desc.parentSessionId);
+                    if (prs.isGlobal)
+                    {
+                        prs.maybeSnapshot(cfs.metadata.cfId, desc.parentSessionId);
+                    }
+                    else
+                    {
+                        final Range<Token> repairingRange = desc.range;
+                        cfs.snapshot(desc.sessionId.toString(), new Predicate<SSTableReader>()
                         {
-                            return sstable != null &&
-                                    !(sstable.partitioner instanceof LocalPartitioner) && // exclude SSTables from 2i
-                                    new Bounds<>(sstable.first.getToken(), sstable.last.getToken()).intersects(Collections.singleton(repairingRange));
-                        }
-                    }, true); //ephemeral snapshot, if repair fails, it will be cleaned next startup
-
+                            public boolean apply(SSTableReader sstable)
+                            {
+                                return sstable != null &&
+                                       !(sstable.partitioner instanceof LocalPartitioner) && // exclude SSTables from 2i
+                                       new Bounds<>(sstable.first.getToken(), sstable.last.getToken()).intersects(Collections.singleton(repairingRange));
+                            }
+                        }, true); //ephemeral snapshot, if repair fails, it will be cleaned next startup
+                    }
                     logger.debug("Enqueuing response to snapshot request {} to {}", desc.sessionId, message.from);
                     MessagingService.instance().sendReply(new MessageOut(MessagingService.Verb.INTERNAL_RESPONSE), id, message.from);
                     break;
 
                 case VALIDATION_REQUEST:
                     ValidationRequest validationRequest = (ValidationRequest) message.payload;
+                    logger.debug("Validating {}", validationRequest);
                     // trigger read-only compaction
-                    ColumnFamilyStore store = Keyspace.open(desc.keyspace).getColumnFamilyStore(desc.columnFamily);
+                    ColumnFamilyStore store = ColumnFamilyStore.getIfExists(desc.keyspace, desc.columnFamily);
+                    if (store == null)
+                    {
+                        logger.error("Table {}.{} was dropped during snapshot phase of repair", desc.keyspace, desc.columnFamily);
+                        MessagingService.instance().sendOneWay(new ValidationComplete(desc).createMessage(), message.from);
+                        return;
+                    }
 
                     Validator validator = new Validator(desc, message.from, validationRequest.gcBefore);
                     CompactionManager.instance.submitValidation(store, validator);
@@ -106,14 +140,19 @@
                 case SYNC_REQUEST:
                     // forwarded sync request
                     SyncRequest request = (SyncRequest) message.payload;
-                    StreamingRepairTask task = new StreamingRepairTask(desc, request);
+                    logger.debug("Syncing {}", request);
+                    long repairedAt = ActiveRepairService.UNREPAIRED_SSTABLE;
+                    if (desc.parentSessionId != null && ActiveRepairService.instance.getParentRepairSession(desc.parentSessionId) != null)
+                        repairedAt = ActiveRepairService.instance.getParentRepairSession(desc.parentSessionId).getRepairedAt();
+
+                    StreamingRepairTask task = new StreamingRepairTask(desc, request, repairedAt);
                     task.run();
                     break;
 
                 case ANTICOMPACTION_REQUEST:
-                    logger.debug("Got anticompaction request");
                     AnticompactionRequest anticompactionRequest = (AnticompactionRequest) message.payload;
-                    ListenableFuture<?> compactionDone = ActiveRepairService.instance.doAntiCompaction(anticompactionRequest.parentRepairSession);
+                    logger.debug("Got anticompaction request {}", anticompactionRequest);
+                    ListenableFuture<?> compactionDone = ActiveRepairService.instance.doAntiCompaction(anticompactionRequest.parentRepairSession, anticompactionRequest.successfulRanges);
                     compactionDone.addListener(new Runnable()
                     {
                         @Override
@@ -139,9 +178,17 @@
         catch (Exception e)
         {
             logger.error("Got error, removing parent repair session");
-            if (desc!=null && desc.parentSessionId != null)
+            if (desc != null && desc.parentSessionId != null)
                 ActiveRepairService.instance.removeParentRepairSession(desc.parentSessionId);
             throw new RuntimeException(e);
         }
     }
+
+    private void logErrorAndSendFailureResponse(String errorMessage, InetAddress to, int id)
+    {
+        logger.error(errorMessage);
+        MessageOut reply = new MessageOut(MessagingService.Verb.INTERNAL_RESPONSE)
+                               .withParameter(MessagingService.FAILURE_RESPONSE_PARAM, MessagingService.ONE_BYTE);
+        MessagingService.instance().sendReply(reply, id, to);
+    }
 }
diff --git a/src/java/org/apache/cassandra/repair/RepairParallelism.java b/src/java/org/apache/cassandra/repair/RepairParallelism.java
index 0afde28..3094302 100644
--- a/src/java/org/apache/cassandra/repair/RepairParallelism.java
+++ b/src/java/org/apache/cassandra/repair/RepairParallelism.java
@@ -28,15 +28,50 @@
     /**
      * One node at a time
      */
-    SEQUENTIAL,
+    SEQUENTIAL("sequential"),
 
     /**
      * All nodes at the same time
      */
-    PARALLEL,
+    PARALLEL("parallel"),
 
     /**
      * One node per data center at a time
      */
-    DATACENTER_AWARE
+    DATACENTER_AWARE("dc_parallel");
+
+    private final String name;
+
+    /**
+     * Return RepairParallelism that match given name.
+     * If name is null, or does not match any, this returns default "sequential" parallelism,
+     *
+     * @param name name of repair parallelism
+     * @return RepairParallelism that match given name
+     */
+    public static RepairParallelism fromName(String name)
+    {
+        if (PARALLEL.getName().equals(name))
+            return PARALLEL;
+        else if (DATACENTER_AWARE.getName().equals(name))
+            return DATACENTER_AWARE;
+        else
+            return SEQUENTIAL;
+    }
+
+    private RepairParallelism(String name)
+    {
+        this.name = name;
+    }
+
+    public String getName()
+    {
+        return name;
+    }
+
+    @Override
+    public String toString()
+    {
+        return getName();
+    }
 }
diff --git a/src/java/org/apache/cassandra/repair/RepairResult.java b/src/java/org/apache/cassandra/repair/RepairResult.java
new file mode 100644
index 0000000..333b48a
--- /dev/null
+++ b/src/java/org/apache/cassandra/repair/RepairResult.java
@@ -0,0 +1,35 @@
+/*
+ * 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.repair;
+
+import java.util.List;
+
+/**
+ * RepairJob's result
+ */
+public class RepairResult
+{
+    public final RepairJobDesc desc;
+    public final List<SyncStat> stats;
+
+    public RepairResult(RepairJobDesc desc, List<SyncStat> stats)
+    {
+        this.desc = desc;
+        this.stats = stats;
+    }
+}
diff --git a/src/java/org/apache/cassandra/repair/RepairRunnable.java b/src/java/org/apache/cassandra/repair/RepairRunnable.java
new file mode 100644
index 0000000..0f2e839
--- /dev/null
+++ b/src/java/org/apache/cassandra/repair/RepairRunnable.java
@@ -0,0 +1,424 @@
+/*
+ * 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.repair;
+
+import java.net.InetAddress;
+import java.nio.ByteBuffer;
+import java.util.*;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import com.google.common.util.concurrent.*;
+import org.apache.commons.lang3.time.DurationFormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.concurrent.JMXConfigurableThreadPoolExecutor;
+import org.apache.cassandra.concurrent.NamedThreadFactory;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.cql3.statements.SelectStatement;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.repair.messages.RepairOption;
+import org.apache.cassandra.service.ActiveRepairService;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.tracing.TraceKeyspace;
+import org.apache.cassandra.tracing.TraceState;
+import org.apache.cassandra.tracing.Tracing;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.UUIDGen;
+import org.apache.cassandra.utils.WrappedRunnable;
+import org.apache.cassandra.utils.progress.ProgressEvent;
+import org.apache.cassandra.utils.progress.ProgressEventNotifier;
+import org.apache.cassandra.utils.progress.ProgressEventType;
+import org.apache.cassandra.utils.progress.ProgressListener;
+
+public class RepairRunnable extends WrappedRunnable implements ProgressEventNotifier
+{
+    private static final Logger logger = LoggerFactory.getLogger(RepairRunnable.class);
+
+    private StorageService storageService;
+    private final int cmd;
+    private final RepairOption options;
+    private final String keyspace;
+
+    private final List<ProgressListener> listeners = new ArrayList<>();
+
+    public RepairRunnable(StorageService storageService, int cmd, RepairOption options, String keyspace)
+    {
+        this.storageService = storageService;
+        this.cmd = cmd;
+        this.options = options;
+        this.keyspace = keyspace;
+    }
+
+    @Override
+    public void addProgressListener(ProgressListener listener)
+    {
+        listeners.add(listener);
+    }
+
+    @Override
+    public void removeProgressListener(ProgressListener listener)
+    {
+        listeners.remove(listener);
+    }
+
+    protected void fireProgressEvent(String tag, ProgressEvent event)
+    {
+        for (ProgressListener listener : listeners)
+        {
+            listener.progress(tag, event);
+        }
+    }
+
+    protected void fireErrorAndComplete(String tag, int progressCount, int totalProgress, String message)
+    {
+        fireProgressEvent(tag, new ProgressEvent(ProgressEventType.ERROR, progressCount, totalProgress, message));
+        fireProgressEvent(tag, new ProgressEvent(ProgressEventType.COMPLETE, progressCount, totalProgress, String.format("Repair command #%d finished with error", cmd)));
+    }
+
+    protected void runMayThrow() throws Exception
+    {
+        final TraceState traceState;
+
+        final String tag = "repair:" + cmd;
+
+        final AtomicInteger progress = new AtomicInteger();
+        final int totalProgress = 4 + options.getRanges().size(); // get valid column families, calculate neighbors, validation, prepare for repair + number of ranges to repair
+
+        String[] columnFamilies = options.getColumnFamilies().toArray(new String[options.getColumnFamilies().size()]);
+        Iterable<ColumnFamilyStore> validColumnFamilies;
+        try
+        {
+            validColumnFamilies = storageService.getValidColumnFamilies(false, false, keyspace, columnFamilies);
+            progress.incrementAndGet();
+        }
+        catch (IllegalArgumentException e)
+        {
+            logger.error("Repair failed:", e);
+            fireErrorAndComplete(tag, progress.get(), totalProgress, e.getMessage());
+            return;
+        }
+
+        final long startTime = System.currentTimeMillis();
+        String message = String.format("Starting repair command #%d, repairing keyspace %s with %s", cmd, keyspace,
+                                       options);
+        logger.info(message);
+        fireProgressEvent(tag, new ProgressEvent(ProgressEventType.START, 0, 100, message));
+        if (options.isTraced())
+        {
+            StringBuilder cfsb = new StringBuilder();
+            for (ColumnFamilyStore cfs : validColumnFamilies)
+                cfsb.append(", ").append(cfs.keyspace.getName()).append(".").append(cfs.name);
+
+            UUID sessionId = Tracing.instance.newSession(Tracing.TraceType.REPAIR);
+            traceState = Tracing.instance.begin("repair", ImmutableMap.of("keyspace", keyspace, "columnFamilies",
+                                                                          cfsb.substring(2)));
+            Tracing.traceRepair(message);
+            traceState.enableActivityNotification(tag);
+            for (ProgressListener listener : listeners)
+                traceState.addProgressListener(listener);
+            Thread queryThread = createQueryThread(cmd, sessionId);
+            queryThread.setName("RepairTracePolling");
+            queryThread.start();
+        }
+        else
+        {
+            traceState = null;
+        }
+
+        final Set<InetAddress> allNeighbors = new HashSet<>();
+        Map<Range, Set<InetAddress>> rangeToNeighbors = new HashMap<>();
+
+        //pre-calculate output of getLocalRanges and pass it to getNeighbors to increase performance and prevent
+        //calculation multiple times
+        Collection<Range<Token>> keyspaceLocalRanges = storageService.getLocalRanges(keyspace);
+
+        try
+        {
+            for (Range<Token> range : options.getRanges())
+            {
+                    Set<InetAddress> neighbors = ActiveRepairService.getNeighbors(keyspace, keyspaceLocalRanges,
+                                                                                  range, options.getDataCenters(),
+                                                                                  options.getHosts());
+                    rangeToNeighbors.put(range, neighbors);
+                    allNeighbors.addAll(neighbors);
+            }
+            progress.incrementAndGet();
+        }
+        catch (IllegalArgumentException e)
+        {
+            logger.error("Repair failed:", e);
+            fireErrorAndComplete(tag, progress.get(), totalProgress, e.getMessage());
+            return;
+        }
+
+        // Validate columnfamilies
+        List<ColumnFamilyStore> columnFamilyStores = new ArrayList<>();
+        try
+        {
+            Iterables.addAll(columnFamilyStores, validColumnFamilies);
+            progress.incrementAndGet();
+        }
+        catch (IllegalArgumentException e)
+        {
+            fireErrorAndComplete(tag, progress.get(), totalProgress, e.getMessage());
+            return;
+        }
+
+        String[] cfnames = new String[columnFamilyStores.size()];
+        for (int i = 0; i < columnFamilyStores.size(); i++)
+        {
+            cfnames[i] = columnFamilyStores.get(i).name;
+        }
+
+        final UUID parentSession = UUIDGen.getTimeUUID();
+        SystemDistributedKeyspace.startParentRepair(parentSession, keyspace, cfnames, options.getRanges());
+        long repairedAt;
+        try
+        {
+            ActiveRepairService.instance.prepareForRepair(parentSession, FBUtilities.getBroadcastAddress(), allNeighbors, options, columnFamilyStores);
+            repairedAt = ActiveRepairService.instance.getParentRepairSession(parentSession).getRepairedAt();
+            progress.incrementAndGet();
+        }
+        catch (Throwable t)
+        {
+            SystemDistributedKeyspace.failParentRepair(parentSession, t);
+            fireErrorAndComplete(tag, progress.get(), totalProgress, t.getMessage());
+            return;
+        }
+
+        // Set up RepairJob executor for this repair command.
+        final ListeningExecutorService executor = MoreExecutors.listeningDecorator(new JMXConfigurableThreadPoolExecutor(options.getJobThreads(),
+                                                                                                                         Integer.MAX_VALUE,
+                                                                                                                         TimeUnit.SECONDS,
+                                                                                                                         new LinkedBlockingQueue<Runnable>(),
+                                                                                                                         new NamedThreadFactory("Repair#" + cmd),
+                                                                                                                         "internal"));
+
+        List<ListenableFuture<RepairSessionResult>> futures = new ArrayList<>(options.getRanges().size());
+        for (Range<Token> range : options.getRanges())
+        {
+            final RepairSession session = ActiveRepairService.instance.submitRepairSession(parentSession,
+                                                              range,
+                                                              keyspace,
+                                                              options.getParallelism(),
+                                                              rangeToNeighbors.get(range),
+                                                              repairedAt,
+                                                              executor,
+                                                              cfnames);
+            if (session == null)
+                continue;
+            // After repair session completes, notify client its result
+            Futures.addCallback(session, new FutureCallback<RepairSessionResult>()
+            {
+                public void onSuccess(RepairSessionResult result)
+                {
+                    /**
+                     * If the success message below is modified, it must also be updated on
+                     * {@link org.apache.cassandra.utils.progress.jmx.LegacyJMXProgressSupport}
+                     * for backward-compatibility support.
+                     */
+                    String message = String.format("Repair session %s for range %s finished", session.getId(),
+                                                   session.getRange().toString());
+                    logger.info(message);
+                    fireProgressEvent(tag, new ProgressEvent(ProgressEventType.PROGRESS,
+                                                             progress.incrementAndGet(),
+                                                             totalProgress,
+                                                             message));
+                }
+
+                public void onFailure(Throwable t)
+                {
+                    /**
+                     * If the failure message below is modified, it must also be updated on
+                     * {@link org.apache.cassandra.utils.progress.jmx.LegacyJMXProgressSupport}
+                     * for backward-compatibility support.
+                     */
+                    String message = String.format("Repair session %s for range %s failed with error %s",
+                                                   session.getId(), session.getRange().toString(), t.getMessage());
+                    logger.error(message, t);
+                    fireProgressEvent(tag, new ProgressEvent(ProgressEventType.PROGRESS,
+                                                             progress.incrementAndGet(),
+                                                             totalProgress,
+                                                             message));
+                }
+            });
+            futures.add(session);
+        }
+
+        // After all repair sessions completes(successful or not),
+        // run anticompaction if necessary and send finish notice back to client
+        final Collection<Range<Token>> successfulRanges = new ArrayList<>();
+        final AtomicBoolean hasFailure = new AtomicBoolean();
+        final ListenableFuture<List<RepairSessionResult>> allSessions = Futures.successfulAsList(futures);
+        ListenableFuture anticompactionResult = Futures.transform(allSessions, new AsyncFunction<List<RepairSessionResult>, Object>()
+        {
+            @SuppressWarnings("unchecked")
+            public ListenableFuture apply(List<RepairSessionResult> results) throws Exception
+            {
+                // filter out null(=failed) results and get successful ranges
+                for (RepairSessionResult sessionResult : results)
+                {
+                    if (sessionResult != null)
+                    {
+                        successfulRanges.add(sessionResult.range);
+                    }
+                    else
+                    {
+                        hasFailure.compareAndSet(false, true);
+                    }
+                }
+                return ActiveRepairService.instance.finishParentSession(parentSession, allNeighbors, successfulRanges);
+            }
+        });
+        Futures.addCallback(anticompactionResult, new FutureCallback<Object>()
+        {
+            public void onSuccess(Object result)
+            {
+                SystemDistributedKeyspace.successfulParentRepair(parentSession, successfulRanges);
+                if (hasFailure.get())
+                {
+                    fireProgressEvent(tag, new ProgressEvent(ProgressEventType.ERROR, progress.get(), totalProgress,
+                                                             "Some repair failed"));
+                }
+                else
+                {
+                    fireProgressEvent(tag, new ProgressEvent(ProgressEventType.SUCCESS, progress.get(), totalProgress,
+                                                             "Repair completed successfully"));
+                }
+                repairComplete();
+            }
+
+            public void onFailure(Throwable t)
+            {
+                fireProgressEvent(tag, new ProgressEvent(ProgressEventType.ERROR, progress.get(), totalProgress, t.getMessage()));
+                SystemDistributedKeyspace.failParentRepair(parentSession, t);
+                repairComplete();
+            }
+
+            private void repairComplete()
+            {
+                String duration = DurationFormatUtils.formatDurationWords(System.currentTimeMillis() - startTime,
+                                                                          true, true);
+                String message = String.format("Repair command #%d finished in %s", cmd, duration);
+                fireProgressEvent(tag, new ProgressEvent(ProgressEventType.COMPLETE, progress.get(), totalProgress, message));
+                logger.info(message);
+                if (options.isTraced() && traceState != null)
+                {
+                    for (ProgressListener listener : listeners)
+                        traceState.removeProgressListener(listener);
+                    // Because DebuggableThreadPoolExecutor#afterExecute and this callback
+                    // run in a nondeterministic order (within the same thread), the
+                    // TraceState may have been nulled out at this point. The TraceState
+                    // should be traceState, so just set it without bothering to check if it
+                    // actually was nulled out.
+                    Tracing.instance.set(traceState);
+                    Tracing.traceRepair(message);
+                    Tracing.instance.stopSession();
+                }
+                executor.shutdownNow();
+            }
+        });
+    }
+
+    private Thread createQueryThread(final int cmd, final UUID sessionId)
+    {
+        return new Thread(new WrappedRunnable()
+        {
+            // Query events within a time interval that overlaps the last by one second. Ignore duplicates. Ignore local traces.
+            // Wake up upon local trace activity. Query when notified of trace activity with a timeout that doubles every two timeouts.
+            public void runMayThrow() throws Exception
+            {
+                TraceState state = Tracing.instance.get(sessionId);
+                if (state == null)
+                    throw new Exception("no tracestate");
+
+                String format = "select event_id, source, activity from %s.%s where session_id = ? and event_id > ? and event_id < ?;";
+                String query = String.format(format, TraceKeyspace.NAME, TraceKeyspace.EVENTS);
+                SelectStatement statement = (SelectStatement) QueryProcessor.parseStatement(query).prepare().statement;
+
+                ByteBuffer sessionIdBytes = ByteBufferUtil.bytes(sessionId);
+                InetAddress source = FBUtilities.getBroadcastAddress();
+
+                HashSet<UUID>[] seen = new HashSet[] { new HashSet<>(), new HashSet<>() };
+                int si = 0;
+                UUID uuid;
+
+                long tlast = System.currentTimeMillis(), tcur;
+
+                TraceState.Status status;
+                long minWaitMillis = 125;
+                long maxWaitMillis = 1000 * 1024L;
+                long timeout = minWaitMillis;
+                boolean shouldDouble = false;
+
+                while ((status = state.waitActivity(timeout)) != TraceState.Status.STOPPED)
+                {
+                    if (status == TraceState.Status.IDLE)
+                    {
+                        timeout = shouldDouble ? Math.min(timeout * 2, maxWaitMillis) : timeout;
+                        shouldDouble = !shouldDouble;
+                    }
+                    else
+                    {
+                        timeout = minWaitMillis;
+                        shouldDouble = false;
+                    }
+                    ByteBuffer tminBytes = ByteBufferUtil.bytes(UUIDGen.minTimeUUID(tlast - 1000));
+                    ByteBuffer tmaxBytes = ByteBufferUtil.bytes(UUIDGen.maxTimeUUID(tcur = System.currentTimeMillis()));
+                    QueryOptions options = QueryOptions.forInternalCalls(ConsistencyLevel.ONE, Lists.newArrayList(sessionIdBytes,
+                                                                                                                  tminBytes,
+                                                                                                                  tmaxBytes));
+                    ResultMessage.Rows rows = statement.execute(QueryState.forInternalCalls(), options);
+                    UntypedResultSet result = UntypedResultSet.create(rows.result);
+
+                    for (UntypedResultSet.Row r : result)
+                    {
+                        if (source.equals(r.getInetAddress("source")))
+                            continue;
+                        if ((uuid = r.getUUID("event_id")).timestamp() > (tcur - 1000) * 10000)
+                            seen[si].add(uuid);
+                        if (seen[si == 0 ? 1 : 0].contains(uuid))
+                            continue;
+                        String message = String.format("%s: %s", r.getInetAddress("source"), r.getString("activity"));
+                        fireProgressEvent("repair:" + cmd,
+                                          new ProgressEvent(ProgressEventType.NOTIFICATION, 0, 0, message));
+                    }
+                    tlast = tcur;
+
+                    si = si == 0 ? 1 : 0;
+                    seen[si].clear();
+                }
+            }
+        });
+    }
+}
diff --git a/src/java/org/apache/cassandra/repair/RepairSession.java b/src/java/org/apache/cassandra/repair/RepairSession.java
index 0580ebb..70bfaa6 100644
--- a/src/java/org/apache/cassandra/repair/RepairSession.java
+++ b/src/java/org/apache/cassandra/repair/RepairSession.java
@@ -21,42 +21,43 @@
 import java.net.InetAddress;
 import java.util.*;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.Executors;
+import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.locks.Condition;
 
-import com.google.common.util.concurrent.ListeningExecutorService;
-import com.google.common.util.concurrent.MoreExecutors;
+import com.google.common.collect.Lists;
+import com.google.common.util.concurrent.*;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.concurrent.NamedThreadFactory;
+import org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.exceptions.RepairException;
 import org.apache.cassandra.gms.*;
-import org.apache.cassandra.service.ActiveRepairService;
-import org.apache.cassandra.utils.*;
-import org.apache.cassandra.utils.concurrent.SimpleCondition;
+import org.apache.cassandra.tracing.Tracing;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.MerkleTree;
+import org.apache.cassandra.utils.Pair;
 
 /**
  * Coordinates the (active) repair of a token range.
  *
  * A given RepairSession repairs a set of replicas for a given range on a list
  * of column families. For each of the column family to repair, RepairSession
- * creates a RepairJob that handles the repair of that CF.
+ * creates a {@link RepairJob} that handles the repair of that CF.
  *
  * A given RepairJob has the 2 main phases:
- *   1. Validation phase: the job requests merkle trees from each of the replica involves
- *      (RepairJob.sendTreeRequests()) and waits until all trees are received (in
+ * <ol>
+ *   <li>Validation phase: the job requests merkle trees from each of the replica involves
+ *      ({@link org.apache.cassandra.repair.ValidationTask}) and waits until all trees are received (in
  *      validationComplete()).
- *   2. Synchonization phase: once all trees are received, the job compares each tree with
- *      all the other using a so-called Differencer (started by submitDifferencers()). If
- *      differences there is between 2 trees, the concerned Differencer will start a streaming
- *      of the difference between the 2 endpoint concerned (Differencer.performStreamingRepair).
- * The job is done once all its Differencer are done (i.e. have either computed no differences
+ *   </li>
+ *   <li>Synchronization phase: once all trees are received, the job compares each tree with
+ *      all the other using a so-called {@link SyncTask}. If there is difference between 2 trees, the
+ *      concerned SyncTask will start a streaming of the difference between the 2 endpoint concerned.
+ *   </li>
+ * </ol>
+ * The job is done once all its SyncTasks are done (i.e. have either computed no differences
  * or the streaming they started is done (syncComplete())).
  *
  * A given session will execute the first phase (validation phase) of each of it's job
@@ -71,15 +72,15 @@
  * we still first send a message to each node to flush and snapshot data so each merkle tree
  * creation is still done on similar data, even if the actual creation is not
  * done simulatneously). If not sequential, all merkle tree are requested in parallel.
- * Similarly, if a job is sequential, it will handle one Differencer at a time, but will handle
+ * Similarly, if a job is sequential, it will handle one SyncTask at a time, but will handle
  * all of them in parallel otherwise.
  */
-public class RepairSession extends WrappedRunnable implements IEndpointStateChangeSubscriber,
-                                                              IFailureDetectionEventListener,
-                                                              IRepairJobEventListener
+public class RepairSession extends AbstractFuture<RepairSessionResult> implements IEndpointStateChangeSubscriber,
+                                                                                 IFailureDetectionEventListener
 {
     private static Logger logger = LoggerFactory.getLogger(RepairSession.class);
 
+    public final UUID parentRepairSession;
     /** Repair session ID */
     private final UUID id;
     public final String keyspace;
@@ -88,51 +89,51 @@
     /** Range to repair */
     public final Range<Token> range;
     public final Set<InetAddress> endpoints;
+    private final long repairedAt;
 
-    private volatile Exception exception;
     private final AtomicBoolean isFailed = new AtomicBoolean(false);
-    private final AtomicBoolean fdUnregistered = new AtomicBoolean(false);
 
-    // First, all RepairJobs are added to this queue,
-    final Queue<RepairJob> jobs = new ConcurrentLinkedQueue<>();
-
-    // and after receiving all validation, the job is moved to
-    // this map, keyed by CF name.
-    final Map<String, RepairJob> syncingJobs = new ConcurrentHashMap<>();
+    // Each validation task waits response from replica in validating ConcurrentMap (keyed by CF name and endpoint address)
+    private final ConcurrentMap<Pair<RepairJobDesc, InetAddress>, ValidationTask> validating = new ConcurrentHashMap<>();
+    // Remote syncing jobs wait response in syncingTasks map
+    private final ConcurrentMap<Pair<RepairJobDesc, NodePair>, RemoteSyncTask> syncingTasks = new ConcurrentHashMap<>();
 
     // Tasks(snapshot, validate request, differencing, ...) are run on taskExecutor
-    private final ListeningExecutorService taskExecutor = MoreExecutors.listeningDecorator(Executors.newCachedThreadPool(new NamedThreadFactory("RepairJobTask")));
-
-    private final SimpleCondition completed = new SimpleCondition();
-    public final Condition differencingDone = new SimpleCondition();
-    public final UUID parentRepairSession;
+    private final ListeningExecutorService taskExecutor = MoreExecutors.listeningDecorator(DebuggableThreadPoolExecutor.createCachedThreadpoolWithMaxSize("RepairJobTask"));
 
     private volatile boolean terminated = false;
 
     /**
      * Create new repair session.
      *
+     * @param parentRepairSession the parent sessions id
+     * @param id this sessions id
      * @param range range to repair
      * @param keyspace name of keyspace
      * @param parallelismDegree specifies the degree of parallelism when calculating the merkle trees
      * @param endpoints the data centers that should be part of the repair; null for all DCs
+     * @param repairedAt when the repair occurred (millis)
      * @param cfnames names of columnfamilies
      */
-    public RepairSession(UUID parentRepairSession, Range<Token> range, String keyspace, RepairParallelism parallelismDegree, Set<InetAddress> endpoints, String... cfnames)
+    public RepairSession(UUID parentRepairSession,
+                         UUID id,
+                         Range<Token> range,
+                         String keyspace,
+                         RepairParallelism parallelismDegree,
+                         Set<InetAddress> endpoints,
+                         long repairedAt,
+                         String... cfnames)
     {
-        this(parentRepairSession, UUIDGen.getTimeUUID(), range, keyspace, parallelismDegree, endpoints, cfnames);
-    }
+        assert cfnames.length > 0 : "Repairing no column families seems pointless, doesn't it";
 
-    public RepairSession(UUID parentRepairSession, UUID id, Range<Token> range, String keyspace, RepairParallelism parallelismDegree, Set<InetAddress> endpoints, String[] cfnames)
-    {
         this.parentRepairSession = parentRepairSession;
         this.id = id;
         this.parallelismDegree = parallelismDegree;
         this.keyspace = keyspace;
         this.cfnames = cfnames;
-        assert cfnames.length > 0 : "Repairing no column families seems pointless, doesn't it";
         this.range = range;
         this.endpoints = endpoints;
+        this.repairedAt = repairedAt;
     }
 
     public UUID getId()
@@ -145,6 +146,16 @@
         return range;
     }
 
+    public void waitForValidation(Pair<RepairJobDesc, InetAddress> key, ValidationTask task)
+    {
+        validating.put(key, task);
+    }
+
+    public void waitForSync(Pair<RepairJobDesc, NodePair> key, RemoteSyncTask task)
+    {
+        syncingTasks.put(key, task);
+    }
+
     /**
      * Receive merkle tree response or failed response from {@code endpoint} for current repair job.
      *
@@ -154,52 +165,17 @@
      */
     public void validationComplete(RepairJobDesc desc, InetAddress endpoint, MerkleTree tree)
     {
-        RepairJob job = jobs.peek();
-        if (job == null)
+        ValidationTask task = validating.remove(Pair.create(desc, endpoint));
+        if (task == null)
         {
             assert terminated;
             return;
         }
 
-        if (tree == null)
-        {
-            exception = new RepairException(desc, "Validation failed in " + endpoint);
-            forceShutdown();
-            return;
-        }
-
-        logger.info(String.format("[repair #%s] Received merkle tree for %s from %s", getId(), desc.columnFamily, endpoint));
-
-        assert job.desc.equals(desc);
-        if (job.addTree(endpoint, tree) == 0)
-        {
-            logger.debug("All responses received for {}/{}", getId(), desc.columnFamily);
-            if (!job.isFailed())
-            {
-                syncingJobs.put(job.desc.columnFamily, job);
-                job.submitDifferencers();
-            }
-
-            // This job is complete, switching to next in line (note that only one thread will ever do this)
-            jobs.poll();
-            RepairJob nextJob = jobs.peek();
-            if (nextJob == null)
-            {
-                // Unregister from FailureDetector once we've completed synchronizing Merkle trees.
-                // After this point, we rely on tcp_keepalive for individual sockets to notify us when a connection is down.
-                // See CASSANDRA-3569
-                if (fdUnregistered.compareAndSet(false, true))
-                    FailureDetector.instance.unregisterFailureDetectionEventListener(this);
-
-                // We are done with this repair session as far as differencing
-                // is considered. Just inform the session
-                differencingDone.signalAll();
-            }
-            else
-            {
-                nextJob.sendTreeRequests(endpoints);
-            }
-        }
+        String message = String.format("Received merkle tree for %s from %s", desc.columnFamily, endpoint);
+        logger.info("[repair #{}] {}", getId(), message);
+        Tracing.traceRepair(message);
+        task.treeReceived(tree);
     }
 
     /**
@@ -211,38 +187,15 @@
      */
     public void syncComplete(RepairJobDesc desc, NodePair nodes, boolean success)
     {
-        RepairJob job = syncingJobs.get(desc.columnFamily);
-        if (job == null)
+        RemoteSyncTask task = syncingTasks.get(Pair.create(desc, nodes));
+        if (task == null)
         {
             assert terminated;
             return;
         }
 
-        if (!success)
-        {
-            exception = new RepairException(desc, String.format("Sync failed between %s and %s", nodes.endpoint1, nodes.endpoint2));
-            forceShutdown();
-            return;
-        }
-
         logger.debug(String.format("[repair #%s] Repair completed between %s and %s on %s", getId(), nodes.endpoint1, nodes.endpoint2, desc.columnFamily));
-
-        if (job.completedSynchronization())
-        {
-            RepairJob completedJob = syncingJobs.remove(job.desc.columnFamily);
-            String remaining = syncingJobs.size() == 0 ? "" : String.format(" (%d remaining column family to sync for this session)", syncingJobs.size());
-            if (completedJob != null && completedJob.isFailed())
-                logger.warn(String.format("[repair #%s] %s sync failed%s", getId(), desc.columnFamily, remaining));
-            else
-                logger.info(String.format("[repair #%s] %s is fully synced%s", getId(), desc.columnFamily, remaining));
-
-            if (jobs.isEmpty() && syncingJobs.isEmpty())
-            {
-                taskExecutor.shutdown();
-                // this repair session is completed
-                completed.signalAll();
-            }
-        }
+        task.syncComplete(success);
     }
 
     private String repairedNodes()
@@ -254,15 +207,30 @@
         return sb.toString();
     }
 
-    // we don't care about the return value but care about it throwing exception
-    public void runMayThrow() throws Exception
+    /**
+     * Start RepairJob on given ColumnFamilies.
+     *
+     * This first validates if all replica are available, and if they are,
+     * creates RepairJobs and submit to run on given executor.
+     *
+     * @param executor Executor to run validation
+     */
+    public void start(ListeningExecutorService executor)
     {
+        String message;
+        if (terminated)
+            return;
+
         logger.info(String.format("[repair #%s] new session: will sync %s on range %s for %s.%s", getId(), repairedNodes(), range, keyspace, Arrays.toString(cfnames)));
+        Tracing.traceRepair("Syncing range {}", range);
+        SystemDistributedKeyspace.startRepairs(getId(), parentRepairSession, keyspace, cfnames, range, endpoints);
 
         if (endpoints.isEmpty())
         {
-            differencingDone.signalAll();
-            logger.info(String.format("[repair #%s] No neighbors to repair with on range %s: session completed", getId(), range));
+            logger.info("[repair #{}] {}", getId(), message = String.format("No neighbors to repair with on range %s: session completed", range));
+            Tracing.traceRepair(message);
+            set(new RepairSessionResult(id, keyspace, range, Lists.<RepairResult>newArrayList()));
+            SystemDistributedKeyspace.failRepairs(getId(), keyspace, cfnames, new RuntimeException(message));
             return;
         }
 
@@ -271,86 +239,65 @@
         {
             if (!FailureDetector.instance.isAlive(endpoint))
             {
-                String message = String.format("Cannot proceed on repair because a neighbor (%s) is dead: session failed", endpoint);
-                differencingDone.signalAll();
+                message = String.format("Cannot proceed on repair because a neighbor (%s) is dead: session failed", endpoint);
                 logger.error("[repair #{}] {}", getId(), message);
-                throw new IOException(message);
+                Exception e = new IOException(message);
+                setException(e);
+                SystemDistributedKeyspace.failRepairs(getId(), keyspace, cfnames, e);
+                return;
             }
         }
 
-        ActiveRepairService.instance.addToActiveSessions(this);
-        try
+        // Create and submit RepairJob for each ColumnFamily
+        List<ListenableFuture<RepairResult>> jobs = new ArrayList<>(cfnames.length);
+        for (String cfname : cfnames)
         {
-            // Create and queue a RepairJob for each column family
-            for (String cfname : cfnames)
-            {
-                RepairJob job = new RepairJob(this, parentRepairSession, id, keyspace, cfname, range, parallelismDegree, taskExecutor);
-                jobs.offer(job);
-            }
-            logger.debug("Sending tree requests to endpoints {}", endpoints);
-            jobs.peek().sendTreeRequests(endpoints);
-
-            // block whatever thread started this session until all requests have been returned:
-            // if this thread dies, the session will still complete in the background
-            completed.await();
-
-            if (exception == null)
-            {
-                logger.info(String.format("[repair #%s] session completed successfully", getId()));
-            }
-            else
-            {
-                logger.error(String.format("[repair #%s] session completed with the following error", getId()), exception);
-                throw exception;
-            }
+            RepairJob job = new RepairJob(this, cfname, parallelismDegree, repairedAt, taskExecutor);
+            executor.execute(job);
+            jobs.add(job);
         }
-        catch (InterruptedException e)
+
+        // When all RepairJobs are done without error, cleanup and set the final result
+        Futures.addCallback(Futures.allAsList(jobs), new FutureCallback<List<RepairResult>>()
         {
-            throw new RuntimeException("Interrupted while waiting for repair.");
-        }
-        finally
-        {
-            // mark this session as terminated
-            terminate();
+            public void onSuccess(List<RepairResult> results)
+            {
+                // this repair session is completed
+                logger.info("[repair #{}] {}", getId(), "Session completed successfully");
+                Tracing.traceRepair("Completed sync of range {}", range);
+                set(new RepairSessionResult(id, keyspace, range, results));
 
-            ActiveRepairService.instance.removeFromActiveSessions(this);
+                taskExecutor.shutdown();
+                // mark this session as terminated
+                terminate();
+            }
 
-            // If we've reached here in an exception state without completing Merkle Tree sync, we'll still be registered
-            // with the FailureDetector.
-            if (fdUnregistered.compareAndSet(false, true))
-                FailureDetector.instance.unregisterFailureDetectionEventListener(this);
-        }
+            public void onFailure(Throwable t)
+            {
+                logger.error(String.format("[repair #%s] Session completed with the following error", getId()), t);
+                Tracing.traceRepair("Session completed with the following error: {}", t);
+                forceShutdown(t);
+            }
+        });
     }
 
     public void terminate()
     {
         terminated = true;
-        jobs.clear();
-        syncingJobs.clear();
+        validating.clear();
+        syncingTasks.clear();
     }
 
     /**
      * clear all RepairJobs and terminate this session.
+     *
+     * @param reason Cause of error for shutdown
      */
-    public void forceShutdown()
+    public void forceShutdown(Throwable reason)
     {
+        setException(reason);
         taskExecutor.shutdownNow();
-        differencingDone.signalAll();
-        completed.signalAll();
-    }
-
-    public void failedSnapshot()
-    {
-        exception = new IOException("Failed during snapshot creation.");
-        forceShutdown();
-    }
-
-    void failedNode(InetAddress remote)
-    {
-        String errorMsg = String.format("Endpoint %s died", remote);
-        exception = new IOException(errorMsg);
-        // If a node failed during Merkle creation, we stop everything (though there could still be some activity in the background)
-        forceShutdown();
+        terminate();
     }
 
     public void onJoin(InetAddress endpoint, EndpointState epState) {}
@@ -383,6 +330,9 @@
         if (!isFailed.compareAndSet(false, true))
             return;
 
-        failedNode(endpoint);
+        Exception exception = new IOException(String.format("Endpoint %s died", endpoint));
+        logger.error(String.format("[repair #%s] session completed with the following error", getId()), exception);
+        // If a node failed, we stop everything (though there could still be some activity in the background)
+        forceShutdown(exception);
     }
 }
diff --git a/src/java/org/apache/cassandra/repair/RepairSessionResult.java b/src/java/org/apache/cassandra/repair/RepairSessionResult.java
new file mode 100644
index 0000000..4551608
--- /dev/null
+++ b/src/java/org/apache/cassandra/repair/RepairSessionResult.java
@@ -0,0 +1,43 @@
+/*
+ * 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.repair;
+
+import java.util.Collection;
+import java.util.UUID;
+
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+
+/**
+ * Repair session result
+ */
+public class RepairSessionResult
+{
+    public final UUID sessionId;
+    public final String keyspace;
+    public final Range<Token> range;
+    public final Collection<RepairResult> repairJobResults;
+
+    public RepairSessionResult(UUID sessionId, String keyspace, Range<Token> range, Collection<RepairResult> repairJobResults)
+    {
+        this.sessionId = sessionId;
+        this.keyspace = keyspace;
+        this.range = range;
+        this.repairJobResults = repairJobResults;
+    }
+}
diff --git a/src/java/org/apache/cassandra/repair/SequentialRequestCoordinator.java b/src/java/org/apache/cassandra/repair/SequentialRequestCoordinator.java
deleted file mode 100644
index 6bb5de9..0000000
--- a/src/java/org/apache/cassandra/repair/SequentialRequestCoordinator.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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.repair;
-
-import java.util.LinkedList;
-import java.util.Queue;
-
-public class SequentialRequestCoordinator<R> implements IRequestCoordinator<R>
-{
-    private final Queue<R> requests = new LinkedList<>();
-    private final IRequestProcessor<R> processor;
-
-    public SequentialRequestCoordinator(IRequestProcessor<R> processor)
-    {
-        this.processor = processor;
-    }
-
-    @Override
-    public void add(R request)
-    {
-        requests.add(request);
-    }
-
-    @Override
-    public void start()
-    {
-        if (requests.isEmpty())
-            return;
-
-        processor.process(requests.peek());
-    }
-
-    @Override
-    public int completed(R request)
-    {
-        assert request.equals(requests.peek());
-        requests.poll();
-        int remaining = requests.size();
-        if (remaining != 0)
-            processor.process(requests.peek());
-        return remaining;
-    }
-}
diff --git a/src/java/org/apache/cassandra/repair/SnapshotTask.java b/src/java/org/apache/cassandra/repair/SnapshotTask.java
index 53e048d..94361d8 100644
--- a/src/java/org/apache/cassandra/repair/SnapshotTask.java
+++ b/src/java/org/apache/cassandra/repair/SnapshotTask.java
@@ -75,6 +75,7 @@
 
         public void onFailure(InetAddress from)
         {
+            //listener.failedSnapshot();
             task.setException(new RuntimeException("Could not create snapshot at " + from));
         }
     }
diff --git a/src/java/org/apache/cassandra/repair/StreamingRepairTask.java b/src/java/org/apache/cassandra/repair/StreamingRepairTask.java
index 25ec698..25ef06e 100644
--- a/src/java/org/apache/cassandra/repair/StreamingRepairTask.java
+++ b/src/java/org/apache/cassandra/repair/StreamingRepairTask.java
@@ -27,59 +27,48 @@
 import org.apache.cassandra.repair.messages.SyncComplete;
 import org.apache.cassandra.repair.messages.SyncRequest;
 import org.apache.cassandra.service.ActiveRepairService;
-import org.apache.cassandra.streaming.*;
-import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.streaming.StreamEvent;
+import org.apache.cassandra.streaming.StreamEventHandler;
+import org.apache.cassandra.streaming.StreamPlan;
+import org.apache.cassandra.streaming.StreamState;
 
 /**
- * Task that make two nodes exchange (stream) some ranges (for a given table/cf).
- * This handle the case where the local node is neither of the two nodes that
- * must stream their range, and allow to register a callback to be called on
- * completion.
+ * StreamingRepairTask performs data streaming between two remote replica which neither is not repair coordinator.
+ * Task will send {@link SyncComplete} message back to coordinator upon streaming completion.
  */
 public class StreamingRepairTask implements Runnable, StreamEventHandler
 {
     private static final Logger logger = LoggerFactory.getLogger(StreamingRepairTask.class);
 
-    /** Repair session ID that this streaming task belongs */
-    public final RepairJobDesc desc;
-    public final SyncRequest request;
+    private final RepairJobDesc desc;
+    private final SyncRequest request;
+    private final long repairedAt;
 
-    public StreamingRepairTask(RepairJobDesc desc, SyncRequest request)
+    public StreamingRepairTask(RepairJobDesc desc, SyncRequest request, long repairedAt)
     {
         this.desc = desc;
         this.request = request;
+        this.repairedAt = repairedAt;
     }
 
     public void run()
     {
-        if (request.src.equals(FBUtilities.getBroadcastAddress()))
-            initiateStreaming();
-        else
-            forwardToSource();
-    }
-
-    private void initiateStreaming()
-    {
-        long repairedAt = ActiveRepairService.UNREPAIRED_SSTABLE;
         InetAddress dest = request.dst;
         InetAddress preferred = SystemKeyspace.getPreferredIP(dest);
-        if (desc.parentSessionId != null && ActiveRepairService.instance.getParentRepairSession(desc.parentSessionId) != null)
-            repairedAt = ActiveRepairService.instance.getParentRepairSession(desc.parentSessionId).repairedAt;
         logger.info(String.format("[streaming task #%s] Performing streaming repair of %d ranges with %s", desc.sessionId, request.ranges.size(), request.dst));
-        StreamResultFuture op = new StreamPlan("Repair", repairedAt, 1)
-                                    .flushBeforeTransfer(true)
-                                    // request ranges from the remote node
-                                    .requestRanges(dest, preferred, desc.keyspace, request.ranges, desc.columnFamily)
-                                    // send ranges to the remote node
-                                    .transferRanges(dest, preferred, desc.keyspace, request.ranges, desc.columnFamily)
-                                    .execute();
-        op.addEventListener(this);
-    }
-
-    private void forwardToSource()
-    {
-        logger.info(String.format("[repair #%s] Forwarding streaming repair of %d ranges to %s (to be streamed with %s)", desc.sessionId, request.ranges.size(), request.src, request.dst));
-        MessagingService.instance().sendOneWay(request.createMessage(), request.src);
+        boolean isIncremental = false;
+        if (desc.parentSessionId != null)
+        {
+            ActiveRepairService.ParentRepairSession prs = ActiveRepairService.instance.getParentRepairSession(desc.parentSessionId);
+            isIncremental = prs.isIncremental;
+        }
+        new StreamPlan("Repair", repairedAt, 1, false, isIncremental).listeners(this)
+                                            .flushBeforeTransfer(true)
+                                            // request ranges from the remote node
+                                            .requestRanges(dest, preferred, desc.keyspace, request.ranges, desc.columnFamily)
+                                            // send ranges to the remote node
+                                            .transferRanges(dest, preferred, desc.keyspace, request.ranges, desc.columnFamily)
+                                            .execute();
     }
 
     public void handleStreamEvent(StreamEvent event)
@@ -89,7 +78,7 @@
     }
 
     /**
-     * If we succeeded on both stream in and out, reply back to the initiator.
+     * If we succeeded on both stream in and out, reply back to coordinator
      */
     public void onSuccess(StreamState state)
     {
@@ -98,7 +87,7 @@
     }
 
     /**
-     * If we failed on either stream in or out, reply fail to the initiator.
+     * If we failed on either stream in or out, reply fail to coordinator
      */
     public void onFailure(Throwable t)
     {
diff --git a/src/java/org/apache/cassandra/repair/SyncStat.java b/src/java/org/apache/cassandra/repair/SyncStat.java
new file mode 100644
index 0000000..5721a20
--- /dev/null
+++ b/src/java/org/apache/cassandra/repair/SyncStat.java
@@ -0,0 +1,33 @@
+/*
+ * 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.repair;
+
+/**
+ * Statistics about synchronizing two replica
+ */
+public class SyncStat
+{
+    public final NodePair nodes;
+    public final long numberOfDifferences;
+
+    public SyncStat(NodePair nodes, long numberOfDifferences)
+    {
+        this.nodes = nodes;
+        this.numberOfDifferences = numberOfDifferences;
+    }
+}
diff --git a/src/java/org/apache/cassandra/repair/SyncTask.java b/src/java/org/apache/cassandra/repair/SyncTask.java
new file mode 100644
index 0000000..7350a66
--- /dev/null
+++ b/src/java/org/apache/cassandra/repair/SyncTask.java
@@ -0,0 +1,86 @@
+/*
+ * 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.repair;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import com.google.common.util.concurrent.AbstractFuture;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.tracing.Tracing;
+import org.apache.cassandra.utils.MerkleTree;
+
+/**
+ * SyncTask will calculate the difference of MerkleTree between two nodes
+ * and perform necessary operation to repair replica.
+ */
+public abstract class SyncTask extends AbstractFuture<SyncStat> implements Runnable
+{
+    private static Logger logger = LoggerFactory.getLogger(SyncTask.class);
+
+    protected final RepairJobDesc desc;
+    protected final TreeResponse r1;
+    protected final TreeResponse r2;
+
+    protected volatile SyncStat stat;
+
+    public SyncTask(RepairJobDesc desc, TreeResponse r1, TreeResponse r2)
+    {
+        this.desc = desc;
+        this.r1 = r1;
+        this.r2 = r2;
+    }
+
+    /**
+     * Compares trees, and triggers repairs for any ranges that mismatch.
+     */
+    public void run()
+    {
+        // compare trees, and collect differences
+        List<Range<Token>> differences = new ArrayList<>();
+        differences.addAll(MerkleTree.difference(r1.tree, r2.tree));
+
+        stat = new SyncStat(new NodePair(r1.endpoint, r2.endpoint), differences.size());
+
+        // choose a repair method based on the significance of the difference
+        String format = String.format("[repair #%s] Endpoints %s and %s %%s for %s", desc.sessionId, r1.endpoint, r2.endpoint, desc.columnFamily);
+        if (differences.isEmpty())
+        {
+            logger.info(String.format(format, "are consistent"));
+            Tracing.traceRepair("Endpoint {} is consistent with {} for {}", r1.endpoint, r2.endpoint, desc.columnFamily);
+            set(stat);
+            return;
+        }
+
+        // non-0 difference: perform streaming repair
+        logger.info(String.format(format, "have " + differences.size() + " range(s) out of sync"));
+        Tracing.traceRepair("Endpoint {} has {} range(s) out of sync with {} for {}", r1.endpoint, differences.size(), r2.endpoint, desc.columnFamily);
+        startSync(differences);
+    }
+
+    public SyncStat getCurrentStat()
+    {
+        return stat;
+    }
+
+    protected abstract void startSync(List<Range<Token>> differences);
+}
diff --git a/src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java b/src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java
new file mode 100644
index 0000000..2509597
--- /dev/null
+++ b/src/java/org/apache/cassandra/repair/SystemDistributedKeyspace.java
@@ -0,0 +1,212 @@
+/*
+ * 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.repair;
+
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.net.InetAddress;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Sets;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+
+public final class SystemDistributedKeyspace
+{
+    private static Logger logger = LoggerFactory.getLogger(SystemDistributedKeyspace.class);
+
+    public static final String NAME = "system_distributed";
+
+    public static final String REPAIR_HISTORY = "repair_history";
+
+    public static final String PARENT_REPAIR_HISTORY = "parent_repair_history";
+
+    private static final CFMetaData RepairHistory =
+        compile(REPAIR_HISTORY,
+                "Repair history",
+                "CREATE TABLE %s ("
+                     + "keyspace_name text,"
+                     + "columnfamily_name text,"
+                     + "id timeuuid,"
+                     + "parent_id timeuuid,"
+                     + "range_begin text,"
+                     + "range_end text,"
+                     + "coordinator inet,"
+                     + "participants set<inet>,"
+                     + "exception_message text,"
+                     + "exception_stacktrace text,"
+                     + "status text,"
+                     + "started_at timestamp,"
+                     + "finished_at timestamp,"
+                     + "PRIMARY KEY ((keyspace_name, columnfamily_name), id))");
+
+    private static final CFMetaData ParentRepairHistory =
+        compile(PARENT_REPAIR_HISTORY,
+                "Repair history",
+                "CREATE TABLE %s ("
+                     + "parent_id timeuuid,"
+                     + "keyspace_name text,"
+                     + "columnfamily_names set<text>,"
+                     + "started_at timestamp,"
+                     + "finished_at timestamp,"
+                     + "exception_message text,"
+                     + "exception_stacktrace text,"
+                     + "requested_ranges set<text>,"
+                     + "successful_ranges set<text>,"
+                     + "PRIMARY KEY (parent_id))");
+
+    private static CFMetaData compile(String name, String description, String schema)
+    {
+        return CFMetaData.compile(String.format(schema, name), NAME)
+                         .comment(description);
+    }
+
+    public static KSMetaData definition()
+    {
+        List<CFMetaData> tables = Arrays.asList(RepairHistory, ParentRepairHistory);
+        return new KSMetaData(NAME, SimpleStrategy.class, ImmutableMap.of("replication_factor", "3"), true, tables);
+    }
+
+    public static void startParentRepair(UUID parent_id, String keyspaceName, String[] cfnames, Collection<Range<Token>> ranges)
+    {
+
+        String query = "INSERT INTO %s.%s (parent_id, keyspace_name, columnfamily_names, requested_ranges, started_at)"+
+                                 " VALUES (%s,        '%s',          { '%s' },           { '%s' },          toTimestamp(now()))";
+        String fmtQry = String.format(query, NAME, PARENT_REPAIR_HISTORY, parent_id.toString(), keyspaceName, Joiner.on("','").join(cfnames), Joiner.on("','").join(ranges));
+        processSilent(fmtQry);
+    }
+
+    public static void failParentRepair(UUID parent_id, Throwable t)
+    {
+        String query = "UPDATE %s.%s SET finished_at = toTimestamp(now()), exception_message=?, exception_stacktrace=? WHERE parent_id=%s";
+
+        StringWriter sw = new StringWriter();
+        PrintWriter pw = new PrintWriter(sw);
+        t.printStackTrace(pw);
+        String fmtQuery = String.format(query, NAME, PARENT_REPAIR_HISTORY, parent_id.toString());
+        processSilent(fmtQuery, t.getMessage(), sw.toString());
+    }
+
+    public static void successfulParentRepair(UUID parent_id, Collection<Range<Token>> successfulRanges)
+    {
+        String query = "UPDATE %s.%s SET finished_at = toTimestamp(now()), successful_ranges = {'%s'} WHERE parent_id=%s";
+        String fmtQuery = String.format(query, NAME, PARENT_REPAIR_HISTORY, Joiner.on("','").join(successfulRanges), parent_id.toString());
+        processSilent(fmtQuery);
+    }
+
+    public static void startRepairs(UUID id, UUID parent_id, String keyspaceName, String[] cfnames, Range<Token> range, Iterable<InetAddress> endpoints)
+    {
+        String coordinator = FBUtilities.getBroadcastAddress().getHostAddress();
+        Set<String> participants = Sets.newHashSet(coordinator);
+
+        for (InetAddress endpoint : endpoints)
+            participants.add(endpoint.getHostAddress());
+
+        String query =
+                "INSERT INTO %s.%s (keyspace_name, columnfamily_name, id, parent_id, range_begin, range_end, coordinator, participants, status, started_at) " +
+                        "VALUES (   '%s',          '%s',              %s, %s,        '%s',        '%s',      '%s',        { '%s' },     '%s',   toTimestamp(now()))";
+
+        for (String cfname : cfnames)
+        {
+            String fmtQry = String.format(query, NAME, REPAIR_HISTORY,
+                                          keyspaceName,
+                                          cfname,
+                                          id.toString(),
+                                          parent_id.toString(),
+                                          range.left.toString(),
+                                          range.right.toString(),
+                                          coordinator,
+                                          Joiner.on("', '").join(participants),
+                    RepairState.STARTED.toString());
+            processSilent(fmtQry);
+        }
+    }
+
+    public static void failRepairs(UUID id, String keyspaceName, String[] cfnames, Throwable t)
+    {
+        for (String cfname : cfnames)
+            failedRepairJob(id, keyspaceName, cfname, t);
+    }
+
+    public static void successfulRepairJob(UUID id, String keyspaceName, String cfname)
+    {
+        String query = "UPDATE %s.%s SET status = '%s', finished_at = toTimestamp(now()) WHERE keyspace_name = '%s' AND columnfamily_name = '%s' AND id = %s";
+        String fmtQuery = String.format(query, NAME, REPAIR_HISTORY,
+                                        RepairState.SUCCESS.toString(),
+                                        keyspaceName,
+                                        cfname,
+                                        id.toString());
+        processSilent(fmtQuery);
+    }
+
+    public static void failedRepairJob(UUID id, String keyspaceName, String cfname, Throwable t)
+    {
+        String query = "UPDATE %s.%s SET status = '%s', finished_at = toTimestamp(now()), exception_message=?, exception_stacktrace=? WHERE keyspace_name = '%s' AND columnfamily_name = '%s' AND id = %s";
+        StringWriter sw = new StringWriter();
+        PrintWriter pw = new PrintWriter(sw);
+        t.printStackTrace(pw);
+        String fmtQry = String.format(query, NAME, REPAIR_HISTORY,
+                RepairState.FAILED.toString(),
+                keyspaceName,
+                cfname,
+                id.toString());
+        processSilent(fmtQry, t.getMessage(), sw.toString());
+    }
+
+    private static void processSilent(String fmtQry, String... values)
+    {
+        try
+        {
+            List<ByteBuffer> valueList = new ArrayList<>();
+            for (String v : values)
+            {
+                valueList.add(ByteBufferUtil.bytes(v));
+            }
+            QueryProcessor.process(fmtQry, ConsistencyLevel.ONE, valueList);
+        }
+        catch (Throwable t)
+        {
+            logger.error("Error executing query "+fmtQry, t);
+        }
+    }
+
+
+    private enum RepairState
+    {
+        STARTED, SUCCESS, FAILED
+    }
+}
diff --git a/src/java/org/apache/cassandra/repair/ValidationTask.java b/src/java/org/apache/cassandra/repair/ValidationTask.java
new file mode 100644
index 0000000..a52ec4f
--- /dev/null
+++ b/src/java/org/apache/cassandra/repair/ValidationTask.java
@@ -0,0 +1,71 @@
+/*
+ * 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.repair;
+
+import java.net.InetAddress;
+
+import com.google.common.util.concurrent.AbstractFuture;
+
+import org.apache.cassandra.exceptions.RepairException;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.repair.messages.ValidationRequest;
+import org.apache.cassandra.utils.MerkleTree;
+
+/**
+ * ValidationTask sends {@link ValidationRequest} to a replica.
+ * When a replica sends back message, task completes.
+ */
+public class ValidationTask extends AbstractFuture<TreeResponse> implements Runnable
+{
+    private final RepairJobDesc desc;
+    private final InetAddress endpoint;
+    private final int gcBefore;
+
+    public ValidationTask(RepairJobDesc desc, InetAddress endpoint, int gcBefore)
+    {
+        this.desc = desc;
+        this.endpoint = endpoint;
+        this.gcBefore = gcBefore;
+    }
+
+    /**
+     * Send ValidationRequest to replica
+     */
+    public void run()
+    {
+        ValidationRequest request = new ValidationRequest(desc, gcBefore);
+        MessagingService.instance().sendOneWay(request.createMessage(), endpoint);
+    }
+
+    /**
+     * Receive MerkleTree from replica node.
+     *
+     * @param tree MerkleTree that is sent from replica. Null if validation failed on replica node.
+     */
+    public void treeReceived(MerkleTree tree)
+    {
+        if (tree == null)
+        {
+            setException(new RepairException(desc, "Validation failed in " + endpoint));
+        }
+        else
+        {
+            set(new TreeResponse(endpoint, tree));
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/repair/Validator.java b/src/java/org/apache/cassandra/repair/Validator.java
index 8f8bf6d..8dbb4cf 100644
--- a/src/java/org/apache/cassandra/repair/Validator.java
+++ b/src/java/org/apache/cassandra/repair/Validator.java
@@ -34,6 +34,7 @@
 import org.apache.cassandra.db.compaction.AbstractCompactedRow;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.repair.messages.ValidationComplete;
+import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.MerkleTree;
 import org.apache.cassandra.utils.MerkleTree.RowHash;
@@ -53,6 +54,7 @@
     public final RepairJobDesc desc;
     public final InetAddress initiator;
     public final int gcBefore;
+    private final boolean evenTreeDistribution;
 
     // null when all rows with the min token have been consumed
     private long validated;
@@ -66,19 +68,25 @@
 
     public Validator(RepairJobDesc desc, InetAddress initiator, int gcBefore)
     {
+        this(desc, initiator, gcBefore, false);
+    }
+
+    public Validator(RepairJobDesc desc, InetAddress initiator, int gcBefore, boolean evenTreeDistribution)
+    {
         this.desc = desc;
         this.initiator = initiator;
         this.gcBefore = gcBefore;
         validated = 0;
         range = null;
         ranges = null;
+        this.evenTreeDistribution = evenTreeDistribution;
     }
 
     public void prepare(ColumnFamilyStore cfs, MerkleTree tree)
     {
         this.tree = tree;
 
-        if (!tree.partitioner().preservesOrder())
+        if (!tree.partitioner().preservesOrder() || evenTreeDistribution)
         {
             // You can't beat an even tree distribution for md5
             tree.init();
@@ -254,7 +262,10 @@
     {
         // respond to the request that triggered this validation
         if (!initiator.equals(FBUtilities.getBroadcastAddress()))
-            logger.info(String.format("[repair #%s] Sending completed merkle tree to %s for %s/%s", desc.sessionId, initiator, desc.keyspace, desc.columnFamily));
+        {
+            logger.info(String.format("[repair #%s] Sending completed merkle tree to %s for %s.%s", desc.sessionId, initiator, desc.keyspace, desc.columnFamily));
+            Tracing.traceRepair("Sending completed merkle tree to {} for {}.{}", initiator, desc.keyspace, desc.columnFamily);
+        }
         MessagingService.instance().sendOneWay(new ValidationComplete(desc, tree).createMessage(), initiator);
     }
 }
diff --git a/src/java/org/apache/cassandra/repair/messages/AnticompactionRequest.java b/src/java/org/apache/cassandra/repair/messages/AnticompactionRequest.java
index 34ea5a5..b554500 100644
--- a/src/java/org/apache/cassandra/repair/messages/AnticompactionRequest.java
+++ b/src/java/org/apache/cassandra/repair/messages/AnticompactionRequest.java
@@ -19,20 +19,31 @@
 
 import java.io.DataInput;
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
 import java.util.UUID;
 
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.utils.UUIDSerializer;
 
 public class AnticompactionRequest extends RepairMessage
 {
     public static MessageSerializer serializer = new AnticompactionRequestSerializer();
     public final UUID parentRepairSession;
+    /**
+     * Successfully repaired ranges. Does not contain null.
+     */
+    public final Collection<Range<Token>> successfulRanges;
 
-    public AnticompactionRequest(UUID parentRepairSession)
+    public AnticompactionRequest(UUID parentRepairSession, Collection<Range<Token>> ranges)
     {
         super(Type.ANTICOMPACTION_REQUEST, null);
         this.parentRepairSession = parentRepairSession;
+        this.successfulRanges = ranges;
     }
 
     public static class AnticompactionRequestSerializer implements MessageSerializer<AnticompactionRequest>
@@ -40,17 +51,38 @@
         public void serialize(AnticompactionRequest message, DataOutputPlus out, int version) throws IOException
         {
             UUIDSerializer.serializer.serialize(message.parentRepairSession, out, version);
+            out.writeInt(message.successfulRanges.size());
+            for (Range<Token> r : message.successfulRanges)
+            {
+                MessagingService.validatePartitioner(r);
+                Range.tokenSerializer.serialize(r, out, version);
+            }
         }
 
         public AnticompactionRequest deserialize(DataInput in, int version) throws IOException
         {
             UUID parentRepairSession = UUIDSerializer.serializer.deserialize(in, version);
-            return new AnticompactionRequest(parentRepairSession);
+            int rangeCount = in.readInt();
+            List<Range<Token>> ranges = new ArrayList<>(rangeCount);
+            for (int i = 0; i < rangeCount; i++)
+                ranges.add((Range<Token>) Range.tokenSerializer.deserialize(in, MessagingService.globalPartitioner(), version));
+            return new AnticompactionRequest(parentRepairSession, ranges);
         }
 
         public long serializedSize(AnticompactionRequest message, int version)
         {
-            return UUIDSerializer.serializer.serializedSize(message.parentRepairSession, version);
+            long size = UUIDSerializer.serializer.serializedSize(message.parentRepairSession, version);
+            for (Range<Token> r : message.successfulRanges)
+                size += Range.tokenSerializer.serializedSize(r, version);
+            return size;
         }
     }
-}
\ No newline at end of file
+
+    @Override
+    public String toString()
+    {
+        return "AnticompactionRequest{" +
+                "parentRepairSession=" + parentRepairSession +
+                "} " + super.toString();
+    }
+}
diff --git a/src/java/org/apache/cassandra/repair/messages/PrepareMessage.java b/src/java/org/apache/cassandra/repair/messages/PrepareMessage.java
index 5699677..3a00376 100644
--- a/src/java/org/apache/cassandra/repair/messages/PrepareMessage.java
+++ b/src/java/org/apache/cassandra/repair/messages/PrepareMessage.java
@@ -28,27 +28,38 @@
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.utils.UUIDSerializer;
 
 
 public class PrepareMessage extends RepairMessage
 {
-    public final static MessageSerializer serializer = new PrepareMessageSerializer();
+    public final static MessageSerializer serializer = new PrepareMessageSerializer(false);
+    public final static MessageSerializer globalSerializer = new PrepareMessageSerializer(true);
     public final List<UUID> cfIds;
     public final Collection<Range<Token>> ranges;
 
     public final UUID parentRepairSession;
+    public final boolean isIncremental;
 
-    public PrepareMessage(UUID parentRepairSession, List<UUID> cfIds, Collection<Range<Token>> ranges)
+    public PrepareMessage(UUID parentRepairSession, List<UUID> cfIds, Collection<Range<Token>> ranges, boolean isIncremental, boolean isGlobal)
     {
-        super(Type.PREPARE_MESSAGE, null);
+        super(isGlobal ? Type.PREPARE_GLOBAL_MESSAGE : Type.PREPARE_MESSAGE, null);
         this.parentRepairSession = parentRepairSession;
         this.cfIds = cfIds;
         this.ranges = ranges;
+        this.isIncremental = isIncremental;
     }
 
     public static class PrepareMessageSerializer implements MessageSerializer<PrepareMessage>
     {
+        private final boolean isGlobal;
+
+        public PrepareMessageSerializer(boolean global)
+        {
+            this.isGlobal = global;
+        }
+
         public void serialize(PrepareMessage message, DataOutputPlus out, int version) throws IOException
         {
             out.writeInt(message.cfIds.size());
@@ -56,8 +67,12 @@
                 UUIDSerializer.serializer.serialize(cfId, out, version);
             UUIDSerializer.serializer.serialize(message.parentRepairSession, out, version);
             out.writeInt(message.ranges.size());
-            for (Range r : message.ranges)
-                Range.serializer.serialize(r, out, version);
+            for (Range<Token> r : message.ranges)
+            {
+                MessagingService.validatePartitioner(r);
+                Range.tokenSerializer.serialize(r, out, version);
+            }
+            out.writeBoolean(message.isIncremental);
         }
 
         public PrepareMessage deserialize(DataInput in, int version) throws IOException
@@ -70,8 +85,10 @@
             int rangeCount = in.readInt();
             List<Range<Token>> ranges = new ArrayList<>(rangeCount);
             for (int i = 0; i < rangeCount; i++)
-                ranges.add((Range<Token>) Range.serializer.deserialize(in, version).toTokenBounds());
-            return new PrepareMessage(parentRepairSession, cfIds, ranges);
+                ranges.add((Range<Token>) Range.tokenSerializer.deserialize(in, MessagingService.globalPartitioner(), version));
+            boolean isIncremental = in.readBoolean();
+
+            return new PrepareMessage(parentRepairSession, cfIds, ranges, isIncremental, isGlobal);
         }
 
         public long serializedSize(PrepareMessage message, int version)
@@ -83,8 +100,9 @@
                 size += UUIDSerializer.serializer.serializedSize(cfId, version);
             size += UUIDSerializer.serializer.serializedSize(message.parentRepairSession, version);
             size += sizes.sizeof(message.ranges.size());
-            for (Range r : message.ranges)
-                size += Range.serializer.serializedSize(r, version);
+            for (Range<Token> r : message.ranges)
+                size += Range.tokenSerializer.serializedSize(r, version);
+            size += sizes.sizeof(message.isIncremental);
             return size;
         }
     }
@@ -96,6 +114,7 @@
                 "cfIds='" + cfIds + '\'' +
                 ", ranges=" + ranges +
                 ", parentRepairSession=" + parentRepairSession +
+                ", isIncremental="+isIncremental +
                 '}';
     }
 }
diff --git a/src/java/org/apache/cassandra/repair/messages/RepairMessage.java b/src/java/org/apache/cassandra/repair/messages/RepairMessage.java
index b49f1f3..82e474f 100644
--- a/src/java/org/apache/cassandra/repair/messages/RepairMessage.java
+++ b/src/java/org/apache/cassandra/repair/messages/RepairMessage.java
@@ -45,6 +45,7 @@
         SYNC_COMPLETE(3, SyncComplete.serializer),
         ANTICOMPACTION_REQUEST(4, AnticompactionRequest.serializer),
         PREPARE_MESSAGE(5, PrepareMessage.serializer),
+        PREPARE_GLOBAL_MESSAGE(8, PrepareMessage.globalSerializer),
         SNAPSHOT(6, SnapshotMessage.serializer),
         CLEANUP(7, CleanupMessage.serializer);
 
@@ -82,7 +83,7 @@
         return new MessageOut<>(MessagingService.Verb.REPAIR_MESSAGE, this, RepairMessage.serializer);
     }
 
-    public static class RepairMessageSerializer implements IVersionedSerializer<RepairMessage>
+    public static class RepairMessageSerializer implements MessageSerializer<RepairMessage>
     {
         public void serialize(RepairMessage message, DataOutputPlus out, int version) throws IOException
         {
diff --git a/src/java/org/apache/cassandra/repair/messages/RepairOption.java b/src/java/org/apache/cassandra/repair/messages/RepairOption.java
new file mode 100644
index 0000000..44a1e57
--- /dev/null
+++ b/src/java/org/apache/cassandra/repair/messages/RepairOption.java
@@ -0,0 +1,324 @@
+/*
+ * 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.repair.messages;
+
+import java.util.*;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.repair.RepairParallelism;
+import org.apache.cassandra.tools.nodetool.Repair;
+import org.apache.cassandra.utils.FBUtilities;
+
+/**
+ * Repair options.
+ */
+public class RepairOption
+{
+    public static final String PARALLELISM_KEY = "parallelism";
+    public static final String PRIMARY_RANGE_KEY = "primaryRange";
+    public static final String INCREMENTAL_KEY = "incremental";
+    public static final String JOB_THREADS_KEY = "jobThreads";
+    public static final String RANGES_KEY = "ranges";
+    public static final String COLUMNFAMILIES_KEY = "columnFamilies";
+    public static final String DATACENTERS_KEY = "dataCenters";
+    public static final String HOSTS_KEY = "hosts";
+    public static final String TRACE_KEY = "trace";
+
+    // we don't want to push nodes too much for repair
+    public static final int MAX_JOB_THREADS = 4;
+
+    private static final Logger logger = LoggerFactory.getLogger(RepairOption.class);
+
+    /**
+     * Construct RepairOptions object from given map of Strings.
+     * <p>
+     * Available options are:
+     *
+     * <table>
+     *     <caption>Repair Options</caption>
+     *     <thead>
+     *         <tr>
+     *             <th>key</th>
+     *             <th>value</th>
+     *             <th>default (when key not given)</th>
+     *         </tr>
+     *     </thead>
+     *     <tbody>
+     *         <tr>
+     *             <td>parallelism</td>
+     *             <td>"sequential", "parallel" or "dc_parallel"</td>
+     *             <td>"sequential"</td>
+     *         </tr>
+     *         <tr>
+     *             <td>primaryRange</td>
+     *             <td>"true" if perform repair only on primary range.</td>
+     *             <td>false</td>
+     *         </tr>
+     *         <tr>
+     *             <td>incremental</td>
+     *             <td>"true" if perform incremental repair.</td>
+     *             <td>false</td>
+     *         </tr>
+     *         <tr>
+     *             <td>trace</td>
+     *             <td>"true" if repair is traced.</td>
+     *             <td>false</td>
+     *         </tr>
+     *         <tr>
+     *             <td>jobThreads</td>
+     *             <td>Number of threads to use to run repair job.</td>
+     *             <td>1</td>
+     *         </tr>
+     *         <tr>
+     *             <td>ranges</td>
+     *             <td>Ranges to repair. A range is expressed as &lt;start token&gt;:&lt;end token&gt;
+     *             and multiple ranges can be given as comma separated ranges(e.g. aaa:bbb,ccc:ddd).</td>
+     *             <td></td>
+     *         </tr>
+     *         <tr>
+     *             <td>columnFamilies</td>
+     *             <td>Specify names of ColumnFamilies to repair.
+     *             Multiple ColumnFamilies can be given as comma separated values(e.g. cf1,cf2,cf3).</td>
+     *             <td></td>
+     *         </tr>
+     *         <tr>
+     *             <td>dataCenters</td>
+     *             <td>Specify names of data centers who participate in this repair.
+     *             Multiple data centers can be given as comma separated values(e.g. dc1,dc2,dc3).</td>
+     *             <td></td>
+     *         </tr>
+     *         <tr>
+     *             <td>hosts</td>
+     *             <td>Specify names of hosts who participate in this repair.
+     *             Multiple hosts can be given as comma separated values(e.g. cass1,cass2).</td>
+     *             <td></td>
+     *         </tr>
+     *     </tbody>
+     * </table>
+     *
+     * @param options options to parse
+     * @param partitioner partitioner is used to construct token ranges
+     * @return RepairOptions object
+     */
+    public static RepairOption parse(Map<String, String> options, IPartitioner partitioner)
+    {
+        // if no parallel option is given, then this will be "sequential" by default.
+        RepairParallelism parallelism = RepairParallelism.fromName(options.get(PARALLELISM_KEY));
+        boolean primaryRange = Boolean.parseBoolean(options.get(PRIMARY_RANGE_KEY));
+        boolean incremental = Boolean.parseBoolean(options.get(INCREMENTAL_KEY));
+        boolean trace = Boolean.parseBoolean(options.get(TRACE_KEY));
+
+        int jobThreads = 1;
+        if (options.containsKey(JOB_THREADS_KEY))
+        {
+            try
+            {
+                jobThreads = Integer.parseInt(options.get(JOB_THREADS_KEY));
+            }
+            catch (NumberFormatException ignore) {}
+        }
+        // ranges
+        String rangesStr = options.get(RANGES_KEY);
+        Set<Range<Token>> ranges = new HashSet<>();
+        if (rangesStr != null)
+        {
+            if (incremental)
+                logger.warn("Incremental repair can't be requested with subrange repair " +
+                            "because each subrange repair would generate an anti-compacted table. " +
+                            "The repair will occur but without anti-compaction.");
+            StringTokenizer tokenizer = new StringTokenizer(rangesStr, ",");
+            while (tokenizer.hasMoreTokens())
+            {
+                String[] rangeStr = tokenizer.nextToken().split(":", 2);
+                if (rangeStr.length < 2)
+                {
+                    continue;
+                }
+                Token parsedBeginToken = partitioner.getTokenFactory().fromString(rangeStr[0].trim());
+                Token parsedEndToken = partitioner.getTokenFactory().fromString(rangeStr[1].trim());
+                ranges.add(new Range<>(parsedBeginToken, parsedEndToken));
+            }
+        }
+
+        RepairOption option = new RepairOption(parallelism, primaryRange, incremental, trace, jobThreads, ranges, !ranges.isEmpty());
+
+        // data centers
+        String dataCentersStr = options.get(DATACENTERS_KEY);
+        Collection<String> dataCenters = new HashSet<>();
+        if (dataCentersStr != null)
+        {
+            StringTokenizer tokenizer = new StringTokenizer(dataCentersStr, ",");
+            while (tokenizer.hasMoreTokens())
+            {
+                dataCenters.add(tokenizer.nextToken().trim());
+            }
+            option.getDataCenters().addAll(dataCenters);
+        }
+
+        // hosts
+        String hostsStr = options.get(HOSTS_KEY);
+        Collection<String> hosts = new HashSet<>();
+        if (hostsStr != null)
+        {
+            StringTokenizer tokenizer = new StringTokenizer(hostsStr, ",");
+            while (tokenizer.hasMoreTokens())
+            {
+                hosts.add(tokenizer.nextToken().trim());
+            }
+            option.getHosts().addAll(hosts);
+        }
+
+        // columnfamilies
+        String cfStr = options.get(COLUMNFAMILIES_KEY);
+        if (cfStr != null)
+        {
+            Collection<String> columnFamilies = new HashSet<>();
+            StringTokenizer tokenizer = new StringTokenizer(cfStr, ",");
+            while (tokenizer.hasMoreTokens())
+            {
+                columnFamilies.add(tokenizer.nextToken().trim());
+            }
+            option.getColumnFamilies().addAll(columnFamilies);
+        }
+
+        // validate options
+        if (jobThreads > MAX_JOB_THREADS)
+        {
+            throw new IllegalArgumentException("Too many job threads. Max is " + MAX_JOB_THREADS);
+        }
+        if (primaryRange && ((!dataCenters.isEmpty() && !option.isInLocalDCOnly()) || !hosts.isEmpty()))
+        {
+            throw new IllegalArgumentException("You need to run primary range repair on all nodes in the cluster.");
+        }
+
+        return option;
+    }
+
+    private final RepairParallelism parallelism;
+    private final boolean primaryRange;
+    private final boolean incremental;
+    private final boolean trace;
+    private final int jobThreads;
+    private final boolean isSubrangeRepair;
+
+    private final Collection<String> columnFamilies = new HashSet<>();
+    private final Collection<String> dataCenters = new HashSet<>();
+    private final Collection<String> hosts = new HashSet<>();
+    private final Collection<Range<Token>> ranges = new HashSet<>();
+
+    public RepairOption(RepairParallelism parallelism, boolean primaryRange, boolean incremental, boolean trace, int jobThreads, Collection<Range<Token>> ranges, boolean isSubrangeRepair)
+    {
+        if (FBUtilities.isWindows() &&
+            (DatabaseDescriptor.getDiskAccessMode() != Config.DiskAccessMode.standard || DatabaseDescriptor.getIndexAccessMode() != Config.DiskAccessMode.standard) &&
+            parallelism == RepairParallelism.SEQUENTIAL)
+        {
+            logger.warn("Sequential repair disabled when memory-mapped I/O is configured on Windows. Reverting to parallel.");
+            this.parallelism = RepairParallelism.PARALLEL;
+        }
+        else
+            this.parallelism = parallelism;
+
+        this.primaryRange = primaryRange;
+        this.incremental = incremental;
+        this.trace = trace;
+        this.jobThreads = jobThreads;
+        this.ranges.addAll(ranges);
+        this.isSubrangeRepair = isSubrangeRepair;
+    }
+
+    public RepairParallelism getParallelism()
+    {
+        return parallelism;
+    }
+
+    public boolean isPrimaryRange()
+    {
+        return primaryRange;
+    }
+
+    public boolean isIncremental()
+    {
+        return incremental;
+    }
+
+    public boolean isTraced()
+    {
+        return trace;
+    }
+
+    public int getJobThreads()
+    {
+        return jobThreads;
+    }
+
+    public Collection<String> getColumnFamilies()
+    {
+        return columnFamilies;
+    }
+
+    public Collection<Range<Token>> getRanges()
+    {
+        return ranges;
+    }
+
+    public Collection<String> getDataCenters()
+    {
+        return dataCenters;
+    }
+
+    public Collection<String> getHosts()
+    {
+        return hosts;
+    }
+
+    public boolean isGlobal()
+    {
+        return dataCenters.isEmpty() && hosts.isEmpty() && !isSubrangeRepair();
+    }
+
+    public boolean isSubrangeRepair()
+    {
+        return isSubrangeRepair;
+    }
+
+    public boolean isInLocalDCOnly() {
+        return dataCenters.size() == 1 && dataCenters.contains(DatabaseDescriptor.getLocalDataCenter());
+    }
+
+    @Override
+    public String toString()
+    {
+        return "repair options (" +
+                       "parallelism: " + parallelism +
+                       ", primary range: " + primaryRange +
+                       ", incremental: " + incremental +
+                       ", job threads: " + jobThreads +
+                       ", ColumnFamilies: " + columnFamilies +
+                       ", dataCenters: " + dataCenters +
+                       ", hosts: " + hosts +
+                       ", # of ranges: " + ranges.size() +
+                       ')';
+    }
+}
diff --git a/src/java/org/apache/cassandra/repair/messages/SyncRequest.java b/src/java/org/apache/cassandra/repair/messages/SyncRequest.java
index e677cd8..68aaf4d 100644
--- a/src/java/org/apache/cassandra/repair/messages/SyncRequest.java
+++ b/src/java/org/apache/cassandra/repair/messages/SyncRequest.java
@@ -30,6 +30,7 @@
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.net.CompactEndpointSerializationHelper;
+import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.repair.RepairJobDesc;
 
 /**
@@ -66,7 +67,10 @@
             CompactEndpointSerializationHelper.serialize(message.dst, out);
             out.writeInt(message.ranges.size());
             for (Range<Token> range : message.ranges)
-                AbstractBounds.serializer.serialize(range, out, version);
+            {
+                MessagingService.validatePartitioner(range);
+                AbstractBounds.tokenSerializer.serialize(range, out, version);
+            }
         }
 
         public SyncRequest deserialize(DataInput in, int version) throws IOException
@@ -78,7 +82,7 @@
             int rangesCount = in.readInt();
             List<Range<Token>> ranges = new ArrayList<>(rangesCount);
             for (int i = 0; i < rangesCount; ++i)
-                ranges.add((Range<Token>) AbstractBounds.serializer.deserialize(in, version).toTokenBounds());
+                ranges.add((Range<Token>) AbstractBounds.tokenSerializer.deserialize(in, MessagingService.globalPartitioner(), version));
             return new SyncRequest(desc, owner, src, dst, ranges);
         }
 
@@ -88,8 +92,19 @@
             size += 3 * CompactEndpointSerializationHelper.serializedSize(message.initiator);
             size += TypeSizes.NATIVE.sizeof(message.ranges.size());
             for (Range<Token> range : message.ranges)
-                size += AbstractBounds.serializer.serializedSize(range, version);
+                size += AbstractBounds.tokenSerializer.serializedSize(range, version);
             return size;
         }
     }
+
+    @Override
+    public String toString()
+    {
+        return "SyncRequest{" +
+                "initiator=" + initiator +
+                ", src=" + src +
+                ", dst=" + dst +
+                ", ranges=" + ranges +
+                "} " + super.toString();
+    }
 }
diff --git a/src/java/org/apache/cassandra/repair/messages/ValidationRequest.java b/src/java/org/apache/cassandra/repair/messages/ValidationRequest.java
index c73b708..43bcf23 100644
--- a/src/java/org/apache/cassandra/repair/messages/ValidationRequest.java
+++ b/src/java/org/apache/cassandra/repair/messages/ValidationRequest.java
@@ -42,6 +42,14 @@
     }
 
     @Override
+    public String toString()
+    {
+        return "ValidationRequest{" +
+                "gcBefore=" + gcBefore +
+                "} " + super.toString();
+    }
+
+    @Override
     public boolean equals(Object o)
     {
         if (this == o) return true;
diff --git a/src/java/org/apache/cassandra/scheduler/RoundRobinScheduler.java b/src/java/org/apache/cassandra/scheduler/RoundRobinScheduler.java
index 2968672..c98c0fe 100644
--- a/src/java/org/apache/cassandra/scheduler/RoundRobinScheduler.java
+++ b/src/java/org/apache/cassandra/scheduler/RoundRobinScheduler.java
@@ -86,12 +86,7 @@
                 weightedQueue.put(t, timeoutMS);
                 // the scheduler will release us when a slot is available
             }
-            catch (TimeoutException e)
-            {
-                queueSize.acquireUninterruptibly();
-                throw e;
-            }
-            catch (InterruptedException e)
+            catch (TimeoutException | InterruptedException e)
             {
                 queueSize.acquireUninterruptibly();
                 throw e;
@@ -145,8 +140,6 @@
         weightedQueue = queues.putIfAbsent(id, maybenew);
         if (weightedQueue == null)
         {
-            // created new queue: register for monitoring
-            maybenew.register();
             return maybenew;
         }
 
diff --git a/src/java/org/apache/cassandra/scheduler/WeightedQueue.java b/src/java/org/apache/cassandra/scheduler/WeightedQueue.java
index 6ab5215..298938d 100644
--- a/src/java/org/apache/cassandra/scheduler/WeightedQueue.java
+++ b/src/java/org/apache/cassandra/scheduler/WeightedQueue.java
@@ -26,7 +26,7 @@
 
 import org.apache.cassandra.metrics.LatencyMetrics;
 
-class WeightedQueue implements WeightedQueueMBean
+class WeightedQueue
 {
     private final LatencyMetrics metric;
 
@@ -41,20 +41,6 @@
         this.metric =  new LatencyMetrics("scheduler", "WeightedQueue", key);
     }
 
-    public void register()
-    {
-        // expose monitoring data
-        MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
-        try
-        {
-            mbs.registerMBean(this, new ObjectName("org.apache.cassandra.scheduler:type=WeightedQueue,queue=" + key));
-        }
-        catch (Exception e)
-        {
-            throw new RuntimeException(e);
-        }
-    }
-
     public void put(Thread t, long timeoutMS) throws InterruptedException, TimeoutException
     {
         if (!queue.offer(new WeightedQueue.Entry(t), timeoutMS, TimeUnit.MILLISECONDS))
@@ -85,31 +71,4 @@
             this.thread = thread;
         }
     }
-
-    /** MBean related methods */
-
-    public long getOperations()
-    {
-        return metric.latency.count();
-    }
-
-    public long getTotalLatencyMicros()
-    {
-        return metric.totalLatency.count();
-    }
-
-    public double getRecentLatencyMicros()
-    {
-        return metric.getRecentLatency();
-    }
-
-    public long[] getTotalLatencyHistogramMicros()
-    {
-        return metric.totalLatencyHistogram.getBuckets(false);
-    }
-
-    public long[] getRecentLatencyHistogramMicros()
-    {
-        return metric.recentLatencyHistogram.getBuckets(true);
-    }
 }
diff --git a/src/java/org/apache/cassandra/scheduler/WeightedQueueMBean.java b/src/java/org/apache/cassandra/scheduler/WeightedQueueMBean.java
deleted file mode 100644
index d16d007..0000000
--- a/src/java/org/apache/cassandra/scheduler/WeightedQueueMBean.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * 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.scheduler;
-
-/**
- * Exposes client request scheduling metrics for a particular scheduler queue.
- * @see org.apache.cassandra.metrics.LatencyMetrics
- */
-@Deprecated
-public interface WeightedQueueMBean
-{
-    public long getOperations();
-    public long getTotalLatencyMicros();
-    public double getRecentLatencyMicros();
-    public long[] getTotalLatencyHistogramMicros();
-    public long[] getRecentLatencyHistogramMicros();
-}
diff --git a/src/java/org/apache/cassandra/schema/LegacySchemaTables.java b/src/java/org/apache/cassandra/schema/LegacySchemaTables.java
new file mode 100644
index 0000000..8d5bf4f
--- /dev/null
+++ b/src/java/org/apache/cassandra/schema/LegacySchemaTables.java
@@ -0,0 +1,1491 @@
+/*
+ * 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.schema;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.*;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.MapDifference;
+import com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.cache.CachingOptions;
+import org.apache.cassandra.config.*;
+import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.cql3.functions.*;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.columniterator.IdentityQueryFilter;
+import org.apache.cassandra.db.composites.CellNameType;
+import org.apache.cassandra.db.composites.CellNames;
+import org.apache.cassandra.db.composites.Composite;
+import org.apache.cassandra.db.filter.QueryFilter;
+import org.apache.cassandra.db.index.SecondaryIndexManager;
+import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.io.compress.CompressionParameters;
+import org.apache.cassandra.locator.AbstractReplicationStrategy;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+
+import static org.apache.cassandra.cql3.QueryProcessor.executeOnceInternal;
+import static org.apache.cassandra.utils.FBUtilities.fromJsonMap;
+import static org.apache.cassandra.utils.FBUtilities.json;
+
+/** system.schema_* tables used to store keyspace/table/type attributes prior to C* 3.0 */
+public class LegacySchemaTables
+{
+    private static final Logger logger = LoggerFactory.getLogger(LegacySchemaTables.class);
+
+    public static final String KEYSPACES = "schema_keyspaces";
+    public static final String COLUMNFAMILIES = "schema_columnfamilies";
+    public static final String COLUMNS = "schema_columns";
+    public static final String TRIGGERS = "schema_triggers";
+    public static final String USERTYPES = "schema_usertypes";
+    public static final String FUNCTIONS = "schema_functions";
+    public static final String AGGREGATES = "schema_aggregates";
+
+    public static final List<String> ALL = Arrays.asList(KEYSPACES, COLUMNFAMILIES, COLUMNS, TRIGGERS, USERTYPES, FUNCTIONS, AGGREGATES);
+
+    private static final CFMetaData Keyspaces =
+        compile(KEYSPACES,
+                "keyspace definitions",
+                "CREATE TABLE %s ("
+                + "keyspace_name text,"
+                + "durable_writes boolean,"
+                + "strategy_class text,"
+                + "strategy_options text,"
+                + "PRIMARY KEY ((keyspace_name))) "
+                + "WITH COMPACT STORAGE");
+
+    private static final CFMetaData Columnfamilies =
+        compile(COLUMNFAMILIES,
+                "table definitions",
+                "CREATE TABLE %s ("
+                + "keyspace_name text,"
+                + "columnfamily_name text,"
+                + "bloom_filter_fp_chance double,"
+                + "caching text,"
+                + "cf_id uuid," // post-2.1 UUID cfid
+                + "comment text,"
+                + "compaction_strategy_class text,"
+                + "compaction_strategy_options text,"
+                + "comparator text,"
+                + "compression_parameters text,"
+                + "default_time_to_live int,"
+                + "default_validator text,"
+                + "dropped_columns map<text, bigint>,"
+                + "gc_grace_seconds int,"
+                + "is_dense boolean,"
+                + "key_validator text,"
+                + "local_read_repair_chance double,"
+                + "max_compaction_threshold int,"
+                + "max_index_interval int,"
+                + "memtable_flush_period_in_ms int,"
+                + "min_compaction_threshold int,"
+                + "min_index_interval int,"
+                + "read_repair_chance double,"
+                + "speculative_retry text,"
+                + "subcomparator text,"
+                + "type text,"
+                + "PRIMARY KEY ((keyspace_name), columnfamily_name))");
+
+    private static final CFMetaData Columns =
+        compile(COLUMNS,
+                "column definitions",
+                "CREATE TABLE %s ("
+                + "keyspace_name text,"
+                + "columnfamily_name text,"
+                + "column_name text,"
+                + "component_index int,"
+                + "index_name text,"
+                + "index_options text,"
+                + "index_type text,"
+                + "type text,"
+                + "validator text,"
+                + "PRIMARY KEY ((keyspace_name), columnfamily_name, column_name))");
+
+    private static final CFMetaData Triggers =
+        compile(TRIGGERS,
+                "trigger definitions",
+                "CREATE TABLE %s ("
+                + "keyspace_name text,"
+                + "columnfamily_name text,"
+                + "trigger_name text,"
+                + "trigger_options map<text, text>,"
+                + "PRIMARY KEY ((keyspace_name), columnfamily_name, trigger_name))");
+
+    private static final CFMetaData Usertypes =
+        compile(USERTYPES,
+                "user defined type definitions",
+                "CREATE TABLE %s ("
+                + "keyspace_name text,"
+                + "type_name text,"
+                + "field_names list<text>,"
+                + "field_types list<text>,"
+                + "PRIMARY KEY ((keyspace_name), type_name))");
+
+    private static final CFMetaData Functions =
+        compile(FUNCTIONS,
+                "user defined function definitions",
+                "CREATE TABLE %s ("
+                + "keyspace_name text,"
+                + "function_name text,"
+                + "signature frozen<list<text>>,"
+                + "argument_names list<text>,"
+                + "argument_types list<text>,"
+                + "body text,"
+                + "language text,"
+                + "return_type text,"
+                + "called_on_null_input boolean,"
+                + "PRIMARY KEY ((keyspace_name), function_name, signature))");
+
+    private static final CFMetaData Aggregates =
+        compile(AGGREGATES,
+                "user defined aggregate definitions",
+                "CREATE TABLE %s ("
+                + "keyspace_name text,"
+                + "aggregate_name text,"
+                + "signature frozen<list<text>>,"
+                + "argument_types list<text>,"
+                + "final_func text,"
+                + "initcond blob,"
+                + "return_type text,"
+                + "state_func text,"
+                + "state_type text,"
+                + "PRIMARY KEY ((keyspace_name), aggregate_name, signature))");
+
+    public static final List<CFMetaData> All = Arrays.asList(Keyspaces, Columnfamilies, Columns, Triggers, Usertypes, Functions, Aggregates);
+
+    private static CFMetaData compile(String name, String description, String schema)
+    {
+        return CFMetaData.compile(String.format(schema, name), SystemKeyspace.NAME)
+                         .comment(description)
+                         .gcGraceSeconds((int) TimeUnit.DAYS.toSeconds(7));
+    }
+
+    /** add entries to system.schema_* for the hardcoded system definitions */
+    public static void saveSystemKeyspaceSchema()
+    {
+        KSMetaData keyspace = Schema.instance.getKSMetaData(SystemKeyspace.NAME);
+        long timestamp = FBUtilities.timestampMicros();
+        // delete old, possibly obsolete entries in schema tables
+        for (String table : ALL)
+        {
+            executeOnceInternal(String.format("DELETE FROM system.%s USING TIMESTAMP ? WHERE keyspace_name = ?", table),
+                                timestamp,
+                                keyspace.name);
+        }
+        // (+1 to timestamp to make sure we don't get shadowed by the tombstones we just added)
+        makeCreateKeyspaceMutation(keyspace, timestamp + 1).apply();
+    }
+
+    public static Collection<KSMetaData> readSchemaFromSystemTables()
+    {
+        List<Row> serializedSchema = getSchemaPartitionsForTable(KEYSPACES);
+
+        List<KSMetaData> keyspaces = new ArrayList<>(serializedSchema.size());
+
+        for (Row partition : serializedSchema)
+        {
+            if (isEmptySchemaPartition(partition) || isSystemKeyspaceSchemaPartition(partition))
+                continue;
+
+            keyspaces.add(createKeyspaceFromSchemaPartitions(partition,
+                                                             readSchemaPartitionForKeyspace(COLUMNFAMILIES, partition.key),
+                                                             readSchemaPartitionForKeyspace(USERTYPES, partition.key)));
+
+            // Will be moved away in #6717
+            for (UDFunction function : createFunctionsFromFunctionsPartition(readSchemaPartitionForKeyspace(FUNCTIONS, partition.key)).values())
+                org.apache.cassandra.cql3.functions.Functions.addOrReplaceFunction(function);
+
+            // Will be moved away in #6717
+            for (UDAggregate aggregate : createAggregatesFromAggregatesPartition(readSchemaPartitionForKeyspace(AGGREGATES, partition.key)).values())
+                org.apache.cassandra.cql3.functions.Functions.addOrReplaceFunction(aggregate);
+        }
+
+        return keyspaces;
+    }
+
+    public static void truncateSchemaTables()
+    {
+        for (String table : ALL)
+            getSchemaCFS(table).truncateBlocking();
+    }
+
+    private static void flushSchemaTables()
+    {
+        for (String table : ALL)
+            SystemKeyspace.forceBlockingFlush(table);
+    }
+
+    /**
+     * Read schema from system keyspace and calculate MD5 digest of every row, resulting digest
+     * will be converted into UUID which would act as content-based version of the schema.
+     */
+    public static UUID calculateSchemaDigest()
+    {
+        MessageDigest digest;
+        try
+        {
+            digest = MessageDigest.getInstance("MD5");
+        }
+        catch (NoSuchAlgorithmException e)
+        {
+            throw new RuntimeException(e);
+        }
+
+        for (String table : ALL)
+        {
+            for (Row partition : getSchemaPartitionsForTable(table))
+            {
+                if (isEmptySchemaPartition(partition) || isSystemKeyspaceSchemaPartition(partition))
+                    continue;
+
+                // we want to digest only live columns
+                ColumnFamilyStore.removeDeletedColumnsOnly(partition.cf, Integer.MAX_VALUE, SecondaryIndexManager.nullUpdater);
+                partition.cf.purgeTombstones(Integer.MAX_VALUE);
+                partition.cf.updateDigest(digest);
+            }
+        }
+
+        return UUID.nameUUIDFromBytes(digest.digest());
+    }
+
+    /**
+     * @param schemaTableName The name of the table responsible for part of the schema
+     * @return CFS responsible to hold low-level serialized schema
+     */
+    private static ColumnFamilyStore getSchemaCFS(String schemaTableName)
+    {
+        return Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStore(schemaTableName);
+    }
+
+    /**
+     * @param schemaTableName The name of the table responsible for part of the schema.
+     * @return low-level schema representation
+     */
+    private static List<Row> getSchemaPartitionsForTable(String schemaTableName)
+    {
+        Token minToken = StorageService.getPartitioner().getMinimumToken();
+        return getSchemaCFS(schemaTableName).getRangeSlice(new Range<RowPosition>(minToken.minKeyBound(), minToken.maxKeyBound()),
+                                                           null,
+                                                           new IdentityQueryFilter(),
+                                                           Integer.MAX_VALUE,
+                                                           System.currentTimeMillis());
+    }
+
+    public static Collection<Mutation> convertSchemaToMutations()
+    {
+        Map<DecoratedKey, Mutation> mutationMap = new HashMap<>();
+
+        for (String table : ALL)
+            convertSchemaToMutations(mutationMap, table);
+
+        return mutationMap.values();
+    }
+
+    private static void convertSchemaToMutations(Map<DecoratedKey, Mutation> mutationMap, String schemaTableName)
+    {
+        for (Row partition : getSchemaPartitionsForTable(schemaTableName))
+        {
+            if (isSystemKeyspaceSchemaPartition(partition))
+                continue;
+
+            Mutation mutation = mutationMap.get(partition.key);
+            if (mutation == null)
+            {
+                mutation = new Mutation(SystemKeyspace.NAME, partition.key.getKey());
+                mutationMap.put(partition.key, mutation);
+            }
+
+            mutation.add(partition.cf);
+        }
+    }
+
+    private static Map<DecoratedKey, ColumnFamily> readSchemaForKeyspaces(String schemaTableName, Set<String> keyspaceNames)
+    {
+        Map<DecoratedKey, ColumnFamily> schema = new HashMap<>();
+
+        for (String keyspaceName : keyspaceNames)
+        {
+            Row schemaEntity = readSchemaPartitionForKeyspace(schemaTableName, keyspaceName);
+            if (schemaEntity.cf != null)
+                schema.put(schemaEntity.key, schemaEntity.cf);
+        }
+
+        return schema;
+    }
+
+    private static ByteBuffer getSchemaKSKey(String ksName)
+    {
+        return AsciiType.instance.fromString(ksName);
+    }
+
+    private static Row readSchemaPartitionForKeyspace(String schemaTableName, String keyspaceName)
+    {
+        DecoratedKey keyspaceKey = StorageService.getPartitioner().decorateKey(getSchemaKSKey(keyspaceName));
+        return readSchemaPartitionForKeyspace(schemaTableName, keyspaceKey);
+    }
+
+    private static Row readSchemaPartitionForKeyspace(String schemaTableName, DecoratedKey keyspaceKey)
+    {
+        QueryFilter filter = QueryFilter.getIdentityFilter(keyspaceKey, schemaTableName, System.currentTimeMillis());
+        return new Row(keyspaceKey, getSchemaCFS(schemaTableName).getColumnFamily(filter));
+    }
+
+    private static Row readSchemaPartitionForTable(String schemaTableName, String keyspaceName, String tableName)
+    {
+        DecoratedKey key = StorageService.getPartitioner().decorateKey(getSchemaKSKey(keyspaceName));
+        ColumnFamilyStore store = getSchemaCFS(schemaTableName);
+        Composite prefix = store.getComparator().make(tableName);
+        ColumnFamily cells = store.getColumnFamily(key, prefix, prefix.end(), false, Integer.MAX_VALUE, System.currentTimeMillis());
+        return new Row(key, cells);
+    }
+
+    private static boolean isEmptySchemaPartition(Row partition)
+    {
+        return partition.cf == null || (partition.cf.isMarkedForDelete() && !partition.cf.hasColumns());
+    }
+
+    private static boolean isSystemKeyspaceSchemaPartition(Row partition)
+    {
+        return getSchemaKSKey(SystemKeyspace.NAME).equals(partition.key.getKey());
+    }
+
+    /**
+     * Merge remote schema in form of mutations with local and mutate ks/cf metadata objects
+     * (which also involves fs operations on add/drop ks/cf)
+     *
+     * @param mutations the schema changes to apply
+     *
+     * @throws ConfigurationException If one of metadata attributes has invalid value
+     * @throws IOException If data was corrupted during transportation or failed to apply fs operations
+     */
+    public static synchronized void mergeSchema(Collection<Mutation> mutations) throws ConfigurationException, IOException
+    {
+        mergeSchema(mutations, true);
+        Schema.instance.updateVersionAndAnnounce();
+    }
+
+    public static synchronized void mergeSchema(Collection<Mutation> mutations, boolean doFlush) throws IOException
+    {
+        // compare before/after schemas of the affected keyspaces only
+        Set<String> keyspaces = new HashSet<>(mutations.size());
+        for (Mutation mutation : mutations)
+            keyspaces.add(ByteBufferUtil.string(mutation.key()));
+
+        // current state of the schema
+        Map<DecoratedKey, ColumnFamily> oldKeyspaces = readSchemaForKeyspaces(KEYSPACES, keyspaces);
+        Map<DecoratedKey, ColumnFamily> oldColumnFamilies = readSchemaForKeyspaces(COLUMNFAMILIES, keyspaces);
+        Map<DecoratedKey, ColumnFamily> oldTypes = readSchemaForKeyspaces(USERTYPES, keyspaces);
+        Map<DecoratedKey, ColumnFamily> oldFunctions = readSchemaForKeyspaces(FUNCTIONS, keyspaces);
+        Map<DecoratedKey, ColumnFamily> oldAggregates = readSchemaForKeyspaces(AGGREGATES, keyspaces);
+
+        for (Mutation mutation : mutations)
+            mutation.apply();
+
+        if (doFlush)
+            flushSchemaTables();
+
+        // with new data applied
+        Map<DecoratedKey, ColumnFamily> newKeyspaces = readSchemaForKeyspaces(KEYSPACES, keyspaces);
+        Map<DecoratedKey, ColumnFamily> newColumnFamilies = readSchemaForKeyspaces(COLUMNFAMILIES, keyspaces);
+        Map<DecoratedKey, ColumnFamily> newTypes = readSchemaForKeyspaces(USERTYPES, keyspaces);
+        Map<DecoratedKey, ColumnFamily> newFunctions = readSchemaForKeyspaces(FUNCTIONS, keyspaces);
+        Map<DecoratedKey, ColumnFamily> newAggregates = readSchemaForKeyspaces(AGGREGATES, keyspaces);
+
+        Set<String> keyspacesToDrop = mergeKeyspaces(oldKeyspaces, newKeyspaces);
+        mergeTables(oldColumnFamilies, newColumnFamilies);
+        mergeTypes(oldTypes, newTypes);
+        mergeFunctions(oldFunctions, newFunctions);
+        mergeAggregates(oldAggregates, newAggregates);
+
+        // it is safe to drop a keyspace only when all nested ColumnFamilies where deleted
+        for (String keyspaceToDrop : keyspacesToDrop)
+            Schema.instance.dropKeyspace(keyspaceToDrop);
+    }
+
+    private static Set<String> mergeKeyspaces(Map<DecoratedKey, ColumnFamily> before, Map<DecoratedKey, ColumnFamily> after)
+    {
+        List<Row> created = new ArrayList<>();
+        List<String> altered = new ArrayList<>();
+        Set<String> dropped = new HashSet<>();
+
+        /*
+         * - we don't care about entriesOnlyOnLeft() or entriesInCommon(), because only the changes are of interest to us
+         * - of all entriesOnlyOnRight(), we only care about ones that have live columns; it's possible to have a ColumnFamily
+         *   there that only has the top-level deletion, if:
+         *      a) a pushed DROP KEYSPACE change for a keyspace hadn't ever made it to this node in the first place
+         *      b) a pulled dropped keyspace that got dropped before it could find a way to this node
+         * - of entriesDiffering(), we don't care about the scenario where both pre and post-values have zero live columns:
+         *   that means that a keyspace had been recreated and dropped, and the recreated keyspace had never found a way
+         *   to this node
+         */
+        MapDifference<DecoratedKey, ColumnFamily> diff = Maps.difference(before, after);
+
+        for (Map.Entry<DecoratedKey, ColumnFamily> entry : diff.entriesOnlyOnRight().entrySet())
+            if (entry.getValue().hasColumns())
+                created.add(new Row(entry.getKey(), entry.getValue()));
+
+        for (Map.Entry<DecoratedKey, MapDifference.ValueDifference<ColumnFamily>> entry : diff.entriesDiffering().entrySet())
+        {
+            String keyspaceName = AsciiType.instance.compose(entry.getKey().getKey());
+
+            ColumnFamily pre  = entry.getValue().leftValue();
+            ColumnFamily post = entry.getValue().rightValue();
+
+            if (pre.hasColumns() && post.hasColumns())
+                altered.add(keyspaceName);
+            else if (pre.hasColumns())
+                dropped.add(keyspaceName);
+            else if (post.hasColumns()) // a (re)created keyspace
+                created.add(new Row(entry.getKey(), post));
+        }
+
+        for (Row row : created)
+            Schema.instance.addKeyspace(createKeyspaceFromSchemaPartition(row));
+        for (String name : altered)
+            Schema.instance.updateKeyspace(name);
+        return dropped;
+    }
+
+    // see the comments for mergeKeyspaces()
+    private static void mergeTables(Map<DecoratedKey, ColumnFamily> before, Map<DecoratedKey, ColumnFamily> after)
+    {
+        List<CFMetaData> created = new ArrayList<>();
+        List<CFMetaData> altered = new ArrayList<>();
+        List<CFMetaData> dropped = new ArrayList<>();
+
+        MapDifference<DecoratedKey, ColumnFamily> diff = Maps.difference(before, after);
+
+        for (Map.Entry<DecoratedKey, ColumnFamily> entry : diff.entriesOnlyOnRight().entrySet())
+            if (entry.getValue().hasColumns())
+                created.addAll(createTablesFromTablesPartition(new Row(entry.getKey(), entry.getValue())).values());
+
+        for (Map.Entry<DecoratedKey, MapDifference.ValueDifference<ColumnFamily>> entry : diff.entriesDiffering().entrySet())
+        {
+            String keyspaceName = AsciiType.instance.compose(entry.getKey().getKey());
+
+            ColumnFamily pre  = entry.getValue().leftValue();
+            ColumnFamily post = entry.getValue().rightValue();
+
+            if (pre.hasColumns() && post.hasColumns())
+            {
+                MapDifference<String, CFMetaData> delta =
+                    Maps.difference(Schema.instance.getKSMetaData(keyspaceName).cfMetaData(),
+                                    createTablesFromTablesPartition(new Row(entry.getKey(), post)));
+
+                dropped.addAll(delta.entriesOnlyOnLeft().values());
+                created.addAll(delta.entriesOnlyOnRight().values());
+                Iterables.addAll(altered, Iterables.transform(delta.entriesDiffering().values(), new Function<MapDifference.ValueDifference<CFMetaData>, CFMetaData>()
+                {
+                    public CFMetaData apply(MapDifference.ValueDifference<CFMetaData> pair)
+                    {
+                        return pair.rightValue();
+                    }
+                }));
+            }
+            else if (pre.hasColumns())
+            {
+                dropped.addAll(Schema.instance.getKSMetaData(keyspaceName).cfMetaData().values());
+            }
+            else if (post.hasColumns())
+            {
+                created.addAll(createTablesFromTablesPartition(new Row(entry.getKey(), post)).values());
+            }
+        }
+
+        for (CFMetaData cfm : created)
+            Schema.instance.addTable(cfm);
+        for (CFMetaData cfm : altered)
+            Schema.instance.updateTable(cfm.ksName, cfm.cfName);
+        for (CFMetaData cfm : dropped)
+            Schema.instance.dropTable(cfm.ksName, cfm.cfName);
+    }
+
+    // see the comments for mergeKeyspaces()
+    private static void mergeTypes(Map<DecoratedKey, ColumnFamily> before, Map<DecoratedKey, ColumnFamily> after)
+    {
+        List<UserType> created = new ArrayList<>();
+        List<UserType> altered = new ArrayList<>();
+        List<UserType> dropped = new ArrayList<>();
+
+        MapDifference<DecoratedKey, ColumnFamily> diff = Maps.difference(before, after);
+
+        // New keyspace with types
+        for (Map.Entry<DecoratedKey, ColumnFamily> entry : diff.entriesOnlyOnRight().entrySet())
+            if (entry.getValue().hasColumns())
+                created.addAll(createTypesFromPartition(new Row(entry.getKey(), entry.getValue())).values());
+
+        for (Map.Entry<DecoratedKey, MapDifference.ValueDifference<ColumnFamily>> entry : diff.entriesDiffering().entrySet())
+        {
+            String keyspaceName = AsciiType.instance.compose(entry.getKey().getKey());
+
+            ColumnFamily pre  = entry.getValue().leftValue();
+            ColumnFamily post = entry.getValue().rightValue();
+
+            if (pre.hasColumns() && post.hasColumns())
+            {
+                MapDifference<ByteBuffer, UserType> delta =
+                    Maps.difference(Schema.instance.getKSMetaData(keyspaceName).userTypes.getAllTypes(),
+                                    createTypesFromPartition(new Row(entry.getKey(), post)));
+
+                dropped.addAll(delta.entriesOnlyOnLeft().values());
+                created.addAll(delta.entriesOnlyOnRight().values());
+                Iterables.addAll(altered, Iterables.transform(delta.entriesDiffering().values(), new Function<MapDifference.ValueDifference<UserType>, UserType>()
+                {
+                    public UserType apply(MapDifference.ValueDifference<UserType> pair)
+                    {
+                        return pair.rightValue();
+                    }
+                }));
+            }
+            else if (pre.hasColumns())
+            {
+                dropped.addAll(Schema.instance.getKSMetaData(keyspaceName).userTypes.getAllTypes().values());
+            }
+            else if (post.hasColumns())
+            {
+                created.addAll(createTypesFromPartition(new Row(entry.getKey(), post)).values());
+            }
+        }
+
+        for (UserType type : created)
+            Schema.instance.addType(type);
+        for (UserType type : altered)
+            Schema.instance.updateType(type);
+        for (UserType type : dropped)
+            Schema.instance.dropType(type);
+    }
+
+    // see the comments for mergeKeyspaces()
+    private static void mergeFunctions(Map<DecoratedKey, ColumnFamily> before, Map<DecoratedKey, ColumnFamily> after)
+    {
+        List<UDFunction> created = new ArrayList<>();
+        List<UDFunction> altered = new ArrayList<>();
+        List<UDFunction> dropped = new ArrayList<>();
+
+        MapDifference<DecoratedKey, ColumnFamily> diff = Maps.difference(before, after);
+
+        // New keyspace with functions
+        for (Map.Entry<DecoratedKey, ColumnFamily> entry : diff.entriesOnlyOnRight().entrySet())
+            if (entry.getValue().hasColumns())
+                created.addAll(createFunctionsFromFunctionsPartition(new Row(entry.getKey(), entry.getValue())).values());
+
+        for (Map.Entry<DecoratedKey, MapDifference.ValueDifference<ColumnFamily>> entry : diff.entriesDiffering().entrySet())
+        {
+            ColumnFamily pre = entry.getValue().leftValue();
+            ColumnFamily post = entry.getValue().rightValue();
+
+            if (pre.hasColumns() && post.hasColumns())
+            {
+                MapDifference<ByteBuffer, UDFunction> delta =
+                    Maps.difference(createFunctionsFromFunctionsPartition(new Row(entry.getKey(), pre)),
+                                    createFunctionsFromFunctionsPartition(new Row(entry.getKey(), post)));
+
+                dropped.addAll(delta.entriesOnlyOnLeft().values());
+                created.addAll(delta.entriesOnlyOnRight().values());
+                Iterables.addAll(altered, Iterables.transform(delta.entriesDiffering().values(), new Function<MapDifference.ValueDifference<UDFunction>, UDFunction>()
+                {
+                    public UDFunction apply(MapDifference.ValueDifference<UDFunction> pair)
+                    {
+                        return pair.rightValue();
+                    }
+                }));
+            }
+            else if (pre.hasColumns())
+            {
+                dropped.addAll(createFunctionsFromFunctionsPartition(new Row(entry.getKey(), pre)).values());
+            }
+            else if (post.hasColumns())
+            {
+                created.addAll(createFunctionsFromFunctionsPartition(new Row(entry.getKey(), post)).values());
+            }
+        }
+
+        for (UDFunction udf : created)
+            Schema.instance.addFunction(udf);
+        for (UDFunction udf : altered)
+            Schema.instance.updateFunction(udf);
+        for (UDFunction udf : dropped)
+            Schema.instance.dropFunction(udf);
+    }
+
+    // see the comments for mergeKeyspaces()
+    private static void mergeAggregates(Map<DecoratedKey, ColumnFamily> before, Map<DecoratedKey, ColumnFamily> after)
+    {
+        List<UDAggregate> created = new ArrayList<>();
+        List<UDAggregate> altered = new ArrayList<>();
+        List<UDAggregate> dropped = new ArrayList<>();
+
+        MapDifference<DecoratedKey, ColumnFamily> diff = Maps.difference(before, after);
+
+        // New keyspace with functions
+        for (Map.Entry<DecoratedKey, ColumnFamily> entry : diff.entriesOnlyOnRight().entrySet())
+            if (entry.getValue().hasColumns())
+                created.addAll(createAggregatesFromAggregatesPartition(new Row(entry.getKey(), entry.getValue())).values());
+
+        for (Map.Entry<DecoratedKey, MapDifference.ValueDifference<ColumnFamily>> entry : diff.entriesDiffering().entrySet())
+        {
+            ColumnFamily pre = entry.getValue().leftValue();
+            ColumnFamily post = entry.getValue().rightValue();
+
+            if (pre.hasColumns() && post.hasColumns())
+            {
+                MapDifference<ByteBuffer, UDAggregate> delta =
+                    Maps.difference(createAggregatesFromAggregatesPartition(new Row(entry.getKey(), pre)),
+                                    createAggregatesFromAggregatesPartition(new Row(entry.getKey(), post)));
+
+                dropped.addAll(delta.entriesOnlyOnLeft().values());
+                created.addAll(delta.entriesOnlyOnRight().values());
+                Iterables.addAll(altered, Iterables.transform(delta.entriesDiffering().values(), new Function<MapDifference.ValueDifference<UDAggregate>, UDAggregate>()
+                {
+                    public UDAggregate apply(MapDifference.ValueDifference<UDAggregate> pair)
+                    {
+                        return pair.rightValue();
+                    }
+                }));
+            }
+            else if (pre.hasColumns())
+            {
+                dropped.addAll(createAggregatesFromAggregatesPartition(new Row(entry.getKey(), pre)).values());
+            }
+            else if (post.hasColumns())
+            {
+                created.addAll(createAggregatesFromAggregatesPartition(new Row(entry.getKey(), post)).values());
+            }
+        }
+
+        for (UDAggregate udf : created)
+            Schema.instance.addAggregate(udf);
+        for (UDAggregate udf : altered)
+            Schema.instance.updateAggregate(udf);
+        for (UDAggregate udf : dropped)
+            Schema.instance.dropAggregate(udf);
+    }
+
+    /*
+     * Keyspace metadata serialization/deserialization.
+     */
+
+    public static Mutation makeCreateKeyspaceMutation(KSMetaData keyspace, long timestamp)
+    {
+        return makeCreateKeyspaceMutation(keyspace, timestamp, true);
+    }
+
+    private static Mutation makeCreateKeyspaceMutation(KSMetaData keyspace, long timestamp, boolean withTablesAndTypesAndFunctions)
+    {
+        Mutation mutation = new Mutation(SystemKeyspace.NAME, getSchemaKSKey(keyspace.name));
+        ColumnFamily cells = mutation.addOrGet(Keyspaces);
+        CFRowAdder adder = new CFRowAdder(cells, Keyspaces.comparator.builder().build(), timestamp);
+
+        adder.add("durable_writes", keyspace.durableWrites);
+        adder.add("strategy_class", keyspace.strategyClass.getName());
+        adder.add("strategy_options", json(keyspace.strategyOptions));
+
+        if (withTablesAndTypesAndFunctions)
+        {
+            for (UserType type : keyspace.userTypes.getAllTypes().values())
+                addTypeToSchemaMutation(type, timestamp, mutation);
+
+            for (CFMetaData table : keyspace.cfMetaData().values())
+                addTableToSchemaMutation(table, timestamp, true, mutation);
+        }
+
+        return mutation;
+    }
+
+    public static Mutation makeDropKeyspaceMutation(KSMetaData keyspace, long timestamp)
+    {
+        Mutation mutation = new Mutation(SystemKeyspace.NAME, getSchemaKSKey(keyspace.name));
+        for (String schemaTable : ALL)
+            mutation.delete(schemaTable, timestamp);
+        mutation.delete(SystemKeyspace.BUILT_INDEXES, timestamp);
+        return mutation;
+    }
+
+    private static KSMetaData createKeyspaceFromSchemaPartitions(Row serializedKeyspace, Row serializedTables, Row serializedTypes)
+    {
+        Collection<CFMetaData> tables = createTablesFromTablesPartition(serializedTables).values();
+        UTMetaData types = new UTMetaData(createTypesFromPartition(serializedTypes));
+        return createKeyspaceFromSchemaPartition(serializedKeyspace).cloneWith(tables, types);
+    }
+
+    public static KSMetaData createKeyspaceFromName(String keyspace)
+    {
+        Row partition = readSchemaPartitionForKeyspace(KEYSPACES, keyspace);
+
+        if (isEmptySchemaPartition(partition))
+            throw new RuntimeException(String.format("%s not found in the schema definitions keyspaceName (%s).", keyspace, KEYSPACES));
+
+        return createKeyspaceFromSchemaPartition(partition);
+    }
+
+    /**
+     * Deserialize only Keyspace attributes without nested tables or types
+     *
+     * @param partition Keyspace attributes in serialized form
+     */
+    private static KSMetaData createKeyspaceFromSchemaPartition(Row partition)
+    {
+        String query = String.format("SELECT * FROM %s.%s", SystemKeyspace.NAME, KEYSPACES);
+        UntypedResultSet.Row row = QueryProcessor.resultify(query, partition).one();
+        return new KSMetaData(row.getString("keyspace_name"),
+                              AbstractReplicationStrategy.getClass(row.getString("strategy_class")),
+                              fromJsonMap(row.getString("strategy_options")),
+                              row.getBoolean("durable_writes"));
+    }
+
+    /*
+     * User type metadata serialization/deserialization.
+     */
+
+    public static Mutation makeCreateTypeMutation(KSMetaData keyspace, UserType type, long timestamp)
+    {
+        // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
+        Mutation mutation = makeCreateKeyspaceMutation(keyspace, timestamp, false);
+        addTypeToSchemaMutation(type, timestamp, mutation);
+        return mutation;
+    }
+
+    private static void addTypeToSchemaMutation(UserType type, long timestamp, Mutation mutation)
+    {
+        ColumnFamily cells = mutation.addOrGet(Usertypes);
+
+        Composite prefix = Usertypes.comparator.make(type.name);
+        CFRowAdder adder = new CFRowAdder(cells, prefix, timestamp);
+
+        adder.resetCollection("field_names");
+        adder.resetCollection("field_types");
+
+        for (int i = 0; i < type.size(); i++)
+        {
+            adder.addListEntry("field_names", type.fieldName(i));
+            adder.addListEntry("field_types", type.fieldType(i).toString());
+        }
+    }
+
+    public static Mutation dropTypeFromSchemaMutation(KSMetaData keyspace, UserType type, long timestamp)
+    {
+        // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
+        Mutation mutation = makeCreateKeyspaceMutation(keyspace, timestamp, false);
+
+        ColumnFamily cells = mutation.addOrGet(Usertypes);
+        int ldt = (int) (System.currentTimeMillis() / 1000);
+
+        Composite prefix = Usertypes.comparator.make(type.name);
+        cells.addAtom(new RangeTombstone(prefix, prefix.end(), timestamp, ldt));
+
+        return mutation;
+    }
+
+    private static Map<ByteBuffer, UserType> createTypesFromPartition(Row partition)
+    {
+        String query = String.format("SELECT * FROM %s.%s", SystemKeyspace.NAME, USERTYPES);
+        Map<ByteBuffer, UserType> types = new HashMap<>();
+        for (UntypedResultSet.Row row : QueryProcessor.resultify(query, partition))
+        {
+            UserType type = createTypeFromRow(row);
+            types.put(type.name, type);
+        }
+        return types;
+    }
+
+    private static UserType createTypeFromRow(UntypedResultSet.Row row)
+    {
+        String keyspace = row.getString("keyspace_name");
+        ByteBuffer name = ByteBufferUtil.bytes(row.getString("type_name"));
+        List<String> rawColumns = row.getList("field_names", UTF8Type.instance);
+        List<String> rawTypes = row.getList("field_types", UTF8Type.instance);
+
+        List<ByteBuffer> columns = new ArrayList<>(rawColumns.size());
+        for (String rawColumn : rawColumns)
+            columns.add(ByteBufferUtil.bytes(rawColumn));
+
+        List<AbstractType<?>> types = new ArrayList<>(rawTypes.size());
+        for (String rawType : rawTypes)
+            types.add(parseType(rawType));
+
+        return new UserType(keyspace, name, columns, types);
+    }
+
+    /*
+     * Table metadata serialization/deserialization.
+     */
+
+    public static Mutation makeCreateTableMutation(KSMetaData keyspace, CFMetaData table, long timestamp)
+    {
+        // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
+        Mutation mutation = makeCreateKeyspaceMutation(keyspace, timestamp, false);
+        addTableToSchemaMutation(table, timestamp, true, mutation);
+        return mutation;
+    }
+
+    private static void addTableToSchemaMutation(CFMetaData table, long timestamp, boolean withColumnsAndTriggers, Mutation mutation)
+    {
+        // For property that can be null (and can be changed), we insert tombstones, to make sure
+        // we don't keep a property the user has removed
+        ColumnFamily cells = mutation.addOrGet(Columnfamilies);
+        Composite prefix = Columnfamilies.comparator.make(table.cfName);
+        CFRowAdder adder = new CFRowAdder(cells, prefix, timestamp);
+
+        adder.add("cf_id", table.cfId);
+        adder.add("type", table.cfType.toString());
+
+        if (table.isSuper())
+        {
+            // We need to continue saving the comparator and subcomparator separatly, otherwise
+            // we won't know at deserialization if the subcomparator should be taken into account
+            // TODO: we should implement an on-start migration if we want to get rid of that.
+            adder.add("comparator", table.comparator.subtype(0).toString());
+            adder.add("subcomparator", table.comparator.subtype(1).toString());
+        }
+        else
+        {
+            adder.add("comparator", table.comparator.toString());
+        }
+
+        adder.add("bloom_filter_fp_chance", table.getBloomFilterFpChance());
+        adder.add("caching", table.getCaching().toString());
+        adder.add("comment", table.getComment());
+        adder.add("compaction_strategy_class", table.compactionStrategyClass.getName());
+        adder.add("compaction_strategy_options", json(table.compactionStrategyOptions));
+        adder.add("compression_parameters", json(table.compressionParameters.asThriftOptions()));
+        adder.add("default_time_to_live", table.getDefaultTimeToLive());
+        adder.add("default_validator", table.getDefaultValidator().toString());
+        adder.add("gc_grace_seconds", table.getGcGraceSeconds());
+        adder.add("key_validator", table.getKeyValidator().toString());
+        adder.add("local_read_repair_chance", table.getDcLocalReadRepairChance());
+        adder.add("max_compaction_threshold", table.getMaxCompactionThreshold());
+        adder.add("max_index_interval", table.getMaxIndexInterval());
+        adder.add("memtable_flush_period_in_ms", table.getMemtableFlushPeriod());
+        adder.add("min_compaction_threshold", table.getMinCompactionThreshold());
+        adder.add("min_index_interval", table.getMinIndexInterval());
+        adder.add("read_repair_chance", table.getReadRepairChance());
+        adder.add("speculative_retry", table.getSpeculativeRetry().toString());
+
+        for (Map.Entry<ColumnIdentifier, Long> entry : table.getDroppedColumns().entrySet())
+            adder.addMapEntry("dropped_columns", entry.getKey().toString(), entry.getValue());
+
+        adder.add("is_dense", table.getIsDense());
+
+        if (withColumnsAndTriggers)
+        {
+            for (ColumnDefinition column : table.allColumns())
+                addColumnToSchemaMutation(table, column, timestamp, mutation);
+
+            for (TriggerDefinition trigger : table.getTriggers().values())
+                addTriggerToSchemaMutation(table, trigger, timestamp, mutation);
+        }
+    }
+
+    public static Mutation makeUpdateTableMutation(KSMetaData keyspace,
+                                                   CFMetaData oldTable,
+                                                   CFMetaData newTable,
+                                                   long timestamp)
+    {
+        Mutation mutation = makeCreateKeyspaceMutation(keyspace, timestamp, false);
+
+        addTableToSchemaMutation(newTable, timestamp, false, mutation);
+
+        MapDifference<ByteBuffer, ColumnDefinition> columnDiff = Maps.difference(oldTable.getColumnMetadata(),
+                                                                                 newTable.getColumnMetadata());
+
+        // columns that are no longer needed
+        for (ColumnDefinition column : columnDiff.entriesOnlyOnLeft().values())
+            dropColumnFromSchemaMutation(oldTable, column, timestamp, mutation);
+
+        // newly added columns
+        for (ColumnDefinition column : columnDiff.entriesOnlyOnRight().values())
+            addColumnToSchemaMutation(newTable, column, timestamp, mutation);
+
+        // old columns with updated attributes
+        for (ByteBuffer name : columnDiff.entriesDiffering().keySet())
+            addColumnToSchemaMutation(newTable, newTable.getColumnDefinition(name), timestamp, mutation);
+
+        MapDifference<String, TriggerDefinition> triggerDiff = Maps.difference(oldTable.getTriggers(), newTable.getTriggers());
+
+        // dropped triggers
+        for (TriggerDefinition trigger : triggerDiff.entriesOnlyOnLeft().values())
+            dropTriggerFromSchemaMutation(oldTable, trigger, timestamp, mutation);
+
+        // newly created triggers
+        for (TriggerDefinition trigger : triggerDiff.entriesOnlyOnRight().values())
+            addTriggerToSchemaMutation(newTable, trigger, timestamp, mutation);
+
+        return mutation;
+    }
+
+    public static Mutation makeDropTableMutation(KSMetaData keyspace, CFMetaData table, long timestamp)
+    {
+        // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
+        Mutation mutation = makeCreateKeyspaceMutation(keyspace, timestamp, false);
+
+        ColumnFamily cells = mutation.addOrGet(Columnfamilies);
+        int ldt = (int) (System.currentTimeMillis() / 1000);
+
+        Composite prefix = Columnfamilies.comparator.make(table.cfName);
+        cells.addAtom(new RangeTombstone(prefix, prefix.end(), timestamp, ldt));
+
+        for (ColumnDefinition column : table.allColumns())
+            dropColumnFromSchemaMutation(table, column, timestamp, mutation);
+
+        for (TriggerDefinition trigger : table.getTriggers().values())
+            dropTriggerFromSchemaMutation(table, trigger, timestamp, mutation);
+
+        // TODO: get rid of in #6717
+        ColumnFamily indexCells = mutation.addOrGet(SystemKeyspace.BuiltIndexes);
+        for (String indexName : Keyspace.open(keyspace.name).getColumnFamilyStore(table.cfName).getBuiltIndexes())
+            indexCells.addTombstone(indexCells.getComparator().makeCellName(indexName), ldt, timestamp);
+
+        return mutation;
+    }
+
+    public static CFMetaData createTableFromName(String keyspace, String table)
+    {
+        Row partition = readSchemaPartitionForTable(COLUMNFAMILIES, keyspace, table);
+
+        if (isEmptySchemaPartition(partition))
+            throw new RuntimeException(String.format("%s:%s not found in the schema definitions keyspace.", keyspace, table));
+
+        return createTableFromTablePartition(partition);
+    }
+
+    /**
+     * Deserialize tables from low-level schema representation, all of them belong to the same keyspace
+     *
+     * @return map containing name of the table and its metadata for faster lookup
+     */
+    private static Map<String, CFMetaData> createTablesFromTablesPartition(Row partition)
+    {
+        if (partition.cf == null)
+            return Collections.emptyMap();
+
+        String query = String.format("SELECT * FROM %s.%s", SystemKeyspace.NAME, COLUMNFAMILIES);
+        Map<String, CFMetaData> tables = new HashMap<>();
+        for (UntypedResultSet.Row row : QueryProcessor.resultify(query, partition))
+        {
+            CFMetaData cfm = createTableFromTableRow(row);
+            tables.put(cfm.cfName, cfm);
+        }
+        return tables;
+    }
+
+    public static CFMetaData createTableFromTablePartitionAndColumnsPartition(Row serializedTable, Row serializedColumns)
+    {
+        String query = String.format("SELECT * FROM %s.%s", SystemKeyspace.NAME, COLUMNFAMILIES);
+        return createTableFromTableRowAndColumnsPartition(QueryProcessor.resultify(query, serializedTable).one(), serializedColumns);
+    }
+
+    private static CFMetaData createTableFromTableRowAndColumnsPartition(UntypedResultSet.Row tableRow, Row serializedColumns)
+    {
+        String query = String.format("SELECT * FROM %s.%s", SystemKeyspace.NAME, COLUMNS);
+        return createTableFromTableRowAndColumnRows(tableRow, QueryProcessor.resultify(query, serializedColumns));
+    }
+
+    private static CFMetaData createTableFromTablePartition(Row row)
+    {
+        String query = String.format("SELECT * FROM %s.%s", SystemKeyspace.NAME, COLUMNFAMILIES);
+        return createTableFromTableRow(QueryProcessor.resultify(query, row).one());
+    }
+
+    /**
+     * Deserialize table metadata from low-level representation
+     *
+     * @return Metadata deserialized from schema
+     */
+    private static CFMetaData createTableFromTableRow(UntypedResultSet.Row result)
+    {
+        String ksName = result.getString("keyspace_name");
+        String cfName = result.getString("columnfamily_name");
+
+        Row serializedColumns = readSchemaPartitionForTable(COLUMNS, ksName, cfName);
+        CFMetaData cfm = createTableFromTableRowAndColumnsPartition(result, serializedColumns);
+
+        Row serializedTriggers = readSchemaPartitionForTable(TRIGGERS, ksName, cfName);
+        for (TriggerDefinition trigger : createTriggersFromTriggersPartition(serializedTriggers))
+            cfm.addTriggerDefinition(trigger);
+
+        return cfm;
+    }
+
+    public static CFMetaData createTableFromTableRowAndColumnRows(UntypedResultSet.Row result,
+                                                                  UntypedResultSet serializedColumnDefinitions)
+    {
+        String ksName = result.getString("keyspace_name");
+        String cfName = result.getString("columnfamily_name");
+
+        AbstractType<?> rawComparator = TypeParser.parse(result.getString("comparator"));
+        AbstractType<?> subComparator = result.has("subcomparator") ? TypeParser.parse(result.getString("subcomparator")) : null;
+        ColumnFamilyType cfType = ColumnFamilyType.valueOf(result.getString("type"));
+
+        AbstractType<?> fullRawComparator = CFMetaData.makeRawAbstractType(rawComparator, subComparator);
+
+        List<ColumnDefinition> columnDefs = createColumnsFromColumnRows(serializedColumnDefinitions,
+                                                                        ksName,
+                                                                        cfName,
+                                                                        fullRawComparator,
+                                                                        cfType == ColumnFamilyType.Super);
+
+        boolean isDense = result.has("is_dense")
+                        ? result.getBoolean("is_dense")
+                        : CFMetaData.calculateIsDense(fullRawComparator, columnDefs);
+
+        CellNameType comparator = CellNames.fromAbstractType(fullRawComparator, isDense);
+
+        // if we are upgrading, we use id generated from names initially
+        UUID cfId = result.has("cf_id")
+                  ? result.getUUID("cf_id")
+                  : CFMetaData.generateLegacyCfId(ksName, cfName);
+
+        CFMetaData cfm = new CFMetaData(ksName, cfName, cfType, comparator, cfId);
+        cfm.isDense(isDense);
+
+        cfm.readRepairChance(result.getDouble("read_repair_chance"));
+        cfm.dcLocalReadRepairChance(result.getDouble("local_read_repair_chance"));
+        cfm.gcGraceSeconds(result.getInt("gc_grace_seconds"));
+        cfm.defaultValidator(TypeParser.parse(result.getString("default_validator")));
+        cfm.keyValidator(TypeParser.parse(result.getString("key_validator")));
+        cfm.minCompactionThreshold(result.getInt("min_compaction_threshold"));
+        cfm.maxCompactionThreshold(result.getInt("max_compaction_threshold"));
+        if (result.has("comment"))
+            cfm.comment(result.getString("comment"));
+        if (result.has("memtable_flush_period_in_ms"))
+            cfm.memtableFlushPeriod(result.getInt("memtable_flush_period_in_ms"));
+        cfm.caching(CachingOptions.fromString(result.getString("caching")));
+        if (result.has("default_time_to_live"))
+            cfm.defaultTimeToLive(result.getInt("default_time_to_live"));
+        if (result.has("speculative_retry"))
+            cfm.speculativeRetry(CFMetaData.SpeculativeRetry.fromString(result.getString("speculative_retry")));
+        cfm.compactionStrategyClass(CFMetaData.createCompactionStrategy(result.getString("compaction_strategy_class")));
+        cfm.compressionParameters(CompressionParameters.create(fromJsonMap(result.getString("compression_parameters"))));
+        cfm.compactionStrategyOptions(fromJsonMap(result.getString("compaction_strategy_options")));
+
+        if (result.has("min_index_interval"))
+            cfm.minIndexInterval(result.getInt("min_index_interval"));
+
+        if (result.has("max_index_interval"))
+            cfm.maxIndexInterval(result.getInt("max_index_interval"));
+
+        if (result.has("bloom_filter_fp_chance"))
+            cfm.bloomFilterFpChance(result.getDouble("bloom_filter_fp_chance"));
+        else
+            cfm.bloomFilterFpChance(cfm.getBloomFilterFpChance());
+
+        if (result.has("dropped_columns"))
+            cfm.droppedColumns(convertDroppedColumns(result.getMap("dropped_columns", UTF8Type.instance, LongType.instance)));
+
+        for (ColumnDefinition cd : columnDefs)
+            cfm.addOrReplaceColumnDefinition(cd);
+
+        return cfm.rebuild();
+    }
+
+    private static Map<ColumnIdentifier, Long> convertDroppedColumns(Map<String, Long> raw)
+    {
+        Map<ColumnIdentifier, Long> converted = Maps.newHashMap();
+        for (Map.Entry<String, Long> entry : raw.entrySet())
+            converted.put(new ColumnIdentifier(entry.getKey(), true), entry.getValue());
+        return converted;
+    }
+
+    /*
+     * Column metadata serialization/deserialization.
+     */
+
+    private static void addColumnToSchemaMutation(CFMetaData table, ColumnDefinition column, long timestamp, Mutation mutation)
+    {
+        ColumnFamily cells = mutation.addOrGet(Columns);
+        Composite prefix = Columns.comparator.make(table.cfName, column.name.toString());
+        CFRowAdder adder = new CFRowAdder(cells, prefix, timestamp);
+
+        adder.add("validator", column.type.toString());
+        adder.add("type", serializeKind(column.kind));
+        adder.add("component_index", column.isOnAllComponents() ? null : column.position());
+        adder.add("index_name", column.getIndexName());
+        adder.add("index_type", column.getIndexType() == null ? null : column.getIndexType().toString());
+        adder.add("index_options", json(column.getIndexOptions()));
+    }
+
+    private static String serializeKind(ColumnDefinition.Kind kind)
+    {
+        // For backward compatibility we need to special case CLUSTERING_COLUMN
+        return kind == ColumnDefinition.Kind.CLUSTERING_COLUMN ? "clustering_key" : kind.toString().toLowerCase();
+    }
+
+    private static ColumnDefinition.Kind deserializeKind(String kind)
+    {
+        if (kind.equalsIgnoreCase("clustering_key"))
+            return ColumnDefinition.Kind.CLUSTERING_COLUMN;
+        return Enum.valueOf(ColumnDefinition.Kind.class, kind.toUpperCase());
+    }
+
+    private static void dropColumnFromSchemaMutation(CFMetaData table, ColumnDefinition column, long timestamp, Mutation mutation)
+    {
+        ColumnFamily cells = mutation.addOrGet(Columns);
+        int ldt = (int) (System.currentTimeMillis() / 1000);
+
+        // Note: we do want to use name.toString(), not name.bytes directly for backward compatibility (For CQL3, this won't make a difference).
+        Composite prefix = Columns.comparator.make(table.cfName, column.name.toString());
+        cells.addAtom(new RangeTombstone(prefix, prefix.end(), timestamp, ldt));
+    }
+
+    private static List<ColumnDefinition> createColumnsFromColumnRows(UntypedResultSet rows,
+                                                                      String keyspace,
+                                                                      String table,
+                                                                      AbstractType<?> rawComparator,
+                                                                      boolean isSuper)
+    {
+        List<ColumnDefinition> columns = new ArrayList<>();
+        for (UntypedResultSet.Row row : rows)
+            columns.add(createColumnFromColumnRow(row, keyspace, table, rawComparator, isSuper));
+        return columns;
+    }
+
+    private static ColumnDefinition createColumnFromColumnRow(UntypedResultSet.Row row,
+                                                              String keyspace,
+                                                              String table,
+                                                              AbstractType<?> rawComparator,
+                                                              boolean isSuper)
+    {
+        ColumnDefinition.Kind kind = deserializeKind(row.getString("type"));
+
+        Integer componentIndex = null;
+        if (kind == ColumnDefinition.Kind.REGULAR && isSuper)
+            componentIndex = 1; // A ColumnDefinition for super columns applies to the column component
+        else if (row.has("component_index"))
+            componentIndex = row.getInt("component_index");
+
+        // Note: we save the column name as string, but we should not assume that it is an UTF8 name, we
+        // we need to use the comparator fromString method
+        AbstractType<?> comparator = kind == ColumnDefinition.Kind.REGULAR
+                                   ? getComponentComparator(rawComparator, componentIndex)
+                                   : UTF8Type.instance;
+        ColumnIdentifier name = new ColumnIdentifier(comparator.fromString(row.getString("column_name")), comparator);
+
+        AbstractType<?> validator = parseType(row.getString("validator"));
+
+        IndexType indexType = null;
+        if (row.has("index_type"))
+            indexType = IndexType.valueOf(row.getString("index_type"));
+
+        Map<String, String> indexOptions = null;
+        if (row.has("index_options"))
+            indexOptions = fromJsonMap(row.getString("index_options"));
+
+        String indexName = null;
+        if (row.has("index_name"))
+            indexName = row.getString("index_name");
+
+        return new ColumnDefinition(keyspace, table, name, validator, indexType, indexOptions, indexName, componentIndex, kind);
+    }
+
+    private static AbstractType<?> getComponentComparator(AbstractType<?> rawComparator, Integer componentIndex)
+    {
+        return (componentIndex == null || (componentIndex == 0 && !(rawComparator instanceof CompositeType)))
+               ? rawComparator
+               : ((CompositeType)rawComparator).types.get(componentIndex);
+    }
+
+    /*
+     * Trigger metadata serialization/deserialization.
+     */
+
+    private static void addTriggerToSchemaMutation(CFMetaData table, TriggerDefinition trigger, long timestamp, Mutation mutation)
+    {
+        ColumnFamily cells = mutation.addOrGet(Triggers);
+        Composite prefix = Triggers.comparator.make(table.cfName, trigger.name);
+        CFRowAdder adder = new CFRowAdder(cells, prefix, timestamp);
+        adder.addMapEntry("trigger_options", "class", trigger.classOption);
+    }
+
+    private static void dropTriggerFromSchemaMutation(CFMetaData table, TriggerDefinition trigger, long timestamp, Mutation mutation)
+    {
+        ColumnFamily cells = mutation.addOrGet(Triggers);
+        int ldt = (int) (System.currentTimeMillis() / 1000);
+
+        Composite prefix = Triggers.comparator.make(table.cfName, trigger.name);
+        cells.addAtom(new RangeTombstone(prefix, prefix.end(), timestamp, ldt));
+    }
+
+    /**
+     * Deserialize triggers from storage-level representation.
+     *
+     * @param partition storage-level partition containing the trigger definitions
+     * @return the list of processed TriggerDefinitions
+     */
+    private static List<TriggerDefinition> createTriggersFromTriggersPartition(Row partition)
+    {
+        List<TriggerDefinition> triggers = new ArrayList<>();
+        String query = String.format("SELECT * FROM %s.%s", SystemKeyspace.NAME, TRIGGERS);
+        for (UntypedResultSet.Row row : QueryProcessor.resultify(query, partition))
+        {
+            String name = row.getString("trigger_name");
+            String classOption = row.getMap("trigger_options", UTF8Type.instance, UTF8Type.instance).get("class");
+            triggers.add(new TriggerDefinition(name, classOption));
+        }
+        return triggers;
+    }
+
+    /*
+     * UDF metadata serialization/deserialization.
+     */
+
+    public static Mutation makeCreateFunctionMutation(KSMetaData keyspace, UDFunction function, long timestamp)
+    {
+        // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
+        Mutation mutation = makeCreateKeyspaceMutation(keyspace, timestamp, false);
+        addFunctionToSchemaMutation(function, timestamp, mutation);
+        return mutation;
+    }
+
+    private static void addFunctionToSchemaMutation(UDFunction function, long timestamp, Mutation mutation)
+    {
+        ColumnFamily cells = mutation.addOrGet(Functions);
+        Composite prefix = Functions.comparator.make(function.name().name, functionSignatureWithTypes(function));
+        CFRowAdder adder = new CFRowAdder(cells, prefix, timestamp);
+
+        adder.resetCollection("argument_names");
+        adder.resetCollection("argument_types");
+
+        for (int i = 0; i < function.argNames().size(); i++)
+        {
+            adder.addListEntry("argument_names", function.argNames().get(i).bytes);
+            adder.addListEntry("argument_types", function.argTypes().get(i).toString());
+        }
+
+        adder.add("body", function.body());
+        adder.add("language", function.language());
+        adder.add("return_type", function.returnType().toString());
+        adder.add("called_on_null_input", function.isCalledOnNullInput());
+    }
+
+    public static Mutation makeDropFunctionMutation(KSMetaData keyspace, UDFunction function, long timestamp)
+    {
+        // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
+        Mutation mutation = makeCreateKeyspaceMutation(keyspace, timestamp, false);
+
+        ColumnFamily cells = mutation.addOrGet(Functions);
+        int ldt = (int) (System.currentTimeMillis() / 1000);
+
+        Composite prefix = Functions.comparator.make(function.name().name, functionSignatureWithTypes(function));
+        cells.addAtom(new RangeTombstone(prefix, prefix.end(), timestamp, ldt));
+
+        return mutation;
+    }
+
+    private static Map<ByteBuffer, UDFunction> createFunctionsFromFunctionsPartition(Row partition)
+    {
+        Map<ByteBuffer, UDFunction> functions = new HashMap<>();
+        String query = String.format("SELECT * FROM %s.%s", SystemKeyspace.NAME, FUNCTIONS);
+        for (UntypedResultSet.Row row : QueryProcessor.resultify(query, partition))
+        {
+            UDFunction function = createFunctionFromFunctionRow(row);
+            functions.put(functionSignatureWithNameAndTypes(function), function);
+        }
+        return functions;
+    }
+
+    private static UDFunction createFunctionFromFunctionRow(UntypedResultSet.Row row)
+    {
+        String ksName = row.getString("keyspace_name");
+        String functionName = row.getString("function_name");
+        FunctionName name = new FunctionName(ksName, functionName);
+
+        List<ColumnIdentifier> argNames = new ArrayList<>();
+        if (row.has("argument_names"))
+            for (String arg : row.getList("argument_names", UTF8Type.instance))
+                argNames.add(new ColumnIdentifier(arg, true));
+
+        List<AbstractType<?>> argTypes = new ArrayList<>();
+        if (row.has("argument_types"))
+            for (String type : row.getList("argument_types", UTF8Type.instance))
+                argTypes.add(parseType(type));
+
+        AbstractType<?> returnType = parseType(row.getString("return_type"));
+
+        String language = row.getString("language");
+        String body = row.getString("body");
+        boolean calledOnNullInput = row.getBoolean("called_on_null_input");
+
+        org.apache.cassandra.cql3.functions.Function existing = org.apache.cassandra.cql3.functions.Functions.find(name, argTypes);
+        if (existing instanceof UDFunction)
+        {
+            // This check prevents duplicate compilation of effectively the same UDF.
+            // Duplicate compilation attempts can occur on the coordinator node handling the CREATE FUNCTION
+            // statement, since CreateFunctionStatement needs to execute UDFunction.create but schema migration
+            // also needs that (since it needs to handle its own change).
+            UDFunction udf = (UDFunction) existing;
+            if (udf.argNames().equals(argNames) && // arg types checked in Functions.find call
+                udf.returnType().equals(returnType) &&
+                !udf.isAggregate() &&
+                udf.language().equals(language) &&
+                udf.body().equals(body) &&
+                udf.isCalledOnNullInput() == calledOnNullInput)
+            {
+                logger.trace("Skipping duplicate compilation of already existing UDF {}", name);
+                return udf;
+            }
+        }
+
+        try
+        {
+            return UDFunction.create(name, argNames, argTypes, returnType, calledOnNullInput, language, body);
+        }
+        catch (InvalidRequestException e)
+        {
+            logger.error(String.format("Cannot load function '%s' from schema: this function won't be available (on this node)", name), e);
+            return UDFunction.createBrokenFunction(name, argNames, argTypes, returnType, calledOnNullInput, language, body, e);
+        }
+    }
+
+    /*
+     * Aggregate UDF metadata serialization/deserialization.
+     */
+
+    public static Mutation makeCreateAggregateMutation(KSMetaData keyspace, UDAggregate aggregate, long timestamp)
+    {
+        // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
+        Mutation mutation = makeCreateKeyspaceMutation(keyspace, timestamp, false);
+        addAggregateToSchemaMutation(aggregate, timestamp, mutation);
+        return mutation;
+    }
+
+    private static void addAggregateToSchemaMutation(UDAggregate aggregate, long timestamp, Mutation mutation)
+    {
+        ColumnFamily cells = mutation.addOrGet(Aggregates);
+        Composite prefix = Aggregates.comparator.make(aggregate.name().name, functionSignatureWithTypes(aggregate));
+        CFRowAdder adder = new CFRowAdder(cells, prefix, timestamp);
+
+        adder.resetCollection("argument_types");
+        adder.add("return_type", aggregate.returnType().toString());
+        adder.add("state_func", aggregate.stateFunction().name().name);
+        adder.add("state_type", aggregate.stateType().toString());
+        adder.add("final_func", aggregate.finalFunction() != null ? aggregate.finalFunction().name().name : null);
+        adder.add("initcond", aggregate.initialCondition() != null ? aggregate.initialCondition() : null);
+
+        for (AbstractType<?> argType : aggregate.argTypes())
+            adder.addListEntry("argument_types", argType.toString());
+    }
+
+    private static Map<ByteBuffer, UDAggregate> createAggregatesFromAggregatesPartition(Row partition)
+    {
+        Map<ByteBuffer, UDAggregate> aggregates = new HashMap<>();
+        String query = String.format("SELECT * FROM %s.%s", SystemKeyspace.NAME, AGGREGATES);
+        for (UntypedResultSet.Row row : QueryProcessor.resultify(query, partition))
+        {
+            UDAggregate aggregate = createAggregateFromAggregateRow(row);
+            aggregates.put(functionSignatureWithNameAndTypes(aggregate), aggregate);
+        }
+        return aggregates;
+    }
+
+    private static UDAggregate createAggregateFromAggregateRow(UntypedResultSet.Row row)
+    {
+        String ksName = row.getString("keyspace_name");
+        String functionName = row.getString("aggregate_name");
+        FunctionName name = new FunctionName(ksName, functionName);
+
+        List<String> types = row.getList("argument_types", UTF8Type.instance);
+
+        List<AbstractType<?>> argTypes;
+        if (types == null)
+        {
+            argTypes = Collections.emptyList();
+        }
+        else
+        {
+            argTypes = new ArrayList<>(types.size());
+            for (String type : types)
+                argTypes.add(parseType(type));
+        }
+
+        AbstractType<?> returnType = parseType(row.getString("return_type"));
+
+        FunctionName stateFunc = new FunctionName(ksName, row.getString("state_func"));
+        FunctionName finalFunc = row.has("final_func") ? new FunctionName(ksName, row.getString("final_func")) : null;
+        AbstractType<?> stateType = row.has("state_type") ? parseType(row.getString("state_type")) : null;
+        ByteBuffer initcond = row.has("initcond") ? row.getBytes("initcond") : null;
+
+        try
+        {
+            return UDAggregate.create(name, argTypes, returnType, stateFunc, finalFunc, stateType, initcond);
+        }
+        catch (InvalidRequestException reason)
+        {
+            return UDAggregate.createBroken(name, argTypes, returnType, initcond, reason);
+        }
+    }
+
+    public static Mutation makeDropAggregateMutation(KSMetaData keyspace, UDAggregate aggregate, long timestamp)
+    {
+        // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
+        Mutation mutation = makeCreateKeyspaceMutation(keyspace, timestamp, false);
+
+        ColumnFamily cells = mutation.addOrGet(Aggregates);
+        int ldt = (int) (System.currentTimeMillis() / 1000);
+
+        Composite prefix = Aggregates.comparator.make(aggregate.name().name, functionSignatureWithTypes(aggregate));
+        cells.addAtom(new RangeTombstone(prefix, prefix.end(), timestamp, ldt));
+
+        return mutation;
+    }
+
+    private static AbstractType<?> parseType(String str)
+    {
+        return TypeParser.parse(str);
+    }
+
+    // We allow method overloads, so a function is not uniquely identified by its name only, but
+    // also by its argument types. To distinguish overloads of given function name in the schema
+    // we use a "signature" which is just a list of it's CQL argument types (we could replace that by
+    // using a "signature" UDT that would be comprised of the function name and argument types,
+    // which we could then use as clustering column. But as we haven't yet used UDT in system tables,
+    // We'll leave that decision to #6717).
+    public static ByteBuffer functionSignatureWithTypes(AbstractFunction fun)
+    {
+        ListType<String> list = ListType.getInstance(UTF8Type.instance, false);
+        List<String> strList = new ArrayList<>(fun.argTypes().size());
+        for (AbstractType<?> argType : fun.argTypes())
+            strList.add(argType.asCQL3Type().toString());
+        return list.decompose(strList);
+    }
+
+    public static ByteBuffer functionSignatureWithNameAndTypes(AbstractFunction fun)
+    {
+        ListType<String> list = ListType.getInstance(UTF8Type.instance, false);
+        List<String> strList = new ArrayList<>(fun.argTypes().size() + 2);
+        strList.add(fun.name().keyspace);
+        strList.add(fun.name().name);
+        for (AbstractType<?> argType : fun.argTypes())
+            strList.add(argType.asCQL3Type().toString());
+        return list.decompose(strList);
+    }
+
+}
diff --git a/src/java/org/apache/cassandra/security/SSLFactory.java b/src/java/org/apache/cassandra/security/SSLFactory.java
index 956ba67..a327de9 100644
--- a/src/java/org/apache/cassandra/security/SSLFactory.java
+++ b/src/java/org/apache/cassandra/security/SSLFactory.java
@@ -24,9 +24,10 @@
 import java.net.InetSocketAddress;
 import java.security.KeyStore;
 import java.security.cert.X509Certificate;
+import java.util.Arrays;
 import java.util.Date;
 import java.util.Enumeration;
-import java.util.Set;
+import java.util.List;
 
 import javax.net.ssl.KeyManagerFactory;
 import javax.net.ssl.SSLContext;
@@ -37,10 +38,12 @@
 
 import org.apache.cassandra.config.EncryptionOptions;
 import org.apache.cassandra.io.util.FileUtils;
-import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.base.Predicates;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
 import com.google.common.collect.Sets;
 
 /**
@@ -58,8 +61,8 @@
         SSLContext ctx = createSSLContext(options, true);
         SSLServerSocket serverSocket = (SSLServerSocket)ctx.getServerSocketFactory().createServerSocket();
         serverSocket.setReuseAddress(true);
-        String[] suits = filterCipherSuites(serverSocket.getSupportedCipherSuites(), options.cipher_suites);
-        serverSocket.setEnabledCipherSuites(suits);
+        String[] suites = filterCipherSuites(serverSocket.getSupportedCipherSuites(), options.cipher_suites);
+        serverSocket.setEnabledCipherSuites(suites);
         serverSocket.setNeedClientAuth(options.require_client_auth);
         serverSocket.setEnabledProtocols(ACCEPTED_PROTOCOLS);
         serverSocket.bind(new InetSocketAddress(address, port), 500);
@@ -71,8 +74,8 @@
     {
         SSLContext ctx = createSSLContext(options, true);
         SSLSocket socket = (SSLSocket) ctx.getSocketFactory().createSocket(address, port, localAddress, localPort);
-        String[] suits = filterCipherSuites(socket.getSupportedCipherSuites(), options.cipher_suites);
-        socket.setEnabledCipherSuites(suits);
+        String[] suites = filterCipherSuites(socket.getSupportedCipherSuites(), options.cipher_suites);
+        socket.setEnabledCipherSuites(suites);
         socket.setEnabledProtocols(ACCEPTED_PROTOCOLS);
         return socket;
     }
@@ -82,8 +85,8 @@
     {
         SSLContext ctx = createSSLContext(options, true);
         SSLSocket socket = (SSLSocket) ctx.getSocketFactory().createSocket(address, port);
-        String[] suits = filterCipherSuites(socket.getSupportedCipherSuites(), options.cipher_suites);
-        socket.setEnabledCipherSuites(suits);
+        String[] suites = filterCipherSuites(socket.getSupportedCipherSuites(), options.cipher_suites);
+        socket.setEnabledCipherSuites(suites);
         socket.setEnabledProtocols(ACCEPTED_PROTOCOLS);
         return socket;
     }
@@ -93,12 +96,13 @@
     {
         SSLContext ctx = createSSLContext(options, true);
         SSLSocket socket = (SSLSocket) ctx.getSocketFactory().createSocket();
-        String[] suits = filterCipherSuites(socket.getSupportedCipherSuites(), options.cipher_suites);
-        socket.setEnabledCipherSuites(suits);
+        String[] suites = filterCipherSuites(socket.getSupportedCipherSuites(), options.cipher_suites);
+        socket.setEnabledCipherSuites(suites);
         socket.setEnabledProtocols(ACCEPTED_PROTOCOLS);
         return socket;
     }
 
+    @SuppressWarnings("resource")
     public static SSLContext createSSLContext(EncryptionOptions options, boolean buildTruststore) throws IOException
     {
         FileInputStream tsf = null;
@@ -154,12 +158,18 @@
         return ctx;
     }
 
-    private static String[] filterCipherSuites(String[] supported, String[] desired)
+    public static String[] filterCipherSuites(String[] supported, String[] desired)
     {
-        Set<String> des = Sets.newHashSet(desired);
-        Set<String> toReturn = Sets.intersection(Sets.newHashSet(supported), des);
-        if (des.size() > toReturn.size())
-            logger.warn("Filtering out {} as it isnt supported by the socket", StringUtils.join(Sets.difference(des, toReturn), ","));
-        return toReturn.toArray(new String[toReturn.size()]);
+        if (Arrays.equals(supported, desired))
+            return desired;
+        List<String> ldesired = Arrays.asList(desired);
+        ImmutableSet<String> ssupported = ImmutableSet.copyOf(supported);
+        String[] ret = Iterables.toArray(Iterables.filter(ldesired, Predicates.in(ssupported)), String.class);
+        if (desired.length > ret.length && logger.isWarnEnabled())
+        {
+            Iterable<String> missing = Iterables.filter(ldesired, Predicates.not(Predicates.in(Sets.newHashSet(ret))));
+            logger.warn("Filtering out {} as it isn't supported by the socket", Iterables.toString(missing));
+        }
+        return ret;
     }
 }
diff --git a/src/java/org/apache/cassandra/serializers/BooleanSerializer.java b/src/java/org/apache/cassandra/serializers/BooleanSerializer.java
index dffecd6..0d6580e 100644
--- a/src/java/org/apache/cassandra/serializers/BooleanSerializer.java
+++ b/src/java/org/apache/cassandra/serializers/BooleanSerializer.java
@@ -30,7 +30,7 @@
 
     public Boolean deserialize(ByteBuffer bytes)
     {
-        if (bytes.remaining() == 0)
+        if (bytes == null || bytes.remaining() == 0)
             return null;
 
         byte value = bytes.get(bytes.position());
diff --git a/src/java/org/apache/cassandra/serializers/ByteSerializer.java b/src/java/org/apache/cassandra/serializers/ByteSerializer.java
new file mode 100644
index 0000000..8c736cb
--- /dev/null
+++ b/src/java/org/apache/cassandra/serializers/ByteSerializer.java
@@ -0,0 +1,54 @@
+/*
+ * 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.serializers;
+
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import java.nio.ByteBuffer;
+
+public class ByteSerializer implements TypeSerializer<Byte>
+{
+    public static final ByteSerializer instance = new ByteSerializer();
+
+    public Byte deserialize(ByteBuffer bytes)
+    {
+        return bytes.remaining() == 0 ? null : bytes.get(bytes.position());
+    }
+
+    public ByteBuffer serialize(Byte value)
+    {
+        return value == null ? ByteBufferUtil.EMPTY_BYTE_BUFFER : ByteBuffer.allocate(1).put(0, value);
+    }
+
+    public void validate(ByteBuffer bytes) throws MarshalException
+    {
+        if (bytes.remaining() != 1)
+            throw new MarshalException(String.format("Expected 1 byte for a tinyint (%d)", bytes.remaining()));
+    }
+
+    public String toString(Byte value)
+    {
+        return value == null ? "" : String.valueOf(value);
+    }
+
+    public Class<Byte> getType()
+    {
+        return Byte.class;
+    }
+}
diff --git a/src/java/org/apache/cassandra/serializers/CollectionSerializer.java b/src/java/org/apache/cassandra/serializers/CollectionSerializer.java
index 29ae2fd..5fb3e0a 100644
--- a/src/java/org/apache/cassandra/serializers/CollectionSerializer.java
+++ b/src/java/org/apache/cassandra/serializers/CollectionSerializer.java
@@ -92,9 +92,9 @@
         return version >= Server.VERSION_3 ? 4 : 2;
     }
 
-    protected static void writeValue(ByteBuffer output, ByteBuffer value, int version)
+    public static void writeValue(ByteBuffer output, ByteBuffer value, int version)
     {
-        if (version >= 3)
+        if (version >= Server.VERSION_3)
         {
             if (value == null)
             {
@@ -129,7 +129,7 @@
         }
     }
 
-    protected static int sizeOfValue(ByteBuffer value, int version)
+    public static int sizeOfValue(ByteBuffer value, int version)
     {
         if (version >= Server.VERSION_3)
         {
diff --git a/src/java/org/apache/cassandra/serializers/ListSerializer.java b/src/java/org/apache/cassandra/serializers/ListSerializer.java
index aeee2b9..d2d0610 100644
--- a/src/java/org/apache/cassandra/serializers/ListSerializer.java
+++ b/src/java/org/apache/cassandra/serializers/ListSerializer.java
@@ -84,7 +84,15 @@
         {
             ByteBuffer input = bytes.duplicate();
             int n = readCollectionSize(input, version);
-            List<T> l = new ArrayList<T>(n);
+
+            if (n < 0)
+                throw new MarshalException("The data cannot be deserialized as a list");
+
+            // If the received bytes are not corresponding to a list, n might be a huge number.
+            // In such a case we do not want to initialize the list with that size as it can result
+            // in an OOM (see CASSANDRA-12618). On the other hand we do not want to have to resize the list
+            // if we can avoid it, so we put a reasonable limit on the initialCapacity.
+            List<T> l = new ArrayList<T>(Math.min(n, 256));
             for (int i = 0; i < n; i++)
             {
                 // We can have nulls in lists that are used for IN values
diff --git a/src/java/org/apache/cassandra/serializers/MapSerializer.java b/src/java/org/apache/cassandra/serializers/MapSerializer.java
index bc8e509..66831a3 100644
--- a/src/java/org/apache/cassandra/serializers/MapSerializer.java
+++ b/src/java/org/apache/cassandra/serializers/MapSerializer.java
@@ -24,7 +24,6 @@
 
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.transport.Server;
-import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.Pair;
 
 public class MapSerializer<K, V> extends CollectionSerializer<Map<K, V>>
@@ -95,7 +94,15 @@
         {
             ByteBuffer input = bytes.duplicate();
             int n = readCollectionSize(input, version);
-            Map<K, V> m = new LinkedHashMap<K, V>(n);
+
+            if (n < 0)
+                throw new MarshalException("The data cannot be deserialized as a map");
+
+            // If the received bytes are not corresponding to a map, n might be a huge number.
+            // In such a case we do not want to initialize the map with that initialCapacity as it can result
+            // in an OOM when put is called (see CASSANDRA-12618). On the other hand we do not want to have to resize
+            // the map if we can avoid it, so we put a reasonable limit on the initialCapacity.
+            Map<K, V> m = new LinkedHashMap<K, V>(Math.min(n, 256));
             for (int i = 0; i < n; i++)
             {
                 ByteBuffer kbb = readValue(input, version);
diff --git a/src/java/org/apache/cassandra/serializers/SetSerializer.java b/src/java/org/apache/cassandra/serializers/SetSerializer.java
index de05a66..4aaf36a 100644
--- a/src/java/org/apache/cassandra/serializers/SetSerializer.java
+++ b/src/java/org/apache/cassandra/serializers/SetSerializer.java
@@ -22,8 +22,6 @@
 import java.nio.ByteBuffer;
 import java.util.*;
 
-import org.apache.cassandra.utils.ByteBufferUtil;
-
 public class SetSerializer<T> extends CollectionSerializer<Set<T>>
 {
     // interning instances
@@ -83,7 +81,16 @@
         {
             ByteBuffer input = bytes.duplicate();
             int n = readCollectionSize(input, version);
-            Set<T> l = new LinkedHashSet<T>(n);
+
+            if (n < 0)
+                throw new MarshalException("The data cannot be deserialized as a set");
+
+            // If the received bytes are not corresponding to a set, n might be a huge number.
+            // In such a case we do not want to initialize the set with that initialCapacity as it can result
+            // in an OOM when add is called (see CASSANDRA-12618). On the other hand we do not want to have to resize
+            // the set if we can avoid it, so we put a reasonable limit on the initialCapacity.
+            Set<T> l = new LinkedHashSet<T>(Math.min(n, 256));
+
             for (int i = 0; i < n; i++)
             {
                 ByteBuffer databb = readValue(input, version);
@@ -96,7 +103,7 @@
         }
         catch (BufferUnderflowException e)
         {
-            throw new MarshalException("Not enough bytes to read a list");
+            throw new MarshalException("Not enough bytes to read a set");
         }
     }
 
diff --git a/src/java/org/apache/cassandra/serializers/ShortSerializer.java b/src/java/org/apache/cassandra/serializers/ShortSerializer.java
new file mode 100644
index 0000000..f12affa
--- /dev/null
+++ b/src/java/org/apache/cassandra/serializers/ShortSerializer.java
@@ -0,0 +1,54 @@
+/*
+ * 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.serializers;
+
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import java.nio.ByteBuffer;
+
+public class ShortSerializer implements TypeSerializer<Short>
+{
+    public static final ShortSerializer instance = new ShortSerializer();
+
+    public Short deserialize(ByteBuffer bytes)
+    {
+        return bytes.remaining() == 0 ? null : ByteBufferUtil.toShort(bytes);
+    }
+
+    public ByteBuffer serialize(Short value)
+    {
+        return value == null ? ByteBufferUtil.EMPTY_BYTE_BUFFER : ByteBufferUtil.bytes(value.shortValue());
+    }
+
+    public void validate(ByteBuffer bytes) throws MarshalException
+    {
+        if (bytes.remaining() != 2)
+            throw new MarshalException(String.format("Expected 2 bytes for a smallint (%d)", bytes.remaining()));
+    }
+
+    public String toString(Short value)
+    {
+        return value == null ? "" : String.valueOf(value);
+    }
+
+    public Class<Short> getType()
+    {
+        return Short.class;
+    }
+}
diff --git a/src/java/org/apache/cassandra/serializers/SimpleDateSerializer.java b/src/java/org/apache/cassandra/serializers/SimpleDateSerializer.java
new file mode 100644
index 0000000..075094c
--- /dev/null
+++ b/src/java/org/apache/cassandra/serializers/SimpleDateSerializer.java
@@ -0,0 +1,127 @@
+/*
+ * 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.serializers;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Pattern;
+
+import org.joda.time.DateTime;
+import org.joda.time.DateTimeZone;
+import org.joda.time.LocalDate;
+import org.joda.time.format.DateTimeFormat;
+import org.joda.time.format.DateTimeFormatter;
+
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+// For byte-order comparability, we shift by Integer.MIN_VALUE and treat the data as an unsigned integer ranging from
+// min date to max date w/epoch sitting in the center @ 2^31
+public class SimpleDateSerializer implements TypeSerializer<Integer>
+{
+    private static final DateTimeFormatter formatter = DateTimeFormat.forPattern("yyyy-MM-dd").withZone(DateTimeZone.UTC);
+    private static final long minSupportedDateMillis = TimeUnit.DAYS.toMillis(Integer.MIN_VALUE);
+    private static final long maxSupportedDateMillis = TimeUnit.DAYS.toMillis(Integer.MAX_VALUE);
+    private static final long maxSupportedDays = (long)Math.pow(2,32) - 1;
+    private static final long byteOrderShift = (long)Math.pow(2,31) * 2;
+
+    private static final Pattern rawPattern = Pattern.compile("^-?\\d+$");
+    public static final SimpleDateSerializer instance = new SimpleDateSerializer();
+
+    public Integer deserialize(ByteBuffer bytes)
+    {
+        return bytes.remaining() == 0 ? null : ByteBufferUtil.toInt(bytes);
+    }
+
+    public ByteBuffer serialize(Integer value)
+    {
+        return value == null ? ByteBufferUtil.EMPTY_BYTE_BUFFER : ByteBufferUtil.bytes(value);
+    }
+
+    public static int dateStringToDays(String source) throws MarshalException
+    {
+        // Raw day value in unsigned int form, epoch @ 2^31
+        if (rawPattern.matcher(source).matches())
+        {
+            try
+            {
+                long result = Long.parseLong(source);
+
+                if (result < 0 || result > maxSupportedDays)
+                    throw new NumberFormatException("Input out of bounds: " + source);
+
+                // Shift > epoch days into negative portion of Integer result for byte order comparability
+                if (result >= Integer.MAX_VALUE)
+                    result -= byteOrderShift;
+
+                return (int) result;
+            }
+            catch (NumberFormatException e)
+            {
+                throw new MarshalException(String.format("Unable to make unsigned int (for date) from: '%s'", source), e);
+            }
+        }
+
+        // Attempt to parse as date string
+        try
+        {
+            DateTime parsed = formatter.parseDateTime(source);
+            long millis = parsed.getMillis();
+            if (millis < minSupportedDateMillis)
+                throw new MarshalException(String.format("Input date %s is less than min supported date %s", source, new LocalDate(minSupportedDateMillis).toString()));
+            if (millis > maxSupportedDateMillis)
+                throw new MarshalException(String.format("Input date %s is greater than max supported date %s", source, new LocalDate(maxSupportedDateMillis).toString()));
+
+            return timeInMillisToDay(millis);
+        }
+        catch (IllegalArgumentException e1)
+        {
+            throw new MarshalException(String.format("Unable to coerce '%s' to a formatted date (long)", source), e1);
+        }
+    }
+
+    public static int timeInMillisToDay(long millis)
+    {
+        Integer result = (int) TimeUnit.MILLISECONDS.toDays(millis);
+        result -= Integer.MIN_VALUE;
+        return result;
+    }
+
+    public static long dayToTimeInMillis(int days)
+    {
+        return TimeUnit.DAYS.toMillis(days - Integer.MIN_VALUE);
+    }
+
+    public void validate(ByteBuffer bytes) throws MarshalException
+    {
+        if (bytes.remaining() != 4)
+            throw new MarshalException(String.format("Expected 4 byte long for date (%d)", bytes.remaining()));
+    }
+
+    public String toString(Integer value)
+    {
+        if (value == null)
+            return "";
+
+        return formatter.print(new LocalDate(dayToTimeInMillis(value), DateTimeZone.UTC));
+    }
+
+    public Class<Integer> getType()
+    {
+        return Integer.class;
+    }
+}
diff --git a/src/java/org/apache/cassandra/serializers/TimeSerializer.java b/src/java/org/apache/cassandra/serializers/TimeSerializer.java
new file mode 100644
index 0000000..1857be3
--- /dev/null
+++ b/src/java/org/apache/cassandra/serializers/TimeSerializer.java
@@ -0,0 +1,195 @@
+/*
+ * 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.serializers;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.TimeUnit;
+import java.util.regex.Pattern;
+
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class TimeSerializer implements TypeSerializer<Long>
+{
+    public static final Pattern timePattern = Pattern.compile("^-?\\d+$");
+    public static final TimeSerializer instance = new TimeSerializer();
+
+    public Long deserialize(ByteBuffer bytes)
+    {
+        return bytes.remaining() == 0 ? null : ByteBufferUtil.toLong(bytes);
+    }
+
+    public ByteBuffer serialize(Long value)
+    {
+        return value == null ? ByteBufferUtil.EMPTY_BYTE_BUFFER : ByteBufferUtil.bytes(value);
+    }
+
+    public static Long timeStringToLong(String source) throws MarshalException
+    {
+        // nano since start of day, raw
+        if (timePattern.matcher(source).matches())
+        {
+            try
+            {
+                long result = Long.parseLong(source);
+                if (result < 0 || result >= TimeUnit.DAYS.toNanos(1))
+                    throw new NumberFormatException("Input long out of bounds: " + source);
+                return result;
+            }
+            catch (NumberFormatException e)
+            {
+                throw new MarshalException(String.format("Unable to make long (for time) from: '%s'", source), e);
+            }
+        }
+
+        // Last chance, attempt to parse as time string
+        try
+        {
+            return parseTimeStrictly(source);
+        }
+        catch (IllegalArgumentException e1)
+        {
+            throw new MarshalException(String.format("(TimeType) Unable to coerce '%s' to a formatted time (long)", source), e1);
+        }
+    }
+
+    public void validate(ByteBuffer bytes) throws MarshalException
+    {
+        if (bytes.remaining() != 8)
+            throw new MarshalException(String.format("Expected 8 byte long for time (%d)", bytes.remaining()));
+    }
+
+    public String toString(Long value)
+    {
+        if (value == null)
+            return "null";
+
+        int nano = (int)(value % 1000);
+        value -= nano;
+        value /= 1000;
+        int micro = (int)(value % 1000);
+        value -= micro;
+        value /= 1000;
+        int milli = (int)(value % 1000);
+        value -= milli;
+        value /= 1000;
+        int seconds = (int)(value % 60);
+        value -= seconds;
+        value /= 60;
+        int minutes = (int)(value % 60);
+        value -= minutes;
+        value /= 60;
+        int hours = (int)(value % 24);
+        value -= hours;
+        value /= 24;
+        assert(value == 0);
+
+        StringBuilder sb = new StringBuilder();
+        leftPadZeros(hours, 2, sb);
+        sb.append(":");
+        leftPadZeros(minutes, 2, sb);
+        sb.append(":");
+        leftPadZeros(seconds, 2, sb);
+        sb.append(".");
+        leftPadZeros(milli, 3, sb);
+        leftPadZeros(micro, 3, sb);
+        leftPadZeros(nano, 3, sb);
+        return sb.toString();
+    }
+
+    private void leftPadZeros(int value, int digits, StringBuilder sb)
+    {
+        for (int i = 1; i < digits; ++i)
+        {
+            if (value < Math.pow(10, i))
+                sb.append("0");
+        }
+        sb.append(value);
+    }
+
+    public Class<Long> getType()
+    {
+        return Long.class;
+    }
+
+    // Time specific parsing loosely based on java.sql.Timestamp
+    private static Long parseTimeStrictly(String s) throws IllegalArgumentException
+    {
+        String nanos_s;
+
+        long hour;
+        long minute;
+        long second;
+        long a_nanos = 0;
+
+        String formatError = "Timestamp format must be hh:mm:ss[.fffffffff]";
+        String zeros = "000000000";
+
+        if (s == null)
+            throw new java.lang.IllegalArgumentException(formatError);
+        s = s.trim();
+
+        // Parse the time
+        int firstColon = s.indexOf(':');
+        int secondColon = s.indexOf(':', firstColon+1);
+
+        // Convert the time; default missing nanos
+        if (firstColon > 0 && secondColon > 0 && secondColon < s.length() - 1)
+        {
+            int period = s.indexOf('.', secondColon+1);
+            hour = Integer.parseInt(s.substring(0, firstColon));
+            if (hour < 0 || hour >= 24)
+                throw new IllegalArgumentException("Hour out of bounds.");
+
+            minute = Integer.parseInt(s.substring(firstColon + 1, secondColon));
+            if (minute < 0 || minute >= 60)
+                throw new IllegalArgumentException("Minute out of bounds.");
+
+            if (period > 0 && period < s.length() - 1)
+            {
+                second = Integer.parseInt(s.substring(secondColon + 1, period));
+                if (second < 0 || second >= 60)
+                    throw new IllegalArgumentException("Second out of bounds.");
+
+                nanos_s = s.substring(period + 1);
+                if (nanos_s.length() > 9)
+                    throw new IllegalArgumentException(formatError);
+                if (!Character.isDigit(nanos_s.charAt(0)))
+                    throw new IllegalArgumentException(formatError);
+                nanos_s = nanos_s + zeros.substring(0, 9 - nanos_s.length());
+                a_nanos = Integer.parseInt(nanos_s);
+            }
+            else if (period > 0)
+                throw new IllegalArgumentException(formatError);
+            else
+            {
+                second = Integer.parseInt(s.substring(secondColon + 1));
+                if (second < 0 || second >= 60)
+                    throw new IllegalArgumentException("Second out of bounds.");
+            }
+        }
+        else
+            throw new IllegalArgumentException(formatError);
+
+        long rawTime = 0;
+        rawTime += TimeUnit.HOURS.toNanos(hour);
+        rawTime += TimeUnit.MINUTES.toNanos(minute);
+        rawTime += TimeUnit.SECONDS.toNanos(second);
+        rawTime += a_nanos;
+        return rawTime;
+    }
+}
diff --git a/src/java/org/apache/cassandra/serializers/TimestampSerializer.java b/src/java/org/apache/cassandra/serializers/TimestampSerializer.java
index 5cb9586..77a5df9 100644
--- a/src/java/org/apache/cassandra/serializers/TimestampSerializer.java
+++ b/src/java/org/apache/cassandra/serializers/TimestampSerializer.java
@@ -22,45 +22,70 @@
 import java.nio.ByteBuffer;
 import java.text.SimpleDateFormat;
 import java.text.ParseException;
-import java.util.Date;
+import java.util.*;
 import java.util.regex.Pattern;
 
 import org.apache.commons.lang3.time.DateUtils;
 
 public class TimestampSerializer implements TypeSerializer<Date>
 {
+
+    //NOTE: This list is used below and if you change the order
+    //      you need to update the default format and json formats in the code below.
     private static final String[] dateStringPatterns = new String[] {
             "yyyy-MM-dd HH:mm",
             "yyyy-MM-dd HH:mm:ss",
+            "yyyy-MM-dd HH:mm z",
+            "yyyy-MM-dd HH:mm zz",
+            "yyyy-MM-dd HH:mm zzz",
             "yyyy-MM-dd HH:mmX",
-            "yyyy-MM-dd HH:mmXX",
+            "yyyy-MM-dd HH:mmXX",  // DEFAULT_FORMAT
             "yyyy-MM-dd HH:mmXXX",
+            "yyyy-MM-dd HH:mm:ss",
+            "yyyy-MM-dd HH:mm:ss z",
+            "yyyy-MM-dd HH:mm:ss zz",
+            "yyyy-MM-dd HH:mm:ss zzz",
             "yyyy-MM-dd HH:mm:ssX",
             "yyyy-MM-dd HH:mm:ssXX",
             "yyyy-MM-dd HH:mm:ssXXX",
             "yyyy-MM-dd HH:mm:ss.SSS",
-            "yyyy-MM-dd HH:mm:ss.SSSX",
+            "yyyy-MM-dd HH:mm:ss.SSS z",
+            "yyyy-MM-dd HH:mm:ss.SSS zz",
+            "yyyy-MM-dd HH:mm:ss.SSS zzz",
+            "yyyy-MM-dd HH:mm:ss.SSSX", // TO_JSON_FORMAT
             "yyyy-MM-dd HH:mm:ss.SSSXX",
             "yyyy-MM-dd HH:mm:ss.SSSXXX",
             "yyyy-MM-dd'T'HH:mm",
+            "yyyy-MM-dd'T'HH:mm z",
+            "yyyy-MM-dd'T'HH:mm zz",
+            "yyyy-MM-dd'T'HH:mm zzz",
             "yyyy-MM-dd'T'HH:mmX",
             "yyyy-MM-dd'T'HH:mmXX",
             "yyyy-MM-dd'T'HH:mmXXX",
             "yyyy-MM-dd'T'HH:mm:ss",
+            "yyyy-MM-dd'T'HH:mm:ss z",
+            "yyyy-MM-dd'T'HH:mm:ss zz",
+            "yyyy-MM-dd'T'HH:mm:ss zzz",
             "yyyy-MM-dd'T'HH:mm:ssX",
             "yyyy-MM-dd'T'HH:mm:ssXX",
             "yyyy-MM-dd'T'HH:mm:ssXXX",
             "yyyy-MM-dd'T'HH:mm:ss.SSS",
+            "yyyy-MM-dd'T'HH:mm:ss.SSS z",
+            "yyyy-MM-dd'T'HH:mm:ss.SSS zz",
+            "yyyy-MM-dd'T'HH:mm:ss.SSS zzz",
             "yyyy-MM-dd'T'HH:mm:ss.SSSX",
             "yyyy-MM-dd'T'HH:mm:ss.SSSXX",
             "yyyy-MM-dd'T'HH:mm:ss.SSSXXX",
             "yyyy-MM-dd",
+            "yyyy-MM-dd z",
+            "yyyy-MM-dd zz",
+            "yyyy-MM-dd zzz",
             "yyyy-MM-ddX",
             "yyyy-MM-ddXX",
             "yyyy-MM-ddXXX"
     };
 
-    private static final String DEFAULT_FORMAT = dateStringPatterns[3];
+    private static final String DEFAULT_FORMAT = dateStringPatterns[6];
     private static final Pattern timestampPattern = Pattern.compile("^-?\\d+$");
 
     private static final ThreadLocal<SimpleDateFormat> FORMATTER = new ThreadLocal<SimpleDateFormat>()
@@ -71,6 +96,17 @@
         }
     };
 
+    private static final String TO_JSON_FORMAT = dateStringPatterns[19];
+    private static final ThreadLocal<SimpleDateFormat> FORMATTER_TO_JSON = new ThreadLocal<SimpleDateFormat>()
+    {
+        protected SimpleDateFormat initialValue()
+        {
+            SimpleDateFormat sdf = new SimpleDateFormat(TO_JSON_FORMAT);
+            sdf.setTimeZone(TimeZone.getTimeZone("UTC"));
+            return sdf;
+        }
+    };
+    
     public static final TimestampSerializer instance = new TimestampSerializer();
 
     public Date deserialize(ByteBuffer bytes)
@@ -97,7 +133,7 @@
             }
             catch (NumberFormatException e)
             {
-                throw new MarshalException(String.format("unable to make long (for date) from: '%s'", source), e);
+                throw new MarshalException(String.format("Unable to make long (for date) from: '%s'", source), e);
             }
         }
 
@@ -108,9 +144,14 @@
         }
         catch (ParseException e1)
         {
-            throw new MarshalException(String.format("unable to coerce '%s' to a  formatted date (long)", source), e1);
+            throw new MarshalException(String.format("Unable to coerce '%s' to a formatted date (long)", source), e1);
         }
     }
+    
+    public static SimpleDateFormat getJsonDateFormatter() 
+    {
+    	return FORMATTER_TO_JSON.get();
+    }
 
     public void validate(ByteBuffer bytes) throws MarshalException
     {
diff --git a/src/java/org/apache/cassandra/serializers/TypeSerializer.java b/src/java/org/apache/cassandra/serializers/TypeSerializer.java
index 7b037c0..cddef08 100644
--- a/src/java/org/apache/cassandra/serializers/TypeSerializer.java
+++ b/src/java/org/apache/cassandra/serializers/TypeSerializer.java
@@ -35,3 +35,4 @@
 
     public Class<T> getType();
 }
+
diff --git a/src/java/org/apache/cassandra/serializers/UTF8Serializer.java b/src/java/org/apache/cassandra/serializers/UTF8Serializer.java
index be5d2d6..7c41b94 100644
--- a/src/java/org/apache/cassandra/serializers/UTF8Serializer.java
+++ b/src/java/org/apache/cassandra/serializers/UTF8Serializer.java
@@ -52,6 +52,9 @@
         // buf has already been sliced/duplicated.
         static boolean validate(ByteBuffer buf)
         {
+            if (buf == null)
+                return false;
+
             buf = buf.slice();
             int b = 0;
             State state = State.START;
@@ -71,11 +74,12 @@
                         {
                             // validate first byte of 2-byte char, 0xc2-0xdf
                             if (b == (byte) 0xc0)
-                                // speical case: modified utf8 null is 0xc080.
+                                // special case: modified utf8 null is 0xc080.
                                 state = State.TWO_80;
                             else if ((b & 0x1e) == 0)
                                 return false;
-                            state = State.TWO;
+                            else
+                                state = State.TWO;
                         }
                         else if ((b >> 4) == -2)
                         {
diff --git a/src/java/org/apache/cassandra/service/AbstractReadExecutor.java b/src/java/org/apache/cassandra/service/AbstractReadExecutor.java
index 8159cb8..2bfd059 100644
--- a/src/java/org/apache/cassandra/service/AbstractReadExecutor.java
+++ b/src/java/org/apache/cassandra/service/AbstractReadExecutor.java
@@ -37,6 +37,7 @@
 import org.apache.cassandra.db.ReadResponse;
 import org.apache.cassandra.db.Row;
 import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.exceptions.ReadFailureException;
 import org.apache.cassandra.exceptions.ReadTimeoutException;
 import org.apache.cassandra.exceptions.UnavailableException;
 import org.apache.cassandra.metrics.ReadRepairMetrics;
@@ -69,7 +70,7 @@
     {
         this.command = command;
         this.targetReplicas = targetReplicas;
-        resolver = new RowDigestResolver(command.ksName, command.key);
+        resolver = new RowDigestResolver(command.ksName, command.key, targetReplicas.size());
         traceState = Tracing.instance.get();
         handler = new ReadCallback<>(resolver, consistencyLevel, command, targetReplicas);
     }
@@ -108,7 +109,7 @@
             logger.trace("reading {} from {}", readCommand.isDigestQuery() ? "digest" : "data", endpoint);
             if (message == null)
                 message = readCommand.createMessage();
-            MessagingService.instance().sendRR(message, endpoint, handler);
+            MessagingService.instance().sendRRWithFailure(message, endpoint, handler);
         }
 
         // We delay the local (potentially blocking) read till the end to avoid stalling remote requests.
@@ -141,7 +142,7 @@
      * wait for an answer.  Blocks until success or timeout, so it is caller's
      * responsibility to call maybeTryAdditionalReplicas first.
      */
-    public Row get() throws ReadTimeoutException, DigestMismatchException
+    public Row get() throws ReadFailureException, ReadTimeoutException, DigestMismatchException
     {
         return handler.get();
     }
@@ -159,11 +160,6 @@
         // Throw UAE early if we don't have enough replicas.
         consistencyLevel.assureSufficientLiveNodes(keyspace, targetReplicas);
 
-        // Fat client. Speculating read executors need access to cfs metrics and sampled latency, and fat clients
-        // can't provide that. So, for now, fat clients will always use NeverSpeculatingReadExecutor.
-        if (StorageService.instance.isClientMode())
-            return new NeverSpeculatingReadExecutor(command, consistencyLevel, targetReplicas);
-
         if (repairDecision != ReadRepairDecision.NONE)
         {
             Tracing.trace("Read-repair {}", repairDecision);
@@ -289,7 +285,7 @@
                 if (traceState != null)
                     traceState.trace("speculating read retry on {}", extraReplica);
                 logger.trace("speculating read retry on {}", extraReplica);
-                MessagingService.instance().sendRR(retryCommand.createMessage(), extraReplica, handler);
+                MessagingService.instance().sendRRWithFailure(retryCommand.createMessage(), extraReplica, handler);
                 speculated = true;
 
                 cfs.metric.speculativeRetries.inc();
diff --git a/src/java/org/apache/cassandra/service/AbstractRowResolver.java b/src/java/org/apache/cassandra/service/AbstractRowResolver.java
index fbbf473..f362047 100644
--- a/src/java/org/apache/cassandra/service/AbstractRowResolver.java
+++ b/src/java/org/apache/cassandra/service/AbstractRowResolver.java
@@ -18,8 +18,6 @@
 package org.apache.cassandra.service;
 
 import java.nio.ByteBuffer;
-import java.util.Queue;
-import java.util.concurrent.ConcurrentLinkedQueue;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -28,20 +26,22 @@
 import org.apache.cassandra.db.ReadResponse;
 import org.apache.cassandra.db.Row;
 import org.apache.cassandra.net.MessageIn;
+import org.apache.cassandra.utils.concurrent.Accumulator;
 
 public abstract class AbstractRowResolver implements IResponseResolver<ReadResponse, Row>
 {
     protected static final Logger logger = LoggerFactory.getLogger(AbstractRowResolver.class);
 
     protected final String keyspaceName;
-    // CLQ gives us thread-safety without the overhead of guaranteeing uniqueness like a Set would
-    protected final Queue<MessageIn<ReadResponse>> replies = new ConcurrentLinkedQueue<>();
+    // Accumulator gives us non-blocking thread-safety with optimal algorithmic constraints
+    protected final Accumulator<MessageIn<ReadResponse>> replies;
     protected final DecoratedKey key;
 
-    public AbstractRowResolver(ByteBuffer key, String keyspaceName)
+    public AbstractRowResolver(ByteBuffer key, String keyspaceName, int maxResponseCount)
     {
         this.key = StorageService.getPartitioner().decorateKey(key);
         this.keyspaceName = keyspaceName;
+        this.replies = new Accumulator<>(maxResponseCount);
     }
 
     public void preprocess(MessageIn<ReadResponse> message)
diff --git a/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java b/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java
index 72e5b9c..8978034 100644
--- a/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java
+++ b/src/java/org/apache/cassandra/service/AbstractWriteResponseHandler.java
@@ -20,8 +20,11 @@
 import java.net.InetAddress;
 import java.util.Collection;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
 
 import com.google.common.collect.Iterables;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ConsistencyLevel;
@@ -29,11 +32,14 @@
 import org.apache.cassandra.db.WriteType;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.net.IAsyncCallback;
+import org.apache.cassandra.net.IAsyncCallbackWithFailure;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.utils.concurrent.SimpleCondition;
 
-public abstract class AbstractWriteResponseHandler implements IAsyncCallback
+public abstract class AbstractWriteResponseHandler<T> implements IAsyncCallbackWithFailure<T>
 {
+    protected static final Logger logger = LoggerFactory.getLogger( AbstractWriteResponseHandler.class );
+
     private final SimpleCondition condition = new SimpleCondition();
     protected final Keyspace keyspace;
     protected final long start;
@@ -42,6 +48,9 @@
     protected final Runnable callback;
     protected final Collection<InetAddress> pendingEndpoints;
     private final WriteType writeType;
+    private static final AtomicIntegerFieldUpdater<AbstractWriteResponseHandler> failuresUpdater
+        = AtomicIntegerFieldUpdater.newUpdater(AbstractWriteResponseHandler.class, "failures");
+    private volatile int failures = 0;
 
     /**
      * @param callback A callback to be called when the write is successful.
@@ -62,7 +71,7 @@
         this.writeType = writeType;
     }
 
-    public void get() throws WriteTimeoutException
+    public void get() throws WriteTimeoutException, WriteFailureException
     {
         long requestTimeout = writeType == WriteType.COUNTER
                             ? DatabaseDescriptor.getCounterWriteRpcTimeout()
@@ -82,8 +91,8 @@
 
         if (!success)
         {
-            int acks = ackCount();
             int blockedFor = totalBlockFor();
+            int acks = ackCount();
             // It's pretty unlikely, but we can race between exiting await above and here, so
             // that we could now have enough acks. In that case, we "lie" on the acks count to
             // avoid sending confusing info to the user (see CASSANDRA-6491).
@@ -91,8 +100,16 @@
                 acks = blockedFor - 1;
             throw new WriteTimeoutException(writeType, consistencyLevel, acks, blockedFor);
         }
+
+        if (totalBlockFor() + failures > totalEndpoints())
+        {
+            throw new WriteFailureException(consistencyLevel, ackCount(), failures, totalBlockFor(), writeType);
+        }
     }
 
+    /** 
+     * @return the minimum number of endpoints that must reply. 
+     */
     protected int totalBlockFor()
     {
         // During bootstrap, we have to include the pending endpoints or we may fail the consistency level
@@ -100,10 +117,29 @@
         return consistencyLevel.blockFor(keyspace) + pendingEndpoints.size();
     }
 
+    /** 
+     * @return the total number of endpoints the request has been sent to. 
+     */
+    protected int totalEndpoints()
+    {
+        return naturalEndpoints.size() + pendingEndpoints.size();
+    }
+
+    /**
+     * @return true if the message counts towards the totalBlockFor() threshold
+     */
+    protected boolean waitingFor(InetAddress from)
+    {
+        return true;
+    }
+
+    /**
+     * @return number of responses received
+     */
     protected abstract int ackCount();
 
     /** null message means "response from local write" */
-    public abstract void response(MessageIn msg);
+    public abstract void response(MessageIn<T> msg);
 
     public void assureSufficientLiveNodes() throws UnavailableException
     {
@@ -116,4 +152,17 @@
         if (callback != null)
             callback.run();
     }
+
+    @Override
+    public void onFailure(InetAddress from)
+    {
+        logger.trace("Got failure from {}", from);
+
+        int n = waitingFor(from)
+              ? failuresUpdater.incrementAndGet(this)
+              : failures;
+
+        if (totalBlockFor() + n > totalEndpoints())
+            signal();
+    }
 }
diff --git a/src/java/org/apache/cassandra/service/ActiveRepairService.java b/src/java/org/apache/cassandra/service/ActiveRepairService.java
index 4ca1e42..7d56e4b 100644
--- a/src/java/org/apache/cassandra/service/ActiveRepairService.java
+++ b/src/java/org/apache/cassandra/service/ActiveRepairService.java
@@ -17,51 +17,59 @@
  */
 package org.apache.cassandra.service;
 
+import java.io.File;
+import java.io.IOException;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.*;
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import com.google.common.base.Predicate;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Multimap;
 import com.google.common.collect.Sets;
-
+import com.google.common.util.concurrent.AbstractFuture;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
 import com.google.common.util.concurrent.MoreExecutors;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.concurrent.JMXConfigurableThreadPoolExecutor;
-import org.apache.cassandra.concurrent.NamedThreadFactory;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.ColumnFamily;
 import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.dht.Bounds;
+import org.apache.cassandra.dht.LocalPartitioner;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.gms.ApplicationState;
 import org.apache.cassandra.gms.EndpointState;
 import org.apache.cassandra.gms.FailureDetector;
 import org.apache.cassandra.gms.Gossiper;
+import org.apache.cassandra.gms.IFailureDetector;
 import org.apache.cassandra.gms.IEndpointStateChangeSubscriber;
 import org.apache.cassandra.gms.IFailureDetectionEventListener;
 import org.apache.cassandra.gms.VersionedValue;
-import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.locator.TokenMetadata;
 import org.apache.cassandra.net.IAsyncCallbackWithFailure;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.repair.*;
-import org.apache.cassandra.repair.messages.PrepareMessage;
-import org.apache.cassandra.repair.messages.RepairMessage;
-import org.apache.cassandra.repair.messages.SyncComplete;
-import org.apache.cassandra.repair.messages.ValidationComplete;
+import org.apache.cassandra.repair.AnticompactionTask;
+import org.apache.cassandra.repair.RepairJobDesc;
+import org.apache.cassandra.repair.RepairParallelism;
+import org.apache.cassandra.repair.RepairSession;
+import org.apache.cassandra.repair.messages.*;
+import org.apache.cassandra.utils.CassandraVersion;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.UUIDGen;
 import org.apache.cassandra.utils.concurrent.Ref;
-
 import org.apache.cassandra.utils.concurrent.Refs;
 
 /**
@@ -80,44 +88,40 @@
  */
 public class ActiveRepairService implements IEndpointStateChangeSubscriber, IFailureDetectionEventListener
 {
-    private static final Logger logger = LoggerFactory.getLogger(ActiveRepairService.class);
-    // singleton enforcement
-    public static final ActiveRepairService instance = new ActiveRepairService();
-
-    public static final long UNREPAIRED_SSTABLE = 0;
-
-    private static final ThreadPoolExecutor executor;
-    private boolean registeredForEndpointChanges = false;
-
-    static
-    {
-        executor = new JMXConfigurableThreadPoolExecutor(4,
-                                                         60,
-                                                         TimeUnit.SECONDS,
-                                                         new LinkedBlockingQueue<Runnable>(),
-                                                         new NamedThreadFactory("AntiEntropySessions"),
-                                                         "internal");
-    }
-
+    /**
+     * @deprecated this statuses are from the previous JMX notification service,
+     * which will be deprecated on 4.0. For statuses of the new notification
+     * service, see {@link org.apache.cassandra.streaming.StreamEvent.ProgressEvent}
+     */
+    @Deprecated
     public static enum Status
     {
         STARTED, SESSION_SUCCESS, SESSION_FAILED, FINISHED
     }
+    private boolean registeredForEndpointChanges = false;
+
+    public static CassandraVersion SUPPORTS_GLOBAL_PREPARE_FLAG_VERSION = new CassandraVersion("2.2.1");
+
+    private static final Logger logger = LoggerFactory.getLogger(ActiveRepairService.class);
+    // singleton enforcement
+    public static final ActiveRepairService instance = new ActiveRepairService(FailureDetector.instance, Gossiper.instance);
+
+    public static final long UNREPAIRED_SSTABLE = 0;
 
     /**
      * A map of active coordinator session.
      */
-    private final ConcurrentMap<UUID, RepairSession> sessions;
+    private final ConcurrentMap<UUID, RepairSession> sessions = new ConcurrentHashMap<>();
 
-    private final ConcurrentMap<UUID, ParentRepairSession> parentRepairSessions;
+    private final ConcurrentMap<UUID, ParentRepairSession> parentRepairSessions = new ConcurrentHashMap<>();
 
-    /**
-     * Protected constructor. Use ActiveRepairService.instance.
-     */
-    protected ActiveRepairService()
+    private final IFailureDetector failureDetector;
+    private final Gossiper gossiper;
+
+    public ActiveRepairService(IFailureDetector failureDetector, Gossiper gossiper)
     {
-        sessions = new ConcurrentHashMap<>();
-        parentRepairSessions = new ConcurrentHashMap<>();
+        this.failureDetector = failureDetector;
+        this.gossiper = gossiper;
     }
 
     /**
@@ -125,54 +129,73 @@
      *
      * @return Future for asynchronous call or null if there is no need to repair
      */
-    public RepairFuture submitRepairSession(UUID parentRepairSession, Range<Token> range, String keyspace, RepairParallelism parallelismDegree, Set<InetAddress> endpoints, String... cfnames)
+    public RepairSession submitRepairSession(UUID parentRepairSession,
+                                             Range<Token> range,
+                                             String keyspace,
+                                             RepairParallelism parallelismDegree,
+                                             Set<InetAddress> endpoints,
+                                             long repairedAt,
+                                             ListeningExecutorService executor,
+                                             String... cfnames)
     {
+        if (endpoints.isEmpty())
+            return null;
+
         if (cfnames.length == 0)
             return null;
-        RepairSession session = new RepairSession(parentRepairSession, range, keyspace, parallelismDegree, endpoints, cfnames);
-        if (session.endpoints.isEmpty())
-            return null;
-        RepairFuture futureTask = new RepairFuture(session);
-        executor.execute(futureTask);
-        return futureTask;
-    }
 
-    public void addToActiveSessions(RepairSession session)
-    {
+        final RepairSession session = new RepairSession(parentRepairSession, UUIDGen.getTimeUUID(), range, keyspace, parallelismDegree, endpoints, repairedAt, cfnames);
+
         sessions.put(session.getId(), session);
-        Gossiper.instance.register(session);
-        FailureDetector.instance.registerFailureDetectionEventListener(session);
+        // register listeners
+        registerOnFdAndGossip(session);
+
+        // remove session at completion
+        session.addListener(new Runnable()
+        {
+            /**
+             * When repair finished, do clean up
+             */
+            public void run()
+            {
+                sessions.remove(session.getId());
+            }
+        }, MoreExecutors.sameThreadExecutor());
+        session.start(executor);
+        return session;
     }
 
-    public void removeFromActiveSessions(RepairSession session)
+    private <T extends AbstractFuture &
+               IEndpointStateChangeSubscriber &
+               IFailureDetectionEventListener> void registerOnFdAndGossip(final T task)
     {
-        Gossiper.instance.unregister(session);
-        sessions.remove(session.getId());
+        gossiper.register(task);
+        failureDetector.registerFailureDetectionEventListener(task);
+
+        // unregister listeners at completion
+        task.addListener(new Runnable()
+        {
+            /**
+             * When repair finished, do clean up
+             */
+            public void run()
+            {
+                failureDetector.unregisterFailureDetectionEventListener(task);
+                gossiper.unregister(task);
+            }
+        }, MoreExecutors.sameThreadExecutor());
     }
 
     public synchronized void terminateSessions()
     {
+        Throwable cause = new IOException("Terminate session is called");
         for (RepairSession session : sessions.values())
         {
-            session.forceShutdown();
+            session.forceShutdown(cause);
         }
         parentRepairSessions.clear();
     }
 
-    // for testing only. Create a session corresponding to a fake request and
-    // add it to the sessions (avoid NPE in tests)
-    RepairFuture submitArtificialRepairSession(RepairJobDesc desc)
-    {
-        Set<InetAddress> neighbours = new HashSet<>();
-        Collection<Range<Token>> keyspaceLocalRanges = StorageService.instance.getLocalRanges(desc.keyspace);
-        neighbours.addAll(ActiveRepairService.getNeighbors(desc.keyspace, keyspaceLocalRanges, desc.range, null, null));
-        RepairSession session = new RepairSession(desc.parentSessionId, desc.sessionId, desc.range, desc.keyspace, RepairParallelism.PARALLEL, neighbours, new String[]{desc.columnFamily});
-        sessions.put(session.getId(), session);
-        RepairFuture futureTask = new RepairFuture(session);
-        executor.execute(futureTask);
-        return futureTask;
-    }
-
     /**
      * Return all of the neighbors with whom we share the provided range.
      *
@@ -206,7 +229,7 @@
         Set<InetAddress> neighbors = new HashSet<>(replicaSets.get(rangeSuperSet));
         neighbors.remove(FBUtilities.getBroadcastAddress());
 
-        if (dataCenters != null)
+        if (dataCenters != null && !dataCenters.isEmpty())
         {
             TokenMetadata.Topology topology = ss.getTokenMetadata().cloneOnlyTokenMap().getTopology();
             Set<InetAddress> dcEndpoints = Sets.newHashSet();
@@ -219,7 +242,7 @@
             }
             return Sets.intersection(neighbors, dcEndpoints);
         }
-        else if (hosts != null)
+        else if (hosts != null && !hosts.isEmpty())
         {
             Set<InetAddress> specifiedHost = new HashSet<>();
             for (final String host : hosts)
@@ -254,10 +277,9 @@
         return neighbors;
     }
 
-    public synchronized UUID prepareForRepair(InetAddress coordinator, Set<InetAddress> endpoints, Collection<Range<Token>> ranges, List<ColumnFamilyStore> columnFamilyStores)
+    public synchronized UUID prepareForRepair(UUID parentRepairSession, InetAddress coordinator, Set<InetAddress> endpoints, RepairOption options, List<ColumnFamilyStore> columnFamilyStores)
     {
-        UUID parentRepairSession = UUIDGen.getTimeUUID();
-        registerParentRepairSession(parentRepairSession, coordinator, columnFamilyStores, ranges);
+        registerParentRepairSession(parentRepairSession, coordinator, columnFamilyStores, options.getRanges(), options.isIncremental(), options.isGlobal());
         final CountDownLatch prepareLatch = new CountDownLatch(endpoints.size());
         final AtomicBoolean status = new AtomicBoolean(true);
         final Set<String> failedNodes = Collections.synchronizedSet(new HashSet<String>());
@@ -289,7 +311,10 @@
         {
             if (FailureDetector.instance.isAlive(neighbour))
             {
-                PrepareMessage message = new PrepareMessage(parentRepairSession, cfIds, ranges);
+                CassandraVersion peerVersion = SystemKeyspace.getReleaseVersion(neighbour);
+                boolean isGlobal = options.isGlobal() && peerVersion != null && peerVersion.compareTo(SUPPORTS_GLOBAL_PREPARE_FLAG_VERSION) >= 0;
+                logger.debug("Sending prepare message: options.isGlobal = {}, peerVersion = {}", options.isGlobal(), peerVersion);
+                PrepareMessage message = new PrepareMessage(parentRepairSession, cfIds, options.getRanges(), options.isIncremental(), isGlobal);
                 MessageOut<RepairMessage> msg = message.createMessage();
                 MessagingService.instance().sendRR(msg, neighbour, callback, TimeUnit.HOURS.toMillis(1), true);
             }
@@ -306,20 +331,20 @@
         }
         catch (InterruptedException e)
         {
-            parentRepairSessions.remove(parentRepairSession);
+            removeParentRepairSession(parentRepairSession);
             throw new RuntimeException("Did not get replies from all endpoints. List of failed endpoint(s): " + failedNodes.toString(), e);
         }
 
         if (!status.get())
         {
-            parentRepairSessions.remove(parentRepairSession);
+            removeParentRepairSession(parentRepairSession);
             throw new RuntimeException("Did not get positive replies from all endpoints. List of failed endpoint(s): " + failedNodes.toString());
         }
 
         return parentRepairSession;
     }
 
-    public synchronized void registerParentRepairSession(UUID parentRepairSession, InetAddress coordinator, List<ColumnFamilyStore> columnFamilyStores, Collection<Range<Token>> ranges)
+    public void registerParentRepairSession(UUID parentRepairSession, InetAddress coordinator, List<ColumnFamilyStore> columnFamilyStores, Collection<Range<Token>> ranges, boolean isIncremental, boolean isGlobal)
     {
         if (!registeredForEndpointChanges)
         {
@@ -327,28 +352,7 @@
             FailureDetector.instance.registerFailureDetectionEventListener(this);
             registeredForEndpointChanges = true;
         }
-
-        cleanupOldParentRepairSessions();
-
-        parentRepairSessions.put(parentRepairSession, new ParentRepairSession(coordinator, columnFamilyStores, ranges, System.currentTimeMillis()));
-    }
-
-    /**
-     * Cleans up old failed parent repair sessions - if it is 24h old, we remove it from the map
-     */
-    private void cleanupOldParentRepairSessions()
-    {
-        long currentTime = System.currentTimeMillis();
-
-        Set<UUID> expired = new HashSet<>();
-        for (Map.Entry<UUID, ParentRepairSession> entry : parentRepairSessions.entrySet())
-        {
-            ParentRepairSession session = entry.getValue();
-            if (session.failed && currentTime - session.repairedAt > TimeUnit.HOURS.toMillis(24))
-                expired.add(entry.getKey());
-        }
-        for (UUID remove : expired)
-            parentRepairSessions.remove(remove);
+        parentRepairSessions.put(parentRepairSession, new ParentRepairSession(coordinator, columnFamilyStores, ranges, isIncremental, isGlobal, System.currentTimeMillis()));
     }
 
     public Set<SSTableReader> currentlyRepairing(UUID cfId, UUID parentRepairSession)
@@ -369,28 +373,19 @@
      *
      * @param parentSession Parent session ID
      * @param neighbors Repair participants (not including self)
-     * @throws InterruptedException
-     * @throws ExecutionException
+     * @param successfulRanges Ranges that repaired successfully
      */
-    public synchronized ListenableFuture<?> finishParentSession(UUID parentSession, Set<InetAddress> neighbors, boolean doAntiCompaction) throws InterruptedException, ExecutionException
+    public synchronized ListenableFuture finishParentSession(UUID parentSession, Set<InetAddress> neighbors, Collection<Range<Token>> successfulRanges)
     {
-        // We want to remove parent repair session whether we succeeded or not, so send AnticompactionRequest anyway.
-        // Each replica node determines if anticompaction is needed.
         List<ListenableFuture<?>> tasks = new ArrayList<>(neighbors.size() + 1);
         for (InetAddress neighbor : neighbors)
         {
-            AnticompactionTask task = new AnticompactionTask(parentSession, neighbor, doAntiCompaction);
+            AnticompactionTask task = new AnticompactionTask(parentSession, neighbor, successfulRanges);
+            registerOnFdAndGossip(task);
             tasks.add(task);
             task.run(); // 'run' is just sending message
         }
-        if (doAntiCompaction)
-        {
-            tasks.add(doAntiCompaction(parentSession));
-        }
-        else
-        {
-            removeParentRepairSession(parentSession);
-        }
+        tasks.add(doAntiCompaction(parentSession, successfulRanges));
         return Futures.successfulAsList(tasks);
     }
 
@@ -399,14 +394,27 @@
         ParentRepairSession session = parentRepairSessions.get(parentSessionId);
         // this can happen if a node thinks that the coordinator was down, but that coordinator got back before noticing
         // that it was down itself.
-        if (session != null && session.failed)
+        if (session == null)
             throw new RuntimeException("Parent repair session with id = " + parentSessionId + " has failed.");
 
         return session;
     }
 
+    /**
+     * called when the repair session is done - either failed or anticompaction has completed
+     *
+     * clears out any snapshots created by this repair
+     *
+     * @param parentSessionId
+     * @return
+     */
     public synchronized ParentRepairSession removeParentRepairSession(UUID parentSessionId)
     {
+        for (ColumnFamilyStore cfs : getParentRepairSession(parentSessionId).columnFamilyStores.values())
+        {
+            if (cfs.snapshotExists(parentSessionId.toString()))
+                cfs.clearSnapshot(parentSessionId.toString());
+        }
         return parentRepairSessions.remove(parentSessionId);
     }
 
@@ -417,17 +425,31 @@
      * @param parentRepairSession parent repair session ID
      * @return Future result of all anti-compaction jobs.
      */
-    public ListenableFuture<List<Object>> doAntiCompaction(final UUID parentRepairSession)
+    @SuppressWarnings("resource")
+    public ListenableFuture<List<Object>> doAntiCompaction(final UUID parentRepairSession, Collection<Range<Token>> successfulRanges)
     {
         assert parentRepairSession != null;
         ParentRepairSession prs = getParentRepairSession(parentRepairSession);
+        //A repair will be marked as not global if it is a subrange repair to avoid many small anti-compactions
+        //in addition to other scenarios such as repairs not involving all DCs or hosts
+        if (!prs.isGlobal)
+        {
+            logger.info("Not a global repair, will not do anticompaction");
+            removeParentRepairSession(parentRepairSession);
+            return Futures.immediateFuture(Collections.emptyList());
+        }
+        assert prs.ranges.containsAll(successfulRanges) : "Trying to perform anticompaction on unknown ranges";
 
         List<ListenableFuture<?>> futures = new ArrayList<>();
-        for (Map.Entry<UUID, ColumnFamilyStore> columnFamilyStoreEntry : prs.columnFamilyStores.entrySet())
+        // if we don't have successful repair ranges, then just skip anticompaction
+        if (!successfulRanges.isEmpty())
         {
-            Refs<SSTableReader> sstables = prs.getActiveRepairedSSTableRefsForAntiCompaction(columnFamilyStoreEntry.getKey());
-            ColumnFamilyStore cfs = columnFamilyStoreEntry.getValue();
-            futures.add(CompactionManager.instance.submitAntiCompaction(cfs, prs.ranges, sstables, prs.repairedAt));
+            for (Map.Entry<UUID, ColumnFamilyStore> columnFamilyStoreEntry : prs.columnFamilyStores.entrySet())
+            {
+                Refs<SSTableReader> sstables = prs.getActiveRepairedSSTableRefsForAntiCompaction(columnFamilyStoreEntry.getKey(), parentRepairSession);
+                ColumnFamilyStore cfs = columnFamilyStoreEntry.getValue();
+                futures.add(CompactionManager.instance.submitAntiCompaction(cfs, successfulRanges, sstables, prs.repairedAt));
+            }
         }
 
         ListenableFuture<List<Object>> allAntiCompactionResults = Futures.successfulAsList(futures);
@@ -479,25 +501,19 @@
      */
     public static class ParentRepairSession
     {
-        public final Map<UUID, ColumnFamilyStore> columnFamilyStores = new HashMap<>();
-        public final Collection<Range<Token>> ranges;
+        private final Map<UUID, ColumnFamilyStore> columnFamilyStores = new HashMap<>();
+        private final Collection<Range<Token>> ranges;
         public final Map<UUID, Set<String>> sstableMap = new HashMap<>();
-        /**
-         * used as fail time if failed is true
-         */
+        public final boolean isIncremental;
+        public final boolean isGlobal;
         public final long repairedAt;
         public final InetAddress coordinator;
         /**
-         * Used to mark a repair as failed - if the coordinator thinks that the repair is still ongoing and sends a
-         * request, we need to fail the coordinator as well.
-         */
-        public final boolean failed;
-        /**
          * Indicates whether we have marked sstables as repairing. Can only be done once per table per ParentRepairSession
          */
         private final Set<UUID> marked = new HashSet<>();
 
-        public ParentRepairSession(InetAddress coordinator, List<ColumnFamilyStore> columnFamilyStores, Collection<Range<Token>> ranges, long repairedAt, boolean failed)
+        public ParentRepairSession(InetAddress coordinator, List<ColumnFamilyStore> columnFamilyStores, Collection<Range<Token>> ranges, boolean isIncremental, boolean isGlobal, long repairedAt)
         {
             this.coordinator = coordinator;
             for (ColumnFamilyStore cfs : columnFamilyStores)
@@ -507,30 +523,55 @@
             }
             this.ranges = ranges;
             this.repairedAt = repairedAt;
-            this.failed = failed;
-        }
-
-        public ParentRepairSession(InetAddress coordinator, List<ColumnFamilyStore> columnFamilyStores, Collection<Range<Token>> ranges, long repairedAt)
-        {
-            this(coordinator, columnFamilyStores, ranges, repairedAt, false);
+            this.isGlobal = isGlobal;
+            this.isIncremental = isIncremental;
         }
 
         /**
-         * Gets the repairing sstables for anticompaction.
+         * Mark sstables repairing - either all sstables or only the unrepaired ones depending on
          *
-         * Note that validation and streaming uses the real unrepaired sstables.
+         * whether this is an incremental or full repair
+         *
+         * @param cfId the column family
+         * @param parentSessionId the parent repair session id, used to make sure we don't start multiple repairs over the same sstables
+         */
+        public synchronized void markSSTablesRepairing(UUID cfId, UUID parentSessionId)
+        {
+            if (!marked.contains(cfId))
+            {
+                List<SSTableReader> sstables = columnFamilyStores.get(cfId).select(isIncremental ? ColumnFamilyStore.UNREPAIRED_SSTABLES : ColumnFamilyStore.CANONICAL_SSTABLES).sstables;
+                Set<SSTableReader> currentlyRepairing = ActiveRepairService.instance.currentlyRepairing(cfId, parentSessionId);
+                if (!Sets.intersection(currentlyRepairing, Sets.newHashSet(sstables)).isEmpty())
+                {
+                    logger.error("Cannot start multiple repair sessions over the same sstables");
+                    throw new RuntimeException("Cannot start multiple repair sessions over the same sstables");
+                }
+                addSSTables(cfId, sstables);
+                marked.add(cfId);
+            }
+        }
+
+        /**
+         * Get the still active sstables we should run anticompaction on
+         *
+         * note that validation and streaming do not call this method - they have to work on the actual active sstables on the node, we only call this
+         * to know which sstables are still there that were there when we started the repair
          *
          * @param cfId
+         * @param parentSessionId for checking if there exists a snapshot for this repair
          * @return
          */
         @SuppressWarnings("resource")
-        public synchronized Refs<SSTableReader> getActiveRepairedSSTableRefsForAntiCompaction(UUID cfId)
+        public synchronized Refs<SSTableReader> getActiveRepairedSSTableRefsForAntiCompaction(UUID cfId, UUID parentSessionId)
         {
             assert marked.contains(cfId);
-            ImmutableMap.Builder<SSTableReader, Ref<SSTableReader>> references = ImmutableMap.builder();
-            Iterable<SSTableReader> sstables = getActiveSSTables(cfId);
-            if (sstables == null)
+            if (!columnFamilyStores.containsKey(cfId))
                 throw new RuntimeException("Not possible to get sstables for anticompaction for " + cfId);
+            boolean isSnapshotRepair = columnFamilyStores.get(cfId).snapshotExists(parentSessionId.toString());
+            ImmutableMap.Builder<SSTableReader, Ref<SSTableReader>> references = ImmutableMap.builder();
+            Iterable<SSTableReader> sstables = isSnapshotRepair ? getSSTablesForSnapshotRepair(cfId, parentSessionId) : getActiveSSTables(cfId);
+            // we check this above - if columnFamilyStores contains the cfId sstables will not be null
+            assert sstables != null;
             for (SSTableReader sstable : sstables)
             {
                 Ref<SSTableReader> ref = sstable.tryRef();
@@ -543,40 +584,101 @@
         }
 
         /**
-         * Marks all the unrepaired sstables as repairing unless we have already done so.
+         * If we are running a snapshot repair we need to find the 'real' sstables when we start anticompaction
          *
-         * Any of these sstables that are still on disk are then anticompacted once the streaming and validation phases are done.
+         * We use the generation of the sstables as identifiers instead of the file name to avoid having to parse out the
+         * actual filename.
          *
          * @param cfId
-         * @param parentSessionId used to check that we don't start multiple inc repair sessions over the same sstables
+         * @param parentSessionId
+         * @return
          */
-        public synchronized void markSSTablesRepairing(UUID cfId, UUID parentSessionId)
+        private Set<SSTableReader> getSSTablesForSnapshotRepair(UUID cfId, UUID parentSessionId)
         {
-            if (!marked.contains(cfId))
+            Set<SSTableReader> activeSSTables = new HashSet<>();
+            ColumnFamilyStore cfs = columnFamilyStores.get(cfId);
+            if (cfs == null)
+                return null;
+
+            Set<Integer> snapshotGenerations = new HashSet<>();
+            try (Refs<SSTableReader> snapshottedSSTables = cfs.getSnapshotSSTableReader(parentSessionId.toString()))
             {
-                List<SSTableReader> sstables = columnFamilyStores.get(cfId).select(ColumnFamilyStore.UNREPAIRED_SSTABLES).sstables;
-                Set<SSTableReader> currentlyRepairing = ActiveRepairService.instance.currentlyRepairing(cfId, parentSessionId);
-                if (!Sets.intersection(currentlyRepairing, Sets.newHashSet(sstables)).isEmpty())
+                for (SSTableReader sstable : snapshottedSSTables)
                 {
+                    snapshotGenerations.add(sstable.descriptor.generation);
+                }
+            }
+            catch (IOException e)
+            {
+                throw new RuntimeException(e);
+            }
+            for (SSTableReader sstable : cfs.select(ColumnFamilyStore.CANONICAL_SSTABLES).sstables)
+                if (snapshotGenerations.contains(sstable.descriptor.generation))
+                    activeSSTables.add(sstable);
+            return activeSSTables;
+        }
+
+        public synchronized void maybeSnapshot(UUID cfId, UUID parentSessionId)
+        {
+            String snapshotName = parentSessionId.toString();
+            if (!columnFamilyStores.get(cfId).snapshotExists(snapshotName))
+            {
+                Set<SSTableReader> snapshottedSSTables = columnFamilyStores.get(cfId).snapshot(snapshotName, new Predicate<SSTableReader>()
+                {
+                    public boolean apply(SSTableReader sstable)
+                    {
+                        return sstable != null &&
+                               (!isIncremental || !sstable.isRepaired()) &&
+                               !(sstable.partitioner instanceof LocalPartitioner) && // exclude SSTables from 2i
+                               new Bounds<>(sstable.first.getToken(), sstable.last.getToken()).intersects(ranges);
+                    }
+                }, true);
+
+                if (isAlreadyRepairing(cfId, parentSessionId, snapshottedSSTables))
+                {
+                    columnFamilyStores.get(cfId).clearSnapshot(parentSessionId.toString());
                     logger.error("Cannot start multiple repair sessions over the same sstables");
                     throw new RuntimeException("Cannot start multiple repair sessions over the same sstables");
                 }
-                addSSTables(cfId, sstables);
+                addSSTables(cfId, snapshottedSSTables);
                 marked.add(cfId);
             }
         }
 
+
+        /**
+         * Compares other repairing sstables *generation* to the ones we just snapshotted
+         *
+         * we compare generations since the sstables have different paths due to snapshot names
+         *
+         * @param cfId id of the column family store
+         * @param parentSessionId parent repair session
+         * @param sstables the newly snapshotted sstables
+         * @return
+         */
+        private boolean isAlreadyRepairing(UUID cfId, UUID parentSessionId, Collection<SSTableReader> sstables)
+        {
+            Set<SSTableReader> currentlyRepairing = ActiveRepairService.instance.currentlyRepairing(cfId, parentSessionId);
+            Set<Integer> currentlyRepairingGenerations = new HashSet<>();
+            Set<Integer> newRepairingGenerations = new HashSet<>();
+            for (SSTableReader sstable : currentlyRepairing)
+                currentlyRepairingGenerations.add(sstable.descriptor.generation);
+            for (SSTableReader sstable : sstables)
+                newRepairingGenerations.add(sstable.descriptor.generation);
+
+            return !Sets.intersection(currentlyRepairingGenerations, newRepairingGenerations).isEmpty();
+        }
+
         private Set<SSTableReader> getActiveSSTables(UUID cfId)
         {
-            if (failed)
-                return Collections.emptySet();
             if (!columnFamilyStores.containsKey(cfId))
                 return null;
 
             Set<String> repairedSSTables = sstableMap.get(cfId);
             Set<SSTableReader> activeSSTables = new HashSet<>();
             Set<String> activeSSTableNames = new HashSet<>();
-            for (SSTableReader sstable : columnFamilyStores.get(cfId).getSSTables())
+            ColumnFamilyStore cfs = columnFamilyStores.get(cfId);
+            for (SSTableReader sstable : cfs.select(ColumnFamilyStore.CANONICAL_SSTABLES).sstables)
             {
                 if (repairedSSTables.contains(sstable.getFilename()))
                 {
@@ -594,10 +696,14 @@
                 sstableMap.get(cfId).add(sstable.getFilename());
         }
 
-        public ParentRepairSession asFailed()
+
+        public long getRepairedAt()
         {
-            return new ParentRepairSession(coordinator, Collections.<ColumnFamilyStore>emptyList(), Collections.<Range<Token>>emptyList(), System.currentTimeMillis(), true);
+            if (isGlobal)
+                return repairedAt;
+            return ActiveRepairService.UNREPAIRED_SSTABLE;
         }
+
         @Override
         public String toString()
         {
@@ -608,7 +714,6 @@
                     ", repairedAt=" + repairedAt +
                     '}';
         }
-
     }
 
     /*
@@ -658,12 +763,9 @@
 
         if (!toRemove.isEmpty())
         {
-            logger.debug("Failing {} in parent repair sessions", toRemove);
+            logger.debug("Removing {} in parent repair sessions", toRemove);
             for (UUID id : toRemove)
-            {
-                ParentRepairSession failed = parentRepairSessions.get(id);
-                parentRepairSessions.replace(id, failed, failed.asFailed());
-            }
+                removeParentRepairSession(id);
         }
     }
 
diff --git a/src/java/org/apache/cassandra/service/CacheService.java b/src/java/org/apache/cassandra/service/CacheService.java
index 50d8903..a13a52d 100644
--- a/src/java/org/apache/cassandra/service/CacheService.java
+++ b/src/java/org/apache/cassandra/service/CacheService.java
@@ -32,6 +32,8 @@
 import javax.management.ObjectName;
 
 import com.google.common.util.concurrent.Futures;
+
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -47,7 +49,6 @@
 import org.apache.cassandra.db.context.CounterContext;
 import org.apache.cassandra.db.filter.QueryFilter;
 import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
@@ -131,10 +132,22 @@
     {
         logger.info("Initializing row cache with capacity of {} MBs", DatabaseDescriptor.getRowCacheSizeInMB());
 
-        long rowCacheInMemoryCapacity = DatabaseDescriptor.getRowCacheSizeInMB() * 1024 * 1024;
+        CacheProvider<RowCacheKey, IRowCacheEntry> cacheProvider;
+        String cacheProviderClassName = DatabaseDescriptor.getRowCacheSizeInMB() > 0
+                                        ? DatabaseDescriptor.getRowCacheClassName() : "org.apache.cassandra.cache.NopCacheProvider";
+        try
+        {
+            Class<CacheProvider<RowCacheKey, IRowCacheEntry>> cacheProviderClass =
+                (Class<CacheProvider<RowCacheKey, IRowCacheEntry>>) Class.forName(cacheProviderClassName);
+            cacheProvider = cacheProviderClass.newInstance();
+        }
+        catch (Exception e)
+        {
+            throw new RuntimeException("Cannot find configured row cache provider class " + DatabaseDescriptor.getRowCacheClassName());
+        }
 
         // cache object
-        ICache<RowCacheKey, IRowCacheEntry> rc = new SerializingCacheProvider().create(rowCacheInMemoryCapacity);
+        ICache<RowCacheKey, IRowCacheEntry> rc = cacheProvider.create();
         AutoSavingCache<RowCacheKey, IRowCacheEntry> rowCache = new AutoSavingCache<>(rc, CacheType.ROW_CACHE, new RowCacheSerializer());
 
         int rowCacheKeysToSave = DatabaseDescriptor.getRowCacheKeysToSave();
@@ -166,35 +179,6 @@
         return cache;
     }
 
-    public long getKeyCacheHits()
-    {
-        return keyCache.getMetrics().hits.count();
-    }
-
-    public long getRowCacheHits()
-    {
-        return rowCache.getMetrics().hits.count();
-    }
-
-    public long getKeyCacheRequests()
-    {
-        return keyCache.getMetrics().requests.count();
-    }
-
-    public long getRowCacheRequests()
-    {
-        return rowCache.getMetrics().requests.count();
-    }
-
-    public double getKeyCacheRecentHitRate()
-    {
-        return keyCache.getMetrics().getRecentHitRate();
-    }
-
-    public double getRowCacheRecentHitRate()
-    {
-        return rowCache.getMetrics().getRecentHitRate();
-    }
 
     public int getRowCacheSavePeriodInSeconds()
     {
@@ -284,7 +268,7 @@
 
     public void invalidateKeyCacheForCf(Pair<String, String> ksAndCFName)
     {
-        Iterator<KeyCacheKey> keyCacheIterator = keyCache.getKeySet().iterator();
+        Iterator<KeyCacheKey> keyCacheIterator = keyCache.keyIterator();
         while (keyCacheIterator.hasNext())
         {
             KeyCacheKey key = keyCacheIterator.next();
@@ -300,7 +284,7 @@
 
     public void invalidateRowCacheForCf(Pair<String, String> ksAndCFName)
     {
-        Iterator<RowCacheKey> rowCacheIterator = rowCache.getKeySet().iterator();
+        Iterator<RowCacheKey> rowCacheIterator = rowCache.keyIterator();
         while (rowCacheIterator.hasNext())
         {
             RowCacheKey rowCacheKey = rowCacheIterator.next();
@@ -311,7 +295,7 @@
 
     public void invalidateCounterCacheForCf(Pair<String, String> ksAndCFName)
     {
-        Iterator<CounterCacheKey> counterCacheIterator = counterCache.getKeySet().iterator();
+        Iterator<CounterCacheKey> counterCacheIterator = counterCache.keyIterator();
         while (counterCacheIterator.hasNext())
         {
             CounterCacheKey counterCacheKey = counterCacheIterator.next();
@@ -325,15 +309,8 @@
         counterCache.clear();
     }
 
-    public long getRowCacheCapacityInBytes()
-    {
-        return rowCache.getMetrics().capacity.value();
-    }
 
-    public long getRowCacheCapacityInMB()
-    {
-        return getRowCacheCapacityInBytes() / 1024 / 1024;
-    }
+
 
     public void setRowCacheCapacityInMB(long capacity)
     {
@@ -343,15 +320,6 @@
         rowCache.setCapacity(capacity * 1024 * 1024);
     }
 
-    public long getKeyCacheCapacityInBytes()
-    {
-        return keyCache.getMetrics().capacity.value();
-    }
-
-    public long getKeyCacheCapacityInMB()
-    {
-        return getKeyCacheCapacityInBytes() / 1024 / 1024;
-    }
 
     public void setKeyCacheCapacityInMB(long capacity)
     {
@@ -369,26 +337,6 @@
         counterCache.setCapacity(capacity * 1024 * 1024);
     }
 
-    public long getRowCacheSize()
-    {
-        return rowCache.getMetrics().size.value();
-    }
-
-    public long getRowCacheEntries()
-    {
-        return rowCache.size();
-    }
-
-    public long getKeyCacheSize()
-    {
-        return keyCache.getMetrics().size.value();
-    }
-
-    public long getKeyCacheEntries()
-    {
-        return keyCache.size();
-    }
-
     public void saveCaches() throws ExecutionException, InterruptedException
     {
         List<Future<?>> futures = new ArrayList<>(3);
@@ -487,7 +435,7 @@
             ByteBufferUtil.writeWithLength(key.key, out);
             out.writeInt(key.desc.generation);
             out.writeBoolean(true);
-            cfs.metadata.comparator.rowIndexEntrySerializer().serialize(entry, out);
+            key.desc.getFormat().getIndexSerializer(cfs.metadata).serialize(entry, out);
         }
 
         public Future<Pair<KeyCacheKey, RowIndexEntry>> deserialize(DataInputStream input, ColumnFamilyStore cfs) throws IOException
@@ -509,7 +457,7 @@
                 RowIndexEntry.Serializer.skip(input);
                 return null;
             }
-            RowIndexEntry entry = reader.metadata.comparator.rowIndexEntrySerializer().deserialize(input, reader.descriptor.version);
+            RowIndexEntry entry = reader.descriptor.getFormat().getIndexSerializer(reader.metadata).deserialize(input, reader.descriptor.version);
             return Futures.immediateFuture(Pair.create(new KeyCacheKey(cfs.metadata.ksAndCFName, reader.descriptor, key), entry));
         }
 
diff --git a/src/java/org/apache/cassandra/service/CacheServiceMBean.java b/src/java/org/apache/cassandra/service/CacheServiceMBean.java
index 28e9d3b..8ac0694 100644
--- a/src/java/org/apache/cassandra/service/CacheServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/CacheServiceMBean.java
@@ -64,90 +64,4 @@
      * @throws InterruptedException when a thread is waiting, sleeping, or otherwise occupied, and the thread is interrupted, either before or during the activity.
      */
     public void saveCaches() throws ExecutionException, InterruptedException;
-
-    //
-    // remaining methods are provided for backwards compatibility; modern clients should use CacheMetrics instead
-    //
-
-    /**
-     * @see org.apache.cassandra.metrics.CacheMetrics#hits
-     */
-    @Deprecated
-    public long getKeyCacheHits();
-
-    /**
-     * @see org.apache.cassandra.metrics.CacheMetrics#hits
-     */
-    @Deprecated
-    public long getRowCacheHits();
-
-    /**
-     * @see org.apache.cassandra.metrics.CacheMetrics#requests
-     */
-    @Deprecated
-    public long getKeyCacheRequests();
-
-    /**
-     * @see org.apache.cassandra.metrics.CacheMetrics#requests
-     */
-    @Deprecated
-    public long getRowCacheRequests();
-
-    /**
-     * @see org.apache.cassandra.metrics.CacheMetrics#hitRate
-     */
-    @Deprecated
-    public double getKeyCacheRecentHitRate();
-
-    /**
-     * @see org.apache.cassandra.metrics.CacheMetrics#hitRate
-     */
-    @Deprecated
-    public double getRowCacheRecentHitRate();
-
-    /**
-     * @see org.apache.cassandra.metrics.CacheMetrics#capacity
-     */
-    @Deprecated
-    public long getRowCacheCapacityInMB();
-    /**
-     * @see org.apache.cassandra.metrics.CacheMetrics#capacity
-     */
-    @Deprecated
-    public long getRowCacheCapacityInBytes();
-
-    /**
-     * @see org.apache.cassandra.metrics.CacheMetrics#capacity
-     */
-    @Deprecated
-    public long getKeyCacheCapacityInMB();
-    /**
-     * @see org.apache.cassandra.metrics.CacheMetrics#capacity
-     */
-    @Deprecated
-    public long getKeyCacheCapacityInBytes();
-
-    /**
-     * @see org.apache.cassandra.metrics.CacheMetrics#size
-     */
-    @Deprecated
-    public long getRowCacheSize();
-
-    /**
-     * @see org.apache.cassandra.metrics.CacheMetrics#entries
-     */
-    @Deprecated
-    public long getRowCacheEntries();
-
-    /**
-     * @see org.apache.cassandra.metrics.CacheMetrics#size
-     */
-    @Deprecated
-    public long getKeyCacheSize();
-
-    /**
-     * @see org.apache.cassandra.metrics.CacheMetrics#entries
-     */
-    @Deprecated
-    public long getKeyCacheEntries();
 }
diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java
index dc22834..18b568d 100644
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@ -25,8 +25,10 @@
 import java.net.UnknownHostException;
 import java.rmi.registry.LocateRegistry;
 import java.rmi.server.RMIServerSocketFactory;
+import java.util.Collections;
 import java.util.List;
-import java.util.*;
+import java.util.Map;
+import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 
 import javax.management.MBeanServer;
@@ -36,28 +38,34 @@
 import javax.management.remote.JMXServiceURL;
 import javax.management.remote.rmi.RMIConnectorServer;
 
+import com.addthis.metrics3.reporter.config.ReporterConfig;
+import com.codahale.metrics.Meter;
+import com.codahale.metrics.MetricRegistryListener;
+import com.codahale.metrics.SharedMetricRegistries;
+import com.codahale.metrics.jvm.BufferPoolMetricSet;
+import com.codahale.metrics.jvm.FileDescriptorRatioGauge;
+import com.codahale.metrics.jvm.GarbageCollectorMetricSet;
+import com.codahale.metrics.jvm.MemoryUsageGaugeSet;
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.Iterables;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.Uninterruptibles;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.addthis.metrics.reporter.config.ReporterConfig;
-import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
-import org.apache.cassandra.concurrent.ScheduledExecutors;
-import org.apache.cassandra.concurrent.Stage;
-import org.apache.cassandra.concurrent.StageManager;
+import org.apache.cassandra.concurrent.*;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.commitlog.CommitLog;
 import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.StartupException;
 import org.apache.cassandra.io.FSError;
 import org.apache.cassandra.io.sstable.CorruptSSTableException;
 import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.metrics.CassandraMetricsRegistry;
+import org.apache.cassandra.metrics.DefaultNameFactory;
 import org.apache.cassandra.metrics.StorageMetrics;
 import org.apache.cassandra.thrift.ThriftServer;
 import org.apache.cassandra.tracing.Tracing;
@@ -72,155 +80,111 @@
 public class CassandraDaemon
 {
     public static final String MBEAN_NAME = "org.apache.cassandra.db:type=NativeAccess";
-    public static JMXConnectorServer jmxServer = null;
+    private static JMXConnectorServer jmxServer = null;
 
-    private static final Logger logger = LoggerFactory.getLogger(CassandraDaemon.class);
+    private static final Logger logger;
+    static {
+        // Need to register metrics before instrumented appender is created(first access to LoggerFactory).
+        SharedMetricRegistries.getOrCreate("logback-metrics").addListener(new MetricRegistryListener.Base()
+        {
+            @Override
+            public void onMeterAdded(String metricName, Meter meter)
+            {
+                // Given metricName consists of appender name in logback.xml + "." + metric name.
+                // We first separate appender name
+                int separator = metricName.lastIndexOf('.');
+                String appenderName = metricName.substring(0, separator);
+                String metric = metricName.substring(separator + 1); // remove "."
+                ObjectName name = DefaultNameFactory.createMetricName(appenderName, metric, null).getMBeanName();
+                CassandraMetricsRegistry.Metrics.registerMBean(meter, name);
+            }
+        });
+        logger = LoggerFactory.getLogger(CassandraDaemon.class);
+    }
 
-    private static void maybeInitJmx()
+    private void maybeInitJmx()
     {
-        String jmxPort = System.getProperty("com.sun.management.jmxremote.port");
+        if (System.getProperty("com.sun.management.jmxremote.port") != null)
+            return;
 
+        String jmxPort = System.getProperty("cassandra.jmx.local.port");
         if (jmxPort == null)
+            return;
+
+        System.setProperty("java.rmi.server.hostname", InetAddress.getLoopbackAddress().getHostAddress());
+        RMIServerSocketFactory serverFactory = new RMIServerSocketFactoryImpl();
+        Map<String, ?> env = Collections.singletonMap(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE, serverFactory);
+        try
         {
-            logger.warn("JMX is not enabled to receive remote connections. Please see cassandra-env.sh for more info.");
-
-            jmxPort = System.getProperty("cassandra.jmx.local.port");
-
-            if (jmxPort == null)
-            {
-                logger.error("cassandra.jmx.local.port missing from cassandra-env.sh, unable to start local JMX service." + jmxPort);
-            }
-            else
-            {
-                System.setProperty("java.rmi.server.hostname", InetAddress.getLoopbackAddress().getHostAddress());
-
-                try
-                {
-                    RMIServerSocketFactory serverFactory = new RMIServerSocketFactoryImpl();
-                    LocateRegistry.createRegistry(Integer.valueOf(jmxPort), null, serverFactory);
-
-                    StringBuffer url = new StringBuffer();
-                    url.append("service:jmx:");
-                    url.append("rmi://localhost/jndi/");
-                    url.append("rmi://localhost:").append(jmxPort).append("/jmxrmi");
-
-                    Map env = new HashMap();
-                    env.put(RMIConnectorServer.RMI_SERVER_SOCKET_FACTORY_ATTRIBUTE, serverFactory);
-
-                    jmxServer = new RMIConnectorServer(
-                            new JMXServiceURL(url.toString()),
-                            env,
-                            ManagementFactory.getPlatformMBeanServer()
-                    );
-
-                    jmxServer.start();
-                }
-                catch (IOException e)
-                {
-                    logger.error("Error starting local jmx server: ", e);
-                }
-            }
+            LocateRegistry.createRegistry(Integer.valueOf(jmxPort), null, serverFactory);
+            JMXServiceURL url = new JMXServiceURL(String.format("service:jmx:rmi://localhost/jndi/rmi://localhost:%s/jmxrmi", jmxPort));
+            jmxServer = new RMIConnectorServer(url, env, ManagementFactory.getPlatformMBeanServer());
+            jmxServer.start();
         }
-        else
+        catch (IOException e)
         {
-            logger.info("JMX is enabled to receive remote connections on port: " + jmxPort);
+            exitOrFail(1, e.getMessage(), e.getCause());
         }
     }
 
     private static final CassandraDaemon instance = new CassandraDaemon();
 
-    /**
-     * The earliest legit timestamp a casandra instance could have ever launched.
-     * Date roughly taken from http://perspectives.mvdirona.com/2008/07/12/FacebookReleasesCassandraAsOpenSource.aspx
-     * We use this to ensure the system clock is at least somewhat correct at startup.
-     */
-    private static final long EARLIEST_LAUNCH_DATE = 1215820800000L;
-
     public Server thriftServer;
     public Server nativeServer;
-    private boolean setupCompleted = false;
+
+    private final boolean runManaged;
+    protected final StartupChecks startupChecks;
+    private boolean setupCompleted;
+
+    public CassandraDaemon() {
+        this(false);
+    }
+
+    public CassandraDaemon(boolean runManaged) {
+        this.runManaged = runManaged;
+        this.startupChecks = new StartupChecks().withDefaultTests();
+        this.setupCompleted = false;
+    }
+
     /**
      * This is a hook for concrete daemons to initialize themselves suitably.
      *
      * Subclasses should override this to finish the job (listening on ports, etc.)
-     *
-     * @throws IOException
      */
     protected void setup()
     {
         FileUtils.setFSErrorHandler(new DefaultFSErrorHandler());
 
-        try
-        {
-            logger.info("Hostname: {}", InetAddress.getLocalHost().getHostName());
-        }
-        catch (UnknownHostException e1)
-        {
-            logger.info("Could not resolve local host");
-        }
+        // Delete any failed snapshot deletions on Windows - see CASSANDRA-9658
+        if (FBUtilities.isWindows())
+            WindowsFailedSnapshotTracker.deleteOldSnapshots();
 
-        long now = System.currentTimeMillis();
-        if (now < EARLIEST_LAUNCH_DATE)
-        {
-            logger.error("current machine time is {}, but that is seemingly incorrect. exiting now.", new Date(now));
-            System.exit(3);
-        }
-
-        // log warnings for different kinds of sub-optimal JVMs.  tldr use 64-bit Oracle >= 1.6u32
-        if (!DatabaseDescriptor.hasLargeAddressSpace())
-            logger.info("32bit JVM detected.  It is recommended to run Cassandra on a 64bit JVM for better performance.");
-        String javaVersion = System.getProperty("java.version");
-        String javaVmName = System.getProperty("java.vm.name");
-        logger.info("JVM vendor/version: {}/{}", javaVmName, javaVersion);
-        if (javaVmName.contains("OpenJDK"))
-        {
-            // There is essentially no QA done on OpenJDK builds, and
-            // clusters running OpenJDK have seen many heap and load issues.
-            logger.warn("OpenJDK is not recommended. Please upgrade to the newest Oracle Java release");
-        }
-        else if (!javaVmName.contains("HotSpot"))
-        {
-            logger.warn("Non-Oracle JVM detected.  Some features, such as immediate unmap of compacted SSTables, may not work as intended");
-        }
-     /*   else
-        {
-            String[] java_version = javaVersion.split("_");
-            String java_major = java_version[0];
-            int java_minor;
-            try
-            {
-                java_minor = (java_version.length > 1) ? Integer.parseInt(java_version[1]) : 0;
-            }
-            catch (NumberFormatException e)
-            {
-                // have only seen this with java7 so far but no doubt there are other ways to break this
-                logger.info("Unable to parse java version {}", Arrays.toString(java_version));
-                java_minor = 32;
-            }
-        }
-     */
-        logger.info("Heap size: {}/{}", Runtime.getRuntime().totalMemory(), Runtime.getRuntime().maxMemory());
-        for(MemoryPoolMXBean pool: ManagementFactory.getMemoryPoolMXBeans())
-            logger.info("{} {}: {}", pool.getName(), pool.getType(), pool.getPeakUsage());
-        logger.info("Classpath: {}", System.getProperty("java.class.path"));
-
-        logger.info("JVM Arguments: {}", ManagementFactory.getRuntimeMXBean().getInputArguments());
-
-        // Fail-fast if JNA is not available or failing to initialize properly
-        // except with -Dcassandra.boot_without_jna=true. See CASSANDRA-6575.
-        if (!CLibrary.jnaAvailable())
-        {
-            boolean jnaRequired = !Boolean.getBoolean("cassandra.boot_without_jna");
-
-            if (jnaRequired)
-            {
-                logger.error("JNA failing to initialize properly. Use -Dcassandra.boot_without_jna=true to bootstrap even so.");
-                System.exit(3);
-            }
-        }
+        logSystemInfo();
 
         CLibrary.tryMlockall();
 
+        try
+        {
+            startupChecks.verify();
+        }
+        catch (StartupException e)
+        {
+            exitOrFail(e.returnCode, e.getMessage(), e.getCause());
+        }
+
+        try
+        {
+            SystemKeyspace.snapshotOnVersionChange();
+        }
+        catch (IOException e)
+        {
+            exitOrFail(3, e.getMessage(), e.getCause());
+        }
+
+        // We need to persist this as soon as possible after startup checks.
+        // This should be the first write to SystemKeyspace (CASSANDRA-11742)
+        SystemKeyspace.persistLocalMetadata();
+
         maybeInitJmx();
 
         Thread.setDefaultUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler()
@@ -251,54 +215,8 @@
             }
         });
 
-        // check all directories(data, commitlog, saved cache) for existence and permission
-        Iterable<String> dirs = Iterables.concat(Arrays.asList(DatabaseDescriptor.getAllDataFileLocations()),
-                                                 Arrays.asList(DatabaseDescriptor.getCommitLogLocation(),
-                                                               DatabaseDescriptor.getSavedCachesLocation()));
-        for (String dataDir : dirs)
-        {
-            logger.debug("Checking directory {}", dataDir);
-            File dir = new File(dataDir);
-
-            // check that directories exist.
-            if (!dir.exists())
-            {
-                logger.error("Directory {} doesn't exist", dataDir);
-                // if they don't, failing their creation, stop cassandra.
-                if (!dir.mkdirs())
-                {
-                    logger.error("Has no permission to create {} directory", dataDir);
-                    System.exit(3);
-                }
-            }
-            // if directories exist verify their permissions
-            if (!Directories.verifyFullPermissions(dir, dataDir))
-            {
-                // if permissions aren't sufficient, stop cassandra.
-                System.exit(3);
-            }
-        }
-
-        if (CacheService.instance == null) // should never happen
-            throw new RuntimeException("Failed to initialize Cache Service.");
-
-        // check the system keyspace to keep user from shooting self in foot by changing partitioner, cluster name, etc.
-        // we do a one-off scrub of the system keyspace first; we can't load the list of the rest of the keyspaces,
-        // until system keyspace is opened.
-        for (CFMetaData cfm : Schema.instance.getKeyspaceMetaData(Keyspace.SYSTEM_KS).values())
-            ColumnFamilyStore.scrubDataDirectories(cfm);
-        try
-        {
-            SystemKeyspace.checkHealth();
-        }
-        catch (ConfigurationException e)
-        {
-            logger.error("Fatal exception during initialization", e);
-            System.exit(100);
-        }
-
-        // load keyspace descriptions.
-        DatabaseDescriptor.loadSchemas();
+        // load schema from disk
+        Schema.instance.loadFromDisk();
 
         // clean up compaction leftovers
         Map<Pair<String, String>, Map<Integer, UUID>> unfinishedCompactions = SystemKeyspace.getUnfinishedCompactions();
@@ -315,7 +233,7 @@
         for (String keyspaceName : Schema.instance.getKeyspaces())
         {
             // Skip system as we've already cleaned it
-            if (keyspaceName.equals(Keyspace.SYSTEM_KS))
+            if (keyspaceName.equals(SystemKeyspace.NAME))
                 continue;
 
             for (CFMetaData cfm : Schema.instance.getKeyspaceMetaData(keyspaceName).values())
@@ -323,6 +241,7 @@
         }
 
         Keyspace.setInitialized();
+
         // initialize keyspaces
         for (String keyspaceName : Schema.instance.getKeyspaces())
         {
@@ -392,9 +311,8 @@
         }
         catch (ConfigurationException e)
         {
-            logger.error("Fatal configuration error", e);
             System.err.println(e.getMessage() + "\nFatal configuration error; unable to start server.  See log for stacktrace.");
-            System.exit(1);
+            exitOrFail(1, "Fatal configuration error", e);
         }
 
         Mx4jTool.maybeLoad();
@@ -406,8 +324,14 @@
             logger.info("Trying to load metrics-reporter-config from file: {}", metricsReporterConfigFile);
             try
             {
+                // enable metrics provided by metrics-jvm.jar
+                CassandraMetricsRegistry.Metrics.register("jvm.buffers", new BufferPoolMetricSet(ManagementFactory.getPlatformMBeanServer()));
+                CassandraMetricsRegistry.Metrics.register("jvm.gc", new GarbageCollectorMetricSet());
+                CassandraMetricsRegistry.Metrics.register("jvm.memory", new MemoryUsageGaugeSet());
+                CassandraMetricsRegistry.Metrics.register("jvm.fd.usage", new FileDescriptorRatioGauge());
+                // initialize metrics-reporter-config from yaml file
                 String reportFileLocation = CassandraDaemon.class.getClassLoader().getResource(metricsReporterConfigFile).getFile();
-                ReporterConfig.loadFromFile(reportFileLocation).enableAll();
+                ReporterConfig.loadFromFile(reportFileLocation).enableAll(CassandraMetricsRegistry.Metrics);
             }
             catch (Exception e)
             {
@@ -469,6 +393,31 @@
         return setupCompleted;
     }
 
+    private void logSystemInfo()
+    {
+    	if (logger.isInfoEnabled())
+    	{
+	        try
+	        {
+	            logger.info("Hostname: {}", InetAddress.getLocalHost().getHostName());
+	        }
+	        catch (UnknownHostException e1)
+	        {
+	            logger.info("Could not resolve local host");
+	        }
+
+	        logger.info("JVM vendor/version: {}/{}", System.getProperty("java.vm.name"), System.getProperty("java.version"));
+	        logger.info("Heap size: {}/{}", Runtime.getRuntime().totalMemory(), Runtime.getRuntime().maxMemory());
+
+	        for(MemoryPoolMXBean pool: ManagementFactory.getMemoryPoolMXBeans())
+	            logger.info("{} {}: {}", pool.getName(), pool.getType(), pool.getPeakUsage());
+
+	        logger.info("Classpath: {}", System.getProperty("java.class.path"));
+
+            logger.info("JVM Arguments: {}", ManagementFactory.getRuntimeMXBean().getInputArguments());
+    	}
+    }
+
     /**
      * Initialize the Cassandra Daemon based on the given <a
      * href="http://commons.apache.org/daemon/jsvc.html">Commons
@@ -493,7 +442,9 @@
     {
         String nativeFlag = System.getProperty("cassandra.start_native_transport");
         if ((nativeFlag != null && Boolean.parseBoolean(nativeFlag)) || (nativeFlag == null && DatabaseDescriptor.startNativeTransport()))
+        {
             nativeServer.start();
+        }
         else
             logger.info("Not starting native transport as requested. Use JMX (StorageService->startNativeTransport()) or nodetool (enablebinary) to start it");
 
@@ -548,12 +499,21 @@
      */
     public void activate()
     {
-        String pidFile = System.getProperty("cassandra-pidfile");
-
+        // Do not put any references to DatabaseDescriptor above the forceStaticInitialization call.
         try
         {
             try
             {
+                DatabaseDescriptor.forceStaticInitialization();
+                DatabaseDescriptor.setDaemonInitialized();
+            }
+            catch (ExceptionInInitializerError e)
+            {
+                throw e.getCause();
+            }
+
+            try
+            {
                 MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
                 mbs.registerMBean(new StandardMBean(new NativeAccess(), NativeAccessMBean.class), new ObjectName(MBEAN_NAME));
             }
@@ -563,8 +523,17 @@
                 //Allow the server to start even if the bean can't be registered
             }
 
+            if (FBUtilities.isWindows())
+            {
+                // We need to adjust the system timer on windows from the default 15ms down to the minimum of 1ms as this
+                // impacts timer intervals, thread scheduling, driver interrupts, etc.
+                WindowsTimer.startTimerPeriod(DatabaseDescriptor.getWindowsTimerInterval());
+            }
+
             setup();
 
+            String pidFile = System.getProperty("cassandra-pidfile");
+
             if (pidFile != null)
             {
                 new File(pidFile).deleteOnExit();
@@ -580,13 +549,27 @@
         }
         catch (Throwable e)
         {
-            logger.error("Exception encountered during startup", e);
+            boolean logStackTrace =
+                    e instanceof ConfigurationException ? ((ConfigurationException)e).logStackTrace : true;
 
-            // try to warn user on stdout too, if we haven't already detached
-            e.printStackTrace();
-            System.out.println("Exception encountered during startup: " + e.getMessage());
+            System.out.println("Exception (" + e.getClass().getName() + ") encountered during startup: " + e.getMessage());
 
-            System.exit(3);
+            if (logStackTrace)
+            {
+                if (runManaged)
+                    logger.error("Exception encountered during startup", e);
+                // try to warn user on stdout too, if we haven't already detached
+                e.printStackTrace();
+                exitOrFail(3, "Exception encountered during startup", e);
+            }
+            else
+            {
+                if (runManaged)
+                    logger.error("Exception encountered during startup: {}", e.getMessage());
+                // try to warn user on stdout too, if we haven't already detached
+                System.err.println(e.getMessage());
+                exitOrFail(3, "Exception encountered during startup: " + e.getMessage());
+            }
         }
     }
 
@@ -597,6 +580,10 @@
     {
         stop();
         destroy();
+        // completely shut down cassandra
+        if(!runManaged) {
+            System.exit(0);
+        }
     }
 
     private void waitForGossipToSettle()
@@ -618,9 +605,9 @@
         while (numOkay < GOSSIP_SETTLE_POLL_SUCCESSES_REQUIRED)
         {
             Uninterruptibles.sleepUninterruptibly(GOSSIP_SETTLE_POLL_INTERVAL_MS, TimeUnit.MILLISECONDS);
-            long completed = gossipStage.getCompletedTasks();
-            long active = gossipStage.getActiveCount();
-            long pending = gossipStage.getPendingTasks();
+            long completed = gossipStage.metrics.completedTasks.getValue();
+            long active = gossipStage.metrics.activeTasks.getValue();
+            long pending = gossipStage.metrics.pendingTasks.getValue();
             totalPolls++;
             if (active == 0 && pending == 0)
             {
@@ -655,6 +642,22 @@
         instance.activate();
     }
 
+    private void exitOrFail(int code, String message) {
+        exitOrFail(code, message, null);
+    }
+
+    private void exitOrFail(int code, String message, Throwable cause) {
+            if(runManaged) {
+                RuntimeException t = cause!=null ? new RuntimeException(message, cause) : new RuntimeException(message);
+                throw t;
+            }
+            else {
+                logger.error(message, cause);
+                System.exit(code);
+            }
+
+        }
+
     static class NativeAccess implements NativeAccessMBean
     {
         public boolean isAvailable()
diff --git a/src/java/org/apache/cassandra/service/ClientState.java b/src/java/org/apache/cassandra/service/ClientState.java
index f2e3f1c..1218928 100644
--- a/src/java/org/apache/cassandra/service/ClientState.java
+++ b/src/java/org/apache/cassandra/service/ClientState.java
@@ -17,31 +17,34 @@
  */
 package org.apache.cassandra.service;
 
+import java.net.InetSocketAddress;
 import java.net.SocketAddress;
-import java.util.*;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
 import java.util.concurrent.atomic.AtomicLong;
 
 import com.google.common.collect.Iterables;
-import com.google.common.collect.Sets;
-import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.auth.*;
+import org.apache.cassandra.config.Config;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.QueryHandler;
 import org.apache.cassandra.cql3.QueryProcessor;
-import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.exceptions.AuthenticationException;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.UnauthorizedException;
-import org.apache.cassandra.tracing.Tracing;
+import org.apache.cassandra.schema.LegacySchemaTables;
 import org.apache.cassandra.thrift.ThriftValidation;
+import org.apache.cassandra.tracing.TraceKeyspace;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.JVMStabilityInspector;
-import org.apache.cassandra.utils.SemanticVersion;
+import org.apache.cassandra.utils.CassandraVersion;
 
 /**
  * State related to a client connection.
@@ -49,27 +52,40 @@
 public class ClientState
 {
     private static final Logger logger = LoggerFactory.getLogger(ClientState.class);
-    public static final SemanticVersion DEFAULT_CQL_VERSION = org.apache.cassandra.cql3.QueryProcessor.CQL_VERSION;
+    public static final CassandraVersion DEFAULT_CQL_VERSION = org.apache.cassandra.cql3.QueryProcessor.CQL_VERSION;
 
     private static final Set<IResource> READABLE_SYSTEM_RESOURCES = new HashSet<>();
     private static final Set<IResource> PROTECTED_AUTH_RESOURCES = new HashSet<>();
-
+    private static final Set<String> ALTERABLE_SYSTEM_KEYSPACES = new HashSet<>();
+    private static final Set<IResource> DROPPABLE_SYSTEM_TABLES = new HashSet<>();
     static
     {
         // We want these system cfs to be always readable to authenticated users since many tools rely on them
         // (nodetool, cqlsh, bulkloader, etc.)
-        for (String cf : Iterables.concat(Arrays.asList(SystemKeyspace.LOCAL_CF, SystemKeyspace.PEERS_CF), SystemKeyspace.allSchemaCfs))
-            READABLE_SYSTEM_RESOURCES.add(DataResource.columnFamily(Keyspace.SYSTEM_KS, cf));
+        for (String cf : Iterables.concat(Arrays.asList(SystemKeyspace.LOCAL, SystemKeyspace.PEERS), LegacySchemaTables.ALL))
+            READABLE_SYSTEM_RESOURCES.add(DataResource.table(SystemKeyspace.NAME, cf));
 
-        PROTECTED_AUTH_RESOURCES.addAll(DatabaseDescriptor.getAuthenticator().protectedResources());
-        PROTECTED_AUTH_RESOURCES.addAll(DatabaseDescriptor.getAuthorizer().protectedResources());
+        if (!Config.isClientMode())
+        {
+            PROTECTED_AUTH_RESOURCES.addAll(DatabaseDescriptor.getAuthenticator().protectedResources());
+            PROTECTED_AUTH_RESOURCES.addAll(DatabaseDescriptor.getAuthorizer().protectedResources());
+            PROTECTED_AUTH_RESOURCES.addAll(DatabaseDescriptor.getRoleManager().protectedResources());
+        }
+
+        // allow users with sufficient privileges to alter KS level options on AUTH_KS and
+        // TRACING_KS, and also to drop legacy tables (users, credentials, permissions) from
+        // AUTH_KS
+        ALTERABLE_SYSTEM_KEYSPACES.add(AuthKeyspace.NAME);
+        ALTERABLE_SYSTEM_KEYSPACES.add(TraceKeyspace.NAME);
+        DROPPABLE_SYSTEM_TABLES.add(DataResource.table(AuthKeyspace.NAME, PasswordAuthenticator.LEGACY_CREDENTIALS_TABLE));
+        DROPPABLE_SYSTEM_TABLES.add(DataResource.table(AuthKeyspace.NAME, CassandraRoleManager.LEGACY_USERS_TABLE));
+        DROPPABLE_SYSTEM_TABLES.add(DataResource.table(AuthKeyspace.NAME, CassandraAuthorizer.USER_PERMISSIONS));
     }
 
     // Current user for the session
     private volatile AuthenticatedUser user;
     private volatile String keyspace;
 
-    private SemanticVersion cqlVersion;
     private static final QueryHandler cqlQueryHandler;
     static
     {
@@ -79,7 +95,7 @@
         {
             try
             {
-                handler = (QueryHandler)FBUtilities.construct(customHandlerClass, "QueryHandler");
+                handler = FBUtilities.construct(customHandlerClass, "QueryHandler");
                 logger.info("Using {} as query handler for native protocol queries (as requested with -Dcassandra.custom_query_handler_class)", customHandlerClass);
             }
             catch (Exception e)
@@ -96,7 +112,7 @@
     public final boolean isInternal;
 
     // The remote address of the client - null for internal clients.
-    private final SocketAddress remoteAddress;
+    private final InetSocketAddress remoteAddress;
 
     // The biggest timestamp that was returned by getTimestamp/assigned to a query. This is global to ensure that the
     // timestamp assigned are strictly monotonic on a node, which is likely what user expect intuitively (more likely,
@@ -113,7 +129,7 @@
         this.remoteAddress = null;
     }
 
-    protected ClientState(SocketAddress remoteAddress)
+    protected ClientState(InetSocketAddress remoteAddress)
     {
         this.isInternal = false;
         this.remoteAddress = remoteAddress;
@@ -134,7 +150,7 @@
      */
     public static ClientState forExternalCalls(SocketAddress remoteAddress)
     {
-        return new ClientState(remoteAddress);
+        return new ClientState((InetSocketAddress)remoteAddress);
     }
 
     /**
@@ -216,7 +232,7 @@
         return cqlQueryHandler;
     }
 
-    public SocketAddress getRemoteAddress()
+    public InetSocketAddress getRemoteAddress()
     {
         return remoteAddress;
     }
@@ -247,10 +263,13 @@
      */
     public void login(AuthenticatedUser user) throws AuthenticationException
     {
-        if (!user.isAnonymous() && !Auth.isExistingUser(user.getName()))
-           throw new AuthenticationException(String.format("User %s doesn't exist - create it with CREATE USER query first",
-                                                           user.getName()));
-        this.user = user;
+        // Login privilege is not inherited via granted roles, so just
+        // verify that the role with the credentials that were actually
+        // supplied has it
+        if (user.isAnonymous() || DatabaseDescriptor.getRoleManager().canLogin(user.getPrimaryRole()))
+            this.user = user;
+        else
+            throw new AuthenticationException(String.format("%s is not permitted to log in", user.getName()));
     }
 
     public void hasAllKeyspacesAccess(Permission perm) throws UnauthorizedException
@@ -270,7 +289,7 @@
     throws UnauthorizedException, InvalidRequestException
     {
         ThriftValidation.validateColumnFamily(keyspace, columnFamily);
-        hasAccess(keyspace, perm, DataResource.columnFamily(keyspace, columnFamily));
+        hasAccess(keyspace, perm, DataResource.table(keyspace, columnFamily));
     }
 
     private void hasAccess(String keyspace, Permission perm, DataResource resource)
@@ -281,16 +300,46 @@
             return;
         validateLogin();
         preventSystemKSSchemaModification(keyspace, resource, perm);
-        if (perm.equals(Permission.SELECT) && READABLE_SYSTEM_RESOURCES.contains(resource))
+        if ((perm == Permission.SELECT) && READABLE_SYSTEM_RESOURCES.contains(resource))
             return;
         if (PROTECTED_AUTH_RESOURCES.contains(resource))
-            if (perm.equals(Permission.CREATE) || perm.equals(Permission.ALTER) || perm.equals(Permission.DROP))
+            if ((perm == Permission.CREATE) || (perm == Permission.ALTER) || (perm == Permission.DROP))
                 throw new UnauthorizedException(String.format("%s schema is protected", resource));
         ensureHasPermission(perm, resource);
     }
 
     public void ensureHasPermission(Permission perm, IResource resource) throws UnauthorizedException
     {
+        if (DatabaseDescriptor.getAuthorizer() instanceof AllowAllAuthorizer)
+            return;
+
+        // Access to built in functions is unrestricted
+        if(resource instanceof FunctionResource && resource.hasParent())
+            if (((FunctionResource)resource).getKeyspace().equals(SystemKeyspace.NAME))
+                return;
+
+        checkPermissionOnResourceChain(perm, resource);
+    }
+
+    // Convenience method called from checkAccess method of CQLStatement
+    // Also avoids needlessly creating lots of FunctionResource objects
+    public void ensureHasPermission(Permission permission, Function function)
+    {
+        // Save creating a FunctionResource is we don't need to
+        if (DatabaseDescriptor.getAuthorizer() instanceof AllowAllAuthorizer)
+            return;
+
+        // built in functions are always available to all
+        if (function.isNative())
+            return;
+
+        checkPermissionOnResourceChain(permission, FunctionResource.function(function.name().keyspace,
+                                                                             function.name().name,
+                                                                             function.argTypes()));
+    }
+
+    private void checkPermissionOnResourceChain(Permission perm, IResource resource)
+    {
         for (IResource r : Resources.chain(resource))
             if (authorize(r).contains(perm))
                 return;
@@ -304,17 +353,22 @@
     private void preventSystemKSSchemaModification(String keyspace, DataResource resource, Permission perm) throws UnauthorizedException
     {
         // we only care about schema modification.
-        if (!(perm.equals(Permission.ALTER) || perm.equals(Permission.DROP) || perm.equals(Permission.CREATE)))
+        if (!((perm == Permission.ALTER) || (perm == Permission.DROP) || (perm == Permission.CREATE)))
             return;
 
         // prevent system keyspace modification
-        if (Keyspace.SYSTEM_KS.equalsIgnoreCase(keyspace))
+        if (SystemKeyspace.NAME.equalsIgnoreCase(keyspace))
             throw new UnauthorizedException(keyspace + " keyspace is not user-modifiable.");
 
-        // we want to allow altering AUTH_KS and TRACING_KS.
-        Set<String> allowAlter = Sets.newHashSet(Auth.AUTH_KS, Tracing.TRACE_KS);
-        if (allowAlter.contains(keyspace.toLowerCase()) && !(resource.isKeyspaceLevel() && perm.equals(Permission.ALTER)))
+        // allow users with sufficient privileges to alter KS level options on AUTH_KS and
+        // TRACING_KS, and also to drop legacy tables (users, credentials, permissions) from
+        // AUTH_KS
+        if (ALTERABLE_SYSTEM_KEYSPACES.contains(resource.getKeyspace().toLowerCase())
+           && ((perm == Permission.ALTER && !resource.isKeyspaceLevel())
+               || (perm == Permission.DROP && !DROPPABLE_SYSTEM_TABLES.contains(resource))))
+        {
             throw new UnauthorizedException(String.format("Cannot %s %s", perm, resource));
+        }
     }
 
     public void validateLogin() throws UnauthorizedException
@@ -342,58 +396,18 @@
             throw new InvalidRequestException("You have not set a keyspace for this session");
     }
 
-    public void setCQLVersion(String str) throws InvalidRequestException
-    {
-        SemanticVersion version;
-        try
-        {
-            version = new SemanticVersion(str);
-        }
-        catch (IllegalArgumentException e)
-        {
-            throw new InvalidRequestException(e.getMessage());
-        }
-
-        SemanticVersion cql = org.apache.cassandra.cql.QueryProcessor.CQL_VERSION;
-        SemanticVersion cql3 = org.apache.cassandra.cql3.QueryProcessor.CQL_VERSION;
-
-        // We've made some backward incompatible changes between CQL3 beta1 and the final.
-        // It's ok because it was a beta, but it still mean we don't support 3.0.0-beta1 so reject it.
-        SemanticVersion cql3Beta = new SemanticVersion("3.0.0-beta1");
-        if (version.equals(cql3Beta))
-            throw new InvalidRequestException(String.format("There has been a few syntax breaking changes between 3.0.0-beta1 and 3.0.0 "
-                                                           + "(mainly the syntax for options of CREATE KEYSPACE and CREATE TABLE). 3.0.0-beta1 "
-                                                           + " is not supported; please upgrade to 3.0.0"));
-        if (version.isSupportedBy(cql))
-            cqlVersion = cql;
-        else if (version.isSupportedBy(cql3))
-            cqlVersion = cql3;
-        else
-            throw new InvalidRequestException(String.format("Provided version %s is not supported by this server (supported: %s)",
-                                                            version,
-                                                            StringUtils.join(getCQLSupportedVersion(), ", ")));
-    }
-
     public AuthenticatedUser getUser()
     {
         return user;
     }
 
-    public SemanticVersion getCQLVersion()
+    public static CassandraVersion[] getCQLSupportedVersion()
     {
-        return cqlVersion;
-    }
-
-    public static SemanticVersion[] getCQLSupportedVersion()
-    {
-        SemanticVersion cql = org.apache.cassandra.cql.QueryProcessor.CQL_VERSION;
-        SemanticVersion cql3 = org.apache.cassandra.cql3.QueryProcessor.CQL_VERSION;
-
-        return new SemanticVersion[]{ cql, cql3 };
+        return new CassandraVersion[]{ QueryProcessor.CQL_VERSION };
     }
 
     private Set<Permission> authorize(IResource resource)
     {
-        return Auth.getPermissions(user, resource);
+        return user.getPermissions(resource);
     }
 }
diff --git a/src/java/org/apache/cassandra/service/ClientWarn.java b/src/java/org/apache/cassandra/service/ClientWarn.java
new file mode 100644
index 0000000..ddad197
--- /dev/null
+++ b/src/java/org/apache/cassandra/service/ClientWarn.java
@@ -0,0 +1,87 @@
+/*
+ * 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.service;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.concurrent.ExecutorLocal;
+import org.apache.cassandra.utils.FBUtilities;
+
+public class ClientWarn implements ExecutorLocal<ClientWarn.State>
+{
+    private static final String TRUNCATED = " [truncated]";
+    private static final ThreadLocal<ClientWarn.State> warnLocal = new ThreadLocal<>();
+    public static ClientWarn instance = new ClientWarn();
+
+    private ClientWarn()
+    {
+    }
+
+    public State get() {
+        return warnLocal.get();
+    }
+
+    public void set(State value) {
+        warnLocal.set(value);
+    }
+
+    public void warn(String text)
+    {
+        State state = warnLocal.get();
+        if (state != null)
+            state.add(text);
+    }
+
+    public void captureWarnings()
+    {
+        warnLocal.set(new State());
+    }
+
+    public List<String> getWarnings()
+    {
+        State state = warnLocal.get();
+        if (state == null || state.warnings.isEmpty())
+            return null;
+        return state.warnings;
+    }
+
+    public void resetWarnings()
+    {
+        warnLocal.remove();
+    }
+
+    public static class State
+    {
+        private final List<String> warnings = new ArrayList<>();
+
+        private void add(String warning)
+        {
+            if (warnings.size() < FBUtilities.MAX_UNSIGNED_SHORT)
+                warnings.add(maybeTruncate(warning));
+        }
+
+        private static String maybeTruncate(String warning)
+        {
+            return warning.length() > FBUtilities.MAX_UNSIGNED_SHORT
+                   ? warning.substring(0, FBUtilities.MAX_UNSIGNED_SHORT - TRUNCATED.length()) + TRUNCATED
+                   : warning;
+        }
+
+    }
+}
diff --git a/src/java/org/apache/cassandra/service/DatacenterSyncWriteResponseHandler.java b/src/java/org/apache/cassandra/service/DatacenterSyncWriteResponseHandler.java
index 81ae1f3..b095c7f 100644
--- a/src/java/org/apache/cassandra/service/DatacenterSyncWriteResponseHandler.java
+++ b/src/java/org/apache/cassandra/service/DatacenterSyncWriteResponseHandler.java
@@ -20,6 +20,7 @@
 import java.net.InetAddress;
 import java.util.Collection;
 import java.util.HashMap;
+import java.util.Map;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
@@ -33,12 +34,11 @@
 /**
  * This class blocks for a quorum of responses _in all datacenters_ (CL.EACH_QUORUM).
  */
-public class DatacenterSyncWriteResponseHandler extends AbstractWriteResponseHandler
+public class DatacenterSyncWriteResponseHandler<T> extends AbstractWriteResponseHandler<T>
 {
     private static final IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
 
-    private final NetworkTopologyStrategy strategy;
-    private final HashMap<String, AtomicInteger> responses = new HashMap<String, AtomicInteger>();
+    private final Map<String, AtomicInteger> responses = new HashMap<String, AtomicInteger>();
     private final AtomicInteger acks = new AtomicInteger(0);
 
     public DatacenterSyncWriteResponseHandler(Collection<InetAddress> naturalEndpoints,
@@ -52,7 +52,7 @@
         super(keyspace, naturalEndpoints, pendingEndpoints, consistencyLevel, callback, writeType);
         assert consistencyLevel == ConsistencyLevel.EACH_QUORUM;
 
-        strategy = (NetworkTopologyStrategy) keyspace.getReplicationStrategy();
+        NetworkTopologyStrategy strategy = (NetworkTopologyStrategy) keyspace.getReplicationStrategy();
 
         for (String dc : strategy.getDatacenters())
         {
@@ -68,7 +68,7 @@
         }
     }
 
-    public void response(MessageIn message)
+    public void response(MessageIn<T> message)
     {
         String dataCenter = message == null
                             ? DatabaseDescriptor.getLocalDataCenter()
diff --git a/src/java/org/apache/cassandra/service/DatacenterWriteResponseHandler.java b/src/java/org/apache/cassandra/service/DatacenterWriteResponseHandler.java
index fb8f992..b1b7b10 100644
--- a/src/java/org/apache/cassandra/service/DatacenterWriteResponseHandler.java
+++ b/src/java/org/apache/cassandra/service/DatacenterWriteResponseHandler.java
@@ -28,7 +28,7 @@
 /**
  * This class blocks for a quorum of responses _in the local datacenter only_ (CL.LOCAL_QUORUM).
  */
-public class DatacenterWriteResponseHandler extends WriteResponseHandler
+public class DatacenterWriteResponseHandler<T> extends WriteResponseHandler<T>
 {
     public DatacenterWriteResponseHandler(Collection<InetAddress> naturalEndpoints,
                                           Collection<InetAddress> pendingEndpoints,
@@ -42,9 +42,9 @@
     }
 
     @Override
-    public void response(MessageIn message)
+    public void response(MessageIn<T> message)
     {
-        if (message == null || consistencyLevel.isLocal(message.from))
+        if (message == null || waitingFor(message.from))
             super.response(message);
     }
 
@@ -55,4 +55,10 @@
         // or we may fail the consistency level guarantees (see #833, #8058)
         return consistencyLevel.blockFor(keyspace) + consistencyLevel.countLocalEndpoints(pendingEndpoints);
     }
+
+    @Override
+    protected boolean waitingFor(InetAddress from)
+    {
+        return consistencyLevel.isLocal(from);
+    }
 }
diff --git a/src/java/org/apache/cassandra/service/EchoVerbHandler.java b/src/java/org/apache/cassandra/service/EchoVerbHandler.java
index 57e7a0d..3d3f69e 100644
--- a/src/java/org/apache/cassandra/service/EchoVerbHandler.java
+++ b/src/java/org/apache/cassandra/service/EchoVerbHandler.java
@@ -35,7 +35,7 @@
 
     public void doVerb(MessageIn<EchoMessage> message, int id)
     {
-        MessageOut<EchoMessage> echoMessage = new MessageOut<EchoMessage>(MessagingService.Verb.REQUEST_RESPONSE, new EchoMessage(), EchoMessage.serializer);
+        MessageOut<EchoMessage> echoMessage = new MessageOut<EchoMessage>(MessagingService.Verb.REQUEST_RESPONSE, EchoMessage.instance, EchoMessage.serializer);
         logger.trace("Sending a EchoMessage reply {}", message.from);
         MessagingService.instance().sendReply(echoMessage, id, message.from);
     }
diff --git a/src/java/org/apache/cassandra/service/EmbeddedCassandraService.java b/src/java/org/apache/cassandra/service/EmbeddedCassandraService.java
index 659d851..6c154cd 100644
--- a/src/java/org/apache/cassandra/service/EmbeddedCassandraService.java
+++ b/src/java/org/apache/cassandra/service/EmbeddedCassandraService.java
@@ -19,6 +19,7 @@
 
 import java.io.IOException;
 
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.service.CassandraDaemon;
 
 /**
@@ -49,6 +50,7 @@
     public void start() throws IOException
     {
         cassandraDaemon = new CassandraDaemon();
+        DatabaseDescriptor.setDaemonInitialized();
         cassandraDaemon.init(null);
         cassandraDaemon.start();
     }
diff --git a/src/java/org/apache/cassandra/service/FileCacheService.java b/src/java/org/apache/cassandra/service/FileCacheService.java
index 250e625..19d6a70 100644
--- a/src/java/org/apache/cassandra/service/FileCacheService.java
+++ b/src/java/org/apache/cassandra/service/FileCacheService.java
@@ -143,11 +143,12 @@
         }
     }
 
+    @SuppressWarnings("resource")
     public void put(CacheKey cacheKey, RandomAccessReader instance)
     {
         int memoryUsed = memoryUsage.get();
-        if (logger.isDebugEnabled())
-            logger.debug("Estimated memory usage is {} compared to actual usage {}", memoryUsed, sizeInBytes());
+        if (logger.isTraceEnabled())
+            logger.trace("Estimated memory usage is {} compared to actual usage {}", memoryUsed, sizeInBytes());
 
         CacheBucket bucket = cache.getIfPresent(cacheKey);
         if (memoryUsed >= MEMORY_USAGE_THRESHOLD || bucket == null)
diff --git a/src/java/org/apache/cassandra/service/GCInspector.java b/src/java/org/apache/cassandra/service/GCInspector.java
index 2d3d4f3..31de151 100644
--- a/src/java/org/apache/cassandra/service/GCInspector.java
+++ b/src/java/org/apache/cassandra/service/GCInspector.java
@@ -20,6 +20,7 @@
 import java.lang.management.GarbageCollectorMXBean;
 import java.lang.management.ManagementFactory;
 import java.lang.management.MemoryUsage;
+import java.lang.reflect.Field;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
@@ -39,6 +40,7 @@
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
+
 import org.apache.cassandra.io.sstable.SSTableDeletingTask;
 import org.apache.cassandra.utils.StatusLogger;
 
@@ -46,10 +48,34 @@
 {
     public static final String MBEAN_NAME = "org.apache.cassandra.service:type=GCInspector";
     private static final Logger logger = LoggerFactory.getLogger(GCInspector.class);
-    final static long MIN_LOG_DURATION = 200;
+    final static long MIN_LOG_DURATION = DatabaseDescriptor.getGCLogThreshold();
     final static long GC_WARN_THRESHOLD_IN_MS = DatabaseDescriptor.getGCWarnThreshold();
     final static long STAT_THRESHOLD = GC_WARN_THRESHOLD_IN_MS != 0 ? GC_WARN_THRESHOLD_IN_MS : MIN_LOG_DURATION;
 
+    /*
+     * The field from java.nio.Bits that tracks the total number of allocated
+     * bytes of direct memory requires via ByteBuffer.allocateDirect that have not been GCed.
+     */
+    final static Field BITS_TOTAL_CAPACITY;
+
+    static
+    {
+        Field temp = null;
+        try
+        {
+            Class<?> bitsClass = Class.forName("java.nio.Bits");
+            Field f = bitsClass.getDeclaredField("totalCapacity");
+            f.setAccessible(true);
+            temp = f;
+        }
+        catch (Throwable t)
+        {
+            logger.debug("Error accessing field of java.nio.Bits", t);
+            //Don't care, will just return the dummy value -1 if we can't get at the field in this JVM
+        }
+        BITS_TOTAL_CAPACITY = temp;
+    }
+
     static final class State
     {
         final double maxRealTimeElapsed;
@@ -256,8 +282,8 @@
                 logger.warn(st);
             else if (duration > MIN_LOG_DURATION)
                 logger.info(st);
-            else if (logger.isDebugEnabled())
-                logger.debug(st);
+            else if (logger.isTraceEnabled())
+                logger.trace(st);
 
             if (duration > STAT_THRESHOLD)
                 StatusLogger.log();
@@ -276,13 +302,30 @@
     public double[] getAndResetStats()
     {
         State state = getTotalSinceLastCheck();
-        double[] r = new double[6];
+        double[] r = new double[7];
         r[0] = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - state.startNanos);
         r[1] = state.maxRealTimeElapsed;
         r[2] = state.totalRealTimeElapsed;
         r[3] = state.sumSquaresRealTimeElapsed;
         r[4] = state.totalBytesReclaimed;
         r[5] = state.count;
+        r[6] = getAllocatedDirectMemory();
+
         return r;
     }
+
+    private static long getAllocatedDirectMemory()
+    {
+        if (BITS_TOTAL_CAPACITY == null) return -1;
+        try
+        {
+            return BITS_TOTAL_CAPACITY.getLong(null);
+        }
+        catch (Throwable t)
+        {
+            logger.trace("Error accessing field of java.nio.Bits", t);
+            //Don't care how or why we failed to get the value in this JVM. Return -1 to indicate failure
+            return -1;
+        }
+    }
 }
diff --git a/src/java/org/apache/cassandra/service/LoadBroadcaster.java b/src/java/org/apache/cassandra/service/LoadBroadcaster.java
index d12ffba..945dd2f 100644
--- a/src/java/org/apache/cassandra/service/LoadBroadcaster.java
+++ b/src/java/org/apache/cassandra/service/LoadBroadcaster.java
@@ -23,6 +23,7 @@
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.cassandra.metrics.StorageMetrics;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -31,7 +32,7 @@
 
 public class LoadBroadcaster implements IEndpointStateChangeSubscriber
 {
-    static final int BROADCAST_INTERVAL = 60 * 1000;
+    static final int BROADCAST_INTERVAL = Integer.getInteger("cassandra.broadcast_interval_ms", 60 * 1000);
 
     public static final LoadBroadcaster instance = new LoadBroadcaster();
 
@@ -86,10 +87,10 @@
         {
             public void run()
             {
-                if (logger.isDebugEnabled())
-                    logger.debug("Disseminating load info ...");
+                if (logger.isTraceEnabled())
+                    logger.trace("Disseminating load info ...");
                 Gossiper.instance.addLocalApplicationState(ApplicationState.LOAD,
-                                                           StorageService.instance.valueFactory.load(StorageService.instance.getLoad()));
+                                                           StorageService.instance.valueFactory.load(StorageMetrics.load.getCount()));
             }
         };
         ScheduledExecutors.scheduledTasks.scheduleWithFixedDelay(runnable, 2 * Gossiper.intervalInMillis, BROADCAST_INTERVAL, TimeUnit.MILLISECONDS);
diff --git a/src/java/org/apache/cassandra/service/MigrationListener.java b/src/java/org/apache/cassandra/service/MigrationListener.java
index 1dcf44a..358b236 100644
--- a/src/java/org/apache/cassandra/service/MigrationListener.java
+++ b/src/java/org/apache/cassandra/service/MigrationListener.java
@@ -17,17 +17,69 @@
  */
 package org.apache.cassandra.service;
 
+import java.util.List;
+
+import org.apache.cassandra.db.marshal.AbstractType;
+
 public abstract class MigrationListener
 {
-    public void onCreateKeyspace(String ksName) {}
-    public void onCreateColumnFamily(String ksName, String cfName) {}
-    public void onCreateUserType(String ksName, String typeName) {}
+    public void onCreateKeyspace(String ksName)
+    {
+    }
 
-    public void onUpdateKeyspace(String ksName) {}
-    public void onUpdateColumnFamily(String ksName, String cfName, boolean columnsDidChange) {}
-    public void onUpdateUserType(String ksName, String typeName) {}
+    public void onCreateColumnFamily(String ksName, String cfName)
+    {
+    }
 
-    public void onDropKeyspace(String ksName) {}
-    public void onDropColumnFamily(String ksName, String cfName) {}
-    public void onDropUserType(String ksName, String typeName) {}
+    public void onCreateUserType(String ksName, String typeName)
+    {
+    }
+
+    public void onCreateFunction(String ksName, String functionName, List<AbstractType<?>> argTypes)
+    {
+    }
+
+    public void onCreateAggregate(String ksName, String aggregateName, List<AbstractType<?>> argTypes)
+    {
+    }
+
+    public void onUpdateKeyspace(String ksName)
+    {
+    }
+
+    public void onUpdateColumnFamily(String ksName, String cfName, boolean columnsDidChange)
+    {
+    }
+
+    public void onUpdateUserType(String ksName, String typeName)
+    {
+    }
+
+    public void onUpdateFunction(String ksName, String functionName, List<AbstractType<?>> argTypes)
+    {
+    }
+
+    public void onUpdateAggregate(String ksName, String aggregateName, List<AbstractType<?>> argTypes)
+    {
+    }
+
+    public void onDropKeyspace(String ksName)
+    {
+    }
+
+    public void onDropColumnFamily(String ksName, String cfName)
+    {
+    }
+
+    public void onDropUserType(String ksName, String typeName)
+    {
+    }
+
+    public void onDropFunction(String ksName, String functionName, List<AbstractType<?>> argTypes)
+    {
+    }
+
+    public void onDropAggregate(String ksName, String aggregateName, List<AbstractType<?>> argTypes)
+    {
+    }
 }
diff --git a/src/java/org/apache/cassandra/service/MigrationManager.java b/src/java/org/apache/cassandra/service/MigrationManager.java
index bebfa43..4cb5ed7 100644
--- a/src/java/org/apache/cassandra/service/MigrationManager.java
+++ b/src/java/org/apache/cassandra/service/MigrationManager.java
@@ -37,8 +37,9 @@
 import org.apache.cassandra.concurrent.StageManager;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.KSMetaData;
-import org.apache.cassandra.config.UTMetaData;
 import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.cql3.functions.UDAggregate;
+import org.apache.cassandra.cql3.functions.UDFunction;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.marshal.UserType;
 import org.apache.cassandra.exceptions.AlreadyExistsException;
@@ -48,6 +49,7 @@
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.LegacySchemaTables;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.WrappedRunnable;
 
@@ -61,7 +63,7 @@
 
     public static final int MIGRATION_DELAY_IN_MS = 60000;
 
-    private final List<MigrationListener> listeners = new CopyOnWriteArrayList<MigrationListener>();
+    private final List<MigrationListener> listeners = new CopyOnWriteArrayList<>();
     
     private MigrationManager() {}
 
@@ -127,7 +129,7 @@
                     submitMigrationTask(endpoint);
                 }
             };
-            ScheduledExecutors.optionalTasks.schedule(runnable, MIGRATION_DELAY_IN_MS, TimeUnit.MILLISECONDS);
+            ScheduledExecutors.nonPeriodicTasks.schedule(runnable, MIGRATION_DELAY_IN_MS, TimeUnit.MILLISECONDS);
         }
     }
 
@@ -148,7 +150,7 @@
          */
         return MessagingService.instance().knowsVersion(endpoint)
                 && MessagingService.instance().getRawVersion(endpoint) == MessagingService.current_version
-                && !Gossiper.instance.isFatClient(endpoint);
+                && !Gossiper.instance.isGossipOnlyMember(endpoint);
     }
 
     public static boolean isReadyForBootstrap()
@@ -174,6 +176,19 @@
             listener.onCreateUserType(ut.keyspace, ut.getNameAsString());
     }
 
+    public void notifyCreateFunction(UDFunction udf)
+    {
+        for (MigrationListener listener : listeners)
+            listener.onCreateFunction(udf.name().keyspace, udf.name().name, udf.argTypes());
+    }
+
+
+    public void notifyCreateAggregate(UDAggregate udf)
+    {
+        for (MigrationListener listener : listeners)
+            listener.onCreateAggregate(udf.name().keyspace, udf.name().name, udf.argTypes());
+    }
+
     public void notifyUpdateKeyspace(KSMetaData ksm)
     {
         for (MigrationListener listener : listeners)
@@ -192,6 +207,18 @@
             listener.onUpdateUserType(ut.keyspace, ut.getNameAsString());
     }
 
+    public void notifyUpdateFunction(UDFunction udf)
+    {
+        for (MigrationListener listener : listeners)
+            listener.onUpdateFunction(udf.name().keyspace, udf.name().name, udf.argTypes());
+    }
+
+    public void notifyUpdateAggregate(UDAggregate udf)
+    {
+        for (MigrationListener listener : listeners)
+            listener.onUpdateAggregate(udf.name().keyspace, udf.name().name, udf.argTypes());
+    }
+
     public void notifyDropKeyspace(KSMetaData ksm)
     {
         for (MigrationListener listener : listeners)
@@ -210,6 +237,18 @@
             listener.onDropUserType(ut.keyspace, ut.getNameAsString());
     }
 
+    public void notifyDropFunction(UDFunction udf)
+    {
+        for (MigrationListener listener : listeners)
+            listener.onDropFunction(udf.name().keyspace, udf.name().name, udf.argTypes());
+    }
+
+    public void notifyDropAggregate(UDAggregate udf)
+    {
+        for (MigrationListener listener : listeners)
+            listener.onDropAggregate(udf.name().keyspace, udf.name().name, udf.argTypes());
+    }
+
     public static void announceNewKeyspace(KSMetaData ksm) throws ConfigurationException
     {
         announceNewKeyspace(ksm, false);
@@ -228,7 +267,7 @@
             throw new AlreadyExistsException(ksm.name);
 
         logger.info(String.format("Create new Keyspace: %s", ksm));
-        announce(ksm.toSchema(timestamp), announceLocally);
+        announce(LegacySchemaTables.makeCreateKeyspaceMutation(ksm, timestamp), announceLocally);
     }
 
     public static void announceNewColumnFamily(CFMetaData cfm) throws ConfigurationException
@@ -238,26 +277,56 @@
 
     public static void announceNewColumnFamily(CFMetaData cfm, boolean announceLocally) throws ConfigurationException
     {
+        announceNewColumnFamily(cfm, announceLocally, true);
+    }
+
+    /**
+     * Announces the table even if the definition is already know locally.
+     * This should generally be avoided but is used internally when we want to force the most up to date version of
+     * a system table schema (Note that we don't know if the schema we force _is_ the most recent version or not, we
+     * just rely on idempotency to basically ignore that announce if it's not. That's why we can't use announceUpdateColumnFamily,
+     * it would for instance delete new columns if this is not called with the most up-to-date version)
+     *
+     * Note that this is only safe for system tables where we know the cfId is fixed and will be the same whatever version
+     * of the definition is used.
+     */
+    public static void forceAnnounceNewColumnFamily(CFMetaData cfm) throws ConfigurationException
+    {
+        announceNewColumnFamily(cfm, false, false);
+    }
+
+    private static void announceNewColumnFamily(CFMetaData cfm, boolean announceLocally, boolean throwOnDuplicate) throws ConfigurationException
+    {
         cfm.validate();
 
         KSMetaData ksm = Schema.instance.getKSMetaData(cfm.ksName);
         if (ksm == null)
-            throw new ConfigurationException(String.format("Cannot add column family '%s' to non existing keyspace '%s'.", cfm.cfName, cfm.ksName));
-        else if (ksm.cfMetaData().containsKey(cfm.cfName))
+            throw new ConfigurationException(String.format("Cannot add table '%s' to non existing keyspace '%s'.", cfm.cfName, cfm.ksName));
+        else if (throwOnDuplicate && ksm.cfMetaData().containsKey(cfm.cfName))
             throw new AlreadyExistsException(cfm.ksName, cfm.cfName);
 
-        logger.info(String.format("Create new ColumnFamily: %s", cfm));
-        announce(addSerializedKeyspace(cfm.toSchema(FBUtilities.timestampMicros()), cfm.ksName), announceLocally);
-    }
-
-    public static void announceNewType(UserType newType)
-    {
-        announceNewType(newType, false);
+        logger.info(String.format("Create new table: %s", cfm));
+        announce(LegacySchemaTables.makeCreateTableMutation(ksm, cfm, FBUtilities.timestampMicros()), announceLocally);
     }
 
     public static void announceNewType(UserType newType, boolean announceLocally)
     {
-        announce(addSerializedKeyspace(UTMetaData.toSchema(newType, FBUtilities.timestampMicros()), newType.keyspace), announceLocally);
+        KSMetaData ksm = Schema.instance.getKSMetaData(newType.keyspace);
+        announce(LegacySchemaTables.makeCreateTypeMutation(ksm, newType, FBUtilities.timestampMicros()), announceLocally);
+    }
+
+    public static void announceNewFunction(UDFunction udf, boolean announceLocally)
+    {
+        logger.info(String.format("Create scalar function '%s'", udf.name()));
+        KSMetaData ksm = Schema.instance.getKSMetaData(udf.name().keyspace);
+        announce(LegacySchemaTables.makeCreateFunctionMutation(ksm, udf, FBUtilities.timestampMicros()), announceLocally);
+    }
+
+    public static void announceNewAggregate(UDAggregate udf, boolean announceLocally)
+    {
+        logger.info(String.format("Create aggregate function '%s'", udf.name()));
+        KSMetaData ksm = Schema.instance.getKSMetaData(udf.name().keyspace);
+        announce(LegacySchemaTables.makeCreateAggregateMutation(ksm, udf, FBUtilities.timestampMicros()), announceLocally);
     }
 
     public static void announceKeyspaceUpdate(KSMetaData ksm) throws ConfigurationException
@@ -274,31 +343,27 @@
             throw new ConfigurationException(String.format("Cannot update non existing keyspace '%s'.", ksm.name));
 
         logger.info(String.format("Update Keyspace '%s' From %s To %s", ksm.name, oldKsm, ksm));
-        announce(oldKsm.toSchemaUpdate(ksm, FBUtilities.timestampMicros()), announceLocally);
+        announce(LegacySchemaTables.makeCreateKeyspaceMutation(ksm, FBUtilities.timestampMicros()), announceLocally);
     }
 
-    public static void announceColumnFamilyUpdate(CFMetaData cfm, boolean fromThrift) throws ConfigurationException
+    public static void announceColumnFamilyUpdate(CFMetaData cfm) throws ConfigurationException
     {
-        announceColumnFamilyUpdate(cfm, fromThrift, false);
+        announceColumnFamilyUpdate(cfm, false);
     }
 
-    public static void announceColumnFamilyUpdate(CFMetaData cfm, boolean fromThrift, boolean announceLocally) throws ConfigurationException
+    public static void announceColumnFamilyUpdate(CFMetaData cfm, boolean announceLocally) throws ConfigurationException
     {
         cfm.validate();
 
         CFMetaData oldCfm = Schema.instance.getCFMetaData(cfm.ksName, cfm.cfName);
         if (oldCfm == null)
-            throw new ConfigurationException(String.format("Cannot update non existing column family '%s' in keyspace '%s'.", cfm.cfName, cfm.ksName));
+            throw new ConfigurationException(String.format("Cannot update non existing table '%s' in keyspace '%s'.", cfm.cfName, cfm.ksName));
+        KSMetaData ksm = Schema.instance.getKSMetaData(cfm.ksName);
 
         oldCfm.validateCompatility(cfm);
 
-        logger.info(String.format("Update ColumnFamily '%s/%s' From %s To %s", cfm.ksName, cfm.cfName, oldCfm, cfm));
-        announce(addSerializedKeyspace(oldCfm.toSchemaUpdate(cfm, FBUtilities.timestampMicros(), fromThrift), cfm.ksName), announceLocally);
-    }
-
-    public static void announceTypeUpdate(UserType updatedType)
-    {
-        announceTypeUpdate(updatedType, false);
+        logger.info(String.format("Update table '%s/%s' From %s To %s", cfm.ksName, cfm.cfName, oldCfm, cfm));
+        announce(LegacySchemaTables.makeUpdateTableMutation(ksm, oldCfm, cfm, FBUtilities.timestampMicros()), announceLocally);
     }
 
     public static void announceTypeUpdate(UserType updatedType, boolean announceLocally)
@@ -318,7 +383,7 @@
             throw new ConfigurationException(String.format("Cannot drop non existing keyspace '%s'.", ksName));
 
         logger.info(String.format("Drop Keyspace '%s'", oldKsm.name));
-        announce(oldKsm.dropFromSchema(FBUtilities.timestampMicros()), announceLocally);
+        announce(LegacySchemaTables.makeDropKeyspaceMutation(oldKsm, FBUtilities.timestampMicros()), announceLocally);
     }
 
     public static void announceColumnFamilyDrop(String ksName, String cfName) throws ConfigurationException
@@ -330,17 +395,11 @@
     {
         CFMetaData oldCfm = Schema.instance.getCFMetaData(ksName, cfName);
         if (oldCfm == null)
-            throw new ConfigurationException(String.format("Cannot drop non existing column family '%s' in keyspace '%s'.", cfName, ksName));
+            throw new ConfigurationException(String.format("Cannot drop non existing table '%s' in keyspace '%s'.", cfName, ksName));
+        KSMetaData ksm = Schema.instance.getKSMetaData(ksName);
 
-        logger.info(String.format("Drop ColumnFamily '%s/%s'", oldCfm.ksName, oldCfm.cfName));
-        announce(addSerializedKeyspace(oldCfm.dropFromSchema(FBUtilities.timestampMicros()), ksName), announceLocally);
-    }
-
-    // Include the serialized keyspace for when a target node missed the CREATE KEYSPACE migration (see #5631).
-    private static Mutation addSerializedKeyspace(Mutation migration, String ksName)
-    {
-        migration.add(SystemKeyspace.readSchemaRow(SystemKeyspace.SCHEMA_KEYSPACES_CF, ksName).cf);
-        return migration;
+        logger.info(String.format("Drop table '%s/%s'", oldCfm.ksName, oldCfm.cfName));
+        announce(LegacySchemaTables.makeDropTableMutation(ksm, oldCfm, FBUtilities.timestampMicros()), announceLocally);
     }
 
     public static void announceTypeDrop(UserType droppedType)
@@ -350,7 +409,22 @@
 
     public static void announceTypeDrop(UserType droppedType, boolean announceLocally)
     {
-        announce(addSerializedKeyspace(UTMetaData.dropFromSchema(droppedType, FBUtilities.timestampMicros()), droppedType.keyspace), announceLocally);
+        KSMetaData ksm = Schema.instance.getKSMetaData(droppedType.keyspace);
+        announce(LegacySchemaTables.dropTypeFromSchemaMutation(ksm, droppedType, FBUtilities.timestampMicros()), announceLocally);
+    }
+
+    public static void announceFunctionDrop(UDFunction udf, boolean announceLocally)
+    {
+        logger.info(String.format("Drop scalar function overload '%s' args '%s'", udf.name(), udf.argTypes()));
+        KSMetaData ksm = Schema.instance.getKSMetaData(udf.name().keyspace);
+        announce(LegacySchemaTables.makeDropFunctionMutation(ksm, udf, FBUtilities.timestampMicros()), announceLocally);
+    }
+
+    public static void announceAggregateDrop(UDAggregate udf, boolean announceLocally)
+    {
+        logger.info(String.format("Drop aggregate function overload '%s' args '%s'", udf.name(), udf.argTypes()));
+        KSMetaData ksm = Schema.instance.getKSMetaData(udf.name().keyspace);
+        announce(LegacySchemaTables.makeDropAggregateMutation(ksm, udf, FBUtilities.timestampMicros()), announceLocally);
     }
 
     /**
@@ -363,9 +437,9 @@
         {
             try
             {
-                DefsTables.mergeSchemaInternal(Collections.singletonList(schema), false);
+                LegacySchemaTables.mergeSchema(Collections.singletonList(schema), false);
             }
-            catch (ConfigurationException | IOException e)
+            catch (IOException e)
             {
                 throw new RuntimeException(e);
             }
@@ -391,7 +465,7 @@
         {
             protected void runMayThrow() throws IOException, ConfigurationException
             {
-                DefsTables.mergeSchema(schema);
+                LegacySchemaTables.mergeSchema(schema);
             }
         });
 
@@ -431,9 +505,7 @@
 
         logger.debug("Truncating schema tables...");
 
-        // truncate schema tables
-        for (String cf : SystemKeyspace.allSchemaCfs)
-            SystemKeyspace.schemaCFS(cf).truncateBlocking();
+        LegacySchemaTables.truncateSchemaTables();
 
         logger.debug("Clearing local schema keyspace definitions...");
 
@@ -470,7 +542,7 @@
         public Collection<Mutation> deserialize(DataInput in, int version) throws IOException
         {
             int count = in.readInt();
-            Collection<Mutation> schema = new ArrayList<Mutation>(count);
+            Collection<Mutation> schema = new ArrayList<>(count);
 
             for (int i = 0; i < count; i++)
                 schema.add(Mutation.serializer.deserialize(in, version));
diff --git a/src/java/org/apache/cassandra/service/MigrationTask.java b/src/java/org/apache/cassandra/service/MigrationTask.java
index 8285d18..b065d90 100644
--- a/src/java/org/apache/cassandra/service/MigrationTask.java
+++ b/src/java/org/apache/cassandra/service/MigrationTask.java
@@ -26,7 +26,7 @@
 
 import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.db.DefsTables;
+import org.apache.cassandra.schema.LegacySchemaTables;
 import org.apache.cassandra.gms.FailureDetector;
 import org.apache.cassandra.net.IAsyncCallback;
 import org.apache.cassandra.net.MessageIn;
@@ -48,6 +48,12 @@
 
     public void runMayThrow() throws Exception
     {
+        if (!FailureDetector.instance.isAlive(endpoint))
+        {
+            logger.warn("Can't send schema pull request: node {} is down.", endpoint);
+            return;
+        }
+
         // There is a chance that quite some time could have passed between now and the MM#maybeScheduleSchemaPull(),
         // potentially enough for the endpoint node to restart - which is an issue if it does restart upgraded, with
         // a higher major.
@@ -57,12 +63,6 @@
             return;
         }
 
-        if (!FailureDetector.instance.isAlive(endpoint))
-        {
-            logger.debug("Can't send schema pull request: node {} is down.", endpoint);
-            return;
-        }
-
         MessageOut message = new MessageOut<>(MessagingService.Verb.MIGRATION_REQUEST, null, MigrationManager.MigrationsSerializer.instance);
 
         IAsyncCallback<Collection<Mutation>> cb = new IAsyncCallback<Collection<Mutation>>()
@@ -72,7 +72,7 @@
             {
                 try
                 {
-                    DefsTables.mergeSchema(message.payload);
+                    LegacySchemaTables.mergeSchema(message.payload);
                 }
                 catch (IOException e)
                 {
diff --git a/src/java/org/apache/cassandra/service/PendingRangeCalculatorService.java b/src/java/org/apache/cassandra/service/PendingRangeCalculatorService.java
index 1e7b7bd..116cede 100644
--- a/src/java/org/apache/cassandra/service/PendingRangeCalculatorService.java
+++ b/src/java/org/apache/cassandra/service/PendingRangeCalculatorService.java
@@ -18,68 +18,79 @@
 
 package org.apache.cassandra.service;
 
-import org.apache.cassandra.utils.BiMultiValMap;
-import com.google.common.collect.HashMultimap;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Multimap;
-import com.google.common.collect.Sets;
-
 import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.locator.AbstractReplicationStrategy;
-import org.apache.cassandra.locator.TokenMetadata;
-import org.apache.cassandra.utils.Pair;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.net.InetAddress;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.Collection;
+import java.util.List;
 import java.util.concurrent.*;
+import java.util.concurrent.atomic.AtomicInteger;
 
 public class PendingRangeCalculatorService
 {
     public static final PendingRangeCalculatorService instance = new PendingRangeCalculatorService();
 
     private static Logger logger = LoggerFactory.getLogger(PendingRangeCalculatorService.class);
+
+    // the executor will only run a single range calculation at a time while keeping at most one task queued in order
+    // to trigger an update only after the most recent state change and not for each update individually
     private final JMXEnabledThreadPoolExecutor executor = new JMXEnabledThreadPoolExecutor(1, Integer.MAX_VALUE, TimeUnit.SECONDS,
             new LinkedBlockingQueue<Runnable>(1), new NamedThreadFactory("PendingRangeCalculator"), "internal");
 
+    private AtomicInteger updateJobs = new AtomicInteger(0);
+
     public PendingRangeCalculatorService()
     {
-        executor.setRejectedExecutionHandler(new ThreadPoolExecutor.DiscardPolicy());
+        executor.setRejectedExecutionHandler(new RejectedExecutionHandler()
+        {
+            public void rejectedExecution(Runnable r, ThreadPoolExecutor e)
+            {
+                PendingRangeCalculatorService.instance.finishUpdate();
+            }
+        }
+        );
     }
 
     private static class PendingRangeTask implements Runnable
     {
         public void run()
         {
-            long start = System.currentTimeMillis();
-            for (String keyspaceName : Schema.instance.getNonSystemKeyspaces())
+            try
             {
-                calculatePendingRanges(Keyspace.open(keyspaceName).getReplicationStrategy(), keyspaceName);
+                long start = System.currentTimeMillis();
+                List<String> keyspaces = Schema.instance.getNonSystemKeyspaces();
+                for (String keyspaceName : keyspaces)
+                    calculatePendingRanges(Keyspace.open(keyspaceName).getReplicationStrategy(), keyspaceName);
+                if (logger.isTraceEnabled())
+                    logger.trace("Finished PendingRangeTask for {} keyspaces in {}ms", keyspaces.size(), System.currentTimeMillis() - start);
             }
-            logger.debug("finished calculation for {} keyspaces in {}ms", Schema.instance.getNonSystemKeyspaces().size(), System.currentTimeMillis() - start);
+            finally
+            {
+                PendingRangeCalculatorService.instance.finishUpdate();
+            }
         }
     }
 
-    public Future<?> update()
+    private void finishUpdate()
     {
-        return executor.submit(new PendingRangeTask());
+        updateJobs.decrementAndGet();
+    }
+
+    public void update()
+    {
+        updateJobs.incrementAndGet();
+        executor.submit(new PendingRangeTask());
     }
 
     public void blockUntilFinished()
     {
-        while (true)
+        // We want to be sure the job we're blocking for is actually finished and we can't trust the TPE's active job count
+        while (updateJobs.get() > 0)
         {
-            if (executor.getActiveCount() + executor.getPendingTasks() == 0)
-                break;
             try
             {
                 Thread.sleep(100);
@@ -91,130 +102,10 @@
         }
     }
 
-    /**
-     * Calculate pending ranges according to bootsrapping and leaving nodes. Reasoning is:
-     *
-     * (1) When in doubt, it is better to write too much to a node than too little. That is, if
-     * there are multiple nodes moving, calculate the biggest ranges a node could have. Cleaning
-     * up unneeded data afterwards is better than missing writes during movement.
-     * (2) When a node leaves, ranges for other nodes can only grow (a node might get additional
-     * ranges, but it will not lose any of its current ranges as a result of a leave). Therefore
-     * we will first remove _all_ leaving tokens for the sake of calculation and then check what
-     * ranges would go where if all nodes are to leave. This way we get the biggest possible
-     * ranges with regard current leave operations, covering all subsets of possible final range
-     * values.
-     * (3) When a node bootstraps, ranges of other nodes can only get smaller. Without doing
-     * complex calculations to see if multiple bootstraps overlap, we simply base calculations
-     * on the same token ring used before (reflecting situation after all leave operations have
-     * completed). Bootstrapping nodes will be added and removed one by one to that metadata and
-     * checked what their ranges would be. This will give us the biggest possible ranges the
-     * node could have. It might be that other bootstraps make our actual final ranges smaller,
-     * but it does not matter as we can clean up the data afterwards.
-     *
-     * NOTE: This is heavy and ineffective operation. This will be done only once when a node
-     * changes state in the cluster, so it should be manageable.
-     */
+
     // public & static for testing purposes
     public static void calculatePendingRanges(AbstractReplicationStrategy strategy, String keyspaceName)
     {
-        TokenMetadata tm = StorageService.instance.getTokenMetadata();
-        Multimap<Range<Token>, InetAddress> pendingRanges = HashMultimap.create();
-        BiMultiValMap<Token, InetAddress> bootstrapTokens = tm.getBootstrapTokens();
-        Set<InetAddress> leavingEndpoints = tm.getLeavingEndpoints();
-
-        if (bootstrapTokens.isEmpty() && leavingEndpoints.isEmpty() && tm.getMovingEndpoints().isEmpty())
-        {
-            if (logger.isDebugEnabled())
-                logger.debug("No bootstrapping, leaving or moving nodes, and no relocating tokens -> empty pending ranges for {}", keyspaceName);
-            tm.setPendingRanges(keyspaceName, pendingRanges);
-            return;
-        }
-
-        Multimap<InetAddress, Range<Token>> addressRanges = strategy.getAddressRanges();
-
-        // Copy of metadata reflecting the situation after all leave operations are finished.
-        TokenMetadata allLeftMetadata = tm.cloneAfterAllLeft();
-
-        // get all ranges that will be affected by leaving nodes
-        Set<Range<Token>> affectedRanges = new HashSet<Range<Token>>();
-        for (InetAddress endpoint : leavingEndpoints)
-            affectedRanges.addAll(addressRanges.get(endpoint));
-
-        // for each of those ranges, find what new nodes will be responsible for the range when
-        // all leaving nodes are gone.
-        TokenMetadata metadata = tm.cloneOnlyTokenMap(); // don't do this in the loop! #7758
-        for (Range<Token> range : affectedRanges)
-        {
-            Set<InetAddress> currentEndpoints = ImmutableSet.copyOf(strategy.calculateNaturalEndpoints(range.right, metadata));
-            Set<InetAddress> newEndpoints = ImmutableSet.copyOf(strategy.calculateNaturalEndpoints(range.right, allLeftMetadata));
-            pendingRanges.putAll(range, Sets.difference(newEndpoints, currentEndpoints));
-        }
-
-        // At this stage pendingRanges has been updated according to leave operations. We can
-        // now continue the calculation by checking bootstrapping nodes.
-
-        // For each of the bootstrapping nodes, simply add and remove them one by one to
-        // allLeftMetadata and check in between what their ranges would be.
-        Multimap<InetAddress, Token> bootstrapAddresses = bootstrapTokens.inverse();
-        for (InetAddress endpoint : bootstrapAddresses.keySet())
-        {
-            Collection<Token> tokens = bootstrapAddresses.get(endpoint);
-
-            allLeftMetadata.updateNormalTokens(tokens, endpoint);
-            for (Range<Token> range : strategy.getAddressRanges(allLeftMetadata).get(endpoint))
-                pendingRanges.put(range, endpoint);
-            allLeftMetadata.removeEndpoint(endpoint);
-        }
-
-        // At this stage pendingRanges has been updated according to leaving and bootstrapping nodes.
-        // We can now finish the calculation by checking moving and relocating nodes.
-
-        for (Pair<Token, InetAddress> moving : tm.getMovingEndpoints())
-        {
-            //Calculate all the ranges which will could be affected. This will include the ranges before and after the move.
-            Set<Range<Token>> moveAffectedRanges = new HashSet<>();
-            InetAddress endpoint = moving.right; // address of the moving node
-            //Add ranges before the move
-            for (Range<Token> range : strategy.getAddressRanges(allLeftMetadata).get(endpoint))
-            {
-                moveAffectedRanges.add(range);
-            }
-
-            allLeftMetadata.updateNormalToken(moving.left, endpoint);
-            //Add ranges after the move
-            for (Range<Token> range : strategy.getAddressRanges(allLeftMetadata).get(endpoint))
-            {
-                moveAffectedRanges.add(range);
-            }
-
-            for(Range<Token> range : moveAffectedRanges)
-            {
-                Set<InetAddress> currentEndpoints = ImmutableSet.copyOf(strategy.calculateNaturalEndpoints(range.right, metadata));
-                Set<InetAddress> newEndpoints = ImmutableSet.copyOf(strategy.calculateNaturalEndpoints(range.right, allLeftMetadata));
-                Set<InetAddress> difference = Sets.difference(newEndpoints, currentEndpoints);
-                for(final InetAddress address : difference)
-                {
-                    Collection<Range<Token>> newRanges = strategy.getAddressRanges(allLeftMetadata).get(address);
-                    Collection<Range<Token>> oldRanges = strategy.getAddressRanges(metadata).get(address);
-                    //We want to get rid of any ranges which the node is currently getting.
-                    newRanges.removeAll(oldRanges);
-
-                    for(Range<Token> newRange : newRanges)
-                    {
-                        for(Range<Token> pendingRange : newRange.subtractAll(oldRanges))
-                        {
-                            pendingRanges.put(pendingRange, address);
-                        }
-                    }
-                }
-            }
-
-            allLeftMetadata.removeEndpoint(endpoint);
-        }
-
-        tm.setPendingRanges(keyspaceName, pendingRanges);
-
-        if (logger.isDebugEnabled())
-            logger.debug("Pending ranges:\n" + (pendingRanges.isEmpty() ? "<empty>" : tm.printPendingRanges()));
+        StorageService.instance.getTokenMetadata().calculatePendingRanges(strategy, keyspaceName);
     }
 }
diff --git a/src/java/org/apache/cassandra/service/QueryState.java b/src/java/org/apache/cassandra/service/QueryState.java
index 9891ba0..ddbc959 100644
--- a/src/java/org/apache/cassandra/service/QueryState.java
+++ b/src/java/org/apache/cassandra/service/QueryState.java
@@ -17,11 +17,11 @@
  */
 package org.apache.cassandra.service;
 
+import java.net.InetAddress;
 import java.util.UUID;
 import java.util.concurrent.ThreadLocalRandom;
 
 import org.apache.cassandra.tracing.Tracing;
-import org.apache.cassandra.utils.FBUtilities;
 
 /**
  * Represents the state related to a given query.
@@ -65,8 +65,8 @@
             return true;
         }
 
-        double tracingProbability = StorageService.instance.getTracingProbability();
-        return tracingProbability != 0 && ThreadLocalRandom.current().nextDouble() < tracingProbability;
+        double traceProbability = StorageService.instance.getTraceProbability();
+        return traceProbability != 0 && ThreadLocalRandom.current().nextDouble() < traceProbability;
     }
 
     public void prepareTracingSession(UUID sessionId)
@@ -76,15 +76,22 @@
 
     public void createTracingSession()
     {
-        if (this.preparedTracingSession == null)
+        UUID session = this.preparedTracingSession;
+        if (session == null)
         {
             Tracing.instance.newSession();
         }
         else
         {
-            UUID session = this.preparedTracingSession;
-            this.preparedTracingSession = null;
             Tracing.instance.newSession(session);
+            this.preparedTracingSession = null;
         }
     }
+
+    public InetAddress getClientAddress()
+    {
+        return clientState.isInternal
+             ? null
+             : clientState.getRemoteAddress().getAddress();
+    }
 }
diff --git a/src/java/org/apache/cassandra/service/RangeSliceVerbHandler.java b/src/java/org/apache/cassandra/service/RangeSliceVerbHandler.java
index f1fd1f9..0f3726c 100644
--- a/src/java/org/apache/cassandra/service/RangeSliceVerbHandler.java
+++ b/src/java/org/apache/cassandra/service/RangeSliceVerbHandler.java
@@ -19,7 +19,6 @@
 
 import org.apache.cassandra.db.AbstractRangeCommand;
 import org.apache.cassandra.db.RangeSliceReply;
-import org.apache.cassandra.db.filter.TombstoneOverwhelmingException;
 import org.apache.cassandra.net.IVerbHandler;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessagingService;
@@ -29,24 +28,13 @@
 {
     public void doVerb(MessageIn<AbstractRangeCommand> message, int id)
     {
-        try
+        if (StorageService.instance.isBootstrapMode())
         {
-            if (StorageService.instance.isBootstrapMode())
-            {
-                /* Don't service reads! */
-                throw new RuntimeException("Cannot service reads while bootstrapping!");
-            }
-            RangeSliceReply reply = new RangeSliceReply(message.payload.executeLocally());
-            Tracing.trace("Enqueuing response to {}", message.from);
-            MessagingService.instance().sendReply(reply.createMessage(), id, message.from);
+            /* Don't service reads! */
+            throw new RuntimeException("Cannot service reads while bootstrapping!");
         }
-        catch (TombstoneOverwhelmingException e)
-        {
-            // error already logged.  Drop the request
-        }
-        catch (Exception ex)
-        {
-            throw new RuntimeException(ex);
-        }
+        RangeSliceReply reply = new RangeSliceReply(message.payload.executeLocally());
+        Tracing.trace("Enqueuing response to {}", message.from);
+        MessagingService.instance().sendReply(reply.createMessage(), id, message.from);
     }
 }
diff --git a/src/java/org/apache/cassandra/service/ReadCallback.java b/src/java/org/apache/cassandra/service/ReadCallback.java
index e0646a9..145679d 100644
--- a/src/java/org/apache/cassandra/service/ReadCallback.java
+++ b/src/java/org/apache/cassandra/service/ReadCallback.java
@@ -32,10 +32,11 @@
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ReadCommand;
 import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.exceptions.ReadFailureException;
 import org.apache.cassandra.exceptions.ReadTimeoutException;
 import org.apache.cassandra.exceptions.UnavailableException;
 import org.apache.cassandra.metrics.ReadRepairMetrics;
-import org.apache.cassandra.net.IAsyncCallback;
+import org.apache.cassandra.net.IAsyncCallbackWithFailure;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
@@ -45,7 +46,7 @@
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.concurrent.SimpleCondition;
 
-public class ReadCallback<TMessage, TResolved> implements IAsyncCallback<TMessage>
+public class ReadCallback<TMessage, TResolved> implements IAsyncCallbackWithFailure<TMessage>
 {
     protected static final Logger logger = LoggerFactory.getLogger( ReadCallback.class );
 
@@ -59,6 +60,10 @@
     private static final AtomicIntegerFieldUpdater<ReadCallback> recievedUpdater
             = AtomicIntegerFieldUpdater.newUpdater(ReadCallback.class, "received");
     private volatile int received = 0;
+    private static final AtomicIntegerFieldUpdater<ReadCallback> failuresUpdater
+            = AtomicIntegerFieldUpdater.newUpdater(ReadCallback.class, "failures");
+    private volatile int failures = 0;
+
     private final Keyspace keyspace; // TODO push this into ConsistencyLevel?
 
     /**
@@ -97,15 +102,24 @@
         }
     }
 
-    public TResolved get() throws ReadTimeoutException, DigestMismatchException
+    public TResolved get() throws ReadFailureException, ReadTimeoutException, DigestMismatchException
     {
         if (!await(command.getTimeout(), TimeUnit.MILLISECONDS))
         {
             // Same as for writes, see AbstractWriteResponseHandler
             ReadTimeoutException ex = new ReadTimeoutException(consistencyLevel, received, blockfor, resolver.isDataPresent());
             Tracing.trace("Read timeout: {}", ex.toString());
-            if (logger.isDebugEnabled())
-                logger.debug("Read timeout: {}", ex.toString());
+            if (logger.isTraceEnabled())
+                logger.trace("Read timeout: {}", ex.toString());
+            throw ex;
+        }
+
+        if (blockfor + failures > endpoints.size())
+        {
+            ReadFailureException ex = new ReadFailureException(consistencyLevel, received, failures, blockfor, resolver.isDataPresent());
+
+            if (logger.isTraceEnabled())
+                logger.trace("Read failure: {}", ex.toString());
             throw ex;
         }
 
@@ -115,7 +129,7 @@
     public void response(MessageIn<TMessage> message)
     {
         resolver.preprocess(message);
-        int n = waitingFor(message)
+        int n = waitingFor(message.from)
               ? recievedUpdater.incrementAndGet(this)
               : received;
         if (n >= blockfor && resolver.isDataPresent())
@@ -137,10 +151,10 @@
     /**
      * @return true if the message counts towards the blockfor threshold
      */
-    private boolean waitingFor(MessageIn message)
+    private boolean waitingFor(InetAddress from)
     {
         return consistencyLevel.isDatacenterLocal()
-             ? DatabaseDescriptor.getLocalDataCenter().equals(DatabaseDescriptor.getEndpointSnitch().getDatacenter(message.from))
+             ? DatabaseDescriptor.getLocalDataCenter().equals(DatabaseDescriptor.getEndpointSnitch().getDatacenter(from))
              : true;
     }
 
@@ -196,13 +210,13 @@
 
                 if (traceState != null)
                     traceState.trace("Digest mismatch: {}", e.toString());
-                if (logger.isDebugEnabled())
-                    logger.debug("Digest mismatch:", e);
+                if (logger.isTraceEnabled())
+                    logger.trace("Digest mismatch:", e);
                 
                 ReadRepairMetrics.repairedBackground.mark();
                 
                 ReadCommand readCommand = (ReadCommand) command;
-                final RowDataResolver repairResolver = new RowDataResolver(readCommand.ksName, readCommand.key, readCommand.filter(), readCommand.timestamp);
+                final RowDataResolver repairResolver = new RowDataResolver(readCommand.ksName, readCommand.key, readCommand.filter(), readCommand.timestamp, endpoints.size());
                 AsyncRepairCallback repairHandler = new AsyncRepairCallback(repairResolver, endpoints.size());
 
                 MessageOut<ReadCommand> message = ((ReadCommand) command).createMessage();
@@ -211,4 +225,15 @@
             }
         }
     }
+
+    @Override
+    public void onFailure(InetAddress from)
+    {
+        int n = waitingFor(from)
+              ? failuresUpdater.incrementAndGet(this)
+              : failures;
+
+        if (blockfor + n > endpoints.size())
+            condition.signalAll();
+    }
 }
diff --git a/src/java/org/apache/cassandra/service/RowDataResolver.java b/src/java/org/apache/cassandra/service/RowDataResolver.java
index 394a4c4..e15302b 100644
--- a/src/java/org/apache/cassandra/service/RowDataResolver.java
+++ b/src/java/org/apache/cassandra/service/RowDataResolver.java
@@ -42,9 +42,9 @@
     private final IDiskAtomFilter filter;
     private final long timestamp;
 
-    public RowDataResolver(String keyspaceName, ByteBuffer key, IDiskAtomFilter qFilter, long timestamp)
+    public RowDataResolver(String keyspaceName, ByteBuffer key, IDiskAtomFilter qFilter, long timestamp, int maxResponseCount)
     {
-        super(key, keyspaceName);
+        super(key, keyspaceName, maxResponseCount);
         this.filter = qFilter;
         this.timestamp = timestamp;
     }
@@ -59,8 +59,8 @@
     public Row resolve() throws DigestMismatchException
     {
         int replyCount = replies.size();
-        if (logger.isDebugEnabled())
-            logger.debug("resolving {} responses", replyCount);
+        if (logger.isTraceEnabled())
+            logger.trace("resolving {} responses", replyCount);
         long start = System.nanoTime();
 
         ColumnFamily resolved;
@@ -84,8 +84,8 @@
             }
 
             resolved = resolveSuperset(versions, timestamp);
-            if (logger.isDebugEnabled())
-                logger.debug("versions merged");
+            if (logger.isTraceEnabled())
+                logger.trace("versions merged");
 
             // send updates to any replica that was missing part of the full row
             // (resolved can be null even if versions doesn't have all nulls because of the call to removeDeleted in resolveSuperSet)
@@ -94,11 +94,11 @@
         }
         else
         {
-            resolved = replies.iterator().next().payload.row().cf;
+            resolved = replies.get(0).payload.row().cf;
         }
 
-        if (logger.isDebugEnabled())
-            logger.debug("resolve: {} ms.", TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start));
+        if (logger.isTraceEnabled())
+            logger.trace("resolve: {} ms.", TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start));
 
         return new Row(key, resolved);
     }
@@ -162,7 +162,7 @@
     public Row getData()
     {
         assert !replies.isEmpty();
-        return replies.peek().payload.row();
+        return replies.get(0).payload.row();
     }
 
     public boolean isDataPresent()
diff --git a/src/java/org/apache/cassandra/service/RowDigestResolver.java b/src/java/org/apache/cassandra/service/RowDigestResolver.java
index 7f2e17d..32b26e1 100644
--- a/src/java/org/apache/cassandra/service/RowDigestResolver.java
+++ b/src/java/org/apache/cassandra/service/RowDigestResolver.java
@@ -27,9 +27,9 @@
 
 public class RowDigestResolver extends AbstractRowResolver
 {
-    public RowDigestResolver(String keyspaceName, ByteBuffer key)
+    public RowDigestResolver(String keyspaceName, ByteBuffer key, int maxResponseCount)
     {
-        super(key, keyspaceName);
+        super(key, keyspaceName, maxResponseCount);
     }
 
     /**
@@ -63,8 +63,8 @@
      */
     public Row resolve() throws DigestMismatchException
     {
-        if (logger.isDebugEnabled())
-            logger.debug("resolving {} responses", replies.size());
+        if (logger.isTraceEnabled())
+            logger.trace("resolving {} responses", replies.size());
 
         long start = System.nanoTime();
 
@@ -98,8 +98,8 @@
                 throw new DigestMismatchException(key, digest, newDigest);
         }
 
-        if (logger.isDebugEnabled())
-            logger.debug("resolve: {} ms.", TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start));
+        if (logger.isTraceEnabled())
+            logger.trace("resolve: {} ms.", TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start));
         return new Row(key, data);
     }
 
diff --git a/src/java/org/apache/cassandra/service/StartupCheck.java b/src/java/org/apache/cassandra/service/StartupCheck.java
new file mode 100644
index 0000000..4d23098
--- /dev/null
+++ b/src/java/org/apache/cassandra/service/StartupCheck.java
@@ -0,0 +1,47 @@
+/*
+ * 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.service;
+
+import org.apache.cassandra.exceptions.StartupException;
+
+/**
+ * A test to determine if the system is in a valid state to start up.
+ * Some implementations may not actually halt startup, but provide
+ * information or advice on tuning and non-fatal environmental issues (e.g. like
+ * checking for and warning about suboptimal JVM settings).
+ * Other checks may indicate that they system is not in a correct state to be started.
+ * Examples include inability to load JNA when the cassandra.boot_without_jna option
+ * is not set, missing or unaccessible data directories, unreadable sstables and
+ * misconfiguration of cluster_name in cassandra.yaml.
+ *
+ * The StartupChecks class manages a collection of these tests, which it executes
+ * right at the beginning of the server settup process.
+ */
+public interface StartupCheck
+{
+    /**
+     * Run some test to determine whether the system is safe to be started
+     * In the case where a test determines it is not safe to proceed, the
+     * test should log a message regarding the reason for the failure and
+     * ideally the steps required to remedy the problem.
+     *
+     * @throws org.apache.cassandra.exceptions.StartupException if the test determines
+     * that the environement or system is not in a safe state to startup
+     */
+    void execute() throws StartupException;
+}
diff --git a/src/java/org/apache/cassandra/service/StartupChecks.java b/src/java/org/apache/cassandra/service/StartupChecks.java
new file mode 100644
index 0000000..34bc824
--- /dev/null
+++ b/src/java/org/apache/cassandra/service/StartupChecks.java
@@ -0,0 +1,358 @@
+/*
+ * 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.service;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.*;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.*;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.StartupException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.*;
+
+/**
+ * Verifies that the system and environment is in a fit state to be started.
+ * Used in CassandraDaemon#setup() to check various settings and invariants.
+ *
+ * Each individual test is modelled as an implementation of StartupCheck, these are run
+ * at the start of CassandraDaemon#setup() before any local state is mutated. The default
+ * checks are a mix of informational tests (inspectJvmOptions), initialization
+ * (initSigarLibrary, checkCacheServiceInitialization) and invariant checking
+ * (checkValidLaunchDate, checkSystemKeyspaceState, checkSSTablesFormat).
+ *
+ * In addition, if checkSystemKeyspaceState determines that the release version has
+ * changed since last startup (i.e. the node has been upgraded) it snapshots the system
+ * keyspace to make it easier to back out if necessary.
+ *
+ * If any check reports a failure, then the setup method exits with an error (after
+ * logging any output from the tests). If all tests report success, setup can continue.
+ * We should be careful in future to ensure anything which mutates local state (such as
+ * writing new sstables etc) only happens after we've verified the initial setup.
+ */
+public class StartupChecks
+{
+    private static final Logger logger = LoggerFactory.getLogger(StartupChecks.class);
+
+    // List of checks to run before starting up. If any test reports failure, startup will be halted.
+    private final List<StartupCheck> preFlightChecks = new ArrayList<>();
+
+    // The default set of pre-flight checks to run. Order is somewhat significant in that we probably
+    // always want the system keyspace check run last, as this actually loads the schema for that
+    // keyspace. All other checks should not require any schema initialization.
+    private final List<StartupCheck> DEFAULT_TESTS = ImmutableList.of(checkJemalloc,
+                                                                      checkValidLaunchDate,
+                                                                      checkJMXPorts,
+                                                                      inspectJvmOptions,
+                                                                      checkJnaInitialization,
+                                                                      initSigarLibrary,
+                                                                      checkDataDirs,
+                                                                      checkSSTablesFormat,
+                                                                      checkSystemKeyspaceState,
+                                                                      checkDatacenter,
+                                                                      checkRack);
+
+    public StartupChecks withDefaultTests()
+    {
+        preFlightChecks.addAll(DEFAULT_TESTS);
+        return this;
+    }
+
+    /**
+     * Add system test to be run before schema is loaded during startup
+     * @param test the system test to include
+     */
+    public StartupChecks withTest(StartupCheck test)
+    {
+        preFlightChecks.add(test);
+        return this;
+    }
+
+    /**
+     * Run the configured tests and return a report detailing the results.
+     * @throws org.apache.cassandra.exceptions.StartupException if any test determines that the
+     * system is not in an valid state to startup
+     */
+    public void verify() throws StartupException
+    {
+        for (StartupCheck test : preFlightChecks)
+            test.execute();
+    }
+
+    public static final StartupCheck checkJemalloc = new StartupCheck()
+    {
+        public void execute() throws StartupException
+        {
+            if (FBUtilities.isWindows())
+                return;
+            String jemalloc = System.getProperty("cassandra.libjemalloc");
+            if (jemalloc == null)
+                logger.warn("jemalloc shared library could not be preloaded to speed up memory allocations");
+            else if ("-".equals(jemalloc))
+                logger.info("jemalloc preload explicitly disabled");
+            else
+                logger.info("jemalloc seems to be preloaded from {}", jemalloc);
+        }
+    };
+
+    public static final StartupCheck checkValidLaunchDate = new StartupCheck()
+    {
+        /**
+         * The earliest legit timestamp a casandra instance could have ever launched.
+         * Date roughly taken from http://perspectives.mvdirona.com/2008/07/12/FacebookReleasesCassandraAsOpenSource.aspx
+         * We use this to ensure the system clock is at least somewhat correct at startup.
+         */
+        private static final long EARLIEST_LAUNCH_DATE = 1215820800000L;
+        public void execute() throws StartupException
+        {
+            long now = System.currentTimeMillis();
+            if (now < EARLIEST_LAUNCH_DATE)
+                throw new StartupException(1, String.format("current machine time is %s, but that is seemingly incorrect. exiting now.",
+                                                            new Date(now).toString()));
+        }
+    };
+
+    public static final StartupCheck checkJMXPorts = new StartupCheck()
+    {
+        public void execute()
+        {
+            String jmxPort = System.getProperty("com.sun.management.jmxremote.port");
+            if (jmxPort == null)
+            {
+                logger.warn("JMX is not enabled to receive remote connections. Please see cassandra-env.sh for more info.");
+                jmxPort = System.getProperty("cassandra.jmx.local.port");
+                if (jmxPort == null)
+                    logger.error("cassandra.jmx.local.port missing from cassandra-env.sh, unable to start local JMX service.");
+            }
+            else
+            {
+                logger.info("JMX is enabled to receive remote connections on port: " + jmxPort);
+            }
+        }
+    };
+
+    public static final StartupCheck inspectJvmOptions = new StartupCheck()
+    {
+        public void execute()
+        {
+            // log warnings for different kinds of sub-optimal JVMs.  tldr use 64-bit Oracle >= 1.6u32
+            if (!DatabaseDescriptor.hasLargeAddressSpace())
+                logger.warn("32bit JVM detected.  It is recommended to run Cassandra on a 64bit JVM for better performance.");
+
+            String javaVmName = System.getProperty("java.vm.name");
+            if (javaVmName.contains("OpenJDK"))
+            {
+                // There is essentially no QA done on OpenJDK builds, and
+                // clusters running OpenJDK have seen many heap and load issues.
+                logger.warn("OpenJDK is not recommended. Please upgrade to the newest Oracle Java release");
+            }
+            else if (!javaVmName.contains("HotSpot"))
+            {
+                logger.warn("Non-Oracle JVM detected.  Some features, such as immediate unmap of compacted SSTables, may not work as intended");
+            }
+        }
+    };
+
+    public static final StartupCheck checkJnaInitialization = new StartupCheck()
+    {
+        public void execute() throws StartupException
+        {
+            // Fail-fast if JNA is not available or failing to initialize properly
+            if (!CLibrary.jnaAvailable())
+                throw new StartupException(3, "JNA failing to initialize properly. ");
+        }
+    };
+
+    public static final StartupCheck initSigarLibrary = new StartupCheck()
+    {
+        public void execute()
+        {
+            SigarLibrary.instance.warnIfRunningInDegradedMode();
+        }
+    };
+
+    public static final StartupCheck checkDataDirs = new StartupCheck()
+    {
+        public void execute() throws StartupException
+        {
+            // check all directories(data, commitlog, saved cache) for existence and permission
+            Iterable<String> dirs = Iterables.concat(Arrays.asList(DatabaseDescriptor.getAllDataFileLocations()),
+                                                     Arrays.asList(DatabaseDescriptor.getCommitLogLocation(),
+                                                                   DatabaseDescriptor.getSavedCachesLocation()));
+            for (String dataDir : dirs)
+            {
+                logger.debug("Checking directory {}", dataDir);
+                File dir = new File(dataDir);
+
+                // check that directories exist.
+                if (!dir.exists())
+                {
+                    logger.warn("Directory {} doesn't exist", dataDir);
+                    // if they don't, failing their creation, stop cassandra.
+                    if (!dir.mkdirs())
+                        throw new StartupException(3, "Has no permission to create directory "+ dataDir);
+                }
+
+                // if directories exist verify their permissions
+                if (!Directories.verifyFullPermissions(dir, dataDir))
+                    throw new StartupException(3, "Insufficient permissions on directory " + dataDir);
+
+            }
+        }
+    };
+
+    public static final StartupCheck checkSSTablesFormat = new StartupCheck()
+    {
+        public void execute() throws StartupException
+        {
+            final Set<String> invalid = new HashSet<>();
+            final Set<String> nonSSTablePaths = new HashSet<>();
+            nonSSTablePaths.add(FileUtils.getCanonicalPath(DatabaseDescriptor.getCommitLogLocation()));
+            nonSSTablePaths.add(FileUtils.getCanonicalPath(DatabaseDescriptor.getSavedCachesLocation()));
+
+            FileVisitor<Path> sstableVisitor = new SimpleFileVisitor<Path>()
+            {
+                public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException
+                {
+                    if (!file.toString().endsWith(".db"))
+                        return FileVisitResult.CONTINUE;
+
+                    try
+                    {
+                        if (!Descriptor.fromFilename(file.toString()).isCompatible())
+                            invalid.add(file.toString());
+                    }
+                    catch (Exception e)
+                    {
+                        invalid.add(file.toString());
+                    }
+                    return FileVisitResult.CONTINUE;
+                }
+
+                public FileVisitResult preVisitDirectory(Path dir, BasicFileAttributes attrs) throws IOException
+                {
+                    String name = dir.getFileName().toString();
+                    return (name.equals("snapshots")
+                            || name.equals("backups")
+                            || nonSSTablePaths.contains(dir.toFile().getCanonicalPath()))
+                           ? FileVisitResult.SKIP_SUBTREE
+                           : FileVisitResult.CONTINUE;
+                }
+            };
+
+            for (String dataDir : DatabaseDescriptor.getAllDataFileLocations())
+            {
+                try
+                {
+                    Files.walkFileTree(Paths.get(dataDir), sstableVisitor);
+                }
+                catch (IOException e)
+                {
+                    throw new StartupException(3, "Unable to verify sstable files on disk", e);
+                }
+            }
+
+            if (!invalid.isEmpty())
+                throw new StartupException(3, String.format("Detected unreadable sstables %s, please check " +
+                                                            "NEWS.txt and ensure that you have upgraded through " +
+                                                            "all required intermediate versions, running " +
+                                                            "upgradesstables",
+                                                            Joiner.on(",").join(invalid)));
+
+        }
+    };
+
+    public static final StartupCheck checkSystemKeyspaceState = new StartupCheck()
+    {
+        public void execute() throws StartupException
+        {
+            // check the system keyspace to keep user from shooting self in foot by changing partitioner, cluster name, etc.
+            // we do a one-off scrub of the system keyspace first; we can't load the list of the rest of the keyspaces,
+            // until system keyspace is opened.
+
+            for (CFMetaData cfm : Schema.instance.getKeyspaceMetaData(SystemKeyspace.NAME).values())
+                ColumnFamilyStore.scrubDataDirectories(cfm);
+
+            try
+            {
+                SystemKeyspace.checkHealth();
+            }
+            catch (ConfigurationException e)
+            {
+                throw new StartupException(100, "Fatal exception during initialization", e);
+            }
+        }
+    };
+
+    public static final StartupCheck checkDatacenter = new StartupCheck()
+    {
+        public void execute() throws StartupException
+        {
+            if (!Boolean.getBoolean("cassandra.ignore_dc"))
+            {
+                String storedDc = SystemKeyspace.getDatacenter();
+                if (storedDc != null)
+                {
+                    String currentDc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddress());
+                    if (!storedDc.equals(currentDc))
+                    {
+                        String formatMessage = "Cannot start node if snitch's data center (%s) differs from previous data center (%s). " +
+                                               "Please fix the snitch configuration, decommission and rebootstrap this node or use the flag -Dcassandra.ignore_dc=true.";
+
+                        throw new StartupException(100, String.format(formatMessage, currentDc, storedDc));
+                    }
+                }
+            }
+        }
+    };
+
+    public static final StartupCheck checkRack = new StartupCheck()
+    {
+        public void execute() throws StartupException
+        {
+            if (!Boolean.getBoolean("cassandra.ignore_rack"))
+            {
+                String storedRack = SystemKeyspace.getRack();
+                if (storedRack != null)
+                {
+                    String currentRack = DatabaseDescriptor.getEndpointSnitch().getRack(FBUtilities.getBroadcastAddress());
+                    if (!storedRack.equals(currentRack))
+                    {
+                        String formatMessage = "Cannot start node if snitch's rack (%s) differs from previous rack (%s). " +
+                                               "Please fix the snitch configuration, decommission and rebootstrap this node or use the flag -Dcassandra.ignore_rack=true.";
+
+                        throw new StartupException(100, String.format(formatMessage, currentRack, storedRack));
+                    }
+                }
+            }
+        }
+    };
+}
diff --git a/src/java/org/apache/cassandra/service/StorageProxy.java b/src/java/org/apache/cassandra/service/StorageProxy.java
index cddc7e9..7b7979d 100644
--- a/src/java/org/apache/cassandra/service/StorageProxy.java
+++ b/src/java/org/apache/cassandra/service/StorageProxy.java
@@ -31,7 +31,6 @@
 import com.google.common.cache.CacheLoader;
 import com.google.common.collect.*;
 import com.google.common.util.concurrent.Uninterruptibles;
-import org.apache.cassandra.metrics.*;
 import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -42,7 +41,7 @@
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.filter.TombstoneOverwhelmingException;
 import org.apache.cassandra.db.index.SecondaryIndex;
 import org.apache.cassandra.db.index.SecondaryIndexSearcher;
 import org.apache.cassandra.db.marshal.UUIDType;
@@ -58,10 +57,10 @@
 import org.apache.cassandra.locator.IEndpointSnitch;
 import org.apache.cassandra.locator.LocalStrategy;
 import org.apache.cassandra.locator.TokenMetadata;
+import org.apache.cassandra.metrics.*;
 import org.apache.cassandra.net.*;
 import org.apache.cassandra.net.MessagingService.Verb;
 import org.apache.cassandra.service.paxos.*;
-import org.apache.cassandra.sink.SinkManager;
 import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.triggers.TriggerExecutor;
 import org.apache.cassandra.utils.*;
@@ -113,7 +112,7 @@
         {
             public void apply(IMutation mutation,
                               Iterable<InetAddress> targets,
-                              AbstractWriteResponseHandler responseHandler,
+                              AbstractWriteResponseHandler<IMutation> responseHandler,
                               String localDataCenter,
                               ConsistencyLevel consistency_level)
             throws OverloadedException
@@ -133,7 +132,7 @@
         {
             public void apply(IMutation mutation,
                               Iterable<InetAddress> targets,
-                              AbstractWriteResponseHandler responseHandler,
+                              AbstractWriteResponseHandler<IMutation> responseHandler,
                               String localDataCenter,
                               ConsistencyLevel consistencyLevel)
             {
@@ -145,7 +144,7 @@
         {
             public void apply(IMutation mutation,
                               Iterable<InetAddress> targets,
-                              AbstractWriteResponseHandler responseHandler,
+                              AbstractWriteResponseHandler<IMutation> responseHandler,
                               String localDataCenter,
                               ConsistencyLevel consistencyLevel)
             {
@@ -203,7 +202,7 @@
                                    ConsistencyLevel consistencyForPaxos,
                                    ConsistencyLevel consistencyForCommit,
                                    ClientState state)
-    throws UnavailableException, IsBootstrappingException, ReadTimeoutException, WriteTimeoutException, InvalidRequestException
+    throws UnavailableException, IsBootstrappingException, RequestFailureException, RequestTimeoutException, InvalidRequestException
     {
         final long start = System.nanoTime();
         int contentions = 0;
@@ -279,6 +278,11 @@
             casWriteMetrics.timeouts.mark();
             throw e;
         }
+        catch (WriteFailureException|ReadFailureException e)
+        {
+            casWriteMetrics.failures.mark();
+            throw e;
+        }
         catch(UnavailableException e)
         {
             casWriteMetrics.unavailables.mark();
@@ -351,7 +355,7 @@
                                                            ConsistencyLevel consistencyForCommit,
                                                            final boolean isWrite,
                                                            ClientState state)
-    throws WriteTimeoutException
+    throws WriteTimeoutException, WriteFailureException
     {
         long timeout = TimeUnit.MILLISECONDS.toNanos(DatabaseDescriptor.getCasContentionTimeout());
 
@@ -488,7 +492,7 @@
         List<InetAddress> naturalEndpoints = StorageService.instance.getNaturalEndpoints(keyspace.getName(), tk);
         Collection<InetAddress> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, keyspace.getName());
 
-        AbstractWriteResponseHandler responseHandler = null;
+        AbstractWriteResponseHandler<Commit> responseHandler = null;
         if (shouldBlock)
         {
             AbstractReplicationStrategy rs = keyspace.getReplicationStrategy();
@@ -557,13 +561,13 @@
      * @param consistency_level the consistency level for the operation
      */
     public static void mutate(Collection<? extends IMutation> mutations, ConsistencyLevel consistency_level)
-    throws UnavailableException, OverloadedException, WriteTimeoutException
+    throws UnavailableException, OverloadedException, WriteTimeoutException, WriteFailureException
     {
         Tracing.trace("Determining replicas for mutation");
         final String localDataCenter = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddress());
 
         long startTime = System.nanoTime();
-        List<AbstractWriteResponseHandler> responseHandlers = new ArrayList<>(mutations.size());
+        List<AbstractWriteResponseHandler<IMutation>> responseHandlers = new ArrayList<>(mutations.size());
 
         try
         {
@@ -581,54 +585,44 @@
             }
 
             // wait for writes.  throws TimeoutException if necessary
-            for (AbstractWriteResponseHandler responseHandler : responseHandlers)
+            for (AbstractWriteResponseHandler<IMutation> responseHandler : responseHandlers)
             {
                 responseHandler.get();
             }
         }
-        catch (WriteTimeoutException ex)
+        catch (WriteTimeoutException|WriteFailureException ex)
         {
             if (consistency_level == ConsistencyLevel.ANY)
             {
-                // hint all the mutations (except counters, which can't be safely retried).  This means
-                // we'll re-hint any successful ones; doesn't seem worth it to track individual success
-                // just for this unusual case.
-                for (IMutation mutation : mutations)
-                {
-                    if (mutation instanceof CounterMutation)
-                        continue;
-
-                    Token tk = StorageService.getPartitioner().getToken(mutation.key());
-                    List<InetAddress> naturalEndpoints = StorageService.instance.getNaturalEndpoints(mutation.getKeyspaceName(), tk);
-                    Collection<InetAddress> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, mutation.getKeyspaceName());
-                    for (InetAddress target : Iterables.concat(naturalEndpoints, pendingEndpoints))
-                    {
-                        // local writes can timeout, but cannot be dropped (see LocalMutationRunnable and
-                        // CASSANDRA-6510), so there is no need to hint or retry
-                        if (!target.equals(FBUtilities.getBroadcastAddress()) && shouldHint(target))
-                            submitHint((Mutation) mutation, target, null);
-                    }
-                }
-                Tracing.trace("Wrote hint to satisfy CL.ANY after no replicas acknowledged the write");
+                hintMutations(mutations);
             }
             else
             {
-                writeMetrics.timeouts.mark();
-                ClientRequestMetrics.writeTimeouts.inc();
-                Tracing.trace("Write timeout; received {} of {} required replies", ex.received, ex.blockFor);
+                if (ex instanceof WriteFailureException)
+                {
+                    writeMetrics.failures.mark();
+                    WriteFailureException fe = (WriteFailureException)ex;
+                    Tracing.trace("Write failure; received {} of {} required replies, failed {} requests",
+                                  fe.received, fe.blockFor, fe.failures);
+                }
+                else
+                {
+                    writeMetrics.timeouts.mark();
+                    WriteTimeoutException te = (WriteTimeoutException)ex;
+                    Tracing.trace("Write timeout; received {} of {} required replies", te.received, te.blockFor);
+                }
                 throw ex;
             }
         }
         catch (UnavailableException e)
         {
             writeMetrics.unavailables.mark();
-            ClientRequestMetrics.writeUnavailables.inc();
             Tracing.trace("Unavailable");
             throw e;
         }
         catch (OverloadedException e)
         {
-            ClientRequestMetrics.writeUnavailables.inc();
+            writeMetrics.unavailables.mark();
             Tracing.trace("Overloaded");
             throw e;
         }
@@ -638,11 +632,39 @@
         }
     }
 
+    /** hint all the mutations (except counters, which can't be safely retried).  This means
+      * we'll re-hint any successful ones; doesn't seem worth it to track individual success
+      * just for this unusual case.
+
+      * @param mutations the mutations that require hints
+      */
+    private static void hintMutations(Collection<? extends IMutation> mutations)
+    {
+        for (IMutation mutation : mutations)
+        {
+            if (mutation instanceof CounterMutation)
+                continue;
+
+            Token tk = StorageService.getPartitioner().getToken(mutation.key());
+            List<InetAddress> naturalEndpoints = StorageService.instance.getNaturalEndpoints(mutation.getKeyspaceName(), tk);
+            Collection<InetAddress> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, mutation.getKeyspaceName());
+            for (InetAddress target : Iterables.concat(naturalEndpoints, pendingEndpoints))
+            {
+                // local writes can timeout, but cannot be dropped (see LocalMutationRunnable and
+                // CASSANDRA-6510), so there is no need to hint or retry
+                if (!target.equals(FBUtilities.getBroadcastAddress()) && shouldHint(target))
+                    submitHint((Mutation) mutation, target, null);
+            }
+        }
+
+        Tracing.trace("Wrote hint to satisfy CL.ANY after no replicas acknowledged the write");
+    }
+
     @SuppressWarnings("unchecked")
     public static void mutateWithTriggers(Collection<? extends IMutation> mutations,
                                           ConsistencyLevel consistencyLevel,
                                           boolean mutateAtomically)
-    throws WriteTimeoutException, UnavailableException, OverloadedException, InvalidRequestException
+    throws WriteTimeoutException, WriteFailureException, UnavailableException, OverloadedException, InvalidRequestException
     {
         Collection<Mutation> augmented = TriggerExecutor.instance.execute(mutations);
 
@@ -697,17 +719,21 @@
         catch (UnavailableException e)
         {
             writeMetrics.unavailables.mark();
-            ClientRequestMetrics.writeUnavailables.inc();
             Tracing.trace("Unavailable");
             throw e;
         }
         catch (WriteTimeoutException e)
         {
             writeMetrics.timeouts.mark();
-            ClientRequestMetrics.writeTimeouts.inc();
             Tracing.trace("Write timeout; received {} of {} required replies", e.received, e.blockFor);
             throw e;
         }
+        catch (WriteFailureException e)
+        {
+            writeMetrics.failures.mark();
+            Tracing.trace("Write failure; received {} of {} required replies", e.received, e.blockFor);
+            throw e;
+        }
         finally
         {
             writeMetrics.addNano(System.nanoTime() - startTime);
@@ -715,12 +741,12 @@
     }
 
     private static void syncWriteToBatchlog(Collection<Mutation> mutations, Collection<InetAddress> endpoints, UUID uuid)
-    throws WriteTimeoutException
+    throws WriteTimeoutException, WriteFailureException
     {
-        AbstractWriteResponseHandler handler = new WriteResponseHandler(endpoints,
+        AbstractWriteResponseHandler<IMutation> handler = new WriteResponseHandler<>(endpoints,
                                                                         Collections.<InetAddress>emptyList(),
                                                                         ConsistencyLevel.ONE,
-                                                                        Keyspace.open(Keyspace.SYSTEM_KS),
+                                                                        Keyspace.open(SystemKeyspace.NAME),
                                                                         null,
                                                                         WriteType.BATCH_LOG);
 
@@ -752,14 +778,14 @@
 
     private static void asyncRemoveFromBatchlog(Collection<InetAddress> endpoints, UUID uuid)
     {
-        AbstractWriteResponseHandler handler = new WriteResponseHandler(endpoints,
+        AbstractWriteResponseHandler<IMutation> handler = new WriteResponseHandler<>(endpoints,
                                                                         Collections.<InetAddress>emptyList(),
                                                                         ConsistencyLevel.ANY,
-                                                                        Keyspace.open(Keyspace.SYSTEM_KS),
+                                                                        Keyspace.open(SystemKeyspace.NAME),
                                                                         null,
                                                                         WriteType.SIMPLE);
-        Mutation mutation = new Mutation(Keyspace.SYSTEM_KS, UUIDType.instance.decompose(uuid));
-        mutation.delete(SystemKeyspace.BATCHLOG_CF, FBUtilities.timestampMicros());
+        Mutation mutation = new Mutation(SystemKeyspace.NAME, UUIDType.instance.decompose(uuid));
+        mutation.delete(SystemKeyspace.BATCHLOG, FBUtilities.timestampMicros());
         MessageOut<Mutation> message = mutation.createMessage();
         for (InetAddress target : endpoints)
         {
@@ -797,7 +823,7 @@
      * @param callback an optional callback to be run if and when the write is
      * successful.
      */
-    public static AbstractWriteResponseHandler performWrite(IMutation mutation,
+    public static AbstractWriteResponseHandler<IMutation> performWrite(IMutation mutation,
                                                             ConsistencyLevel consistency_level,
                                                             String localDataCenter,
                                                             WritePerformer performer,
@@ -812,7 +838,7 @@
         List<InetAddress> naturalEndpoints = StorageService.instance.getNaturalEndpoints(keyspaceName, tk);
         Collection<InetAddress> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, keyspaceName);
 
-        AbstractWriteResponseHandler responseHandler = rs.getWriteResponseHandler(naturalEndpoints, pendingEndpoints, consistency_level, callback, writeType);
+        AbstractWriteResponseHandler<IMutation> responseHandler = rs.getWriteResponseHandler(naturalEndpoints, pendingEndpoints, consistency_level, callback, writeType);
 
         // exit early if we can't fulfill the CL at this time
         responseHandler.assureSufficientLiveNodes();
@@ -829,17 +855,17 @@
         Token tk = StorageService.getPartitioner().getToken(mutation.key());
         List<InetAddress> naturalEndpoints = StorageService.instance.getNaturalEndpoints(keyspaceName, tk);
         Collection<InetAddress> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, keyspaceName);
-        AbstractWriteResponseHandler responseHandler = rs.getWriteResponseHandler(naturalEndpoints, pendingEndpoints, consistency_level, null, writeType);
+        AbstractWriteResponseHandler<IMutation> responseHandler = rs.getWriteResponseHandler(naturalEndpoints, pendingEndpoints, consistency_level, null, writeType);
         return new WriteResponseHandlerWrapper(responseHandler, mutation);
     }
 
     // used by atomic_batch_mutate to decouple availability check from the write itself, caches consistency level and endpoints.
     private static class WriteResponseHandlerWrapper
     {
-        final AbstractWriteResponseHandler handler;
+        final AbstractWriteResponseHandler<IMutation> handler;
         final Mutation mutation;
 
-        WriteResponseHandlerWrapper(AbstractWriteResponseHandler handler, Mutation mutation)
+        WriteResponseHandlerWrapper(AbstractWriteResponseHandler<IMutation> handler, Mutation mutation)
         {
             this.handler = handler;
             this.mutation = mutation;
@@ -877,18 +903,21 @@
      * is not available.
      *
      * Note about hints:
-     *
+     * <pre>
+     * {@code
      * | Hinted Handoff | Consist. Level |
      * | on             |       >=1      | --> wait for hints. We DO NOT notify the handler with handler.response() for hints;
      * | on             |       ANY      | --> wait for hints. Responses count towards consistency.
      * | off            |       >=1      | --> DO NOT fire hints. And DO NOT wait for them to complete.
      * | off            |       ANY      | --> DO NOT fire hints. And DO NOT wait for them to complete.
+     * }
+     * </pre>
      *
      * @throws OverloadedException if the hints cannot be written/enqueued
      */
     public static void sendToHintedEndpoints(final Mutation mutation,
                                              Iterable<InetAddress> targets,
-                                             AbstractWriteResponseHandler responseHandler,
+                                             AbstractWriteResponseHandler<IMutation> responseHandler,
                                              String localDataCenter)
     throws OverloadedException
     {
@@ -964,10 +993,10 @@
         // The idea is that if we have over maxHintsInProgress hints in flight, this is probably due to
         // a small number of nodes causing problems, so we should avoid shutting down writes completely to
         // healthy nodes.  Any node with no hintsInProgress is considered healthy.
-        if (StorageMetrics.totalHintsInProgress.count() > maxHintsInProgress
+        if (StorageMetrics.totalHintsInProgress.getCount() > maxHintsInProgress
                 && (getHintsInProgressFor(destination).get() > 0 && shouldHint(destination)))
         {
-            throw new OverloadedException("Too many in flight hints: " + StorageMetrics.totalHintsInProgress.count() +
+            throw new OverloadedException("Too many in flight hints: " + StorageMetrics.totalHintsInProgress.getCount() +
                                           " destination: " + destination +
                                           " destination hints: " + getHintsInProgressFor(destination).get());
         }
@@ -987,7 +1016,7 @@
 
     public static Future<Void> submitHint(final Mutation mutation,
                                           final InetAddress target,
-                                          final AbstractWriteResponseHandler responseHandler)
+                                          final AbstractWriteResponseHandler<IMutation> responseHandler)
     {
         // local write that time out should be handled by LocalMutationRunnable
         assert !target.equals(FBUtilities.getBroadcastAddress()) : target;
@@ -999,7 +1028,7 @@
                 int ttl = HintedHandOffManager.calculateHintTTL(mutation);
                 if (ttl > 0)
                 {
-                    logger.debug("Adding hint for {}", target);
+                    logger.trace("Adding hint for {}", target);
                     writeHintForMutation(mutation, System.currentTimeMillis(), ttl, target);
                     // Notify the handler only for CL == ANY
                     if (responseHandler != null && responseHandler.consistencyLevel == ConsistencyLevel.ANY)
@@ -1038,14 +1067,15 @@
             logger.debug("Discarding hint for endpoint not part of ring: {}", target);
     }
 
-    private static void sendMessagesToNonlocalDC(MessageOut<? extends IMutation> message, Collection<InetAddress> targets, AbstractWriteResponseHandler handler)
+    private static void sendMessagesToNonlocalDC(MessageOut<? extends IMutation> message,
+                                                 Collection<InetAddress> targets,
+                                                 AbstractWriteResponseHandler<IMutation> handler)
     {
         Iterator<InetAddress> iter = targets.iterator();
         InetAddress target = iter.next();
 
         // Add the other destinations of the same message as a FORWARD_HEADER entry
-        DataOutputBuffer out = new DataOutputBuffer();
-        try
+        try (DataOutputBuffer out = new DataOutputBuffer())
         {
             out.writeInt(targets.size() - 1);
             while (iter.hasNext())
@@ -1073,19 +1103,23 @@
         }
     }
 
-    private static void insertLocal(final Mutation mutation, final AbstractWriteResponseHandler responseHandler)
+    private static void insertLocal(final Mutation mutation, final AbstractWriteResponseHandler<IMutation> responseHandler)
     {
 
         StageManager.getStage(Stage.MUTATION).maybeExecuteImmediately(new LocalMutationRunnable()
         {
             public void runMayThrow()
             {
-                IMutation processed = SinkManager.processWriteRequest(mutation);
-                if (processed != null)
+                try
                 {
-                    ((Mutation) processed).apply();
+                    mutation.apply();
                     responseHandler.response(null);
                 }
+                catch (Exception ex)
+                {
+                    logger.error("Failed to apply mutation locally : {}", ex.getMessage());
+                    responseHandler.onFailure(FBUtilities.getBroadcastAddress());
+                }
             }
 
             @Override
@@ -1110,7 +1144,7 @@
      * quicker response and because the WriteResponseHandlers don't make it easy to send back an error. We also always gather
      * the write latencies at the coordinator node to make gathering point similar to the case of standard writes.
      */
-    public static AbstractWriteResponseHandler mutateCounter(CounterMutation cm, String localDataCenter) throws UnavailableException, OverloadedException
+    public static AbstractWriteResponseHandler<IMutation> mutateCounter(CounterMutation cm, String localDataCenter) throws UnavailableException, OverloadedException
     {
         InetAddress endpoint = findSuitableEndpoint(cm.getKeyspaceName(), cm.key(), localDataCenter, cm.consistency());
 
@@ -1130,7 +1164,7 @@
             rs.getWriteResponseHandler(naturalEndpoints, pendingEndpoints, cm.consistency(), null, WriteType.COUNTER).assureSufficientLiveNodes();
 
             // Forward the actual update to the chosen leader replica
-            AbstractWriteResponseHandler responseHandler = new WriteResponseHandler(endpoint, WriteType.COUNTER);
+            AbstractWriteResponseHandler<IMutation> responseHandler = new WriteResponseHandler<>(endpoint, WriteType.COUNTER);
 
             Tracing.trace("Enqueuing counter update to {}", endpoint);
             MessagingService.instance().sendRR(cm.makeMutationMessage(), endpoint, responseHandler, false);
@@ -1177,7 +1211,7 @@
 
     // Must be called on a replica of the mutation. This replica becomes the
     // leader of this mutation.
-    public static AbstractWriteResponseHandler applyCounterMutationOnLeader(CounterMutation cm, String localDataCenter, Runnable callback)
+    public static AbstractWriteResponseHandler<IMutation> applyCounterMutationOnLeader(CounterMutation cm, String localDataCenter, Runnable callback)
     throws UnavailableException, OverloadedException
     {
         return performWrite(cm, cm.consistency(), localDataCenter, counterWritePerformer, callback, WriteType.COUNTER);
@@ -1185,7 +1219,7 @@
 
     // Same as applyCounterMutationOnLeader but must with the difference that it use the MUTATION stage to execute the write (while
     // applyCounterMutationOnLeader assumes it is on the MUTATION stage already)
-    public static AbstractWriteResponseHandler applyCounterMutationOnCoordinator(CounterMutation cm, String localDataCenter)
+    public static AbstractWriteResponseHandler<IMutation> applyCounterMutationOnCoordinator(CounterMutation cm, String localDataCenter)
     throws UnavailableException, OverloadedException
     {
         return performWrite(cm, cm.consistency(), localDataCenter, counterWriteOnCoordinatorPerformer, null, WriteType.COUNTER);
@@ -1193,7 +1227,7 @@
 
     private static Runnable counterWriteTask(final IMutation mutation,
                                              final Iterable<InetAddress> targets,
-                                             final AbstractWriteResponseHandler responseHandler,
+                                             final AbstractWriteResponseHandler<IMutation> responseHandler,
                                              final String localDataCenter)
     {
         return new DroppableRunnable(MessagingService.Verb.COUNTER_MUTATION)
@@ -1201,18 +1235,13 @@
             @Override
             public void runMayThrow() throws OverloadedException, WriteTimeoutException
             {
-                IMutation processed = SinkManager.processWriteRequest(mutation);
-                if (processed == null)
-                    return;
+                assert mutation instanceof CounterMutation;
 
-                assert processed instanceof CounterMutation;
-                CounterMutation cm = (CounterMutation) processed;
-
-                Mutation result = cm.apply();
+                Mutation result = ((CounterMutation) mutation).apply();
                 responseHandler.response(null);
 
                 Set<InetAddress> remotes = Sets.difference(ImmutableSet.copyOf(targets),
-                            ImmutableSet.of(FBUtilities.getBroadcastAddress()));
+                                                           ImmutableSet.of(FBUtilities.getBroadcastAddress()));
                 if (!remotes.isEmpty())
                     sendToHintedEndpoints(result, remotes, responseHandler, localDataCenter);
             }
@@ -1222,13 +1251,13 @@
     private static boolean systemKeyspaceQuery(List<ReadCommand> cmds)
     {
         for (ReadCommand cmd : cmds)
-            if (!cmd.ksName.equals(Keyspace.SYSTEM_KS))
+            if (!cmd.ksName.equals(SystemKeyspace.NAME))
                 return false;
         return true;
     }
 
     public static List<Row> read(List<ReadCommand> commands, ConsistencyLevel consistencyLevel)
-    throws UnavailableException, IsBootstrappingException, ReadTimeoutException, InvalidRequestException
+    throws UnavailableException, IsBootstrappingException, ReadFailureException, ReadTimeoutException, InvalidRequestException
     {
         // When using serial CL, the ClientState should be provided
         assert !consistencyLevel.isSerialConsistency();
@@ -1240,12 +1269,11 @@
      * a specific set of column names from a given column family.
      */
     public static List<Row> read(List<ReadCommand> commands, ConsistencyLevel consistencyLevel, ClientState state)
-    throws UnavailableException, IsBootstrappingException, ReadTimeoutException, InvalidRequestException
+    throws UnavailableException, IsBootstrappingException, ReadFailureException, ReadTimeoutException, InvalidRequestException
     {
         if (StorageService.instance.isBootstrapMode() && !systemKeyspaceQuery(commands))
         {
             readMetrics.unavailables.mark();
-            ClientRequestMetrics.readUnavailables.inc();
             throw new IsBootstrappingException();
         }
 
@@ -1255,7 +1283,7 @@
     }
 
     private static List<Row> readWithPaxos(List<ReadCommand> commands, ConsistencyLevel consistencyLevel, ClientState state)
-    throws InvalidRequestException, UnavailableException, ReadTimeoutException
+    throws InvalidRequestException, UnavailableException, ReadFailureException, ReadTimeoutException
     {
         assert state != null;
 
@@ -1288,23 +1316,31 @@
             {
                 throw new ReadTimeoutException(consistencyLevel, 0, consistencyLevel.blockFor(Keyspace.open(command.ksName)), false);
             }
+            catch (WriteFailureException e)
+            {
+                throw new ReadFailureException(consistencyLevel, e.received, e.failures, e.blockFor, false);
+            }
 
             rows = fetchRows(commands, consistencyForCommitOrFetch);
         }
         catch (UnavailableException e)
         {
             readMetrics.unavailables.mark();
-            ClientRequestMetrics.readUnavailables.inc();
             casReadMetrics.unavailables.mark();
             throw e;
         }
         catch (ReadTimeoutException e)
         {
             readMetrics.timeouts.mark();
-            ClientRequestMetrics.readTimeouts.inc();
             casReadMetrics.timeouts.mark();
             throw e;
         }
+        catch (ReadFailureException e)
+        {
+            readMetrics.failures.mark();
+            casReadMetrics.failures.mark();
+            throw e;
+        }
         finally
         {
             long latency = System.nanoTime() - start;
@@ -1319,7 +1355,7 @@
     }
 
     private static List<Row> readRegular(List<ReadCommand> commands, ConsistencyLevel consistencyLevel)
-    throws UnavailableException, ReadTimeoutException
+    throws UnavailableException, ReadFailureException, ReadTimeoutException
     {
         long start = System.nanoTime();
         List<Row> rows = null;
@@ -1331,13 +1367,16 @@
         catch (UnavailableException e)
         {
             readMetrics.unavailables.mark();
-            ClientRequestMetrics.readUnavailables.inc();
             throw e;
         }
         catch (ReadTimeoutException e)
         {
             readMetrics.timeouts.mark();
-            ClientRequestMetrics.readTimeouts.inc();
+            throw e;
+        }
+        catch (ReadFailureException e)
+        {
+            readMetrics.failures.mark();
             throw e;
         }
         finally
@@ -1364,7 +1403,7 @@
      * 5. else carry out read repair by getting data from all the nodes.
      */
     private static List<Row> fetchRows(List<ReadCommand> initialCommands, ConsistencyLevel consistencyLevel)
-    throws UnavailableException, ReadTimeoutException
+    throws UnavailableException, ReadFailureException, ReadTimeoutException
     {
         List<Row> rows = new ArrayList<>(initialCommands.size());
         // (avoid allocating a new list in the common case of nothing-to-retry)
@@ -1406,10 +1445,10 @@
                         rows.add(row);
                     }
 
-                    if (logger.isDebugEnabled())
-                        logger.debug("Read: {} ms.", TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - exec.handler.start));
+                    if (logger.isTraceEnabled())
+                        logger.trace("Read: {} ms.", TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - exec.handler.start));
                 }
-                catch (ReadTimeoutException ex)
+                catch (ReadTimeoutException|ReadFailureException ex)
                 {
                     int blockFor = consistencyLevel.blockFor(Keyspace.open(exec.command.getKeyspace()));
                     int responseCount = exec.handler.getReceivedCount();
@@ -1417,14 +1456,15 @@
                                    ? exec.resolver.isDataPresent() ? " (including data)" : " (only digests)"
                                    : "";
 
+                    boolean isTimeout = ex instanceof ReadTimeoutException;
                     if (Tracing.isTracing())
                     {
-                        Tracing.trace("Timed out; received {} of {} responses{}",
-                                      new Object[]{ responseCount, blockFor, gotData });
+                        Tracing.trace("{}; received {} of {} responses{}",
+                                      isTimeout ? "Timed out" : "Failed", responseCount, blockFor, gotData);
                     }
                     else if (logger.isDebugEnabled())
                     {
-                        logger.debug("Read timeout; received {} of {} responses{}", responseCount, blockFor, gotData);
+                        logger.debug("Read {}; received {} of {} responses{}", (isTimeout ? "timeout" : "failure"), responseCount, blockFor, gotData);
                     }
                     throw ex;
                 }
@@ -1435,7 +1475,7 @@
                     ReadRepairMetrics.repairedBlocking.mark();
 
                     // Do a full data read to resolve the correct response (and repair node that need be)
-                    RowDataResolver resolver = new RowDataResolver(exec.command.ksName, exec.command.key, exec.command.filter(), exec.command.timestamp);
+                    RowDataResolver resolver = new RowDataResolver(exec.command.ksName, exec.command.key, exec.command.filter(), exec.command.timestamp, exec.handler.endpoints.size());
                     ReadCallback<ReadResponse, Row> repairHandler = new ReadCallback<>(resolver,
                                                                                        ConsistencyLevel.ALL,
                                                                                        exec.getContactedReplicas().size(),
@@ -1455,7 +1495,7 @@
                     for (InetAddress endpoint : exec.getContactedReplicas())
                     {
                         Tracing.trace("Enqueuing full data read to {}", endpoint);
-                        MessagingService.instance().sendRR(message, endpoint, repairHandler);
+                        MessagingService.instance().sendRRWithFailure(message, endpoint, repairHandler);
                     }
                 }
             }
@@ -1484,7 +1524,7 @@
                         if (Tracing.isTracing())
                             Tracing.trace("Timed out waiting on digest mismatch repair requests");
                         else
-                            logger.debug("Timed out waiting on digest mismatch repair requests");
+                            logger.trace("Timed out waiting on digest mismatch repair requests");
                         // the caught exception here will have CL.ALL from the repair command,
                         // not whatever CL the initial command was at (CASSANDRA-7947)
                         int blockFor = consistencyLevel.blockFor(Keyspace.open(command.getKeyspace()));
@@ -1503,7 +1543,7 @@
                         if (Tracing.isTracing())
                             Tracing.trace("Timed out waiting on digest mismatch repair acknowledgements");
                         else
-                            logger.debug("Timed out waiting on digest mismatch repair acknowledgements");
+                            logger.trace("Timed out waiting on digest mismatch repair acknowledgements");
                         int blockFor = consistencyLevel.blockFor(Keyspace.open(command.getKeyspace()));
                         throw new ReadTimeoutException(consistencyLevel, blockFor-1, blockFor, true);
                     }
@@ -1546,11 +1586,22 @@
 
         protected void runMayThrow()
         {
-            Keyspace keyspace = Keyspace.open(command.ksName);
-            Row r = command.getRow(keyspace);
-            ReadResponse result = ReadVerbHandler.getResponse(command, r);
-            MessagingService.instance().addLatency(FBUtilities.getBroadcastAddress(), TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start));
-            handler.response(result);
+            try
+            {
+                Keyspace keyspace = Keyspace.open(command.ksName);
+                Row r = command.getRow(keyspace);
+                ReadResponse result = ReadVerbHandler.getResponse(command, r);
+                MessagingService.instance().addLatency(FBUtilities.getBroadcastAddress(), TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start));
+                handler.response(result);
+            }
+            catch (Throwable t)
+            {
+                handler.onFailure(FBUtilities.getBroadcastAddress());
+                if (t instanceof TombstoneOverwhelmingException)
+                    logger.error(t.getMessage());
+                else
+                    throw t;
+            }
         }
     }
 
@@ -1569,9 +1620,20 @@
 
         protected void runMayThrow()
         {
-            RangeSliceReply result = new RangeSliceReply(command.executeLocally());
-            MessagingService.instance().addLatency(FBUtilities.getBroadcastAddress(), TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start));
-            handler.response(result);
+            try
+            {
+                RangeSliceReply result = new RangeSliceReply(command.executeLocally());
+                MessagingService.instance().addLatency(FBUtilities.getBroadcastAddress(), TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start));
+                handler.response(result);
+            }
+            catch (Throwable t)
+            {
+                handler.onFailure(FBUtilities.getBroadcastAddress());
+                if (t instanceof TombstoneOverwhelmingException)
+                    logger.error(t.getMessage());
+                else
+                    throw t;
+            }
         }
     }
 
@@ -1655,7 +1717,7 @@
     }
 
     public static List<Row> getRangeSlice(AbstractRangeCommand command, ConsistencyLevel consistency_level)
-    throws UnavailableException, ReadTimeoutException
+    throws UnavailableException, ReadFailureException, ReadTimeoutException
     {
         Tracing.trace("Computing ranges to query");
         long startTime = System.nanoTime();
@@ -1690,7 +1752,7 @@
                              ranges.size(),
                              concurrencyFactor);
                 Tracing.trace("Submitting range requests on {} ranges with a concurrency of {}",
-                              new Object[]{ ranges.size(), concurrencyFactor});
+                              ranges.size(), concurrencyFactor);
             }
             else
             {
@@ -1702,13 +1764,16 @@
                 concurrencyFactor = resultRowsPerRange == 0.0
                                   ? 1
                                   : Math.max(1, Math.min(ranges.size(), (int) Math.ceil(command.limit() / resultRowsPerRange)));
-                logger.debug("Estimated result rows per range: {}; requested rows: {}, ranges.size(): {}; concurrent range requests: {}",
+
+                logger.trace("Estimated result rows per range: {}; requested rows: {}, ranges.size(): {}; concurrent range requests: {}",
                              resultRowsPerRange,
                              command.limit(),
                              ranges.size(),
                              concurrencyFactor);
                 Tracing.trace("Submitting range requests on {} ranges with a concurrency of {} ({} rows per range expected)",
-                              new Object[]{ ranges.size(), concurrencyFactor, resultRowsPerRange});
+                              ranges.size(),
+                              concurrencyFactor,
+                              resultRowsPerRange);
             }
 
             boolean haveSufficientRows = false;
@@ -1782,7 +1847,7 @@
                     if (filteredEndpoints.size() == 1
                         && filteredEndpoints.get(0).equals(FBUtilities.getBroadcastAddress()))
                     {
-                        StageManager.getStage(Stage.READ).execute(new LocalRangeSliceRunnable(nodeCmd, handler), Tracing.instance.get());
+                        StageManager.getStage(Stage.READ).execute(new LocalRangeSliceRunnable(nodeCmd, handler));
                     }
                     else
                     {
@@ -1790,7 +1855,7 @@
                         for (InetAddress endpoint : filteredEndpoints)
                         {
                             Tracing.trace("Enqueuing request to {}", endpoint);
-                            MessagingService.instance().sendRR(message, endpoint, handler);
+                            MessagingService.instance().sendRRWithFailure(message, endpoint, handler);
                         }
                     }
                     scanHandlers.add(Pair.create(nodeCmd, handler));
@@ -1813,24 +1878,25 @@
                         }
                         repairResponses.addAll(resolver.repairResults);
                     }
-                    catch (ReadTimeoutException ex)
+                    catch (ReadTimeoutException|ReadFailureException ex)
                     {
-                        // we timed out waiting for responses
+                        // we timed out or failed waiting for responses
                         int blockFor = consistency_level.blockFor(keyspace);
                         int responseCount = resolver.responses.size();
                         String gotData = responseCount > 0
                                          ? resolver.isDataPresent() ? " (including data)" : " (only digests)"
                                          : "";
 
+                        boolean isTimeout = ex instanceof ReadTimeoutException;
                         if (Tracing.isTracing())
                         {
-                            Tracing.trace("Timed out; received {} of {} responses{} for range {} of {}",
-                                          new Object[]{ responseCount, blockFor, gotData, i, ranges.size() });
+                            Tracing.trace("{}; received {} of {} responses{} for range {} of {}",
+                                          (isTimeout ? "Timed out" : "Failed"), responseCount, blockFor, gotData, i, ranges.size());
                         }
                         else if (logger.isDebugEnabled())
                         {
-                            logger.debug("Range slice timeout; received {} of {} responses{} for range {} of {}",
-                                         responseCount, blockFor, gotData, i, ranges.size());
+                            logger.debug("Range slice {}; received {} of {} responses{} for range {} of {}",
+                                         (isTimeout ? "timeout" : "failure"), responseCount, blockFor, gotData, i, ranges.size());
                         }
                         throw ex;
                     }
@@ -1884,11 +1950,26 @@
                         actualRowsPerRange = fetchedRows / i;
                         concurrencyFactor = Math.max(1, Math.min(ranges.size() - i, Math.round(remainingRows / actualRowsPerRange)));
                     }
-                    logger.debug("Didn't get enough response rows; actual rows per range: {}; remaining rows: {}, new concurrent requests: {}",
+                    logger.trace("Didn't get enough response rows; actual rows per range: {}; remaining rows: {}, new concurrent requests: {}",
                                  actualRowsPerRange, (int) remainingRows, concurrencyFactor);
                 }
             }
         }
+        catch (ReadTimeoutException e)
+        {
+            rangeMetrics.timeouts.mark();
+            throw e;
+        }
+        catch (UnavailableException e)
+        {
+            rangeMetrics.unavailables.mark();
+            throw e;
+        }
+        catch (ReadFailureException e)
+        {
+            rangeMetrics.failures.mark();
+            throw e;
+        }
         finally
         {
             long latency = System.nanoTime() - startTime;
@@ -1984,7 +2065,7 @@
     static <T extends RingPosition<T>> List<AbstractBounds<T>> getRestrictedRanges(final AbstractBounds<T> queryRange)
     {
         // special case for bounds containing exactly 1 (non-minimum) token
-        if (queryRange instanceof Bounds && queryRange.left.equals(queryRange.right) && !queryRange.left.isMinimum(StorageService.getPartitioner()))
+        if (queryRange instanceof Bounds && queryRange.left.equals(queryRange.right) && !queryRange.left.isMinimum())
         {
             return Collections.singletonList(queryRange);
         }
@@ -2024,81 +2105,6 @@
         return ranges;
     }
 
-    public long getReadOperations()
-    {
-        return readMetrics.latency.count();
-    }
-
-    public long getTotalReadLatencyMicros()
-    {
-        return readMetrics.totalLatency.count();
-    }
-
-    public double getRecentReadLatencyMicros()
-    {
-        return readMetrics.getRecentLatency();
-    }
-
-    public long[] getTotalReadLatencyHistogramMicros()
-    {
-        return readMetrics.totalLatencyHistogram.getBuckets(false);
-    }
-
-    public long[] getRecentReadLatencyHistogramMicros()
-    {
-        return readMetrics.recentLatencyHistogram.getBuckets(true);
-    }
-
-    public long getRangeOperations()
-    {
-        return rangeMetrics.latency.count();
-    }
-
-    public long getTotalRangeLatencyMicros()
-    {
-        return rangeMetrics.totalLatency.count();
-    }
-
-    public double getRecentRangeLatencyMicros()
-    {
-        return rangeMetrics.getRecentLatency();
-    }
-
-    public long[] getTotalRangeLatencyHistogramMicros()
-    {
-        return rangeMetrics.totalLatencyHistogram.getBuckets(false);
-    }
-
-    public long[] getRecentRangeLatencyHistogramMicros()
-    {
-        return rangeMetrics.recentLatencyHistogram.getBuckets(true);
-    }
-
-    public long getWriteOperations()
-    {
-        return writeMetrics.latency.count();
-    }
-
-    public long getTotalWriteLatencyMicros()
-    {
-        return writeMetrics.totalLatency.count();
-    }
-
-    public double getRecentWriteLatencyMicros()
-    {
-        return writeMetrics.getRecentLatency();
-    }
-
-    public long[] getTotalWriteLatencyHistogramMicros()
-    {
-        return writeMetrics.totalLatencyHistogram.getBuckets(false);
-    }
-
-    public long[] getRecentWriteLatencyHistogramMicros()
-    {
-        return writeMetrics.recentLatencyHistogram.getBuckets(true);
-    }
-
     public boolean getHintedHandoffEnabled()
     {
         return DatabaseDescriptor.hintedHandoffEnabled();
@@ -2131,29 +2137,30 @@
 
     public static boolean shouldHint(InetAddress ep)
     {
-        if (DatabaseDescriptor.shouldHintByDC())
+        if (DatabaseDescriptor.hintedHandoffEnabled())
         {
-            final String dc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(ep);
-            //Disable DC specific hints
-            if(!DatabaseDescriptor.hintedHandoffEnabled(dc))
+            if (DatabaseDescriptor.shouldHintByDC())
+            {
+                final String dc = DatabaseDescriptor.getEndpointSnitch().getDatacenter(ep);
+                // Disable DC specific hints
+                if (!DatabaseDescriptor.hintedHandoffEnabled(dc))
+                {
+                    return false;
+                }
+            }
+
+            boolean hintWindowExpired = Gossiper.instance.getEndpointDowntime(ep) > DatabaseDescriptor.getMaxHintWindow();
+            if (hintWindowExpired)
             {
                 HintedHandOffManager.instance.metrics.incrPastWindow(ep);
-                return false;
+                Tracing.trace("Not hinting {} which has been down {}ms", ep, Gossiper.instance.getEndpointDowntime(ep));
             }
+            return !hintWindowExpired;
         }
-        else if (!DatabaseDescriptor.hintedHandoffEnabled())
+        else
         {
-            HintedHandOffManager.instance.metrics.incrPastWindow(ep);
             return false;
         }
-
-        boolean hintWindowExpired = Gossiper.instance.getEndpointDowntime(ep) > DatabaseDescriptor.getMaxHintWindow();
-        if (hintWindowExpired)
-        {
-            HintedHandOffManager.instance.metrics.incrPastWindow(ep);
-            Tracing.trace("Not hinting {} which has been down {}ms", ep, Gossiper.instance.getEndpointDowntime(ep));
-        }
-        return !hintWindowExpired;
     }
 
     /**
@@ -2215,7 +2222,7 @@
     {
         public void apply(IMutation mutation,
                           Iterable<InetAddress> targets,
-                          AbstractWriteResponseHandler responseHandler,
+                          AbstractWriteResponseHandler<IMutation> responseHandler,
                           String localDataCenter,
                           ConsistencyLevel consistencyLevel) throws OverloadedException;
     }
@@ -2244,7 +2251,8 @@
             try
             {
                 runMayThrow();
-            } catch (Exception e)
+            }
+            catch (Exception e)
             {
                 throw new RuntimeException(e);
             }
@@ -2328,7 +2336,7 @@
 
     public long getTotalHints()
     {
-        return StorageMetrics.totalHints.count();
+        return StorageMetrics.totalHints.getCount();
     }
 
     public int getMaxHintsInProgress()
@@ -2343,7 +2351,7 @@
 
     public int getHintsInProgress()
     {
-        return (int) StorageMetrics.totalHintsInProgress.count();
+        return (int) StorageMetrics.totalHintsInProgress.getCount();
     }
 
     public void verifyNoHintsInProgress()
@@ -2382,14 +2390,14 @@
     public void reloadTriggerClasses() { TriggerExecutor.instance.reloadClasses(); }
 
     public long getReadRepairAttempted() {
-        return ReadRepairMetrics.attempted.count();
+        return ReadRepairMetrics.attempted.getCount();
     }
 
     public long getReadRepairRepairedBlocking() {
-        return ReadRepairMetrics.repairedBlocking.count();
+        return ReadRepairMetrics.repairedBlocking.getCount();
     }
 
     public long getReadRepairRepairedBackground() {
-        return ReadRepairMetrics.repairedBackground.count();
+        return ReadRepairMetrics.repairedBackground.getCount();
     }
 }
diff --git a/src/java/org/apache/cassandra/service/StorageProxyMBean.java b/src/java/org/apache/cassandra/service/StorageProxyMBean.java
index 08be118..e619892 100644
--- a/src/java/org/apache/cassandra/service/StorageProxyMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageProxyMBean.java
@@ -23,54 +23,6 @@
 
 public interface StorageProxyMBean
 {
-    /**
-     * @see org.apache.cassandra.metrics.LatencyMetrics#lastOpCount
-     */
-    @Deprecated
-    public long getReadOperations();
-    /**
-     * @see org.apache.cassandra.metrics.LatencyMetrics#totalLatencyHistogram
-     */
-    @Deprecated
-    public long getTotalReadLatencyMicros();
-    /**
-     * @see org.apache.cassandra.metrics.LatencyMetrics#recentLatencyHistogram
-     */
-    @Deprecated
-    public double getRecentReadLatencyMicros();
-    /**
-     * @see org.apache.cassandra.metrics.LatencyMetrics#totalLatencyHistogram
-     */
-    @Deprecated
-    public long[] getTotalReadLatencyHistogramMicros();
-    /**
-     * @see org.apache.cassandra.metrics.LatencyMetrics#recentLatencyHistogram
-     */
-    @Deprecated
-    public long[] getRecentReadLatencyHistogramMicros();
-
-    @Deprecated
-    public long getRangeOperations();
-    @Deprecated
-    public long getTotalRangeLatencyMicros();
-    @Deprecated
-    public double getRecentRangeLatencyMicros();
-    @Deprecated
-    public long[] getTotalRangeLatencyHistogramMicros();
-    @Deprecated
-    public long[] getRecentRangeLatencyHistogramMicros();
-
-    @Deprecated
-    public long getWriteOperations();
-    @Deprecated
-    public long getTotalWriteLatencyMicros();
-    @Deprecated
-    public double getRecentWriteLatencyMicros();
-    @Deprecated
-    public long[] getTotalWriteLatencyHistogramMicros();
-    @Deprecated
-    public long[] getRecentWriteLatencyHistogramMicros();
-
     public long getTotalHints();
     public boolean getHintedHandoffEnabled();
     public Set<String> getHintedHandoffEnabledByDC();
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index 27939f9..9d2d7bb 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -17,133 +17,24 @@
  */
 package org.apache.cassandra.service;
 
-import java.io.ByteArrayInputStream;
-import java.io.DataInputStream;
-import java.io.File;
-import java.io.IOException;
+import java.io.*;
 import java.lang.management.ManagementFactory;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.EnumMap;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
+import java.util.*;
 import java.util.Map.Entry;
-import java.util.Set;
-import java.util.SortedMap;
-import java.util.TreeMap;
-import java.util.UUID;
-import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Future;
-import java.util.concurrent.FutureTask;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
+import java.util.concurrent.*;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-
-import javax.management.JMX;
-import javax.management.MBeanServer;
-import javax.management.Notification;
-import javax.management.NotificationBroadcasterSupport;
-import javax.management.ObjectName;
+import javax.management.*;
 import javax.management.openmbean.TabularData;
 import javax.management.openmbean.TabularDataSupport;
 
-import org.apache.cassandra.auth.Auth;
-import org.apache.cassandra.concurrent.ScheduledExecutors;
-import org.apache.cassandra.concurrent.Stage;
-import org.apache.cassandra.concurrent.StageManager;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.KSMetaData;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.db.BatchlogManager;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.CounterMutationVerbHandler;
-import org.apache.cassandra.db.DecoratedKey;
-import org.apache.cassandra.db.DefinitionsUpdateVerbHandler;
-import org.apache.cassandra.db.HintedHandOffManager;
-import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.MigrationRequestVerbHandler;
-import org.apache.cassandra.db.MutationVerbHandler;
-import org.apache.cassandra.db.ReadRepairVerbHandler;
-import org.apache.cassandra.db.ReadVerbHandler;
-import org.apache.cassandra.db.SchemaCheckVerbHandler;
-import org.apache.cassandra.db.SizeEstimatesRecorder;
-import org.apache.cassandra.db.SnapshotDetailsTabularData;
-import org.apache.cassandra.db.SystemKeyspace;
-import org.apache.cassandra.db.TruncateVerbHandler;
-import org.apache.cassandra.db.commitlog.CommitLog;
-import org.apache.cassandra.db.compaction.CompactionManager;
-import org.apache.cassandra.db.index.SecondaryIndex;
-import org.apache.cassandra.dht.BootStrapper;
-import org.apache.cassandra.dht.IPartitioner;
-import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.dht.RangeStreamer;
-import org.apache.cassandra.dht.RingPosition;
-import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.UnavailableException;
-import org.apache.cassandra.gms.ApplicationState;
-import org.apache.cassandra.gms.EndpointState;
-import org.apache.cassandra.gms.FailureDetector;
-import org.apache.cassandra.gms.GossipDigestAck2VerbHandler;
-import org.apache.cassandra.gms.GossipDigestAckVerbHandler;
-import org.apache.cassandra.gms.GossipDigestSynVerbHandler;
-import org.apache.cassandra.gms.GossipShutdownVerbHandler;
-import org.apache.cassandra.gms.Gossiper;
-import org.apache.cassandra.gms.IEndpointStateChangeSubscriber;
-import org.apache.cassandra.gms.IFailureDetector;
-import org.apache.cassandra.gms.TokenSerializer;
-import org.apache.cassandra.gms.VersionedValue;
-import org.apache.cassandra.io.sstable.SSTableDeletingTask;
-import org.apache.cassandra.io.sstable.SSTableLoader;
-import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.locator.AbstractReplicationStrategy;
-import org.apache.cassandra.locator.DynamicEndpointSnitch;
-import org.apache.cassandra.locator.IEndpointSnitch;
-import org.apache.cassandra.locator.LocalStrategy;
-import org.apache.cassandra.locator.TokenMetadata;
-import org.apache.cassandra.metrics.StorageMetrics;
-import org.apache.cassandra.net.AsyncOneResponse;
-import org.apache.cassandra.net.MessageOut;
-import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.net.ResponseVerbHandler;
-import org.apache.cassandra.repair.RepairFuture;
-import org.apache.cassandra.repair.RepairMessageVerbHandler;
-import org.apache.cassandra.repair.RepairParallelism;
-import org.apache.cassandra.service.paxos.CommitVerbHandler;
-import org.apache.cassandra.service.paxos.PrepareVerbHandler;
-import org.apache.cassandra.service.paxos.ProposeVerbHandler;
-import org.apache.cassandra.streaming.ReplicationFinishedVerbHandler;
-import org.apache.cassandra.streaming.StreamManager;
-import org.apache.cassandra.streaming.StreamPlan;
-import org.apache.cassandra.streaming.StreamResultFuture;
-import org.apache.cassandra.streaming.StreamState;
-import org.apache.cassandra.thrift.EndpointDetails;
-import org.apache.cassandra.thrift.TokenRange;
-import org.apache.cassandra.thrift.cassandraConstants;
-import org.apache.cassandra.tracing.Tracing;
-import org.apache.cassandra.utils.BackgroundActivityMonitor;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.JVMStabilityInspector;
-import org.apache.cassandra.utils.OutputHandler;
-import org.apache.cassandra.utils.Pair;
-import org.apache.cassandra.utils.WrappedRunnable;
-
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Optional;
+import com.google.common.base.Predicate;
+import com.google.common.collect.*;
 import com.google.common.util.concurrent.*;
 import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
@@ -153,19 +44,40 @@
 import ch.qos.logback.classic.jmx.JMXConfiguratorMBean;
 import ch.qos.logback.classic.spi.ILoggingEvent;
 import ch.qos.logback.core.Appender;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Predicate;
-import com.google.common.collect.ArrayListMultimap;
-import com.google.common.collect.Collections2;
-import com.google.common.collect.HashMultimap;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Iterators;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Multimap;
-import com.google.common.collect.Sets;
+import org.apache.cassandra.auth.AuthKeyspace;
+import org.apache.cassandra.auth.AuthMigrationListener;
+import org.apache.cassandra.concurrent.ScheduledExecutors;
+import org.apache.cassandra.concurrent.Stage;
+import org.apache.cassandra.concurrent.StageManager;
+import org.apache.cassandra.config.*;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.commitlog.CommitLog;
+import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.dht.*;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.exceptions.*;
+import org.apache.cassandra.gms.*;
+import org.apache.cassandra.io.sstable.SSTableDeletingTask;
+import org.apache.cassandra.io.sstable.SSTableLoader;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.locator.*;
+import org.apache.cassandra.metrics.StorageMetrics;
+import org.apache.cassandra.net.*;
+import org.apache.cassandra.repair.*;
+import org.apache.cassandra.repair.messages.RepairOption;
+import org.apache.cassandra.service.paxos.CommitVerbHandler;
+import org.apache.cassandra.service.paxos.PrepareVerbHandler;
+import org.apache.cassandra.service.paxos.ProposeVerbHandler;
+import org.apache.cassandra.streaming.*;
+import org.apache.cassandra.thrift.EndpointDetails;
+import org.apache.cassandra.thrift.TokenRange;
+import org.apache.cassandra.thrift.cassandraConstants;
+import org.apache.cassandra.tracing.TraceKeyspace;
+import org.apache.cassandra.utils.*;
+import org.apache.cassandra.utils.progress.ProgressEvent;
+import org.apache.cassandra.utils.progress.ProgressEventType;
+import org.apache.cassandra.utils.progress.jmx.JMXProgressSupport;
+import org.apache.cassandra.utils.progress.jmx.LegacyJMXProgressSupport;
 
 /**
  * This abstraction contains the token/identifier of this node
@@ -179,8 +91,14 @@
 
     public static final int RING_DELAY = getRingDelay(); // delay after which we assume ring has stablized
 
-    /* JMX notification serial number counter */
-    private final AtomicLong notificationSerialNumber = new AtomicLong();
+    private final JMXProgressSupport progressSupport = new JMXProgressSupport(this);
+
+    /**
+     * @deprecated backward support to previous notification interface
+     * Will be removed on 4.0
+     */
+    @Deprecated
+    private final LegacyJMXProgressSupport legacyProgressSupport;
 
     private static int getRingDelay()
     {
@@ -200,7 +118,7 @@
     public volatile VersionedValue.VersionedValueFactory valueFactory = new VersionedValue.VersionedValueFactory(getPartitioner());
 
     private Thread drainOnShutdown = null;
-    private boolean inShutdownHook = false;
+    private volatile boolean inShutdownHook = false;
 
     public static final StorageService instance = new StorageService();
 
@@ -235,22 +153,20 @@
     private InetAddress removingNode;
 
     /* Are we starting this node in bootstrap mode? */
-    private boolean isBootstrapMode;
+    private volatile boolean isBootstrapMode;
 
     /* we bootstrap but do NOT join the ring unless told to do so */
     private boolean isSurveyMode = Boolean.parseBoolean(System.getProperty("cassandra.write_survey", "false"));
     /* true if node is rebuilding and receiving data */
     private final AtomicBoolean isRebuilding = new AtomicBoolean();
 
-    /* when intialized as a client, we shouldn't write to the system keyspace. */
-    private boolean isClientMode;
     private boolean initialized;
     private volatile boolean joined = false;
 
     /* the probability for tracing any particular request, 0 disables tracing and 1 enables for all */
-    private double tracingProbability = 0.0;
+    private double traceProbability = 0.0;
 
-    private static enum Mode { STARTING, NORMAL, CLIENT, JOINING, LEAVING, DECOMMISSIONED, MOVING, DRAINING, DRAINED }
+    private static enum Mode { STARTING, NORMAL, JOINING, LEAVING, DECOMMISSIONED, MOVING, DRAINING, DRAINED }
     private Mode operationMode = Mode.STARTING;
 
     /* Used for tracking drain progress */
@@ -266,20 +182,27 @@
 
     private Collection<Token> bootstrapTokens = null;
 
-    public void finishBootstrapping()
-    {
-        isBootstrapMode = false;
-    }
+    // true when keeping strict consistency while bootstrapping
+    private boolean useStrictConsistency = Boolean.parseBoolean(System.getProperty("cassandra.consistent.rangemovement", "true"));
+    private static final boolean allowSimultaneousMoves = Boolean.valueOf(System.getProperty("cassandra.consistent.simultaneousmoves.allow","false"));
+    private static final boolean joinRing = Boolean.parseBoolean(System.getProperty("cassandra.join_ring", "true"));
+    private boolean replacing;
+    private UUID replacingId;
+
+    private final StreamStateStore streamStateStore = new StreamStateStore();
+
+    private final AtomicBoolean doneAuthSetup = new AtomicBoolean(false);
 
     /** This method updates the local token on disk  */
     public void setTokens(Collection<Token> tokens)
     {
+        assert tokens != null && !tokens.isEmpty() : "Node needs at least one token.";
         if (logger.isDebugEnabled())
             logger.debug("Setting tokens to {}", tokens);
         SystemKeyspace.updateTokens(tokens);
-        tokenMetadata.updateNormalTokens(tokens, FBUtilities.getBroadcastAddress());
         Collection<Token> localTokens = getLocalTokens();
         setGossipTokens(localTokens);
+        tokenMetadata.updateNormalTokens(tokens, FBUtilities.getBroadcastAddress());
         setMode(Mode.NORMAL, false);
     }
 
@@ -293,6 +216,9 @@
 
     public StorageService()
     {
+        // use dedicated executor for sending JMX notifications
+        super(Executors.newSingleThreadExecutor());
+
         MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
         try
         {
@@ -305,6 +231,8 @@
             throw new RuntimeException(e);
         }
 
+        legacyProgressSupport = new LegacyJMXProgressSupport(this, jmxObjectName);
+
         /* register the verb handlers */
         MessagingService.instance().registerVerbHandlers(MessagingService.Verb.MUTATION, new MutationVerbHandler());
         MessagingService.instance().registerVerbHandlers(MessagingService.Verb.READ_REPAIR, new ReadRepairVerbHandler());
@@ -368,7 +296,17 @@
         if (!initialized)
         {
             logger.warn("Starting gossip by operator request");
-            setGossipTokens(getLocalTokens());
+            Collection<Token> tokens = SystemKeyspace.getSavedTokens();
+
+            boolean validTokens = tokens != null && !tokens.isEmpty();
+
+            // shouldn't be called before these are set if we intend to join the ring/are in the process of doing so
+            if (joined || joinRing)
+                assert validTokens : "Cannot start gossiping for a node intended to join without valid tokens";
+
+            if (validTokens)
+                setGossipTokens(tokens);
+
             Gossiper.instance.forceNewerGeneration();
             Gossiper.instance.start((int) (System.currentTimeMillis() / 1000));
             initialized = true;
@@ -498,33 +436,32 @@
         if (daemon == null)
             throw new IllegalStateException("No configured daemon");
         daemon.deactivate();
-        // completely shut down cassandra
-        System.exit(0);
     }
 
     public synchronized Collection<Token> prepareReplacementInfo() throws ConfigurationException
     {
         logger.info("Gathering node replacement information for {}", DatabaseDescriptor.getReplaceAddress());
         if (!MessagingService.instance().isListening())
-            MessagingService.instance().listen(FBUtilities.getLocalAddress());
+            MessagingService.instance().listen();
 
         // make magic happen
-        Gossiper.instance.doShadowRound();
+        Map<InetAddress, EndpointState> epStates = Gossiper.instance.doShadowRound();
 
-        UUID hostId = null;
         // now that we've gossiped at least once, we should be able to find the node we're replacing
-        if (Gossiper.instance.getEndpointStateForEndpoint(DatabaseDescriptor.getReplaceAddress())== null)
+        if (epStates.get(DatabaseDescriptor.getReplaceAddress())== null)
             throw new RuntimeException("Cannot replace_address " + DatabaseDescriptor.getReplaceAddress() + " because it doesn't exist in gossip");
-        hostId = Gossiper.instance.getHostId(DatabaseDescriptor.getReplaceAddress());
+        replacingId = Gossiper.instance.getHostId(DatabaseDescriptor.getReplaceAddress(), epStates);
         try
         {
-            VersionedValue tokensVersionedValue = Gossiper.instance.getEndpointStateForEndpoint(DatabaseDescriptor.getReplaceAddress()).getApplicationState(ApplicationState.TOKENS);
+            VersionedValue tokensVersionedValue = epStates.get(DatabaseDescriptor.getReplaceAddress()).getApplicationState(ApplicationState.TOKENS);
             if (tokensVersionedValue == null)
                 throw new RuntimeException("Could not find tokens for " + DatabaseDescriptor.getReplaceAddress() + " to replace");
             Collection<Token> tokens = TokenSerializer.deserialize(getPartitioner(), new DataInputStream(new ByteArrayInputStream(tokensVersionedValue.toBytes())));
 
-            SystemKeyspace.setLocalHostId(hostId); // use the replacee's host Id as our own so we receive hints, etc
-            Gossiper.instance.resetEndpointStateMap(); // clean up since we have what we need
+            if (isReplacingSameAddress())
+            {
+                SystemKeyspace.setLocalHostId(replacingId); // use the replacee's host Id as our own so we receive hints, etc
+            }
             return tokens;
         }
         catch (IOException e)
@@ -537,79 +474,44 @@
     {
         logger.debug("Starting shadow gossip round to check for endpoint collision");
         if (!MessagingService.instance().isListening())
-            MessagingService.instance().listen(FBUtilities.getLocalAddress());
-        Gossiper.instance.doShadowRound();
-        if (!Gossiper.instance.isSafeForBootstrap(FBUtilities.getBroadcastAddress()))
+            MessagingService.instance().listen();
+        Map<InetAddress, EndpointState> epStates = Gossiper.instance.doShadowRound();
+        if (!Gossiper.instance.isSafeForBootstrap(FBUtilities.getBroadcastAddress(), epStates))
         {
             throw new RuntimeException(String.format("A node with address %s already exists, cancelling join. " +
                                                      "Use cassandra.replace_address if you want to replace this node.",
                                                      FBUtilities.getBroadcastAddress()));
         }
-        if (RangeStreamer.useStrictConsistency && !allowSimultaneousMoves())
+        if (useStrictConsistency && !allowSimultaneousMoves())
         {
-            for (Map.Entry<InetAddress, EndpointState> entry : Gossiper.instance.getEndpointStates())
+            for (Map.Entry<InetAddress, EndpointState> entry : epStates.entrySet())
             {
-
+                // ignore local node or empty status
                 if (entry.getKey().equals(FBUtilities.getBroadcastAddress()) || entry.getValue().getApplicationState(ApplicationState.STATUS) == null)
-                        continue;
-                String[] pieces = entry.getValue().getApplicationState(ApplicationState.STATUS).value.split(VersionedValue.DELIMITER_STR, -1);
+                    continue;
+                String[] pieces = splitValue(entry.getValue().getApplicationState(ApplicationState.STATUS));
                 assert (pieces.length > 0);
                 String state = pieces[0];
                 if (state.equals(VersionedValue.STATUS_BOOTSTRAPPING) || state.equals(VersionedValue.STATUS_LEAVING) || state.equals(VersionedValue.STATUS_MOVING))
                     throw new UnsupportedOperationException("Other bootstrapping/leaving/moving nodes detected, cannot bootstrap while cassandra.consistent.rangemovement is true");
             }
         }
-        Gossiper.instance.resetEndpointStateMap();
     }
 
     private boolean allowSimultaneousMoves()
     {
-        return RangeStreamer.allowSimultaneousMoves && DatabaseDescriptor.getNumTokens() == 1;
+        return allowSimultaneousMoves && DatabaseDescriptor.getNumTokens() == 1;
     }
 
-    public synchronized void initClient() throws ConfigurationException
+    // for testing only
+    public void unsafeInitialize() throws ConfigurationException
     {
-        // We don't wait, because we're going to actually try to work on
-        initClient(0);
-
-        // sleep a while to allow gossip to warm up (the other nodes need to know about this one before they can reply).
-        outer:
-        while (true)
-        {
-            Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS);
-            for (InetAddress address : Gossiper.instance.getLiveMembers())
-            {
-                if (!Gossiper.instance.isFatClient(address))
-                    break outer;
-            }
-        }
-
-        // sleep until any schema migrations have finished
-        while (!MigrationManager.isReadyForBootstrap())
-        {
-            Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS);
-        }
-    }
-
-    public synchronized void initClient(int ringDelay) throws ConfigurationException
-    {
-        if (initialized)
-        {
-            if (!isClientMode)
-                throw new UnsupportedOperationException("StorageService does not support switching modes.");
-            return;
-        }
         initialized = true;
-        isClientMode = true;
-        logger.info("Starting up client gossip");
-        setMode(Mode.CLIENT, false);
         Gossiper.instance.register(this);
         Gossiper.instance.start((int) (System.currentTimeMillis() / 1000)); // needed for node-ring gathering.
         Gossiper.instance.addLocalApplicationState(ApplicationState.NET_VERSION, valueFactory.networkVersion());
-
         if (!MessagingService.instance().isListening())
-            MessagingService.instance().listen(FBUtilities.getLocalAddress());
-        Uninterruptibles.sleepUninterruptibly(ringDelay, TimeUnit.MILLISECONDS);
+            MessagingService.instance().listen();
     }
 
     public synchronized void initServer() throws ConfigurationException
@@ -621,16 +523,10 @@
     {
         logger.info("Cassandra version: {}", FBUtilities.getReleaseVersionString());
         logger.info("Thrift API version: {}", cassandraConstants.VERSION);
-        logger.info("CQL supported versions: {} (default: {})", StringUtils.join(ClientState.getCQLSupportedVersion(), ","), ClientState.DEFAULT_CQL_VERSION);
+        logger.info("CQL supported versions: {} (default: {})",
+                    StringUtils.join(ClientState.getCQLSupportedVersion(), ","), ClientState.DEFAULT_CQL_VERSION);
 
-        if (initialized)
-        {
-            if (isClientMode)
-                throw new UnsupportedOperationException("StorageService does not support switching modes.");
-            return;
-        }
         initialized = true;
-        isClientMode = false;
 
         try
         {
@@ -715,6 +611,9 @@
 
                 CommitLog.instance.shutdownBlocking();
 
+                if (FBUtilities.isWindows())
+                    WindowsTimer.endTimerPeriod(DatabaseDescriptor.getWindowsTimerInterval());
+
                 // wait for miscellaneous tasks like sstable and commitlog segment deletion
                 ScheduledExecutors.nonPeriodicTasks.shutdown();
                 if (!ScheduledExecutors.nonPeriodicTasks.awaitTermination(1, TimeUnit.MINUTES))
@@ -723,8 +622,17 @@
         }, "StorageServiceShutdownHook");
         Runtime.getRuntime().addShutdownHook(drainOnShutdown);
 
+        replacing = DatabaseDescriptor.isReplacing();
+
+        if (!Boolean.parseBoolean(System.getProperty("cassandra.start_gossip", "true")))
+        {
+            logger.info("Not starting gossip as requested.");
+            return;
+        }
+
         prepareToJoin();
 
+        // Has to be called after the host id has potentially changed in prepareToJoin().
         try
         {
             CacheService.instance.counterCache.loadSavedAsync().get();
@@ -735,7 +643,7 @@
             logger.warn("Error loading counter cache", t);
         }
 
-        if (Boolean.parseBoolean(System.getProperty("cassandra.join_ring", "true")))
+        if (joinRing)
         {
             joinTokenRing(delay);
         }
@@ -751,6 +659,7 @@
                 states.add(Pair.create(ApplicationState.STATUS, valueFactory.hibernate(true)));
                 Gossiper.instance.addLocalApplicationStates(states);
             }
+            doAuthSetup();
             logger.info("Not joining ring as requested. Use JMX (StorageService->joinRing()) to initiate ring joining");
         }
     }
@@ -762,6 +671,9 @@
     {
         if (drainOnShutdown != null)
             Runtime.getRuntime().removeShutdownHook(drainOnShutdown);
+
+        if (FBUtilities.isWindows())
+            WindowsTimer.endTimerPeriod(DatabaseDescriptor.getWindowsTimerInterval());
     }
 
     private boolean shouldBootstrap()
@@ -775,19 +687,26 @@
         {
             Map<ApplicationState, VersionedValue> appStates = new EnumMap<>(ApplicationState.class);
 
-            if (DatabaseDescriptor.isReplacing() && !(Boolean.parseBoolean(System.getProperty("cassandra.join_ring", "true"))))
+            if (replacing && !joinRing)
                 throw new ConfigurationException("Cannot set both join_ring=false and attempt to replace a node");
             if (DatabaseDescriptor.getReplaceTokens().size() > 0 || DatabaseDescriptor.getReplaceNode() != null)
                 throw new RuntimeException("Replace method removed; use cassandra.replace_address instead");
-            if (DatabaseDescriptor.isReplacing())
+            if (replacing)
             {
                 if (SystemKeyspace.bootstrapComplete())
                     throw new RuntimeException("Cannot replace address with a node that is already bootstrapped");
                 if (!DatabaseDescriptor.isAutoBootstrap())
                     throw new RuntimeException("Trying to replace_address with auto_bootstrap disabled will not work, check your configuration");
                 bootstrapTokens = prepareReplacementInfo();
-                appStates.put(ApplicationState.TOKENS, valueFactory.tokens(bootstrapTokens));
-                appStates.put(ApplicationState.STATUS, valueFactory.hibernate(true));
+                if (isReplacingSameAddress())
+                {
+                    logger.warn("Writes will not be forwarded to this node during replacement because it has the same address as " +
+                                "the node to be replaced ({}). If the previous node has been down for longer than max_hint_window_in_ms, " +
+                                "repair must be run after the replacement process in order to make this node consistent.",
+                                DatabaseDescriptor.getReplaceAddress());
+                    appStates.put(ApplicationState.TOKENS, valueFactory.tokens(bootstrapTokens));
+                    appStates.put(ApplicationState.STATUS, valueFactory.hibernate(true));
+                }
             }
             else if (shouldBootstrap())
             {
@@ -802,7 +721,7 @@
             getTokenMetadata().updateHostId(localHostId, FBUtilities.getBroadcastAddress());
             appStates.put(ApplicationState.NET_VERSION, valueFactory.networkVersion());
             appStates.put(ApplicationState.HOST_ID, valueFactory.hostId(localHostId));
-            appStates.put(ApplicationState.RPC_ADDRESS, valueFactory.rpcaddress(DatabaseDescriptor.getBroadcastRpcAddress()));
+            appStates.put(ApplicationState.RPC_ADDRESS, valueFactory.rpcaddress(FBUtilities.getBroadcastRpcAddress()));
             appStates.put(ApplicationState.RELEASE_VERSION, valueFactory.releaseVersion());
             logger.info("Starting up server gossip");
             Gossiper.instance.register(this);
@@ -813,7 +732,7 @@
             Schema.instance.updateVersionAndAnnounce(); // Ensure we know our own actual Schema UUID in preparation for updates
 
             if (!MessagingService.instance().isListening())
-                MessagingService.instance().listen(FBUtilities.getLocalAddress());
+                MessagingService.instance().listen();
             LoadBroadcaster.instance.startBroadcasting();
 
             HintedHandOffManager.instance.start();
@@ -835,13 +754,20 @@
         // We attempted to replace this with a schema-presence check, but you need a meaningful sleep
         // to get schema info from gossip which defeats the purpose.  See CASSANDRA-4427 for the gory details.
         Set<InetAddress> current = new HashSet<>();
-        logger.debug("Bootstrap variables: {} {} {} {}",
-                     DatabaseDescriptor.isAutoBootstrap(),
-                     SystemKeyspace.bootstrapInProgress(),
-                     SystemKeyspace.bootstrapComplete(),
-                     DatabaseDescriptor.getSeeds().contains(FBUtilities.getBroadcastAddress()));
+        if (logger.isDebugEnabled())
+        {
+            logger.debug("Bootstrap variables: {} {} {} {}",
+                         DatabaseDescriptor.isAutoBootstrap(),
+                         SystemKeyspace.bootstrapInProgress(),
+                         SystemKeyspace.bootstrapComplete(),
+                         DatabaseDescriptor.getSeeds().contains(FBUtilities.getBroadcastAddress()));
+        }
         if (DatabaseDescriptor.isAutoBootstrap() && !SystemKeyspace.bootstrapComplete() && DatabaseDescriptor.getSeeds().contains(FBUtilities.getBroadcastAddress()))
+        {
             logger.info("This node will not auto bootstrap because it is configured to be a seed node.");
+        }
+
+        boolean dataAvailable = true; // make this to false when bootstrap streaming failed
         if (shouldBootstrap())
         {
             if (SystemKeyspace.bootstrapInProgress())
@@ -872,19 +798,20 @@
             PendingRangeCalculatorService.instance.blockUntilFinished();
             setMode(Mode.JOINING, "calculation complete, ready to bootstrap", true);
 
+            logger.debug("... got ring + schema info");
 
-            if (logger.isDebugEnabled())
-                logger.debug("... got ring + schema info");
-
-            if (Boolean.parseBoolean(System.getProperty("cassandra.consistent.rangemovement", "true")) &&
+            if (useStrictConsistency && !allowSimultaneousMoves() &&
                     (
                         tokenMetadata.getBootstrapTokens().valueSet().size() > 0 ||
                         tokenMetadata.getLeavingEndpoints().size() > 0 ||
                         tokenMetadata.getMovingEndpoints().size() > 0
                     ))
+            {
                 throw new UnsupportedOperationException("Other bootstrapping/leaving/moving nodes detected, cannot bootstrap while cassandra.consistent.rangemovement is true");
+            }
 
-            if (!DatabaseDescriptor.isReplacing())
+            // get bootstrap tokens
+            if (!replacing)
             {
                 if (tokenMetadata.isMember(FBUtilities.getBroadcastAddress()))
                 {
@@ -896,7 +823,7 @@
             }
             else
             {
-                if (!DatabaseDescriptor.getReplaceAddress().equals(FBUtilities.getBroadcastAddress()))
+                if (!isReplacingSameAddress())
                 {
                     try
                     {
@@ -941,8 +868,7 @@
                 setMode(Mode.JOINING, "Replacing a node with token(s): " + bootstrapTokens, true);
             }
 
-            bootstrap(bootstrapTokens);
-            assert !isBootstrapMode; // bootstrap will block until finished
+            dataAvailable = bootstrap(bootstrapTokens);
         }
         else
         {
@@ -954,13 +880,13 @@
                 {
                     bootstrapTokens = BootStrapper.getRandomTokens(tokenMetadata, DatabaseDescriptor.getNumTokens());
                     if (DatabaseDescriptor.getNumTokens() == 1)
-                        logger.warn("Generated random token " + bootstrapTokens + ". Random tokens will result in an unbalanced ring; see http://wiki.apache.org/cassandra/Operations");
+                        logger.warn("Generated random token {}. Random tokens will result in an unbalanced ring; see http://wiki.apache.org/cassandra/Operations", bootstrapTokens);
                     else
                         logger.info("Generated random tokens. tokens are {}", bootstrapTokens);
                 }
                 else
                 {
-                    bootstrapTokens = new ArrayList<Token>(initialTokens.size());
+                    bootstrapTokens = new ArrayList<>(initialTokens.size());
                     for (String token : initialTokens)
                         bootstrapTokens.add(getPartitioner().getTokenFactory().fromString(token));
                     logger.info("Saved tokens not found. Using configuration value: {}", bootstrapTokens);
@@ -971,29 +897,31 @@
                 if (bootstrapTokens.size() != DatabaseDescriptor.getNumTokens())
                     throw new ConfigurationException("Cannot change the number of tokens from " + bootstrapTokens.size() + " to " + DatabaseDescriptor.getNumTokens());
                 else
-                    logger.info("Using saved tokens " + bootstrapTokens);
+                    logger.info("Using saved tokens {}", bootstrapTokens);
             }
         }
 
         // if we don't have system_traces keyspace at this point, then create it manually
-        if (Schema.instance.getKSMetaData(Tracing.TRACE_KS) == null)
-        {
-            KSMetaData tracingKeyspace = KSMetaData.traceKeyspace();
-            MigrationManager.announceNewKeyspace(tracingKeyspace, 0, false);
-        }
+        maybeAddOrUpdateKeyspace(TraceKeyspace.definition());
+        maybeAddOrUpdateKeyspace(SystemDistributedKeyspace.definition());
 
         if (!isSurveyMode)
         {
-            // start participating in the ring.
-            SystemKeyspace.setBootstrapState(SystemKeyspace.BootstrapState.COMPLETED);
-            setTokens(bootstrapTokens);
-            // remove the existing info about the replaced node.
-            if (!current.isEmpty())
-                for (InetAddress existing : current)
-                    Gossiper.instance.replacedEndpoint(existing);
-            assert tokenMetadata.sortedTokens().size() > 0;
+            if (dataAvailable)
+            {
+                finishJoiningRing(bootstrapTokens);
 
-            Auth.setup();
+                // remove the existing info about the replaced node.
+                if (!current.isEmpty())
+                {
+                    for (InetAddress existing : current)
+                        Gossiper.instance.replacedEndpoint(existing);
+                }
+            }
+            else
+            {
+                logger.warn("Some data streaming failed. Use nodetool to check bootstrap state and resume. For more, see `nodetool help bootstrap`. {}", SystemKeyspace.getBootstrapState());
+            }
         }
         else
         {
@@ -1001,6 +929,11 @@
         }
     }
 
+    public static boolean isReplacingSameAddress()
+    {
+        return DatabaseDescriptor.getReplaceAddress().equals(FBUtilities.getBroadcastAddress());
+    }
+
     public void gossipSnitchInfo()
     {
         IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
@@ -1026,19 +959,83 @@
         }
         else if (isSurveyMode)
         {
-            setTokens(SystemKeyspace.getSavedTokens());
-            SystemKeyspace.setBootstrapState(SystemKeyspace.BootstrapState.COMPLETED);
             isSurveyMode = false;
             logger.info("Leaving write survey mode and joining ring at operator request");
-            assert tokenMetadata.sortedTokens().size() > 0;
+            finishJoiningRing(SystemKeyspace.getSavedTokens());
+        }
+    }
 
-            Auth.setup();
+    private void finishJoiningRing(Collection<Token> tokens)
+    {
+        // start participating in the ring.
+        SystemKeyspace.setBootstrapState(SystemKeyspace.BootstrapState.COMPLETED);
+        setTokens(tokens);
+
+        assert tokenMetadata.sortedTokens().size() > 0;
+        doAuthSetup();
+    }
+
+    private void doAuthSetup()
+    {
+        if (!doneAuthSetup.getAndSet(true))
+        {
+            maybeAddOrUpdateKeyspace(AuthKeyspace.definition());
+
+            DatabaseDescriptor.getRoleManager().setup();
+            DatabaseDescriptor.getAuthenticator().setup();
+            DatabaseDescriptor.getAuthorizer().setup();
+            MigrationManager.instance.register(new AuthMigrationListener());
+        }
+    }
+
+    private void maybeAddKeyspace(KSMetaData ksm)
+    {
+        try
+        {
+            MigrationManager.announceNewKeyspace(ksm, 0, false);
+        }
+        catch (AlreadyExistsException e)
+        {
+            logger.debug("Attempted to create new keyspace {}, but it already exists", ksm.name);
+        }
+    }
+
+    /**
+     * Ensure the schema of a pseudo-system keyspace (a distributed system keyspace: traces, auth and the so-called distributedKeyspace),
+     * is up to date with what we expected (creating it if it doesn't exist and updating tables that may have been upgraded).
+     */
+    private void maybeAddOrUpdateKeyspace(KSMetaData expected)
+    {
+        // Note that want to deal with the keyspace and its table a bit differently: for the keyspace definition
+        // itself, we want to create it if it doesn't exist yet, but if it does exist, we don't want to modify it,
+        // because user can modify the definition to change the replication factor (#6016) and we don't want to
+        // override it. For the tables however, we have to deal with the fact that new version can add new columns
+        // (#8162 being an example), so even if the table definition exists, we still need to force the "current"
+        // version of the schema, the one the node will be expecting.
+
+        KSMetaData defined = Schema.instance.getKSMetaData(expected.name);
+        // If the keyspace doesn't exist, create it
+        if (defined == null)
+        {
+            maybeAddKeyspace(expected);
+            defined = Schema.instance.getKSMetaData(expected.name);
+        }
+
+        // While the keyspace exists, it might miss table or have outdated one
+        // There is also the potential for a race, as schema migrations add the bare
+        // keyspace into Schema.instance before adding its tables, so double check that
+        // all the expected tables are present
+        for (CFMetaData expectedTable : expected.cfMetaData().values())
+        {
+            CFMetaData definedTable = defined.cfMetaData().get(expectedTable.cfName);
+            if (definedTable == null || !definedTable.equals(expectedTable))
+                MigrationManager.forceAnnounceNewColumnFamily(expectedTable);
         }
     }
 
     public boolean isJoined()
     {
-        return tokenMetadata.isMember(FBUtilities.getBroadcastAddress());
+        return tokenMetadata.isMember(FBUtilities.getBroadcastAddress()) && !isSurveyMode;
     }
 
     public void rebuild(String sourceDc)
@@ -1053,7 +1050,13 @@
 
         try
         {
-            RangeStreamer streamer = new RangeStreamer(tokenMetadata, FBUtilities.getBroadcastAddress(), "Rebuild");
+            RangeStreamer streamer = new RangeStreamer(tokenMetadata,
+                                                       null,
+                                                       FBUtilities.getBroadcastAddress(),
+                                                       "Rebuild",
+                                                       !replacing && useStrictConsistency,
+                                                       DatabaseDescriptor.getEndpointSnitch(),
+                                                       streamStateStore);
             streamer.addSourceFilter(new RangeStreamer.FailureDetectorSourceFilter(FailureDetector.instance));
             if (sourceDc != null)
                 streamer.addSourceFilter(new RangeStreamer.SingleDatacenterFilter(DatabaseDescriptor.getEndpointSnitch(), sourceDc));
@@ -1113,6 +1116,7 @@
     public void setCompactionThroughputMbPerSec(int value)
     {
         DatabaseDescriptor.setCompactionThroughputMbPerSec(value);
+        CompactionManager.instance.setRate(value);
     }
 
     public boolean isIncrementalBackupsEnabled()
@@ -1140,16 +1144,28 @@
             logger.debug(logMsg);
     }
 
-    private void bootstrap(Collection<Token> tokens)
+    /**
+     * Bootstrap node by fetching data from other nodes.
+     * If node is bootstrapping as a new node, then this also announces bootstrapping to the cluster.
+     *
+     * This blocks until streaming is done.
+     *
+     * @param tokens bootstrapping tokens
+     * @return true if bootstrap succeeds.
+     */
+    private boolean bootstrap(final Collection<Token> tokens)
     {
         isBootstrapMode = true;
         SystemKeyspace.updateTokens(tokens); // DON'T use setToken, that makes us part of the ring locally which is incorrect until we are done bootstrapping
-        if (!DatabaseDescriptor.isReplacing())
+
+        if (!replacing || !isReplacingSameAddress())
         {
             // if not an existing token then bootstrap
-            List<Pair<ApplicationState, VersionedValue>> states = new ArrayList<Pair<ApplicationState, VersionedValue>>();
+            List<Pair<ApplicationState, VersionedValue>> states = new ArrayList<>();
             states.add(Pair.create(ApplicationState.TOKENS, valueFactory.tokens(tokens)));
-            states.add(Pair.create(ApplicationState.STATUS, valueFactory.bootstrapping(tokens)));
+            states.add(Pair.create(ApplicationState.STATUS, replacing?
+                                                            valueFactory.bootReplacing(DatabaseDescriptor.getReplaceAddress()) :
+                                                            valueFactory.bootstrapping(tokens)));
             Gossiper.instance.addLocalApplicationStates(states);
             setMode(Mode.JOINING, "sleeping " + RING_DELAY + " ms for pending range setup", true);
             Uninterruptibles.sleepUninterruptibly(RING_DELAY, TimeUnit.MILLISECONDS);
@@ -1160,11 +1176,97 @@
             tokenMetadata.updateNormalTokens(tokens, FBUtilities.getBroadcastAddress());
             SystemKeyspace.removeEndpoint(DatabaseDescriptor.getReplaceAddress());
         }
+
         if (!Gossiper.instance.seenAnySeed())
             throw new IllegalStateException("Unable to contact any seeds!");
+
+        if (Boolean.getBoolean("cassandra.reset_bootstrap_progress"))
+        {
+            logger.info("Resetting bootstrap progress to start fresh");
+            SystemKeyspace.resetAvailableRanges();
+        }
+
         setMode(Mode.JOINING, "Starting to bootstrap...", true);
-        new BootStrapper(FBUtilities.getBroadcastAddress(), tokens, tokenMetadata).bootstrap(); // handles token update
-        logger.info("Bootstrap completed! for the tokens {}", tokens);
+        BootStrapper bootstrapper = new BootStrapper(FBUtilities.getBroadcastAddress(), tokens, tokenMetadata);
+        bootstrapper.addProgressListener(progressSupport);
+        ListenableFuture<StreamState> bootstrapStream = bootstrapper.bootstrap(streamStateStore, !replacing && useStrictConsistency); // handles token update
+        Futures.addCallback(bootstrapStream, new FutureCallback<StreamState>()
+        {
+            @Override
+            public void onSuccess(StreamState streamState)
+            {
+                isBootstrapMode = false;
+                logger.info("Bootstrap completed! for the tokens {}", tokens);
+            }
+
+            @Override
+            public void onFailure(Throwable e)
+            {
+                logger.warn("Error during bootstrap.", e);
+            }
+        });
+        try
+        {
+            bootstrapStream.get();
+            return true;
+        }
+        catch (Throwable e)
+        {
+            logger.error("Error while waiting on bootstrap to complete. Bootstrap will have to be restarted.", e);
+            return false;
+        }
+    }
+
+    public boolean resumeBootstrap()
+    {
+        if (isBootstrapMode && SystemKeyspace.bootstrapInProgress())
+        {
+            logger.info("Resuming bootstrap...");
+
+            // get bootstrap tokens saved in system keyspace
+            final Collection<Token> tokens = SystemKeyspace.getSavedTokens();
+            // already bootstrapped ranges are filtered during bootstrap
+            BootStrapper bootstrapper = new BootStrapper(FBUtilities.getBroadcastAddress(), tokens, tokenMetadata);
+            bootstrapper.addProgressListener(progressSupport);
+            ListenableFuture<StreamState> bootstrapStream = bootstrapper.bootstrap(streamStateStore, !replacing && useStrictConsistency); // handles token update
+            Futures.addCallback(bootstrapStream, new FutureCallback<StreamState>()
+            {
+                @Override
+                public void onSuccess(StreamState streamState)
+                {
+                    isBootstrapMode = false;
+                    // start participating in the ring.
+                    // pretend we are in survey mode so we can use joinRing() here
+                    isSurveyMode = true;
+                    try
+                    {
+                        progressSupport.progress("bootstrap", ProgressEvent.createNotification("Joining ring..."));
+                        joinRing();
+                    }
+                    catch (IOException ignore)
+                    {
+                        // joinRing with survey mode does not throw IOException
+                    }
+                    progressSupport.progress("bootstrap", new ProgressEvent(ProgressEventType.COMPLETE, 1, 1, "Resume bootstrap complete"));
+                    logger.info("Resume complete");
+                }
+
+                @Override
+                public void onFailure(Throwable e)
+                {
+                    String message = "Error during bootstrap: " + e.getCause().getMessage();
+                    logger.error(message, e.getCause());
+                    progressSupport.progress("bootstrap", new ProgressEvent(ProgressEventType.ERROR, 1, 1, message));
+                    progressSupport.progress("bootstrap", new ProgressEvent(ProgressEventType.COMPLETE, 1, 1, "Resume bootstrap complete"));
+                }
+            });
+            return true;
+        }
+        else
+        {
+            logger.info("Resuming bootstrap is requested, but the node is already bootstrapped.");
+            return false;
+        }
     }
 
     public boolean isBootstrapMode()
@@ -1219,7 +1321,7 @@
     public String getRpcaddress(InetAddress endpoint)
     {
         if (endpoint.equals(FBUtilities.getBroadcastAddress()))
-            return DatabaseDescriptor.getBroadcastRpcAddress().getHostAddress();
+            return FBUtilities.getBroadcastRpcAddress().getHostAddress();
         else if (Gossiper.instance.getEndpointStateForEndpoint(endpoint).getApplicationState(ApplicationState.RPC_ADDRESS) == null)
             return endpoint.getHostAddress();
         else
@@ -1255,7 +1357,7 @@
             keyspace = Schema.instance.getNonSystemKeyspaces().get(0);
 
         Map<List<String>, List<String>> map = new HashMap<>();
-        for (Map.Entry<Range<Token>, Collection<InetAddress>> entry : tokenMetadata.getPendingRanges(keyspace).entrySet())
+        for (Map.Entry<Range<Token>, Collection<InetAddress>> entry : tokenMetadata.getPendingRangesMM(keyspace).asMap().entrySet())
         {
             List<InetAddress> l = new ArrayList<>(entry.getValue());
             map.put(entry.getKey().asList(), stringify(l));
@@ -1386,7 +1488,7 @@
 
         for (Map.Entry<Range<Token>, List<InetAddress>> entry : rangeToAddressMap.entrySet())
         {
-            Range range = entry.getKey();
+            Range<Token> range = entry.getKey();
             List<InetAddress> addresses = entry.getValue();
             List<String> endpoints = new ArrayList<>(addresses.size());
             List<String> rpc_endpoints = new ArrayList<>(addresses.size());
@@ -1463,7 +1565,7 @@
     */
     private Map<Range<Token>, List<InetAddress>> constructRangeToEndpointMap(String keyspace, List<Range<Token>> ranges)
     {
-        Map<Range<Token>, List<InetAddress>> rangeToEndpointMap = new HashMap<>();
+        Map<Range<Token>, List<InetAddress>> rangeToEndpointMap = new HashMap<>(ranges.size());
         for (Range<Token> range : ranges)
         {
             rangeToEndpointMap.put(range, Keyspace.open(keyspace).getReplicationStrategy().getNaturalEndpoints(range.right));
@@ -1510,16 +1612,18 @@
      */
     public void onChange(InetAddress endpoint, ApplicationState state, VersionedValue value)
     {
-        if (state.equals(ApplicationState.STATUS))
+        if (state == ApplicationState.STATUS)
         {
-            String apStateValue = value.value;
-            String[] pieces = apStateValue.split(VersionedValue.DELIMITER_STR, -1);
+            String[] pieces = splitValue(value);
             assert (pieces.length > 0);
 
             String moveName = pieces[0];
 
             switch (moveName)
             {
+                case VersionedValue.STATUS_BOOTSTRAPPING_REPLACE:
+                    handleStateBootreplacing(endpoint, pieces);
+                    break;
                 case VersionedValue.STATUS_BOOTSTRAPPING:
                     handleStateBootstrap(endpoint);
                     break;
@@ -1585,11 +1689,19 @@
                     case HOST_ID:
                         SystemKeyspace.updatePeerInfo(endpoint, "host_id", UUID.fromString(value.value));
                         break;
+                    case RPC_READY:
+                        notifyRpcChange(endpoint, epState.isRpcReady());
+                        break;
                 }
             }
         }
     }
 
+    private static String[] splitValue(VersionedValue value)
+    {
+        return value.value.split(VersionedValue.DELIMITER_STR, -1);
+    }
+
     public void updateTopology(InetAddress endpoint)
     {
         if (getTokenMetadata().isMember(endpoint))
@@ -1639,6 +1751,66 @@
         }
     }
 
+    private void notifyRpcChange(InetAddress endpoint, boolean ready)
+    {
+        if (ready)
+            notifyUp(endpoint);
+        else
+            notifyDown(endpoint);
+    }
+
+    private void notifyUp(InetAddress endpoint)
+    {
+        if (!isRpcReady(endpoint) || !Gossiper.instance.isAlive(endpoint))
+            return;
+
+        for (IEndpointLifecycleSubscriber subscriber : lifecycleSubscribers)
+            subscriber.onUp(endpoint);
+    }
+
+    private void notifyDown(InetAddress endpoint)
+    {
+        for (IEndpointLifecycleSubscriber subscriber : lifecycleSubscribers)
+            subscriber.onDown(endpoint);
+    }
+
+    private void notifyJoined(InetAddress endpoint)
+    {
+        if (!isStatus(endpoint, VersionedValue.STATUS_NORMAL))
+            return;
+
+        for (IEndpointLifecycleSubscriber subscriber : lifecycleSubscribers)
+            subscriber.onJoinCluster(endpoint);
+    }
+
+    private void notifyMoved(InetAddress endpoint)
+    {
+        for (IEndpointLifecycleSubscriber subscriber : lifecycleSubscribers)
+            subscriber.onMove(endpoint);
+    }
+
+    private void notifyLeft(InetAddress endpoint)
+    {
+        for (IEndpointLifecycleSubscriber subscriber : lifecycleSubscribers)
+            subscriber.onLeaveCluster(endpoint);
+    }
+
+    private boolean isStatus(InetAddress endpoint, String status)
+    {
+        return Gossiper.instance.getEndpointStateForEndpoint(endpoint).getStatus().equals(status);
+    }
+
+    public boolean isRpcReady(InetAddress endpoint)
+    {
+        return MessagingService.instance().getVersion(endpoint) < MessagingService.VERSION_22 ||
+                Gossiper.instance.getEndpointStateForEndpoint(endpoint).isRpcReady();
+    }
+
+    public void setRpcReady(boolean value)
+    {
+        Gossiper.instance.addLocalApplicationState(ApplicationState.RPC_READY, valueFactory.rpcReady(value));
+    }
+
     private Collection<Token> getTokensFor(InetAddress endpoint)
     {
         try
@@ -1691,8 +1863,44 @@
         tokenMetadata.addBootstrapTokens(tokens, endpoint);
         PendingRangeCalculatorService.instance.update();
 
-        if (Gossiper.instance.usesHostId(endpoint))
-            tokenMetadata.updateHostId(Gossiper.instance.getHostId(endpoint), endpoint);
+        tokenMetadata.updateHostId(Gossiper.instance.getHostId(endpoint), endpoint);
+    }
+
+
+    private void handleStateBootreplacing(InetAddress newNode, String[] pieces)
+    {
+        InetAddress oldNode;
+        try
+        {
+            oldNode = InetAddress.getByName(pieces[1]);
+        }
+        catch (Exception e)
+        {
+            logger.error("Node {} tried to replace malformed endpoint {}.", newNode, pieces[1], e);
+            return;
+        }
+
+        if (FailureDetector.instance.isAlive(oldNode))
+        {
+            throw new RuntimeException(String.format("Node %s is trying to replace alive node %s.", newNode, oldNode));
+        }
+
+        Optional<InetAddress> replacingNode = tokenMetadata.getReplacingNode(newNode);
+        if (replacingNode.isPresent() && !replacingNode.get().equals(oldNode))
+        {
+            throw new RuntimeException(String.format("Node %s is already replacing %s but is trying to replace %s.",
+                                                     newNode, replacingNode.get(), oldNode));
+        }
+
+        Collection<Token> tokens = getTokensFor(newNode);
+
+        if (logger.isDebugEnabled())
+            logger.debug("Node {} is replacing {}, tokens {}", newNode, oldNode, tokens);
+
+        tokenMetadata.addReplaceTokens(tokens, newNode, oldNode);
+        PendingRangeCalculatorService.instance.update();
+
+        tokenMetadata.updateHostId(Gossiper.instance.getHostId(newNode), newNode);
     }
 
     /**
@@ -1719,41 +1927,58 @@
                          endpoint,
                          Gossiper.instance.getEndpointStateForEndpoint(endpoint));
 
+        Optional<InetAddress> replacingNode = tokenMetadata.getReplacingNode(endpoint);
+        if (replacingNode.isPresent())
+        {
+            assert !endpoint.equals(replacingNode.get()) : "Pending replacement endpoint with same address is not supported";
+            logger.info("Node {} will complete replacement of {} for tokens {}", endpoint, replacingNode.get(), tokens);
+            if (FailureDetector.instance.isAlive(replacingNode.get()))
+            {
+                logger.error("Node {} cannot complete replacement of alive node {}.", endpoint, replacingNode.get());
+                return;
+            }
+            endpointsToRemove.add(replacingNode.get());
+        }
+
+        Optional<InetAddress> replacementNode = tokenMetadata.getReplacementNode(endpoint);
+        if (replacementNode.isPresent())
+        {
+            logger.warn("Node {} is currently being replaced by node {}.", endpoint, replacementNode.get());
+        }
+
         updatePeerInfo(endpoint);
         // Order Matters, TM.updateHostID() should be called before TM.updateNormalToken(), (see CASSANDRA-4300).
-        if (Gossiper.instance.usesHostId(endpoint))
+        UUID hostId = Gossiper.instance.getHostId(endpoint);
+        InetAddress existing = tokenMetadata.getEndpointForHostId(hostId);
+        if (replacing && isReplacingSameAddress() && Gossiper.instance.getEndpointStateForEndpoint(DatabaseDescriptor.getReplaceAddress()) != null
+            && (hostId.equals(Gossiper.instance.getHostId(DatabaseDescriptor.getReplaceAddress()))))
+            logger.warn("Not updating token metadata for {} because I am replacing it", endpoint);
+        else
         {
-            UUID hostId = Gossiper.instance.getHostId(endpoint);
-            InetAddress existing = tokenMetadata.getEndpointForHostId(hostId);
-            if (DatabaseDescriptor.isReplacing() && Gossiper.instance.getEndpointStateForEndpoint(DatabaseDescriptor.getReplaceAddress()) != null && (hostId.equals(Gossiper.instance.getHostId(DatabaseDescriptor.getReplaceAddress()))))
-                logger.warn("Not updating token metadata for {} because I am replacing it", endpoint);
-            else
+            if (existing != null && !existing.equals(endpoint))
             {
-                if (existing != null && !existing.equals(endpoint))
+                if (existing.equals(FBUtilities.getBroadcastAddress()))
                 {
-                    if (existing.equals(FBUtilities.getBroadcastAddress()))
-                    {
-                        logger.warn("Not updating host ID {} for {} because it's mine", hostId, endpoint);
-                        tokenMetadata.removeEndpoint(endpoint);
-                        endpointsToRemove.add(endpoint);
-                    }
-                    else if (Gossiper.instance.compareEndpointStartup(endpoint, existing) > 0)
-                    {
-                        logger.warn("Host ID collision for {} between {} and {}; {} is the new owner", hostId, existing, endpoint, endpoint);
-                        tokenMetadata.removeEndpoint(existing);
-                        endpointsToRemove.add(existing);
-                        tokenMetadata.updateHostId(hostId, endpoint);
-                    }
-                    else
-                    {
-                        logger.warn("Host ID collision for {} between {} and {}; ignored {}", hostId, existing, endpoint, endpoint);
-                        tokenMetadata.removeEndpoint(endpoint);
-                        endpointsToRemove.add(endpoint);
-                    }
+                    logger.warn("Not updating host ID {} for {} because it's mine", hostId, endpoint);
+                    tokenMetadata.removeEndpoint(endpoint);
+                    endpointsToRemove.add(endpoint);
+                }
+                else if (Gossiper.instance.compareEndpointStartup(endpoint, existing) > 0)
+                {
+                    logger.warn("Host ID collision for {} between {} and {}; {} is the new owner", hostId, existing, endpoint, endpoint);
+                    tokenMetadata.removeEndpoint(existing);
+                    endpointsToRemove.add(existing);
+                    tokenMetadata.updateHostId(hostId, endpoint);
                 }
                 else
-                    tokenMetadata.updateHostId(hostId, endpoint);
+                {
+                    logger.warn("Host ID collision for {} between {} and {}; ignored {}", hostId, existing, endpoint, endpoint);
+                    tokenMetadata.removeEndpoint(endpoint);
+                    endpointsToRemove.add(endpoint);
+                }
             }
+            else
+                tokenMetadata.updateHostId(hostId, endpoint);
         }
 
         for (final Token token : tokens)
@@ -1764,21 +1989,18 @@
             {
                 logger.debug("New node {} at token {}", endpoint, token);
                 tokensToUpdateInMetadata.add(token);
-                if (!isClientMode)
-                    tokensToUpdateInSystemKeyspace.add(token);
+                tokensToUpdateInSystemKeyspace.add(token);
             }
             else if (endpoint.equals(currentOwner))
             {
                 // set state back to normal, since the node may have tried to leave, but failed and is now back up
                 tokensToUpdateInMetadata.add(token);
-                if (!isClientMode)
-                    tokensToUpdateInSystemKeyspace.add(token);
+                tokensToUpdateInSystemKeyspace.add(token);
             }
             else if (Gossiper.instance.compareEndpointStartup(endpoint, currentOwner) > 0)
             {
                 tokensToUpdateInMetadata.add(token);
-                if (!isClientMode)
-                    tokensToUpdateInSystemKeyspace.add(token);
+                tokensToUpdateInSystemKeyspace.add(token);
 
                 // currentOwner is no longer current, endpoint is.  Keep track of these moves, because when
                 // a host no longer has any tokens, we'll want to remove it.
@@ -1803,34 +2025,27 @@
             }
         }
 
-        boolean isMoving = tokenMetadata.isMoving(endpoint); // capture because updateNormalTokens clears moving status
+        // capture because updateNormalTokens clears moving and member status
+        boolean isMember = tokenMetadata.isMember(endpoint);
+        boolean isMoving = tokenMetadata.isMoving(endpoint);
         tokenMetadata.updateNormalTokens(tokensToUpdateInMetadata, endpoint);
         for (InetAddress ep : endpointsToRemove)
         {
             removeEndpoint(ep);
-            if (DatabaseDescriptor.isReplacing() && DatabaseDescriptor.getReplaceAddress().equals(ep))
+            if (replacing && DatabaseDescriptor.getReplaceAddress().equals(ep))
                 Gossiper.instance.replacementQuarantine(ep); // quarantine locally longer than normally; see CASSANDRA-8260
         }
         if (!tokensToUpdateInSystemKeyspace.isEmpty())
-            SystemKeyspace.updateTokens(endpoint, tokensToUpdateInSystemKeyspace);;
+            SystemKeyspace.updateTokens(endpoint, tokensToUpdateInSystemKeyspace);
 
         if (isMoving || operationMode == Mode.MOVING)
         {
             tokenMetadata.removeFromMoving(endpoint);
-
-            if (!isClientMode)
-            {
-                for (IEndpointLifecycleSubscriber subscriber : lifecycleSubscribers)
-                    subscriber.onMove(endpoint);
-            }
+            notifyMoved(endpoint);
         }
-        else
+        else if (!isMember) // prior to this, the node was not a member
         {
-            if (!isClientMode)
-            {
-                for (IEndpointLifecycleSubscriber subscriber : lifecycleSubscribers)
-                    subscriber.onJoinCluster(endpoint);
-            }
+            notifyJoined(endpoint);
         }
 
         PendingRangeCalculatorService.instance.update();
@@ -1946,7 +2161,7 @@
                 PendingRangeCalculatorService.instance.update();
 
                 // find the endpoint coordinating this removal that we need to notify when we're done
-                String[] coordinator = Gossiper.instance.getEndpointStateForEndpoint(endpoint).getApplicationState(ApplicationState.REMOVAL_COORDINATOR).value.split(VersionedValue.DELIMITER_STR, -1);
+                String[] coordinator = splitValue(Gossiper.instance.getEndpointStateForEndpoint(endpoint).getApplicationState(ApplicationState.REMOVAL_COORDINATOR));
                 UUID hostId = UUID.fromString(coordinator[1]);
                 // grab any data we are now responsible for and notify responsible node
                 restoreReplicaCount(endpoint, tokenMetadata.getEndpointForHostId(hostId));
@@ -1969,11 +2184,7 @@
         if (!tokens.isEmpty())
             tokenMetadata.removeBootstrapTokens(tokens);
 
-        if (!isClientMode)
-        {
-            for (IEndpointLifecycleSubscriber subscriber : lifecycleSubscribers)
-                subscriber.onLeaveCluster(endpoint);
-        }
+        notifyLeft(endpoint);
         PendingRangeCalculatorService.instance.update();
     }
 
@@ -1987,8 +2198,7 @@
     private void removeEndpoint(InetAddress endpoint)
     {
         Gossiper.instance.removeEndpoint(endpoint);
-        if (!isClientMode)
-            SystemKeyspace.removeEndpoint(endpoint);
+        SystemKeyspace.removeEndpoint(endpoint);
     }
 
     protected void addExpireTimeIfFound(InetAddress endpoint, long expireTime)
@@ -2137,7 +2347,7 @@
         if (logger.isDebugEnabled())
             logger.debug("Node {} ranges [{}]", endpoint, StringUtils.join(ranges, ", "));
 
-        Map<Range<Token>, List<InetAddress>> currentReplicaEndpoints = new HashMap<>();
+        Map<Range<Token>, List<InetAddress>> currentReplicaEndpoints = new HashMap<>(ranges.size());
 
         // Find (for each range) all nodes that store replicas for these ranges as well
         TokenMetadata metadata = tokenMetadata.cloneOnlyTokenMap(); // don't do this in the loop! #7758
@@ -2186,14 +2396,10 @@
     {
         MigrationManager.instance.scheduleSchemaPull(endpoint, state);
 
-        if (isClientMode)
-            return;
-
         if (tokenMetadata.isMember(endpoint))
         {
             HintedHandOffManager.instance.scheduleHintDelivery(endpoint, true);
-            for (IEndpointLifecycleSubscriber subscriber : lifecycleSubscribers)
-                subscriber.onUp(endpoint);
+            notifyUp(endpoint);
         }
     }
 
@@ -2206,11 +2412,7 @@
     public void onDead(InetAddress endpoint, EndpointState state)
     {
         MessagingService.instance().convict(endpoint);
-        if (!isClientMode)
-        {
-            for (IEndpointLifecycleSubscriber subscriber : lifecycleSubscribers)
-                subscriber.onDown(endpoint);
-        }
+        notifyDown(endpoint);
     }
 
     public void onRestart(InetAddress endpoint, EndpointState state)
@@ -2220,24 +2422,10 @@
             onDead(endpoint, state);
     }
 
-    /** raw load value */
-    public double getLoad()
-    {
-        double bytes = 0;
-        for (String keyspaceName : Schema.instance.getKeyspaces())
-        {
-            Keyspace keyspace = Schema.instance.getKeyspaceInstance(keyspaceName);
-            if (keyspace == null)
-                continue;
-            for (ColumnFamilyStore cfs : keyspace.getColumnFamilyStores())
-                bytes += cfs.getLiveDiskSpaceUsed();
-        }
-        return bytes;
-    }
 
     public String getLoadString()
     {
-        return FileUtils.stringifyFileSize(getLoad());
+        return FileUtils.stringifyFileSize(StorageMetrics.load.getCount());
     }
 
     public Map<String, String> getLoadMap()
@@ -2390,7 +2578,7 @@
 
     public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        if (keyspaceName.equals(Keyspace.SYSTEM_KS))
+        if (keyspaceName.equals(SystemKeyspace.NAME))
             throw new RuntimeException("Cleanup of the system keyspace is neither necessary nor wise");
 
         CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
@@ -2416,7 +2604,7 @@
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
         CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
-        for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
+        for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, false, keyspaceName, columnFamilies))
         {
             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData, jobs);
             if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
@@ -2424,6 +2612,18 @@
         }
         return status.statusCode;
     }
+    public int verify(boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    {
+        CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
+        for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
+        {
+            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.verify(extendedVerify);
+            if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
+                status = oneStatus;
+        }
+        return status.statusCode;
+    }
+
     public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
         return upgradeSSTables(keyspaceName, excludeCurrentVersion, 2, columnFamilies);
@@ -2441,11 +2641,11 @@
         return status.statusCode;
     }
 
-    public void forceKeyspaceCompaction(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public void forceKeyspaceCompaction(boolean splitOutput, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
         for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, false, keyspaceName, columnFamilies))
         {
-            cfStore.forceMajorCompaction();
+            cfStore.forceMajorCompaction(splitOutput);
         }
     }
 
@@ -2457,7 +2657,7 @@
      */
     public void takeSnapshot(String tag, String... keyspaceNames) throws IOException
     {
-        if (operationMode.equals(Mode.JOINING))
+        if (operationMode == Mode.JOINING)
             throw new IOException("Cannot snapshot until bootstrap completes");
         if (tag == null || tag.equals(""))
             throw new IOException("You must supply a snapshot name.");
@@ -2496,13 +2696,13 @@
     {
         if (keyspaceName == null)
             throw new IOException("You must supply a keyspace name");
-        if (operationMode.equals(Mode.JOINING))
+        if (operationMode == Mode.JOINING)
             throw new IOException("Cannot snapshot until bootstrap completes");
 
         if (columnFamilyName == null)
-            throw new IOException("You must supply a column family name");
+            throw new IOException("You must supply a table name");
         if (columnFamilyName.contains("."))
-            throw new IllegalArgumentException("Cannot take a snapshot of a secondary index by itself. Run snapshot on the column family that owns the index.");
+            throw new IllegalArgumentException("Cannot take a snapshot of a secondary index by itself. Run snapshot on the table that owns the index.");
 
         if (tag == null || tag.equals(""))
             throw new IOException("You must supply a snapshot name.");
@@ -2541,6 +2741,7 @@
                     throw new IOException("You must supply a keyspace name");
                 if (operationMode.equals(Mode.JOINING))
                     throw new IOException("Cannot snapshot until bootstrap completes");
+
                 if (columnFamilyName == null)
                     throw new IOException("You must supply a column family name");
                 if (tag == null || tag.equals(""))
@@ -2620,7 +2821,7 @@
         Map<String, TabularData> snapshotMap = new HashMap<>();
         for (Keyspace keyspace : Keyspace.all())
         {
-            if (Keyspace.SYSTEM_KS.equals(keyspace.getName()))
+            if (SystemKeyspace.NAME.equals(keyspace.getName()))
                 continue;
 
             for (ColumnFamilyStore cfStore : keyspace.getColumnFamilyStores())
@@ -2646,7 +2847,7 @@
         long total = 0;
         for (Keyspace keyspace : Keyspace.all())
         {
-            if (Keyspace.SYSTEM_KS.equals(keyspace.getName()))
+            if (SystemKeyspace.NAME.equals(keyspace.getName()))
                 continue;
 
             for (ColumnFamilyStore cfStore : keyspace.getColumnFamilyStores())
@@ -2673,72 +2874,7 @@
     public Iterable<ColumnFamilyStore> getValidColumnFamilies(boolean allowIndexes, boolean autoAddIndexes, String keyspaceName, String... cfNames) throws IOException
     {
         Keyspace keyspace = getValidKeyspace(keyspaceName);
-        Set<ColumnFamilyStore> valid = new HashSet<>();
-
-        if (cfNames.length == 0)
-        {
-            // all stores are interesting
-            for (ColumnFamilyStore cfStore : keyspace.getColumnFamilyStores())
-            {
-                valid.add(cfStore);
-                if (autoAddIndexes)
-                {
-                    for (SecondaryIndex si : cfStore.indexManager.getIndexes())
-                    {
-                        if (si.getIndexCfs() != null) {
-                            logger.info("adding secondary index {} to operation", si.getIndexName());
-                            valid.add(si.getIndexCfs());
-                        }
-                    }
-
-                }
-            }
-            return valid;
-        }
-        // filter out interesting stores
-        for (String cfName : cfNames)
-        {
-            //if the CF name is an index, just flush the CF that owns the index
-            String baseCfName = cfName;
-            String idxName = null;
-            if (cfName.contains(".")) // secondary index
-            {
-                if(!allowIndexes)
-                {
-                   logger.warn("Operation not allowed on secondary Index column family ({})", cfName);
-                    continue;
-                }
-
-                String[] parts = cfName.split("\\.", 2);
-                baseCfName = parts[0];
-                idxName = parts[1];
-            }
-
-            ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore(baseCfName);
-            if (idxName != null)
-            {
-                Collection< SecondaryIndex > indexes = cfStore.indexManager.getIndexesByNames(new HashSet<>(Arrays.asList(cfName)));
-                if (indexes.isEmpty())
-                    logger.warn(String.format("Invalid column family index specified: %s/%s. Proceeding with others.", baseCfName, idxName));
-                else
-                    valid.add(Iterables.get(indexes, 0).getIndexCfs());
-            }
-            else
-            {
-                valid.add(cfStore);
-                if(autoAddIndexes)
-                {
-                    for(SecondaryIndex si : cfStore.indexManager.getIndexes())
-                    {
-                        if (si.getIndexCfs() != null) {
-                            logger.info("adding secondary index {} to operation", si.getIndexName());
-                            valid.add(si.getIndexCfs());
-                        }
-                    }
-                }
-            }
-        }
-        return valid;
+        return keyspace.getValidColumnFamilies(allowIndexes, autoAddIndexes, cfNames);
     }
 
     /**
@@ -2756,141 +2892,189 @@
         }
     }
 
-    /**
-     * Sends JMX notification to subscribers.
-     *
-     * @param type Message type
-     * @param message Message itself
-     * @param userObject Arbitrary object to attach to notification
-     */
-    public void sendNotification(String type, String message, Object userObject)
+    public int repairAsync(String keyspace, Map<String, String> repairSpec)
     {
-        Notification jmxNotification = new Notification(type, jmxObjectName, notificationSerialNumber.incrementAndGet(), message);
-        jmxNotification.setUserData(userObject);
-        sendNotification(jmxNotification);
+        RepairOption option = RepairOption.parse(repairSpec, getPartitioner());
+        // if ranges are not specified
+        if (option.getRanges().isEmpty())
+        {
+            if (option.isPrimaryRange())
+            {
+                // when repairing only primary range, neither dataCenters nor hosts can be set
+                if (option.getDataCenters().isEmpty() && option.getHosts().isEmpty())
+                    option.getRanges().addAll(getPrimaryRanges(keyspace));
+                    // except dataCenters only contain local DC (i.e. -local)
+                else if (option.isInLocalDCOnly())
+                    option.getRanges().addAll(getPrimaryRangesWithinDC(keyspace));
+                else
+                    throw new IllegalArgumentException("You need to run primary range repair on all nodes in the cluster.");
+            }
+            else
+            {
+                option.getRanges().addAll(getLocalRanges(keyspace));
+            }
+        }
+        return forceRepairAsync(keyspace, option, false);
     }
 
-    public int forceRepairAsync(String keyspace, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts, boolean primaryRange, boolean fullRepair, String... columnFamilies) throws IOException
+    @Deprecated
+    public int forceRepairAsync(String keyspace,
+                                boolean isSequential,
+                                Collection<String> dataCenters,
+                                Collection<String> hosts,
+                                boolean primaryRange,
+                                boolean fullRepair,
+                                String... columnFamilies)
     {
         return forceRepairAsync(keyspace, isSequential ? RepairParallelism.SEQUENTIAL.ordinal() : RepairParallelism.PARALLEL.ordinal(), dataCenters, hosts, primaryRange, fullRepair, columnFamilies);
     }
 
-    public int forceRepairAsync(String keyspace, int parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, boolean primaryRange, boolean fullRepair, String... columnFamilies)
+    @Deprecated
+    public int forceRepairAsync(String keyspace,
+                                int parallelismDegree,
+                                Collection<String> dataCenters,
+                                Collection<String> hosts,
+                                boolean primaryRange,
+                                boolean fullRepair,
+                                String... columnFamilies)
     {
         if (parallelismDegree < 0 || parallelismDegree > RepairParallelism.values().length - 1)
         {
             throw new IllegalArgumentException("Invalid parallelism degree specified: " + parallelismDegree);
         }
-        Collection<Range<Token>> ranges;
+        RepairParallelism parallelism = RepairParallelism.values()[parallelismDegree];
+        if (FBUtilities.isWindows() && parallelism != RepairParallelism.PARALLEL)
+        {
+            logger.warn("Snapshot-based repair is not yet supported on Windows.  Reverting to parallel repair.");
+            parallelism = RepairParallelism.PARALLEL;
+        }
+
+        RepairOption options = new RepairOption(parallelism, primaryRange, !fullRepair, false, 1, Collections.<Range<Token>>emptyList(), false);
+        if (dataCenters != null)
+        {
+            options.getDataCenters().addAll(dataCenters);
+        }
+        if (hosts != null)
+        {
+            options.getHosts().addAll(hosts);
+        }
         if (primaryRange)
         {
             // when repairing only primary range, neither dataCenters nor hosts can be set
-            if (dataCenters == null && hosts == null)
-                ranges = getPrimaryRanges(keyspace);
-            // except dataCenters only contain local DC (i.e. -local)
-            else if (dataCenters != null && dataCenters.size() == 1 && dataCenters.contains(DatabaseDescriptor.getLocalDataCenter()))
-                ranges = getPrimaryRangesWithinDC(keyspace);
+            if (options.getDataCenters().isEmpty() && options.getHosts().isEmpty())
+                options.getRanges().addAll(getPrimaryRanges(keyspace));
+                // except dataCenters only contain local DC (i.e. -local)
+            else if (options.getDataCenters().size() == 1 && options.getDataCenters().contains(DatabaseDescriptor.getLocalDataCenter()))
+                options.getRanges().addAll(getPrimaryRangesWithinDC(keyspace));
             else
                 throw new IllegalArgumentException("You need to run primary range repair on all nodes in the cluster.");
         }
         else
         {
-             ranges = getLocalRanges(keyspace);
+            options.getRanges().addAll(getLocalRanges(keyspace));
         }
-
-        return forceRepairAsync(keyspace, RepairParallelism.values()[parallelismDegree], dataCenters, hosts, ranges, fullRepair, columnFamilies);
-    }
-
-    public int forceRepairAsync(String keyspace, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts, Collection<Range<Token>> ranges, boolean fullRepair, String... columnFamilies)
-    {
-        return forceRepairAsync(keyspace, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, dataCenters, hosts, ranges, fullRepair, columnFamilies);
-    }
-
-    public int forceRepairAsync(String keyspace, RepairParallelism parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, Collection<Range<Token>> ranges, boolean fullRepair, String... columnFamilies)
-    {
-        if (ranges.isEmpty() || Keyspace.open(keyspace).getReplicationStrategy().getReplicationFactor() < 2)
-            return 0;
-
-        int cmd = nextRepairCommand.incrementAndGet();
-        if (ranges.size() > 0)
+        if (columnFamilies != null)
         {
-            if (FBUtilities.isWindows() && parallelismDegree != RepairParallelism.PARALLEL)
+            for (String columnFamily : columnFamilies)
             {
-                logger.warn("Snapshot-based repair is not yet supported on Windows.  Reverting to parallel repair.");
-                parallelismDegree = RepairParallelism.PARALLEL;
+                options.getColumnFamilies().add(columnFamily);
             }
-            new Thread(createRepairTask(cmd, keyspace, ranges, parallelismDegree, dataCenters, hosts, fullRepair, columnFamilies)).start();
         }
-        return cmd;
+        return forceRepairAsync(keyspace, options, true);
     }
 
-    public int forceRepairAsync(String keyspace, boolean isSequential, boolean isLocal, boolean primaryRange, boolean fullRepair, String... columnFamilies)
+    @Deprecated
+    public int forceRepairAsync(String keyspace,
+                                boolean isSequential,
+                                boolean isLocal,
+                                boolean primaryRange,
+                                boolean fullRepair,
+                                String... columnFamilies)
     {
-        Collection<Range<Token>> ranges;
-        if (primaryRange)
+        Set<String> dataCenters = null;
+        if (isLocal)
         {
-            ranges = isLocal ? getPrimaryRangesWithinDC(keyspace) : getPrimaryRanges(keyspace);
+            dataCenters = Sets.newHashSet(DatabaseDescriptor.getLocalDataCenter());
         }
-        else
-        {
-            ranges = getLocalRanges(keyspace);
-        }
-
-        return forceRepairAsync(keyspace, isSequential, isLocal, ranges, fullRepair, columnFamilies);
+        return forceRepairAsync(keyspace, isSequential, dataCenters, null, primaryRange, fullRepair, columnFamilies);
     }
 
-    public int forceRepairAsync(String keyspace, boolean isSequential, boolean isLocal, Collection<Range<Token>> ranges, boolean fullRepair, String... columnFamilies)
-    {
-        return forceRepairAsync(keyspace, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, isLocal, ranges, fullRepair, columnFamilies);
-    }
-
-    public int forceRepairAsync(String keyspace, RepairParallelism parallelismDegree, boolean isLocal, Collection<Range<Token>> ranges, boolean fullRepair, String... columnFamilies)
-    {
-        if (ranges.isEmpty() || Keyspace.open(keyspace).getReplicationStrategy().getReplicationFactor() < 2)
-            return 0;
-
-        int cmd = nextRepairCommand.incrementAndGet();
-        if (FBUtilities.isWindows() && parallelismDegree != RepairParallelism.PARALLEL)
-        {
-            logger.warn("Snapshot-based repair is not yet supported on Windows.  Reverting to parallel repair.");
-            parallelismDegree = RepairParallelism.PARALLEL;
-        }
-        new Thread(createRepairTask(cmd, keyspace, ranges, parallelismDegree, isLocal, fullRepair, columnFamilies)).start();
-        return cmd;
-    }
-
-    public int forceRepairRangeAsync(String beginToken, String endToken, String keyspaceName, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts, boolean fullRepair, String... columnFamilies) throws IOException
+    @Deprecated
+    public int forceRepairRangeAsync(String beginToken,
+                                     String endToken,
+                                     String keyspaceName,
+                                     boolean isSequential,
+                                     Collection<String> dataCenters,
+                                     Collection<String> hosts,
+                                     boolean fullRepair,
+                                     String... columnFamilies)
     {
         return forceRepairRangeAsync(beginToken, endToken, keyspaceName,
                                      isSequential ? RepairParallelism.SEQUENTIAL.ordinal() : RepairParallelism.PARALLEL.ordinal(),
                                      dataCenters, hosts, fullRepair, columnFamilies);
     }
 
-    public int forceRepairRangeAsync(String beginToken, String endToken, String keyspaceName, int parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, boolean fullRepair, String... columnFamilies)
+    @Deprecated
+    public int forceRepairRangeAsync(String beginToken,
+                                     String endToken,
+                                     String keyspaceName,
+                                     int parallelismDegree,
+                                     Collection<String> dataCenters,
+                                     Collection<String> hosts,
+                                     boolean fullRepair,
+                                     String... columnFamilies)
     {
         if (parallelismDegree < 0 || parallelismDegree > RepairParallelism.values().length - 1)
         {
             throw new IllegalArgumentException("Invalid parallelism degree specified: " + parallelismDegree);
         }
+        RepairParallelism parallelism = RepairParallelism.values()[parallelismDegree];
+        if (FBUtilities.isWindows() && parallelism != RepairParallelism.PARALLEL)
+        {
+            logger.warn("Snapshot-based repair is not yet supported on Windows.  Reverting to parallel repair.");
+            parallelism = RepairParallelism.PARALLEL;
+        }
+
+        if (!fullRepair)
+            logger.warn("Incremental repair can't be requested with subrange repair " +
+                        "because each subrange repair would generate an anti-compacted table. " +
+                        "The repair will occur but without anti-compaction.");
         Collection<Range<Token>> repairingRange = createRepairRangeFrom(beginToken, endToken);
 
-        logger.info("starting user-requested repair of range {} for keyspace {} and column families {}",
-                           repairingRange, keyspaceName, columnFamilies);
+        RepairOption options = new RepairOption(parallelism, false, !fullRepair, false, 1, repairingRange, true);
+        options.getDataCenters().addAll(dataCenters);
+        if (hosts != null)
+        {
+            options.getHosts().addAll(hosts);
+        }
+        if (columnFamilies != null)
+        {
+            for (String columnFamily : columnFamilies)
+            {
+                options.getColumnFamilies().add(columnFamily);
+            }
+        }
 
-        RepairParallelism parallelism = RepairParallelism.values()[parallelismDegree];
-        return forceRepairAsync(keyspaceName, parallelism, dataCenters, hosts, repairingRange, fullRepair, columnFamilies);
+        logger.info("starting user-requested repair of range {} for keyspace {} and column families {}",
+                    repairingRange, keyspaceName, columnFamilies);
+        return forceRepairAsync(keyspaceName, options, true);
     }
 
-    public int forceRepairRangeAsync(String beginToken, String endToken, String keyspaceName, boolean isSequential, boolean isLocal, boolean fullRepair, String... columnFamilies)
+    @Deprecated
+    public int forceRepairRangeAsync(String beginToken,
+                                     String endToken,
+                                     String keyspaceName,
+                                     boolean isSequential,
+                                     boolean isLocal,
+                                     boolean fullRepair,
+                                     String... columnFamilies)
     {
-        if (!fullRepair)
-            throw new IllegalArgumentException("Incremental repair can't be requested with subrange repair because " +
-                                               "each subrange repair would generate an anti-compacted table");
-        Collection<Range<Token>> repairingRange = createRepairRangeFrom(beginToken, endToken);
-
-        logger.info("starting user-requested repair of range {} for keyspace {} and column families {}",
-                           repairingRange, keyspaceName, columnFamilies);
-        return forceRepairAsync(keyspaceName, isSequential, isLocal, repairingRange, fullRepair, columnFamilies);
+        Set<String> dataCenters = null;
+        if (isLocal)
+        {
+            dataCenters = Sets.newHashSet(DatabaseDescriptor.getLocalDataCenter());
+        }
+        return forceRepairRangeAsync(beginToken, endToken, keyspaceName, isSequential, dataCenters, null, fullRepair, columnFamilies);
     }
 
     /**
@@ -2900,7 +3084,6 @@
      * @param endToken end token of the range
      * @return collection of ranges that match ring layout in TokenMetadata
      */
-    @SuppressWarnings("unchecked")
     @VisibleForTesting
     Collection<Range<Token>> createRepairRangeFrom(String beginToken, String endToken)
     {
@@ -2932,168 +3115,28 @@
         return repairingRange;
     }
 
-    private FutureTask<Object> createRepairTask(int cmd,
-                                                String keyspace,
-                                                Collection<Range<Token>> ranges,
-                                                RepairParallelism parallelismDegree,
-                                                boolean isLocal,
-                                                boolean fullRepair,
-                                                String... columnFamilies)
+    public int forceRepairAsync(String keyspace, RepairOption options, boolean legacy)
     {
-        Set<String> dataCenters = null;
-        if (isLocal)
-        {
-            dataCenters = Sets.newHashSet(DatabaseDescriptor.getLocalDataCenter());
-        }
-        return createRepairTask(cmd, keyspace, ranges, parallelismDegree, dataCenters, null, fullRepair, columnFamilies);
+        if (options.getRanges().isEmpty() || Keyspace.open(keyspace).getReplicationStrategy().getReplicationFactor() < 2)
+            return 0;
+
+        int cmd = nextRepairCommand.incrementAndGet();
+        new Thread(createRepairTask(cmd, keyspace, options, legacy)).start();
+        return cmd;
     }
 
-    private FutureTask<Object> createRepairTask(final int cmd,
-                                                final String keyspace,
-                                                final Collection<Range<Token>> ranges,
-                                                final RepairParallelism parallelismDegree,
-                                                final Collection<String> dataCenters,
-                                                final Collection<String> hosts,
-                                                final boolean fullRepair,
-                                                final String... columnFamilies)
+    private FutureTask<Object> createRepairTask(final int cmd, final String keyspace, final RepairOption options, boolean legacy)
     {
-        if (dataCenters != null && !dataCenters.contains(DatabaseDescriptor.getLocalDataCenter()))
+        if (!options.getDataCenters().isEmpty() && !options.getDataCenters().contains(DatabaseDescriptor.getLocalDataCenter()))
         {
             throw new IllegalArgumentException("the local data center must be part of the repair");
         }
 
-        return new FutureTask<>(new WrappedRunnable()
-        {
-            protected void runMayThrow() throws Exception
-            {
-                String message = String.format("Starting repair command #%d, repairing %d ranges for keyspace %s (parallelism=%s, full=%b)", cmd, ranges.size(), keyspace, parallelismDegree, fullRepair);
-                logger.info(message);
-                sendNotification("repair", message, new int[]{cmd, ActiveRepairService.Status.STARTED.ordinal()});
-
-                if (parallelismDegree != RepairParallelism.PARALLEL && !fullRepair)
-                {
-                    message = "It is not possible to mix sequential repair and incremental repairs.";
-                    logger.error(message);
-                    sendNotification("repair", message, new int[]{cmd, ActiveRepairService.Status.FINISHED.ordinal()});
-                    return;
-                }
-
-                //pre-calculate output of getLocalRanges and pass it to getNeighbors to increase performance and prevent
-                //calculation multiple times
-                Collection<Range<Token>> keyspaceLocalRanges = getLocalRanges(keyspace);
-
-                Set<InetAddress> allNeighbors = new HashSet<>();
-                Map<Range, Set<InetAddress>> rangeToNeighbors = new HashMap<>();
-                for (Range<Token> range : ranges)
-                {
-                    try
-                    {
-                        Set<InetAddress> neighbors = ActiveRepairService.getNeighbors(keyspace, keyspaceLocalRanges, range, dataCenters, hosts);
-                        rangeToNeighbors.put(range, neighbors);
-                        allNeighbors.addAll(neighbors);
-                    }
-                    catch (IllegalArgumentException e)
-                    {
-                        logger.error("Repair failed:", e);
-                        sendNotification("repair", e.getMessage(), new int[]{cmd, ActiveRepairService.Status.FINISHED.ordinal()});
-                        return;
-                    }
-                }
-
-                // Validate columnfamilies
-                List<ColumnFamilyStore> columnFamilyStores = new ArrayList<>();
-                try
-                {
-                    Iterables.addAll(columnFamilyStores, getValidColumnFamilies(false, false, keyspace, columnFamilies));
-                }
-                catch (IllegalArgumentException e)
-                {
-                    sendNotification("repair", e.getMessage(), new int[]{cmd, ActiveRepairService.Status.FINISHED.ordinal()});
-                    return;
-                }
-
-                UUID parentSession = null;
-                if (!fullRepair)
-                {
-                    try
-                    {
-                        parentSession = ActiveRepairService.instance.prepareForRepair(FBUtilities.getBroadcastAddress(), allNeighbors, ranges, columnFamilyStores);
-                    }
-                    catch (Throwable t)
-                    {
-                        sendNotification("repair", String.format("Repair failed with error %s", t.getMessage()), new int[]{cmd, ActiveRepairService.Status.FINISHED.ordinal()});
-                        return;
-                    }
-                }
-
-                List<RepairFuture> futures = new ArrayList<>(ranges.size());
-                String[] cfnames = new String[columnFamilyStores.size()];
-                for (int i = 0; i < columnFamilyStores.size(); i++)
-                {
-                    cfnames[i] = columnFamilyStores.get(i).name;
-                }
-                for (Range<Token> range : ranges)
-                {
-                    RepairFuture future = ActiveRepairService.instance.submitRepairSession(parentSession, range, keyspace, parallelismDegree, rangeToNeighbors.get(range), cfnames);
-                    if (future == null)
-                        continue;
-                    futures.add(future);
-                    // wait for a session to be done with its differencing before starting the next one
-                    try
-                    {
-                        future.session.differencingDone.await();
-                    }
-                    catch (InterruptedException e)
-                    {
-                        message = "Interrupted while waiting for the differencing of repair session " + future.session + " to be done. Repair may be imprecise.";
-                        logger.error(message, e);
-                        sendNotification("repair", message, new int[]{cmd, ActiveRepairService.Status.SESSION_FAILED.ordinal()});
-                    }
-                }
-
-                boolean successful = true;
-                for (RepairFuture future : futures)
-                {
-                    try
-                    {
-                        future.get();
-                        message = String.format("Repair session %s for range %s finished", future.session.getId(), future.session.getRange().toString());
-                        logger.info(message);
-                        sendNotification("repair", message, new int[]{cmd, ActiveRepairService.Status.SESSION_SUCCESS.ordinal()});
-                    }
-                    catch (ExecutionException e)
-                    {
-                        successful = false;
-                        message = String.format("Repair session %s for range %s failed with error %s", future.session.getId(), future.session.getRange().toString(), e.getCause().getMessage());
-                        logger.error(message, e);
-                        sendNotification("repair", message, new int[]{cmd, ActiveRepairService.Status.SESSION_FAILED.ordinal()});
-                    }
-                    catch (Exception e)
-                    {
-                        successful = false;
-                        message = String.format("Repair session %s for range %s failed with error %s", future.session.getId(), future.session.getRange().toString(), e.getMessage());
-                        logger.error(message, e);
-                        sendNotification("repair", message, new int[]{cmd, ActiveRepairService.Status.SESSION_FAILED.ordinal()});
-                    }
-                }
-                if (!fullRepair)
-                {
-                    ListenableFuture future = ActiveRepairService.instance.finishParentSession(parentSession, allNeighbors, successful);
-                    future.addListener(new Runnable()
-                    {
-                        @Override
-                        public void run()
-                        {
-                            sendNotification("repair", String.format("Repair command #%d finished", cmd), new int[]{cmd, ActiveRepairService.Status.FINISHED.ordinal()});
-                        }
-                    }, MoreExecutors.sameThreadExecutor());
-                }
-                else
-                {
-                    sendNotification("repair", String.format("Repair command #%d finished", cmd), new int[]{cmd, ActiveRepairService.Status.FINISHED.ordinal()});
-                }
-            }
-        }, null);
+        RepairRunnable task = new RepairRunnable(this, cmd, options, keyspace);
+        task.addProgressListener(progressSupport);
+        if (legacy)
+            task.addProgressListener(legacyProgressSupport);
+        return new FutureTask<>(task, null);
     }
 
     public void forceTerminateAllRepairSessions() {
@@ -3179,8 +3222,8 @@
     */
     public List<Range<Token>> getAllRanges(List<Token> sortedTokens)
     {
-        if (logger.isDebugEnabled())
-            logger.debug("computing ranges for {}", StringUtils.join(sortedTokens, ", "));
+        if (logger.isTraceEnabled())
+            logger.trace("computing ranges for {}", StringUtils.join(sortedTokens, ", "));
 
         if (sortedTokens.isEmpty())
             return Collections.emptyList();
@@ -3481,7 +3524,7 @@
     private Future<StreamState> streamHints()
     {
         // StreamPlan will not fail if there are zero files to transfer, so flush anyway (need to get any in-memory hints, as well)
-        ColumnFamilyStore hintsCF = Keyspace.open(Keyspace.SYSTEM_KS).getColumnFamilyStore(SystemKeyspace.HINTS_CF);
+        ColumnFamilyStore hintsCF = Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStore(SystemKeyspace.HINTS);
         FBUtilities.waitOnFuture(hintsCF.forceFlush());
 
         // gather all live nodes in the cluster that aren't also leaving
@@ -3512,10 +3555,10 @@
 
             return new StreamPlan("Hints").transferRanges(hintsDestinationHost,
                                                           preferred,
-                                                                      Keyspace.SYSTEM_KS,
-                                                                      ranges,
-                                                                      SystemKeyspace.HINTS_CF)
-                                                      .execute();
+                                                          SystemKeyspace.NAME,
+                                                          ranges,
+                                                          SystemKeyspace.HINTS)
+                                          .execute();
         }
     }
 
@@ -3649,7 +3692,7 @@
                             {
                                 List<InetAddress> endpoints = null;
 
-                                if (RangeStreamer.useStrictConsistency)
+                                if (useStrictConsistency)
                                 {
                                     Set<InetAddress> oldEndpoints = Sets.newHashSet(rangeAddresses.get(range));
                                     Set<InetAddress> newEndpoints = Sets.newHashSet(strategy.calculateNaturalEndpoints(toFetch.right, tokenMetaCloneAllSettled));
@@ -3683,7 +3726,7 @@
                         if (addressList == null || addressList.isEmpty())
                             continue;
 
-                        if (RangeStreamer.useStrictConsistency)
+                        if (useStrictConsistency)
                         {
                             if (addressList.size() > 1)
                                 throw new IllegalStateException("Multiple strict sources found for " + toFetch);
@@ -3718,7 +3761,7 @@
                     }
 
                     // stream requests
-                    Multimap<InetAddress, Range<Token>> workMap = RangeStreamer.getWorkMap(rangesToFetchWithPreferredEndpoints, keyspace);
+                    Multimap<InetAddress, Range<Token>> workMap = RangeStreamer.getWorkMap(rangesToFetchWithPreferredEndpoints, keyspace, FailureDetector.instance, useStrictConsistency);
                     for (InetAddress address : workMap.keySet())
                     {
                         logger.debug("Will request range {} of keyspace {} from endpoint {}", workMap.get(address), keyspace, address);
@@ -3875,11 +3918,6 @@
         }
     }
 
-    public boolean isClientMode()
-    {
-        return isClientMode;
-    }
-
     public String getOperationMode()
     {
         return operationMode.toString();
@@ -3949,8 +3987,17 @@
             FBUtilities.waitOnFuture(f);
             remainingCFs--;
         }
-        // flush the system ones after all the rest are done, just in case flushing modifies any system state
-        // like CASSANDRA-5151. don't bother with progress tracking since system data is tiny.
+
+        BatchlogManager.shutdown();
+
+        // Interrupt on going compaction and shutdown to prevent further compaction
+        CompactionManager.instance.forceShutdown();
+
+        // Flush the system tables after all other tables are flushed, just in case flushing modifies any system state
+        // like CASSANDRA-5151. Don't bother with progress tracking since system data is tiny.
+        // Flush system tables after stopping the batchlog manager and compactions since they both modify
+        // system tables (for example compactions can obsolete sstables and the tidiers in SSTableReader update
+        // system tables, see SSTableReader.GlobalTidy)
         flushes.clear();
         for (Keyspace keyspace : Keyspace.system())
         {
@@ -3959,23 +4006,18 @@
         }
         FBUtilities.waitOnFutures(flushes);
 
-        BatchlogManager.shutdown();
-
-        // Interrupt on going compaction and shutdown to prevent further compaction
-        CompactionManager.instance.forceShutdown();
-
         // whilst we've flushed all the CFs, which will have recycled all completed segments, we want to ensure
         // there are no segments to replay, so we force the recycling of any remaining (should be at most one)
         CommitLog.instance.forceRecycleAllSegments();
 
-        ColumnFamilyStore.shutdownPostFlushExecutor();
-
         CommitLog.instance.shutdownBlocking();
 
         // wait for miscellaneous tasks like sstable and commitlog segment deletion
         ScheduledExecutors.nonPeriodicTasks.shutdown();
         if (!ScheduledExecutors.nonPeriodicTasks.awaitTermination(1, TimeUnit.MINUTES))
-            logger.warn("Miscellaneous task executor still busy after one minute; proceeding with shutdown");
+            logger.warn("Failed to wait for non periodic tasks to shutdown");
+
+        ColumnFamilyStore.shutdownPostFlushExecutor();
 
         setMode(Mode.DRAINED, true);
     }
@@ -4104,30 +4146,18 @@
         return finalOwnership;
     }
 
-
-    private boolean hasSameReplication(List<String> list)
-    {
-        if (list.isEmpty())
-            return false;
-
-        for (int i = 0; i < list.size() -1; i++)
-        {
-            KSMetaData ksm1 = Schema.instance.getKSMetaData(list.get(i));
-            KSMetaData ksm2 = Schema.instance.getKSMetaData(list.get(i + 1));
-            if (!ksm1.strategyClass.equals(ksm2.strategyClass) ||
-                    !Iterators.elementsEqual(ksm1.strategyOptions.entrySet().iterator(),
-                                             ksm2.strategyOptions.entrySet().iterator()))
-                return false;
-        }
-        return true;
-    }
-
     public List<String> getKeyspaces()
     {
         List<String> keyspaceNamesList = new ArrayList<>(Schema.instance.getKeyspaces());
         return Collections.unmodifiableList(keyspaceNamesList);
     }
 
+    public List<String> getNonSystemKeyspaces()
+    {
+        List<String> keyspaceNamesList = new ArrayList<>(Schema.instance.getNonSystemKeyspaces());
+        return Collections.unmodifiableList(keyspaceNamesList);
+    }
+
     public void updateSnitch(String epSnitchClassName, Boolean dynamic, Integer dynamicUpdateInterval, Integer dynamicResetInterval, Double dynamicBadnessThreshold) throws ClassNotFoundException
     {
         IEndpointSnitch oldSnitch = DatabaseDescriptor.getEndpointSnitch();
@@ -4232,10 +4262,10 @@
         Set<Range<Token>> toFetch  = new HashSet<>();
 
 
-        for (Range r1 : current)
+        for (Range<Token> r1 : current)
         {
             boolean intersect = false;
-            for (Range r2 : updated)
+            for (Range<Token> r2 : updated)
             {
                 if (r1.intersects(r2))
                 {
@@ -4250,10 +4280,10 @@
             }
         }
 
-        for (Range r2 : updated)
+        for (Range<Token> r2 : updated)
         {
             boolean intersect = false;
-            for (Range r1 : current)
+            for (Range<Token> r1 : current)
             {
                 if (r2.intersects(r1))
                 {
@@ -4297,8 +4327,11 @@
 
         SSTableLoader.Client client = new SSTableLoader.Client()
         {
+            private String keyspace;
+
             public void init(String keyspace)
             {
+                this.keyspace = keyspace;
                 try
                 {
                     setPartitioner(DatabaseDescriptor.getPartitioner());
@@ -4315,19 +4348,13 @@
                 }
             }
 
-            public CFMetaData getCFMetaData(String keyspace, String cfName)
+            public CFMetaData getTableMetadata(String tableName)
             {
-                return Schema.instance.getCFMetaData(keyspace, cfName);
+                return Schema.instance.getCFMetaData(keyspace, tableName);
             }
         };
 
-        SSTableLoader loader = new SSTableLoader(dir, client, new OutputHandler.LogOutput());
-        return loader.stream();
-    }
-
-    public int getExceptionCount()
-    {
-        return (int)StorageMetrics.exceptions.count();
+        return new SSTableLoader(dir, client, new OutputHandler.LogOutput()).stream();
     }
 
     public void rescheduleFailedDeletions()
@@ -4373,12 +4400,12 @@
 
     public void setTraceProbability(double probability)
     {
-        this.tracingProbability = probability;
+        this.traceProbability = probability;
     }
 
-    public double getTracingProbability()
+    public double getTraceProbability()
     {
-        return tracingProbability;
+        return traceProbability;
     }
 
     public void disableAutoCompaction(String ks, String... columnFamilies) throws IOException
@@ -4429,9 +4456,20 @@
         DatabaseDescriptor.setTombstoneFailureThreshold(threshold);
     }
 
+    public int getBatchSizeFailureThreshold()
+    {
+        return DatabaseDescriptor.getBatchSizeFailThresholdInKB();
+    }
+
+    public void setBatchSizeFailureThreshold(int threshold)
+    {
+        DatabaseDescriptor.setBatchSizeFailThresholdInKB(threshold);
+    }
+
     public void setHintedHandoffThrottleInKB(int throttleInKB)
     {
         DatabaseDescriptor.setHintedHandoffThrottleInKB(throttleInKB);
         logger.info(String.format("Updated hinted_handoff_throttle_in_kb to %d", throttleInKB));
     }
+
 }
diff --git a/src/java/org/apache/cassandra/service/StorageServiceMBean.java b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
index d3a1725..761eed6 100644
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@ -168,13 +168,6 @@
     /** Retrieve the mapping of host ID to endpoint */
     public Map<String, String> getHostIdToEndpoint();
 
-    /**
-     * Numeric load value.
-     * @see org.apache.cassandra.metrics.StorageMetrics#load
-     */
-    @Deprecated
-    public double getLoad();
-
     /** Human-readable load value */
     public String getLoadString();
 
@@ -226,9 +219,7 @@
      *            list of columnfamily from different keyspace in the form of ks1.cf1 ks2.cf2
      */
     public void takeMultipleColumnFamilySnapshot(String tag, String... columnFamilyList) throws IOException;
-    
-    
-    
+
     /**
      * Remove the snapshot with the given name from the given keyspaces.
      * If no tag is specified we will remove all snapshots.
@@ -255,7 +246,7 @@
     /**
      * Forces major compaction of a single keyspace
      */
-    public void forceKeyspaceCompaction(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+    public void forceKeyspaceCompaction(boolean splitOutput, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 
     /**
      * Trigger a cleanup of keys on a single keyspace
@@ -277,6 +268,14 @@
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 
     /**
+     * Verify (checksums of) the given keyspace.
+     * If columnFamilies array is empty, all CFs are verified.
+     *
+     * The entire sstable will be read to ensure each cell validates if extendedVerify is true
+     */
+    public int verify(boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+
+    /**
      * Rewrite all sstables to the latest version.
      * Unlike scrub, it doesn't skip bad rows and do not snapshot sstables first.
      */
@@ -300,9 +299,17 @@
      *   type: "repair"
      *   userObject: int array of length 2, [0]=command number, [1]=ordinal of ActiveRepairService.Status
      *
+     * @param keyspace Keyspace name to repair. Should not be null.
+     * @param options repair option.
      * @return Repair command number, or 0 if nothing to repair
      */
-    public int forceRepairAsync(String keyspace, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts,  boolean primaryRange, boolean repairedAt, String... columnFamilies) throws IOException;
+    public int repairAsync(String keyspace, Map<String, String> options);
+
+    /**
+     * @deprecated use {@link #repairAsync(String keyspace, Map options)} instead.
+     */
+    @Deprecated
+    public int forceRepairAsync(String keyspace, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts,  boolean primaryRange, boolean fullRepair, String... columnFamilies) throws IOException;
 
     /**
      * Invoke repair asynchronously.
@@ -311,38 +318,41 @@
      *   type: "repair"
      *   userObject: int array of length 2, [0]=command number, [1]=ordinal of ActiveRepairService.Status
      *
+     * @deprecated use {@link #repairAsync(String keyspace, Map options)} instead.
+     *
      * @param parallelismDegree 0: sequential, 1: parallel, 2: DC parallel
      * @return Repair command number, or 0 if nothing to repair
      */
+    @Deprecated
     public int forceRepairAsync(String keyspace, int parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, boolean primaryRange, boolean fullRepair, String... columnFamilies);
 
     /**
-     * Same as forceRepairAsync, but handles a specified range
+     * @deprecated use {@link #repairAsync(String keyspace, Map options)} instead.
      */
-    public int forceRepairRangeAsync(String beginToken, String endToken, String keyspaceName, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts, boolean repairedAt, String... columnFamilies) throws IOException;
+    @Deprecated
+    public int forceRepairRangeAsync(String beginToken, String endToken, String keyspaceName, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts, boolean fullRepair, String... columnFamilies) throws IOException;
 
     /**
      * Same as forceRepairAsync, but handles a specified range
      *
+     * @deprecated use {@link #repairAsync(String keyspace, Map options)} instead.
+     *
      * @param parallelismDegree 0: sequential, 1: parallel, 2: DC parallel
      */
+    @Deprecated
     public int forceRepairRangeAsync(String beginToken, String endToken, String keyspaceName, int parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, boolean fullRepair, String... columnFamilies);
 
     /**
-     * Invoke repair asynchronously.
-     * You can track repair progress by subscribing JMX notification sent from this StorageServiceMBean.
-     * Notification format is:
-     *   type: "repair"
-     *   userObject: int array of length 2, [0]=command number, [1]=ordinal of ActiveRepairService.Status
-     *
-     * @return Repair command number, or 0 if nothing to repair
+     * @deprecated use {@link #repairAsync(String keyspace, Map options)} instead.
      */
+    @Deprecated
     public int forceRepairAsync(String keyspace, boolean isSequential, boolean isLocal, boolean primaryRange, boolean fullRepair, String... columnFamilies);
 
     /**
-     * Same as forceRepairAsync, but handles a specified range
+     * @deprecated use {@link #repairAsync(String keyspace, Map options)} instead.
      */
-    public int forceRepairRangeAsync(String beginToken, String endToken, String keyspaceName, boolean isSequential, boolean isLocal, boolean repairedAt, String... columnFamilies);
+    @Deprecated
+    public int forceRepairRangeAsync(String beginToken, String endToken, String keyspaceName, boolean isSequential, boolean isLocal, boolean fullRepair, String... columnFamilies);
 
     public void forceTerminateAllRepairSessions();
 
@@ -434,6 +444,8 @@
 
     public List<String> getKeyspaces();
 
+    public List<String> getNonSystemKeyspaces();
+
     /**
      * Change endpointsnitch class and dynamic-ness (and dynamic attributes) at runtime
      * @param epSnitchClassName        the canonical path name for a class implementing IEndpointSnitch
@@ -476,9 +488,6 @@
     public void joinRing() throws IOException;
     public boolean isJoined();
 
-    @Deprecated
-    public int getExceptionCount();
-
     public void setStreamThroughputMbPerSec(int value);
     public int getStreamThroughputMbPerSec();
 
@@ -548,7 +557,7 @@
     /**
      * Returns the configured tracing probability.
      */
-    public double getTracingProbability();
+    public double getTraceProbability();
 
     void disableAutoCompaction(String ks, String ... columnFamilies) throws IOException;
     void enableAutoCompaction(String ks, String ... columnFamilies) throws IOException;
@@ -570,6 +579,19 @@
     /** Sets the threshold for abandoning queries with many tombstones */
     public void setTombstoneFailureThreshold(int tombstoneDebugThreshold);
 
+    /** Returns the threshold for rejecting queries due to a large batch size */
+    public int getBatchSizeFailureThreshold();
+    /** Sets the threshold for rejecting queries due to a large batch size */
+    public void setBatchSizeFailureThreshold(int batchSizeDebugThreshold);
+
     /** Sets the hinted handoff throttle in kb per second, per delivery thread. */
     public void setHintedHandoffThrottleInKB(int throttleInKB);
+
+    /**
+     * Resume bootstrap streaming when there is failed data streaming.
+     *
+     *
+     * @return true if the node successfully starts resuming. (this does not mean bootstrap streaming was success.)
+     */
+    public boolean resumeBootstrap();
 }
diff --git a/src/java/org/apache/cassandra/service/WriteResponseHandler.java b/src/java/org/apache/cassandra/service/WriteResponseHandler.java
index df23b19..1dc03e0 100644
--- a/src/java/org/apache/cassandra/service/WriteResponseHandler.java
+++ b/src/java/org/apache/cassandra/service/WriteResponseHandler.java
@@ -34,7 +34,7 @@
 /**
  * Handles blocking writes for ONE, ANY, TWO, THREE, QUORUM, and ALL consistency levels.
  */
-public class WriteResponseHandler extends AbstractWriteResponseHandler
+public class WriteResponseHandler<T> extends AbstractWriteResponseHandler<T>
 {
     protected static final Logger logger = LoggerFactory.getLogger(WriteResponseHandler.class);
 
@@ -63,7 +63,7 @@
         this(endpoint, writeType, null);
     }
 
-    public void response(MessageIn m)
+    public void response(MessageIn<T> m)
     {
         if (responsesUpdater.decrementAndGet(this) == 0)
             signal();
diff --git a/src/java/org/apache/cassandra/service/pager/AbstractQueryPager.java b/src/java/org/apache/cassandra/service/pager/AbstractQueryPager.java
index 8bbf6d6..02623eb 100644
--- a/src/java/org/apache/cassandra/service/pager/AbstractQueryPager.java
+++ b/src/java/org/apache/cassandra/service/pager/AbstractQueryPager.java
@@ -301,7 +301,7 @@
         DeletionInfo.InOrderTester tester = cf.deletionInfo().inOrderTester(isReversed);
         return isReversed
              ? discardTail(cf, toDiscard, newCf, cf.reverseIterator(), tester)
-             : discardHead(cf, toDiscard, newCf, cf.iterator(), tester);
+             : discardHead(toDiscard, newCf, cf.iterator(), tester);
     }
 
     private int discardLast(ColumnFamily cf, int toDiscard, ColumnFamily newCf)
@@ -309,11 +309,11 @@
         boolean isReversed = isReversed();
         DeletionInfo.InOrderTester tester = cf.deletionInfo().inOrderTester(isReversed);
         return isReversed
-             ? discardHead(cf, toDiscard, newCf, cf.reverseIterator(), tester)
+             ? discardHead(toDiscard, newCf, cf.reverseIterator(), tester)
              : discardTail(cf, toDiscard, newCf, cf.iterator(), tester);
     }
 
-    private int discardHead(ColumnFamily cf, int toDiscard, ColumnFamily copy, Iterator<Cell> iter, DeletionInfo.InOrderTester tester)
+    private int discardHead(int toDiscard, ColumnFamily copy, Iterator<Cell> iter, DeletionInfo.InOrderTester tester)
     {
         ColumnCounter counter = columnCounter();
 
diff --git a/src/java/org/apache/cassandra/service/pager/PagingState.java b/src/java/org/apache/cassandra/service/pager/PagingState.java
index bbae921..f168880 100644
--- a/src/java/org/apache/cassandra/service/pager/PagingState.java
+++ b/src/java/org/apache/cassandra/service/pager/PagingState.java
@@ -22,6 +22,7 @@
 import java.nio.ByteBuffer;
 
 import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.io.util.DataOutputBufferFixed;
 import org.apache.cassandra.transport.ProtocolException;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
@@ -59,13 +60,12 @@
 
     public ByteBuffer serialize()
     {
-        try
+        try (DataOutputBuffer out = new DataOutputBufferFixed(serializedSize()))
         {
-            DataOutputBuffer out = new DataOutputBuffer(serializedSize());
             ByteBufferUtil.writeWithShortLength(partitionKey, out);
             ByteBufferUtil.writeWithShortLength(cellName, out);
             out.writeInt(remaining);
-            return out.asByteBuffer();
+            return out.buffer();
         }
         catch (IOException e)
         {
diff --git a/src/java/org/apache/cassandra/service/pager/RangeNamesQueryPager.java b/src/java/org/apache/cassandra/service/pager/RangeNamesQueryPager.java
index 50d1280..6b36a25 100644
--- a/src/java/org/apache/cassandra/service/pager/RangeNamesQueryPager.java
+++ b/src/java/org/apache/cassandra/service/pager/RangeNamesQueryPager.java
@@ -81,7 +81,9 @@
     protected boolean containsPreviousLast(Row first)
     {
         // When querying the next page, we create a bound that exclude the lastReturnedKey
-        return false;
+        // but unfortunately ExcludingBounds is serialized as Bounds, which includes both endpoints,
+        // so we may still get a live row with the same key as lastReturnedKey, see CASSANDRA-10509
+        return lastReturnedKey != null && lastReturnedKey.equals(first.key);
     }
 
     protected boolean recordLast(Row last)
@@ -103,11 +105,11 @@
         AbstractBounds<RowPosition> bounds = command.keyRange;
         if (bounds instanceof Range || bounds instanceof Bounds)
         {
-            return new Range<RowPosition>(lastReturnedKey, bounds.right);
+            return new Range<>(lastReturnedKey, bounds.right);
         }
         else
         {
-            return new ExcludingBounds<RowPosition>(lastReturnedKey, bounds.right);
+            return new ExcludingBounds<>(lastReturnedKey, bounds.right);
         }
     }
 }
diff --git a/src/java/org/apache/cassandra/service/pager/RangeSliceQueryPager.java b/src/java/org/apache/cassandra/service/pager/RangeSliceQueryPager.java
index 3ac777e..3b16e0b 100644
--- a/src/java/org/apache/cassandra/service/pager/RangeSliceQueryPager.java
+++ b/src/java/org/apache/cassandra/service/pager/RangeSliceQueryPager.java
@@ -72,7 +72,8 @@
     protected List<Row> queryNextPage(int pageSize, ConsistencyLevel consistencyLevel, boolean localQuery)
     throws RequestExecutionException
     {
-        SliceQueryFilter sf = (SliceQueryFilter)columnFilter;
+        SliceQueryFilter rawFilter = (SliceQueryFilter)columnFilter;
+        SliceQueryFilter sf = rawFilter.withUpdatedCount(Math.min(rawFilter.count, pageSize));
         AbstractBounds<RowPosition> keyRange = lastReturnedKey == null ? command.keyRange : makeIncludingKeyBounds(lastReturnedKey);
         // For DISTINCT queries we can and must ignore the lastReturnedName (see CASSANDRA-13017)
         Composite start = lastReturnedName == null || isDistinct() ? sf.start() : lastReturnedName;
@@ -103,7 +104,6 @@
 
         // Same as SliceQueryPager, we ignore a deleted column
         Cell firstCell = isReversed() ? lastCell(first.cf) : firstNonStaticCell(first.cf);
-
         // If the row was containing only static columns it has already been returned and we can skip it.
         if (firstCell == null)
             return true;
diff --git a/src/java/org/apache/cassandra/service/paxos/PrepareCallback.java b/src/java/org/apache/cassandra/service/paxos/PrepareCallback.java
index 2859a69..081f457 100644
--- a/src/java/org/apache/cassandra/service/paxos/PrepareCallback.java
+++ b/src/java/org/apache/cassandra/service/paxos/PrepareCallback.java
@@ -61,7 +61,7 @@
     public synchronized void response(MessageIn<PrepareResponse> message)
     {
         PrepareResponse response = message.payload;
-        logger.debug("Prepare response {} from {}", response, message.from);
+        logger.trace("Prepare response {} from {}", response, message.from);
 
         // In case of clock skew, another node could be proposing with ballot that are quite a bit
         // older than our own. In that case, we record the more recent commit we've received to make
diff --git a/src/java/org/apache/cassandra/service/paxos/ProposeCallback.java b/src/java/org/apache/cassandra/service/paxos/ProposeCallback.java
index 018dab9..b0bd163 100644
--- a/src/java/org/apache/cassandra/service/paxos/ProposeCallback.java
+++ b/src/java/org/apache/cassandra/service/paxos/ProposeCallback.java
@@ -59,7 +59,7 @@
 
     public void response(MessageIn<Boolean> msg)
     {
-        logger.debug("Propose response {} from {}", msg.payload, msg.from);
+        logger.trace("Propose response {} from {}", msg.payload, msg.from);
 
         if (msg.payload)
             accepts.incrementAndGet();
diff --git a/src/java/org/apache/cassandra/sink/IMessageSink.java b/src/java/org/apache/cassandra/sink/IMessageSink.java
deleted file mode 100644
index 996e7ff..0000000
--- a/src/java/org/apache/cassandra/sink/IMessageSink.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * 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.sink;
-
-import java.net.InetAddress;
-
-import org.apache.cassandra.net.MessageIn;
-import org.apache.cassandra.net.MessageOut;
-
-public interface IMessageSink
-{
-    /**
-     * Transform or drop an outgoing message
-     *
-     * @return null if the message is dropped, or the transformed message to send, which may be just
-     * the original message
-     */
-    MessageOut handleMessage(MessageOut message, int id, InetAddress to);
-
-    /**
-     * Transform or drop an incoming message
-     *
-     * @return null if the message is dropped, or the transformed message to receive, which may be just
-     * the original message
-     */
-    MessageIn handleMessage(MessageIn message, int id, InetAddress to);
-}
diff --git a/src/java/org/apache/cassandra/sink/IRequestSink.java b/src/java/org/apache/cassandra/sink/IRequestSink.java
deleted file mode 100644
index 2873e46..0000000
--- a/src/java/org/apache/cassandra/sink/IRequestSink.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * 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.sink;
-
-import org.apache.cassandra.db.IMutation;
-
-public interface IRequestSink
-{
-    /**
-     * Transform or drop a write request (represented by a Mutation).
-     *
-     * @param mutation the Mutation to be applied locally.
-     * @return null if the mutation is to be dropped, or the transformed mutation to apply, which may be just
-     * the original mutation.
-     */
-    IMutation handleWriteRequest(IMutation mutation);
-}
diff --git a/src/java/org/apache/cassandra/sink/SinkManager.java b/src/java/org/apache/cassandra/sink/SinkManager.java
deleted file mode 100644
index 9b422dc..0000000
--- a/src/java/org/apache/cassandra/sink/SinkManager.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/*
- * 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.sink;
-
-import java.net.InetAddress;
-import java.util.Set;
-import java.util.concurrent.CopyOnWriteArraySet;
-
-import org.apache.cassandra.db.IMutation;
-import org.apache.cassandra.net.MessageIn;
-import org.apache.cassandra.net.MessageOut;
-
-public class SinkManager
-{
-    private static final Set<IMessageSink> messageSinks = new CopyOnWriteArraySet<>();
-    private static final Set<IRequestSink> requestSinks = new CopyOnWriteArraySet<>();
-
-    public static void add(IMessageSink ms)
-    {
-        messageSinks.add(ms);
-    }
-
-    public static void add(IRequestSink rs)
-    {
-        requestSinks.add(rs);
-    }
-
-    public static void remove(IMessageSink ms)
-    {
-        messageSinks.remove(ms);
-    }
-
-    public static void remove(IRequestSink rs)
-    {
-        requestSinks.remove(rs);
-    }
-
-    public static void clear()
-    {
-        messageSinks.clear();
-        requestSinks.clear();
-    }
-
-    public static MessageOut processOutboundMessage(MessageOut message, int id, InetAddress to)
-    {
-        if (messageSinks.isEmpty())
-            return message;
-
-        for (IMessageSink ms : messageSinks)
-        {
-            message = ms.handleMessage(message, id, to);
-            if (message == null)
-                return null;
-        }
-        return message;
-    }
-
-    public static MessageIn processInboundMessage(MessageIn message, int id)
-    {
-        if (messageSinks.isEmpty())
-            return message;
-
-        for (IMessageSink ms : messageSinks)
-        {
-            message = ms.handleMessage(message, id, null);
-            if (message == null)
-                return null;
-        }
-        return message;
-    }
-
-    public static IMutation processWriteRequest(IMutation mutation)
-    {
-        if (requestSinks.isEmpty())
-            return mutation;
-
-        for (IRequestSink rs : requestSinks)
-        {
-            mutation = rs.handleWriteRequest(mutation);
-            if (mutation == null)
-                return null;
-        }
-        return mutation;
-    }
-}
diff --git a/src/java/org/apache/cassandra/streaming/ConnectionHandler.java b/src/java/org/apache/cassandra/streaming/ConnectionHandler.java
index 60ce11e..fe551a8 100644
--- a/src/java/org/apache/cassandra/streaming/ConnectionHandler.java
+++ b/src/java/org/apache/cassandra/streaming/ConnectionHandler.java
@@ -17,6 +17,7 @@
  */
 package org.apache.cassandra.streaming;
 
+import java.io.BufferedOutputStream;
 import java.io.IOException;
 import java.net.Socket;
 import java.net.SocketException;
@@ -33,10 +34,12 @@
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.SettableFuture;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.io.util.DataOutputStreamAndChannel;
+import org.apache.cassandra.io.util.DataOutputStreamPlus;
+import org.apache.cassandra.io.util.BufferedDataOutputStreamPlus;
+import org.apache.cassandra.io.util.WrappedDataOutputStreamPlus;
 import org.apache.cassandra.net.IncomingStreamingConnection;
 import org.apache.cassandra.streaming.messages.StreamInitMessage;
 import org.apache.cassandra.streaming.messages.StreamMessage;
@@ -74,17 +77,16 @@
      *
      * @throws IOException
      */
+    @SuppressWarnings("resource")
     public void initiate() throws IOException
     {
         logger.debug("[Stream #{}] Sending stream init for incoming stream", session.planId());
         Socket incomingSocket = session.createConnection();
-        incoming.start(incomingSocket, StreamMessage.CURRENT_VERSION);
-        incoming.sendInitMessage(incomingSocket, true);
+        incoming.start(incomingSocket, StreamMessage.CURRENT_VERSION, true);
 
         logger.debug("[Stream #{}] Sending stream init for outgoing stream", session.planId());
         Socket outgoingSocket = session.createConnection();
-        outgoing.start(outgoingSocket, StreamMessage.CURRENT_VERSION);
-        outgoing.sendInitMessage(outgoingSocket, false);
+        outgoing.start(outgoingSocket, StreamMessage.CURRENT_VERSION, true);
     }
 
     /**
@@ -155,24 +157,27 @@
 
         protected int protocolVersion;
         protected Socket socket;
+        private final boolean isOutgoingHandler;
 
         private final AtomicReference<SettableFuture<?>> closeFuture = new AtomicReference<>();
         private IncomingStreamingConnection incomingConnection;
 
-        protected MessageHandler(StreamSession session)
+        protected MessageHandler(StreamSession session, boolean isOutgoingHandler)
         {
             this.session = session;
+            this.isOutgoingHandler = isOutgoingHandler;
         }
 
         protected abstract String name();
 
-        protected static DataOutputStreamAndChannel getWriteChannel(Socket socket) throws IOException
+        @SuppressWarnings("resource")
+        protected static DataOutputStreamPlus getWriteChannel(Socket socket) throws IOException
         {
             WritableByteChannel out = socket.getChannel();
             // socket channel is null when encrypted(SSL)
             if (out == null)
-                out = Channels.newChannel(socket.getOutputStream());
-            return new DataOutputStreamAndChannel(socket.getOutputStream(), out);
+                return new WrappedDataOutputStreamPlus(new BufferedOutputStream(socket.getOutputStream()));
+            return new BufferedDataOutputStreamPlus(out);
         }
 
         protected static ReadableByteChannel getReadChannel(Socket socket) throws IOException
@@ -181,28 +186,35 @@
             return Channels.newChannel(socket.getInputStream());
         }
 
-        public void sendInitMessage(Socket socket, boolean isForOutgoing) throws IOException
+        @SuppressWarnings("resource")
+        public void sendInitMessage() throws IOException
         {
             StreamInitMessage message = new StreamInitMessage(
                     FBUtilities.getBroadcastAddress(),
                     session.sessionIndex(),
                     session.planId(),
                     session.description(),
-                    isForOutgoing);
+                    !isOutgoingHandler,
+                    session.keepSSTableLevel(),
+                    session.isIncremental());
             ByteBuffer messageBuf = message.createMessage(false, protocolVersion);
-            getWriteChannel(socket).write(messageBuf);
+            DataOutputStreamPlus out = getWriteChannel(socket);
+            out.write(messageBuf);
+            out.flush();
         }
 
-        public void start(IncomingStreamingConnection connection, int protocolVersion)
+        public void start(IncomingStreamingConnection connection, int protocolVersion) throws IOException
         {
             this.incomingConnection = connection;
-            start(connection.socket, protocolVersion);
+            start(connection.socket, protocolVersion, false);
         }
 
-        public void start(Socket socket, int protocolVersion)
+        public void start(Socket socket, int protocolVersion, boolean initiator) throws IOException
         {
             this.socket = socket;
             this.protocolVersion = protocolVersion;
+            if (initiator)
+                sendInitMessage();
 
             new Thread(this, name() + "-" + session.peer).start();
         }
@@ -223,6 +235,9 @@
 
         protected void signalCloseDone()
         {
+            if (!isClosed())
+                close();
+
             closeFuture.get().set(null);
 
             // We can now close the socket
@@ -257,7 +272,7 @@
     {
         IncomingMessageHandler(StreamSession session)
         {
-            super(session);
+            super(session, false);
         }
 
         protected String name()
@@ -265,6 +280,7 @@
             return "STREAM-IN";
         }
 
+        @SuppressWarnings("resource")
         public void run()
         {
             try
@@ -283,11 +299,6 @@
                     }
                 }
             }
-            catch (SocketException e)
-            {
-                // socket is closed
-                close();
-            }
             catch (Throwable t)
             {
                 JVMStabilityInspector.inspectThrowable(t);
@@ -321,7 +332,7 @@
 
         OutgoingMessageHandler(StreamSession session)
         {
-            super(session);
+            super(session, true);
         }
 
         protected String name()
@@ -334,11 +345,12 @@
             messageQueue.put(message);
         }
 
+        @SuppressWarnings("resource")
         public void run()
         {
             try
             {
-                DataOutputStreamAndChannel out = getWriteChannel(socket);
+                DataOutputStreamPlus out = getWriteChannel(socket);
 
                 StreamMessage next;
                 while (!isClosed())
@@ -370,11 +382,12 @@
             }
         }
 
-        private void sendMessage(DataOutputStreamAndChannel out, StreamMessage message)
+        private void sendMessage(DataOutputStreamPlus out, StreamMessage message)
         {
             try
             {
                 StreamMessage.serialize(message, out, protocolVersion, session);
+                out.flush();
             }
             catch (SocketException e)
             {
diff --git a/src/java/org/apache/cassandra/streaming/DefaultConnectionFactory.java b/src/java/org/apache/cassandra/streaming/DefaultConnectionFactory.java
index f711490..5c27ff3 100644
--- a/src/java/org/apache/cassandra/streaming/DefaultConnectionFactory.java
+++ b/src/java/org/apache/cassandra/streaming/DefaultConnectionFactory.java
@@ -60,7 +60,7 @@
                     throw e;
 
                 long waitms = DatabaseDescriptor.getRpcTimeout() * (long)Math.pow(2, attempts);
-                logger.warn("Failed attempt " + attempts + " to connect to " + peer + ". Retrying in " + waitms + " ms. (" + e + ")");
+                logger.warn("Failed attempt {} to connect to {}. Retrying in {} ms. ({})", attempts, peer, waitms, e);
                 try
                 {
                     Thread.sleep(waitms);
diff --git a/src/java/org/apache/cassandra/streaming/StreamCoordinator.java b/src/java/org/apache/cassandra/streaming/StreamCoordinator.java
index a0c99fe..603366d 100644
--- a/src/java/org/apache/cassandra/streaming/StreamCoordinator.java
+++ b/src/java/org/apache/cassandra/streaming/StreamCoordinator.java
@@ -45,11 +45,15 @@
     private Map<InetAddress, HostStreamingData> peerSessions = new HashMap<>();
     private final int connectionsPerHost;
     private StreamConnectionFactory factory;
+    private final boolean keepSSTableLevel;
+    private final boolean isIncremental;
 
-    public StreamCoordinator(int connectionsPerHost, StreamConnectionFactory factory)
+    public StreamCoordinator(int connectionsPerHost, boolean keepSSTableLevel, boolean isIncremental, StreamConnectionFactory factory)
     {
         this.connectionsPerHost = connectionsPerHost;
         this.factory = factory;
+        this.keepSSTableLevel = keepSSTableLevel;
+        this.isIncremental = isIncremental;
     }
 
     public void setConnectionFactory(StreamConnectionFactory factory)
@@ -179,7 +183,7 @@
     {
         HostStreamingData data = peerSessions.get(peer);
         if (data == null)
-            throw new IllegalArgumentException("Unknown peer requested: " + peer.toString());
+            throw new IllegalArgumentException("Unknown peer requested: " + peer);
         return data;
     }
 
@@ -233,7 +237,7 @@
             // create
             if (streamSessions.size() < connectionsPerHost)
             {
-                StreamSession session = new StreamSession(peer, connecting, factory, streamSessions.size());
+                StreamSession session = new StreamSession(peer, connecting, factory, streamSessions.size(), keepSSTableLevel, isIncremental);
                 streamSessions.put(++lastReturned, session);
                 return session;
             }
@@ -265,7 +269,7 @@
             StreamSession session = streamSessions.get(id);
             if (session == null)
             {
-                session = new StreamSession(peer, connecting, factory, id);
+                session = new StreamSession(peer, connecting, factory, id, keepSSTableLevel, isIncremental);
                 streamSessions.put(id, session);
             }
             return session;
diff --git a/src/java/org/apache/cassandra/streaming/StreamEvent.java b/src/java/org/apache/cassandra/streaming/StreamEvent.java
index 8089323..de3db9c 100644
--- a/src/java/org/apache/cassandra/streaming/StreamEvent.java
+++ b/src/java/org/apache/cassandra/streaming/StreamEvent.java
@@ -18,8 +18,11 @@
 package org.apache.cassandra.streaming;
 
 import java.net.InetAddress;
+import java.util.Set;
 import java.util.UUID;
 
+import com.google.common.collect.ImmutableSet;
+
 public abstract class StreamEvent
 {
     public static enum Type
@@ -43,6 +46,7 @@
         public final InetAddress peer;
         public final boolean success;
         public final int sessionIndex;
+        public final Set<StreamRequest> requests;
 
         public SessionCompleteEvent(StreamSession session)
         {
@@ -50,6 +54,7 @@
             this.peer = session.peer;
             this.success = session.isSuccess();
             this.sessionIndex = session.sessionIndex();
+            this.requests = ImmutableSet.copyOf(session.requests);
         }
     }
 
@@ -66,7 +71,7 @@
         @Override
         public String toString()
         {
-            return "<ProgressEvent " + progress.toString() + ">";
+            return "<ProgressEvent " + progress + ">";
         }
     }
 
diff --git a/src/java/org/apache/cassandra/streaming/StreamLockfile.java b/src/java/org/apache/cassandra/streaming/StreamLockfile.java
index 0b1d832..83ae5f0 100644
--- a/src/java/org/apache/cassandra/streaming/StreamLockfile.java
+++ b/src/java/org/apache/cassandra/streaming/StreamLockfile.java
@@ -28,12 +28,12 @@
 import java.util.UUID;
 
 import com.google.common.base.Charsets;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.sstable.SSTable;
-import org.apache.cassandra.io.sstable.SSTableWriter;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 
@@ -58,7 +58,7 @@
 
     public StreamLockfile(File directory, UUID uuid)
     {
-        lockfile = new File(directory, uuid.toString() + FILE_EXT);
+        lockfile = new File(directory, uuid + FILE_EXT);
     }
 
     public StreamLockfile(File lockfile)
diff --git a/src/java/org/apache/cassandra/streaming/StreamManager.java b/src/java/org/apache/cassandra/streaming/StreamManager.java
index 728631d..dc8ec19 100644
--- a/src/java/org/apache/cassandra/streaming/StreamManager.java
+++ b/src/java/org/apache/cassandra/streaming/StreamManager.java
@@ -70,10 +70,10 @@
 
         public StreamRateLimiter(InetAddress peer)
         {
-            double throughput = ((double) DatabaseDescriptor.getStreamThroughputOutboundMegabitsPerSec()) * BYTES_PER_MEGABIT;
+            double throughput = DatabaseDescriptor.getStreamThroughputOutboundMegabitsPerSec() * BYTES_PER_MEGABIT;
             mayUpdateThroughput(throughput, limiter);
 
-            double interDCThroughput = ((double) DatabaseDescriptor.getInterDCStreamThroughputOutboundMegabitsPerSec()) * BYTES_PER_MEGABIT;
+            double interDCThroughput = DatabaseDescriptor.getInterDCStreamThroughputOutboundMegabitsPerSec() * BYTES_PER_MEGABIT;
             mayUpdateThroughput(interDCThroughput, interDCLimiter);
 
             if (DatabaseDescriptor.getLocalDataCenter() != null && DatabaseDescriptor.getEndpointSnitch() != null)
diff --git a/src/java/org/apache/cassandra/streaming/StreamPlan.java b/src/java/org/apache/cassandra/streaming/StreamPlan.java
index 1bb0ce5..0d963ed 100644
--- a/src/java/org/apache/cassandra/streaming/StreamPlan.java
+++ b/src/java/org/apache/cassandra/streaming/StreamPlan.java
@@ -38,8 +38,6 @@
     private final long repairedAt;
     private final StreamCoordinator coordinator;
 
-    private StreamConnectionFactory connectionFactory = new DefaultConnectionFactory();
-
     private boolean flushBeforeTransfer = true;
 
     /**
@@ -49,14 +47,19 @@
      */
     public StreamPlan(String description)
     {
-        this(description, ActiveRepairService.UNREPAIRED_SSTABLE, 1);
+        this(description, ActiveRepairService.UNREPAIRED_SSTABLE, 1, false, false);
     }
 
-    public StreamPlan(String description, long repairedAt, int connectionsPerHost)
+    public StreamPlan(String description, boolean keepSSTableLevels)
+    {
+        this(description, ActiveRepairService.UNREPAIRED_SSTABLE, 1, keepSSTableLevels, false);
+    }
+
+    public StreamPlan(String description, long repairedAt, int connectionsPerHost, boolean keepSSTableLevels, boolean isIncremental)
     {
         this.description = description;
         this.repairedAt = repairedAt;
-        this.coordinator = new StreamCoordinator(connectionsPerHost, connectionFactory);
+        this.coordinator = new StreamCoordinator(connectionsPerHost, keepSSTableLevels, isIncremental, new DefaultConnectionFactory());
     }
 
     /**
diff --git a/src/java/org/apache/cassandra/streaming/StreamReader.java b/src/java/org/apache/cassandra/streaming/StreamReader.java
index 1e3ba7f..c96ea22 100644
--- a/src/java/org/apache/cassandra/streaming/StreamReader.java
+++ b/src/java/org/apache/cassandra/streaming/StreamReader.java
@@ -17,16 +17,17 @@
  */
 package org.apache.cassandra.streaming;
 
-import java.io.DataInput;
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.io.InputStream;
+import java.io.*;
 import java.nio.channels.Channels;
 import java.nio.channels.ReadableByteChannel;
 import java.util.Collection;
 import java.util.UUID;
 
 import com.google.common.base.Throwables;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.io.sstable.format.Version;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -37,15 +38,15 @@
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.sstable.SSTableWriter;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.streaming.messages.FileMessageHeader;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.BytesReadTracker;
 import org.apache.cassandra.utils.Pair;
 
+import static org.apache.cassandra.utils.Throwables.extractIOExceptionCause;
+
 /**
  * StreamReader reads from stream and writes to SSTable.
  */
@@ -56,8 +57,10 @@
     protected final long estimatedKeys;
     protected final Collection<Pair<Long, Long>> sections;
     protected final StreamSession session;
-    protected final Descriptor.Version inputVersion;
+    protected final Version inputVersion;
     protected final long repairedAt;
+    protected final SSTableFormat.Type format;
+    protected final int sstableLevel;
     protected final int fileSeqNum;
 
     protected Descriptor desc;
@@ -68,8 +71,10 @@
         this.cfId = header.cfId;
         this.estimatedKeys = header.estimatedKeys;
         this.sections = header.sections;
-        this.inputVersion = new Descriptor.Version(header.version);
+        this.inputVersion = header.format.info.getVersion(header.version);
         this.repairedAt = header.repairedAt;
+        this.format = header.format;
+        this.sstableLevel = header.sstableLevel;
         this.fileSeqNum = header.sequenceNumber;
     }
 
@@ -78,6 +83,7 @@
      * @return SSTable transferred
      * @throws IOException if reading the remote sstable fails. Will throw an RTE if local write fails.
      */
+    @SuppressWarnings("resource")
     public SSTableWriter read(ReadableByteChannel channel) throws IOException
     {
         long totalSize = totalSize();
@@ -103,19 +109,19 @@
         DecoratedKey key = null;
         try
         {
-            writer = createWriter(cfs, totalSize, repairedAt);
+            writer = createWriter(cfs, totalSize, repairedAt, format);
             while (in.getBytesRead() < totalSize)
             {
                 key = StorageService.getPartitioner().decorateKey(ByteBufferUtil.readWithShortLength(in));
                 writeRow(key, writer, in, cfs);
+
                 // TODO move this to BytesReadTracker
                 session.progress(desc, ProgressInfo.Direction.IN, in.getBytesRead(), totalSize);
             }
             logger.debug("[Stream #{}] Finished receiving file #{} from {} readBytes = {}, totalSize = {}",
                          session.planId(), fileSeqNum, session.peer, in.getBytesRead(), totalSize);
             return writer;
-        }
-        catch (Throwable e)
+        } catch (Throwable e)
         {
             if (key != null)
                 logger.warn("[Stream {}] Error while reading partition {} from stream on ks='{}' and table='{}'.",
@@ -132,41 +138,18 @@
                     e.addSuppressed(e2);
                 }
             }
-            drain(dis, in.getBytesRead());
-            if (e instanceof IOException)
-                throw (IOException) e;
-            else
-                throw Throwables.propagate(e);
+            throw Throwables.propagate(e);
         }
     }
 
-    protected SSTableWriter createWriter(ColumnFamilyStore cfs, long totalSize, long repairedAt) throws IOException
+    protected SSTableWriter createWriter(ColumnFamilyStore cfs, long totalSize, long repairedAt, SSTableFormat.Type format) throws IOException
     {
         Directories.DataDirectory localDir = cfs.directories.getWriteableLocation(totalSize);
         if (localDir == null)
             throw new IOException("Insufficient disk space to store " + totalSize + " bytes");
-        desc = Descriptor.fromFilename(cfs.getTempSSTablePath(cfs.directories.getLocationForDisk(localDir)));
+        desc = Descriptor.fromFilename(cfs.getTempSSTablePath(cfs.directories.getLocationForDisk(localDir), format));
 
-        return new SSTableWriter(desc.filenameFor(Component.DATA), estimatedKeys, repairedAt);
-    }
-
-    protected void drain(InputStream dis, long bytesRead) throws IOException
-    {
-        long toSkip = totalSize() - bytesRead;
-
-        // InputStream.skip can return -1 if dis is inaccessible.
-        long skipped = dis.skip(toSkip);
-        if (skipped == -1)
-            return;
-
-        toSkip = toSkip - skipped;
-        while (toSkip > 0)
-        {
-            skipped = dis.skip(toSkip);
-            if (skipped == -1)
-                break;
-            toSkip = toSkip - skipped;
-        }
+        return SSTableWriter.create(desc, estimatedKeys, repairedAt, sstableLevel);
     }
 
     protected long totalSize()
diff --git a/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java b/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
index 8773cab..b342edc 100644
--- a/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
+++ b/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
@@ -18,28 +18,25 @@
 package org.apache.cassandra.streaming;
 
 import java.io.File;
-import java.io.IOError;
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 import java.util.Set;
 import java.util.UUID;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor;
+import org.apache.cassandra.concurrent.NamedThreadFactory;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.dht.Bounds;
 import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.io.sstable.SSTableReader;
-import org.apache.cassandra.io.sstable.SSTableWriter;
-import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 import org.apache.cassandra.utils.Pair;
 
@@ -50,10 +47,7 @@
  */
 public class StreamReceiveTask extends StreamTask
 {
-    private static final ThreadPoolExecutor executor = DebuggableThreadPoolExecutor.createWithMaximumPoolSize("StreamReceiveTask",
-                                                                                                              FBUtilities.getAvailableProcessors(),
-                                                                                                              60, TimeUnit.SECONDS);
-
+    private static final ExecutorService executor = Executors.newCachedThreadPool(new NamedThreadFactory("StreamReceiveTask"));
     private static final Logger logger = LoggerFactory.getLogger(StreamReceiveTask.class);
 
     // number of files to receive
@@ -126,19 +120,16 @@
                     for (SSTableWriter writer : task.sstables)
                         writer.abort();
                     task.sstables.clear();
-                    task.session.taskCompleted(task);
                     return;
                 }
                 ColumnFamilyStore cfs = Keyspace.open(kscf.left).getColumnFamilyStore(kscf.right);
 
                 File lockfiledir = cfs.directories.getWriteableLocationAsFile(task.sstables.size() * 256L);
-                if (lockfiledir == null)
-                    throw new IOError(new IOException("All disks full"));
                 StreamLockfile lockfile = new StreamLockfile(lockfiledir, UUID.randomUUID());
                 lockfile.create(task.sstables);
                 List<SSTableReader> readers = new ArrayList<>();
                 for (SSTableWriter writer : task.sstables)
-                    readers.add(writer.closeAndOpenReader());
+                    readers.add(writer.finish(true));
                 lockfile.delete();
                 task.sstables.clear();
 
diff --git a/src/java/org/apache/cassandra/streaming/StreamRequest.java b/src/java/org/apache/cassandra/streaming/StreamRequest.java
index 9c5b974..0fe40cf 100644
--- a/src/java/org/apache/cassandra/streaming/StreamRequest.java
+++ b/src/java/org/apache/cassandra/streaming/StreamRequest.java
@@ -29,6 +29,7 @@
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.net.MessagingService;
 
 public class StreamRequest
 {
@@ -55,8 +56,9 @@
             out.writeInt(request.ranges.size());
             for (Range<Token> range : request.ranges)
             {
-                Token.serializer.serialize(range.left, out);
-                Token.serializer.serialize(range.right, out);
+                MessagingService.validatePartitioner(range);
+                Token.serializer.serialize(range.left, out, version);
+                Token.serializer.serialize(range.right, out, version);
             }
             out.writeInt(request.columnFamilies.size());
             for (String cf : request.columnFamilies)
@@ -71,8 +73,8 @@
             List<Range<Token>> ranges = new ArrayList<>(rangeCount);
             for (int i = 0; i < rangeCount; i++)
             {
-                Token left = Token.serializer.deserialize(in);
-                Token right = Token.serializer.deserialize(in);
+                Token left = Token.serializer.deserialize(in, MessagingService.globalPartitioner(), version);
+                Token right = Token.serializer.deserialize(in, MessagingService.globalPartitioner(), version);
                 ranges.add(new Range<>(left, right));
             }
             int cfCount = in.readInt();
@@ -89,8 +91,8 @@
             size += TypeSizes.NATIVE.sizeof(request.ranges.size());
             for (Range<Token> range : request.ranges)
             {
-                size += Token.serializer.serializedSize(range.left, TypeSizes.NATIVE);
-                size += Token.serializer.serializedSize(range.right, TypeSizes.NATIVE);
+                size += Token.serializer.serializedSize(range.left, version);
+                size += Token.serializer.serializedSize(range.right, version);
             }
             size += TypeSizes.NATIVE.sizeof(request.columnFamilies.size());
             for (String cf : request.columnFamilies)
diff --git a/src/java/org/apache/cassandra/streaming/StreamResultFuture.java b/src/java/org/apache/cassandra/streaming/StreamResultFuture.java
index 5c9c6de..b299b87 100644
--- a/src/java/org/apache/cassandra/streaming/StreamResultFuture.java
+++ b/src/java/org/apache/cassandra/streaming/StreamResultFuture.java
@@ -70,9 +70,9 @@
             set(getCurrentState());
     }
 
-    private StreamResultFuture(UUID planId, String description)
+    private StreamResultFuture(UUID planId, String description, boolean keepSSTableLevels, boolean isIncremental)
     {
-        this(planId, description, new StreamCoordinator(0, new DefaultConnectionFactory()));
+        this(planId, description, new StreamCoordinator(0, keepSSTableLevels, isIncremental, new DefaultConnectionFactory()));
     }
 
     static StreamResultFuture init(UUID planId, String description, Collection<StreamEventHandler> listeners, StreamCoordinator coordinator)
@@ -102,7 +102,9 @@
                                                                     InetAddress from,
                                                                     IncomingStreamingConnection connection,
                                                                     boolean isForOutgoing,
-                                                                    int version) throws IOException
+                                                                    int version,
+                                                                    boolean keepSSTableLevel,
+                                                                    boolean isIncremental) throws IOException
     {
         StreamResultFuture future = StreamManager.instance.getReceivingStream(planId);
         if (future == null)
@@ -110,7 +112,7 @@
             logger.info("[Stream #{} ID#{}] Creating new streaming plan for {}", planId, sessionIndex, description);
 
             // The main reason we create a StreamResultFuture on the receiving side is for JMX exposure.
-            future = new StreamResultFuture(planId, description);
+            future = new StreamResultFuture(planId, description, keepSSTableLevel, isIncremental);
             StreamManager.instance.registerReceiving(future);
         }
         future.attachConnection(from, sessionIndex, connection, isForOutgoing, version);
diff --git a/src/java/org/apache/cassandra/streaming/StreamSession.java b/src/java/org/apache/cassandra/streaming/StreamSession.java
index 273631c..0f43f1f 100644
--- a/src/java/org/apache/cassandra/streaming/StreamSession.java
+++ b/src/java/org/apache/cassandra/streaming/StreamSession.java
@@ -25,35 +25,33 @@
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.AtomicBoolean;
 
-import javax.annotation.Nullable;
-
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Function;
 import com.google.common.collect.*;
+
+import org.apache.cassandra.db.lifecycle.SSTableIntervalTree;
+import org.apache.cassandra.db.lifecycle.View;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.DataTracker;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.RowPosition;
 import org.apache.cassandra.dht.AbstractBounds;
 import org.apache.cassandra.dht.Bounds;
-import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.gms.*;
 import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.metrics.StreamingMetrics;
 import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.streaming.messages.*;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 import org.apache.cassandra.utils.Pair;
-
 import org.apache.cassandra.utils.concurrent.Ref;
 import org.apache.cassandra.utils.concurrent.Refs;
 
@@ -107,10 +105,8 @@
  *       complete (received()). When all files for the StreamReceiveTask have been received, the sstables
  *       are added to the CFS (and 2ndary index are built, StreamReceiveTask.complete()) and the task
  *       is marked complete (taskCompleted())
- *   (b) If during the streaming of a particular file an I/O error occurs on the receiving end of a stream
- *       (FileMessage.deserialize), the node will retry the file (up to DatabaseDescriptor.getMaxStreamingRetries())
- *       by sending a RetryMessage to the sender. On receiving a RetryMessage, the sender simply issue a new
- *       FileMessage for that file.
+ *   (b) If during the streaming of a particular file an error occurs on the receiving end of a stream
+ *       (FileMessage.deserialize), the node will send a SessionFailedMessage to the sender and close the stream session.
  *   (c) When all transfer and receive tasks for a session are complete, the move to the Completion phase
  *       (maybeCompleted()).
  *
@@ -139,9 +135,10 @@
     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 ConcurrentHashMap<UUID, StreamTransferTask> transfers = new ConcurrentHashMap<>();
+    @VisibleForTesting
+    protected 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;
@@ -150,9 +147,9 @@
 
     public final ConnectionHandler handler;
 
-    private int retries;
-
     private AtomicBoolean isAborted = new AtomicBoolean(false);
+    private final boolean keepSSTableLevel;
+    private final boolean isIncremental;
 
     public static enum State
     {
@@ -174,7 +171,7 @@
      * @param connecting Actual connecting address
      * @param factory is used for establishing connection
      */
-    public StreamSession(InetAddress peer, InetAddress connecting, StreamConnectionFactory factory, int index)
+    public StreamSession(InetAddress peer, InetAddress connecting, StreamConnectionFactory factory, int index, boolean keepSSTableLevel, boolean isIncremental)
     {
         this.peer = peer;
         this.connecting = connecting;
@@ -182,6 +179,8 @@
         this.factory = factory;
         this.handler = new ConnectionHandler(this);
         this.metrics = StreamingMetrics.get(connecting);
+        this.keepSSTableLevel = keepSSTableLevel;
+        this.isIncremental = isIncremental;
     }
 
     public UUID planId()
@@ -199,6 +198,17 @@
         return streamResult == null ? null : streamResult.description;
     }
 
+    public boolean keepSSTableLevel()
+    {
+        return keepSSTableLevel;
+    }
+
+    public boolean isIncremental()
+    {
+        return isIncremental;
+    }
+
+
     /**
      * Bind this session to report to specific {@link StreamResultFuture} and
      * perform pre-streaming initialization.
@@ -270,7 +280,7 @@
             flushSSTables(stores);
 
         List<Range<Token>> normalizedRanges = Range.normalize(ranges);
-        List<SSTableStreamingSections> sections = getSSTableSectionsForRanges(normalizedRanges, stores, repairedAt, repairedAt != ActiveRepairService.UNREPAIRED_SSTABLE);
+        List<SSTableStreamingSections> sections = getSSTableSectionsForRanges(normalizedRanges, stores, repairedAt, isIncremental);
         try
         {
             addTransferFiles(sections);
@@ -306,24 +316,31 @@
         {
             for (ColumnFamilyStore cfStore : stores)
             {
-                final List<AbstractBounds<RowPosition>> rowBoundsList = new ArrayList<>(ranges.size());
-                final IPartitioner partitioner = cfStore.partitioner;
+                final List<Range<RowPosition>> keyRanges = new ArrayList<>(ranges.size());
                 for (Range<Token> range : ranges)
-                    rowBoundsList.add(range.toRowBounds());
-                refs.addAll(cfStore.selectAndReference(new Function<DataTracker.View, List<SSTableReader>>()
+                    keyRanges.add(Range.makeRowRange(range));
+                refs.addAll(cfStore.selectAndReference(new Function<View, List<SSTableReader>>()
                 {
-                    public List<SSTableReader> apply(DataTracker.View view)
+                    public List<SSTableReader> apply(View view)
                     {
-                        DataTracker.SSTableIntervalTree intervalTree = DataTracker.buildIntervalTree(ColumnFamilyStore.CANONICAL_SSTABLES.apply(view));
+                        SSTableIntervalTree intervalTree = SSTableIntervalTree.build(ColumnFamilyStore.CANONICAL_SSTABLES.apply(view));
                         Set<SSTableReader> sstables = Sets.newHashSet();
-                        for (AbstractBounds<RowPosition> rowBounds : rowBoundsList)
+                        for (Range<RowPosition> keyRange : keyRanges)
                         {
-                            for (SSTableReader sstable : DataTracker.View.sstablesInBounds(rowBounds, intervalTree, partitioner))
+                            // keyRange excludes its start, while sstableInBounds is inclusive (of both start and end).
+                            // This is fine however, because keyRange has been created from a token range through Range.makeRowRange (see above).
+                            // And that later method uses the Token.maxKeyBound() method to creates the range, which return a "fake" key that
+                            // sort after all keys having the token. That "fake" key cannot however be equal to any real key, so that even
+                            // including keyRange.left will still exclude any key having the token of the original token range, and so we're
+                            // still actually selecting what we wanted.
+                            for (SSTableReader sstable : View.sstablesInBounds(keyRange.left, keyRange.right, intervalTree))
                             {
                                 if (!isIncremental || !sstable.isRepaired())
                                     sstables.add(sstable);
                             }
                         }
+
+                        logger.debug("ViewFilter for {}/{} sstables", sstables.size(), view.sstables.size());
                         return ImmutableList.copyOf(sstables);
                     }
                 }).refs);
@@ -460,11 +477,6 @@
                 received(received.cfId, received.sequenceNumber);
                 break;
 
-            case RETRY:
-                RetryMessage retry = (RetryMessage) message;
-                retry(retry.cfId, retry.sequenceNumber);
-                break;
-
             case COMPLETE:
                 complete();
                 break;
@@ -589,18 +601,6 @@
     }
 
     /**
-     * Call back on receiving {@code StreamMessage.Type.RETRY} message.
-     *
-     * @param cfId ColumnFamily ID
-     * @param sequenceNumber Sequence number to indicate which file to stream again
-     */
-    public void retry(UUID cfId, int sequenceNumber)
-    {
-        OutgoingFileMessage message = transfers.get(cfId).createMessageForRetry(sequenceNumber);
-        handler.sendMessage(message);
-    }
-
-    /**
      * Check if session is completed on receiving {@code StreamMessage.Type.COMPLETE} message.
      */
     public synchronized void complete()
@@ -630,17 +630,6 @@
         closeSession(State.FAILED);
     }
 
-    public void doRetry(FileMessageHeader header, Throwable e)
-    {
-        logger.warn("[Stream #{}] Retrying for following error", planId(), e);
-        // retry
-        retries++;
-        if (retries > DatabaseDescriptor.getMaxStreamingRetries())
-            onError(new IOException("Too many retries for " + header, e));
-        else
-            handler.sendMessage(new RetryMessage(header.cfId, header.sequenceNumber));
-    }
-
     /**
      * @return Current snapshot of this session info.
      */
diff --git a/src/java/org/apache/cassandra/streaming/StreamTransferTask.java b/src/java/org/apache/cassandra/streaming/StreamTransferTask.java
index 1727bae..c1c5055 100644
--- a/src/java/org/apache/cassandra/streaming/StreamTransferTask.java
+++ b/src/java/org/apache/cassandra/streaming/StreamTransferTask.java
@@ -22,11 +22,12 @@
 import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Throwables;
 import com.google.common.collect.Iterables;
 
 import org.apache.cassandra.concurrent.NamedThreadFactory;
-import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.streaming.messages.OutgoingFileMessage;
 import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.concurrent.Ref;
@@ -42,7 +43,8 @@
     private final AtomicInteger sequenceNumber = new AtomicInteger(0);
     private boolean aborted = false;
 
-    private final Map<Integer, OutgoingFileMessage> files = new HashMap<>();
+    @VisibleForTesting
+    protected final Map<Integer, OutgoingFileMessage> files = new HashMap<>();
     private final Map<Integer, ScheduledFuture> timeoutTasks = new HashMap<>();
 
     private long totalSize;
@@ -55,7 +57,7 @@
     public synchronized void addTransferFile(Ref<SSTableReader> ref, long estimatedKeys, List<Pair<Long, Long>> sections, long repairedAt)
     {
         assert ref.get() != null && cfId.equals(ref.get().metadata.cfId);
-        OutgoingFileMessage message = new OutgoingFileMessage(ref, sequenceNumber.getAndIncrement(), estimatedKeys, sections, repairedAt);
+        OutgoingFileMessage message = new OutgoingFileMessage(ref, sequenceNumber.getAndIncrement(), estimatedKeys, sections, repairedAt, session.keepSSTableLevel());
         files.put(message.header.sequenceNumber, message);
         totalSize += message.header.size();
     }
diff --git a/src/java/org/apache/cassandra/streaming/StreamWriter.java b/src/java/org/apache/cassandra/streaming/StreamWriter.java
index 2579414..721ae1e 100644
--- a/src/java/org/apache/cassandra/streaming/StreamWriter.java
+++ b/src/java/org/apache/cassandra/streaming/StreamWriter.java
@@ -20,8 +20,6 @@
 import java.io.File;
 import java.io.IOException;
 import java.io.OutputStream;
-import java.nio.channels.Channels;
-import java.nio.channels.WritableByteChannel;
 import java.util.Collection;
 
 import org.slf4j.Logger;
@@ -30,9 +28,10 @@
 import com.ning.compress.lzf.LZFOutputStream;
 
 import org.apache.cassandra.io.sstable.Component;
-import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.util.DataIntegrityMetadata;
 import org.apache.cassandra.io.util.DataIntegrityMetadata.ChecksumValidator;
+import org.apache.cassandra.io.util.DataOutputStreamPlus;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.io.util.RandomAccessReader;
 import org.apache.cassandra.streaming.StreamManager.StreamRateLimiter;
@@ -70,26 +69,26 @@
      *
      * StreamWriter uses LZF compression on wire to decrease size to transfer.
      *
-     * @param channel where this writes data to
+     * @param output where this writes data to
      * @throws IOException on any I/O error
      */
-    public void write(WritableByteChannel channel) throws IOException
+    public void write(DataOutputStreamPlus output) throws IOException
     {
         long totalSize = totalSize();
         logger.debug("[Stream #{}] Start streaming file {} to {}, repairedAt = {}, totalSize = {}", session.planId(),
                      sstable.getFilename(), session.peer, sstable.getSSTableMetadata().repairedAt, totalSize);
-        RandomAccessReader file = sstable.openDataReader();
-        ChecksumValidator validator = new File(sstable.descriptor.filenameFor(Component.CRC)).exists()
-                                    ? DataIntegrityMetadata.checksumValidator(sstable.descriptor)
-                                    : null;
-        transferBuffer = validator == null ? new byte[DEFAULT_CHUNK_SIZE] : new byte[validator.chunkSize];
 
-        // setting up data compression stream
-        compressedOutput = new LZFOutputStream(Channels.newOutputStream(channel));
-        long progress = 0L;
-
-        try
+        try(RandomAccessReader file = sstable.openDataReader();
+            ChecksumValidator validator = new File(sstable.descriptor.filenameFor(Component.CRC)).exists()
+                                          ? DataIntegrityMetadata.checksumValidator(sstable.descriptor)
+                                          : null;)
         {
+            transferBuffer = validator == null ? new byte[DEFAULT_CHUNK_SIZE] : new byte[validator.chunkSize];
+
+            // setting up data compression stream
+            compressedOutput = new LZFOutputStream(output);
+            long progress = 0L;
+
             // stream each of the required sections of the file
             for (Pair<Long, Long> section : sections)
             {
@@ -113,18 +112,12 @@
                     readOffset = 0;
                 }
 
-                // make sure that current section is send
+                // make sure that current section is sent
                 compressedOutput.flush();
             }
             logger.debug("[Stream #{}] Finished streaming file {} to {}, bytesTransferred = {}, totalSize = {}",
                          session.planId(), sstable.getFilename(), session.peer, progress, totalSize);
         }
-        finally
-        {
-            // no matter what happens close file
-            FileUtils.closeQuietly(file);
-            FileUtils.closeQuietly(validator);
-        }
     }
 
     protected long totalSize()
diff --git a/src/java/org/apache/cassandra/streaming/compress/CompressedInputStream.java b/src/java/org/apache/cassandra/streaming/compress/CompressedInputStream.java
index 6280ccd..d08ffa9 100644
--- a/src/java/org/apache/cassandra/streaming/compress/CompressedInputStream.java
+++ b/src/java/org/apache/cassandra/streaming/compress/CompressedInputStream.java
@@ -25,7 +25,6 @@
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.zip.Adler32;
-import java.util.zip.CRC32;
 import java.util.zip.Checksum;
 
 import com.google.common.collect.Iterators;
@@ -35,7 +34,6 @@
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.io.compress.CompressionMetadata;
-import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.WrappedRunnable;
 
 /**
@@ -65,20 +63,27 @@
     // raw checksum bytes
     private final byte[] checksumBytes = new byte[4];
 
+    /**
+     * Indicates there was a problem when reading from source stream.
+     * When this is added to the <code>dataBuffer</code> by the stream Reader,
+     * it is expected that the <code>readException</code> variable is populated
+     * with the cause of the error when reading from source stream, so it is
+     * thrown to the consumer on subsequent read operation.
+     */
     private static final byte[] POISON_PILL = new byte[0];
 
+    protected volatile IOException readException = null;
+
     private long totalCompressedBytesRead;
-    private final boolean hasPostCompressionAdlerChecksums;
 
     /**
      * @param source Input source to read compressed data from
      * @param info Compression info
      */
-    public CompressedInputStream(InputStream source, CompressionInfo info, boolean hasPostCompressionAdlerChecksums)
+    public CompressedInputStream(InputStream source, CompressionInfo info)
     {
         this.info = info;
-        this.checksum = hasPostCompressionAdlerChecksums ? new Adler32() : new CRC32();
-        this.hasPostCompressionAdlerChecksums = hasPostCompressionAdlerChecksums;
+        this.checksum =  new Adler32();
         this.buffer = new byte[info.parameters.chunkLength()];
         // buffer is limited to store up to 1024 chunks
         this.dataBuffer = new ArrayBlockingQueue<byte[]>(Math.min(info.chunks.length, 1024));
@@ -88,13 +93,19 @@
 
     public int read() throws IOException
     {
+        if (readException != null)
+            throw readException;
+
         if (current >= bufferOffset + buffer.length || validBufferBytes == -1)
         {
             try
             {
                 byte[] compressedWithCRC = dataBuffer.take();
                 if (compressedWithCRC == POISON_PILL)
-                    throw new EOFException("No chunk available");
+                {
+                    assert readException != null;
+                    throw readException;
+                }
                 decompress(compressedWithCRC);
             }
             catch (InterruptedException e)
@@ -123,14 +134,7 @@
         // validate crc randomly
         if (info.parameters.getCrcCheckChance() > ThreadLocalRandom.current().nextDouble())
         {
-            if (hasPostCompressionAdlerChecksums)
-            {
-                checksum.update(compressed, 0, compressed.length - checksumBytes.length);
-            }
-            else
-            {
-                checksum.update(buffer, 0, validBufferBytes);
-            }
+            checksum.update(compressed, 0, compressed.length - checksumBytes.length);
 
             System.arraycopy(compressed, compressed.length - checksumBytes.length, checksumBytes, 0, checksumBytes.length);
             if (Ints.fromByteArray(checksumBytes) != (int) checksum.getValue())
@@ -149,7 +153,7 @@
         return totalCompressedBytesRead;
     }
 
-    static class Reader extends WrappedRunnable
+    class Reader extends WrappedRunnable
     {
         private final InputStream source;
         private final Iterator<CompressionMetadata.Chunk> chunks;
@@ -180,6 +184,7 @@
                         int r = source.read(compressedWithCRC, bufferRead, readLength - bufferRead);
                         if (r < 0)
                         {
+                            readException = new EOFException("No chunk available");
                             dataBuffer.put(POISON_PILL);
                             return; // throw exception where we consume dataBuffer
                         }
@@ -188,6 +193,7 @@
                     catch (IOException e)
                     {
                         logger.warn("Error while reading compressed input stream.", e);
+                        readException = e;
                         dataBuffer.put(POISON_PILL);
                         return; // throw exception where we consume dataBuffer
                     }
diff --git a/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java b/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
index fd0d9c8..fa1022d 100644
--- a/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
+++ b/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
@@ -18,6 +18,7 @@
 package org.apache.cassandra.streaming.compress;
 
 import java.io.DataInputStream;
+
 import java.io.IOException;
 import java.nio.channels.Channels;
 import java.nio.channels.ReadableByteChannel;
@@ -25,6 +26,8 @@
 import com.google.common.base.Throwables;
 
 import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -32,7 +35,6 @@
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.io.compress.CompressionMetadata;
-import org.apache.cassandra.io.sstable.SSTableWriter;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.streaming.ProgressInfo;
 import org.apache.cassandra.streaming.StreamReader;
@@ -42,6 +44,8 @@
 import org.apache.cassandra.utils.BytesReadTracker;
 import org.apache.cassandra.utils.Pair;
 
+import static org.apache.cassandra.utils.Throwables.extractIOExceptionCause;
+
 /**
  * StreamReader that reads from streamed compressed SSTable
  */
@@ -62,6 +66,7 @@
      * @throws java.io.IOException if reading the remote sstable fails. Will throw an RTE if local write fails.
      */
     @Override
+    @SuppressWarnings("resource")
     public SSTableWriter read(ReadableByteChannel channel) throws IOException
     {
         long totalSize = totalSize();
@@ -81,22 +86,25 @@
                      session.planId(), fileSeqNum, session.peer, repairedAt, totalSize, cfs.keyspace.getName(),
                      cfs.getColumnFamilyName());
 
-        CompressedInputStream cis = new CompressedInputStream(Channels.newInputStream(channel), compressionInfo, inputVersion.hasPostCompressionAdlerChecksums);
+        CompressedInputStream cis = new CompressedInputStream(Channels.newInputStream(channel), compressionInfo);
         BytesReadTracker in = new BytesReadTracker(new DataInputStream(cis));
         SSTableWriter writer = null;
         DecoratedKey key = null;
         try
         {
-            writer = createWriter(cfs, totalSize, repairedAt);
+            writer = createWriter(cfs, totalSize, repairedAt, format);
             int sectionIdx = 0;
             for (Pair<Long, Long> section : sections)
             {
-                long length = section.right - section.left;
+                assert cis.getTotalCompressedBytesRead() <= totalSize;
+                long sectionLength = section.right - section.left;
+
+                logger.trace("[Stream #{}] Reading section {} with length {} from stream.", session.planId(), sectionIdx++, sectionLength);
                 // skip to beginning of section inside chunk
                 cis.position(section.left);
                 in.reset(0);
-                logger.trace("[Stream #{}] Reading section {} with length {} from stream.", session.planId(), sectionIdx++, length);
-                while (in.getBytesRead() < length)
+
+                while (in.getBytesRead() < sectionLength)
                 {
                     key = StorageService.getPartitioner().decorateKey(ByteBufferUtil.readWithShortLength(in));
                     writeRow(key, writer, in, cfs);
@@ -126,11 +134,9 @@
                     e.addSuppressed(e2);
                 }
             }
-            drain(cis, in.getBytesRead());
-            if (e instanceof IOException)
-                throw (IOException) e;
-            else
-                throw Throwables.propagate(e);
+            if (extractIOExceptionCause(e).isPresent())
+                throw e;
+            throw Throwables.propagate(e);
         }
     }
 
diff --git a/src/java/org/apache/cassandra/streaming/compress/CompressedStreamWriter.java b/src/java/org/apache/cassandra/streaming/compress/CompressedStreamWriter.java
index 6fe08e6..99e9bd6 100644
--- a/src/java/org/apache/cassandra/streaming/compress/CompressedStreamWriter.java
+++ b/src/java/org/apache/cassandra/streaming/compress/CompressedStreamWriter.java
@@ -18,18 +18,20 @@
 package org.apache.cassandra.streaming.compress;
 
 import java.io.IOException;
-import java.nio.channels.FileChannel;
 import java.nio.channels.WritableByteChannel;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 
+import com.google.common.base.Function;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.io.compress.CompressionMetadata;
-import org.apache.cassandra.io.sstable.SSTableReader;
-import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.ChannelProxy;
+import org.apache.cassandra.io.util.DataOutputStreamPlus;
 import org.apache.cassandra.io.util.RandomAccessReader;
 import org.apache.cassandra.streaming.ProgressInfo;
 import org.apache.cassandra.streaming.StreamSession;
@@ -54,34 +56,41 @@
     }
 
     @Override
-    public void write(WritableByteChannel channel) throws IOException
+    public void write(DataOutputStreamPlus out) throws IOException
     {
         long totalSize = totalSize();
         logger.debug("[Stream #{}] Start streaming file {} to {}, repairedAt = {}, totalSize = {}", session.planId(),
                      sstable.getFilename(), session.peer, sstable.getSSTableMetadata().repairedAt, totalSize);
-        RandomAccessReader file = sstable.openDataReader();
-        FileChannel fc = file.getChannel();
-
-        long progress = 0L;
-        // calculate chunks to transfer. we want to send continuous chunks altogether.
-        List<Pair<Long, Long>> sections = getTransferSections(compressionInfo.chunks);
-        int sectionIdx = 0;
-        try
+        try (RandomAccessReader file = sstable.openDataReader(); final ChannelProxy fc = file.getChannel())
         {
+            long progress = 0L;
+            // calculate chunks to transfer. we want to send continuous chunks altogether.
+            List<Pair<Long, Long>> sections = getTransferSections(compressionInfo.chunks);
+
+            int sectionIdx = 0;
+
             // stream each of the required sections of the file
-            for (Pair<Long, Long> section : sections)
+            for (final Pair<Long, Long> section : sections)
             {
                 // length of the section to stream
                 long length = section.right - section.left;
+
                 logger.trace("[Stream #{}] Writing section {} with length {} to stream.", session.planId(), sectionIdx++, length);
 
                 // tracks write progress
                 long bytesTransferred = 0;
                 while (bytesTransferred < length)
                 {
-                    int toTransfer = (int) Math.min(CHUNK_SIZE, length - bytesTransferred);
+                    final long bytesTransferredFinal = bytesTransferred;
+                    final int toTransfer = (int) Math.min(CHUNK_SIZE, length - bytesTransferred);
                     limiter.acquire(toTransfer);
-                    long lastWrite = fc.transferTo(section.left + bytesTransferred, toTransfer, channel);
+                    long lastWrite = out.applyToChannel(new Function<WritableByteChannel, Long>()
+                    {
+                        public Long apply(WritableByteChannel wbc)
+                        {
+                            return fc.transferTo(section.left + bytesTransferredFinal, toTransfer, wbc);
+                        }
+                    });
                     bytesTransferred += lastWrite;
                     progress += lastWrite;
                     session.progress(sstable.descriptor, ProgressInfo.Direction.OUT, progress, totalSize);
@@ -90,11 +99,6 @@
             logger.debug("[Stream #{}] Finished streaming file {} to {}, bytesTransferred = {}, totalSize = {}",
                          session.planId(), sstable.getFilename(), session.peer, progress, totalSize);
         }
-        finally
-        {
-            // no matter what happens close file
-            FileUtils.closeQuietly(file);
-        }
     }
 
     @Override
diff --git a/src/java/org/apache/cassandra/streaming/compress/CompressionInfo.java b/src/java/org/apache/cassandra/streaming/compress/CompressionInfo.java
index 8cfcd95..907a1c7 100644
--- a/src/java/org/apache/cassandra/streaming/compress/CompressionInfo.java
+++ b/src/java/org/apache/cassandra/streaming/compress/CompressionInfo.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.streaming.compress;
 
 import java.io.DataInput;
-import java.io.DataOutput;
 import java.io.IOException;
 
 import org.apache.cassandra.db.TypeSizes;
diff --git a/src/java/org/apache/cassandra/streaming/messages/CompleteMessage.java b/src/java/org/apache/cassandra/streaming/messages/CompleteMessage.java
index ec9c66c..b555f64 100644
--- a/src/java/org/apache/cassandra/streaming/messages/CompleteMessage.java
+++ b/src/java/org/apache/cassandra/streaming/messages/CompleteMessage.java
@@ -20,7 +20,7 @@
 import java.io.IOException;
 import java.nio.channels.ReadableByteChannel;
 
-import org.apache.cassandra.io.util.DataOutputStreamAndChannel;
+import org.apache.cassandra.io.util.DataOutputStreamPlus;
 import org.apache.cassandra.streaming.StreamSession;
 
 public class CompleteMessage extends StreamMessage
@@ -32,7 +32,7 @@
             return new CompleteMessage();
         }
 
-        public void serialize(CompleteMessage message, DataOutputStreamAndChannel out, int version, StreamSession session) throws IOException {}
+        public void serialize(CompleteMessage message, DataOutputStreamPlus out, int version, StreamSession session) throws IOException {}
     };
 
     public CompleteMessage()
diff --git a/src/java/org/apache/cassandra/streaming/messages/FileMessageHeader.java b/src/java/org/apache/cassandra/streaming/messages/FileMessageHeader.java
index 34d9a01..b2af699 100644
--- a/src/java/org/apache/cassandra/streaming/messages/FileMessageHeader.java
+++ b/src/java/org/apache/cassandra/streaming/messages/FileMessageHeader.java
@@ -26,6 +26,7 @@
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.compress.CompressionMetadata;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.streaming.compress.CompressionInfo;
@@ -43,6 +44,9 @@
     public final int sequenceNumber;
     /** SSTable version */
     public final String version;
+
+    /** SSTable format **/
+    public final SSTableFormat.Type format;
     public final long estimatedKeys;
     public final List<Pair<Long, Long>> sections;
     /**
@@ -53,41 +57,55 @@
     public final CompressionInfo compressionInfo;
     private final CompressionMetadata compressionMetadata;
     public final long repairedAt;
+    public final int sstableLevel;
+
+    /* cached size value */
+    private transient final long size;
 
     public FileMessageHeader(UUID cfId,
                              int sequenceNumber,
                              String version,
+                             SSTableFormat.Type format,
                              long estimatedKeys,
                              List<Pair<Long, Long>> sections,
                              CompressionInfo compressionInfo,
-                             long repairedAt)
+                             long repairedAt,
+                             int sstableLevel)
     {
         this.cfId = cfId;
         this.sequenceNumber = sequenceNumber;
         this.version = version;
+        this.format = format;
         this.estimatedKeys = estimatedKeys;
         this.sections = sections;
         this.compressionInfo = compressionInfo;
         this.compressionMetadata = null;
         this.repairedAt = repairedAt;
+        this.sstableLevel = sstableLevel;
+        this.size = calculateSize();
     }
 
     public FileMessageHeader(UUID cfId,
                              int sequenceNumber,
                              String version,
+                             SSTableFormat.Type format,
                              long estimatedKeys,
                              List<Pair<Long, Long>> sections,
                              CompressionMetadata compressionMetadata,
-                             long repairedAt)
+                             long repairedAt,
+                             int sstableLevel)
     {
         this.cfId = cfId;
         this.sequenceNumber = sequenceNumber;
         this.version = version;
+        this.format = format;
         this.estimatedKeys = estimatedKeys;
         this.sections = sections;
         this.compressionInfo = null;
         this.compressionMetadata = compressionMetadata;
         this.repairedAt = repairedAt;
+        this.sstableLevel = sstableLevel;
+        this.size = calculateSize();
     }
 
     public boolean isCompressed()
@@ -100,23 +118,28 @@
      */
     public long size()
     {
-        long size = 0;
+        return size;
+    }
+
+    private long calculateSize()
+    {
+        long transferSize = 0;
         if (compressionInfo != null)
         {
             // calculate total length of transferring chunks
             for (CompressionMetadata.Chunk chunk : compressionInfo.chunks)
-                size += chunk.length + 4; // 4 bytes for CRC
+                transferSize += chunk.length + 4; // 4 bytes for CRC
         }
         else if (compressionMetadata != null)
         {
-            size = compressionMetadata.getTotalSizeForSections(sections);
+            transferSize = compressionMetadata.getTotalSizeForSections(sections);
         }
         else
         {
             for (Pair<Long, Long> section : sections)
-                size += section.right - section.left;
+                transferSize += section.right - section.left;
         }
-        return size;
+        return transferSize;
     }
 
     @Override
@@ -126,10 +149,12 @@
         sb.append("cfId: ").append(cfId);
         sb.append(", #").append(sequenceNumber);
         sb.append(", version: ").append(version);
+        sb.append(", format: ").append(format);
         sb.append(", estimated keys: ").append(estimatedKeys);
         sb.append(", transfer size: ").append(size());
         sb.append(", compressed?: ").append(isCompressed());
         sb.append(", repairedAt: ").append(repairedAt);
+        sb.append(", level: ").append(sstableLevel);
         sb.append(')');
         return sb.toString();
     }
@@ -158,8 +183,15 @@
             UUIDSerializer.serializer.serialize(header.cfId, out, version);
             out.writeInt(header.sequenceNumber);
             out.writeUTF(header.version);
-            out.writeLong(header.estimatedKeys);
 
+            //We can't stream to a node that doesn't understand a new sstable format
+            if (version < StreamMessage.VERSION_22 && header.format != SSTableFormat.Type.LEGACY && header.format != SSTableFormat.Type.BIG)
+                throw new UnsupportedOperationException("Can't stream non-legacy sstables to nodes < 2.2");
+
+            if (version >= StreamMessage.VERSION_22)
+                out.writeUTF(header.format.name);
+
+            out.writeLong(header.estimatedKeys);
             out.writeInt(header.sections.size());
             for (Pair<Long, Long> section : header.sections)
             {
@@ -172,6 +204,7 @@
                 compressionInfo = new CompressionInfo(header.compressionMetadata.getChunksForSections(header.sections), header.compressionMetadata.parameters);
             CompressionInfo.serializer.serialize(compressionInfo, out, version);
             out.writeLong(header.repairedAt);
+            out.writeInt(header.sstableLevel);
             return compressionInfo;
         }
 
@@ -180,6 +213,11 @@
             UUID cfId = UUIDSerializer.serializer.deserialize(in, MessagingService.current_version);
             int sequenceNumber = in.readInt();
             String sstableVersion = in.readUTF();
+
+            SSTableFormat.Type format = SSTableFormat.Type.LEGACY;
+            if (version >= StreamMessage.VERSION_22)
+                format = SSTableFormat.Type.validate(in.readUTF());
+
             long estimatedKeys = in.readLong();
             int count = in.readInt();
             List<Pair<Long, Long>> sections = new ArrayList<>(count);
@@ -187,7 +225,8 @@
                 sections.add(Pair.create(in.readLong(), in.readLong()));
             CompressionInfo compressionInfo = CompressionInfo.serializer.deserialize(in, MessagingService.current_version);
             long repairedAt = in.readLong();
-            return new FileMessageHeader(cfId, sequenceNumber, sstableVersion, estimatedKeys, sections, compressionInfo, repairedAt);
+            int sstableLevel = in.readInt();
+            return new FileMessageHeader(cfId, sequenceNumber, sstableVersion, format, estimatedKeys, sections, compressionInfo, repairedAt, sstableLevel);
         }
 
         public long serializedSize(FileMessageHeader header, int version)
@@ -195,6 +234,10 @@
             long size = UUIDSerializer.serializer.serializedSize(header.cfId, version);
             size += TypeSizes.NATIVE.sizeof(header.sequenceNumber);
             size += TypeSizes.NATIVE.sizeof(header.version);
+
+            if (version >= StreamMessage.VERSION_22)
+                size += TypeSizes.NATIVE.sizeof(header.format.name);
+
             size += TypeSizes.NATIVE.sizeof(header.estimatedKeys);
 
             size += TypeSizes.NATIVE.sizeof(header.sections.size());
@@ -204,7 +247,8 @@
                 size += TypeSizes.NATIVE.sizeof(section.right);
             }
             size += CompressionInfo.serializer.serializedSize(header.compressionInfo, version);
+            size += TypeSizes.NATIVE.sizeof(header.sstableLevel);
             return size;
         }
     }
-}
\ No newline at end of file
+}
diff --git a/src/java/org/apache/cassandra/streaming/messages/IncomingFileMessage.java b/src/java/org/apache/cassandra/streaming/messages/IncomingFileMessage.java
index cb39275..2870c03 100644
--- a/src/java/org/apache/cassandra/streaming/messages/IncomingFileMessage.java
+++ b/src/java/org/apache/cassandra/streaming/messages/IncomingFileMessage.java
@@ -22,13 +22,17 @@
 import java.nio.channels.Channels;
 import java.nio.channels.ReadableByteChannel;
 
-import org.apache.cassandra.io.sstable.SSTableWriter;
-import org.apache.cassandra.io.util.DataOutputStreamAndChannel;
+import com.google.common.base.Optional;
+
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.io.util.DataOutputStreamPlus;
 import org.apache.cassandra.streaming.StreamReader;
 import org.apache.cassandra.streaming.StreamSession;
 import org.apache.cassandra.streaming.compress.CompressedStreamReader;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 
+import static org.apache.cassandra.utils.Throwables.extractIOExceptionCause;
+
 /**
  * IncomingFileMessage is used to receive the part(or whole) of a SSTable data file.
  */
@@ -36,6 +40,7 @@
 {
     public static Serializer<IncomingFileMessage> serializer = new Serializer<IncomingFileMessage>()
     {
+        @SuppressWarnings("resource")
         public IncomingFileMessage deserialize(ReadableByteChannel in, int version, StreamSession session) throws IOException
         {
             DataInputStream input = new DataInputStream(Channels.newInputStream(in));
@@ -47,30 +52,14 @@
             {
                 return new IncomingFileMessage(reader.read(in), header);
             }
-            catch (IOException eof)
-            {
-                // Reading from remote failed(i.e. reached EOF before reading expected length of data).
-                // This can be caused by network/node failure thus we are not retrying
-                throw eof;
-            }
             catch (Throwable t)
             {
-                // Throwable can be Runtime error containing IOException.
-                // In that case we don't want to retry.
-                Throwable cause = t;
-                while ((cause = cause.getCause()) != null)
-                {
-                   if (cause instanceof IOException)
-                       throw (IOException) cause;
-                }
                 JVMStabilityInspector.inspectThrowable(t);
-                // Otherwise, we can retry
-                session.doRetry(header, t);
-                return null;
+                throw t;
             }
         }
 
-        public void serialize(IncomingFileMessage message, DataOutputStreamAndChannel out, int version, StreamSession session) throws IOException
+        public void serialize(IncomingFileMessage message, DataOutputStreamPlus out, int version, StreamSession session) throws IOException
         {
             throw new UnsupportedOperationException("Not allowed to call serialize on an incoming file");
         }
diff --git a/src/java/org/apache/cassandra/streaming/messages/OutgoingFileMessage.java b/src/java/org/apache/cassandra/streaming/messages/OutgoingFileMessage.java
index 71902e1..30b79f8 100644
--- a/src/java/org/apache/cassandra/streaming/messages/OutgoingFileMessage.java
+++ b/src/java/org/apache/cassandra/streaming/messages/OutgoingFileMessage.java
@@ -21,15 +21,16 @@
 import java.nio.channels.ReadableByteChannel;
 import java.util.List;
 
+import com.google.common.annotations.VisibleForTesting;
+
 import org.apache.cassandra.io.compress.CompressionMetadata;
-import org.apache.cassandra.io.sstable.SSTableReader;
-import org.apache.cassandra.io.util.DataOutputStreamAndChannel;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.DataOutputStreamPlus;
 import org.apache.cassandra.streaming.StreamSession;
 import org.apache.cassandra.streaming.StreamWriter;
 import org.apache.cassandra.streaming.compress.CompressedStreamWriter;
 import org.apache.cassandra.streaming.compress.CompressionInfo;
 import org.apache.cassandra.utils.Pair;
-
 import org.apache.cassandra.utils.concurrent.Ref;
 
 /**
@@ -44,10 +45,18 @@
             throw new UnsupportedOperationException("Not allowed to call deserialize on an outgoing file");
         }
 
-        public void serialize(OutgoingFileMessage message, DataOutputStreamAndChannel out, int version, StreamSession session) throws IOException
+        public void serialize(OutgoingFileMessage message, DataOutputStreamPlus out, int version, StreamSession session) throws IOException
         {
-            message.serialize(out, version, session);
-            session.fileSent(message.header);
+            message.startTransfer();
+            try
+            {
+                message.serialize(out, version, session);
+                session.fileSent(message.header);
+            }
+            finally
+            {
+                message.finishTransfer();
+            }
         }
     };
 
@@ -55,8 +64,9 @@
     private final Ref<SSTableReader> ref;
     private final String filename;
     private boolean completed = false;
+    private boolean transferring = false;
 
-    public OutgoingFileMessage(Ref<SSTableReader> ref, int sequenceNumber, long estimatedKeys, List<Pair<Long, Long>> sections, long repairedAt)
+    public OutgoingFileMessage(Ref<SSTableReader> ref, int sequenceNumber, long estimatedKeys, List<Pair<Long, Long>> sections, long repairedAt, boolean keepSSTableLevel)
     {
         super(Type.FILE);
         this.ref = ref;
@@ -66,13 +76,15 @@
         this.header = new FileMessageHeader(sstable.metadata.cfId,
                                             sequenceNumber,
                                             sstable.descriptor.version.toString(),
+                                            sstable.descriptor.formatType,
                                             estimatedKeys,
                                             sections,
                                             sstable.compression ? sstable.getCompressionMetadata() : null,
-                                            repairedAt);
+                                            repairedAt,
+                                            keepSSTableLevel ? sstable.getSSTableLevel() : 0);
     }
 
-    public synchronized void serialize(DataOutputStreamAndChannel out, int version, StreamSession session) throws IOException
+    public synchronized void serialize(DataOutputStreamPlus out, int version, StreamSession session) throws IOException
     {
         if (completed)
         {
@@ -84,8 +96,29 @@
         final SSTableReader reader = ref.get();
         StreamWriter writer = compressionInfo == null ?
                                       new StreamWriter(reader, header.sections, session) :
-                                      new CompressedStreamWriter(reader, header.sections, compressionInfo, session);
-        writer.write(out.getChannel());
+                                      new CompressedStreamWriter(reader, header.sections,
+                                                                 compressionInfo, session);
+        writer.write(out);
+    }
+
+    @VisibleForTesting
+    public synchronized void finishTransfer()
+    {
+        transferring = false;
+        //session was aborted mid-transfer, now it's safe to release
+        if (completed)
+        {
+            ref.release();
+        }
+    }
+
+    @VisibleForTesting
+    public synchronized void startTransfer()
+    {
+        if (completed)
+            throw new RuntimeException(String.format("Transfer of file %s already completed or aborted (perhaps session failed?).",
+                                                     filename));
+        transferring = true;
     }
 
     public synchronized void complete()
@@ -93,7 +126,11 @@
         if (!completed)
         {
             completed = true;
-            ref.release();
+            //release only if not transferring
+            if (!transferring)
+            {
+                ref.release();
+            }
         }
     }
 
diff --git a/src/java/org/apache/cassandra/streaming/messages/PrepareMessage.java b/src/java/org/apache/cassandra/streaming/messages/PrepareMessage.java
index 0bc5982..004df18 100644
--- a/src/java/org/apache/cassandra/streaming/messages/PrepareMessage.java
+++ b/src/java/org/apache/cassandra/streaming/messages/PrepareMessage.java
@@ -23,8 +23,7 @@
 import java.util.ArrayList;
 import java.util.Collection;
 
-import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.io.util.DataOutputStreamAndChannel;
+import org.apache.cassandra.io.util.DataOutputStreamPlus;
 import org.apache.cassandra.streaming.StreamRequest;
 import org.apache.cassandra.streaming.StreamSession;
 import org.apache.cassandra.streaming.StreamSummary;
@@ -48,7 +47,7 @@
             return message;
         }
 
-        public void serialize(PrepareMessage message, DataOutputStreamAndChannel out, int version, StreamSession session) throws IOException
+        public void serialize(PrepareMessage message, DataOutputStreamPlus out, int version, StreamSession session) throws IOException
         {
             // requests
             out.writeInt(message.requests.size());
diff --git a/src/java/org/apache/cassandra/streaming/messages/ReceivedMessage.java b/src/java/org/apache/cassandra/streaming/messages/ReceivedMessage.java
index e556651..1255947 100644
--- a/src/java/org/apache/cassandra/streaming/messages/ReceivedMessage.java
+++ b/src/java/org/apache/cassandra/streaming/messages/ReceivedMessage.java
@@ -22,8 +22,7 @@
 import java.nio.channels.ReadableByteChannel;
 import java.util.UUID;
 
-import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.io.util.DataOutputStreamAndChannel;
+import org.apache.cassandra.io.util.DataOutputStreamPlus;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.streaming.StreamSession;
 import org.apache.cassandra.utils.UUIDSerializer;
@@ -38,7 +37,7 @@
             return new ReceivedMessage(UUIDSerializer.serializer.deserialize(input, MessagingService.current_version), input.readInt());
         }
 
-        public void serialize(ReceivedMessage message, DataOutputStreamAndChannel out, int version, StreamSession session) throws IOException
+        public void serialize(ReceivedMessage message, DataOutputStreamPlus out, int version, StreamSession session) throws IOException
         {
             UUIDSerializer.serializer.serialize(message.cfId, out, MessagingService.current_version);
             out.writeInt(message.sequenceNumber);
diff --git a/src/java/org/apache/cassandra/streaming/messages/RetryMessage.java b/src/java/org/apache/cassandra/streaming/messages/RetryMessage.java
index 50b8873..6673aa1 100644
--- a/src/java/org/apache/cassandra/streaming/messages/RetryMessage.java
+++ b/src/java/org/apache/cassandra/streaming/messages/RetryMessage.java
@@ -22,12 +22,15 @@
 import java.nio.channels.ReadableByteChannel;
 import java.util.UUID;
 
-import org.apache.cassandra.io.util.DataOutputPlus;
-import org.apache.cassandra.io.util.DataOutputStreamAndChannel;
+import org.apache.cassandra.io.util.DataOutputStreamPlus;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.streaming.StreamSession;
 import org.apache.cassandra.utils.UUIDSerializer;
 
+/**
+ * @deprecated retry support removed on CASSANDRA-10992
+ */
+@Deprecated
 public class RetryMessage extends StreamMessage
 {
     public static Serializer<RetryMessage> serializer = new Serializer<RetryMessage>()
@@ -38,7 +41,7 @@
             return new RetryMessage(UUIDSerializer.serializer.deserialize(input, MessagingService.current_version), input.readInt());
         }
 
-        public void serialize(RetryMessage message, DataOutputStreamAndChannel out, int version, StreamSession session) throws IOException
+        public void serialize(RetryMessage message, DataOutputStreamPlus out, int version, StreamSession session) throws IOException
         {
             UUIDSerializer.serializer.serialize(message.cfId, out, MessagingService.current_version);
             out.writeInt(message.sequenceNumber);
diff --git a/src/java/org/apache/cassandra/streaming/messages/SessionFailedMessage.java b/src/java/org/apache/cassandra/streaming/messages/SessionFailedMessage.java
index ae15620..46f49d6 100644
--- a/src/java/org/apache/cassandra/streaming/messages/SessionFailedMessage.java
+++ b/src/java/org/apache/cassandra/streaming/messages/SessionFailedMessage.java
@@ -20,7 +20,7 @@
 import java.io.IOException;
 import java.nio.channels.ReadableByteChannel;
 
-import org.apache.cassandra.io.util.DataOutputStreamAndChannel;
+import org.apache.cassandra.io.util.DataOutputStreamPlus;
 import org.apache.cassandra.streaming.StreamSession;
 
 public class SessionFailedMessage extends StreamMessage
@@ -32,7 +32,7 @@
             return new SessionFailedMessage();
         }
 
-        public void serialize(SessionFailedMessage message, DataOutputStreamAndChannel out, int version, StreamSession session) throws IOException {}
+        public void serialize(SessionFailedMessage message, DataOutputStreamPlus out, int version, StreamSession session) throws IOException {}
     };
 
     public SessionFailedMessage()
diff --git a/src/java/org/apache/cassandra/streaming/messages/StreamInitMessage.java b/src/java/org/apache/cassandra/streaming/messages/StreamInitMessage.java
index a9ec4ae..e8b3f82 100644
--- a/src/java/org/apache/cassandra/streaming/messages/StreamInitMessage.java
+++ b/src/java/org/apache/cassandra/streaming/messages/StreamInitMessage.java
@@ -26,6 +26,7 @@
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.io.util.DataOutputBufferFixed;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.net.CompactEndpointSerializationHelper;
 import org.apache.cassandra.net.MessagingService;
@@ -46,14 +47,18 @@
 
     // true if this init message is to connect for outgoing message on receiving side
     public final boolean isForOutgoing;
+    public final boolean keepSSTableLevel;
+    public final boolean isIncremental;
 
-    public StreamInitMessage(InetAddress from, int sessionIndex, UUID planId, String description, boolean isForOutgoing)
+    public StreamInitMessage(InetAddress from, int sessionIndex, UUID planId, String description, boolean isForOutgoing, boolean keepSSTableLevel, boolean isIncremental)
     {
         this.from = from;
         this.sessionIndex = sessionIndex;
         this.planId = planId;
         this.description = description;
         this.isForOutgoing = isForOutgoing;
+        this.keepSSTableLevel = keepSSTableLevel;
+        this.isIncremental = isIncremental;
     }
 
     /**
@@ -78,9 +83,11 @@
         try
         {
             int size = (int)StreamInitMessage.serializer.serializedSize(this, version);
-            DataOutputBuffer buffer = new DataOutputBuffer(size);
-            StreamInitMessage.serializer.serialize(this, buffer, version);
-            bytes = buffer.getData();
+            try (DataOutputBuffer buffer = new DataOutputBufferFixed(size))
+            {
+                StreamInitMessage.serializer.serialize(this, buffer, version);
+                bytes = buffer.getData();
+            }
         }
         catch (IOException e)
         {
@@ -105,6 +112,8 @@
             UUIDSerializer.serializer.serialize(message.planId, out, MessagingService.current_version);
             out.writeUTF(message.description);
             out.writeBoolean(message.isForOutgoing);
+            out.writeBoolean(message.keepSSTableLevel);
+            out.writeBoolean(message.isIncremental);
         }
 
         public StreamInitMessage deserialize(DataInput in, int version) throws IOException
@@ -114,7 +123,9 @@
             UUID planId = UUIDSerializer.serializer.deserialize(in, MessagingService.current_version);
             String description = in.readUTF();
             boolean sentByInitiator = in.readBoolean();
-            return new StreamInitMessage(from, sessionIndex, planId, description, sentByInitiator);
+            boolean keepSSTableLevel = in.readBoolean();
+            boolean isIncremental = in.readBoolean();
+            return new StreamInitMessage(from, sessionIndex, planId, description, sentByInitiator, keepSSTableLevel, isIncremental);
         }
 
         public long serializedSize(StreamInitMessage message, int version)
@@ -124,6 +135,8 @@
             size += UUIDSerializer.serializer.serializedSize(message.planId, MessagingService.current_version);
             size += TypeSizes.NATIVE.sizeof(message.description);
             size += TypeSizes.NATIVE.sizeof(message.isForOutgoing);
+            size += TypeSizes.NATIVE.sizeof(message.keepSSTableLevel);
+            size += TypeSizes.NATIVE.sizeof(message.isIncremental);
             return size;
         }
     }
diff --git a/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java b/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java
index 9724bf1..f0a9ef3 100644
--- a/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java
+++ b/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java
@@ -22,7 +22,7 @@
 import java.nio.ByteBuffer;
 import java.nio.channels.ReadableByteChannel;
 
-import org.apache.cassandra.io.util.DataOutputStreamAndChannel;
+import org.apache.cassandra.io.util.DataOutputStreamPlus;
 import org.apache.cassandra.streaming.StreamSession;
 
 /**
@@ -33,9 +33,11 @@
 public abstract class StreamMessage
 {
     /** Streaming protocol version */
-    public static final int CURRENT_VERSION = 2;
+    public static final int VERSION_20 = 2;
+    public static final int VERSION_22 = 3;
+    public static final int CURRENT_VERSION = VERSION_22;
 
-    public static void serialize(StreamMessage message, DataOutputStreamAndChannel out, int version, StreamSession session) throws IOException
+    public static void serialize(StreamMessage message, DataOutputStreamPlus out, int version, StreamSession session) throws IOException
     {
         ByteBuffer buff = ByteBuffer.allocate(1);
         // message type
@@ -71,7 +73,7 @@
     public static interface Serializer<V extends StreamMessage>
     {
         V deserialize(ReadableByteChannel in, int version, StreamSession session) throws IOException;
-        void serialize(V message, DataOutputStreamAndChannel out, int version, StreamSession session) throws IOException;
+        void serialize(V message, DataOutputStreamPlus out, int version, StreamSession session) throws IOException;
     }
 
     /** StreamMessage types */
diff --git a/src/java/org/apache/cassandra/thrift/CassandraServer.java b/src/java/org/apache/cassandra/thrift/CassandraServer.java
index 7103721..36664ae 100644
--- a/src/java/org/apache/cassandra/thrift/CassandraServer.java
+++ b/src/java/org/apache/cassandra/thrift/CassandraServer.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.thrift;
 
 import java.io.IOException;
-import java.io.UnsupportedEncodingException;
 import java.nio.ByteBuffer;
 import java.nio.charset.CharacterCodingException;
 import java.nio.charset.StandardCharsets;
@@ -30,50 +29,34 @@
 
 import com.google.common.base.Function;
 import com.google.common.base.Joiner;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSortedSet;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
+import com.google.common.collect.*;
 import com.google.common.primitives.Longs;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.auth.AuthenticatedUser;
 import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.KSMetaData;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.cql.CQLStatement;
-import org.apache.cassandra.cql.QueryProcessor;
+import org.apache.cassandra.config.*;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.statements.ParsedStatement;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.db.context.CounterContext;
 import org.apache.cassandra.db.filter.ColumnSlice;
-import org.apache.cassandra.db.filter.IDiskAtomFilter;
-import org.apache.cassandra.db.filter.NamesQueryFilter;
-import org.apache.cassandra.db.filter.SliceQueryFilter;
+import org.apache.cassandra.db.filter.*;
 import org.apache.cassandra.db.marshal.TimeUUIDType;
 import org.apache.cassandra.dht.*;
+import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.locator.DynamicEndpointSnitch;
 import org.apache.cassandra.metrics.ClientMetrics;
 import org.apache.cassandra.scheduler.IRequestScheduler;
 import org.apache.cassandra.serializers.MarshalException;
-import org.apache.cassandra.service.CASRequest;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.MigrationManager;
-import org.apache.cassandra.service.StorageProxy;
-import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.service.*;
 import org.apache.cassandra.service.pager.QueryPagers;
 import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.Pair;
-import org.apache.cassandra.utils.SemanticVersion;
 import org.apache.cassandra.utils.UUIDGen;
 import org.apache.thrift.TException;
 
@@ -85,8 +68,6 @@
 
     private final static List<ColumnOrSuperColumn> EMPTY_COLUMNS = Collections.emptyList();
 
-    private volatile boolean loggedCQL2Warning = false;
-
     /*
      * RequestScheduler to perform the scheduling of incoming requests
      */
@@ -314,7 +295,7 @@
         }
         else
         {
-            logger.debug("get_slice");
+            logger.trace("get_slice");
         }
 
         try
@@ -362,7 +343,7 @@
         }
         else
         {
-            logger.debug("multiget_slice");
+            logger.trace("multiget_slice");
         }
 
         try
@@ -469,7 +450,7 @@
         }
         else
         {
-            logger.debug("get");
+            logger.trace("get");
         }
 
         try
@@ -538,7 +519,7 @@
         }
         else
         {
-            logger.debug("get_count");
+            logger.trace("get_count");
         }
 
         try
@@ -557,10 +538,10 @@
             // request by page if this is a large row
             if (cfs.getMeanColumns() > 0)
             {
-                int averageColumnSize = (int) (cfs.getMeanRowSize() / cfs.getMeanColumns());
+                int averageColumnSize = (int) (cfs.metric.meanRowSize.getValue() / cfs.getMeanColumns());
                 pageSize = Math.min(COUNT_PAGE_SIZE, 4 * 1024 * 1024 / averageColumnSize);
                 pageSize = Math.max(2, pageSize);
-                logger.debug("average row column size is {}; using pageSize of {}", averageColumnSize, pageSize);
+                logger.trace("average row column size is {}; using pageSize of {}", averageColumnSize, pageSize);
             }
             else
             {
@@ -600,13 +581,6 @@
         }
     }
 
-    private static ByteBuffer getName(ColumnOrSuperColumn cosc)
-    {
-        return cosc.isSetSuper_column() ? cosc.super_column.name :
-                   (cosc.isSetColumn() ? cosc.column.name :
-                       (cosc.isSetCounter_column() ? cosc.counter_column.name : cosc.counter_super_column.name));
-    }
-
     public Map<ByteBuffer, Integer> multiget_count(List<ByteBuffer> keys, ColumnParent column_parent, SlicePredicate predicate, ConsistencyLevel consistency_level)
     throws InvalidRequestException, UnavailableException, TimedOutException
     {
@@ -625,7 +599,7 @@
         }
         else
         {
-            logger.debug("multiget_count");
+            logger.trace("multiget_count");
         }
 
         try
@@ -710,7 +684,7 @@
         }
         else
         {
-            logger.debug("insert");
+            logger.trace("insert");
         }
 
         try
@@ -750,7 +724,7 @@
         }
         else
         {
-            logger.debug("cas");
+            logger.trace("cas");
         }
 
         try
@@ -984,7 +958,7 @@
         }
         else
         {
-            logger.debug("batch_mutate");
+            logger.trace("batch_mutate");
         }
 
         try
@@ -1017,7 +991,7 @@
         }
         else
         {
-            logger.debug("atomic_batch_mutate");
+            logger.trace("atomic_batch_mutate");
         }
 
         try
@@ -1076,7 +1050,7 @@
         }
         else
         {
-            logger.debug("remove");
+            logger.trace("remove");
         }
 
         try
@@ -1146,7 +1120,7 @@
         if (ksm == null)
             throw new NotFoundException();
 
-        return ksm.toThrift();
+        return ThriftConversion.toThrift(ksm);
     }
 
     public List<KeySlice> get_range_slices(ColumnParent column_parent, SlicePredicate predicate, KeyRange range, ConsistencyLevel consistency_level)
@@ -1163,7 +1137,7 @@
         }
         else
         {
-            logger.debug("range_slice");
+            logger.trace("range_slice");
         }
 
         try
@@ -1189,12 +1163,12 @@
                 Token.TokenFactory tokenFactory = p.getTokenFactory();
                 Token left = tokenFactory.fromString(range.start_token);
                 Token right = tokenFactory.fromString(range.end_token);
-                bounds = Range.makeRowRange(left, right, p);
+                bounds = Range.makeRowRange(left, right);
             }
             else
             {
                 RowPosition end = range.end_key == null
-                                ? p.getTokenFactory().fromString(range.end_token).maxKeyBound(p)
+                                ? p.getTokenFactory().fromString(range.end_token).maxKeyBound()
                                 : RowPosition.ForKey.get(range.end_key, p);
                 bounds = new Bounds<RowPosition>(RowPosition.ForKey.get(range.start_key, p), end);
             }
@@ -1208,7 +1182,7 @@
                                                                         now,
                                                                         filter,
                                                                         bounds,
-                                                                        ThriftConversion.fromThrift(range.row_filter),
+                                                                        ThriftConversion.indexExpressionsFromThrift(range.row_filter),
                                                                         range.count),
                                                   consistencyLevel);
             }
@@ -1224,13 +1198,9 @@
         {
             throw ThriftConversion.toThrift(e);
         }
-        catch (ReadTimeoutException e)
+        catch (RequestExecutionException e)
         {
-            throw ThriftConversion.toThrift(e);
-        }
-        catch (org.apache.cassandra.exceptions.UnavailableException e)
-        {
-            throw ThriftConversion.toThrift(e);
+            throw ThriftConversion.rethrow(e);
         }
         finally
         {
@@ -1251,7 +1221,7 @@
         }
         else
         {
-            logger.debug("get_paged_slice");
+            logger.trace("get_paged_slice");
         }
 
         try
@@ -1277,12 +1247,12 @@
                 Token.TokenFactory tokenFactory = p.getTokenFactory();
                 Token left = tokenFactory.fromString(range.start_token);
                 Token right = tokenFactory.fromString(range.end_token);
-                bounds = Range.makeRowRange(left, right, p);
+                bounds = Range.makeRowRange(left, right);
             }
             else
             {
                 RowPosition end = range.end_key == null
-                                ? p.getTokenFactory().fromString(range.end_token).maxKeyBound(p)
+                                ? p.getTokenFactory().fromString(range.end_token).maxKeyBound()
                                 : RowPosition.ForKey.get(range.end_key, p);
                 bounds = new Bounds<RowPosition>(RowPosition.ForKey.get(range.start_key, p), end);
             }
@@ -1310,13 +1280,9 @@
         {
             throw ThriftConversion.toThrift(e);
         }
-        catch (ReadTimeoutException e)
+        catch (RequestExecutionException e)
         {
-            throw ThriftConversion.toThrift(e);
-        }
-        catch (org.apache.cassandra.exceptions.UnavailableException e)
-        {
-            throw ThriftConversion.toThrift(e);
+            throw ThriftConversion.rethrow(e);
         }
         finally
         {
@@ -1350,7 +1316,7 @@
         }
         else
         {
-            logger.debug("scan");
+            logger.trace("scan");
         }
 
         try
@@ -1376,7 +1342,7 @@
                                                               now,
                                                               filter,
                                                               bounds,
-                                                              ThriftConversion.fromThrift(index_clause.expressions),
+                                                              ThriftConversion.indexExpressionsFromThrift(index_clause.expressions),
                                                               index_clause.count);
 
             List<Row> rows = StorageProxy.getRangeSlice(command, consistencyLevel);
@@ -1386,13 +1352,9 @@
         {
             throw ThriftConversion.toThrift(e);
         }
-        catch (ReadTimeoutException e)
+        catch (RequestExecutionException e)
         {
-            throw ThriftConversion.toThrift(e);
-        }
-        catch (org.apache.cassandra.exceptions.UnavailableException e)
-        {
-            throw ThriftConversion.toThrift(e);
+            throw ThriftConversion.rethrow(e);
         }
         finally
         {
@@ -1506,12 +1468,11 @@
         }
     }
 
-    public void login(AuthenticationRequest auth_request) throws AuthenticationException, AuthorizationException, TException
+    public void login(AuthenticationRequest auth_request) throws TException
     {
         try
         {
-            AuthenticatedUser user = DatabaseDescriptor.getAuthenticator().authenticate(auth_request.getCredentials());
-            state().login(user);
+            state().login(DatabaseDescriptor.getAuthenticator().legacyAuthenticate(auth_request.getCredentials()));
         }
         catch (org.apache.cassandra.exceptions.AuthenticationException e)
         {
@@ -1545,7 +1506,7 @@
     public String system_add_column_family(CfDef cf_def)
     throws InvalidRequestException, SchemaDisagreementException, TException
     {
-        logger.debug("add_column_family");
+        logger.trace("add_column_family");
 
         try
         {
@@ -1553,7 +1514,7 @@
             String keyspace = cState.getKeyspace();
             cState.hasKeyspaceAccess(keyspace, Permission.CREATE);
             cf_def.unsetId(); // explicitly ignore any id set by client (Hector likes to set zero)
-            CFMetaData cfm = CFMetaData.fromThrift(cf_def);
+            CFMetaData cfm = ThriftConversion.fromThrift(cf_def);
             CFMetaData.validateCompactionOptions(cfm.compactionStrategyClass, cfm.compactionStrategyOptions);
             cfm.addDefaultIndexNames();
 
@@ -1572,7 +1533,7 @@
     public String system_drop_column_family(String column_family)
     throws InvalidRequestException, SchemaDisagreementException, TException
     {
-        logger.debug("drop_column_family");
+        logger.trace("drop_column_family");
 
         ThriftClientState cState = state();
 
@@ -1592,7 +1553,7 @@
     public String system_add_keyspace(KsDef ks_def)
     throws InvalidRequestException, SchemaDisagreementException, TException
     {
-        logger.debug("add_keyspace");
+        logger.trace("add_keyspace");
 
         try
         {
@@ -1613,7 +1574,7 @@
             for (CfDef cf_def : ks_def.cf_defs)
             {
                 cf_def.unsetId(); // explicitly ignore any id set by client (same as system_add_column_family)
-                CFMetaData cfm = CFMetaData.fromThrift(cf_def);
+                CFMetaData cfm = ThriftConversion.fromThrift(cf_def);
                 cfm.addDefaultIndexNames();
 
                 if (!cfm.getTriggers().isEmpty())
@@ -1621,7 +1582,7 @@
 
                 cfDefs.add(cfm);
             }
-            MigrationManager.announceNewKeyspace(KSMetaData.fromThrift(ks_def, cfDefs.toArray(new CFMetaData[cfDefs.size()])));
+            MigrationManager.announceNewKeyspace(ThriftConversion.fromThrift(ks_def, cfDefs.toArray(new CFMetaData[cfDefs.size()])));
             return Schema.instance.getVersion().toString();
         }
         catch (RequestValidationException e)
@@ -1633,7 +1594,7 @@
     public String system_drop_keyspace(String keyspace)
     throws InvalidRequestException, SchemaDisagreementException, TException
     {
-        logger.debug("drop_keyspace");
+        logger.trace("drop_keyspace");
 
         try
         {
@@ -1655,7 +1616,7 @@
     public String system_update_keyspace(KsDef ks_def)
     throws InvalidRequestException, SchemaDisagreementException, TException
     {
-        logger.debug("update_keyspace");
+        logger.trace("update_keyspace");
 
         try
         {
@@ -1663,9 +1624,9 @@
             state().hasKeyspaceAccess(ks_def.name, Permission.ALTER);
             ThriftValidation.validateKeyspace(ks_def.name);
             if (ks_def.getCf_defs() != null && ks_def.getCf_defs().size() > 0)
-                throw new InvalidRequestException("Keyspace update must not contain any column family definitions.");
+                throw new InvalidRequestException("Keyspace update must not contain any table definitions.");
 
-            MigrationManager.announceKeyspaceUpdate(KSMetaData.fromThrift(ks_def));
+            MigrationManager.announceKeyspaceUpdate(ThriftConversion.fromThrift(ks_def));
             return Schema.instance.getVersion().toString();
         }
         catch (RequestValidationException e)
@@ -1677,7 +1638,7 @@
     public String system_update_column_family(CfDef cf_def)
     throws InvalidRequestException, SchemaDisagreementException, TException
     {
-        logger.debug("update_column_family");
+        logger.trace("update_column_family");
 
         try
         {
@@ -1688,19 +1649,19 @@
             CFMetaData oldCfm = Schema.instance.getCFMetaData(cf_def.keyspace, cf_def.name);
 
             if (oldCfm == null)
-                throw new InvalidRequestException("Could not find column family definition to modify.");
+                throw new InvalidRequestException("Could not find table definition to modify.");
 
             if (!oldCfm.isThriftCompatible())
                 throw new InvalidRequestException("Cannot modify CQL3 table " + oldCfm.cfName + " as it may break the schema. You should use cqlsh to modify CQL3 tables instead.");
 
-            CFMetaData cfm = CFMetaData.fromThriftForUpdate(cf_def, oldCfm);
+            CFMetaData cfm = ThriftConversion.fromThriftForUpdate(cf_def, oldCfm);
             CFMetaData.validateCompactionOptions(cfm.compactionStrategyClass, cfm.compactionStrategyOptions);
             cfm.addDefaultIndexNames();
 
             if (!oldCfm.getTriggers().equals(cfm.getTriggers()))
                 state().ensureIsSuper("Only superusers are allowed to add or remove triggers.");
 
-            MigrationManager.announceColumnFamilyUpdate(cfm, true);
+            MigrationManager.announceColumnFamilyUpdate(cfm);
             return Schema.instance.getVersion().toString();
         }
         catch (RequestValidationException e)
@@ -1724,7 +1685,7 @@
             }
             else
             {
-                logger.debug("truncating {}.{}", cState.getKeyspace(), cfname);
+                logger.trace("truncating {}.{}", cState.getKeyspace(), cfname);
             }
 
             schedule(DatabaseDescriptor.getTruncateRpcTimeout());
@@ -1773,7 +1734,7 @@
 
     public Map<String, List<String>> describe_schema_versions() throws TException, InvalidRequestException
     {
-        logger.debug("checking schema agreement");
+        logger.trace("checking schema agreement");
         return StorageProxy.describeSchemaVersions();
     }
 
@@ -1791,7 +1752,7 @@
         }
         else
         {
-            logger.debug("add");
+            logger.trace("add");
         }
 
         try
@@ -1847,7 +1808,7 @@
         }
         else
         {
-            logger.debug("remove_counter");
+            logger.trace("remove_counter");
         }
 
         try
@@ -1915,70 +1876,20 @@
         {
             throw new InvalidRequestException("Error deflating query string.");
         }
+        catch (IOException e)
+        {
+            throw new AssertionError(e);
+        }
         return queryString;
     }
 
-    private void validateCQLVersion(int major) throws InvalidRequestException
+    public CqlResult execute_cql_query(ByteBuffer query, Compression compression) throws TException
     {
-        /*
-         * The rules are:
-         *   - If no version are set, we don't validate anything. The reason is
-         *     that 1) old CQL2 client might not have called set_cql_version
-         *     and 2) some client may have removed the set_cql_version for CQL3
-         *     when updating to 1.2.0. A CQL3 client upgrading from pre-1.2
-         *     shouldn't be in that case however since set_cql_version uses to
-         *     be mandatory (for CQL3).
-         *   - Otherwise, checks the major matches whatever was set.
-         */
-        SemanticVersion versionSet = state().getCQLVersion();
-        if (versionSet == null)
-            return;
-
-        if (versionSet.major != major)
-            throw new InvalidRequestException(
-                "Cannot execute/prepare CQL" + major + " statement since the CQL has been set to CQL" + versionSet.major
-              + "(This might mean your client hasn't been upgraded correctly to use the new CQL3 methods introduced in Cassandra 1.2+).");
+        throw new InvalidRequestException("CQL2 has been removed in Cassandra 2.2. Please use CQL3 instead");
     }
 
-    public CqlResult execute_cql_query(ByteBuffer query, Compression compression)
-    throws InvalidRequestException, UnavailableException, TimedOutException, SchemaDisagreementException, TException
+    public CqlResult execute_cql3_query(ByteBuffer query, Compression compression, ConsistencyLevel cLevel) throws TException
     {
-        validateCQLVersion(2);
-        maybeLogCQL2Warning();
-
-        try
-        {
-            String queryString = uncompress(query, compression);
-            if (startSessionIfRequested())
-            {
-                Tracing.instance.begin("execute_cql_query",
-                                       ImmutableMap.of("query", queryString));
-            }
-            else
-            {
-                logger.debug("execute_cql_query");
-            }
-
-            return QueryProcessor.process(queryString, state());
-        }
-        catch (RequestExecutionException e)
-        {
-            throw ThriftConversion.rethrow(e);
-        }
-        catch (RequestValidationException e)
-        {
-            throw ThriftConversion.toThrift(e);
-        }
-        finally
-        {
-            Tracing.instance.stopSession();
-        }
-    }
-
-    public CqlResult execute_cql3_query(ByteBuffer query, Compression compression, ConsistencyLevel cLevel)
-    throws InvalidRequestException, UnavailableException, TimedOutException, SchemaDisagreementException, TException
-    {
-        validateCQLVersion(3);
         try
         {
             String queryString = uncompress(query, compression);
@@ -1990,11 +1901,14 @@
             }
             else
             {
-                logger.debug("execute_cql3_query");
+                logger.trace("execute_cql3_query");
             }
 
             ThriftClientState cState = state();
-            return cState.getCQLQueryHandler().process(queryString, cState.getQueryState(), QueryOptions.fromProtocolV2(ThriftConversion.fromThrift(cLevel), Collections.<ByteBuffer>emptyList())).toThriftResult();
+            return ClientState.getCQLQueryHandler().process(queryString,
+                                                            cState.getQueryState(),
+                                                            QueryOptions.fromProtocolV2(ThriftConversion.fromThrift(cLevel), Collections.<ByteBuffer>emptyList()),
+                                                            null).toThriftResult();
         }
         catch (RequestExecutionException e)
         {
@@ -2010,14 +1924,14 @@
         }
     }
 
-    public CqlPreparedResult prepare_cql_query(ByteBuffer query, Compression compression)
-    throws InvalidRequestException, TException
+    public CqlPreparedResult prepare_cql_query(ByteBuffer query, Compression compression) throws TException
     {
-        if (logger.isDebugEnabled())
-            logger.debug("prepare_cql_query");
+        throw new InvalidRequestException("CQL2 has been removed in Cassandra 2.2. Please use CQL3 instead");
+    }
 
-        validateCQLVersion(2);
-        maybeLogCQL2Warning();
+    public CqlPreparedResult prepare_cql3_query(ByteBuffer query, Compression compression) throws TException
+    {
+        logger.trace("prepare_cql3_query");
 
         String queryString = uncompress(query, compression);
         ThriftClientState cState = state();
@@ -2025,7 +1939,9 @@
         try
         {
             cState.validateLogin();
-            return QueryProcessor.prepare(queryString, cState);
+            return ClientState.getCQLQueryHandler().prepare(queryString,
+                                                       cState.getQueryState(),
+                                                       null).toThriftPreparedResult();
         }
         catch (RequestValidationException e)
         {
@@ -2033,26 +1949,52 @@
         }
     }
 
-    public CqlPreparedResult prepare_cql3_query(ByteBuffer query, Compression compression)
-    throws InvalidRequestException, TException
+    public CqlResult execute_prepared_cql_query(int itemId, List<ByteBuffer> bindVariables) throws TException
     {
-        if (logger.isDebugEnabled())
-            logger.debug("prepare_cql3_query");
+        throw new InvalidRequestException("CQL2 has been removed in Cassandra 2.2. Please use CQL3 instead");
+    }
 
-        validateCQLVersion(3);
-
-        String queryString = uncompress(query, compression);
-        ThriftClientState cState = state();
+    public CqlResult execute_prepared_cql3_query(int itemId, List<ByteBuffer> bindVariables, ConsistencyLevel cLevel) throws TException
+    {
+        if (startSessionIfRequested())
+        {
+            // TODO we don't have [typed] access to CQL bind variables here.  CASSANDRA-4560 is open to add support.
+            Tracing.instance.begin("execute_prepared_cql3_query", ImmutableMap.of("consistency_level", cLevel.name()));
+        }
+        else
+        {
+            logger.trace("execute_prepared_cql3_query");
+        }
 
         try
         {
-            cState.validateLogin();
-            return cState.getCQLQueryHandler().prepare(queryString, cState.getQueryState()).toThriftPreparedResult();
+            ThriftClientState cState = state();
+            ParsedStatement.Prepared prepared = ClientState.getCQLQueryHandler().getPreparedForThrift(itemId);
+
+            if (prepared == null)
+                throw new InvalidRequestException(String.format("Prepared query with ID %d not found" +
+                                                                " (either the query was not prepared on this host (maybe the host has been restarted?)" +
+                                                                " or you have prepared too many queries and it has been evicted from the internal cache)",
+                                                                itemId));
+            logger.trace("Retrieved prepared statement #{} with {} bind markers", itemId, prepared.statement.getBoundTerms());
+
+            return ClientState.getCQLQueryHandler().processPrepared(prepared.statement,
+                                                                    cState.getQueryState(),
+                                                                    QueryOptions.fromProtocolV2(ThriftConversion.fromThrift(cLevel), bindVariables),
+                                                                    null).toThriftResult();
+        }
+        catch (RequestExecutionException e)
+        {
+            throw ThriftConversion.rethrow(e);
         }
         catch (RequestValidationException e)
         {
             throw ThriftConversion.toThrift(e);
         }
+        finally
+        {
+            Tracing.instance.stopSession();
+        }
     }
 
     @Override
@@ -2070,9 +2012,9 @@
         }
         else
         {
-            logger.debug("get_multi_slice");
+            logger.trace("get_multi_slice");
         }
-        try
+        try 
         {
             ClientState cState = state();
             String keyspace = cState.getKeyspace();
@@ -2100,7 +2042,6 @@
                 }
                 slices[i] = new ColumnSlice(start, finish);
             }
-
             ColumnSlice[] deoverlapped = ColumnSlice.deoverlapSlices(slices, request.reversed ? metadata.comparator.reverseComparator() : metadata.comparator);
             SliceQueryFilter filter = new SliceQueryFilter(deoverlapped, request.reversed, request.count);
             ThriftValidation.validateKey(metadata, request.key);
@@ -2110,8 +2051,8 @@
         catch (RequestValidationException e)
         {
             throw ThriftConversion.toThrift(e);
-        }
-        finally
+        } 
+        finally 
         {
             Tracing.instance.stopSession();
         }
@@ -2128,118 +2069,11 @@
             columnSlice.setFinish(new byte[0]);
     }
 
-    public CqlResult execute_prepared_cql_query(int itemId, List<ByteBuffer> bindVariables)
-    throws InvalidRequestException, UnavailableException, TimedOutException, SchemaDisagreementException, TException
-    {
-        validateCQLVersion(2);
-        maybeLogCQL2Warning();
-
-        if (startSessionIfRequested())
-        {
-            // TODO we don't have [typed] access to CQL bind variables here.  CASSANDRA-4560 is open to add support.
-            Tracing.instance.begin("execute_prepared_cql_query", Collections.<String, String>emptyMap());
-        }
-        else
-        {
-            logger.debug("execute_prepared_cql_query");
-        }
-
-        try
-        {
-            ThriftClientState cState = state();
-            CQLStatement statement = cState.getPrepared().get(itemId);
-
-            if (statement == null)
-                throw new InvalidRequestException(String.format("Prepared query with ID %d not found", itemId));
-            logger.trace("Retrieved prepared statement #{} with {} bind markers", itemId, statement.boundTerms);
-
-            return QueryProcessor.processPrepared(statement, cState, bindVariables);
-        }
-        catch (RequestExecutionException e)
-        {
-            throw ThriftConversion.rethrow(e);
-        }
-        catch (RequestValidationException e)
-        {
-            throw ThriftConversion.toThrift(e);
-        }
-        finally
-        {
-            Tracing.instance.stopSession();
-        }
-    }
-
-    public CqlResult execute_prepared_cql3_query(int itemId, List<ByteBuffer> bindVariables, ConsistencyLevel cLevel)
-    throws InvalidRequestException, UnavailableException, TimedOutException, SchemaDisagreementException, TException
-    {
-        validateCQLVersion(3);
-
-        if (startSessionIfRequested())
-        {
-            // TODO we don't have [typed] access to CQL bind variables here.  CASSANDRA-4560 is open to add support.
-            Tracing.instance.begin("execute_prepared_cql3_query", ImmutableMap.of("consistency_level", cLevel.name()));
-        }
-        else
-        {
-            logger.debug("execute_prepared_cql3_query");
-        }
-
-        try
-        {
-            ThriftClientState cState = state();
-            ParsedStatement.Prepared prepared = cState.getCQLQueryHandler().getPreparedForThrift(itemId);
-
-            if (prepared == null)
-                throw new InvalidRequestException(String.format("Prepared query with ID %d not found" +
-                                                                " (either the query was not prepared on this host (maybe the host has been restarted?)" +
-                                                                " or you have prepared too many queries and it has been evicted from the internal cache)",
-                                                                itemId));
-            logger.trace("Retrieved prepared statement #{} with {} bind markers", itemId, prepared.statement.getBoundTerms());
-
-            return cState.getCQLQueryHandler().processPrepared(prepared.statement,
-                                                               cState.getQueryState(),
-                                                               QueryOptions.fromProtocolV2(ThriftConversion.fromThrift(cLevel), bindVariables)).toThriftResult();
-        }
-        catch (RequestExecutionException e)
-        {
-            throw ThriftConversion.rethrow(e);
-        }
-        catch (RequestValidationException e)
-        {
-            throw ThriftConversion.toThrift(e);
-        }
-        finally
-        {
-            Tracing.instance.stopSession();
-        }
-    }
-
     /*
-     * Deprecated, but if a client sets CQL2, it is a no-op for compatibility sake.
-     * If it sets CQL3 however, we throw an IRE because this mean the client
-     * hasn't been updated for Cassandra 1.2 and should start using the new
-     * execute_cql3_query, etc... and there is no point no warning it early.
+     * No-op since 2.2.
      */
-    public void set_cql_version(String version) throws InvalidRequestException
+    public void set_cql_version(String version)
     {
-        try
-        {
-            state().setCQLVersion(version);
-        }
-        catch (org.apache.cassandra.exceptions.InvalidRequestException e)
-        {
-            throw new InvalidRequestException(e.getMessage());
-        }
-    }
-
-    private void maybeLogCQL2Warning()
-    {
-        if (!loggedCQL2Warning)
-        {
-            logger.warn("CQL2 has been deprecated since Cassandra 2.0, and will be removed entirely in version 2.2."
-                        + " Please switch to CQL3 before then.");
-            loggedCQL2Warning = true;
-        }
     }
 
     public ByteBuffer trace_next_query() throws TException
diff --git a/src/java/org/apache/cassandra/thrift/CustomTNonBlockingServer.java b/src/java/org/apache/cassandra/thrift/CustomTNonBlockingServer.java
index 11fcc5e..de8df57 100644
--- a/src/java/org/apache/cassandra/thrift/CustomTNonBlockingServer.java
+++ b/src/java/org/apache/cassandra/thrift/CustomTNonBlockingServer.java
@@ -37,6 +37,7 @@
     }
 
     @Override
+    @SuppressWarnings("resource")
     protected boolean requestInvoke(FrameBuffer frameBuffer)
     {
         TNonblockingSocket socket = (TNonblockingSocket)((CustomFrameBuffer)frameBuffer).getTransport();
@@ -47,6 +48,7 @@
 
     public static class Factory implements TServerFactory
     {
+        @SuppressWarnings("resource")
         public TServer buildTServer(Args args)
         {
             if (DatabaseDescriptor.getClientEncryptionOptions().enabled)
diff --git a/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java b/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
index 905a981..efa9330 100644
--- a/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
+++ b/src/java/org/apache/cassandra/thrift/CustomTThreadPoolServer.java
@@ -54,7 +54,7 @@
 
 /**
  * Slightly modified version of the Apache Thrift TThreadPoolServer.
- * <p/>
+ * <p>
  * This allows passing an executor so you have more control over the actual
  * behaviour of the tasks being run.
  * <p/>
@@ -86,6 +86,7 @@
         this.activeClients = new AtomicInteger(0);
     }
 
+    @SuppressWarnings("resource")
     public void serve()
     {
         try
@@ -126,7 +127,7 @@
             catch (RejectedExecutionException e)
             {
                 // worker thread decremented activeClients but hadn't finished exiting
-                logger.debug("Dropping client connection because our limit of {} has been reached", args.maxWorkerThreads);
+                logger.trace("Dropping client connection because our limit of {} has been reached", args.maxWorkerThreads);
                 continue;
             }
 
@@ -185,18 +186,16 @@
         public void run()
         {
             TProcessor processor = null;
-            TTransport inputTransport = null;
-            TTransport outputTransport = null;
             TProtocol inputProtocol = null;
             TProtocol outputProtocol = null;
             SocketAddress socket = null;
-            try
+            try (TTransport inputTransport = inputTransportFactory_.getTransport(client_);
+                 TTransport outputTransport = outputTransportFactory_.getTransport(client_))
             {
                 socket = ((TCustomSocket) client_).getSocket().getRemoteSocketAddress();
                 ThriftSessionManager.instance.setCurrentSocket(socket);
                 processor = processorFactory_.getProcessor(client_);
-                inputTransport = inputTransportFactory_.getTransport(client_);
-                outputTransport = outputTransportFactory_.getTransport(client_);
+
                 inputProtocol = inputProtocolFactory_.getProtocol(inputTransport);
                 outputProtocol = outputProtocolFactory_.getProtocol(outputTransport);
                 // we check stopped first to make sure we're not supposed to be shutting
@@ -213,7 +212,7 @@
             {
                 // Assume the client died and continue silently
                 // Log at debug to allow debugging of "frame too large" errors (see CASSANDRA-3142).
-                logger.debug("Thrift transport error occurred during processing of message.", ttx);
+                logger.trace("Thrift transport error occurred during processing of message.", ttx);
             }
             catch (TException tx)
             {
@@ -228,10 +227,7 @@
             {
                 if (socket != null)
                     ThriftSessionManager.instance.connectionComplete(socket);
-                if (inputTransport != null)
-                    inputTransport.close();
-                if (outputTransport != null)
-                    outputTransport.close();
+
                 activeClients.decrementAndGet();
             }
         }
@@ -239,6 +235,7 @@
 
     public static class Factory implements TServerFactory
     {
+        @SuppressWarnings("resource")
         public TServer buildTServer(Args args)
         {
             final InetSocketAddress addr = args.addr;
@@ -249,7 +246,7 @@
                 if (clientEnc.enabled)
                 {
                     logger.info("enabling encrypted thrift connections between client and server");
-                    TSSLTransportParameters params = new TSSLTransportParameters(clientEnc.protocol, clientEnc.cipher_suites);
+                    TSSLTransportParameters params = new TSSLTransportParameters(clientEnc.protocol, new String[0]);
                     params.setKeyStore(clientEnc.keystore, clientEnc.keystore_password);
                     if (clientEnc.require_client_auth)
                     {
@@ -258,6 +255,8 @@
                     }
                     TServerSocket sslServer = TSSLTransportFactory.getServerSocket(addr.getPort(), 0, addr.getAddress(), params);
                     SSLServerSocket sslServerSocket = (SSLServerSocket) sslServer.getServerSocket();
+                    String[] suites = SSLFactory.filterCipherSuites(sslServerSocket.getSupportedCipherSuites(), clientEnc.cipher_suites);
+                    sslServerSocket.setEnabledCipherSuites(suites);
                     sslServerSocket.setEnabledProtocols(SSLFactory.ACCEPTED_PROTOCOLS);
                     serverTransport = new TCustomServerSocket(sslServer.getServerSocket(), args.keepAlive, args.sendBufferSize, args.recvBufferSize);
                 }
diff --git a/src/java/org/apache/cassandra/thrift/SSLTransportFactory.java b/src/java/org/apache/cassandra/thrift/SSLTransportFactory.java
index d1bdead..ea74b94 100644
--- a/src/java/org/apache/cassandra/thrift/SSLTransportFactory.java
+++ b/src/java/org/apache/cassandra/thrift/SSLTransportFactory.java
@@ -27,6 +27,8 @@
 
 public class SSLTransportFactory implements ITransportFactory
 {
+    public static final int DEFAULT_MAX_FRAME_SIZE = 15 * 1024 * 1024; // 15 MiB
+
     public static final String TRUSTSTORE = "enc.truststore";
     public static final String TRUSTSTORE_PASSWORD = "enc.truststore.password";
     public static final String KEYSTORE = "enc.keystore";
@@ -50,6 +52,7 @@
     private String[] cipherSuites;
 
     @Override
+    @SuppressWarnings("resource")
     public TTransport openTransport(String host, int port) throws Exception
     {
         TSSLTransportFactory.TSSLTransportParameters params = new TSSLTransportFactory.TSSLTransportParameters(protocol, cipherSuites);
@@ -57,8 +60,7 @@
         if (null != keystore)
             params.setKeyStore(keystore, keystorePassword);
         TTransport trans = TSSLTransportFactory.getClientSocket(host, port, SOCKET_TIMEOUT, params);
-        int frameSize = 15 * 1024 * 1024; // 15 MiB
-        return new TFramedTransport(trans, frameSize);
+        return new TFramedTransport(trans, DEFAULT_MAX_FRAME_SIZE);
     }
 
     @Override
diff --git a/src/java/org/apache/cassandra/thrift/TCustomNonblockingServerSocket.java b/src/java/org/apache/cassandra/thrift/TCustomNonblockingServerSocket.java
index 63466b8..a430721 100644
--- a/src/java/org/apache/cassandra/thrift/TCustomNonblockingServerSocket.java
+++ b/src/java/org/apache/cassandra/thrift/TCustomNonblockingServerSocket.java
@@ -43,6 +43,7 @@
     }
 
     @Override
+    @SuppressWarnings("resource")
     protected TNonblockingSocket acceptImpl() throws TTransportException
     {
         TNonblockingSocket tsocket = super.acceptImpl();
diff --git a/src/java/org/apache/cassandra/thrift/TCustomServerSocket.java b/src/java/org/apache/cassandra/thrift/TCustomServerSocket.java
index d88cf71..8e27481 100644
--- a/src/java/org/apache/cassandra/thrift/TCustomServerSocket.java
+++ b/src/java/org/apache/cassandra/thrift/TCustomServerSocket.java
@@ -72,7 +72,7 @@
         catch (IOException ioe)
         {
             serverSocket = null;
-            throw new TTransportException("Could not create ServerSocket on address " + bindAddr.toString() + ".");
+            throw new TTransportException("Could not create ServerSocket on address " + bindAddr + ".");
         }
 
         this.keepAlive = keepAlive;
@@ -89,6 +89,7 @@
     }
 
     @Override
+    @SuppressWarnings("resource")
     protected TCustomSocket acceptImpl() throws TTransportException
     {
 
diff --git a/src/java/org/apache/cassandra/thrift/TFramedTransportFactory.java b/src/java/org/apache/cassandra/thrift/TFramedTransportFactory.java
index a4c6bb7..7bf0b96 100644
--- a/src/java/org/apache/cassandra/thrift/TFramedTransportFactory.java
+++ b/src/java/org/apache/cassandra/thrift/TFramedTransportFactory.java
@@ -34,6 +34,7 @@
     private static final String THRIFT_FRAMED_TRANSPORT_SIZE_IN_MB = "cassandra.thrift.framed.size_mb";
     private int thriftFramedTransportSizeMb = 15; // 15Mb is the default for C* & Hadoop ConfigHelper
 
+    @SuppressWarnings("resource")
     public TTransport openTransport(String host, int port) throws TTransportException
     {
         TSocket socket = new TSocket(host, port);
diff --git a/src/java/org/apache/cassandra/thrift/THsHaDisruptorServer.java b/src/java/org/apache/cassandra/thrift/THsHaDisruptorServer.java
index dd501ec..37bc440 100644
--- a/src/java/org/apache/cassandra/thrift/THsHaDisruptorServer.java
+++ b/src/java/org/apache/cassandra/thrift/THsHaDisruptorServer.java
@@ -67,6 +67,7 @@
 
     public static class Factory implements TServerFactory
     {
+        @SuppressWarnings("resource")
         public TServer buildTServer(Args args)
         {
             if (DatabaseDescriptor.getClientEncryptionOptions().enabled)
diff --git a/src/java/org/apache/cassandra/thrift/ThriftClientState.java b/src/java/org/apache/cassandra/thrift/ThriftClientState.java
index 56dcf6f..6a3c50f 100644
--- a/src/java/org/apache/cassandra/thrift/ThriftClientState.java
+++ b/src/java/org/apache/cassandra/thrift/ThriftClientState.java
@@ -17,11 +17,9 @@
  */
 package org.apache.cassandra.thrift;
 
-import java.net.SocketAddress;
-import java.util.*;
+import java.net.InetSocketAddress;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.cql.CQLStatement;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
 
@@ -34,20 +32,9 @@
  */
 public class ThriftClientState extends ClientState
 {
-    private static final int MAX_CACHE_PREPARED = 10000;    // Enough to keep buggy clients from OOM'ing us
-
     private final QueryState queryState;
 
-    // An LRU map of prepared statements
-    private final Map<Integer, CQLStatement> prepared = new LinkedHashMap<Integer, CQLStatement>(16, 0.75f, true)
-    {
-        protected boolean removeEldestEntry(Map.Entry<Integer, CQLStatement> eldest)
-        {
-            return size() > MAX_CACHE_PREPARED;
-        }
-    };
-
-    public ThriftClientState(SocketAddress remoteAddress)
+    public ThriftClientState(InetSocketAddress remoteAddress)
     {
         super(remoteAddress);
         this.queryState = new QueryState(this);
@@ -58,11 +45,6 @@
         return queryState;
     }
 
-    public Map<Integer, CQLStatement> getPrepared()
-    {
-        return prepared;
-    }
-
     public String getSchedulingValue()
     {
         switch(DatabaseDescriptor.getRequestSchedulerId())
diff --git a/src/java/org/apache/cassandra/thrift/ThriftConversion.java b/src/java/org/apache/cassandra/thrift/ThriftConversion.java
index 2aca45a..04eae38 100644
--- a/src/java/org/apache/cassandra/thrift/ThriftConversion.java
+++ b/src/java/org/apache/cassandra/thrift/ThriftConversion.java
@@ -17,16 +17,31 @@
  */
 package org.apache.cassandra.thrift;
 
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
+import java.nio.ByteBuffer;
+import java.util.*;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Strings;
+import com.google.common.collect.Maps;
+
+import org.apache.cassandra.cache.CachingOptions;
+import org.apache.cassandra.config.*;
+import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.cql3.Operator;
+import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.db.ColumnFamilyType;
+import org.apache.cassandra.schema.LegacySchemaTables;
 import org.apache.cassandra.db.WriteType;
-import org.apache.cassandra.exceptions.RequestExecutionException;
-import org.apache.cassandra.exceptions.RequestTimeoutException;
-import org.apache.cassandra.exceptions.RequestValidationException;
-import org.apache.cassandra.exceptions.WriteTimeoutException;
+import org.apache.cassandra.db.composites.CellNameType;
+import org.apache.cassandra.db.composites.CellNames;
+import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.exceptions.*;
+import org.apache.cassandra.io.compress.CompressionParameters;
+import org.apache.cassandra.locator.AbstractReplicationStrategy;
+import org.apache.cassandra.locator.LocalStrategy;
+import org.apache.cassandra.serializers.MarshalException;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.UUIDGen;
 
 /**
  * Static utility methods to convert internal structure to and from thrift ones.
@@ -75,7 +90,9 @@
     // for methods that have a return value.
     public static RuntimeException rethrow(RequestExecutionException e) throws UnavailableException, TimedOutException
     {
-        if (e instanceof RequestTimeoutException)
+        if (e instanceof RequestFailureException)
+            throw toThrift((RequestFailureException)e);
+        else if (e instanceof RequestTimeoutException)
             throw toThrift((RequestTimeoutException)e);
         else
             throw new UnavailableException();
@@ -113,7 +130,13 @@
         return toe;
     }
 
-    public static List<org.apache.cassandra.db.IndexExpression> fromThrift(List<IndexExpression> exprs)
+    // Thrift does not support RequestFailureExceptions, so we translate them into timeouts
+    public static TimedOutException toThrift(RequestFailureException e)
+    {
+        return new TimedOutException();
+    }
+
+    public static List<org.apache.cassandra.db.IndexExpression> indexExpressionsFromThrift(List<IndexExpression> exprs)
     {
         if (exprs == null)
             return null;
@@ -130,4 +153,355 @@
         }
         return converted;
     }
+
+    public static KSMetaData fromThrift(KsDef ksd, CFMetaData... cfDefs) throws ConfigurationException
+    {
+        Class<? extends AbstractReplicationStrategy> cls = AbstractReplicationStrategy.getClass(ksd.strategy_class);
+        if (cls.equals(LocalStrategy.class))
+            throw new ConfigurationException("Unable to use given strategy class: LocalStrategy is reserved for internal use.");
+
+        return new KSMetaData(ksd.name,
+                              cls,
+                              ksd.strategy_options == null ? Collections.<String, String>emptyMap() : ksd.strategy_options,
+                              ksd.durable_writes,
+                              Arrays.asList(cfDefs));
+    }
+
+    public static KsDef toThrift(KSMetaData ksm)
+    {
+        List<CfDef> cfDefs = new ArrayList<>(ksm.cfMetaData().size());
+        for (CFMetaData cfm : ksm.cfMetaData().values())
+            if (cfm.isThriftCompatible()) // Don't expose CF that cannot be correctly handle by thrift; see CASSANDRA-4377 for further details
+                cfDefs.add(toThrift(cfm));
+
+        KsDef ksdef = new KsDef(ksm.name, ksm.strategyClass.getName(), cfDefs);
+        ksdef.setStrategy_options(ksm.strategyOptions);
+        ksdef.setDurable_writes(ksm.durableWrites);
+
+        return ksdef;
+    }
+
+    public static CFMetaData fromThrift(CfDef cf_def)
+    throws org.apache.cassandra.exceptions.InvalidRequestException, ConfigurationException
+    {
+        return internalFromThrift(cf_def, Collections.<ColumnDefinition>emptyList());
+    }
+
+    public static CFMetaData fromThriftForUpdate(CfDef cf_def, CFMetaData toUpdate)
+    throws org.apache.cassandra.exceptions.InvalidRequestException, ConfigurationException
+    {
+        return internalFromThrift(cf_def, toUpdate.allColumns());
+    }
+
+    // Convert a thrift CfDef, given a list of ColumnDefinitions to copy over to the created CFMetadata before the CQL metadata are rebuild
+    private static CFMetaData internalFromThrift(CfDef cf_def, Collection<ColumnDefinition> previousCQLMetadata)
+    throws org.apache.cassandra.exceptions.InvalidRequestException, ConfigurationException
+    {
+        ColumnFamilyType cfType = ColumnFamilyType.create(cf_def.column_type);
+        if (cfType == null)
+            throw new org.apache.cassandra.exceptions.InvalidRequestException("Invalid column type " + cf_def.column_type);
+
+        applyImplicitDefaults(cf_def);
+
+        try
+        {
+            AbstractType<?> rawComparator = TypeParser.parse(cf_def.comparator_type);
+            AbstractType<?> subComparator = cfType == ColumnFamilyType.Standard
+                    ? null
+                    : cf_def.subcomparator_type == null ? BytesType.instance : TypeParser.parse(cf_def.subcomparator_type);
+
+            AbstractType<?> fullRawComparator = CFMetaData.makeRawAbstractType(rawComparator, subComparator);
+
+            AbstractType<?> keyValidator = cf_def.isSetKey_validation_class() ? TypeParser.parse(cf_def.key_validation_class) : null;
+
+            // Convert the REGULAR definitions from the input CfDef
+            List<ColumnDefinition> defs = fromThrift(cf_def.keyspace, cf_def.name, rawComparator, subComparator, cf_def.column_metadata);
+
+            // Add the keyAlias if there is one, since that's on CQL metadata that thrift can actually change (for
+            // historical reasons)
+            boolean hasKeyAlias = cf_def.isSetKey_alias() && keyValidator != null && !(keyValidator instanceof CompositeType);
+            if (hasKeyAlias)
+                defs.add(ColumnDefinition.partitionKeyDef(cf_def.keyspace, cf_def.name, cf_def.key_alias, keyValidator, null));
+
+            // for Thrift updates, we should be calculating denseness from just the regular columns & comparator
+            boolean isDense = CFMetaData.calculateIsDense(fullRawComparator, defs);
+
+            // Now add any CQL metadata that we want to copy, skipping the keyAlias if there was one
+            for (ColumnDefinition def : previousCQLMetadata)
+            {
+                // skip all pre-existing REGULAR columns
+                if (def.kind == ColumnDefinition.Kind.REGULAR)
+                    continue;
+
+                // skip previous PARTITION_KEY column def if key_alias has been set by this update already (overwritten)
+                if (def.kind == ColumnDefinition.Kind.PARTITION_KEY && hasKeyAlias)
+                    continue;
+
+                // the table switched from DENSE to SPARSE by adding one or more REGULAR columns;
+                // in this case we should now drop the COMPACT_VALUE column
+                if (def.kind == ColumnDefinition.Kind.COMPACT_VALUE && !isDense)
+                    continue;
+
+                // skip CLUSTERING_COLUMN column(s) of a sparse table, if:
+                // a) this is a Standard columnfamily *OR* b) it's a Super columnfamily and the second (subcolumn) component;
+                // in other words, only keep the clustering column in sparse tables if it's the first (super) component
+                // of a super column family
+                if (def.kind == ColumnDefinition.Kind.CLUSTERING_COLUMN && !isDense)
+                    if (cfType == ColumnFamilyType.Standard || def.position() != 0)
+                        continue;
+
+                defs.add(def);
+            }
+
+            CellNameType comparator = CellNames.fromAbstractType(fullRawComparator, CFMetaData.calculateIsDense(fullRawComparator, defs));
+
+            UUID cfId = Schema.instance.getId(cf_def.keyspace, cf_def.name);
+            if (cfId == null)
+                cfId = UUIDGen.getTimeUUID();
+
+            // set isDense now so that it doesn't get re-calculated incorrectly later in rebuild() b/c of defined clusterings
+            CFMetaData newCFMD = new CFMetaData(cf_def.keyspace, cf_def.name, cfType, comparator, cfId).isDense(isDense);
+
+            newCFMD.addAllColumnDefinitions(defs);
+
+            if (keyValidator != null)
+                newCFMD.keyValidator(keyValidator);
+            if (cf_def.isSetGc_grace_seconds())
+                newCFMD.gcGraceSeconds(cf_def.gc_grace_seconds);
+            if (cf_def.isSetMin_compaction_threshold())
+                newCFMD.minCompactionThreshold(cf_def.min_compaction_threshold);
+            if (cf_def.isSetMax_compaction_threshold())
+                newCFMD.maxCompactionThreshold(cf_def.max_compaction_threshold);
+            if (cf_def.isSetCompaction_strategy())
+                newCFMD.compactionStrategyClass(CFMetaData.createCompactionStrategy(cf_def.compaction_strategy));
+            if (cf_def.isSetCompaction_strategy_options())
+                newCFMD.compactionStrategyOptions(new HashMap<>(cf_def.compaction_strategy_options));
+            if (cf_def.isSetBloom_filter_fp_chance())
+                newCFMD.bloomFilterFpChance(cf_def.bloom_filter_fp_chance);
+            if (cf_def.isSetMemtable_flush_period_in_ms())
+                newCFMD.memtableFlushPeriod(cf_def.memtable_flush_period_in_ms);
+            if (cf_def.isSetCaching() || cf_def.isSetCells_per_row_to_cache())
+                newCFMD.caching(CachingOptions.fromThrift(cf_def.caching, cf_def.cells_per_row_to_cache));
+            if (cf_def.isSetRead_repair_chance())
+                newCFMD.readRepairChance(cf_def.read_repair_chance);
+            if (cf_def.isSetDefault_time_to_live())
+                newCFMD.defaultTimeToLive(cf_def.default_time_to_live);
+            if (cf_def.isSetDclocal_read_repair_chance())
+                newCFMD.dcLocalReadRepairChance(cf_def.dclocal_read_repair_chance);
+            if (cf_def.isSetMin_index_interval())
+                newCFMD.minIndexInterval(cf_def.min_index_interval);
+            if (cf_def.isSetMax_index_interval())
+                newCFMD.maxIndexInterval(cf_def.max_index_interval);
+            if (cf_def.isSetSpeculative_retry())
+                newCFMD.speculativeRetry(CFMetaData.SpeculativeRetry.fromString(cf_def.speculative_retry));
+            if (cf_def.isSetTriggers())
+                newCFMD.triggers(triggerDefinitionsFromThrift(cf_def.triggers));
+
+            return newCFMD.comment(cf_def.comment)
+                          .defaultValidator(TypeParser.parse(cf_def.default_validation_class))
+                          .compressionParameters(CompressionParameters.create(cf_def.compression_options))
+                          .rebuild();
+        }
+        catch (SyntaxException | MarshalException e)
+        {
+            throw new ConfigurationException(e.getMessage());
+        }
+    }
+
+    /** applies implicit defaults to cf definition. useful in updates */
+    private static void applyImplicitDefaults(org.apache.cassandra.thrift.CfDef cf_def)
+    {
+        if (!cf_def.isSetComment())
+            cf_def.setComment("");
+        if (!cf_def.isSetMin_compaction_threshold())
+            cf_def.setMin_compaction_threshold(CFMetaData.DEFAULT_MIN_COMPACTION_THRESHOLD);
+        if (!cf_def.isSetMax_compaction_threshold())
+            cf_def.setMax_compaction_threshold(CFMetaData.DEFAULT_MAX_COMPACTION_THRESHOLD);
+        if (cf_def.compaction_strategy == null)
+            cf_def.compaction_strategy = CFMetaData.DEFAULT_COMPACTION_STRATEGY_CLASS.getSimpleName();
+        if (cf_def.compaction_strategy_options == null)
+            cf_def.compaction_strategy_options = Collections.emptyMap();
+        if (!cf_def.isSetCompression_options())
+            cf_def.setCompression_options(Collections.singletonMap(CompressionParameters.SSTABLE_COMPRESSION, CFMetaData.DEFAULT_COMPRESSOR));
+        if (!cf_def.isSetDefault_time_to_live())
+            cf_def.setDefault_time_to_live(CFMetaData.DEFAULT_DEFAULT_TIME_TO_LIVE);
+        if (!cf_def.isSetDclocal_read_repair_chance())
+            cf_def.setDclocal_read_repair_chance(CFMetaData.DEFAULT_DCLOCAL_READ_REPAIR_CHANCE);
+
+        // if index_interval was set, use that for the min_index_interval default
+        if (!cf_def.isSetMin_index_interval())
+        {
+            if (cf_def.isSetIndex_interval())
+                cf_def.setMin_index_interval(cf_def.getIndex_interval());
+            else
+                cf_def.setMin_index_interval(CFMetaData.DEFAULT_MIN_INDEX_INTERVAL);
+        }
+
+        if (!cf_def.isSetMax_index_interval())
+        {
+            // ensure the max is at least as large as the min
+            cf_def.setMax_index_interval(Math.max(cf_def.min_index_interval, CFMetaData.DEFAULT_MAX_INDEX_INTERVAL));
+        }
+    }
+
+    /**
+     * Create CFMetaData from thrift {@link CqlRow} that contains columns from schema_columnfamilies.
+     *
+     * @param columnsRes CqlRow containing columns from schema_columnfamilies.
+     * @return CFMetaData derived from CqlRow
+     */
+    public static CFMetaData fromThriftCqlRow(CqlRow cf, CqlResult columnsRes)
+    {
+        UntypedResultSet.Row cfRow = new UntypedResultSet.Row(convertThriftCqlRow(cf));
+
+        List<Map<String, ByteBuffer>> cols = new ArrayList<>(columnsRes.rows.size());
+        for (CqlRow row : columnsRes.rows)
+            cols.add(convertThriftCqlRow(row));
+        UntypedResultSet colsRows = UntypedResultSet.create(cols);
+
+        return LegacySchemaTables.createTableFromTableRowAndColumnRows(cfRow, colsRows);
+    }
+
+    private static Map<String, ByteBuffer> convertThriftCqlRow(CqlRow row)
+    {
+        Map<String, ByteBuffer> m = new HashMap<>();
+        for (org.apache.cassandra.thrift.Column column : row.getColumns())
+            m.put(UTF8Type.instance.getString(column.bufferForName()), column.value);
+        return m;
+    }
+
+    public static CfDef toThrift(CFMetaData cfm)
+    {
+        CfDef def = new CfDef(cfm.ksName, cfm.cfName);
+        def.setColumn_type(cfm.cfType.name());
+
+        if (cfm.isSuper())
+        {
+            def.setComparator_type(cfm.comparator.subtype(0).toString());
+            def.setSubcomparator_type(cfm.comparator.subtype(1).toString());
+        }
+        else
+        {
+            def.setComparator_type(cfm.comparator.toString());
+        }
+
+        def.setComment(Strings.nullToEmpty(cfm.getComment()));
+        def.setRead_repair_chance(cfm.getReadRepairChance());
+        def.setDclocal_read_repair_chance(cfm.getDcLocalReadRepairChance());
+        def.setGc_grace_seconds(cfm.getGcGraceSeconds());
+        def.setDefault_validation_class(cfm.getDefaultValidator().toString());
+        def.setKey_validation_class(cfm.getKeyValidator().toString());
+        def.setMin_compaction_threshold(cfm.getMinCompactionThreshold());
+        def.setMax_compaction_threshold(cfm.getMaxCompactionThreshold());
+        // We only return the alias if only one is set since thrift don't know about multiple key aliases
+        if (cfm.partitionKeyColumns().size() == 1)
+            def.setKey_alias(cfm.partitionKeyColumns().get(0).name.bytes);
+        def.setColumn_metadata(columnDefinitionsToThrift(cfm.allColumns()));
+        def.setCompaction_strategy(cfm.compactionStrategyClass.getName());
+        def.setCompaction_strategy_options(new HashMap<>(cfm.compactionStrategyOptions));
+        def.setCompression_options(cfm.compressionParameters.asThriftOptions());
+        def.setBloom_filter_fp_chance(cfm.getBloomFilterFpChance());
+        def.setMin_index_interval(cfm.getMinIndexInterval());
+        def.setMax_index_interval(cfm.getMaxIndexInterval());
+        def.setMemtable_flush_period_in_ms(cfm.getMemtableFlushPeriod());
+        def.setCaching(cfm.getCaching().toThriftCaching());
+        def.setCells_per_row_to_cache(cfm.getCaching().toThriftCellsPerRow());
+        def.setDefault_time_to_live(cfm.getDefaultTimeToLive());
+        def.setSpeculative_retry(cfm.getSpeculativeRetry().toString());
+        def.setTriggers(triggerDefinitionsToThrift(cfm.getTriggers().values()));
+
+        return def;
+    }
+
+    public static ColumnDefinition fromThrift(String ksName,
+                                              String cfName,
+                                              AbstractType<?> thriftComparator,
+                                              AbstractType<?> thriftSubcomparator,
+                                              ColumnDef thriftColumnDef)
+    throws SyntaxException, ConfigurationException
+    {
+        // For super columns, the componentIndex is 1 because the ColumnDefinition applies to the column component.
+        Integer componentIndex = thriftSubcomparator != null ? 1 : null;
+        AbstractType<?> comparator = thriftSubcomparator == null ? thriftComparator : thriftSubcomparator;
+        try
+        {
+            comparator.validate(thriftColumnDef.name);
+        }
+        catch (MarshalException e)
+        {
+            throw new ConfigurationException(String.format("Column name %s is not valid for comparator %s", ByteBufferUtil.bytesToHex(thriftColumnDef.name), comparator));
+        }
+
+        return new ColumnDefinition(ksName,
+                                    cfName,
+                                    new ColumnIdentifier(ByteBufferUtil.clone(thriftColumnDef.name), comparator),
+                                    TypeParser.parse(thriftColumnDef.validation_class),
+                                    thriftColumnDef.index_type == null ? null : org.apache.cassandra.config.IndexType.valueOf(thriftColumnDef.index_type.name()),
+                                    thriftColumnDef.index_options,
+                                    thriftColumnDef.index_name,
+                                    componentIndex,
+                                    ColumnDefinition.Kind.REGULAR);
+    }
+
+    private static List<ColumnDefinition> fromThrift(String ksName,
+                                                     String cfName,
+                                                     AbstractType<?> thriftComparator,
+                                                     AbstractType<?> thriftSubcomparator,
+                                                     List<ColumnDef> thriftDefs)
+    throws SyntaxException, ConfigurationException
+    {
+        if (thriftDefs == null)
+            return new ArrayList<>();
+
+        List<ColumnDefinition> defs = new ArrayList<>(thriftDefs.size());
+        for (ColumnDef thriftColumnDef : thriftDefs)
+            defs.add(fromThrift(ksName, cfName, thriftComparator, thriftSubcomparator, thriftColumnDef));
+
+        return defs;
+    }
+
+    @VisibleForTesting
+    public static ColumnDef toThrift(ColumnDefinition column)
+    {
+        ColumnDef cd = new ColumnDef();
+
+        cd.setName(ByteBufferUtil.clone(column.name.bytes));
+        cd.setValidation_class(column.type.toString());
+        cd.setIndex_type(column.getIndexType() == null ? null : org.apache.cassandra.thrift.IndexType.valueOf(column.getIndexType().name()));
+        cd.setIndex_name(column.getIndexName());
+        cd.setIndex_options(column.getIndexOptions() == null ? null : Maps.newHashMap(column.getIndexOptions()));
+
+        return cd;
+    }
+
+    private static List<ColumnDef> columnDefinitionsToThrift(Collection<ColumnDefinition> columns)
+    {
+        List<ColumnDef> thriftDefs = new ArrayList<>(columns.size());
+        for (ColumnDefinition def : columns)
+            if (def.kind == ColumnDefinition.Kind.REGULAR)
+                thriftDefs.add(ThriftConversion.toThrift(def));
+        return thriftDefs;
+    }
+
+    private static Map<String, TriggerDefinition> triggerDefinitionsFromThrift(List<TriggerDef> thriftDefs)
+    {
+        Map<String, TriggerDefinition> triggerDefinitions = new HashMap<>();
+        for (TriggerDef thriftDef : thriftDefs)
+            triggerDefinitions.put(thriftDef.getName(),
+                                   new TriggerDefinition(thriftDef.getName(), thriftDef.getOptions().get(TriggerDefinition.CLASS)));
+        return triggerDefinitions;
+    }
+
+    private static List<TriggerDef> triggerDefinitionsToThrift(Collection<TriggerDefinition> triggers)
+    {
+        List<TriggerDef> thriftDefs = new ArrayList<>(triggers.size());
+        for (TriggerDefinition def : triggers)
+        {
+            TriggerDef td = new TriggerDef();
+            td.setName(def.name);
+            td.setOptions(Collections.singletonMap(TriggerDefinition.CLASS, def.classOption));
+            thriftDefs.add(td);
+        }
+        return thriftDefs;
+    }
 }
diff --git a/src/java/org/apache/cassandra/thrift/ThriftServer.java b/src/java/org/apache/cassandra/thrift/ThriftServer.java
index 2409606..44ec524 100644
--- a/src/java/org/apache/cassandra/thrift/ThriftServer.java
+++ b/src/java/org/apache/cassandra/thrift/ThriftServer.java
@@ -60,7 +60,7 @@
         }
     }
 
-    public void stop()
+    public synchronized void stop()
     {
         if (server != null)
         {
diff --git a/src/java/org/apache/cassandra/thrift/ThriftSessionManager.java b/src/java/org/apache/cassandra/thrift/ThriftSessionManager.java
index 6caa558..3603ad5 100644
--- a/src/java/org/apache/cassandra/thrift/ThriftSessionManager.java
+++ b/src/java/org/apache/cassandra/thrift/ThriftSessionManager.java
@@ -17,6 +17,7 @@
  */
 package org.apache.cassandra.thrift;
 
+import java.net.InetSocketAddress;
 import java.net.SocketAddress;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
@@ -58,7 +59,7 @@
         if (cState == null)
         {
             //guarantee atomicity
-            ThriftClientState newState = new ThriftClientState(socket);
+            ThriftClientState newState = new ThriftClientState((InetSocketAddress)socket);
             cState = activeSocketSessions.putIfAbsent(socket, newState);
             if (cState == null)
                 cState = newState;
diff --git a/src/java/org/apache/cassandra/thrift/ThriftValidation.java b/src/java/org/apache/cassandra/thrift/ThriftValidation.java
index d5d9f73..d735676 100644
--- a/src/java/org/apache/cassandra/thrift/ThriftValidation.java
+++ b/src/java/org/apache/cassandra/thrift/ThriftValidation.java
@@ -94,12 +94,12 @@
         if (isCommutativeOp)
         {
             if (!metadata.isCounter())
-                throw new org.apache.cassandra.exceptions.InvalidRequestException("invalid operation for non commutative columnfamily " + cfName);
+                throw new org.apache.cassandra.exceptions.InvalidRequestException("invalid operation for non commutative table " + cfName);
         }
         else
         {
             if (metadata.isCounter())
-                throw new org.apache.cassandra.exceptions.InvalidRequestException("invalid operation for commutative columnfamily " + cfName);
+                throw new org.apache.cassandra.exceptions.InvalidRequestException("invalid operation for commutative table " + cfName);
         }
         return metadata;
     }
@@ -109,11 +109,11 @@
     {
         validateKeyspace(keyspaceName);
         if (cfName.isEmpty())
-            throw new org.apache.cassandra.exceptions.InvalidRequestException("non-empty columnfamily is required");
+            throw new org.apache.cassandra.exceptions.InvalidRequestException("non-empty table is required");
 
         CFMetaData metadata = Schema.instance.getCFMetaData(keyspaceName, cfName);
         if (metadata == null)
-            throw new org.apache.cassandra.exceptions.InvalidRequestException("unconfigured columnfamily " + cfName);
+            throw new org.apache.cassandra.exceptions.InvalidRequestException("unconfigured table " + cfName);
 
         return metadata;
     }
@@ -155,7 +155,7 @@
         {
             if (column_parent.super_column != null)
             {
-                throw new org.apache.cassandra.exceptions.InvalidRequestException("columnfamily alone is required for standard CF " + metadata.cfName);
+                throw new org.apache.cassandra.exceptions.InvalidRequestException("table alone is required for standard CF " + metadata.cfName);
             }
         }
 
@@ -208,7 +208,7 @@
             if (superColumnName.remaining() == 0)
                 throw new org.apache.cassandra.exceptions.InvalidRequestException("supercolumn name must not be empty");
             if (metadata.cfType == ColumnFamilyType.Standard)
-                throw new org.apache.cassandra.exceptions.InvalidRequestException("supercolumn specified to ColumnFamily " + metadata.cfName + " containing normal columns");
+                throw new org.apache.cassandra.exceptions.InvalidRequestException("supercolumn specified to table " + metadata.cfName + " containing normal columns");
         }
         AbstractType<?> comparator = SuperColumns.getComparatorFor(metadata, superColumnName);
         boolean isCQL3Table = !metadata.isThriftCompatible();
@@ -312,7 +312,7 @@
         if (cosc.column != null)
         {
             if (isCommutative)
-                throw new org.apache.cassandra.exceptions.InvalidRequestException("invalid operation for commutative columnfamily " + metadata.cfName);
+                throw new org.apache.cassandra.exceptions.InvalidRequestException("invalid operation for commutative table " + metadata.cfName);
 
             validateTtl(cosc.column);
             validateColumnPath(metadata, new ColumnPath(metadata.cfName).setSuper_column((ByteBuffer)null).setColumn(cosc.column.name));
@@ -322,7 +322,7 @@
         if (cosc.super_column != null)
         {
             if (isCommutative)
-                throw new org.apache.cassandra.exceptions.InvalidRequestException("invalid operation for commutative columnfamily " + metadata.cfName);
+                throw new org.apache.cassandra.exceptions.InvalidRequestException("invalid operation for commutative table " + metadata.cfName);
 
             for (Column c : cosc.super_column.columns)
             {
@@ -334,7 +334,7 @@
         if (cosc.counter_column != null)
         {
             if (!isCommutative)
-                throw new org.apache.cassandra.exceptions.InvalidRequestException("invalid operation for non commutative columnfamily " + metadata.cfName);
+                throw new org.apache.cassandra.exceptions.InvalidRequestException("invalid operation for non commutative table " + metadata.cfName);
 
             validateColumnPath(metadata, new ColumnPath(metadata.cfName).setSuper_column((ByteBuffer)null).setColumn(cosc.counter_column.name));
         }
@@ -342,7 +342,7 @@
         if (cosc.counter_super_column != null)
         {
             if (!isCommutative)
-                throw new org.apache.cassandra.exceptions.InvalidRequestException("invalid operation for non commutative columnfamily " + metadata.cfName);
+                throw new org.apache.cassandra.exceptions.InvalidRequestException("invalid operation for non commutative table " + metadata.cfName);
 
             for (CounterColumn c : cosc.counter_super_column.columns)
                 validateColumnPath(metadata, new ColumnPath(metadata.cfName).setSuper_column(cosc.counter_super_column.name).setColumn(c.name));
@@ -402,7 +402,7 @@
 
         if (metadata.cfType == ColumnFamilyType.Standard && del.super_column != null)
         {
-            String msg = String.format("Deletion of super columns is not possible on a standard ColumnFamily (KeySpace=%s ColumnFamily=%s Deletion=%s)", metadata.ksName, metadata.cfName, del);
+            String msg = String.format("Deletion of super columns is not possible on a standard table (KeySpace=%s Table=%s Deletion=%s)", metadata.ksName, metadata.cfName, del);
             throw new org.apache.cassandra.exceptions.InvalidRequestException(msg);
         }
 
@@ -413,7 +413,7 @@
         }
         else if (!del.isSetTimestamp())
         {
-            throw new org.apache.cassandra.exceptions.InvalidRequestException("Deletion timestamp is not optional for non commutative column family " + metadata.cfName);
+            throw new org.apache.cassandra.exceptions.InvalidRequestException("Deletion timestamp is not optional for non commutative table " + metadata.cfName);
         }
     }
 
@@ -451,8 +451,8 @@
         }
         catch (MarshalException me)
         {
-            if (logger.isDebugEnabled())
-                logger.debug("rejecting invalid value {}", ByteBufferUtil.bytesToHex(summarize(column.value)));
+            if (logger.isTraceEnabled())
+                logger.trace("rejecting invalid value {}", ByteBufferUtil.bytesToHex(summarize(column.value)));
 
             throw new org.apache.cassandra.exceptions.InvalidRequestException(String.format("(%s) [%s][%s][%s] failed validation",
                                                                       me.getMessage(),
@@ -524,7 +524,7 @@
         {
             Token startToken = p.getToken(range.start_key);
             Token endToken = p.getToken(range.end_key);
-            if (startToken.compareTo(endToken) > 0 && !endToken.isMinimum(p))
+            if (startToken.compareTo(endToken) > 0 && !endToken.isMinimum())
             {
                 if (p.preservesOrder())
                     throw new org.apache.cassandra.exceptions.InvalidRequestException("start key must sort before (or equal to) finish key in your partitioner!");
@@ -535,7 +535,7 @@
         else if (range.start_key != null && range.end_token != null)
         {
             // start_token/end_token can wrap, but key/token should not
-            RowPosition stop = p.getTokenFactory().fromString(range.end_token).maxKeyBound(p);
+            RowPosition stop = p.getTokenFactory().fromString(range.end_token).maxKeyBound();
             if (RowPosition.ForKey.get(range.start_key, p).compareTo(stop) > 0 && !stop.isMinimum())
                 throw new org.apache.cassandra.exceptions.InvalidRequestException("Start key's token sorts after end token");
         }
@@ -633,7 +633,7 @@
 
     public static void validateKeyspaceNotSystem(String modifiedKeyspace) throws org.apache.cassandra.exceptions.InvalidRequestException
     {
-        if (modifiedKeyspace.equalsIgnoreCase(Keyspace.SYSTEM_KS))
+        if (modifiedKeyspace.equalsIgnoreCase(SystemKeyspace.NAME))
             throw new org.apache.cassandra.exceptions.InvalidRequestException("system keyspace is not user-modifiable");
     }
 
diff --git a/src/java/org/apache/cassandra/tools/BootstrapMonitor.java b/src/java/org/apache/cassandra/tools/BootstrapMonitor.java
new file mode 100644
index 0000000..9719192
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/BootstrapMonitor.java
@@ -0,0 +1,88 @@
+/*
+ * 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.tools;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.SimpleDateFormat;
+import java.util.concurrent.locks.Condition;
+
+import org.apache.cassandra.utils.concurrent.SimpleCondition;
+import org.apache.cassandra.utils.progress.ProgressEvent;
+import org.apache.cassandra.utils.progress.ProgressEventType;
+import org.apache.cassandra.utils.progress.jmx.JMXNotificationProgressListener;
+
+public class BootstrapMonitor extends JMXNotificationProgressListener
+{
+    private final SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss,SSS");
+    private final PrintStream out;
+    private final Condition condition = new SimpleCondition();
+
+    public BootstrapMonitor(PrintStream out)
+    {
+        this.out = out;
+    }
+
+    public void awaitCompletion() throws InterruptedException
+    {
+        condition.await();
+    }
+
+    @Override
+    public boolean isInterestedIn(String tag)
+    {
+        return "bootstrap".equals(tag);
+    }
+
+    @Override
+    public void handleNotificationLost(long timestamp, String message)
+    {
+        super.handleNotificationLost(timestamp, message);
+    }
+
+    @Override
+    public void handleConnectionClosed(long timestamp, String message)
+    {
+        handleConnectionFailed(timestamp, message);
+    }
+
+    @Override
+    public void handleConnectionFailed(long timestamp, String message)
+    {
+        Exception error = new IOException(String.format("[%s] JMX connection closed. (%s)",
+                                              format.format(timestamp), message));
+        out.println(error.getMessage());
+        condition.signalAll();
+    }
+
+    @Override
+    public void progress(String tag, ProgressEvent event)
+    {
+        ProgressEventType type = event.getType();
+        String message = String.format("[%s] %s", format.format(System.currentTimeMillis()), event.getMessage());
+        if (type == ProgressEventType.PROGRESS)
+        {
+            message = message + " (progress: " + (int)event.getProgressPercentage() + "%)";
+        }
+        out.println(message);
+        if (type == ProgressEventType.COMPLETE)
+        {
+            condition.signalAll();
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/tools/BulkLoadConnectionFactory.java b/src/java/org/apache/cassandra/tools/BulkLoadConnectionFactory.java
index 399344e..aa23f45 100644
--- a/src/java/org/apache/cassandra/tools/BulkLoadConnectionFactory.java
+++ b/src/java/org/apache/cassandra/tools/BulkLoadConnectionFactory.java
@@ -22,8 +22,6 @@
 import java.net.InetSocketAddress;
 import java.net.Socket;
 import java.nio.channels.SocketChannel;
-import java.util.HashSet;
-import java.util.Set;
 
 import org.apache.cassandra.config.EncryptionOptions;
 import org.apache.cassandra.security.SSLFactory;
diff --git a/src/java/org/apache/cassandra/tools/BulkLoader.java b/src/java/org/apache/cassandra/tools/BulkLoader.java
index 0b1a1d4..7d0fdc8 100644
--- a/src/java/org/apache/cassandra/tools/BulkLoader.java
+++ b/src/java/org/apache/cassandra/tools/BulkLoader.java
@@ -18,33 +18,25 @@
 package org.apache.cassandra.tools;
 
 import java.io.File;
-import java.net.*;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.MalformedURLException;
+import java.net.UnknownHostException;
 import java.util.*;
 
-import com.google.common.base.Joiner;
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.Multimap;
-
 import org.apache.commons.cli.*;
-import org.apache.commons.lang3.StringUtils;
 
-import org.apache.thrift.protocol.TBinaryProtocol;
-import org.apache.thrift.protocol.TProtocol;
-import org.apache.thrift.transport.TTransport;
-
-import org.apache.cassandra.auth.IAuthenticator;
+import com.datastax.driver.core.SSLOptions;
+import javax.net.ssl.SSLContext;
 import org.apache.cassandra.config.*;
-import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.SystemKeyspace;
-import org.apache.cassandra.db.marshal.UTF8Type;
-import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.SSTableLoader;
+import org.apache.cassandra.security.SSLFactory;
 import org.apache.cassandra.streaming.*;
-import org.apache.cassandra.thrift.*;
-import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.NativeSSTableLoaderClient;
 import org.apache.cassandra.utils.OutputHandler;
 
 public class BulkLoader
@@ -55,14 +47,12 @@
     private static final String NOPROGRESS_OPTION  = "no-progress";
     private static final String IGNORE_NODES_OPTION  = "ignore";
     private static final String INITIAL_HOST_ADDRESS_OPTION = "nodes";
-    private static final String RPC_PORT_OPTION = "port";
+    private static final String NATIVE_PORT_OPTION = "port";
     private static final String USER_OPTION = "username";
     private static final String PASSWD_OPTION = "password";
     private static final String THROTTLE_MBITS = "throttle";
     private static final String INTER_DC_THROTTLE_MBITS = "inter-dc-throttle";
 
-    private static final String TRANSPORT_FACTORY = "transport-factory";
-
     /* client encryption options */
     private static final String SSL_TRUSTSTORE = "truststore";
     private static final String SSL_TRUSTSTORE_PW = "truststore-password";
@@ -84,13 +74,13 @@
                 options.directory,
                 new ExternalClient(
                         options.hosts,
-                        options.rpcPort,
+                        options.nativePort,
                         options.user,
                         options.passwd,
-                        options.transportFactory,
                         options.storagePort,
                         options.sslStoragePort,
-                        options.serverEncOptions),
+                        options.serverEncOptions,
+                        buildSSLOptions(options.clientEncOptions)),
                 handler,
                 options.connectionsPerHost);
         DatabaseDescriptor.setStreamThroughputOutboundMegabitsPerSec(options.throttle);
@@ -157,8 +147,13 @@
             start = lastTime = System.nanoTime();
         }
 
-        public void onSuccess(StreamState finalState) {}
-        public void onFailure(Throwable t) {}
+        public void onSuccess(StreamState finalState)
+        {
+        }
+
+        public void onFailure(Throwable t)
+        {
+        }
 
         public synchronized void handleStreamEvent(StreamEvent event)
         {
@@ -188,7 +183,7 @@
                 // recalculate progress across all sessions in all hosts and display
                 for (InetAddress peer : sessionsByHost.keySet())
                 {
-                    sb.append("[").append(peer.toString()).append("]");
+                    sb.append("[").append(peer).append("]");
 
                     for (SessionInfo session : sessionsByHost.get(peer))
                     {
@@ -257,14 +252,27 @@
         }
     }
 
-    public static class ExternalClient extends SSTableLoader.Client
+    private static SSLOptions buildSSLOptions(EncryptionOptions.ClientEncryptionOptions clientEncryptionOptions)
     {
-        private final Map<String, CFMetaData> knownCfs = new HashMap<>();
-        private final Set<InetAddress> hosts;
-        private final int rpcPort;
-        private final String user;
-        private final String passwd;
-        private final ITransportFactory transportFactory;
+
+        if (!clientEncryptionOptions.enabled)
+            return null;
+
+        SSLContext sslContext;
+        try
+        {
+            sslContext = SSLFactory.createSSLContext(clientEncryptionOptions, true);
+        }
+        catch (IOException e)
+        {
+            throw new RuntimeException("Could not create SSL Context.", e);
+        }
+
+        return new SSLOptions(sslContext, clientEncryptionOptions.cipher_suites);
+    }
+
+    static class ExternalClient extends NativeSSTableLoaderClient
+    {
         private final int storagePort;
         private final int sslStoragePort;
         private final EncryptionOptions.ServerEncryptionOptions serverEncOptions;
@@ -273,123 +281,22 @@
                               int port,
                               String user,
                               String passwd,
-                              ITransportFactory transportFactory,
                               int storagePort,
                               int sslStoragePort,
-                              EncryptionOptions.ServerEncryptionOptions serverEncryptionOptions)
+                              EncryptionOptions.ServerEncryptionOptions serverEncryptionOptions,
+                              SSLOptions sslOptions)
         {
-            super();
-            this.hosts = hosts;
-            this.rpcPort = port;
-            this.user = user;
-            this.passwd = passwd;
-            this.transportFactory = transportFactory;
+            super(hosts, port, user, passwd, sslOptions);
             this.storagePort = storagePort;
             this.sslStoragePort = sslStoragePort;
             this.serverEncOptions = serverEncryptionOptions;
         }
 
         @Override
-        public void init(String keyspace)
-        {
-            Iterator<InetAddress> hostiter = hosts.iterator();
-            while (hostiter.hasNext())
-            {
-                try
-                {
-                    // Query endpoint to ranges map and schemas from thrift
-                    InetAddress host = hostiter.next();
-                    Cassandra.Client client = createThriftClient(host.getHostAddress(), rpcPort, this.user, this.passwd, this.transportFactory);
-
-                    setPartitioner(client.describe_partitioner());
-                    Token.TokenFactory tkFactory = getPartitioner().getTokenFactory();
-
-                    for (TokenRange tr : client.describe_ring(keyspace))
-                    {
-                        Range<Token> range = new Range<>(tkFactory.fromString(tr.start_token), tkFactory.fromString(tr.end_token), getPartitioner());
-                        for (String ep : tr.endpoints)
-                        {
-                            addRangeForEndpoint(range, InetAddress.getByName(ep));
-                        }
-                    }
-
-                    String cfQuery = String.format("SELECT %s FROM %s.%s WHERE keyspace_name = '%s'",
-                                                   StringUtils.join(getCFColumnsWithoutCollections(), ","),
-                                                   Keyspace.SYSTEM_KS,
-                                                   SystemKeyspace.SCHEMA_COLUMNFAMILIES_CF,
-                                                   keyspace);
-                    CqlResult cfRes = client.execute_cql3_query(ByteBufferUtil.bytes(cfQuery), Compression.NONE, ConsistencyLevel.ONE);
-
-
-                    for (CqlRow row : cfRes.rows)
-                    {
-                        String columnFamily = UTF8Type.instance.getString(row.columns.get(1).bufferForName());
-                        String columnsQuery = String.format("SELECT * FROM %s.%s WHERE keyspace_name = '%s' AND columnfamily_name = '%s'",
-                                                            Keyspace.SYSTEM_KS,
-                                                            SystemKeyspace.SCHEMA_COLUMNS_CF,
-                                                            keyspace,
-                                                            columnFamily);
-                        CqlResult columnsRes = client.execute_cql3_query(ByteBufferUtil.bytes(columnsQuery), Compression.NONE, ConsistencyLevel.ONE);
-
-                        CFMetaData metadata = CFMetaData.fromThriftCqlRow(row, columnsRes);
-                        knownCfs.put(metadata.cfName, metadata);
-                    }
-                    break;
-                }
-                catch (Exception e)
-                {
-                    if (!hostiter.hasNext())
-                        throw new RuntimeException("Could not retrieve endpoint ranges: ", e);
-                }
-            }
-        }
-
-        //Remove dropped_columns since we can't parse collections in v2 which is used by thrift
-        //See CASSANDRA-10700
-        List<String> getCFColumnsWithoutCollections()
-        {
-
-            Iterator<ColumnDefinition> allColumns = CFMetaData.SchemaColumnFamiliesCf.allColumnsInSelectOrder();
-            List<String> selectedColumns = new ArrayList<>();
-
-            while (allColumns.hasNext())
-            {
-                ColumnDefinition def = allColumns.next();
-
-                if (!def.type.isCollection())
-                    selectedColumns.add(UTF8Type.instance.getString(def.name.bytes));
-            }
-
-            return selectedColumns;
-        }
-
-        @Override
         public StreamConnectionFactory getConnectionFactory()
         {
             return new BulkLoadConnectionFactory(storagePort, sslStoragePort, serverEncOptions, false);
         }
-
-        @Override
-        public CFMetaData getCFMetaData(String keyspace, String cfName)
-        {
-            return knownCfs.get(cfName);
-        }
-
-        private static Cassandra.Client createThriftClient(String host, int port, String user, String passwd, ITransportFactory transportFactory) throws Exception
-        {
-            TTransport trans = transportFactory.openTransport(host, port);
-            TProtocol protocol = new TBinaryProtocol(trans);
-            Cassandra.Client client = new Cassandra.Client(protocol);
-            if (user != null && passwd != null)
-            {
-                Map<String, String> credentials = new HashMap<>();
-                credentials.put(IAuthenticator.USERNAME_KEY, user);
-                credentials.put(IAuthenticator.PASSWORD_KEY, passwd);
-                AuthenticationRequest authenticationRequest = new AuthenticationRequest(credentials);
-                client.login(authenticationRequest);
-            }
-            return client;
-        }
     }
 
     static class LoaderOptions
@@ -399,15 +306,14 @@
         public boolean debug;
         public boolean verbose;
         public boolean noProgress;
-        public int rpcPort = 9160;
+        public int nativePort = 9042;
         public String user;
         public String passwd;
         public int throttle = 0;
         public int interDcThrottle = 0;
         public int storagePort;
         public int sslStoragePort;
-        public ITransportFactory transportFactory = new TFramedTransportFactory();
-        public EncryptionOptions encOptions = new EncryptionOptions.ClientEncryptionOptions();
+        public EncryptionOptions.ClientEncryptionOptions clientEncOptions = new EncryptionOptions.ClientEncryptionOptions();
         public int connectionsPerHost = 1;
         public EncryptionOptions.ServerEncryptionOptions serverEncOptions = new EncryptionOptions.ServerEncryptionOptions();
 
@@ -462,8 +368,8 @@
                 opts.verbose = cmd.hasOption(VERBOSE_OPTION);
                 opts.noProgress = cmd.hasOption(NOPROGRESS_OPTION);
 
-                if (cmd.hasOption(RPC_PORT_OPTION))
-                    opts.rpcPort = Integer.parseInt(cmd.getOptionValue(RPC_PORT_OPTION));
+                if (cmd.hasOption(NATIVE_PORT_OPTION))
+                    opts.nativePort = Integer.parseInt(cmd.getOptionValue(NATIVE_PORT_OPTION));
 
                 if (cmd.hasOption(USER_OPTION))
                     opts.user = cmd.getOptionValue(USER_OPTION);
@@ -536,7 +442,7 @@
                 opts.sslStoragePort = config.ssl_storage_port;
                 opts.throttle = config.stream_throughput_outbound_megabits_per_sec;
                 opts.interDcThrottle = config.inter_dc_stream_throughput_outbound_megabits_per_sec;
-                opts.encOptions = config.client_encryption_options;
+                opts.clientEncOptions = config.client_encryption_options;
                 opts.serverEncOptions = config.server_encryption_options;
 
                 if (cmd.hasOption(THROTTLE_MBITS))
@@ -549,53 +455,52 @@
                     opts.interDcThrottle = Integer.parseInt(cmd.getOptionValue(INTER_DC_THROTTLE_MBITS));
                 }
 
+                if (cmd.hasOption(SSL_TRUSTSTORE) || cmd.hasOption(SSL_TRUSTSTORE_PW) ||
+                    cmd.hasOption(SSL_KEYSTORE) || cmd.hasOption(SSL_KEYSTORE_PW))
+                {
+                    opts.clientEncOptions.enabled = true;
+                }
+
                 if (cmd.hasOption(SSL_TRUSTSTORE))
                 {
-                    opts.encOptions.truststore = cmd.getOptionValue(SSL_TRUSTSTORE);
+                    opts.clientEncOptions.truststore = cmd.getOptionValue(SSL_TRUSTSTORE);
                 }
 
                 if (cmd.hasOption(SSL_TRUSTSTORE_PW))
                 {
-                    opts.encOptions.truststore_password = cmd.getOptionValue(SSL_TRUSTSTORE_PW);
+                    opts.clientEncOptions.truststore_password = cmd.getOptionValue(SSL_TRUSTSTORE_PW);
                 }
 
                 if (cmd.hasOption(SSL_KEYSTORE))
                 {
-                    opts.encOptions.keystore = cmd.getOptionValue(SSL_KEYSTORE);
+                    opts.clientEncOptions.keystore = cmd.getOptionValue(SSL_KEYSTORE);
                     // if a keystore was provided, lets assume we'll need to use it
-                    opts.encOptions.require_client_auth = true;
+                    opts.clientEncOptions.require_client_auth = true;
                 }
 
                 if (cmd.hasOption(SSL_KEYSTORE_PW))
                 {
-                    opts.encOptions.keystore_password = cmd.getOptionValue(SSL_KEYSTORE_PW);
+                    opts.clientEncOptions.keystore_password = cmd.getOptionValue(SSL_KEYSTORE_PW);
                 }
 
                 if (cmd.hasOption(SSL_PROTOCOL))
                 {
-                    opts.encOptions.protocol = cmd.getOptionValue(SSL_PROTOCOL);
+                    opts.clientEncOptions.protocol = cmd.getOptionValue(SSL_PROTOCOL);
                 }
 
                 if (cmd.hasOption(SSL_ALGORITHM))
                 {
-                    opts.encOptions.algorithm = cmd.getOptionValue(SSL_ALGORITHM);
+                    opts.clientEncOptions.algorithm = cmd.getOptionValue(SSL_ALGORITHM);
                 }
 
                 if (cmd.hasOption(SSL_STORE_TYPE))
                 {
-                    opts.encOptions.store_type = cmd.getOptionValue(SSL_STORE_TYPE);
+                    opts.clientEncOptions.store_type = cmd.getOptionValue(SSL_STORE_TYPE);
                 }
 
                 if (cmd.hasOption(SSL_CIPHER_SUITES))
                 {
-                    opts.encOptions.cipher_suites = cmd.getOptionValue(SSL_CIPHER_SUITES).split(",");
-                }
-
-                if (cmd.hasOption(TRANSPORT_FACTORY))
-                {
-                    ITransportFactory transportFactory = getTransportFactory(cmd.getOptionValue(TRANSPORT_FACTORY));
-                    configureTransportFactory(transportFactory, opts);
-                    opts.transportFactory = transportFactory;
+                    opts.clientEncOptions.cipher_suites = cmd.getOptionValue(SSL_CIPHER_SUITES).split(",");
                 }
 
                 return opts;
@@ -607,50 +512,6 @@
             }
         }
 
-        private static ITransportFactory getTransportFactory(String transportFactory)
-        {
-            try
-            {
-                Class<?> factory = Class.forName(transportFactory);
-                if (!ITransportFactory.class.isAssignableFrom(factory))
-                    throw new IllegalArgumentException(String.format("transport factory '%s' " +
-                            "not derived from ITransportFactory", transportFactory));
-                return (ITransportFactory) factory.newInstance();
-            }
-            catch (Exception e)
-            {
-                throw new IllegalArgumentException(String.format("Cannot create a transport factory '%s'.", transportFactory), e);
-            }
-        }
-
-        private static void configureTransportFactory(ITransportFactory transportFactory, LoaderOptions opts)
-        {
-            Map<String, String> options = new HashMap<>();
-            // If the supplied factory supports the same set of options as our SSL impl, set those 
-            if (transportFactory.supportedOptions().contains(SSLTransportFactory.TRUSTSTORE))
-                options.put(SSLTransportFactory.TRUSTSTORE, opts.encOptions.truststore);
-            if (transportFactory.supportedOptions().contains(SSLTransportFactory.TRUSTSTORE_PASSWORD))
-                options.put(SSLTransportFactory.TRUSTSTORE_PASSWORD, opts.encOptions.truststore_password);
-            if (transportFactory.supportedOptions().contains(SSLTransportFactory.PROTOCOL))
-                options.put(SSLTransportFactory.PROTOCOL, opts.encOptions.protocol);
-            if (transportFactory.supportedOptions().contains(SSLTransportFactory.CIPHER_SUITES))
-                options.put(SSLTransportFactory.CIPHER_SUITES, Joiner.on(',').join(opts.encOptions.cipher_suites));
-
-            if (transportFactory.supportedOptions().contains(SSLTransportFactory.KEYSTORE)
-                    && opts.encOptions.require_client_auth)
-                options.put(SSLTransportFactory.KEYSTORE, opts.encOptions.keystore);
-            if (transportFactory.supportedOptions().contains(SSLTransportFactory.KEYSTORE_PASSWORD)
-                    && opts.encOptions.require_client_auth)
-                options.put(SSLTransportFactory.KEYSTORE_PASSWORD, opts.encOptions.keystore_password);
-
-            // Now check if any of the factory's supported options are set as system properties
-            for (String optionKey : transportFactory.supportedOptions())
-                if (System.getProperty(optionKey) != null)
-                    options.put(optionKey, System.getProperty(optionKey));
-
-            transportFactory.setOptions(options);
-        }
-
         private static void errorMsg(String msg, CmdLineOptions options)
         {
             System.err.println(msg);
@@ -666,12 +527,11 @@
             options.addOption(null, NOPROGRESS_OPTION,   "don't display progress");
             options.addOption("i",  IGNORE_NODES_OPTION, "NODES", "don't stream to this (comma separated) list of nodes");
             options.addOption("d",  INITIAL_HOST_ADDRESS_OPTION, "initial hosts", "Required. try to connect to these hosts (comma separated) initially for ring information");
-            options.addOption("p",  RPC_PORT_OPTION, "rpc port", "port used for rpc (default 9160)");
+            options.addOption("p",  NATIVE_PORT_OPTION, "rpc port", "port used for native connection (default 9042)");
             options.addOption("t",  THROTTLE_MBITS, "throttle", "throttle speed in Mbits (default unlimited)");
             options.addOption("idct",  INTER_DC_THROTTLE_MBITS, "inter-dc-throttle", "inter-datacenter throttle speed in Mbits (default unlimited)");
             options.addOption("u",  USER_OPTION, "username", "username for cassandra authentication");
             options.addOption("pw", PASSWD_OPTION, "password", "password for cassandra authentication");
-            options.addOption("tf", TRANSPORT_FACTORY, "transport factory", "Fully-qualified ITransportFactory class name for creating a connection to cassandra");
             options.addOption("cph", CONNECTIONS_PER_HOST, "connectionsPerHost", "number of concurrent connections-per-host.");
             // ssl connection-related options
             options.addOption("ts", SSL_TRUSTSTORE, "TRUSTSTORE", "Client SSL: full path to truststore");
diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java
index 13c7acf..24c5874 100644
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@ -17,8 +17,6 @@
  */
 package org.apache.cassandra.tools;
 
-import static org.apache.commons.lang3.ArrayUtils.isEmpty;
-
 import java.io.IOException;
 import java.io.PrintStream;
 import java.lang.management.ManagementFactory;
@@ -29,11 +27,8 @@
 import java.net.UnknownHostException;
 import java.rmi.server.RMIClientSocketFactory;
 import java.rmi.server.RMISocketFactory;
-import java.text.SimpleDateFormat;
 import java.util.AbstractMap;
 import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
@@ -45,24 +40,19 @@
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
-import java.util.concurrent.locks.Condition;
 
 import javax.management.JMX;
 import javax.management.MBeanServerConnection;
 import javax.management.MalformedObjectNameException;
-import javax.management.Notification;
-import javax.management.NotificationListener;
 import javax.management.ObjectName;
 import javax.management.openmbean.CompositeData;
 import javax.management.openmbean.OpenDataException;
 import javax.management.openmbean.TabularData;
-import javax.management.remote.JMXConnectionNotification;
 import javax.management.remote.JMXConnector;
 import javax.management.remote.JMXConnectorFactory;
 import javax.management.remote.JMXServiceURL;
 import javax.rmi.ssl.SslRMIClientSocketFactory;
 
-import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutorMBean;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStoreMBean;
 import org.apache.cassandra.db.HintedHandOffManager;
@@ -71,12 +61,15 @@
 import org.apache.cassandra.db.compaction.CompactionManagerMBean;
 import org.apache.cassandra.gms.FailureDetector;
 import org.apache.cassandra.gms.FailureDetectorMBean;
+import org.apache.cassandra.gms.Gossiper;
+import org.apache.cassandra.gms.GossiperMBean;
 import org.apache.cassandra.locator.EndpointSnitchInfoMBean;
+import org.apache.cassandra.metrics.CassandraMetricsRegistry;
 import org.apache.cassandra.metrics.ColumnFamilyMetrics.Sampler;
+import org.apache.cassandra.metrics.StorageMetrics;
+import org.apache.cassandra.metrics.ThreadPoolMetrics;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.net.MessagingServiceMBean;
-import org.apache.cassandra.repair.RepairParallelism;
-import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.service.CacheService;
 import org.apache.cassandra.service.CacheServiceMBean;
 import org.apache.cassandra.service.GCInspector;
@@ -87,16 +80,13 @@
 import org.apache.cassandra.streaming.StreamManagerMBean;
 import org.apache.cassandra.streaming.StreamState;
 import org.apache.cassandra.streaming.management.StreamStateCompositeData;
-import org.apache.cassandra.utils.EstimatedHistogram;
-import org.apache.cassandra.utils.JVMStabilityInspector;
-import org.apache.cassandra.utils.concurrent.SimpleCondition;
 
 import com.google.common.base.Function;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Maps;
+import com.google.common.collect.Multimap;
 import com.google.common.collect.Sets;
 import com.google.common.util.concurrent.Uninterruptibles;
-import com.yammer.metrics.reporting.JmxReporter;
 
 /**
  * JMX client operations for Cassandra.
@@ -115,6 +105,7 @@
     private MBeanServerConnection mbeanServerConn;
     private CompactionManagerMBean compactionProxy;
     private StorageServiceMBean ssProxy;
+    private GossiperMBean gossProxy;
     private MemoryMXBean memProxy;
     private GCInspectorMXBean gcProxy;
     private RuntimeMXBean runtimeProxy;
@@ -212,6 +203,8 @@
             hhProxy = JMX.newMBeanProxy(mbeanServerConn, name, HintedHandOffManagerMBean.class);
             name = new ObjectName(GCInspector.MBEAN_NAME);
             gcProxy = JMX.newMBeanProxy(mbeanServerConn, name, GCInspectorMXBean.class);
+            name = new ObjectName(Gossiper.MBEAN_NAME);
+            gossProxy = JMX.newMBeanProxy(mbeanServerConn, name, GossiperMBean.class);
         }
         catch (MalformedObjectNameException e)
         {
@@ -248,6 +241,11 @@
         return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies);
     }
 
+    public int verify(boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    {
+        return ssProxy.verify(extendedVerify, keyspaceName, columnFamilies);
+    }
+
     public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
         return ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, columnFamilies);
@@ -261,10 +259,11 @@
 
     public void forceKeyspaceCleanup(PrintStream out, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
+        checkJobs(out, jobs);
         if (forceKeyspaceCleanup(jobs, keyspaceName, columnFamilies) != 0)
         {
             failed = true;
-            out.println("Aborted cleaning up atleast one column family in keyspace "+keyspaceName+", check server logs for more information.");
+            out.println("Aborted cleaning up at least one table in keyspace "+keyspaceName+", check server logs for more information.");
         }
     }
 
@@ -274,24 +273,34 @@
         if (scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies) != 0)
         {
             failed = true;
-            out.println("Aborted scrubbing atleast one column family in keyspace "+keyspaceName+", check server logs for more information.");
+            out.println("Aborted scrubbing at least one table in keyspace "+keyspaceName+", check server logs for more information.");
         }
     }
 
+    public void verify(PrintStream out, boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    {
+        if (verify(extendedVerify, keyspaceName, columnFamilies) != 0)
+        {
+            failed = true;
+            out.println("Aborted verifying at least one table in keyspace "+keyspaceName+", check server logs for more information.");
+        }
+    }
+
+
     public void upgradeSSTables(PrintStream out, String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
         checkJobs(out, jobs);
         if (upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, columnFamilies) != 0)
         {
             failed = true;
-            out.println("Aborted upgrading sstables for atleast one column family in keyspace "+keyspaceName+", check server logs for more information.");
+            out.println("Aborted upgrading sstables for atleast one table in keyspace "+keyspaceName+", check server logs for more information.");
         }
     }
 
 
-    public void forceKeyspaceCompaction(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public void forceKeyspaceCompaction(boolean splitOutput, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        ssProxy.forceKeyspaceCompaction(keyspaceName, columnFamilies);
+        ssProxy.forceKeyspaceCompaction(splitOutput, keyspaceName, columnFamilies);
     }
 
     public void forceKeyspaceFlush(String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
@@ -299,54 +308,14 @@
         ssProxy.forceKeyspaceFlush(keyspaceName, columnFamilies);
     }
 
-    public void forceRepairAsync(final PrintStream out, final String keyspaceName, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts, boolean primaryRange, boolean fullRepair, String... columnFamilies) throws IOException
+    public void repairAsync(final PrintStream out, final String keyspace, Map<String, String> options) throws IOException
     {
-        forceRepairAsync(out, keyspaceName, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, dataCenters, hosts, primaryRange, fullRepair, columnFamilies);
-    }
-
-    public void forceRepairAsync(final PrintStream out, final String keyspaceName, RepairParallelism parallelismDegree, Collection<String> dataCenters, final Collection<String> hosts, boolean primaryRange, boolean fullRepair, String... columnFamilies) throws IOException
-    {
-        RepairRunner runner = new RepairRunner(out, keyspaceName, columnFamilies);
+        RepairRunner runner = new RepairRunner(out, ssProxy, keyspace, options);
         try
         {
             jmxc.addConnectionNotificationListener(runner, null, null);
             ssProxy.addNotificationListener(runner, null, null);
-            if (!runner.repairAndWait(ssProxy, parallelismDegree, dataCenters, hosts, primaryRange, fullRepair))
-                failed = true;
-        }
-        catch (Exception e)
-        {
-            throw new IOException(e) ;
-        }
-        finally
-        {
-            try
-            {
-                ssProxy.removeNotificationListener(runner);
-                jmxc.removeConnectionNotificationListener(runner);
-            }
-            catch (Throwable t)
-            {
-                JVMStabilityInspector.inspectThrowable(t);
-                out.println("Exception occurred during clean-up. " + t);
-            }
-        }
-    }
-
-    public void forceRepairRangeAsync(final PrintStream out, final String keyspaceName, boolean isSequential, Collection<String> dataCenters, Collection<String> hosts, final String startToken, final String endToken, boolean fullRepair, String... columnFamilies) throws IOException
-    {
-        forceRepairRangeAsync(out, keyspaceName, isSequential ? RepairParallelism.SEQUENTIAL : RepairParallelism.PARALLEL, dataCenters, hosts, startToken, endToken, fullRepair, columnFamilies);
-    }
-
-    public void forceRepairRangeAsync(final PrintStream out, final String keyspaceName, RepairParallelism parallelismDegree, Collection<String> dataCenters, final Collection<String> hosts, final String startToken, final String endToken, boolean fullRepair, String... columnFamilies) throws IOException
-    {
-        RepairRunner runner = new RepairRunner(out, keyspaceName, columnFamilies);
-        try
-        {
-            jmxc.addConnectionNotificationListener(runner, null, null);
-            ssProxy.addNotificationListener(runner, null, null);
-            if (!runner.repairRangeAndWait(ssProxy, parallelismDegree, dataCenters, hosts, startToken, endToken, fullRepair))
-                failed = true;
+            runner.run();
         }
         catch (Exception e)
         {
@@ -552,7 +521,7 @@
         {
             if (keyspaces.length != 1)
             {
-                throw new IOException("When specifying the column family for a snapshot, you must specify one and only one keyspace");
+                throw new IOException("When specifying the table for a snapshot, you must specify one and only one keyspace");
             }
             ssProxy.takeColumnFamilySnapshot(keyspaces[0], columnFamily, snapshotName);
         }
@@ -562,10 +531,10 @@
 
     /**
      * Take a snapshot of all column family from different keyspaces.
-     * 
+     *
      * @param snapshotName
      *            the name of the snapshot.
-     * @param columnfamilylist
+     * @param columnFamilyList
      *            list of columnfamily from different keyspace in the form of ks1.cf1 ks2.cf2
      */
     public void takeMultipleColumnFamilySnapshot(String snapshotName, String... columnFamilyList)
@@ -635,20 +604,9 @@
         ssProxy.forceRemoveCompletion();
     }
 
-    public Iterator<Map.Entry<String, JMXEnabledThreadPoolExecutorMBean>> getThreadPoolMBeanProxies()
+    public void assassinateEndpoint(String address) throws UnknownHostException
     {
-        try
-        {
-            return new ThreadPoolProxyMBeanIterator(mbeanServerConn);
-        }
-        catch (MalformedObjectNameException e)
-        {
-            throw new RuntimeException("Invalid ObjectName? Please report this as a bug.", e);
-        }
-        catch (IOException e)
-        {
-            throw new RuntimeException("Could not retrieve list of stat mbeans.", e);
-        }
+        gossProxy.assassinateEndpoint(address);
     }
 
     /**
@@ -822,26 +780,12 @@
 
     public String getDataCenter()
     {
-        try
-        {
-            return getEndpointSnitchInfoProxy().getDatacenter(getEndpoint());
-        }
-        catch (UnknownHostException e)
-        {
-            return "Unknown";
-        }
+        return getEndpointSnitchInfoProxy().getDatacenter();
     }
 
     public String getRack()
     {
-        try
-        {
-            return getEndpointSnitchInfoProxy().getRack(getEndpoint());
-        }
-        catch (UnknownHostException e)
-        {
-            return "Unknown";
-        }
+        return getEndpointSnitchInfoProxy().getRack();
     }
 
     public List<String> getKeyspaces()
@@ -849,6 +793,11 @@
         return ssProxy.getKeyspaces();
     }
 
+    public List<String> getNonSystemKeyspaces()
+    {
+        return ssProxy.getNonSystemKeyspaces();
+    }
+
     public String getClusterName()
     {
         return ssProxy.getClusterName();
@@ -900,11 +849,7 @@
         {
             hhProxy.truncateAllHints();
         }
-        catch (ExecutionException e)
-        {
-            throw new RuntimeException("Error while executing truncate hints", e);
-        }
-        catch (InterruptedException e)
+        catch (ExecutionException | InterruptedException e)
         {
             throw new RuntimeException("Error while executing truncate hints", e);
         }
@@ -997,9 +942,14 @@
         return ssProxy.getInterDCStreamThroughputMbPerSec();
     }
 
+    public double getTraceProbability()
+    {
+        return ssProxy.getTraceProbability();
+    }
+
     public int getExceptionCount()
     {
-        return ssProxy.getExceptionCount();
+        return (int)StorageMetrics.exceptions.getCount();
     }
 
     public Map<String, Integer> getDroppedMessages()
@@ -1027,6 +977,11 @@
         compactionProxy.stopCompaction(string);
     }
 
+    public void stopById(String compactionId)
+    {
+        compactionProxy.stopCompactionById(compactionId);
+    }
+
     public void setStreamThroughput(int value)
     {
         ssProxy.setStreamThroughputMbPerSec(value);
@@ -1105,12 +1060,12 @@
                 case "Size":
                     return JMX.newMBeanProxy(mbeanServerConn,
                             new ObjectName("org.apache.cassandra.metrics:type=Cache,scope=" + cacheType + ",name=" + metricName),
-                            JmxReporter.GaugeMBean.class).getValue();
+                            CassandraMetricsRegistry.JmxGaugeMBean.class).getValue();
                 case "Requests":
                 case "Hits":
                     return JMX.newMBeanProxy(mbeanServerConn,
                             new ObjectName("org.apache.cassandra.metrics:type=Cache,scope=" + cacheType + ",name=" + metricName),
-                            JmxReporter.MeterMBean.class).getCount();
+                            CassandraMetricsRegistry.JmxMeterMBean.class).getCount();
                 default:
                     throw new RuntimeException("Unknown cache metric name.");
 
@@ -1122,6 +1077,20 @@
         }
     }
 
+    public Object getThreadPoolMetric(String pathName, String poolName, String metricName)
+    {
+        return ThreadPoolMetrics.getJmxMetric(mbeanServerConn, pathName, poolName, metricName);
+    }
+
+    /**
+     * Retrieve threadpool paths and names for threadpools with metrics.
+     * @return Multimap from path (internal, request, etc.) to name
+     */
+    public Multimap<String, String> getThreadPools()
+    {
+        return ThreadPoolMetrics.getJmxThreadPools(mbeanServerConn);
+    }
+
     /**
      * Retrieve ColumnFamily metrics
      * @param ks Keyspace for which stats are to be displayed.
@@ -1157,7 +1126,7 @@
                 case "RecentBloomFilterFalsePositives":
                 case "RecentBloomFilterFalseRatio":
                 case "SnapshotsSize":
-                    return JMX.newMBeanProxy(mbeanServerConn, oName, JmxReporter.GaugeMBean.class).getValue();
+                    return JMX.newMBeanProxy(mbeanServerConn, oName, CassandraMetricsRegistry.JmxGaugeMBean.class).getValue();
                 case "LiveDiskSpaceUsed":
                 case "MemtableSwitchCount":
                 case "SpeculativeRetries":
@@ -1165,18 +1134,18 @@
                 case "WriteTotalLatency":
                 case "ReadTotalLatency":
                 case "PendingFlushes":
-                    return JMX.newMBeanProxy(mbeanServerConn, oName, JmxReporter.CounterMBean.class).getCount();
-                case "ReadLatency":
+                    return JMX.newMBeanProxy(mbeanServerConn, oName, CassandraMetricsRegistry.JmxCounterMBean.class).getCount();
                 case "CoordinatorReadLatency":
                 case "CoordinatorScanLatency":
+                case "ReadLatency":
                 case "WriteLatency":
-                    return JMX.newMBeanProxy(mbeanServerConn, oName, JmxReporter.TimerMBean.class);
+                    return JMX.newMBeanProxy(mbeanServerConn, oName, CassandraMetricsRegistry.JmxTimerMBean.class);
                 case "LiveScannedHistogram":
                 case "SSTablesPerReadHistogram":
                 case "TombstoneScannedHistogram":
-                    return JMX.newMBeanProxy(mbeanServerConn, oName, JmxReporter.HistogramMBean.class);
+                    return JMX.newMBeanProxy(mbeanServerConn, oName, CassandraMetricsRegistry.JmxHistogramMBean.class);
                 default:
-                    throw new RuntimeException("Unknown column family metric.");
+                    throw new RuntimeException("Unknown table metric.");
             }
         }
         catch (MalformedObjectNameException e)
@@ -1189,13 +1158,13 @@
      * Retrieve Proxy metrics
      * @param scope RangeSlice, Read or Write
      */
-    public JmxReporter.TimerMBean getProxyMetric(String scope)
+    public CassandraMetricsRegistry.JmxTimerMBean getProxyMetric(String scope)
     {
         try
         {
             return JMX.newMBeanProxy(mbeanServerConn,
                     new ObjectName("org.apache.cassandra.metrics:type=ClientRequest,scope=" + scope + ",name=Latency"),
-                    JmxReporter.TimerMBean.class);
+                    CassandraMetricsRegistry.JmxTimerMBean.class);
         }
         catch (MalformedObjectNameException e)
         {
@@ -1216,16 +1185,16 @@
                 case "BytesCompacted":
                     return JMX.newMBeanProxy(mbeanServerConn,
                             new ObjectName("org.apache.cassandra.metrics:type=Compaction,name=" + metricName),
-                            JmxReporter.CounterMBean.class);
+                            CassandraMetricsRegistry.JmxCounterMBean.class);
                 case "CompletedTasks":
                 case "PendingTasks":
                     return JMX.newMBeanProxy(mbeanServerConn,
                             new ObjectName("org.apache.cassandra.metrics:type=Compaction,name=" + metricName),
-                            JmxReporter.GaugeMBean.class).getValue();
+                            CassandraMetricsRegistry.JmxGaugeMBean.class).getValue();
                 case "TotalCompactionsCompleted":
                     return JMX.newMBeanProxy(mbeanServerConn,
                             new ObjectName("org.apache.cassandra.metrics:type=Compaction,name=" + metricName),
-                            JmxReporter.MeterMBean.class);
+                            CassandraMetricsRegistry.JmxMeterMBean.class);
                 default:
                     throw new RuntimeException("Unknown compaction metric.");
             }
@@ -1246,7 +1215,7 @@
         {
             return JMX.newMBeanProxy(mbeanServerConn,
                     new ObjectName("org.apache.cassandra.metrics:type=Storage,name=" + metricName),
-                    JmxReporter.CounterMBean.class).getCount();
+                    CassandraMetricsRegistry.JmxCounterMBean.class).getCount();
         }
         catch (MalformedObjectNameException e)
         {
@@ -1254,34 +1223,26 @@
         }
     }
 
-    public double[] metricPercentilesAsArray(long[] counts)
+    public double[] metricPercentilesAsArray(CassandraMetricsRegistry.JmxHistogramMBean metric)
     {
-        double[] result = new double[7];
+        return new double[]{ metric.get50thPercentile(),
+                metric.get75thPercentile(),
+                metric.get95thPercentile(),
+                metric.get98thPercentile(),
+                metric.get99thPercentile(),
+                metric.getMin(),
+                metric.getMax()};
+    }
 
-        if (isEmpty(counts))
-        {
-            Arrays.fill(result, Double.NaN);
-            return result;
-        }
-
-        double[] offsetPercentiles = new double[] { 0.5, 0.75, 0.95, 0.98, 0.99 };
-        EstimatedHistogram metric = new EstimatedHistogram(counts);
-
-        if (metric.isOverflowed())
-        {
-            System.err.println(String.format("EstimatedHistogram overflowed larger than %s, unable to calculate percentiles",
-                                             metric.getLargestBucketOffset()));
-            for (int i = 0; i < result.length; i++)
-                result[i] = Double.NaN;
-        }
-        else
-        {
-            for (int i = 0; i < offsetPercentiles.length; i++)
-                result[i] = metric.percentile(offsetPercentiles[i]);
-        }
-        result[5] = metric.min();
-        result[6] = metric.max();
-        return result;
+    public double[] metricPercentilesAsArray(CassandraMetricsRegistry.JmxTimerMBean metric)
+    {
+        return new double[]{ metric.get50thPercentile(),
+                metric.get75thPercentile(),
+                metric.get95thPercentile(),
+                metric.get98thPercentile(),
+                metric.get99thPercentile(),
+                metric.getMin(),
+                metric.getMax()};
     }
 
     public TabularData getCompactionHistory()
@@ -1302,7 +1263,7 @@
         }
         catch (Exception e)
         {
-          throw new RuntimeException("Error setting log for " + classQualifier +" on level " + level +". Please check logback configuration and ensure to have <jmxConfigurator /> set", e); 
+          throw new RuntimeException("Error setting log for " + classQualifier +" on level " + level +". Please check logback configuration and ensure to have <jmxConfigurator /> set", e);
         }
     }
 
@@ -1310,6 +1271,53 @@
     {
         return ssProxy.getLoggingLevels();
     }
+
+    public void resumeBootstrap(PrintStream out) throws IOException
+    {
+        BootstrapMonitor monitor = new BootstrapMonitor(out);
+        try
+        {
+            jmxc.addConnectionNotificationListener(monitor, null, null);
+            ssProxy.addNotificationListener(monitor, null, null);
+            if (ssProxy.resumeBootstrap())
+            {
+                out.println("Resuming bootstrap");
+                monitor.awaitCompletion();
+            }
+            else
+            {
+                out.println("Node is already bootstrapped.");
+            }
+        }
+        catch (Exception e)
+        {
+            throw new IOException(e);
+        }
+        finally
+        {
+            try
+            {
+                ssProxy.removeNotificationListener(monitor);
+                jmxc.removeConnectionNotificationListener(monitor);
+            }
+            catch (Throwable e)
+            {
+                out.println("Exception occurred during clean-up. " + e);
+            }
+        }
+    }
+
+    public TabularData getFailureDetectorPhilValues()
+    {
+        try
+        {
+            return fdProxy.getPhiValues();
+        }
+        catch (OpenDataException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
 }
 
 class ColumnFamilyStoreMBeanIterator implements Iterator<Map.Entry<String, ColumnFamilyStoreMBean>>
@@ -1335,7 +1343,7 @@
                 // get CF name and split it for index name
                 String e1CF[] = e1.getValue().getColumnFamilyName().split("\\.");
                 String e2CF[] = e2.getValue().getColumnFamilyName().split("\\.");
-                assert e1CF.length <= 2 && e2CF.length <= 2 : "unexpected split count for column family name";
+                assert e1CF.length <= 2 && e2CF.length <= 2 : "unexpected split count for table name";
 
                 //if neither are indexes, just compare CF names
                 if(e1CF.length == 1 && e2CF.length == 1)
@@ -1387,122 +1395,3 @@
         throw new UnsupportedOperationException();
     }
 }
-
-class ThreadPoolProxyMBeanIterator implements Iterator<Map.Entry<String, JMXEnabledThreadPoolExecutorMBean>>
-{
-    private final Iterator<ObjectName> resIter;
-    private final MBeanServerConnection mbeanServerConn;
-
-    public ThreadPoolProxyMBeanIterator(MBeanServerConnection mbeanServerConn)
-    throws MalformedObjectNameException, NullPointerException, IOException
-    {
-        Set<ObjectName> requests = mbeanServerConn.queryNames(new ObjectName("org.apache.cassandra.request:type=*"), null);
-        Set<ObjectName> internal = mbeanServerConn.queryNames(new ObjectName("org.apache.cassandra.internal:type=*"), null);
-        Set<ObjectName> transport = mbeanServerConn.queryNames(new ObjectName("org.apache.cassandra.transport:type=*"), null);
-        resIter = Iterables.concat(requests, internal, transport).iterator();
-        this.mbeanServerConn = mbeanServerConn;
-    }
-
-    public boolean hasNext()
-    {
-        return resIter.hasNext();
-    }
-
-    public Map.Entry<String, JMXEnabledThreadPoolExecutorMBean> next()
-    {
-        ObjectName objectName = resIter.next();
-        String poolName = objectName.getKeyProperty("type");
-        JMXEnabledThreadPoolExecutorMBean threadPoolProxy = JMX.newMBeanProxy(mbeanServerConn, objectName, JMXEnabledThreadPoolExecutorMBean.class);
-        return new AbstractMap.SimpleImmutableEntry<String, JMXEnabledThreadPoolExecutorMBean>(poolName, threadPoolProxy);
-    }
-
-    public void remove()
-    {
-        throw new UnsupportedOperationException();
-    }
-}
-
-class RepairRunner implements NotificationListener
-{
-    private final SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss,SSS");
-    private final Condition condition = new SimpleCondition();
-    private final PrintStream out;
-    private final String keyspace;
-    private final String[] columnFamilies;
-    private int cmd;
-    private volatile boolean success = true;
-    private volatile Exception error = null;
-
-    RepairRunner(PrintStream out, String keyspace, String... columnFamilies)
-    {
-        this.out = out;
-        this.keyspace = keyspace;
-        this.columnFamilies = columnFamilies;
-    }
-
-    public boolean repairAndWait(StorageServiceMBean ssProxy, RepairParallelism parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, boolean primaryRangeOnly, boolean fullRepair) throws Exception
-    {
-        cmd = ssProxy.forceRepairAsync(keyspace, parallelismDegree.ordinal(), dataCenters, hosts, primaryRangeOnly, fullRepair, columnFamilies);
-        waitForRepair();
-        return success;
-    }
-
-    public boolean repairRangeAndWait(StorageServiceMBean ssProxy, RepairParallelism parallelismDegree, Collection<String> dataCenters, Collection<String> hosts, String startToken, String endToken, boolean fullRepair) throws Exception
-    {
-        cmd = ssProxy.forceRepairRangeAsync(startToken, endToken, keyspace, parallelismDegree.ordinal(), dataCenters, hosts, fullRepair, columnFamilies);
-        waitForRepair();
-        return success;
-    }
-
-    private void waitForRepair() throws Exception
-    {
-        if (cmd > 0)
-        {
-            condition.await();
-        }
-        else
-        {
-            String message = String.format("[%s] Nothing to repair for keyspace '%s'", format.format(System.currentTimeMillis()), keyspace);
-            out.println(message);
-        }
-        if (error != null)
-        {
-            throw error;
-        }
-    }
-
-    public void handleNotification(Notification notification, Object handback)
-    {
-        if ("repair".equals(notification.getType()))
-        {
-            int[] status = (int[]) notification.getUserData();
-            assert status.length == 2;
-            if (cmd == status[0])
-            {
-                String message = String.format("[%s] %s", format.format(notification.getTimeStamp()), notification.getMessage());
-                out.println(message);
-                // repair status is int array with [0] = cmd number, [1] = status
-                if (status[1] == ActiveRepairService.Status.SESSION_FAILED.ordinal())
-                    success = false;
-                else if (status[1] == ActiveRepairService.Status.FINISHED.ordinal())
-                    condition.signalAll();
-            }
-        }
-        else if (JMXConnectionNotification.NOTIFS_LOST.equals(notification.getType()))
-        {
-            String message = String.format("[%s] Lost notification. You should check server log for repair status of keyspace %s",
-                                           format.format(notification.getTimeStamp()),
-                                           keyspace);
-            out.println(message);
-        }
-        else if (JMXConnectionNotification.FAILED.equals(notification.getType())
-                 || JMXConnectionNotification.CLOSED.equals(notification.getType()))
-        {
-            String message = String.format("JMX connection closed. You should check server log for repair status of keyspace %s"
-                                           + "(Subsequent keyspaces are not going to be repaired).",
-                                           keyspace);
-            error = new IOException(message);
-            condition.signalAll();
-        }
-    }
-}
diff --git a/src/java/org/apache/cassandra/tools/NodeTool.java b/src/java/org/apache/cassandra/tools/NodeTool.java
index d1afb6f..966b785 100644
--- a/src/java/org/apache/cassandra/tools/NodeTool.java
+++ b/src/java/org/apache/cassandra/tools/NodeTool.java
@@ -17,91 +17,30 @@
  */
 package org.apache.cassandra.tools;
 
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkState;
+import java.io.*;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.text.SimpleDateFormat;
+import java.util.*;
+import java.util.Map.Entry;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Throwables;
+import com.google.common.collect.*;
+
+import io.airlift.command.*;
+
+import org.apache.cassandra.locator.EndpointSnitchInfoMBean;
+import org.apache.cassandra.tools.nodetool.*;
+import org.apache.cassandra.utils.FBUtilities;
+
 import static com.google.common.base.Throwables.getStackTraceAsString;
 import static com.google.common.collect.Iterables.toArray;
 import static com.google.common.collect.Lists.newArrayList;
 import static java.lang.Integer.parseInt;
 import static java.lang.String.format;
 import static org.apache.commons.lang3.ArrayUtils.EMPTY_STRING_ARRAY;
-import static org.apache.commons.lang3.StringUtils.EMPTY;
-import static org.apache.commons.lang3.StringUtils.isEmpty;
-import static org.apache.commons.lang3.StringUtils.isNotEmpty;
-import static org.apache.commons.lang3.StringUtils.join;
-import io.airlift.command.Arguments;
-import io.airlift.command.Cli;
-import io.airlift.command.Command;
-import io.airlift.command.Help;
-import io.airlift.command.Option;
-import io.airlift.command.OptionType;
-import io.airlift.command.ParseArgumentsMissingException;
-import io.airlift.command.ParseArgumentsUnexpectedException;
-import io.airlift.command.ParseCommandMissingException;
-import io.airlift.command.ParseCommandUnrecognizedException;
-import io.airlift.command.ParseOptionConversionException;
-import io.airlift.command.ParseOptionMissingException;
-import io.airlift.command.ParseOptionMissingValueException;
-
-import java.io.Console;
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.FileWriter;
-import java.io.IOError;
-import java.io.IOException;
-import java.lang.management.MemoryUsage;
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.text.DecimalFormat;
-import java.text.SimpleDateFormat;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Scanner;
-import java.util.Set;
-import java.util.SortedMap;
-import java.util.concurrent.ExecutionException;
-
-import javax.management.InstanceNotFoundException;
-import javax.management.openmbean.CompositeData;
-import javax.management.openmbean.OpenDataException;
-import javax.management.openmbean.TabularData;
-import javax.management.openmbean.TabularDataSupport;
-
-import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutorMBean;
-import org.apache.cassandra.db.ColumnFamilyStoreMBean;
-import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.compaction.CompactionManagerMBean;
-import org.apache.cassandra.db.compaction.OperationType;
-import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.locator.EndpointSnitchInfoMBean;
-import org.apache.cassandra.metrics.ColumnFamilyMetrics.Sampler;
-import org.apache.cassandra.net.MessagingServiceMBean;
-import org.apache.cassandra.repair.RepairParallelism;
-import org.apache.cassandra.service.CacheServiceMBean;
-import org.apache.cassandra.service.StorageProxyMBean;
-import org.apache.cassandra.streaming.ProgressInfo;
-import org.apache.cassandra.streaming.SessionInfo;
-import org.apache.cassandra.streaming.StreamState;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.JVMStabilityInspector;
-import org.apache.commons.lang3.ArrayUtils;
-
-import com.google.common.base.Joiner;
-import com.google.common.base.Throwables;
-import com.google.common.collect.ArrayListMultimap;
-import com.google.common.collect.LinkedHashMultimap;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Ordering;
-import com.yammer.metrics.reporting.JmxReporter;
+import static org.apache.commons.lang3.StringUtils.*;
 
 public class NodeTool
 {
@@ -115,11 +54,14 @@
                 Ring.class,
                 NetStats.class,
                 CfStats.class,
+                TableStats.class,
                 CfHistograms.class,
+                TableHistograms.class,
                 Cleanup.class,
                 ClearSnapshot.class,
                 Compact.class,
                 Scrub.class,
+                Verify.class,
                 Flush.class,
                 UpgradeSSTable.class,
                 DisableAutoCompaction.class,
@@ -138,6 +80,7 @@
                 GetCompactionThreshold.class,
                 GetCompactionThroughput.class,
                 GetStreamThroughput.class,
+                GetTraceProbability.class,
                 GetInterDCStreamThroughput.class,
                 GetEndpoints.class,
                 GetSSTables.class,
@@ -152,9 +95,8 @@
                 ProxyHistograms.class,
                 Rebuild.class,
                 Refresh.class,
-                RefreshSizeEstimates.class,
-                RemoveToken.class,
                 RemoveNode.class,
+                Assassinate.class,
                 Repair.class,
                 SetCacheCapacity.class,
                 SetHintedHandoffThrottleInKB.class,
@@ -189,14 +131,24 @@
                 TpStats.class,
                 TopPartitions.class,
                 SetLoggingLevel.class,
-                GetLoggingLevels.class
+                GetLoggingLevels.class,
+                FailureDetectorInfo.class,
+                RefreshSizeEstimates.class
         );
 
-        Cli<Runnable> parser = Cli.<Runnable>builder("nodetool")
-                .withDescription("Manage your Cassandra cluster")
+        Cli.CliBuilder<Runnable> builder = Cli.builder("nodetool");
+
+        builder.withDescription("Manage your Cassandra cluster")
+                 .withDefaultCommand(Help.class)
+                 .withCommands(commands);
+
+        // bootstrap commands
+        builder.withGroup("bootstrap")
+                .withDescription("Monitor/manage node's bootstrap process")
                 .withDefaultCommand(Help.class)
-                .withCommands(commands)
-                .build();
+                .withCommand(BootstrapResume.class);
+
+        Cli<Runnable> parser = builder.build();
 
         int status = 0;
         try
@@ -362,10 +314,15 @@
 
         protected List<String> parseOptionalKeyspace(List<String> cmdArgs, NodeProbe nodeProbe)
         {
+            return parseOptionalKeyspace(cmdArgs, nodeProbe, false);
+        }
+
+        protected List<String> parseOptionalKeyspace(List<String> cmdArgs, NodeProbe nodeProbe, boolean includeSystemKS)
+        {
             List<String> keyspaces = new ArrayList<>();
 
             if (cmdArgs == null || cmdArgs.isEmpty())
-                keyspaces.addAll(nodeProbe.getKeyspaces());
+                keyspaces.addAll(includeSystemKS ? nodeProbe.getKeyspaces() : nodeProbe.getNonSystemKeyspaces());
             else
                 keyspaces.add(cmdArgs.get(0));
 
@@ -384,1962 +341,9 @@
         }
     }
 
-    @Command(name = "info", description = "Print node information (uptime, load, ...)")
-    public static class Info extends NodeToolCmd
-    {
-        @Option(name = {"-T", "--tokens"}, description = "Display all tokens")
-        private boolean tokens = false;
-
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            boolean gossipInitialized = probe.isInitialized();
-
-            System.out.printf("%-23s: %s%n", "ID", probe.getLocalHostId());
-            System.out.printf("%-23s: %s%n", "Gossip active", gossipInitialized);
-            System.out.printf("%-23s: %s%n", "Thrift active", probe.isThriftServerRunning());
-            System.out.printf("%-23s: %s%n", "Native Transport active", probe.isNativeTransportRunning());
-            System.out.printf("%-23s: %s%n", "Load", probe.getLoadString());
-            if (gossipInitialized)
-                System.out.printf("%-23s: %s%n", "Generation No", probe.getCurrentGenerationNumber());
-            else
-                System.out.printf("%-23s: %s%n", "Generation No", 0);
-
-            // Uptime
-            long secondsUp = probe.getUptime() / 1000;
-            System.out.printf("%-23s: %d%n", "Uptime (seconds)", secondsUp);
-
-            // Memory usage
-            MemoryUsage heapUsage = probe.getHeapMemoryUsage();
-            double memUsed = (double) heapUsage.getUsed() / (1024 * 1024);
-            double memMax = (double) heapUsage.getMax() / (1024 * 1024);
-            System.out.printf("%-23s: %.2f / %.2f%n", "Heap Memory (MB)", memUsed, memMax);
-            try
-            {
-                System.out.printf("%-23s: %.2f%n", "Off Heap Memory (MB)", getOffHeapMemoryUsed(probe));
-            }
-            catch (RuntimeException e)
-            {
-                // offheap-metrics introduced in 2.1.3 - older versions do not have the appropriate mbeans
-                if (!(e.getCause() instanceof InstanceNotFoundException))
-                    throw e;
-            }
-
-            // Data Center/Rack
-            System.out.printf("%-23s: %s%n", "Data Center", probe.getDataCenter());
-            System.out.printf("%-23s: %s%n", "Rack", probe.getRack());
-
-            // Exceptions
-            System.out.printf("%-23s: %s%n", "Exceptions", probe.getStorageMetric("Exceptions"));
-
-            CacheServiceMBean cacheService = probe.getCacheServiceMBean();
-
-            // Key Cache: Hits, Requests, RecentHitRate, SavePeriodInSeconds
-            System.out.printf("%-23s: entries %d, size %s, capacity %s, %d hits, %d requests, %.3f recent hit rate, %d save period in seconds%n",
-                    "Key Cache",
-                    probe.getCacheMetric("KeyCache", "Entries"),
-                    FileUtils.stringifyFileSize((long) probe.getCacheMetric("KeyCache", "Size")),
-                    FileUtils.stringifyFileSize((long) probe.getCacheMetric("KeyCache", "Capacity")),
-                    probe.getCacheMetric("KeyCache", "Hits"),
-                    probe.getCacheMetric("KeyCache", "Requests"),
-                    probe.getCacheMetric("KeyCache", "HitRate"),
-                    cacheService.getKeyCacheSavePeriodInSeconds());
-
-            // Row Cache: Hits, Requests, RecentHitRate, SavePeriodInSeconds
-            System.out.printf("%-23s: entries %d, size %s, capacity %s, %d hits, %d requests, %.3f recent hit rate, %d save period in seconds%n",
-                    "Row Cache",
-                    probe.getCacheMetric("RowCache", "Entries"),
-                    FileUtils.stringifyFileSize((long) probe.getCacheMetric("RowCache", "Size")),
-                    FileUtils.stringifyFileSize((long) probe.getCacheMetric("RowCache", "Capacity")),
-                    probe.getCacheMetric("RowCache", "Hits"),
-                    probe.getCacheMetric("RowCache", "Requests"),
-                    probe.getCacheMetric("RowCache", "HitRate"),
-                    cacheService.getRowCacheSavePeriodInSeconds());
-
-            // Counter Cache: Hits, Requests, RecentHitRate, SavePeriodInSeconds
-            System.out.printf("%-23s: entries %d, size %s, capacity %s, %d hits, %d requests, %.3f recent hit rate, %d save period in seconds%n",
-                    "Counter Cache",
-                    probe.getCacheMetric("CounterCache", "Entries"),
-                    FileUtils.stringifyFileSize((long) probe.getCacheMetric("CounterCache", "Size")),
-                    FileUtils.stringifyFileSize((long) probe.getCacheMetric("CounterCache", "Capacity")),
-                    probe.getCacheMetric("CounterCache", "Hits"),
-                    probe.getCacheMetric("CounterCache", "Requests"),
-                    probe.getCacheMetric("CounterCache", "HitRate"),
-                    cacheService.getCounterCacheSavePeriodInSeconds());
-
-            // check if node is already joined, before getting tokens, since it throws exception if not.
-            if (probe.isJoined())
-            {
-                // Tokens
-                List<String> tokens = probe.getTokens();
-                if (tokens.size() == 1 || this.tokens)
-                    for (String token : tokens)
-                        System.out.printf("%-23s: %s%n", "Token", token);
-                else
-                    System.out.printf("%-23s: (invoke with -T/--tokens to see all %d tokens)%n", "Token",
-                                      tokens.size());
-            }
-            else
-            {
-                System.out.printf("%-23s: (node is not joined to the cluster)%n", "Token");
-            }
-        }
-
-        /**
-         * Returns the total off heap memory used in MB.
-         * @return the total off heap memory used in MB.
-         */
-        private static double getOffHeapMemoryUsed(NodeProbe probe)
-        {
-            long offHeapMemUsedInBytes = 0;
-            // get a list of column family stores
-            Iterator<Map.Entry<String, ColumnFamilyStoreMBean>> cfamilies = probe.getColumnFamilyStoreMBeanProxies();
-
-            while (cfamilies.hasNext())
-            {
-                Entry<String, ColumnFamilyStoreMBean> entry = cfamilies.next();
-                String keyspaceName = entry.getKey();
-                String cfName = entry.getValue().getColumnFamilyName();
-
-                offHeapMemUsedInBytes += (Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "MemtableOffHeapSize");
-                offHeapMemUsedInBytes += (Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "BloomFilterOffHeapMemoryUsed");
-                offHeapMemUsedInBytes += (Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "IndexSummaryOffHeapMemoryUsed");
-                offHeapMemUsedInBytes += (Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "CompressionMetadataOffHeapMemoryUsed");
-            }
-
-            return offHeapMemUsedInBytes / (1024d * 1024);
-        }
-    }
-
-    @Command(name = "ring", description = "Print information about the token ring")
-    public static class Ring extends NodeToolCmd
-    {
-        @Arguments(description = "Specify a keyspace for accurate ownership information (topology awareness)")
-        private String keyspace = null;
-
-        @Option(title = "resolve_ip", name = {"-r", "--resolve-ip"}, description = "Show node domain names instead of IPs")
-        private boolean resolveIp = false;
-
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            Map<String, String> tokensToEndpoints = probe.getTokenToEndpointMap();
-            LinkedHashMultimap<String, String> endpointsToTokens = LinkedHashMultimap.create();
-            boolean haveVnodes = false;
-            for (Map.Entry<String, String> entry : tokensToEndpoints.entrySet())
-            {
-                haveVnodes |= endpointsToTokens.containsKey(entry.getValue());
-                endpointsToTokens.put(entry.getValue(), entry.getKey());
-            }
-
-            int maxAddressLength = Collections.max(endpointsToTokens.keys(), new Comparator<String>()
-            {
-                @Override
-                public int compare(String first, String second)
-                {
-                    return ((Integer) first.length()).compareTo(second.length());
-                }
-            }).length();
-
-            String formatPlaceholder = "%%-%ds  %%-12s%%-7s%%-8s%%-16s%%-20s%%-44s%%n";
-            String format = format(formatPlaceholder, maxAddressLength);
-
-            StringBuffer errors = new StringBuffer();
-            boolean showEffectiveOwnership = true;
-            // Calculate per-token ownership of the ring
-            Map<InetAddress, Float> ownerships;
-            try
-            {
-                ownerships = probe.effectiveOwnership(keyspace);
-            } 
-            catch (IllegalStateException ex)
-            {
-                ownerships = probe.getOwnership();
-                errors.append("Note: " + ex.getMessage() + "%n");
-                showEffectiveOwnership = false;
-            } 
-            catch (IllegalArgumentException ex)
-            {
-                System.out.printf("%nError: " + ex.getMessage() + "%n");
-                return;
-            }
-
-            
-            System.out.println();
-            for (Entry<String, SetHostStat> entry : getOwnershipByDc(probe, resolveIp, tokensToEndpoints, ownerships).entrySet())
-                printDc(probe, format, entry.getKey(), endpointsToTokens, entry.getValue(),showEffectiveOwnership);
-
-            if (haveVnodes)
-            {
-                System.out.println("  Warning: \"nodetool ring\" is used to output all the tokens of a node.");
-                System.out.println("  To view status related info of a node use \"nodetool status\" instead.\n");
-            }
-
-            System.out.printf("%n  " + errors.toString());
-        }
-
-        private void printDc(NodeProbe probe, String format,
-                             String dc,
-                             LinkedHashMultimap<String, String> endpointsToTokens,
-                             SetHostStat hoststats,boolean showEffectiveOwnership)
-        {
-            Collection<String> liveNodes = probe.getLiveNodes();
-            Collection<String> deadNodes = probe.getUnreachableNodes();
-            Collection<String> joiningNodes = probe.getJoiningNodes();
-            Collection<String> leavingNodes = probe.getLeavingNodes();
-            Collection<String> movingNodes = probe.getMovingNodes();
-            Map<String, String> loadMap = probe.getLoadMap();
-
-            System.out.println("Datacenter: " + dc);
-            System.out.println("==========");
-
-            // get the total amount of replicas for this dc and the last token in this dc's ring
-            List<String> tokens = new ArrayList<>();
-            String lastToken = "";
-
-            for (HostStat stat : hoststats)
-            {
-                tokens.addAll(endpointsToTokens.get(stat.endpoint.getHostAddress()));
-                lastToken = tokens.get(tokens.size() - 1);
-            }
-
-            System.out.printf(format, "Address", "Rack", "Status", "State", "Load", "Owns", "Token");
-
-            if (hoststats.size() > 1)
-                System.out.printf(format, "", "", "", "", "", "", lastToken);
-            else
-                System.out.println();
-
-            for (HostStat stat : hoststats)
-            {
-                String endpoint = stat.endpoint.getHostAddress();
-                String rack;
-                try
-                {
-                    rack = probe.getEndpointSnitchInfoProxy().getRack(endpoint);
-                }
-                catch (UnknownHostException e)
-                {
-                    rack = "Unknown";
-                }
-
-                String status = liveNodes.contains(endpoint)
-                        ? "Up"
-                        : deadNodes.contains(endpoint)
-                                ? "Down"
-                                : "?";
-
-                String state = "Normal";
-
-                if (joiningNodes.contains(endpoint))
-                    state = "Joining";
-                else if (leavingNodes.contains(endpoint))
-                    state = "Leaving";
-                else if (movingNodes.contains(endpoint))
-                    state = "Moving";
-
-                String load = loadMap.containsKey(endpoint)
-                        ? loadMap.get(endpoint)
-                        : "?";
-                String owns = stat.owns != null && showEffectiveOwnership? new DecimalFormat("##0.00%").format(stat.owns) : "?";
-                System.out.printf(format, stat.ipOrDns(), rack, status, state, load, owns, stat.token);
-            }
-            System.out.println();
-        }
-    }
-
-    @Command(name = "netstats", description = "Print network information on provided host (connecting node by default)")
-    public static class NetStats extends NodeToolCmd
-    {
-        @Option(title = "human_readable",
-                name = {"-H", "--human-readable"},
-                description = "Display bytes in human readable form, i.e. KB, MB, GB, TB")
-        private boolean humanReadable = false;
-
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            System.out.printf("Mode: %s%n", probe.getOperationMode());
-            Set<StreamState> statuses = probe.getStreamStatus();
-            if (statuses.isEmpty())
-                System.out.println("Not sending any streams.");
-            for (StreamState status : statuses)
-            {
-                System.out.printf("%s %s%n", status.description, status.planId.toString());
-                for (SessionInfo info : status.sessions)
-                {
-                    System.out.printf("    %s", info.peer.toString());
-                    // print private IP when it is used
-                    if (!info.peer.equals(info.connecting))
-                    {
-                        System.out.printf(" (using %s)", info.connecting.toString());
-                    }
-                    System.out.printf("%n");
-                    if (!info.receivingSummaries.isEmpty())
-                    {
-                        if (humanReadable)
-                            System.out.printf("        Receiving %d files, %s total. Already received %d files, %s total%n", info.getTotalFilesToReceive(), FileUtils.stringifyFileSize(info.getTotalSizeToReceive()), info.getTotalFilesReceived(), FileUtils.stringifyFileSize(info.getTotalSizeReceived()));
-                        else
-                            System.out.printf("        Receiving %d files, %d bytes total. Already received %d files, %d bytes total%n", info.getTotalFilesToReceive(), info.getTotalSizeToReceive(), info.getTotalFilesReceived(), info.getTotalSizeReceived());
-                        for (ProgressInfo progress : info.getReceivingFiles())
-                        {
-                            System.out.printf("            %s%n", progress.toString());
-                        }
-                    }
-                    if (!info.sendingSummaries.isEmpty())
-                    {
-                        if (humanReadable)
-                            System.out.printf("        Sending %d files, %s total. Already sent %d files, %s total%n", info.getTotalFilesToSend(), FileUtils.stringifyFileSize(info.getTotalSizeToSend()), info.getTotalFilesSent(), FileUtils.stringifyFileSize(info.getTotalSizeSent()));
-                        else
-                            System.out.printf("        Sending %d files, %d bytes total. Already sent %d files, %d bytes total%n", info.getTotalFilesToSend(), info.getTotalSizeToSend(), info.getTotalFilesSent(), info.getTotalSizeSent());
-                        for (ProgressInfo progress : info.getSendingFiles())
-                        {
-                            System.out.printf("            %s%n", progress.toString());
-                        }
-                    }
-                }
-            }
-
-            if (!probe.isStarting())
-            {
-                System.out.printf("Read Repair Statistics:%nAttempted: %d%nMismatch (Blocking): %d%nMismatch (Background): %d%n", probe.getReadRepairAttempted(), probe.getReadRepairRepairedBlocking(), probe.getReadRepairRepairedBackground());
-
-                MessagingServiceMBean ms = probe.msProxy;
-                System.out.printf("%-25s", "Pool Name");
-                System.out.printf("%10s", "Active");
-                System.out.printf("%10s", "Pending");
-                System.out.printf("%15s", "Completed");
-                System.out.printf("%10s%n", "Dropped");
-
-                int pending;
-                long completed;
-                long dropped;
-
-                pending = 0;
-                for (int n : ms.getCommandPendingTasks().values())
-                    pending += n;
-                completed = 0;
-                for (long n : ms.getCommandCompletedTasks().values())
-                    completed += n;
-                dropped = 0;
-                for (long n : ms.getCommandDroppedTasks().values())
-                    dropped += n;
-                System.out.printf("%-25s%10s%10s%15s%10s%n", "Commands", "n/a", pending, completed, dropped);
-
-                pending = 0;
-                for (int n : ms.getResponsePendingTasks().values())
-                    pending += n;
-                completed = 0;
-                for (long n : ms.getResponseCompletedTasks().values())
-                    completed += n;
-                System.out.printf("%-25s%10s%10s%15s%10s%n", "Responses", "n/a", pending, completed, "n/a");
-            }
-        }
-    }
-
-    @Command(name = "cfstats", description = "Print statistics on column families")
-    public static class CfStats extends NodeToolCmd
-    {
-        @Arguments(usage = "[<keyspace.cfname>...]", description = "List of column families (or keyspace) names")
-        private List<String> cfnames = new ArrayList<>();
-
-        @Option(name = "-i", description = "Ignore the list of column families and display the remaining cfs")
-        private boolean ignore = false;
-
-        @Option(title = "human_readable",
-                name = {"-H", "--human-readable"},
-                description = "Display bytes in human readable form, i.e. KB, MB, GB, TB")
-        private boolean humanReadable = false;
-
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            OptionFilter filter = new OptionFilter(ignore, cfnames);
-            Map<String, List<ColumnFamilyStoreMBean>> cfstoreMap = new HashMap<>();
-
-            // get a list of column family stores
-            Iterator<Map.Entry<String, ColumnFamilyStoreMBean>> cfamilies = probe.getColumnFamilyStoreMBeanProxies();
-
-            while (cfamilies.hasNext())
-            {
-                Map.Entry<String, ColumnFamilyStoreMBean> entry = cfamilies.next();
-                String keyspaceName = entry.getKey();
-                ColumnFamilyStoreMBean cfsProxy = entry.getValue();
-
-                if (!cfstoreMap.containsKey(keyspaceName) && filter.isColumnFamilyIncluded(entry.getKey(), cfsProxy.getColumnFamilyName()))
-                {
-                    List<ColumnFamilyStoreMBean> columnFamilies = new ArrayList<>();
-                    columnFamilies.add(cfsProxy);
-                    cfstoreMap.put(keyspaceName, columnFamilies);
-                } else if (filter.isColumnFamilyIncluded(entry.getKey(), cfsProxy.getColumnFamilyName()))
-                {
-                    cfstoreMap.get(keyspaceName).add(cfsProxy);
-                }
-            }
-
-            // make sure all specified kss and cfs exist
-            filter.verifyKeyspaces(probe.getKeyspaces());
-            filter.verifyColumnFamilies();
-
-            // print out the table statistics
-            for (Map.Entry<String, List<ColumnFamilyStoreMBean>> entry : cfstoreMap.entrySet())
-            {
-                String keyspaceName = entry.getKey();
-                List<ColumnFamilyStoreMBean> columnFamilies = entry.getValue();
-                long keyspaceReadCount = 0;
-                long keyspaceWriteCount = 0;
-                int keyspacePendingFlushes = 0;
-                double keyspaceTotalReadTime = 0.0f;
-                double keyspaceTotalWriteTime = 0.0f;
-
-                System.out.println("Keyspace: " + keyspaceName);
-                for (ColumnFamilyStoreMBean cfstore : columnFamilies)
-                {
-                    String cfName = cfstore.getColumnFamilyName();
-                    long writeCount = ((JmxReporter.TimerMBean) probe.getColumnFamilyMetric(keyspaceName, cfName, "WriteLatency")).getCount();
-                    long readCount = ((JmxReporter.TimerMBean) probe.getColumnFamilyMetric(keyspaceName, cfName, "ReadLatency")).getCount();
-
-                    if (readCount > 0)
-                    {
-                        keyspaceReadCount += readCount;
-                        keyspaceTotalReadTime += (long) probe.getColumnFamilyMetric(keyspaceName, cfName, "ReadTotalLatency");
-                    }
-                    if (writeCount > 0)
-                    {
-                        keyspaceWriteCount += writeCount;
-                        keyspaceTotalWriteTime += (long) probe.getColumnFamilyMetric(keyspaceName, cfName, "WriteTotalLatency");
-                    }
-                    keyspacePendingFlushes += (long) probe.getColumnFamilyMetric(keyspaceName, cfName, "PendingFlushes");
-                }
-
-                double keyspaceReadLatency = keyspaceReadCount > 0
-                                             ? keyspaceTotalReadTime / keyspaceReadCount / 1000
-                                             : Double.NaN;
-                double keyspaceWriteLatency = keyspaceWriteCount > 0
-                                              ? keyspaceTotalWriteTime / keyspaceWriteCount / 1000
-                                              : Double.NaN;
-
-                System.out.println("\tRead Count: " + keyspaceReadCount);
-                System.out.println("\tRead Latency: " + String.format("%s", keyspaceReadLatency) + " ms.");
-                System.out.println("\tWrite Count: " + keyspaceWriteCount);
-                System.out.println("\tWrite Latency: " + String.format("%s", keyspaceWriteLatency) + " ms.");
-                System.out.println("\tPending Flushes: " + keyspacePendingFlushes);
-
-                // print out column family statistics for this keyspace
-                for (ColumnFamilyStoreMBean cfstore : columnFamilies)
-                {
-                    String cfName = cfstore.getColumnFamilyName();
-                    if (cfName.contains("."))
-                        System.out.println("\t\tTable (index): " + cfName);
-                    else
-                        System.out.println("\t\tTable: " + cfName);
-
-                    System.out.println("\t\tSSTable count: " + probe.getColumnFamilyMetric(keyspaceName, cfName, "LiveSSTableCount"));
-
-                    int[] leveledSStables = cfstore.getSSTableCountPerLevel();
-                    if (leveledSStables != null)
-                    {
-                        System.out.print("\t\tSSTables in each level: [");
-                        for (int level = 0; level < leveledSStables.length; level++)
-                        {
-                            int count = leveledSStables[level];
-                            System.out.print(count);
-                            long maxCount = 4L; // for L0
-                            if (level > 0)
-                                maxCount = (long) Math.pow(10, level);
-                            //  show max threshold for level when exceeded
-                            if (count > maxCount)
-                                System.out.print("/" + maxCount);
-
-                            if (level < leveledSStables.length - 1)
-                                System.out.print(", ");
-                            else
-                                System.out.println("]");
-                        }
-                    }
-
-                    Long memtableOffHeapSize = null;
-                    Long bloomFilterOffHeapSize = null;
-                    Long indexSummaryOffHeapSize = null;
-                    Long compressionMetadataOffHeapSize = null;
-
-                    Long offHeapSize = null;
-
-                    try
-                    {
-                        memtableOffHeapSize = (Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "MemtableOffHeapSize");
-                        bloomFilterOffHeapSize = (Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "BloomFilterOffHeapMemoryUsed");
-                        indexSummaryOffHeapSize = (Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "IndexSummaryOffHeapMemoryUsed");
-                        compressionMetadataOffHeapSize = (Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "CompressionMetadataOffHeapMemoryUsed");
-
-                        offHeapSize = memtableOffHeapSize + bloomFilterOffHeapSize + indexSummaryOffHeapSize + compressionMetadataOffHeapSize;
-                    }
-                    catch (RuntimeException e)
-                    {
-                        // offheap-metrics introduced in 2.1.3 - older versions do not have the appropriate mbeans
-                        if (!(e.getCause() instanceof InstanceNotFoundException))
-                            throw e;
-                    }
-
-                    System.out.println("\t\tSpace used (live): " + format((Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "LiveDiskSpaceUsed"), humanReadable));
-                    System.out.println("\t\tSpace used (total): " + format((Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "TotalDiskSpaceUsed"), humanReadable));
-                    System.out.println("\t\tSpace used by snapshots (total): " + format((Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "SnapshotsSize"), humanReadable));
-                    if (offHeapSize != null)
-                        System.out.println("\t\tOff heap memory used (total): " + format(offHeapSize, humanReadable));
-                    System.out.println("\t\tSSTable Compression Ratio: " + probe.getColumnFamilyMetric(keyspaceName, cfName, "CompressionRatio"));
-                    System.out.println("\t\tNumber of keys (estimate): " + probe.getColumnFamilyMetric(keyspaceName, cfName, "EstimatedRowCount"));
-                    System.out.println("\t\tMemtable cell count: " + probe.getColumnFamilyMetric(keyspaceName, cfName, "MemtableColumnsCount"));
-                    System.out.println("\t\tMemtable data size: " + format((Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "MemtableLiveDataSize"), humanReadable));
-                    if (memtableOffHeapSize != null)
-                        System.out.println("\t\tMemtable off heap memory used: " + format(memtableOffHeapSize, humanReadable));
-                    System.out.println("\t\tMemtable switch count: " + probe.getColumnFamilyMetric(keyspaceName, cfName, "MemtableSwitchCount"));
-                    System.out.println("\t\tLocal read count: " + ((JmxReporter.TimerMBean) probe.getColumnFamilyMetric(keyspaceName, cfName, "ReadLatency")).getCount());
-                    double localReadLatency = ((JmxReporter.TimerMBean) probe.getColumnFamilyMetric(keyspaceName, cfName, "ReadLatency")).getMean() / 1000;
-                    double localRLatency = localReadLatency > 0 ? localReadLatency : Double.NaN;
-                    System.out.printf("\t\tLocal read latency: %01.3f ms%n", localRLatency);
-                    System.out.println("\t\tLocal write count: " + ((JmxReporter.TimerMBean) probe.getColumnFamilyMetric(keyspaceName, cfName, "WriteLatency")).getCount());
-                    double localWriteLatency = ((JmxReporter.TimerMBean) probe.getColumnFamilyMetric(keyspaceName, cfName, "WriteLatency")).getMean() / 1000;
-                    double localWLatency = localWriteLatency > 0 ? localWriteLatency : Double.NaN;
-                    System.out.printf("\t\tLocal write latency: %01.3f ms%n", localWLatency);
-                    System.out.println("\t\tPending flushes: " + probe.getColumnFamilyMetric(keyspaceName, cfName, "PendingFlushes"));
-                    System.out.println("\t\tBloom filter false positives: " + probe.getColumnFamilyMetric(keyspaceName, cfName, "BloomFilterFalsePositives"));
-                    System.out.printf("\t\tBloom filter false ratio: %s%n", String.format("%01.5f", probe.getColumnFamilyMetric(keyspaceName, cfName, "RecentBloomFilterFalseRatio")));
-                    System.out.println("\t\tBloom filter space used: " + format((Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "BloomFilterDiskSpaceUsed"), humanReadable));
-                    if (bloomFilterOffHeapSize != null)
-                        System.out.println("\t\tBloom filter off heap memory used: " + format(bloomFilterOffHeapSize, humanReadable));
-                    if (indexSummaryOffHeapSize != null)
-                        System.out.println("\t\tIndex summary off heap memory used: " + format(indexSummaryOffHeapSize, humanReadable));
-                    if (compressionMetadataOffHeapSize != null)
-                        System.out.println("\t\tCompression metadata off heap memory used: " + format(compressionMetadataOffHeapSize, humanReadable));
-
-                    System.out.println("\t\tCompacted partition minimum bytes: " + format((Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "MinRowSize"), humanReadable));
-                    System.out.println("\t\tCompacted partition maximum bytes: " + format((Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "MaxRowSize"), humanReadable));
-                    System.out.println("\t\tCompacted partition mean bytes: " + format((Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "MeanRowSize"), humanReadable));
-                    JmxReporter.HistogramMBean histogram = (JmxReporter.HistogramMBean) probe.getColumnFamilyMetric(keyspaceName, cfName, "LiveScannedHistogram");
-                    System.out.println("\t\tAverage live cells per slice (last five minutes): " + histogram.getMean());
-                    System.out.println("\t\tMaximum live cells per slice (last five minutes): " + histogram.getMax());
-                    histogram = (JmxReporter.HistogramMBean) probe.getColumnFamilyMetric(keyspaceName, cfName, "TombstoneScannedHistogram");
-                    System.out.println("\t\tAverage tombstones per slice (last five minutes): " + histogram.getMean());
-                    System.out.println("\t\tMaximum tombstones per slice (last five minutes): " + histogram.getMax());
-
-                    System.out.println("");
-                }
-                System.out.println("----------------");
-            }
-        }
-
-        private String format(long bytes, boolean humanReadable) {
-            return humanReadable ? FileUtils.stringifyFileSize(bytes) : Long.toString(bytes);
-        }
-
-        /**
-         * Used for filtering keyspaces and columnfamilies to be displayed using the cfstats command.
-         */
-        private static class OptionFilter
-        {
-            private Map<String, List<String>> filter = new HashMap<>();
-            private Map<String, List<String>> verifier = new HashMap<>();
-            private List<String> filterList = new ArrayList<>();
-            private boolean ignoreMode;
-
-            public OptionFilter(boolean ignoreMode, List<String> filterList)
-            {
-                this.filterList.addAll(filterList);
-                this.ignoreMode = ignoreMode;
-
-                for (String s : filterList)
-                {
-                    String[] keyValues = s.split("\\.", 2);
-
-                    // build the map that stores the ks' and cfs to use
-                    if (!filter.containsKey(keyValues[0]))
-                    {
-                        filter.put(keyValues[0], new ArrayList<String>());
-                        verifier.put(keyValues[0], new ArrayList<String>());
-
-                        if (keyValues.length == 2)
-                        {
-                            filter.get(keyValues[0]).add(keyValues[1]);
-                            verifier.get(keyValues[0]).add(keyValues[1]);
-                        }
-                    } else
-                    {
-                        if (keyValues.length == 2)
-                        {
-                            filter.get(keyValues[0]).add(keyValues[1]);
-                            verifier.get(keyValues[0]).add(keyValues[1]);
-                        }
-                    }
-                }
-            }
-
-            public boolean isColumnFamilyIncluded(String keyspace, String columnFamily)
-            {
-                // supplying empty params list is treated as wanting to display all kss & cfs
-                if (filterList.isEmpty())
-                    return !ignoreMode;
-
-                List<String> cfs = filter.get(keyspace);
-
-                // no such keyspace is in the map
-                if (cfs == null)
-                    return ignoreMode;
-                    // only a keyspace with no cfs was supplied
-                    // so ignore or include (based on the flag) every column family in specified keyspace
-                else if (cfs.size() == 0)
-                    return !ignoreMode;
-
-                // keyspace exists, and it contains specific cfs
-                verifier.get(keyspace).remove(columnFamily);
-                return ignoreMode ^ cfs.contains(columnFamily);
-            }
-
-            public void verifyKeyspaces(List<String> keyspaces)
-            {
-                for (String ks : verifier.keySet())
-                    if (!keyspaces.contains(ks))
-                        throw new IllegalArgumentException("Unknown keyspace: " + ks);
-            }
-
-            public void verifyColumnFamilies()
-            {
-                for (String ks : filter.keySet())
-                    if (verifier.get(ks).size() > 0)
-                        throw new IllegalArgumentException("Unknown column families: " + verifier.get(ks).toString() + " in keyspace: " + ks);
-            }
-        }
-    }
-
-    @Command(name = "toppartitions", description = "Sample and print the most active partitions for a given column family")
-    public static class TopPartitions extends NodeToolCmd
-    {
-        @Arguments(usage = "<keyspace> <cfname> <duration>", description = "The keyspace, column family name, and duration in milliseconds")
-        private List<String> args = new ArrayList<>();
-        @Option(name = "-s", description = "Capacity of stream summary, closer to the actual cardinality of partitions will yield more accurate results (Default: 256)")
-        private int size = 256;
-        @Option(name = "-k", description = "Number of the top partitions to list (Default: 10)")
-        private int topCount = 10;
-        @Option(name = "-a", description = "Comma separated list of samplers to use (Default: all)")
-        private String samplers = join(Sampler.values(), ',');
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            checkArgument(args.size() == 3, "toppartitions requires keyspace, column family name, and duration");
-            checkArgument(topCount < size, "TopK count (-k) option must be smaller then the summary capacity (-s)");
-            String keyspace = args.get(0);
-            String cfname = args.get(1);
-            Integer duration = Integer.parseInt(args.get(2));
-            // generate the list of samplers
-            List<Sampler> targets = Lists.newArrayList();
-            for (String s : samplers.split(","))
-            {
-                try
-                {
-                    targets.add(Sampler.valueOf(s.toUpperCase()));
-                } catch (Exception e)
-                {
-                    throw new IllegalArgumentException(s + " is not a valid sampler, choose one of: " + join(Sampler.values(), ", "));
-                }
-            }
-
-            Map<Sampler, CompositeData> results;
-            try
-            {
-                results = probe.getPartitionSample(keyspace, cfname, size, duration, topCount, targets);
-            } catch (OpenDataException e)
-            {
-                throw new RuntimeException(e);
-            }
-            boolean first = true;
-            for(Entry<Sampler, CompositeData> result : results.entrySet())
-            {
-                CompositeData sampling = result.getValue();
-                // weird casting for http://bugs.sun.com/view_bug.do?bug_id=6548436
-                List<CompositeData> topk = (List<CompositeData>) (Object) Lists.newArrayList(((TabularDataSupport) sampling.get("partitions")).values());
-                Collections.sort(topk, new Ordering<CompositeData>()
-                {
-                    public int compare(CompositeData left, CompositeData right)
-                    {
-                        return Long.compare((long) right.get("count"), (long) left.get("count"));
-                    }
-                });
-                if(!first)
-                    System.out.println();
-                System.out.println(result.getKey().toString()+ " Sampler:");
-                System.out.printf("  Cardinality: ~%d (%d capacity)%n", (long) sampling.get("cardinality"), size);
-                System.out.printf("  Top %d partitions:%n", topCount);
-                if (topk.size() == 0)
-                {
-                    System.out.println("\tNothing recorded during sampling period...");
-                } else
-                {
-                    int offset = 0;
-                    for (CompositeData entry : topk)
-                        offset = Math.max(offset, entry.get("string").toString().length());
-                    System.out.printf("\t%-" + offset + "s%10s%10s%n", "Partition", "Count", "+/-");
-                    for (CompositeData entry : topk)
-                        System.out.printf("\t%-" + offset + "s%10d%10d%n", entry.get("string").toString(), entry.get("count"), entry.get("error"));
-                }
-                first = false;
-            }
-        }
-    }
-
-    @Command(name = "cfhistograms", description = "Print statistic histograms for a given column family")
-    public static class CfHistograms extends NodeToolCmd
-    {
-        @Arguments(usage = "<keyspace> <cfname>", description = "The keyspace and column family name")
-        private List<String> args = new ArrayList<>();
-
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            checkArgument(args.size() == 2, "cfhistograms requires ks and cf args");
-
-            String keyspace = args.get(0);
-            String cfname = args.get(1);
-
-            ColumnFamilyStoreMBean store = probe.getCfsProxy(keyspace, cfname);
-
-            long[] estimatedRowSizeHistogram = store.getEstimatedRowSizeHistogram();
-            long[] estimatedColumnCountHistogram = store.getEstimatedColumnCountHistogram();
-
-            if (ArrayUtils.isEmpty(estimatedRowSizeHistogram) || ArrayUtils.isEmpty(estimatedColumnCountHistogram))
-            {
-                System.err.println("No SSTables exists, unable to calculate 'Partition Size' and 'Cell Count' percentiles");
-            }
-
-            // calculate percentile of row size and column count
-            String[] percentiles = new String[]{"50%", "75%", "95%", "98%", "99%", "Min", "Max"};
-            double[] readLatency = probe.metricPercentilesAsArray(store.getRecentReadLatencyHistogramMicros());
-            double[] writeLatency = probe.metricPercentilesAsArray(store.getRecentWriteLatencyHistogramMicros());
-            double[] estimatedRowSizePercentiles = probe.metricPercentilesAsArray(estimatedRowSizeHistogram);
-            double[] estimatedColumnCountPercentiles = probe.metricPercentilesAsArray(estimatedColumnCountHistogram);
-            double[] sstablesPerRead = probe.metricPercentilesAsArray(store.getRecentSSTablesPerReadHistogram());
-
-            System.out.println(format("%s/%s histograms", keyspace, cfname));
-            System.out.println(format("%-10s%10s%18s%18s%18s%18s",
-                    "Percentile", "SSTables", "Write Latency", "Read Latency", "Partition Size", "Cell Count"));
-            System.out.println(format("%-10s%10s%18s%18s%18s%18s",
-                    "", "", "(micros)", "(micros)", "(bytes)", ""));
-
-            for (int i = 0; i < percentiles.length; i++)
-            {
-                System.out.println(format("%-10s%10.2f%18.2f%18.2f%18.0f%18.0f",
-                        percentiles[i],
-                        sstablesPerRead[i],
-                        writeLatency[i],
-                        readLatency[i],
-                        estimatedRowSizePercentiles[i],
-                        estimatedColumnCountPercentiles[i]));
-            }
-            System.out.println();
-        }
-    }
-
-    @Command(name = "cleanup", description = "Triggers the immediate cleanup of keys no longer belonging to a node. By default, clean all keyspaces")
-    public static class Cleanup extends NodeToolCmd
-    {
-        @Arguments(usage = "[<keyspace> <cfnames>...]", description = "The keyspace followed by one or many column families")
-        private List<String> args = new ArrayList<>();
-
-        @Option(title = "jobs",
-                name = {"-j", "--jobs"},
-                description = "Number of sstables to cleanup simultaneusly, set to 0 to use all available compaction threads")
-        private int jobs = 2;
-
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            List<String> keyspaces = parseOptionalKeyspace(args, probe);
-            String[] cfnames = parseOptionalColumnFamilies(args);
-
-            for (String keyspace : keyspaces)
-            {
-                if (Keyspace.SYSTEM_KS.equals(keyspace))
-                    continue;
-
-                try
-                {
-                    probe.forceKeyspaceCleanup(System.out, jobs, keyspace, cfnames);
-                } catch (Exception e)
-                {
-                    throw new RuntimeException("Error occurred during cleanup", e);
-                }
-            }
-        }
-    }
-
-    @Command(name = "clearsnapshot", description = "Remove the snapshot with the given name from the given keyspaces. If no snapshotName is specified we will remove all snapshots")
-    public static class ClearSnapshot extends NodeToolCmd
-    {
-        @Arguments(usage = "[<keyspaces>...] ", description = "Remove snapshots from the given keyspaces")
-        private List<String> keyspaces = new ArrayList<>();
-
-        @Option(title = "snapshot_name", name = "-t", description = "Remove the snapshot with a given name")
-        private String snapshotName = EMPTY;
-
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            StringBuilder sb = new StringBuilder();
-
-            sb.append("Requested clearing snapshot(s) for ");
-
-            if (keyspaces.isEmpty())
-                sb.append("[all keyspaces]");
-            else
-                sb.append("[").append(join(keyspaces, ", ")).append("]");
-
-            if (!snapshotName.isEmpty())
-                sb.append(" with snapshot name [").append(snapshotName).append("]");
-
-            System.out.println(sb.toString());
-
-            try
-            {
-                probe.clearSnapshot(snapshotName, toArray(keyspaces, String.class));
-            } catch (IOException e)
-            {
-                throw new RuntimeException("Error during clearing snapshots", e);
-            }
-        }
-    }
-
-    @Command(name = "compact", description = "Force a (major) compaction on one or more column families")
-    public static class Compact extends NodeToolCmd
-    {
-        @Arguments(usage = "[<keyspace> <cfnames>...]", description = "The keyspace followed by one or many column families")
-        private List<String> args = new ArrayList<>();
-
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            List<String> keyspaces = parseOptionalKeyspace(args, probe);
-            String[] cfnames = parseOptionalColumnFamilies(args);
-
-            for (String keyspace : keyspaces)
-            {
-                try
-                {
-                    probe.forceKeyspaceCompaction(keyspace, cfnames);
-                } catch (Exception e)
-                {
-                    throw new RuntimeException("Error occurred during compaction", e);
-                }
-            }
-        }
-    }
-
-    @Command(name = "flush", description = "Flush one or more column families")
-    public static class Flush extends NodeToolCmd
-    {
-        @Arguments(usage = "[<keyspace> <cfnames>...]", description = "The keyspace followed by one or many column families")
-        private List<String> args = new ArrayList<>();
-
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            List<String> keyspaces = parseOptionalKeyspace(args, probe);
-            String[] cfnames = parseOptionalColumnFamilies(args);
-
-            for (String keyspace : keyspaces)
-            {
-                try
-                {
-                    probe.forceKeyspaceFlush(keyspace, cfnames);
-                } catch (Exception e)
-                {
-                    throw new RuntimeException("Error occurred during flushing", e);
-                }
-            }
-        }
-    }
-
-    @Command(name = "scrub", description = "Scrub (rebuild sstables for) one or more column families")
-    public static class Scrub extends NodeToolCmd
-    {
-        @Arguments(usage = "[<keyspace> <cfnames>...]", description = "The keyspace followed by one or many column families")
-        private List<String> args = new ArrayList<>();
-
-        @Option(title = "disable_snapshot",
-                name = {"-ns", "--no-snapshot"},
-                description = "Scrubbed CFs will be snapshotted first, if disableSnapshot is false. (default false)")
-        private boolean disableSnapshot = false;
-
-        @Option(title = "skip_corrupted",
-                name = {"-s", "--skip-corrupted"},
-                description = "Skip corrupted partitions even when scrubbing counter tables. (default false)")
-        private boolean skipCorrupted = false;
-
-        @Option(title = "no_validate",
-                name = {"-n", "--no-validate"},
-                description = "Do not validate columns using column validator")
-        private boolean noValidation = false;
-
-        @Option(title = "jobs",
-                name = {"-j", "--jobs"},
-                description = "Number of sstables to scrub simultanously, set to 0 to use all available compaction threads")
-        private int jobs = 2;
-
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            List<String> keyspaces = parseOptionalKeyspace(args, probe);
-            String[] cfnames = parseOptionalColumnFamilies(args);
-
-            for (String keyspace : keyspaces)
-            {
-                try
-                {
-                    probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, jobs, keyspace, cfnames);
-                } catch (Exception e)
-                {
-                    throw new RuntimeException("Error occurred during flushing", e);
-                }
-            }
-        }
-    }
-
-    @Command(name = "disableautocompaction", description = "Disable autocompaction for the given keyspace and column family")
-    public static class DisableAutoCompaction extends NodeToolCmd
-    {
-        @Arguments(usage = "[<keyspace> <cfnames>...]", description = "The keyspace followed by one or many column families")
-        private List<String> args = new ArrayList<>();
-
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            List<String> keyspaces = parseOptionalKeyspace(args, probe);
-            String[] cfnames = parseOptionalColumnFamilies(args);
-
-            for (String keyspace : keyspaces)
-            {
-                try
-                {
-                    probe.disableAutoCompaction(keyspace, cfnames);
-                } catch (IOException e)
-                {
-                    throw new RuntimeException("Error occurred during disabling auto-compaction", e);
-                }
-            }
-        }
-    }
-
-    @Command(name = "enableautocompaction", description = "Enable autocompaction for the given keyspace and column family")
-    public static class EnableAutoCompaction extends NodeToolCmd
-    {
-        @Arguments(usage = "[<keyspace> <cfnames>...]", description = "The keyspace followed by one or many column families")
-        private List<String> args = new ArrayList<>();
-
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            List<String> keyspaces = parseOptionalKeyspace(args, probe);
-            String[] cfnames = parseOptionalColumnFamilies(args);
-
-            for (String keyspace : keyspaces)
-            {
-                try
-                {
-                    probe.enableAutoCompaction(keyspace, cfnames);
-                } catch (IOException e)
-                {
-                    throw new RuntimeException("Error occurred during enabling auto-compaction", e);
-                }
-            }
-        }
-    }
-
-    @Command(name = "upgradesstables", description = "Rewrite sstables (for the requested column families) that are not on the current version (thus upgrading them to said current version)")
-    public static class UpgradeSSTable extends NodeToolCmd
-    {
-        @Arguments(usage = "[<keyspace> <cfnames>...]", description = "The keyspace followed by one or many column families")
-        private List<String> args = new ArrayList<>();
-
-        @Option(title = "include_all", name = {"-a", "--include-all-sstables"}, description = "Use -a to include all sstables, even those already on the current version")
-        private boolean includeAll = false;
-
-        @Option(title = "jobs",
-                name = {"-j","--jobs"},
-                description = "Number of sstables to upgrade simultaneously, set to 0 to use all available compaction threads")
-        private int jobs = 2;
-
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            List<String> keyspaces = parseOptionalKeyspace(args, probe);
-            String[] cfnames = parseOptionalColumnFamilies(args);
-
-            for (String keyspace : keyspaces)
-            {
-                try
-                {
-                    probe.upgradeSSTables(System.out, keyspace, !includeAll, jobs, cfnames);
-                } catch (Exception e)
-                {
-                    throw new RuntimeException("Error occurred during enabling auto-compaction", e);
-                }
-            }
-        }
-    }
-
-    @Command(name = "compactionstats", description = "Print statistics on compactions")
-    public static class CompactionStats extends NodeToolCmd
-    {
-        @Option(title = "human_readable",
-                name = {"-H", "--human-readable"},
-                description = "Display bytes in human readable form, i.e. KB, MB, GB, TB")
-        private boolean humanReadable = false;
-
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            int compactionThroughput = probe.getCompactionThroughput();
-            CompactionManagerMBean cm = probe.getCompactionManagerProxy();
-            System.out.println("pending tasks: " + probe.getCompactionMetric("PendingTasks"));
-            long remainingBytes = 0;
-            List<Map<String, String>> compactions = cm.getCompactions();
-            if (!compactions.isEmpty())
-            {
-                List<String[]> lines = new ArrayList<>();
-                int[] columnSizes = new int[] { 0, 0, 0, 0, 0, 0, 0 };
-
-                addLine(lines, columnSizes, "compaction type", "keyspace", "table", "completed", "total", "unit", "progress");
-                for (Map<String, String> c : compactions)
-                {
-                    long total = Long.parseLong(c.get("total"));
-                    long completed = Long.parseLong(c.get("completed"));
-                    String taskType = c.get("taskType");
-                    String keyspace = c.get("keyspace");
-                    String columnFamily = c.get("columnfamily");
-                    String completedStr = humanReadable ? FileUtils.stringifyFileSize(completed) : Long.toString(completed);
-                    String totalStr = humanReadable ? FileUtils.stringifyFileSize(total) : Long.toString(total);
-                    String unit = c.get("unit");
-                    String percentComplete = total == 0 ? "n/a" : new DecimalFormat("0.00").format((double) completed / total * 100) + "%";
-                    addLine(lines, columnSizes, taskType, keyspace, columnFamily, completedStr, totalStr, unit, percentComplete);
-                    if (taskType.equals(OperationType.COMPACTION.toString()))
-                        remainingBytes += total - completed;
-                }
-
-                StringBuilder buffer = new StringBuilder();
-                for (int columnSize : columnSizes) {
-                    buffer.append("%");
-                    buffer.append(columnSize + 3);
-                    buffer.append("s");
-                }
-                buffer.append("%n");
-                String format = buffer.toString();
-
-                for (String[] line : lines)
-                {
-                    System.out.printf(format, line[0], line[1], line[2], line[3], line[4], line[5], line[6]);
-                }
-
-                String remainingTime = "n/a";
-                if (compactionThroughput != 0)
-                {
-                    long remainingTimeInSecs = remainingBytes / (1024L * 1024L * compactionThroughput);
-                    remainingTime = format("%dh%02dm%02ds", remainingTimeInSecs / 3600, (remainingTimeInSecs % 3600) / 60, (remainingTimeInSecs % 60));
-                }
-                System.out.printf("%25s%10s%n", "Active compaction remaining time : ", remainingTime);
-            }
-        }
-
-        private void addLine(List<String[]> lines, int[] columnSizes, String... columns) {
-            lines.add(columns);
-            for (int i = 0; i < columns.length; i++) {
-                columnSizes[i] = Math.max(columnSizes[i], columns[i].length());
-            }
-        }
-    }
-
-    @Command(name = "compactionhistory", description = "Print history of compaction")
-    public static class CompactionHistory extends NodeToolCmd
-    {
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            System.out.println("Compaction History: ");
-
-            TabularData tabularData = probe.getCompactionHistory();
-            if (tabularData.isEmpty())
-            {
-                System.out.printf("There is no compaction history");
-                return;
-            }
-
-            String format = "%-41s%-19s%-29s%-26s%-15s%-15s%s%n";
-            List<String> indexNames = tabularData.getTabularType().getIndexNames();
-            System.out.printf(format, toArray(indexNames, Object.class));
-
-            Set<?> values = tabularData.keySet();
-            for (Object eachValue : values)
-            {
-                List<?> value = (List<?>) eachValue;
-                System.out.printf(format, toArray(value, Object.class));
-            }
-        }
-    }
-
-    @Command(name = "decommission", description = "Decommission the *node I am connecting to*")
-    public static class Decommission extends NodeToolCmd
-    {
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            try
-            {
-                probe.decommission();
-            } catch (InterruptedException e)
-            {
-                throw new RuntimeException("Error decommissioning node", e);
-            } catch (UnsupportedOperationException e)
-            {
-                throw new IllegalStateException("Unsupported operation: " + e.getMessage(), e);
-            }
-        }
-    }
-
-    @Command(name = "describecluster", description = "Print the name, snitch, partitioner and schema version of a cluster")
-    public static class DescribeCluster extends NodeToolCmd
-    {
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            // display cluster name, snitch and partitioner
-            System.out.println("Cluster Information:");
-            System.out.println("\tName: " + probe.getClusterName());
-            System.out.println("\tSnitch: " + probe.getEndpointSnitchInfoProxy().getSnitchName());
-            System.out.println("\tPartitioner: " + probe.getPartitioner());
-
-            // display schema version for each node
-            System.out.println("\tSchema versions:");
-            Map<String, List<String>> schemaVersions = probe.getSpProxy().getSchemaVersions();
-            for (String version : schemaVersions.keySet())
-            {
-                System.out.println(format("\t\t%s: %s%n", version, schemaVersions.get(version)));
-            }
-        }
-    }
-
-    @Command(name = "disablebinary", description = "Disable native transport (binary protocol)")
-    public static class DisableBinary extends NodeToolCmd
-    {
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            probe.stopNativeTransport();
-        }
-    }
-
-    @Command(name = "enablebinary", description = "Reenable native transport (binary protocol)")
-    public static class EnableBinary extends NodeToolCmd
-    {
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            probe.startNativeTransport();
-        }
-    }
-
-    @Command(name = "enablegossip", description = "Reenable gossip")
-    public static class EnableGossip extends NodeToolCmd
-    {
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            probe.startGossiping();
-        }
-    }
-
-    @Command(name = "disablegossip", description = "Disable gossip (effectively marking the node down)")
-    public static class DisableGossip extends NodeToolCmd
-    {
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            probe.stopGossiping();
-        }
-    }
-
-    @Command(name = "enablehandoff", description = "Reenable the future hints storing on the current node")
-    public static class EnableHandoff extends NodeToolCmd
-    {
-        @Arguments(usage = "<dc-name>,<dc-name>", description = "Enable hinted handoff only for these DCs")
-        private List<String> args = new ArrayList<>();
-
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            checkArgument(args.size() <= 1, "enablehandoff does not accept two args");
-            if(args.size() == 1)
-                probe.enableHintedHandoff(args.get(0));
-            else
-                probe.enableHintedHandoff();
-        }
-    }
-
-    @Command(name = "enablethrift", description = "Reenable thrift server")
-    public static class EnableThrift extends NodeToolCmd
-    {
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            probe.startThriftServer();
-        }
-    }
-
-    @Command(name = "getcompactionthreshold", description = "Print min and max compaction thresholds for a given column family")
-    public static class GetCompactionThreshold extends NodeToolCmd
-    {
-        @Arguments(usage = "<keyspace> <cfname>", description = "The keyspace with a column family")
-        private List<String> args = new ArrayList<>();
-
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            checkArgument(args.size() == 2, "getcompactionthreshold requires ks and cf args");
-            String ks = args.get(0);
-            String cf = args.get(1);
-
-            ColumnFamilyStoreMBean cfsProxy = probe.getCfsProxy(ks, cf);
-            System.out.println("Current compaction thresholds for " + ks + "/" + cf + ": \n" +
-                    " min = " + cfsProxy.getMinimumCompactionThreshold() + ", " +
-                    " max = " + cfsProxy.getMaximumCompactionThreshold());
-        }
-    }
-
-    @Command(name = "getcompactionthroughput", description = "Print the MB/s throughput cap for compaction in the system")
-    public static class GetCompactionThroughput extends NodeToolCmd
-    {
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            System.out.println("Current compaction throughput: " + probe.getCompactionThroughput() + " MB/s");
-        }
-    }
-
-    @Command(name = "getstreamthroughput", description = "Print the Mb/s throughput cap for streaming in the system")
-    public static class GetStreamThroughput extends NodeToolCmd
-    {
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            System.out.println("Current stream throughput: " + probe.getStreamThroughput() + " Mb/s");
-        }
-    }
-
-    @Command(name = "getinterdcstreamthroughput", description = "Print the Mb/s throughput cap for inter-datacenter streaming in the system")
-    public static class GetInterDCStreamThroughput extends NodeToolCmd
-    {
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            System.out.println("Current inter-datacenter stream throughput: " + probe.getInterDCStreamThroughput() + " Mb/s");
-        }
-    }
-
-    @Command(name = "getendpoints", description = "Print the end points that owns the key")
-    public static class GetEndpoints extends NodeToolCmd
-    {
-        @Arguments(usage = "<keyspace> <cfname> <key>", description = "The keyspace, the column family, and the partition key for which we need to find the endpoint")
-        private List<String> args = new ArrayList<>();
-
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            checkArgument(args.size() == 3, "getendpoints requires ks, cf and key args");
-            String ks = args.get(0);
-            String cf = args.get(1);
-            String key = args.get(2);
-
-            List<InetAddress> endpoints = probe.getEndpoints(ks, cf, key);
-            for (InetAddress endpoint : endpoints)
-            {
-                System.out.println(endpoint.getHostAddress());
-            }
-        }
-    }
-
-    @Command(name = "getsstables", description = "Print the sstable filenames that own the key")
-    public static class GetSSTables extends NodeToolCmd
-    {
-        @Arguments(usage = "<keyspace> <cfname> <key>", description = "The keyspace, the column family, and the key")
-        private List<String> args = new ArrayList<>();
-
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            checkArgument(args.size() == 3, "getsstables requires ks, cf and key args");
-            String ks = args.get(0);
-            String cf = args.get(1);
-            String key = args.get(2);
-
-            List<String> sstables = probe.getSSTables(ks, cf, key);
-            for (String sstable : sstables)
-            {
-                System.out.println(sstable);
-            }
-        }
-    }
-
-    @Command(name = "gossipinfo", description = "Shows the gossip information for the cluster")
-    public static class GossipInfo extends NodeToolCmd
-    {
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            System.out.println(probe.getGossipInfo());
-        }
-    }
-
-    @Command(name = "invalidatekeycache", description = "Invalidate the key cache")
-    public static class InvalidateKeyCache extends NodeToolCmd
-    {
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            probe.invalidateKeyCache();
-        }
-    }
-
-    @Command(name = "invalidaterowcache", description = "Invalidate the row cache")
-    public static class InvalidateRowCache extends NodeToolCmd
-    {
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            probe.invalidateRowCache();
-        }
-    }
-
-    @Command(name = "invalidatecountercache", description = "Invalidate the counter cache")
-    public static class InvalidateCounterCache extends NodeToolCmd
-    {
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            probe.invalidateCounterCache();
-        }
-    }
-
-    @Command(name = "join", description = "Join the ring")
-    public static class Join extends NodeToolCmd
-    {
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            checkState(!probe.isJoined(), "This node has already joined the ring.");
-
-            try
-            {
-                probe.joinRing();
-            } catch (IOException e)
-            {
-                throw new RuntimeException("Error during joining the ring", e);
-            }
-        }
-    }
-
-    @Command(name = "move", description = "Move node on the token ring to a new token")
-    public static class Move extends NodeToolCmd
-    {
-        @Arguments(usage = "<new token>", description = "The new token.", required = true)
-        private String newToken = EMPTY;
-
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            try
-            {
-                probe.move(newToken);
-            } catch (IOException e)
-            {
-                throw new RuntimeException("Error during moving node", e);
-            }
-        }
-    }
-
-
-
-    @Command(name = "pausehandoff", description = "Pause hints delivery process")
-    public static class PauseHandoff extends NodeToolCmd
-    {
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            probe.pauseHintsDelivery();
-        }
-    }
-
-    @Command(name = "resumehandoff", description = "Resume hints delivery process")
-    public static class ResumeHandoff extends NodeToolCmd
-    {
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            probe.resumeHintsDelivery();
-        }
-    }
-
-
-    @Command(name = "proxyhistograms", description = "Print statistic histograms for network operations")
-    public static class ProxyHistograms extends NodeToolCmd
-    {
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            StorageProxyMBean sp = probe.getSpProxy();
-            String[] percentiles = new String[]{"50%", "75%", "95%", "98%", "99%", "Min", "Max"};
-            double[] readLatency = probe.metricPercentilesAsArray(sp.getRecentReadLatencyHistogramMicros());
-            double[] writeLatency = probe.metricPercentilesAsArray(sp.getRecentWriteLatencyHistogramMicros());
-            double[] rangeLatency = probe.metricPercentilesAsArray(sp.getRecentRangeLatencyHistogramMicros());
-
-            System.out.println("proxy histograms");
-            System.out.println(format("%-10s%18s%18s%18s",
-                    "Percentile", "Read Latency", "Write Latency", "Range Latency"));
-            System.out.println(format("%-10s%18s%18s%18s",
-                    "", "(micros)", "(micros)", "(micros)"));
-            for (int i = 0; i < percentiles.length; i++)
-            {
-                System.out.println(format("%-10s%18.2f%18.2f%18.2f",
-                        percentiles[i],
-                        readLatency[i],
-                        writeLatency[i],
-                        rangeLatency[i]));
-            }
-            System.out.println();
-        }
-    }
-
-    @Command(name = "rebuild", description = "Rebuild data by streaming from other nodes (similarly to bootstrap)")
-    public static class Rebuild extends NodeToolCmd
-    {
-        @Arguments(usage = "<src-dc-name>", description = "Name of DC from which to select sources for streaming. By default, pick any DC")
-        private String sourceDataCenterName = null;
-
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            probe.rebuild(sourceDataCenterName);
-        }
-    }
-
-    @Command(name = "refresh", description = "Load newly placed SSTables to the system without restart")
-    public static class Refresh extends NodeToolCmd
-    {
-        @Arguments(usage = "<keyspace> <cfname>", description = "The keyspace and column family name")
-        private List<String> args = new ArrayList<>();
-
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            checkArgument(args.size() == 2, "refresh requires ks and cf args");
-            probe.loadNewSSTables(args.get(0), args.get(1));
-        }
-    }
-
-    @Command(name = "refreshsizeestimates", description = "Refresh system.size_estimates")
-    public static class RefreshSizeEstimates extends NodeToolCmd
-    {
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            probe.refreshSizeEstimates();
-        }
-    }
-
-    @Deprecated
-    @Command(name = "removetoken", description = "DEPRECATED (see removenode)", hidden = true)
-    public static class RemoveToken extends NodeToolCmd
-    {
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            System.err.println("Warn: removetoken is deprecated, please use removenode instead");
-        }
-    }
-
-    @Command(name = "removenode", description = "Show status of current node removal, force completion of pending removal or remove provided ID")
-    public static class RemoveNode extends NodeToolCmd
-    {
-        @Arguments(title = "remove_operation", usage = "<status>|<force>|<ID>", description = "Show status of current node removal, force completion of pending removal, or remove provided ID", required = true)
-        private String removeOperation = EMPTY;
-
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            switch (removeOperation)
-            {
-                case "status":
-                    System.out.println("RemovalStatus: " + probe.getRemovalStatus());
-                    break;
-                case "force":
-                    System.out.println("RemovalStatus: " + probe.getRemovalStatus());
-                    probe.forceRemoveCompletion();
-                    break;
-                default:
-                    probe.removeNode(removeOperation);
-                    break;
-            }
-        }
-    }
-
-    @Command(name = "repair", description = "Repair one or more column families")
-    public static class Repair extends NodeToolCmd
-    {
-        @Arguments(usage = "[<keyspace> <cfnames>...]", description = "The keyspace followed by one or many column families")
-        private List<String> args = new ArrayList<>();
-
-        @Option(title = "parallel", name = {"-par", "--parallel"}, description = "Use -par to carry out a parallel repair")
-        private boolean parallel = false;
-
-        @Option(title = "dc parallel", name = {"-dcpar", "--dc-parallel"}, description = "Use -dcpar to repair data centers in parallel.")
-        private boolean dcParallel = false;
-
-        @Option(title = "local_dc", name = {"-local", "--in-local-dc"}, description = "Use -local to only repair against nodes in the same datacenter")
-        private boolean localDC = false;
-
-        @Option(title = "specific_dc", name = {"-dc", "--in-dc"}, description = "Use -dc to repair specific datacenters")
-        private List<String> specificDataCenters = new ArrayList<>();
-
-        @Option(title = "specific_host", name = {"-hosts", "--in-hosts"}, description = "Use -hosts to repair specific hosts")
-        private List<String> specificHosts = new ArrayList<>();
-
-        @Option(title = "start_token", name = {"-st", "--start-token"}, description = "Use -st to specify a token at which the repair range starts")
-        private String startToken = EMPTY;
-
-        @Option(title = "end_token", name = {"-et", "--end-token"}, description = "Use -et to specify a token at which repair range ends")
-        private String endToken = EMPTY;
-
-        @Option(title = "primary_range", name = {"-pr", "--partitioner-range"}, description = "Use -pr to repair only the first range returned by the partitioner")
-        private boolean primaryRange = false;
-
-        @Option(title = "incremental_repair", name = {"-inc", "--incremental"}, description = "Use -inc to use the new incremental repair")
-        private boolean incrementalRepair = false;
-
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            List<String> keyspaces = parseOptionalKeyspace(args, probe);
-            String[] cfnames = parseOptionalColumnFamilies(args);
-
-            if (primaryRange && (!specificDataCenters.isEmpty() || !specificHosts.isEmpty()))
-                throw new RuntimeException("Primary range repair should be performed on all nodes in the cluster.");
-
-            for (String keyspace : keyspaces)
-            {
-                try
-                {
-                    RepairParallelism parallelismDegree = RepairParallelism.SEQUENTIAL;
-                    if (parallel)
-                        parallelismDegree = RepairParallelism.PARALLEL;
-                    else if (dcParallel)
-                        parallelismDegree = RepairParallelism.DATACENTER_AWARE;
-
-                    Collection<String> dataCenters = null;
-                    Collection<String> hosts = null;
-                    if (!specificDataCenters.isEmpty())
-                        dataCenters = newArrayList(specificDataCenters);
-                    else if (localDC)
-                        dataCenters = newArrayList(probe.getDataCenter());
-                    else if(!specificHosts.isEmpty())
-                        hosts = newArrayList(specificHosts);
-                    if (!startToken.isEmpty() || !endToken.isEmpty())
-                        probe.forceRepairRangeAsync(System.out, keyspace, parallelismDegree, dataCenters,hosts, startToken, endToken, !incrementalRepair, cfnames);
-                    else
-                        probe.forceRepairAsync(System.out, keyspace, parallelismDegree, dataCenters, hosts, primaryRange, !incrementalRepair, cfnames);
-                } catch (Exception e)
-                {
-                    throw new RuntimeException("Error occurred during repair", e);
-                }
-            }
-        }
-    }
-
-    @Command(name = "setcachecapacity", description = "Set global key, row, and counter cache capacities (in MB units)")
-    public static class SetCacheCapacity extends NodeToolCmd
-    {
-        @Arguments(title = "<key-cache-capacity> <row-cache-capacity> <counter-cache-capacity>",
-                   usage = "<key-cache-capacity> <row-cache-capacity> <counter-cache-capacity>",
-                   description = "Key cache, row cache, and counter cache (in MB)",
-                   required = true)
-        private List<Integer> args = new ArrayList<>();
-
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            checkArgument(args.size() == 3, "setcachecapacity requires key-cache-capacity, row-cache-capacity, and counter-cache-capacity args.");
-            probe.setCacheCapacities(args.get(0), args.get(1), args.get(2));
-        }
-    }
-
-    @Command(name = "setcompactionthreshold", description = "Set min and max compaction thresholds for a given column family")
-    public static class SetCompactionThreshold extends NodeToolCmd
-    {
-        @Arguments(title = "<keyspace> <cfname> <minthreshold> <maxthreshold>", usage = "<keyspace> <cfname> <minthreshold> <maxthreshold>", description = "The keyspace, the column family, min and max threshold", required = true)
-        private List<String> args = new ArrayList<>();
-
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            checkArgument(args.size() == 4, "setcompactionthreshold requires ks, cf, min, and max threshold args.");
-
-            int minthreshold = parseInt(args.get(2));
-            int maxthreshold = parseInt(args.get(3));
-            checkArgument(minthreshold >= 0 && maxthreshold >= 0, "Thresholds must be positive integers");
-            checkArgument(minthreshold <= maxthreshold, "Min threshold cannot be greater than max.");
-            checkArgument(minthreshold >= 2 || maxthreshold == 0, "Min threshold must be at least 2");
-
-            probe.setCompactionThreshold(args.get(0), args.get(1), minthreshold, maxthreshold);
-        }
-    }
-
-    @Command(name = "setcompactionthroughput", description = "Set the MB/s throughput cap for compaction in the system, or 0 to disable throttling")
-    public static class SetCompactionThroughput extends NodeToolCmd
-    {
-        @Arguments(title = "compaction_throughput", usage = "<value_in_mb>", description = "Value in MB, 0 to disable throttling", required = true)
-        private Integer compactionThroughput = null;
-
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            probe.setCompactionThroughput(compactionThroughput);
-        }
-    }
-
-    @Command(name = "sethintedhandoffthrottlekb", description =  "Set hinted handoff throttle in kb per second, per delivery thread.")
-    public static class SetHintedHandoffThrottleInKB extends NodeToolCmd
-    {
-        @Arguments(title = "throttle_in_kb", usage = "<value_in_kb_per_sec>", description = "Value in KB per second", required = true)
-        private Integer throttleInKB = null;
-
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            probe.setHintedHandoffThrottleInKB(throttleInKB);
-        }
-    }
-
-    @Command(name = "setstreamthroughput", description = "Set the Mb/s throughput cap for streaming in the system, or 0 to disable throttling")
-    public static class SetStreamThroughput extends NodeToolCmd
-    {
-        @Arguments(title = "stream_throughput", usage = "<value_in_mb>", description = "Value in Mb, 0 to disable throttling", required = true)
-        private Integer streamThroughput = null;
-
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            probe.setStreamThroughput(streamThroughput);
-        }
-    }
-
-    @Command(name = "setinterdcstreamthroughput", description = "Set the Mb/s throughput cap for inter-datacenter streaming in the system, or 0 to disable throttling")
-    public static class SetInterDCStreamThroughput extends NodeToolCmd
-    {
-        @Arguments(title = "inter_dc_stream_throughput", usage = "<value_in_mb>", description = "Value in Mb, 0 to disable throttling", required = true)
-        private Integer interDCStreamThroughput = null;
-
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            probe.setInterDCStreamThroughput(interDCStreamThroughput);
-        }
-    }
-
-    @Command(name = "settraceprobability", description = "Sets the probability for tracing any given request to value. 0 disables, 1 enables for all requests, 0 is the default")
-    public static class SetTraceProbability extends NodeToolCmd
-    {
-        @Arguments(title = "trace_probability", usage = "<value>", description = "Trace probability between 0 and 1 (ex: 0.2)", required = true)
-        private Double traceProbability = null;
-
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            checkArgument(traceProbability >= 0 && traceProbability <= 1, "Trace probability must be between 0 and 1");
-            probe.setTraceProbability(traceProbability);
-        }
-    }
-
-    @Command(name = "snapshot", description = "Take a snapshot of specified keyspaces or a snapshot of the specified column family")
-    public static class Snapshot extends NodeToolCmd
-    {
-        @Arguments(usage = "[<keyspaces...>]", description = "List of keyspaces. By default, all keyspaces")
-        private List<String> keyspaces = new ArrayList<>();
-
-        @Option(title = "cfname", name = {"-cf", "--column-family"}, description = "The column family name (you must specify one and only one keyspace for using this option)")
-        private String columnFamily = null;
-
-        @Option(title = "tag", name = {"-t", "--tag"}, description = "The name of the snapshot")
-        private String snapshotName = Long.toString(System.currentTimeMillis());
-
-        @Option(title = "kclist", name = { "-kc", "--kc-list" }, description = "The list of Keyspace.Column family to take snapshot.(you must not specify only keyspace)")
-        private String kcList = null;
-
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            try
-            {
-                StringBuilder sb = new StringBuilder();
-
-                sb.append("Requested creating snapshot(s) for ");
-
-                // Create a separate path for kclist to avoid breaking of already existing scripts
-                if (null != kcList && !kcList.isEmpty())
-                {
-                    kcList = kcList.replace(" ", "");
-                    if (keyspaces.isEmpty() && null == columnFamily)
-                        sb.append("[").append(kcList).append("]");
-                    else
-                    {
-                        throw new IOException(
-                                "When specifying the Keyspace columfamily list for a snapshot, you should not specify columnfamily");
-                    }
-                    if (!snapshotName.isEmpty())
-                        sb.append(" with snapshot name [").append(snapshotName).append("]");
-                    System.out.println(sb.toString());
-                    probe.takeMultipleColumnFamilySnapshot(snapshotName, kcList.split(","));
-                    System.out.println("Snapshot directory: " + snapshotName);
-                }
-                else
-                {
-                    if (keyspaces.isEmpty())
-                        sb.append("[all keyspaces]");
-                    else
-                        sb.append("[").append(join(keyspaces, ", ")).append("]");
-
-                    if (!snapshotName.isEmpty())
-                        sb.append(" with snapshot name [").append(snapshotName).append("]");
-
-                    System.out.println(sb.toString());
-
-                    probe.takeSnapshot(snapshotName, columnFamily, toArray(keyspaces, String.class));
-                    System.out.println("Snapshot directory: " + snapshotName);
-                }
-            }
-            catch (IOException e)
-            {
-                throw new RuntimeException("Error during taking a snapshot", e);
-            }
-        }
-    }
-
-    @Command(name = "listsnapshots", description = "Lists all the snapshots along with the size on disk and true size.")
-    public static class ListSnapshots extends NodeToolCmd
-    {
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            try
-            {
-                System.out.println("Snapshot Details: ");
-
-                final Map<String,TabularData> snapshotDetails = probe.getSnapshotDetails();
-                if (snapshotDetails.isEmpty())
-                {
-                    System.out.printf("There are no snapshots");
-                    return;
-                }
-
-                final long trueSnapshotsSize = probe.trueSnapshotsSize();
-                final String format = "%-20s%-29s%-29s%-19s%-19s%n";
-                // display column names only once
-                final List<String> indexNames = snapshotDetails.entrySet().iterator().next().getValue().getTabularType().getIndexNames();
-                System.out.printf(format, (Object[]) indexNames.toArray(new String[indexNames.size()]));
-
-                for (final Map.Entry<String, TabularData> snapshotDetail : snapshotDetails.entrySet())
-                {
-                    Set<?> values = snapshotDetail.getValue().keySet();
-                    for (Object eachValue : values)
-                    {
-                        final List<?> value = (List<?>) eachValue;
-                        System.out.printf(format, value.toArray(new Object[value.size()]));
-                    }
-                }
-
-                System.out.println("\nTotal TrueDiskSpaceUsed: " + FileUtils.stringifyFileSize(trueSnapshotsSize) + "\n");
-            }
-            catch (Exception e)
-            {
-                throw new RuntimeException("Error during list snapshot", e);
-            }
-        }
-    }
-
-    @Command(name = "status", description = "Print cluster information (state, load, IDs, ...)")
-    public static class Status extends NodeToolCmd
-    {
-        @Arguments(usage = "[<keyspace>]", description = "The keyspace name")
-        private String keyspace = null;
-
-        @Option(title = "resolve_ip", name = {"-r", "--resolve-ip"}, description = "Show node domain names instead of IPs")
-        private boolean resolveIp = false;
-
-        private boolean hasEffectiveOwns = false;
-        private boolean isTokenPerNode = true;
-        private int maxAddressLength = 0;
-        private String format = null;
-        private Collection<String> joiningNodes, leavingNodes, movingNodes, liveNodes, unreachableNodes;
-        private Map<String, String> loadMap, hostIDMap, tokensToEndpoints;
-        private EndpointSnitchInfoMBean epSnitchInfo;
-
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            joiningNodes = probe.getJoiningNodes();
-            leavingNodes = probe.getLeavingNodes();
-            movingNodes = probe.getMovingNodes();
-            loadMap = probe.getLoadMap();
-            tokensToEndpoints = probe.getTokenToEndpointMap();
-            liveNodes = probe.getLiveNodes();
-            unreachableNodes = probe.getUnreachableNodes();
-            hostIDMap = probe.getHostIdMap();
-            epSnitchInfo = probe.getEndpointSnitchInfoProxy();
-            
-            StringBuffer errors = new StringBuffer();
-
-            Map<InetAddress, Float> ownerships = null;
-            try
-            {
-                ownerships = probe.effectiveOwnership(keyspace);
-                hasEffectiveOwns = true;
-            } catch (IllegalStateException e)
-            {
-                ownerships = probe.getOwnership();
-                errors.append("Note: " + e.getMessage() + "%n");
-            }
-            catch (IllegalArgumentException ex)
-            {
-                System.out.printf("%nError: " + ex.getMessage() + "%n");
-                System.exit(1);
-            }
-
-            SortedMap<String, SetHostStat> dcs = getOwnershipByDc(probe, resolveIp, tokensToEndpoints, ownerships);
-
-            // More tokens than nodes (aka vnodes)?
-            if (dcs.values().size() < tokensToEndpoints.keySet().size())
-                isTokenPerNode = false;
-
-            findMaxAddressLength(dcs);
-
-            // Datacenters
-            for (Map.Entry<String, SetHostStat> dc : dcs.entrySet())
-            {
-                String dcHeader = String.format("Datacenter: %s%n", dc.getKey());
-                System.out.printf(dcHeader);
-                for (int i = 0; i < (dcHeader.length() - 1); i++) System.out.print('=');
-                System.out.println();
-
-                // Legend
-                System.out.println("Status=Up/Down");
-                System.out.println("|/ State=Normal/Leaving/Joining/Moving");
-
-                printNodesHeader(hasEffectiveOwns, isTokenPerNode);
-
-                ArrayListMultimap<InetAddress, HostStat> hostToTokens = ArrayListMultimap.create();
-                for (HostStat stat : dc.getValue())
-                    hostToTokens.put(stat.endpoint, stat);
-
-                for (InetAddress endpoint : hostToTokens.keySet())
-                {
-                    Float owns = ownerships.get(endpoint);
-                    List<HostStat> tokens = hostToTokens.get(endpoint);
-                    printNode(endpoint.getHostAddress(), owns, tokens, hasEffectiveOwns, isTokenPerNode);
-                }
-            }
-            
-            System.out.printf("%n" + errors.toString());
-            
-        }
-
-        private void findMaxAddressLength(Map<String, SetHostStat> dcs)
-        {
-            maxAddressLength = 0;
-            for (Map.Entry<String, SetHostStat> dc : dcs.entrySet())
-            {
-                for (HostStat stat : dc.getValue())
-                {
-                    maxAddressLength = Math.max(maxAddressLength, stat.ipOrDns().length());
-                }
-            }
-        }
-
-        private void printNodesHeader(boolean hasEffectiveOwns, boolean isTokenPerNode)
-        {
-            String fmt = getFormat(hasEffectiveOwns, isTokenPerNode);
-            String owns = hasEffectiveOwns ? "Owns (effective)" : "Owns";
-
-            if (isTokenPerNode)
-                System.out.printf(fmt, "-", "-", "Address", "Load", owns, "Host ID", "Token", "Rack");
-            else
-                System.out.printf(fmt, "-", "-", "Address", "Load", "Tokens", owns, "Host ID", "Rack");
-        }
-
-        private void printNode(String endpoint, Float owns, List<HostStat> tokens, boolean hasEffectiveOwns, boolean isTokenPerNode)
-        {
-            String status, state, load, strOwns, hostID, rack, fmt;
-            fmt = getFormat(hasEffectiveOwns, isTokenPerNode);
-            if (liveNodes.contains(endpoint)) status = "U";
-            else if (unreachableNodes.contains(endpoint)) status = "D";
-            else status = "?";
-            if (joiningNodes.contains(endpoint)) state = "J";
-            else if (leavingNodes.contains(endpoint)) state = "L";
-            else if (movingNodes.contains(endpoint)) state = "M";
-            else state = "N";
-
-            load = loadMap.containsKey(endpoint) ? loadMap.get(endpoint) : "?";
-            strOwns = owns != null && hasEffectiveOwns ? new DecimalFormat("##0.0%").format(owns) : "?";
-            hostID = hostIDMap.get(endpoint);
-
-            try
-            {
-                rack = epSnitchInfo.getRack(endpoint);
-            } catch (UnknownHostException e)
-            {
-                throw new RuntimeException(e);
-            }
-
-            String endpointDns = tokens.get(0).ipOrDns();
-            if (isTokenPerNode)
-                System.out.printf(fmt, status, state, endpointDns, load, strOwns, hostID, tokens.get(0).token, rack);
-            else
-                System.out.printf(fmt, status, state, endpointDns, load, tokens.size(), strOwns, hostID, rack);
-        }
-
-        private String getFormat(
-                boolean hasEffectiveOwns,
-                boolean isTokenPerNode)
-        {
-            if (format == null)
-            {
-                StringBuilder buf = new StringBuilder();
-                String addressPlaceholder = String.format("%%-%ds  ", maxAddressLength);
-                buf.append("%s%s  ");                         // status
-                buf.append(addressPlaceholder);               // address
-                buf.append("%-9s  ");                         // load
-                if (!isTokenPerNode)
-                    buf.append("%-6s  ");                     // "Tokens"
-                if (hasEffectiveOwns)
-                    buf.append("%-16s  ");                    // "Owns (effective)"
-                else
-                    buf.append("%-6s  ");                     // "Owns
-                buf.append("%-36s  ");                        // Host ID
-                if (isTokenPerNode)
-                    buf.append("%-39s  ");                    // token
-                buf.append("%s%n");                           // "Rack"
-
-                format = buf.toString();
-            }
-
-            return format;
-        }
-    }
-
-    private static SortedMap<String, SetHostStat> getOwnershipByDc(NodeProbe probe, boolean resolveIp,
-                                                                   Map<String, String> tokenToEndpoint,
-                                                                   Map<InetAddress, Float> ownerships)
+    public static SortedMap<String, SetHostStat> getOwnershipByDc(NodeProbe probe, boolean resolveIp,
+                                                                  Map<String, String> tokenToEndpoint,
+                                                                  Map<InetAddress, Float> ownerships)
     {
         SortedMap<String, SetHostStat> ownershipByDc = Maps.newTreeMap();
         EndpointSnitchInfoMBean epSnitchInfo = probe.getEndpointSnitchInfoProxy();
@@ -2359,400 +363,4 @@
         }
         return ownershipByDc;
     }
-
-    static class SetHostStat implements Iterable<HostStat>
-    {
-        final List<HostStat> hostStats = new ArrayList<HostStat>();
-        final boolean resolveIp;
-
-        public SetHostStat(boolean resolveIp)
-        {
-            this.resolveIp = resolveIp;
-        }
-
-        public int size()
-        {
-            return hostStats.size();
-        }
-
-        @Override
-        public Iterator<HostStat> iterator()
-        {
-            return hostStats.iterator();
-        }
-
-        public void add(String token, String host, Map<InetAddress, Float> ownerships) throws UnknownHostException
-        {
-            InetAddress endpoint = InetAddress.getByName(host);
-            Float owns = ownerships.get(endpoint);
-            hostStats.add(new HostStat(token, endpoint, resolveIp, owns));
-        }
-    }
-
-    static class HostStat
-    {
-        public final InetAddress endpoint;
-        public final boolean resolveIp;
-        public final Float owns;
-        public final String token;
-
-        public HostStat(String token, InetAddress endpoint, boolean resolveIp, Float owns)
-        {
-            this.token = token;
-            this.endpoint = endpoint;
-            this.resolveIp = resolveIp;
-            this.owns = owns;
-        }
-
-        public String ipOrDns()
-        {
-            return resolveIp ? endpoint.getHostName() : endpoint.getHostAddress();
-        }
-    }
-
-    @Command(name = "statusbinary", description = "Status of native transport (binary protocol)")
-    public static class StatusBinary extends NodeToolCmd
-    {
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            System.out.println(
-                    probe.isNativeTransportRunning()
-                    ? "running"
-                    : "not running");
-        }
-    }
-
-    @Command(name = "statusgossip", description = "Status of gossip")
-    public static class StatusGossip extends NodeToolCmd
-    {
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            System.out.println(
-                    probe.isGossipRunning()
-                    ? "running"
-                    : "not running");
-        }
-    }
-
-    @Command(name = "statusthrift", description = "Status of thrift server")
-    public static class StatusThrift extends NodeToolCmd
-    {
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            System.out.println(
-                    probe.isThriftServerRunning()
-                    ? "running"
-                    : "not running");
-        }
-    }
-
-    @Command(name = "statusbackup", description = "Status of incremental backup")
-    public static class StatusBackup extends NodeToolCmd
-    {
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            System.out.println(
-                    probe.isIncrementalBackupsEnabled()
-                    ? "running"
-                    : "not running");
-        }
-    }
-
-    @Command(name = "statushandoff", description = "Status of storing future hints on the current node")
-    public static class StatusHandoff extends NodeToolCmd
-    {
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            System.out.println(
-                    probe.isHandoffEnabled()
-                    ? "running"
-                    : "not running");
-        }
-    }
-
-    @Command(name = "stop", description = "Stop compaction")
-    public static class Stop extends NodeToolCmd
-    {
-        @Arguments(title = "compaction_type", usage = "<compaction type>", description = "Supported types are COMPACTION, VALIDATION, CLEANUP, SCRUB, INDEX_BUILD", required = true)
-        private OperationType compactionType = OperationType.UNKNOWN;
-
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            probe.stop(compactionType.name());
-        }
-    }
-
-    @Command(name = "stopdaemon", description = "Stop cassandra daemon")
-    public static class StopDaemon extends NodeToolCmd
-    {
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            try
-            {
-                probe.stopCassandraDaemon();
-            } catch (Exception e)
-            {
-                JVMStabilityInspector.inspectThrowable(e);
-                // ignored
-            }
-            System.out.println("Cassandra has shutdown.");
-        }
-    }
-
-    @Command(name = "version", description = "Print cassandra version")
-    public static class Version extends NodeToolCmd
-    {
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            System.out.println("ReleaseVersion: " + probe.getReleaseVersion());
-        }
-    }
-
-    @Command(name = "describering", description = "Shows the token ranges info of a given keyspace")
-    public static class DescribeRing extends NodeToolCmd
-    {
-        @Arguments(description = "The keyspace name", required = true)
-        String keyspace = EMPTY;
-
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            System.out.println("Schema Version:" + probe.getSchemaVersion());
-            System.out.println("TokenRange: ");
-            try
-            {
-                for (String tokenRangeString : probe.describeRing(keyspace))
-                {
-                    System.out.println("\t" + tokenRangeString);
-                }
-            } catch (IOException e)
-            {
-                throw new RuntimeException(e);
-            }
-        }
-    }
-
-    @Command(name = "rangekeysample", description = "Shows the sampled keys held across all keyspaces")
-    public static class RangeKeySample extends NodeToolCmd
-    {
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            System.out.println("RangeKeySample: ");
-            List<String> tokenStrings = probe.sampleKeyRange();
-            for (String tokenString : tokenStrings)
-            {
-                System.out.println("\t" + tokenString);
-            }
-        }
-    }
-
-    @Command(name = "rebuild_index", description = "A full rebuild of native secondary indexes for a given column family")
-    public static class RebuildIndex extends NodeToolCmd
-    {
-        @Arguments(usage = "<keyspace> <cfname> <indexName...>", description = "The keyspace and column family name followed by a list of index names (IndexNameExample: Standard3.IdxName Standard3.IdxName1)")
-        List<String> args = new ArrayList<>();
-
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            checkArgument(args.size() >= 3, "rebuild_index requires ks, cf and idx args");
-            probe.rebuildIndex(args.get(0), args.get(1), toArray(args.subList(2, args.size()), String.class));
-        }
-    }
-
-    @Command(name = "resetlocalschema", description = "Reset node's local schema and resync")
-    public static class ResetLocalSchema extends NodeToolCmd
-    {
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            try
-            {
-                probe.resetLocalSchema();
-            } catch (IOException e)
-            {
-                throw new RuntimeException(e);
-            }
-        }
-    }
-
-    @Command(name = "enablebackup", description = "Enable incremental backup")
-    public static class EnableBackup extends NodeToolCmd
-    {
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            probe.setIncrementalBackupsEnabled(true);
-        }
-    }
-
-    @Command(name = "disablebackup", description = "Disable incremental backup")
-    public static class DisableBackup extends NodeToolCmd
-    {
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            probe.setIncrementalBackupsEnabled(false);
-        }
-    }
-
-    @Command(name = "setcachekeystosave", description = "Set number of keys saved by each cache for faster post-restart warmup. 0 to disable")
-    public static class SetCacheKeysToSave extends NodeToolCmd
-    {
-        @Arguments(title = "<key-cache-keys-to-save> <row-cache-keys-to-save> <counter-cache-keys-to-save>",
-                   usage = "<key-cache-keys-to-save> <row-cache-keys-to-save> <counter-cache-keys-to-save>",
-                   description = "The number of keys saved by each cache. 0 to disable",
-                   required = true)
-        private List<Integer> args = new ArrayList<>();
-
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            checkArgument(args.size() == 3, "setcachekeystosave requires key-cache-keys-to-save, row-cache-keys-to-save, and counter-cache-keys-to-save args.");
-            probe.setCacheKeysToSave(args.get(0), args.get(1), args.get(2));
-        }
-    }
-
-    @Command(name = "reloadtriggers", description = "Reload trigger classes")
-    public static class ReloadTriggers extends NodeToolCmd
-    {
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            probe.reloadTriggers();
-        }
-    }
-
-    @Command(name = "disablehandoff", description = "Disable storing hinted handoffs")
-    public static class DisableHandoff extends NodeToolCmd
-    {
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            probe.disableHintedHandoff();
-        }
-    }
-
-    @Command(name = "disablethrift", description = "Disable thrift server")
-    public static class DisableThrift extends NodeToolCmd
-    {
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            probe.stopThriftServer();
-        }
-    }
-
-    @Command(name = "drain", description = "Drain the node (stop accepting writes and flush all column families)")
-    public static class Drain extends NodeToolCmd
-    {
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            try
-            {
-                probe.drain();
-            } catch (IOException | InterruptedException | ExecutionException e)
-            {
-                throw new RuntimeException("Error occured during flushing", e);
-            }
-        }
-    }
-
-    @Command(name = "tpstats", description = "Print usage statistics of thread pools")
-    public static class TpStats extends NodeTool.NodeToolCmd
-    {
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            System.out.printf("%-25s%10s%10s%15s%10s%18s%n", "Pool Name", "Active", "Pending", "Completed", "Blocked", "All time blocked");
-
-            Iterator<Map.Entry<String, JMXEnabledThreadPoolExecutorMBean>> threads = probe.getThreadPoolMBeanProxies();
-            while (threads.hasNext())
-            {
-                Map.Entry<String, JMXEnabledThreadPoolExecutorMBean> thread = threads.next();
-                String poolName = thread.getKey();
-                JMXEnabledThreadPoolExecutorMBean threadPoolProxy = thread.getValue();
-                System.out.printf("%-25s%10s%10s%15s%10s%18s%n",
-                        poolName,
-                        threadPoolProxy.getActiveCount(),
-                        threadPoolProxy.getPendingTasks(),
-                        threadPoolProxy.getCompletedTasks(),
-                        threadPoolProxy.getCurrentlyBlockedTasks(),
-                        threadPoolProxy.getTotalBlockedTasks());
-            }
-
-            System.out.printf("%n%-20s%10s%n", "Message type", "Dropped");
-            for (Map.Entry<String, Integer> entry : probe.getDroppedMessages().entrySet())
-                System.out.printf("%-20s%10s%n", entry.getKey(), entry.getValue());
-        }
-    }
-
-    @Command(name = "gcstats", description = "Print GC Statistics")
-    public static class GcStats extends NodeTool.NodeToolCmd
-    {
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            double[] stats = probe.getAndResetGCStats();
-            double mean = stats[2] / stats[5];
-            double stdev = Math.sqrt((stats[3] / stats[5]) - (mean * mean));
-            System.out.printf("%20s%20s%20s%20s%20s%20s%n", "Interval (ms)", "Max GC Elapsed (ms)", "Total GC Elapsed (ms)", "Stdev GC Elapsed (ms)", "GC Reclaimed (MB)", "Collections");
-            System.out.printf("%20.0f%20.0f%20.0f%20.0f%20.0f%20.0f%n", stats[0], stats[1], stats[2], stdev, stats[4], stats[5]);
-        }
-    }
-
-    @Command(name = "truncatehints", description = "Truncate all hints on the local node, or truncate hints for the endpoint(s) specified.")
-    public static class TruncateHints extends NodeToolCmd
-    {
-        @Arguments(usage = "[endpoint ... ]", description = "Endpoint address(es) to delete hints for, either ip address (\"127.0.0.1\") or hostname")
-        private String endpoint = EMPTY;
-
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            if (endpoint.isEmpty())
-                probe.truncateHints();
-            else
-                probe.truncateHints(endpoint);
-        }
-    }
-    
-    @Command(name = "setlogginglevel", description = "Set the log level threshold for a given class. If both class and level are empty/null, it will reset to the initial configuration")
-    public static class SetLoggingLevel extends NodeToolCmd
-    {
-        @Arguments(usage = "<class> <level>", description = "The class to change the level for and the log level threshold to set (can be empty)")
-        private List<String> args = new ArrayList<>();
-
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            String classQualifier = args.size() >= 1 ? args.get(0) : EMPTY;
-            String level = args.size() == 2 ? args.get(1) : EMPTY;
-            probe.setLoggingLevel(classQualifier, level);
-        }
-    }
-    
-    @Command(name = "getlogginglevels", description = "Get the runtime logging levels")
-    public static class GetLoggingLevels extends NodeToolCmd
-    {
-        @Override
-        public void execute(NodeProbe probe)
-        {
-            // what if some one set a very long logger name? 50 space may not be enough...
-            System.out.printf("%n%-50s%10s%n", "Logger Name", "Log Level");
-            for (Map.Entry<String, String> entry : probe.getLoggingLevels().entrySet())
-                System.out.printf("%-50s%10s%n", entry.getKey(), entry.getValue());
-        }
-    }
-
 }
diff --git a/src/java/org/apache/cassandra/tools/RepairRunner.java b/src/java/org/apache/cassandra/tools/RepairRunner.java
new file mode 100644
index 0000000..0813775
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/RepairRunner.java
@@ -0,0 +1,118 @@
+/*
+ * 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.tools;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.text.SimpleDateFormat;
+import java.util.Map;
+import java.util.concurrent.locks.Condition;
+
+import org.apache.cassandra.service.StorageServiceMBean;
+import org.apache.cassandra.utils.concurrent.SimpleCondition;
+import org.apache.cassandra.utils.progress.ProgressEvent;
+import org.apache.cassandra.utils.progress.ProgressEventType;
+import org.apache.cassandra.utils.progress.jmx.JMXNotificationProgressListener;
+
+public class RepairRunner extends JMXNotificationProgressListener
+{
+    private final SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss,SSS");
+
+    private final PrintStream out;
+    private final StorageServiceMBean ssProxy;
+    private final String keyspace;
+    private final Map<String, String> options;
+    private final Condition condition = new SimpleCondition();
+
+    private int cmd;
+    private volatile boolean hasNotificationLost;
+    private volatile Exception error;
+
+    public RepairRunner(PrintStream out, StorageServiceMBean ssProxy, String keyspace, Map<String, String> options)
+    {
+        this.out = out;
+        this.ssProxy = ssProxy;
+        this.keyspace = keyspace;
+        this.options = options;
+    }
+
+    public void run() throws Exception
+    {
+        cmd = ssProxy.repairAsync(keyspace, options);
+        if (cmd <= 0)
+        {
+            String message = String.format("[%s] Nothing to repair for keyspace '%s'", format.format(System.currentTimeMillis()), keyspace);
+            out.println(message);
+        }
+        else
+        {
+            condition.await();
+            if (error != null)
+            {
+                throw error;
+            }
+            if (hasNotificationLost)
+            {
+                out.println(String.format("There were some lost notification(s). You should check server log for repair status of keyspace %s", keyspace));
+            }
+        }
+    }
+
+    @Override
+    public boolean isInterestedIn(String tag)
+    {
+        return tag.equals("repair:" + cmd);
+    }
+
+    @Override
+    public void handleNotificationLost(long timestamp, String message)
+    {
+        hasNotificationLost = true;
+    }
+
+    @Override
+    public void handleConnectionClosed(long timestamp, String message)
+    {
+        handleConnectionFailed(timestamp, message);
+    }
+
+    @Override
+    public void handleConnectionFailed(long timestamp, String message)
+    {
+        error = new IOException(String.format("[%s] JMX connection closed. You should check server log for repair status of keyspace %s"
+                                               + "(Subsequent keyspaces are not going to be repaired).",
+                                  format.format(timestamp), keyspace));
+        condition.signalAll();
+    }
+
+    @Override
+    public void progress(String tag, ProgressEvent event)
+    {
+        ProgressEventType type = event.getType();
+        String message = String.format("[%s] %s", format.format(System.currentTimeMillis()), event.getMessage());
+        if (type == ProgressEventType.PROGRESS)
+        {
+            message = message + " (progress: " + (int)event.getProgressPercentage() + "%)";
+        }
+        out.println(message);
+        if (type == ProgressEventType.COMPLETE)
+        {
+            condition.signalAll();
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/tools/SSTableExpiredBlockers.java b/src/java/org/apache/cassandra/tools/SSTableExpiredBlockers.java
index 8205dbe..0d8c5e5 100644
--- a/src/java/org/apache/cassandra/tools/SSTableExpiredBlockers.java
+++ b/src/java/org/apache/cassandra/tools/SSTableExpiredBlockers.java
@@ -24,7 +24,6 @@
 import java.util.Map;
 import java.util.Set;
 
-import com.google.common.base.Throwables;
 import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.Multimap;
 
@@ -36,7 +35,7 @@
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 
 /**
  * During compaction we can drop entire sstables if they only contain expired tombstones and if it is guaranteed
@@ -56,9 +55,12 @@
             out.println("Usage: sstableexpiredblockers <keyspace> <table>");
             System.exit(1);
         }
+
+        Util.initDatabaseDescriptor();
+
         String keyspace = args[args.length - 2];
         String columnfamily = args[args.length - 1];
-        DatabaseDescriptor.loadSchemas();
+        Schema.instance.loadFromDisk(false);
 
         CFMetaData metadata = Schema.instance.getCFMetaData(keyspace, columnfamily);
         if (metadata == null)
diff --git a/src/java/org/apache/cassandra/tools/SSTableExport.java b/src/java/org/apache/cassandra/tools/SSTableExport.java
index d5c77c1..58070e6 100644
--- a/src/java/org/apache/cassandra/tools/SSTableExport.java
+++ b/src/java/org/apache/cassandra/tools/SSTableExport.java
@@ -22,10 +22,10 @@
 import java.io.PrintStream;
 import java.util.*;
 
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.commons.cli.*;
 
 import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.composites.CellNameType;
@@ -115,7 +115,7 @@
     }
 
     /**
-     * Serialize a given cell to a List of Objects that jsonMapper knows how to turn into strings.  Format is
+     * Serialize a given cell to a List of Objects that jsonMapper knows how to turn into strings.  Type is
      *
      * human_readable_name, value, timestamp, [flag, [options]]
      *
@@ -225,8 +225,7 @@
     public static void enumeratekeys(Descriptor desc, PrintStream outs, CFMetaData metadata)
     throws IOException
     {
-        KeyIterator iter = new KeyIterator(desc);
-        try
+        try (KeyIterator iter = new KeyIterator(desc))
         {
             DecoratedKey lastKey = null;
             while (iter.hasNext())
@@ -242,10 +241,6 @@
                 checkStream(outs); // flushes
             }
         }
-        finally
-        {
-            iter.close();
-        }
     }
 
     /**
@@ -261,8 +256,8 @@
     public static void export(Descriptor desc, PrintStream outs, Collection<String> toExport, String[] excludes, CFMetaData metadata) throws IOException
     {
         SSTableReader sstable = SSTableReader.open(desc);
-        RandomAccessReader dfile = sstable.openDataReader();
-        try
+
+        try (RandomAccessReader dfile = sstable.openDataReader())
         {
             IPartitioner partitioner = sstable.partitioner;
 
@@ -305,20 +300,16 @@
             outs.println("\n]");
             outs.flush();
         }
-        finally
-        {
-            dfile.close();
-        }
     }
 
     // This is necessary to accommodate the test suite since you cannot open a Reader more
     // than once from within the same process.
-    static void export(SSTableReader reader, PrintStream outs, String[] excludes, CFMetaData metadata) throws IOException
+    static void export(SSTableReader reader, PrintStream outs, String[] excludes) throws IOException
     {
         Set<String> excludeSet = new HashSet<String>();
 
         if (excludes != null)
-            excludeSet = new HashSet<String>(Arrays.asList(excludes));
+            excludeSet = new HashSet<>(Arrays.asList(excludes));
 
         SSTableIdentityIterator row;
         ISSTableScanner scanner = reader.getScanner();
@@ -361,12 +352,11 @@
      * @param desc     the descriptor of the sstable to read from
      * @param outs     PrintStream to write the output to
      * @param excludes keys to exclude from export
-     * @param metadata Metadata to print keys in a proper format
      * @throws IOException on failure to read/write input/output
      */
-    public static void export(Descriptor desc, PrintStream outs, String[] excludes, CFMetaData metadata) throws IOException
+    public static void export(Descriptor desc, PrintStream outs, String[] excludes) throws IOException
     {
-        export(SSTableReader.open(desc), outs, excludes, metadata);
+        export(SSTableReader.open(desc), outs, excludes);
     }
 
     /**
@@ -374,12 +364,11 @@
      *
      * @param desc     the descriptor of the sstable to read from
      * @param excludes keys to exclude from export
-     * @param metadata Metadata to print keys in a proper format
      * @throws IOException on failure to read/write SSTable/standard out
      */
-    public static void export(Descriptor desc, String[] excludes, CFMetaData metadata) throws IOException
+    public static void export(Descriptor desc, String[] excludes) throws IOException
     {
-        export(desc, System.out, excludes, metadata);
+        export(desc, System.out, excludes);
     }
 
     /**
@@ -387,11 +376,13 @@
      * export the contents of the SSTable to JSON.
      *
      * @param args command lines arguments
-     * @throws IOException            on failure to open/read/write files or output streams
      * @throws ConfigurationException on configuration failure (wrong params given)
      */
     public static void main(String[] args) throws ConfigurationException
     {
+        System.err.println("WARNING: please note that sstable2json is now deprecated and will be removed in Cassandra 3.0. "
+                         + "Please see https://issues.apache.org/jira/browse/CASSANDRA-9618 for details.");
+
         String usage = String.format("Usage: %s <sstable> [-k key [-k key [...]] -x key [-x key [...]]]%n", SSTableExport.class.getName());
 
         CommandLineParser parser = new PosixParser();
@@ -414,12 +405,13 @@
             System.exit(1);
         }
 
+        Util.initDatabaseDescriptor();
 
         String[] keys = cmd.getOptionValues(KEY_OPTION);
         String[] excludes = cmd.getOptionValues(EXCLUDEKEY_OPTION);
         String ssTableFileName = new File(cmd.getArgs()[0]).getAbsolutePath();
 
-        DatabaseDescriptor.loadSchemas(false);
+        Schema.instance.loadFromDisk(false);
         Descriptor descriptor = Descriptor.fromFilename(ssTableFileName);
 
         // Start by validating keyspace name
@@ -448,7 +440,7 @@
         }
         catch (IllegalArgumentException e)
         {
-            System.err.println(String.format("The provided column family is not part of this cassandra keyspace: keyspace = %s, column family = %s",
+            System.err.println(String.format("The provided table is not part of this cassandra keyspace: keyspace = %s, table = %s",
                                              descriptor.ksname, descriptor.cfname));
             System.exit(1);
         }
@@ -464,7 +456,7 @@
                 if ((keys != null) && (keys.length > 0))
                     export(descriptor, System.out, Arrays.asList(keys), excludes, cfStore.metadata);
                 else
-                    export(descriptor, excludes, cfStore.metadata);
+                    export(descriptor, excludes);
             }
         }
         catch (IOException e)
diff --git a/src/java/org/apache/cassandra/tools/SSTableImport.java b/src/java/org/apache/cassandra/tools/SSTableImport.java
index e0f3ef3..ad0ffa1 100644
--- a/src/java/org/apache/cassandra/tools/SSTableImport.java
+++ b/src/java/org/apache/cassandra/tools/SSTableImport.java
@@ -26,6 +26,8 @@
 import java.util.TreeMap;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLineParser;
 import org.apache.commons.cli.Option;
@@ -42,7 +44,6 @@
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.BytesType;
 import org.apache.cassandra.dht.IPartitioner;
-import org.apache.cassandra.io.sstable.SSTableWriter;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -70,7 +71,7 @@
     private final boolean isSorted;
 
     private static final JsonFactory factory = new MappingJsonFactory().configure(
-            JsonParser.Feature.INTERN_FIELD_NAMES, false);
+                                                                                 JsonParser.Feature.INTERN_FIELD_NAMES, false);
 
     static
     {
@@ -78,7 +79,7 @@
         optKeyspace.setRequired(true);
         options.addOption(optKeyspace);
 
-        Option optColfamily = new Option(COLUMN_FAMILY_OPTION, true, "Column Family name.");
+        Option optColfamily = new Option(COLUMN_FAMILY_OPTION, true, "Table name.");
         optColfamily.setRequired(true);
         options.addOption(optColfamily);
 
@@ -123,7 +124,7 @@
                     }
                     else if (isCounter())
                     {
-                        timestampOfLastDelete = (long) ((Integer) fields.get(4));
+                        timestampOfLastDelete = ((Integer) fields.get(4));
                     }
                     else if (isRangeTombstone())
                     {
@@ -142,10 +143,10 @@
                 else
                 {
                     assert meta.isCQL3Table() || name.hasRemaining() : "Cell name should not be empty";
-                    value = stringAsType((String) fields.get(1), 
-                            meta.getValueValidator(name.hasRemaining() 
-                                    ? comparator.cellFromByteBuffer(name)
-                                    : meta.comparator.rowMarker(Composites.EMPTY)));
+                    value = stringAsType((String) fields.get(1),
+                                         meta.getValueValidator(name.hasRemaining()
+                                                                ? comparator.cellFromByteBuffer(name)
+                                                                : meta.comparator.rowMarker(Composites.EMPTY)));
                 }
             }
         }
@@ -218,10 +219,10 @@
                 cfamily.addAtom(new RangeTombstone(start, end, col.timestamp, col.localExpirationTime));
                 continue;
             }
-            
+
             assert cfm.isCQL3Table() || col.getName().hasRemaining() : "Cell name should not be empty";
-            CellName cname = col.getName().hasRemaining() ? cfm.comparator.cellFromByteBuffer(col.getName()) 
-                    : cfm.comparator.rowMarker(Composites.EMPTY);
+            CellName cname = col.getName().hasRemaining() ? cfm.comparator.cellFromByteBuffer(col.getName())
+                                                          : cfm.comparator.rowMarker(Composites.EMPTY);
 
             if (col.isExpiring())
             {
@@ -303,133 +304,139 @@
         int importedKeys = 0;
         long start = System.nanoTime();
 
-        JsonParser parser = getParser(jsonFile);
-
-        Object[] data = parser.readValueAs(new TypeReference<Object[]>(){});
+        Object[] data;
+        try (JsonParser parser = getParser(jsonFile))
+        {
+            data = parser.readValueAs(new TypeReference<Object[]>(){});
+        }
 
         keyCountToImport = (keyCountToImport == null) ? data.length : keyCountToImport;
-        SSTableWriter writer = new SSTableWriter(ssTablePath, keyCountToImport, ActiveRepairService.UNREPAIRED_SSTABLE);
 
-        System.out.printf("Importing %s keys...%n", keyCountToImport);
-
-        // sort by dk representation, but hold onto the hex version
-        SortedMap<DecoratedKey,Map<?, ?>> decoratedKeys = new TreeMap<DecoratedKey,Map<?, ?>>();
-
-        for (Object row : data)
+        try (SSTableWriter writer = SSTableWriter.create(Descriptor.fromFilename(ssTablePath), keyCountToImport, ActiveRepairService.UNREPAIRED_SSTABLE, 0))
         {
-            Map<?,?> rowAsMap = (Map<?, ?>)row;
-            decoratedKeys.put(partitioner.decorateKey(getKeyValidator(columnFamily).fromString((String) rowAsMap.get("key"))), rowAsMap);
-        }
+            System.out.printf("Importing %s keys...%n", keyCountToImport);
 
-        for (Map.Entry<DecoratedKey, Map<?, ?>> row : decoratedKeys.entrySet())
-        {
-            if (row.getValue().containsKey("metadata"))
+            // sort by dk representation, but hold onto the hex version
+            SortedMap<DecoratedKey, Map<?, ?>> decoratedKeys = new TreeMap<DecoratedKey, Map<?, ?>>();
+
+            for (Object row : data)
             {
-                parseMeta((Map<?, ?>) row.getValue().get("metadata"), columnFamily, null);
+                Map<?, ?> rowAsMap = (Map<?, ?>) row;
+                decoratedKeys.put(partitioner.decorateKey(getKeyValidator(columnFamily).fromString((String) rowAsMap.get("key"))), rowAsMap);
             }
 
-            Object columns = row.getValue().get("cells");
-            addColumnsToCF((List<?>) columns, columnFamily);
-
-
-            writer.append(row.getKey(), columnFamily);
-            columnFamily.clear();
-
-            importedKeys++;
-
-            long current = System.nanoTime();
-
-            if (TimeUnit.NANOSECONDS.toSeconds(current - start) >= 5) // 5 secs.
+            for (Map.Entry<DecoratedKey, Map<?, ?>> row : decoratedKeys.entrySet())
             {
-                System.out.printf("Currently imported %d keys.%n", importedKeys);
-                start = current;
+                if (row.getValue().containsKey("metadata"))
+                {
+                    parseMeta((Map<?, ?>) row.getValue().get("metadata"), columnFamily, null);
+                }
+
+                Object columns = row.getValue().get("cells");
+                addColumnsToCF((List<?>) columns, columnFamily);
+
+
+                writer.append(row.getKey(), columnFamily);
+                columnFamily.clear();
+
+                importedKeys++;
+
+                long current = System.nanoTime();
+
+                if (TimeUnit.NANOSECONDS.toSeconds(current - start) >= 5) // 5 secs.
+                {
+                    System.out.printf("Currently imported %d keys.%n", importedKeys);
+                    start = current;
+                }
+
+                if (keyCountToImport == importedKeys)
+                    break;
             }
 
-            if (keyCountToImport == importedKeys)
-                break;
+            writer.finish(true);
         }
 
-        writer.closeAndOpenReader();
-
         return importedKeys;
     }
 
     private int importSorted(String jsonFile, ColumnFamily columnFamily, String ssTablePath,
-            IPartitioner partitioner) throws IOException
+                             IPartitioner partitioner) throws IOException
     {
         int importedKeys = 0; // already imported keys count
         long start = System.nanoTime();
 
-        JsonParser parser = getParser(jsonFile);
-
-        if (keyCountToImport == null)
+        try (JsonParser parser = getParser(jsonFile))
         {
-            keyCountToImport = 0;
-            System.out.println("Counting keys to import, please wait... (NOTE: to skip this use -n <num_keys>)");
+
+            if (keyCountToImport == null)
+            {
+                keyCountToImport = 0;
+                System.out.println("Counting keys to import, please wait... (NOTE: to skip this use -n <num_keys>)");
+
+                parser.nextToken(); // START_ARRAY
+                while (parser.nextToken() != null)
+                {
+                    parser.skipChildren();
+                    if (parser.getCurrentToken() == JsonToken.END_ARRAY)
+                        break;
+
+                    keyCountToImport++;
+                }
+            }
+            System.out.printf("Importing %s keys...%n", keyCountToImport);
+        }
+
+        try (JsonParser parser = getParser(jsonFile); // renewing parser
+             SSTableWriter writer = SSTableWriter.create(Descriptor.fromFilename(ssTablePath), keyCountToImport, ActiveRepairService.UNREPAIRED_SSTABLE);)
+        {
+            int lineNumber = 1;
+            DecoratedKey prevStoredKey = null;
 
             parser.nextToken(); // START_ARRAY
             while (parser.nextToken() != null)
             {
-                parser.skipChildren();
-                if (parser.getCurrentToken() == JsonToken.END_ARRAY)
+                String key = parser.getCurrentName();
+                Map<?, ?> row = parser.readValueAs(new TypeReference<Map<?, ?>>(){});
+                DecoratedKey currentKey = partitioner.decorateKey(getKeyValidator(columnFamily).fromString((String) row.get("key")));
+
+                if (row.containsKey("metadata"))
+                    parseMeta((Map<?, ?>) row.get("metadata"), columnFamily, null);
+
+                addColumnsToCF((List<?>) row.get("cells"), columnFamily);
+
+                if (prevStoredKey != null && prevStoredKey.compareTo(currentKey) != -1)
+                {
+                    System.err
+                    .printf("Line %d: Key %s is greater than previous, collection is not sorted properly. Aborting import. You might need to delete SSTables manually.%n",
+                            lineNumber, key);
+                    return -1;
+                }
+
+                // saving decorated key
+                writer.append(currentKey, columnFamily);
+                columnFamily.clear();
+
+                prevStoredKey = currentKey;
+                importedKeys++;
+                lineNumber++;
+
+                long current = System.nanoTime();
+
+                if (TimeUnit.NANOSECONDS.toSeconds(current - start) >= 5) // 5 secs.
+                {
+                    System.out.printf("Currently imported %d keys.%n", importedKeys);
+                    start = current;
+                }
+
+                if (keyCountToImport == importedKeys)
                     break;
 
-                keyCountToImport++;
             }
+
+            writer.finish(true);
+
+            return importedKeys;
         }
-
-        System.out.printf("Importing %s keys...%n", keyCountToImport);
-
-        parser = getParser(jsonFile); // renewing parser
-        SSTableWriter writer = new SSTableWriter(ssTablePath, keyCountToImport, ActiveRepairService.UNREPAIRED_SSTABLE);
-
-        int lineNumber = 1;
-        DecoratedKey prevStoredKey = null;
-
-        parser.nextToken(); // START_ARRAY
-        while (parser.nextToken() != null)
-        {
-            String key = parser.getCurrentName();
-            Map<?, ?> row = parser.readValueAs(new TypeReference<Map<?, ?>>(){});
-            DecoratedKey currentKey = partitioner.decorateKey(getKeyValidator(columnFamily).fromString((String) row.get("key")));
-
-            if (row.containsKey("metadata"))
-                parseMeta((Map<?, ?>) row.get("metadata"), columnFamily, null);
-
-            addColumnsToCF((List<?>) row.get("cells"), columnFamily);
-
-            if (prevStoredKey != null && prevStoredKey.compareTo(currentKey) != -1)
-            {
-                System.err
-                        .printf("Line %d: Key %s is greater than previous, collection is not sorted properly. Aborting import. You might need to delete SSTables manually.%n",
-                                lineNumber, key);
-                return -1;
-            }
-
-            // saving decorated key
-            writer.append(currentKey, columnFamily);
-            columnFamily.clear();
-
-            prevStoredKey = currentKey;
-            importedKeys++;
-            lineNumber++;
-
-            long current = System.nanoTime();
-
-            if (TimeUnit.NANOSECONDS.toSeconds(current - start) >= 5) // 5 secs.
-            {
-                System.out.printf("Currently imported %d keys.%n", importedKeys);
-                start = current;
-            }
-
-            if (keyCountToImport == importedKeys)
-                break;
-
-        }
-
-        writer.closeAndOpenReader();
-
-        return importedKeys;
     }
 
     /**
@@ -463,12 +470,15 @@
      * using an optional command line argument, or supplied on standard in.
      *
      * @param args command line arguments
-     * @throws IOException on failure to open/read/write files or output streams
      * @throws ParseException on failure to parse JSON input
      * @throws ConfigurationException on configuration error.
      */
     public static void main(String[] args) throws ParseException, ConfigurationException
     {
+        System.err.println("WARNING: please note that json2sstable is now deprecated and will be removed in Cassandra 3.0. "
+                         + "You should use CQLSSTableWriter if you want to write sstables directly. "
+                         + "Please see https://issues.apache.org/jira/browse/CASSANDRA-9618 for details.");
+
         CommandLineParser parser = new PosixParser();
 
         try
@@ -506,7 +516,9 @@
             isSorted = true;
         }
 
-        DatabaseDescriptor.loadSchemas(false);
+        Util.initDatabaseDescriptor();
+
+        Schema.instance.loadFromDisk(false);
         if (Schema.instance.getNonSystemKeyspaces().size() < 1)
         {
             String msg = "no non-system keyspaces are defined";
@@ -516,7 +528,7 @@
 
         try
         {
-           new SSTableImport(keyCountToImport, isSorted).importJson(json, keyspace, cfamily, ssTable);
+            new SSTableImport(keyCountToImport, isSorted).importJson(json, keyspace, cfamily, ssTable);
         }
         catch (Exception e)
         {
@@ -532,7 +544,7 @@
     private static void printProgramUsage()
     {
         System.out.printf("Usage: %s -s -K <keyspace> -c <column_family> -n <num_keys> <json> <sstable>%n%n",
-                            SSTableImport.class.getName());
+                          SSTableImport.class.getName());
 
         System.out.println("Options:");
         for (Object o :  options.getOptions())
diff --git a/src/java/org/apache/cassandra/tools/SSTableLevelResetter.java b/src/java/org/apache/cassandra/tools/SSTableLevelResetter.java
index 2b273c5..9f0af05 100644
--- a/src/java/org/apache/cassandra/tools/SSTableLevelResetter.java
+++ b/src/java/org/apache/cassandra/tools/SSTableLevelResetter.java
@@ -17,12 +17,10 @@
  */
 package org.apache.cassandra.tools;
 
-import java.io.IOException;
 import java.io.PrintStream;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
@@ -40,13 +38,13 @@
     /**
      * @param args a list of sstables whose metadata we are changing
      */
-    public static void main(String[] args) throws IOException
+    public static void main(String[] args)
     {
         PrintStream out = System.out;
         if (args.length == 0)
         {
             out.println("This command should be run with Cassandra stopped!");
-            out.println("Usage: sstablelevelreset <keyspace> <columnfamily>");
+            out.println("Usage: sstablelevelreset <keyspace> <table>");
             System.exit(1);
         }
 
@@ -54,16 +52,18 @@
         {
             out.println("This command should be run with Cassandra stopped, otherwise you will get very strange behavior");
             out.println("Verify that Cassandra is not running and then execute the command like this:");
-            out.println("Usage: sstablelevelreset --really-reset <keyspace> <columnfamily>");
+            out.println("Usage: sstablelevelreset --really-reset <keyspace> <table>");
             System.exit(1);
         }
 
+        Util.initDatabaseDescriptor();
+
         // TODO several daemon threads will run from here.
         // So we have to explicitly call System.exit.
         try
         {
             // load keyspace descriptions.
-            DatabaseDescriptor.loadSchemas();
+            Schema.instance.loadFromDisk(false);
 
             String keyspaceName = args[1];
             String columnfamily = args[2];
@@ -98,7 +98,7 @@
 
             if (!foundSSTable)
             {
-                out.println("Found no sstables, did you give the correct keyspace/columnfamily?");
+                out.println("Found no sstables, did you give the correct keyspace/table?");
             }
         }
         catch (Throwable t)
diff --git a/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java b/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
index 8b33d50..8319014 100644
--- a/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
+++ b/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
@@ -43,6 +43,8 @@
             System.exit(1);
         }
 
+        Util.initDatabaseDescriptor();
+
         for (String fname : args)
         {
             if (new File(fname).exists())
@@ -68,8 +70,9 @@
                     out.printf("Estimated droppable tombstones: %s%n", stats.getEstimatedDroppableTombstoneRatio((int) (System.currentTimeMillis() / 1000)));
                     out.printf("SSTable Level: %d%n", stats.sstableLevel);
                     out.printf("Repaired at: %d%n", stats.repairedAt);
-                    out.println(stats.replayPosition);
-                    out.println("Estimated tombstone drop times:%n");
+                    out.printf("Minimum replay position: %s\n", stats.commitLogLowerBound);
+                    out.printf("Maximum replay position: %s\n", stats.commitLogUpperBound);
+                    out.println("Estimated tombstone drop times:");
                     for (Map.Entry<Double, Long> entry : stats.estimatedTombstoneDropTime.getAsMap().entrySet())
                     {
                         out.printf("%-10s:%10s%n",entry.getKey().intValue(), entry.getValue());
diff --git a/src/java/org/apache/cassandra/tools/SSTableOfflineRelevel.java b/src/java/org/apache/cassandra/tools/SSTableOfflineRelevel.java
index 81fb2bd..71e4cfc 100644
--- a/src/java/org/apache/cassandra/tools/SSTableOfflineRelevel.java
+++ b/src/java/org/apache/cassandra/tools/SSTableOfflineRelevel.java
@@ -39,7 +39,7 @@
 import org.apache.cassandra.db.compaction.LeveledManifest;
 import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.utils.Pair;
 
 /**
@@ -83,10 +83,13 @@
             out.println("Usage: sstableofflinerelevel [--dry-run] <keyspace> <columnfamily>");
             System.exit(1);
         }
+
+        Util.initDatabaseDescriptor();
+
         boolean dryRun = args[0].equals("--dry-run");
         String keyspace = args[args.length - 2];
         String columnfamily = args[args.length - 1];
-        DatabaseDescriptor.loadSchemas(false);
+        Schema.instance.loadFromDisk(false);
 
         if (Schema.instance.getCFMetaData(keyspace, columnfamily) == null)
             throw new IllegalArgumentException(String.format("Unknown keyspace/columnFamily %s.%s",
diff --git a/src/java/org/apache/cassandra/tools/SSTableRepairedAtSetter.java b/src/java/org/apache/cassandra/tools/SSTableRepairedAtSetter.java
index de1d91d..ff362cc 100644
--- a/src/java/org/apache/cassandra/tools/SSTableRepairedAtSetter.java
+++ b/src/java/org/apache/cassandra/tools/SSTableRepairedAtSetter.java
@@ -36,8 +36,9 @@
  *
  * If you know you ran repair 2 weeks ago, you can do something like
  *
+ * {@code
  * sstablerepairset --is-repaired -f <(find /var/lib/cassandra/data/.../ -iname "*Data.db*" -mtime +14)
- *
+ * }
  */
 public class SSTableRepairedAtSetter
 {
@@ -62,6 +63,8 @@
             System.exit(1);
         }
 
+        Util.initDatabaseDescriptor();
+
         boolean setIsRepaired = args[1].equals("--is-repaired");
 
         List<String> fileNames;
@@ -77,7 +80,7 @@
         for (String fname: fileNames)
         {
             Descriptor descriptor = Descriptor.fromFilename(fname);
-            if (descriptor.version.hasRepairedAt)
+            if (descriptor.version.hasRepairedAt())
             {
                 if (setIsRepaired)
                 {
diff --git a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
index fdf6c8d..a486a13 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
@@ -28,17 +28,13 @@
 import com.google.common.collect.Lists;
 import org.apache.commons.cli.*;
 
-import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.compaction.AbstractCompactionStrategy;
-import org.apache.cassandra.db.compaction.CompactionManager;
-import org.apache.cassandra.db.compaction.LeveledCompactionStrategy;
-import org.apache.cassandra.db.compaction.LeveledManifest;
-import org.apache.cassandra.db.compaction.Scrubber;
-import org.apache.cassandra.db.compaction.WrappingCompactionStrategy;
+import org.apache.cassandra.db.compaction.*;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.*;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 import org.apache.cassandra.utils.OutputHandler;
@@ -58,25 +54,40 @@
     public static void main(String args[])
     {
         Options options = Options.parseArgs(args);
+        Util.initDatabaseDescriptor();
+
         try
         {
             // load keyspace descriptions.
-            DatabaseDescriptor.loadSchemas(false);
+            Schema.instance.loadFromDisk(false);
 
-            if (Schema.instance.getCFMetaData(options.keyspaceName, options.cfName) == null)
-                throw new IllegalArgumentException(String.format("Unknown keyspace/columnFamily %s.%s",
-                                                                 options.keyspaceName,
-                                                                 options.cfName));
+            if (Schema.instance.getKSMetaData(options.keyspaceName) == null)
+                throw new IllegalArgumentException(String.format("Unknown keyspace %s", options.keyspaceName));
 
             // Do not load sstables since they might be broken
             Keyspace keyspace = Keyspace.openWithoutSSTables(options.keyspaceName);
-            ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(options.cfName);
+
+            ColumnFamilyStore cfs = null;
+            for (ColumnFamilyStore c : keyspace.getValidColumnFamilies(true, false, options.cfName))
+            {
+                if (c.name.equals(options.cfName))
+                {
+                    cfs = c;
+                    break;
+                }
+            }
+
+            if (cfs == null)
+                throw new IllegalArgumentException(String.format("Unknown table %s.%s",
+                                                                  options.keyspaceName,
+                                                                  options.cfName));
+
             String snapshotName = "pre-scrub-" + System.currentTimeMillis();
 
             OutputHandler handler = new OutputHandler.SystemOutput(options.verbose, options.debug);
             Directories.SSTableLister lister = cfs.directories.sstableLister().skipTemporary(true);
 
-            List<SSTableReader> sstables = new ArrayList<SSTableReader>();
+            List<SSTableReader> sstables = new ArrayList<>();
 
             // Scrub sstables
             for (Map.Entry<Descriptor, Set<Component>> entry : lister.list().entrySet())
@@ -87,7 +98,7 @@
 
                 try
                 {
-                    SSTableReader sstable = SSTableReader.openNoValidation(entry.getKey(), components, cfs.metadata);
+                    SSTableReader sstable = SSTableReader.openNoValidation(entry.getKey(), components, cfs);
                     sstables.add(sstable);
 
                     File snapshotDirectory = Directories.getSnapshotDirectory(sstable.descriptor, snapshotName);
@@ -108,30 +119,27 @@
             {
                 for (SSTableReader sstable : sstables)
                 {
-                    try
+                    try (LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.SCRUB, sstable))
                     {
-                        Scrubber scrubber = new Scrubber(cfs, sstable, options.skipCorrupted, handler, true, !options.noValidate);
-                        try
+                        txn.obsoleteOriginals(); // make sure originals are deleted and avoid NPE if index is missing, CASSANDRA-9591
+                        try (Scrubber scrubber = new Scrubber(cfs, txn, options.skipCorrupted, handler, !options.noValidate))
                         {
                             scrubber.scrub();
                         }
-                        finally
+                        catch (Throwable t)
                         {
-                            scrubber.close();
+                            if (!cfs.rebuildOnFailedScrub(t))
+                            {
+                                System.out.println(t.getMessage());
+                                throw t;
+                            }
                         }
-
-                        // Remove the sstable (it's been copied by scrub and snapshotted)
-                        sstable.markObsolete(null);
                     }
                     catch (Exception e)
                     {
                         System.err.println(String.format("Error scrubbing %s: %s", sstable, e.getMessage()));
                         e.printStackTrace(System.err);
                     }
-                    finally
-                    {
-                        sstable.selfRef().release();
-                    }
                 }
             }
 
@@ -154,7 +162,7 @@
     {
         WrappingCompactionStrategy wrappingStrategy = (WrappingCompactionStrategy)strategy;
         int maxSizeInMB = (int)((cfs.getCompactionStrategy().getMaxSSTableBytes()) / (1024L * 1024L));
-        if (wrappingStrategy.getWrappedStrategies().size() == 2 && wrappingStrategy.getWrappedStrategies().iterator().next() instanceof LeveledCompactionStrategy)
+        if (wrappingStrategy.getWrappedStrategies().size() == 2 && wrappingStrategy.getWrappedStrategies().get(0) instanceof LeveledCompactionStrategy)
         {
             System.out.println("Checking leveled manifest");
             Predicate<SSTableReader> repairedPredicate = new Predicate<SSTableReader>()
@@ -266,7 +274,7 @@
             String usage = String.format("%s [options] <keyspace> <column_family>", TOOL_NAME);
             StringBuilder header = new StringBuilder();
             header.append("--\n");
-            header.append("Scrub the sstable for the provided column family." );
+            header.append("Scrub the sstable for the provided table." );
             header.append("\n--\n");
             header.append("Options are:");
             new HelpFormatter().printHelp(usage, header.toString(), options, "");
diff --git a/src/java/org/apache/cassandra/tools/StandaloneSplitter.java b/src/java/org/apache/cassandra/tools/StandaloneSplitter.java
index 9c3c58b..95fed3c 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneSplitter.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneSplitter.java
@@ -19,18 +19,20 @@
 package org.apache.cassandra.tools;
 
 import java.io.File;
-import java.io.IOException;
 import java.util.*;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.commons.cli.*;
 
-import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.compaction.SSTableSplitter;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.io.sstable.*;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 import org.apache.cassandra.utils.Pair;
@@ -47,13 +49,15 @@
     private static final String NO_SNAPSHOT_OPTION = "no-snapshot";
     private static final String SIZE_OPTION = "size";
 
-    public static void main(String args[]) throws IOException
+    public static void main(String args[])
     {
         Options options = Options.parseArgs(args);
+        Util.initDatabaseDescriptor();
+
         try
         {
             // load keyspace descriptions.
-            DatabaseDescriptor.loadSchemas(false);
+            Schema.instance.loadFromDisk(false);
 
             String ksName = null;
             String cfName = null;
@@ -81,7 +85,7 @@
                 if (cfName == null)
                     cfName = desc.cfname;
                 else if (!cfName.equals(desc.cfname))
-                    throw new IllegalArgumentException("All sstables must be part of the same column family");
+                    throw new IllegalArgumentException("All sstables must be part of the same table");
 
                 Set<Component> components = new HashSet<Component>(Arrays.asList(new Component[]{
                     Component.DATA,
@@ -111,12 +115,12 @@
             ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
             String snapshotName = "pre-split-" + System.currentTimeMillis();
 
-            List<SSTableReader> sstables = new ArrayList<SSTableReader>();
+            List<SSTableReader> sstables = new ArrayList<>();
             for (Map.Entry<Descriptor, Set<Component>> fn : parsedFilenames.entrySet())
             {
                 try
                 {
-                    SSTableReader sstable = SSTableReader.openNoValidation(fn.getKey(), fn.getValue(), cfs.metadata);
+                    SSTableReader sstable = SSTableReader.openNoValidation(fn.getKey(), fn.getValue(), cfs);
                     if (!isSSTableLargerEnough(sstable, options.sizeInMB)) {
                         System.out.println(String.format("Skipping %s: it's size (%.3f MB) is less than the split size (%d MB)",
                                 sstable.getFilename(), ((sstable.onDiskLength() * 1.0d) / 1024L) / 1024L, options.sizeInMB));
@@ -145,12 +149,11 @@
             if (options.snapshot)
                 System.out.println(String.format("Pre-split sstables snapshotted into snapshot %s", snapshotName));
 
-            cfs.getDataTracker().markCompacting(sstables, false, true);
             for (SSTableReader sstable : sstables)
             {
-                try
+                try (LifecycleTransaction transaction = LifecycleTransaction.offline(OperationType.UNKNOWN, sstable))
                 {
-                    new SSTableSplitter(cfs, sstable, options.sizeInMB).split();
+                    new SSTableSplitter(cfs, transaction, options.sizeInMB).split();
 
                     // Remove the sstable (it's been copied by split and snapshotted)
                     sstable.markObsolete(null);
@@ -223,7 +226,7 @@
                 opts.sizeInMB = DEFAULT_SSTABLE_SIZE;
 
                 if (cmd.hasOption(SIZE_OPTION))
-                    opts.sizeInMB = Integer.valueOf(cmd.getOptionValue(SIZE_OPTION));
+                    opts.sizeInMB = Integer.parseInt(cmd.getOptionValue(SIZE_OPTION));
 
                 return opts;
             }
diff --git a/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java b/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java
index abe5521..23342bc 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.tools;
 
-import java.io.IOException;
 import java.util.*;
 import java.util.concurrent.TimeUnit;
 
@@ -29,8 +28,11 @@
 import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.db.compaction.Upgrader;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.io.sstable.*;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 import org.apache.cassandra.utils.OutputHandler;
 
@@ -43,16 +45,18 @@
     private static final String HELP_OPTION  = "help";
     private static final String KEEP_SOURCE = "keep-source";
 
-    public static void main(String args[]) throws IOException
+    public static void main(String args[])
     {
         Options options = Options.parseArgs(args);
+        Util.initDatabaseDescriptor();
+
         try
         {
             // load keyspace descriptions.
-            DatabaseDescriptor.loadSchemas(false);
+            Schema.instance.loadFromDisk(false);
 
             if (Schema.instance.getCFMetaData(options.keyspace, options.cf) == null)
-                throw new IllegalArgumentException(String.format("Unknown keyspace/columnFamily %s.%s",
+                throw new IllegalArgumentException(String.format("Unknown keyspace/table %s.%s",
                                                                  options.keyspace,
                                                                  options.cf));
 
@@ -66,7 +70,7 @@
             else
                 lister.includeBackups(false);
 
-            Collection<SSTableReader> readers = new ArrayList<SSTableReader>();
+            Collection<SSTableReader> readers = new ArrayList<>();
 
             // Upgrade sstables
             for (Map.Entry<Descriptor, Set<Component>> entry : lister.list().entrySet())
@@ -77,9 +81,12 @@
 
                 try
                 {
-                    SSTableReader sstable = SSTableReader.openNoValidation(entry.getKey(), components, cfs.metadata);
-                    if (sstable.descriptor.version.equals(Descriptor.Version.CURRENT))
+                    SSTableReader sstable = SSTableReader.openNoValidation(entry.getKey(), components, cfs);
+                    if (sstable.descriptor.version.equals(DatabaseDescriptor.getSSTableFormat().info.getLatestVersion()))
+                    {
+                        sstable.selfRef().release();
                         continue;
+                    }
                     readers.add(sstable);
                 }
                 catch (Exception e)
@@ -88,8 +95,6 @@
                     System.err.println(String.format("Error Loading %s: %s", entry.getKey(), e.getMessage()));
                     if (options.debug)
                         e.printStackTrace(System.err);
-
-                    continue;
                 }
             }
 
@@ -98,9 +103,9 @@
 
             for (SSTableReader sstable : readers)
             {
-                try
+                try (LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.UPGRADE_SSTABLES, sstable))
                 {
-                    Upgrader upgrader = new Upgrader(cfs, sstable, handler);
+                    Upgrader upgrader = new Upgrader(cfs, txn, handler);
                     upgrader.upgrade();
 
                     if (!options.keepSource)
diff --git a/src/java/org/apache/cassandra/tools/StandaloneVerifier.java b/src/java/org/apache/cassandra/tools/StandaloneVerifier.java
new file mode 100644
index 0000000..fb7f218
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/StandaloneVerifier.java
@@ -0,0 +1,220 @@
+/**
+ * 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.tools;
+
+import com.google.common.base.Predicate;
+import com.google.common.base.Predicates;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Directories;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.compaction.*;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.commons.cli.*;
+
+import java.io.File;
+import java.util.*;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.cassandra.tools.BulkLoader.CmdLineOptions;
+
+public class StandaloneVerifier
+{
+    private static final String TOOL_NAME = "sstableverify";
+    private static final String VERBOSE_OPTION  = "verbose";
+    private static final String EXTENDED_OPTION = "extended";
+    private static final String DEBUG_OPTION  = "debug";
+    private static final String HELP_OPTION  = "help";
+
+    public static void main(String args[])
+    {
+        Options options = Options.parseArgs(args);
+        Util.initDatabaseDescriptor();
+
+        try
+        {
+            // load keyspace descriptions.
+            Schema.instance.loadFromDisk(false);
+
+            boolean hasFailed = false;
+
+            if (Schema.instance.getCFMetaData(options.keyspaceName, options.cfName) == null)
+                throw new IllegalArgumentException(String.format("Unknown keyspace/table %s.%s",
+                                                                 options.keyspaceName,
+                                                                 options.cfName));
+
+            // Do not load sstables since they might be broken
+            Keyspace keyspace = Keyspace.openWithoutSSTables(options.keyspaceName);
+            ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(options.cfName);
+
+            OutputHandler handler = new OutputHandler.SystemOutput(options.verbose, options.debug);
+            Directories.SSTableLister lister = cfs.directories.sstableLister().skipTemporary(true);
+
+            boolean extended = options.extended;
+
+            List<SSTableReader> sstables = new ArrayList<>();
+
+            // Verify sstables
+            for (Map.Entry<Descriptor, Set<Component>> entry : lister.list().entrySet())
+            {
+                Set<Component> components = entry.getValue();
+                if (!components.contains(Component.DATA) || !components.contains(Component.PRIMARY_INDEX))
+                    continue;
+
+                try
+                {
+                    SSTableReader sstable = SSTableReader.openNoValidation(entry.getKey(), components, cfs);
+                    sstables.add(sstable);
+                }
+                catch (Exception e)
+                {
+                    JVMStabilityInspector.inspectThrowable(e);
+                    System.err.println(String.format("Error Loading %s: %s", entry.getKey(), e.getMessage()));
+                    if (options.debug)
+                        e.printStackTrace(System.err);
+                }
+            }
+
+            for (SSTableReader sstable : sstables)
+            {
+                try
+                {
+
+                    try (Verifier verifier = new Verifier(cfs, sstable, handler, true))
+                    {
+                        verifier.verify(extended);
+                    }
+                    catch (CorruptSSTableException cs)
+                    {
+                        System.err.println(String.format("Error verifying %s: %s", sstable, cs.getMessage()));
+                        hasFailed = true;
+                    }
+                }
+                catch (Exception e)
+                {
+                    System.err.println(String.format("Error verifying %s: %s", sstable, e.getMessage()));
+                    e.printStackTrace(System.err);
+                }
+            }
+
+            CompactionManager.instance.finishCompactionsAndShutdown(5, TimeUnit.MINUTES);
+
+            System.exit( hasFailed ? 1 : 0 ); // We need that to stop non daemonized threads
+        }
+        catch (Exception e)
+        {
+            System.err.println(e.getMessage());
+            if (options.debug)
+                e.printStackTrace(System.err);
+            System.exit(1);
+        }
+    }
+
+    private static class Options
+    {
+        public final String keyspaceName;
+        public final String cfName;
+
+        public boolean debug;
+        public boolean verbose;
+        public boolean extended;
+
+        private Options(String keyspaceName, String cfName)
+        {
+            this.keyspaceName = keyspaceName;
+            this.cfName = cfName;
+        }
+
+        public static Options parseArgs(String cmdArgs[])
+        {
+            CommandLineParser parser = new GnuParser();
+            CmdLineOptions options = getCmdLineOptions();
+            try
+            {
+                CommandLine cmd = parser.parse(options, cmdArgs, false);
+
+                if (cmd.hasOption(HELP_OPTION))
+                {
+                    printUsage(options);
+                    System.exit(0);
+                }
+
+                String[] args = cmd.getArgs();
+                if (args.length != 2)
+                {
+                    String msg = args.length < 2 ? "Missing arguments" : "Too many arguments";
+                    System.err.println(msg);
+                    printUsage(options);
+                    System.exit(1);
+                }
+
+                String keyspaceName = args[0];
+                String cfName = args[1];
+
+                Options opts = new Options(keyspaceName, cfName);
+
+                opts.debug = cmd.hasOption(DEBUG_OPTION);
+                opts.verbose = cmd.hasOption(VERBOSE_OPTION);
+                opts.extended = cmd.hasOption(EXTENDED_OPTION);
+
+                return opts;
+            }
+            catch (ParseException e)
+            {
+                errorMsg(e.getMessage(), options);
+                return null;
+            }
+        }
+
+        private static void errorMsg(String msg, CmdLineOptions options)
+        {
+            System.err.println(msg);
+            printUsage(options);
+            System.exit(1);
+        }
+
+        private static CmdLineOptions getCmdLineOptions()
+        {
+            CmdLineOptions options = new CmdLineOptions();
+            options.addOption(null, DEBUG_OPTION,          "display stack traces");
+            options.addOption("e",  EXTENDED_OPTION,       "extended verification");
+            options.addOption("v",  VERBOSE_OPTION,        "verbose output");
+            options.addOption("h",  HELP_OPTION,           "display this help message");
+            return options;
+        }
+
+        public static void printUsage(CmdLineOptions options)
+        {
+            String usage = String.format("%s [options] <keyspace> <column_family>", TOOL_NAME);
+            StringBuilder header = new StringBuilder();
+            header.append("--\n");
+            header.append("Verify the sstable for the provided table." );
+            header.append("\n--\n");
+            header.append("Options are:");
+            new HelpFormatter().printHelp(usage, header.toString(), options, "");
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/tools/Util.java b/src/java/org/apache/cassandra/tools/Util.java
new file mode 100644
index 0000000..6e23361
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/Util.java
@@ -0,0 +1,58 @@
+/*
+ * 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.tools;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.exceptions.ConfigurationException;
+
+public final class Util
+{
+    private Util()
+    {
+    }
+
+    /**
+     * This is used by standalone tools to force static initialization of DatabaseDescriptor, and fail if configuration
+     * is bad.
+     */
+    public static void initDatabaseDescriptor()
+    {
+        try
+        {
+            DatabaseDescriptor.forceStaticInitialization();
+        }
+        catch (ExceptionInInitializerError e)
+        {
+            Throwable cause = e.getCause();
+            boolean logStackTrace = !(cause instanceof ConfigurationException) || ((ConfigurationException) cause).logStackTrace;
+            System.out.println("Exception (" + cause.getClass().getName() + ") encountered during startup: " + cause.getMessage());
+
+            if (logStackTrace)
+            {
+                cause.printStackTrace();
+                System.exit(3);
+            }
+            else
+            {
+                System.err.println(cause.getMessage());
+                System.exit(3);
+            }
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/Assassinate.java b/src/java/org/apache/cassandra/tools/nodetool/Assassinate.java
new file mode 100644
index 0000000..56fec44
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/Assassinate.java
@@ -0,0 +1,47 @@
+/*
+ * 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.tools.nodetool;
+
+import static org.apache.commons.lang3.StringUtils.EMPTY;
+import io.airlift.command.Arguments;
+import io.airlift.command.Command;
+
+import java.net.UnknownHostException;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "assassinate", description = "Forcefully remove a dead node without re-replicating any data.  Use as a last resort if you cannot removenode")
+public class Assassinate extends NodeToolCmd
+{
+    @Arguments(title = "ip address", usage = "<ip_address>", description = "IP address of the endpoint to assassinate", required = true)
+    private String endpoint = EMPTY;
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        try
+        {
+            probe.assassinateEndpoint(endpoint);
+        }
+        catch (UnknownHostException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/BootstrapResume.java b/src/java/org/apache/cassandra/tools/nodetool/BootstrapResume.java
new file mode 100644
index 0000000..bb47e10
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/BootstrapResume.java
@@ -0,0 +1,43 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Command;
+
+import java.io.IOError;
+import java.io.IOException;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "resume", description = "Resume bootstrap streaming")
+public class BootstrapResume extends NodeToolCmd
+{
+    @Override
+    protected void execute(NodeProbe probe)
+    {
+        try
+        {
+            probe.resumeBootstrap(System.out);
+        }
+        catch (IOException e)
+        {
+            throw new IOError(e);
+        }
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/CfHistograms.java b/src/java/org/apache/cassandra/tools/nodetool/CfHistograms.java
new file mode 100644
index 0000000..69d3b4a
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/CfHistograms.java
@@ -0,0 +1,29 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Command;
+
+/**
+ * @deprecated use TableHistograms
+ */
+@Command(name = "cfhistograms", hidden = true, description = "Print statistic histograms for a given column family")
+@Deprecated
+public class CfHistograms extends TableHistograms
+{
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/CfStats.java b/src/java/org/apache/cassandra/tools/nodetool/CfStats.java
new file mode 100644
index 0000000..15c72ba
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/CfStats.java
@@ -0,0 +1,29 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Command;
+
+/**
+ * @deprecated use TableStats
+ */
+@Command(name = "cfstats", hidden = true, description = "Print statistics on tables")
+@Deprecated
+public class CfStats extends TableStats
+{
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java b/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java
new file mode 100644
index 0000000..6c6676d
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java
@@ -0,0 +1,62 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Arguments;
+import io.airlift.command.Command;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import io.airlift.command.Option;
+import org.apache.cassandra.db.SystemKeyspace;
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "cleanup", description = "Triggers the immediate cleanup of keys no longer belonging to a node. By default, clean all keyspaces")
+public class Cleanup extends NodeToolCmd
+{
+    @Arguments(usage = "[<keyspace> <tables>...]", description = "The keyspace followed by one or many tables")
+    private List<String> args = new ArrayList<>();
+
+    @Option(title = "jobs",
+            name = {"-j", "--jobs"},
+            description = "Number of sstables to cleanup simultanously, set to 0 to use all available compaction threads")
+    private int jobs = 2;
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        List<String> keyspaces = parseOptionalKeyspace(args, probe);
+        String[] cfnames = parseOptionalColumnFamilies(args);
+
+        for (String keyspace : keyspaces)
+        {
+            if (SystemKeyspace.NAME.equals(keyspace))
+                continue;
+
+            try
+            {
+                probe.forceKeyspaceCleanup(System.out, jobs, keyspace, cfnames);
+            } catch (Exception e)
+            {
+                throw new RuntimeException("Error occurred during cleanup", e);
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/ClearSnapshot.java b/src/java/org/apache/cassandra/tools/nodetool/ClearSnapshot.java
new file mode 100644
index 0000000..7167bd9
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/ClearSnapshot.java
@@ -0,0 +1,68 @@
+/*
+ * 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.tools.nodetool;
+
+import static com.google.common.collect.Iterables.toArray;
+import static org.apache.commons.lang3.StringUtils.EMPTY;
+import static org.apache.commons.lang3.StringUtils.join;
+import io.airlift.command.Arguments;
+import io.airlift.command.Command;
+import io.airlift.command.Option;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "clearsnapshot", description = "Remove the snapshot with the given name from the given keyspaces. If no snapshotName is specified we will remove all snapshots")
+public class ClearSnapshot extends NodeToolCmd
+{
+    @Arguments(usage = "[<keyspaces>...] ", description = "Remove snapshots from the given keyspaces")
+    private List<String> keyspaces = new ArrayList<>();
+
+    @Option(title = "snapshot_name", name = "-t", description = "Remove the snapshot with a given name")
+    private String snapshotName = EMPTY;
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        StringBuilder sb = new StringBuilder();
+
+        sb.append("Requested clearing snapshot(s) for ");
+
+        if (keyspaces.isEmpty())
+            sb.append("[all keyspaces]");
+        else
+            sb.append("[").append(join(keyspaces, ", ")).append("]");
+
+        if (!snapshotName.isEmpty())
+            sb.append(" with snapshot name [").append(snapshotName).append("]");
+
+        System.out.println(sb.toString());
+
+        try
+        {
+            probe.clearSnapshot(snapshotName, toArray(keyspaces, String.class));
+        } catch (IOException e)
+        {
+            throw new RuntimeException("Error during clearing snapshots", e);
+        }
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/Compact.java b/src/java/org/apache/cassandra/tools/nodetool/Compact.java
new file mode 100644
index 0000000..4d04ae7
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/Compact.java
@@ -0,0 +1,56 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Arguments;
+import io.airlift.command.Command;
+import io.airlift.command.Option;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "compact", description = "Force a (major) compaction on one or more tables")
+public class Compact extends NodeToolCmd
+{
+    @Arguments(usage = "[<keyspace> <tables>...]", description = "The keyspace followed by one or many tables")
+    private List<String> args = new ArrayList<>();
+
+    @Option(title = "split_output", name = {"-s", "--split-output"}, description = "Use -s to not create a single big file")
+    private boolean splitOutput = false;
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        List<String> keyspaces = parseOptionalKeyspace(args, probe);
+        String[] cfnames = parseOptionalColumnFamilies(args);
+
+        for (String keyspace : keyspaces)
+        {
+            try
+            {
+                probe.forceKeyspaceCompaction(splitOutput, keyspace, cfnames);
+            } catch (Exception e)
+            {
+                throw new RuntimeException("Error occurred during compaction", e);
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/CompactionHistory.java b/src/java/org/apache/cassandra/tools/nodetool/CompactionHistory.java
new file mode 100644
index 0000000..cbb054a
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/CompactionHistory.java
@@ -0,0 +1,57 @@
+/*
+ * 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.tools.nodetool;
+
+import static com.google.common.collect.Iterables.toArray;
+import io.airlift.command.Command;
+
+import java.util.List;
+import java.util.Set;
+
+import javax.management.openmbean.TabularData;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "compactionhistory", description = "Print history of compaction")
+public class CompactionHistory extends NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        System.out.println("Compaction History: ");
+
+        TabularData tabularData = probe.getCompactionHistory();
+        if (tabularData.isEmpty())
+        {
+            System.out.printf("There is no compaction history");
+            return;
+        }
+
+        String format = "%-41s%-19s%-29s%-26s%-15s%-15s%s%n";
+        List<String> indexNames = tabularData.getTabularType().getIndexNames();
+        System.out.printf(format, toArray(indexNames, Object.class));
+
+        Set<?> values = tabularData.keySet();
+        for (Object eachValue : values)
+        {
+            List<?> value = (List<?>) eachValue;
+            System.out.printf(format, toArray(value, Object.class));
+        }
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/CompactionStats.java b/src/java/org/apache/cassandra/tools/nodetool/CompactionStats.java
new file mode 100644
index 0000000..e57d2ee
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/CompactionStats.java
@@ -0,0 +1,105 @@
+/*
+ * 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.tools.nodetool;
+
+import static java.lang.String.format;
+import io.airlift.command.Command;
+import io.airlift.command.Option;
+
+import java.text.DecimalFormat;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.cassandra.db.compaction.CompactionManagerMBean;
+import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "compactionstats", description = "Print statistics on compactions")
+public class CompactionStats extends NodeToolCmd
+{
+    @Option(title = "human_readable",
+            name = {"-H", "--human-readable"},
+            description = "Display bytes in human readable form, i.e. KB, MB, GB, TB")
+    private boolean humanReadable = false;
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        CompactionManagerMBean cm = probe.getCompactionManagerProxy();
+        System.out.println("pending tasks: " + probe.getCompactionMetric("PendingTasks"));
+        long remainingBytes = 0;
+        List<Map<String, String>> compactions = cm.getCompactions();
+        if (!compactions.isEmpty())
+        {
+            int compactionThroughput = probe.getCompactionThroughput();
+            List<String[]> lines = new ArrayList<>();
+            int[] columnSizes = new int[] { 0, 0, 0, 0, 0, 0, 0, 0 };
+
+            addLine(lines, columnSizes, "id", "compaction type", "keyspace", "table", "completed", "total", "unit", "progress");
+            for (Map<String, String> c : compactions)
+            {
+                long total = Long.parseLong(c.get("total"));
+                long completed = Long.parseLong(c.get("completed"));
+                String taskType = c.get("taskType");
+                String keyspace = c.get("keyspace");
+                String columnFamily = c.get("columnfamily");
+                String completedStr = humanReadable ? FileUtils.stringifyFileSize(completed) : Long.toString(completed);
+                String totalStr = humanReadable ? FileUtils.stringifyFileSize(total) : Long.toString(total);
+                String unit = c.get("unit");
+                String percentComplete = total == 0 ? "n/a" : new DecimalFormat("0.00").format((double) completed / total * 100) + "%";
+                String id = c.get("compactionId");
+                addLine(lines, columnSizes, id, taskType, keyspace, columnFamily, completedStr, totalStr, unit, percentComplete);
+                if (taskType.equals(OperationType.COMPACTION.toString()))
+                    remainingBytes += total - completed;
+            }
+
+            StringBuilder buffer = new StringBuilder();
+            for (int columnSize : columnSizes) {
+                buffer.append("%");
+                buffer.append(columnSize + 3);
+                buffer.append("s");
+            }
+            buffer.append("%n");
+            String format = buffer.toString();
+
+            for (String[] line : lines)
+            {
+                System.out.printf(format, line[0], line[1], line[2], line[3], line[4], line[5], line[6], line[7]);
+            }
+
+            String remainingTime = "n/a";
+            if (compactionThroughput != 0)
+            {
+                long remainingTimeInSecs = remainingBytes / (1024L * 1024L * compactionThroughput);
+                remainingTime = format("%dh%02dm%02ds", remainingTimeInSecs / 3600, (remainingTimeInSecs % 3600) / 60, (remainingTimeInSecs % 60));
+            }
+            System.out.printf("%25s%10s%n", "Active compaction remaining time : ", remainingTime);
+        }
+    }
+
+    private void addLine(List<String[]> lines, int[] columnSizes, String... columns) {
+        lines.add(columns);
+        for (int i = 0; i < columns.length; i++) {
+            columnSizes[i] = Math.max(columnSizes[i], columns[i].length());
+        }
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/Decommission.java b/src/java/org/apache/cassandra/tools/nodetool/Decommission.java
new file mode 100644
index 0000000..34890e0
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/Decommission.java
@@ -0,0 +1,42 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Command;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "decommission", description = "Decommission the *node I am connecting to*")
+public class Decommission extends NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        try
+        {
+            probe.decommission();
+        } catch (InterruptedException e)
+        {
+            throw new RuntimeException("Error decommissioning node", e);
+        } catch (UnsupportedOperationException e)
+        {
+            throw new IllegalStateException("Unsupported operation: " + e.getMessage(), e);
+        }
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/DescribeCluster.java b/src/java/org/apache/cassandra/tools/nodetool/DescribeCluster.java
new file mode 100644
index 0000000..81dee20
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/DescribeCluster.java
@@ -0,0 +1,49 @@
+/*
+ * 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.tools.nodetool;
+
+import static java.lang.String.format;
+import io.airlift.command.Command;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "describecluster", description = "Print the name, snitch, partitioner and schema version of a cluster")
+public class DescribeCluster extends NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        // display cluster name, snitch and partitioner
+        System.out.println("Cluster Information:");
+        System.out.println("\tName: " + probe.getClusterName());
+        System.out.println("\tSnitch: " + probe.getEndpointSnitchInfoProxy().getSnitchName());
+        System.out.println("\tPartitioner: " + probe.getPartitioner());
+
+        // display schema version for each node
+        System.out.println("\tSchema versions:");
+        Map<String, List<String>> schemaVersions = probe.getSpProxy().getSchemaVersions();
+        for (String version : schemaVersions.keySet())
+        {
+            System.out.println(format("\t\t%s: %s%n", version, schemaVersions.get(version)));
+        }
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/DescribeRing.java b/src/java/org/apache/cassandra/tools/nodetool/DescribeRing.java
new file mode 100644
index 0000000..a120ffe
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/DescribeRing.java
@@ -0,0 +1,51 @@
+/*
+ * 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.tools.nodetool;
+
+import static org.apache.commons.lang3.StringUtils.EMPTY;
+import io.airlift.command.Arguments;
+import io.airlift.command.Command;
+
+import java.io.IOException;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "describering", description = "Shows the token ranges info of a given keyspace")
+public class DescribeRing extends NodeToolCmd
+{
+    @Arguments(description = "The keyspace name", required = true)
+    String keyspace = EMPTY;
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        System.out.println("Schema Version:" + probe.getSchemaVersion());
+        System.out.println("TokenRange: ");
+        try
+        {
+            for (String tokenRangeString : probe.describeRing(keyspace))
+            {
+                System.out.println("\t" + tokenRangeString);
+            }
+        } catch (IOException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/DisableAutoCompaction.java b/src/java/org/apache/cassandra/tools/nodetool/DisableAutoCompaction.java
new file mode 100644
index 0000000..2f5832d
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/DisableAutoCompaction.java
@@ -0,0 +1,53 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Arguments;
+import io.airlift.command.Command;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "disableautocompaction", description = "Disable autocompaction for the given keyspace and table")
+public class DisableAutoCompaction extends NodeToolCmd
+{
+    @Arguments(usage = "[<keyspace> <tables>...]", description = "The keyspace followed by one or many tables")
+    private List<String> args = new ArrayList<>();
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        List<String> keyspaces = parseOptionalKeyspace(args, probe);
+        String[] cfnames = parseOptionalColumnFamilies(args);
+
+        for (String keyspace : keyspaces)
+        {
+            try
+            {
+                probe.disableAutoCompaction(keyspace, cfnames);
+            } catch (IOException e)
+            {
+                throw new RuntimeException("Error occurred during disabling auto-compaction", e);
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/DisableBackup.java b/src/java/org/apache/cassandra/tools/nodetool/DisableBackup.java
new file mode 100644
index 0000000..74e7f50
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/DisableBackup.java
@@ -0,0 +1,33 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Command;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "disablebackup", description = "Disable incremental backup")
+public class DisableBackup extends NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        probe.setIncrementalBackupsEnabled(false);
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/DisableBinary.java b/src/java/org/apache/cassandra/tools/nodetool/DisableBinary.java
new file mode 100644
index 0000000..dee319b
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/DisableBinary.java
@@ -0,0 +1,33 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Command;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "disablebinary", description = "Disable native transport (binary protocol)")
+public class DisableBinary extends NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        probe.stopNativeTransport();
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/DisableGossip.java b/src/java/org/apache/cassandra/tools/nodetool/DisableGossip.java
new file mode 100644
index 0000000..32448c9
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/DisableGossip.java
@@ -0,0 +1,33 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Command;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "disablegossip", description = "Disable gossip (effectively marking the node down)")
+public class DisableGossip extends NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        probe.stopGossiping();
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/DisableHandoff.java b/src/java/org/apache/cassandra/tools/nodetool/DisableHandoff.java
new file mode 100644
index 0000000..11cd754
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/DisableHandoff.java
@@ -0,0 +1,33 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Command;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "disablehandoff", description = "Disable storing hinted handoffs")
+public class DisableHandoff extends NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        probe.disableHintedHandoff();
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/DisableThrift.java b/src/java/org/apache/cassandra/tools/nodetool/DisableThrift.java
new file mode 100644
index 0000000..148b195
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/DisableThrift.java
@@ -0,0 +1,33 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Command;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "disablethrift", description = "Disable thrift server")
+public class DisableThrift extends NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        probe.stopThriftServer();
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/Drain.java b/src/java/org/apache/cassandra/tools/nodetool/Drain.java
new file mode 100644
index 0000000..5562e6d
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/Drain.java
@@ -0,0 +1,42 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Command;
+
+import java.io.IOException;
+import java.util.concurrent.ExecutionException;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "drain", description = "Drain the node (stop accepting writes and flush all tables)")
+public class Drain extends NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        try
+        {
+            probe.drain();
+        } catch (IOException | InterruptedException | ExecutionException e)
+        {
+            throw new RuntimeException("Error occurred during flushing", e);
+        }
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/EnableAutoCompaction.java b/src/java/org/apache/cassandra/tools/nodetool/EnableAutoCompaction.java
new file mode 100644
index 0000000..e846187
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/EnableAutoCompaction.java
@@ -0,0 +1,53 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Arguments;
+import io.airlift.command.Command;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "enableautocompaction", description = "Enable autocompaction for the given keyspace and table")
+public class EnableAutoCompaction extends NodeToolCmd
+{
+    @Arguments(usage = "[<keyspace> <tables>...]", description = "The keyspace followed by one or many tables")
+    private List<String> args = new ArrayList<>();
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        List<String> keyspaces = parseOptionalKeyspace(args, probe);
+        String[] cfnames = parseOptionalColumnFamilies(args);
+
+        for (String keyspace : keyspaces)
+        {
+            try
+            {
+                probe.enableAutoCompaction(keyspace, cfnames);
+            } catch (IOException e)
+            {
+                throw new RuntimeException("Error occurred during enabling auto-compaction", e);
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/EnableBackup.java b/src/java/org/apache/cassandra/tools/nodetool/EnableBackup.java
new file mode 100644
index 0000000..4847fa5
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/EnableBackup.java
@@ -0,0 +1,33 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Command;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "enablebackup", description = "Enable incremental backup")
+public class EnableBackup extends NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        probe.setIncrementalBackupsEnabled(true);
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/EnableBinary.java b/src/java/org/apache/cassandra/tools/nodetool/EnableBinary.java
new file mode 100644
index 0000000..f1d5d9c
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/EnableBinary.java
@@ -0,0 +1,33 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Command;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "enablebinary", description = "Reenable native transport (binary protocol)")
+public class EnableBinary extends NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        probe.startNativeTransport();
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/EnableGossip.java b/src/java/org/apache/cassandra/tools/nodetool/EnableGossip.java
new file mode 100644
index 0000000..16a9f4b
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/EnableGossip.java
@@ -0,0 +1,33 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Command;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "enablegossip", description = "Reenable gossip")
+public class EnableGossip extends NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        probe.startGossiping();
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/EnableHandoff.java b/src/java/org/apache/cassandra/tools/nodetool/EnableHandoff.java
new file mode 100644
index 0000000..d18d77a
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/EnableHandoff.java
@@ -0,0 +1,45 @@
+/*
+ * 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.tools.nodetool;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import io.airlift.command.Arguments;
+import io.airlift.command.Command;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "enablehandoff", description = "Reenable the future hints storing on the current node")
+public class EnableHandoff extends NodeToolCmd
+{
+    @Arguments(usage = "<dc-name>,<dc-name>", description = "Enable hinted handoff only for these DCs")
+    private List<String> args = new ArrayList<>();
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        checkArgument(args.size() <= 1, "enablehandoff does not accept two args");
+        if(args.size() == 1)
+            probe.enableHintedHandoff(args.get(0));
+        else
+            probe.enableHintedHandoff();
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/EnableThrift.java b/src/java/org/apache/cassandra/tools/nodetool/EnableThrift.java
new file mode 100644
index 0000000..780b36d
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/EnableThrift.java
@@ -0,0 +1,33 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Command;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "enablethrift", description = "Reenable thrift server")
+public class EnableThrift extends NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        probe.startThriftServer();
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/FailureDetectorInfo.java b/src/java/org/apache/cassandra/tools/nodetool/FailureDetectorInfo.java
new file mode 100644
index 0000000..3c0303d
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/FailureDetectorInfo.java
@@ -0,0 +1,46 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Command;
+
+import java.util.List;
+
+import javax.management.openmbean.CompositeData;
+import javax.management.openmbean.TabularData;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "failuredetector", description = "Shows the failure detector information for the cluster")
+public class FailureDetectorInfo extends NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        TabularData data = probe.getFailureDetectorPhilValues();
+        System.out.printf("%10s,%16s%n", "Endpoint", "Phi");
+        for (Object o : data.keySet())
+        {
+            @SuppressWarnings({ "rawtypes", "unchecked" })
+            CompositeData datum = data.get(((List) o).toArray(new Object[((List) o).size()]));
+            System.out.printf("%10s,%16.8f%n",datum.get("Endpoint"), datum.get("PHI"));
+        }
+    }
+}
+
diff --git a/src/java/org/apache/cassandra/tools/nodetool/Flush.java b/src/java/org/apache/cassandra/tools/nodetool/Flush.java
new file mode 100644
index 0000000..e9038f7
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/Flush.java
@@ -0,0 +1,52 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Arguments;
+import io.airlift.command.Command;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "flush", description = "Flush one or more tables")
+public class Flush extends NodeToolCmd
+{
+    @Arguments(usage = "[<keyspace> <tables>...]", description = "The keyspace followed by one or many tables")
+    private List<String> args = new ArrayList<>();
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        List<String> keyspaces = parseOptionalKeyspace(args, probe);
+        String[] cfnames = parseOptionalColumnFamilies(args);
+
+        for (String keyspace : keyspaces)
+        {
+            try
+            {
+                probe.forceKeyspaceFlush(keyspace, cfnames);
+            } catch (Exception e)
+            {
+                throw new RuntimeException("Error occurred during flushing", e);
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/GcStats.java b/src/java/org/apache/cassandra/tools/nodetool/GcStats.java
new file mode 100644
index 0000000..dd38fe7
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/GcStats.java
@@ -0,0 +1,37 @@
+/*
+ * 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.tools.nodetool;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+import io.airlift.command.Command;
+
+@Command(name = "gcstats", description = "Print GC Statistics")
+public class GcStats extends NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        double[] stats = probe.getAndResetGCStats();
+        double mean = stats[2] / stats[5];
+        double stdev = Math.sqrt((stats[3] / stats[5]) - (mean * mean));
+        System.out.printf("%20s%20s%20s%20s%20s%20s%25s%n", "Interval (ms)", "Max GC Elapsed (ms)", "Total GC Elapsed (ms)", "Stdev GC Elapsed (ms)", "GC Reclaimed (MB)", "Collections", "Direct Memory Bytes");
+        System.out.printf("%20.0f%20.0f%20.0f%20.0f%20.0f%20.0f%25d%n", stats[0], stats[1], stats[2], stdev, stats[4], stats[5], (long)stats[6]);
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/GetCompactionThreshold.java b/src/java/org/apache/cassandra/tools/nodetool/GetCompactionThreshold.java
new file mode 100644
index 0000000..6c629de
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/GetCompactionThreshold.java
@@ -0,0 +1,49 @@
+/*
+ * 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.tools.nodetool;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import io.airlift.command.Arguments;
+import io.airlift.command.Command;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.db.ColumnFamilyStoreMBean;
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "getcompactionthreshold", description = "Print min and max compaction thresholds for a given table")
+public class GetCompactionThreshold extends NodeToolCmd
+{
+    @Arguments(usage = "<keyspace> <table>", description = "The keyspace with a table")
+    private List<String> args = new ArrayList<>();
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        checkArgument(args.size() == 2, "getcompactionthreshold requires ks and cf args");
+        String ks = args.get(0);
+        String cf = args.get(1);
+
+        ColumnFamilyStoreMBean cfsProxy = probe.getCfsProxy(ks, cf);
+        System.out.println("Current compaction thresholds for " + ks + "/" + cf + ": \n" +
+                " min = " + cfsProxy.getMinimumCompactionThreshold() + ", " +
+                " max = " + cfsProxy.getMaximumCompactionThreshold());
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/GetCompactionThroughput.java b/src/java/org/apache/cassandra/tools/nodetool/GetCompactionThroughput.java
new file mode 100644
index 0000000..c3af184
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/GetCompactionThroughput.java
@@ -0,0 +1,33 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Command;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "getcompactionthroughput", description = "Print the MB/s throughput cap for compaction in the system")
+public class GetCompactionThroughput extends NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        System.out.println("Current compaction throughput: " + probe.getCompactionThroughput() + " MB/s");
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/GetEndpoints.java b/src/java/org/apache/cassandra/tools/nodetool/GetEndpoints.java
new file mode 100644
index 0000000..49d2148
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/GetEndpoints.java
@@ -0,0 +1,51 @@
+/*
+ * 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.tools.nodetool;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import io.airlift.command.Arguments;
+import io.airlift.command.Command;
+
+import java.net.InetAddress;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "getendpoints", description = "Print the end points that owns the key")
+public class GetEndpoints extends NodeToolCmd
+{
+    @Arguments(usage = "<keyspace> <table> <key>", description = "The keyspace, the table, and the partition key for which we need to find the endpoint")
+    private List<String> args = new ArrayList<>();
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        checkArgument(args.size() == 3, "getendpoints requires keyspace, table and partition key arguments");
+        String ks = args.get(0);
+        String table = args.get(1);
+        String key = args.get(2);
+
+        List<InetAddress> endpoints = probe.getEndpoints(ks, table, key);
+        for (InetAddress endpoint : endpoints)
+        {
+            System.out.println(endpoint.getHostAddress());
+        }
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/GetInterDCStreamThroughput.java b/src/java/org/apache/cassandra/tools/nodetool/GetInterDCStreamThroughput.java
new file mode 100644
index 0000000..4c354c0
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/GetInterDCStreamThroughput.java
@@ -0,0 +1,33 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Command;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "getinterdcstreamthroughput", description = "Print the Mb/s throughput cap for inter-datacenter streaming in the system")
+public class GetInterDCStreamThroughput extends NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        System.out.println("Current inter-datacenter stream throughput: " + probe.getInterDCStreamThroughput() + " Mb/s");
+    }
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/GetLoggingLevels.java b/src/java/org/apache/cassandra/tools/nodetool/GetLoggingLevels.java
new file mode 100644
index 0000000..7ce0017
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/GetLoggingLevels.java
@@ -0,0 +1,38 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Command;
+
+import java.util.Map;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "getlogginglevels", description = "Get the runtime logging levels")
+public class GetLoggingLevels extends NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        // what if some one set a very long logger name? 50 space may not be enough...
+        System.out.printf("%n%-50s%10s%n", "Logger Name", "Log Level");
+        for (Map.Entry<String, String> entry : probe.getLoggingLevels().entrySet())
+            System.out.printf("%-50s%10s%n", entry.getKey(), entry.getValue());
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/GetSSTables.java b/src/java/org/apache/cassandra/tools/nodetool/GetSSTables.java
new file mode 100644
index 0000000..2c5d46b
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/GetSSTables.java
@@ -0,0 +1,50 @@
+/*
+ * 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.tools.nodetool;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import io.airlift.command.Arguments;
+import io.airlift.command.Command;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "getsstables", description = "Print the sstable filenames that own the key")
+public class GetSSTables extends NodeToolCmd
+{
+    @Arguments(usage = "<keyspace> <table> <key>", description = "The keyspace, the table, and the key")
+    private List<String> args = new ArrayList<>();
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        checkArgument(args.size() == 3, "getsstables requires ks, cf and key args");
+        String ks = args.get(0);
+        String cf = args.get(1);
+        String key = args.get(2);
+
+        List<String> sstables = probe.getSSTables(ks, cf, key);
+        for (String sstable : sstables)
+        {
+            System.out.println(sstable);
+        }
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/GetStreamThroughput.java b/src/java/org/apache/cassandra/tools/nodetool/GetStreamThroughput.java
new file mode 100644
index 0000000..437eb54
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/GetStreamThroughput.java
@@ -0,0 +1,33 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Command;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "getstreamthroughput", description = "Print the Mb/s throughput cap for streaming in the system")
+public class GetStreamThroughput extends NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        System.out.println("Current stream throughput: " + probe.getStreamThroughput() + " Mb/s");
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/GetTraceProbability.java b/src/java/org/apache/cassandra/tools/nodetool/GetTraceProbability.java
new file mode 100644
index 0000000..3940790
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/GetTraceProbability.java
@@ -0,0 +1,33 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Command;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "gettraceprobability", description = "Print the current trace probability value")
+public class GetTraceProbability extends NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        System.out.println("Current trace probability: " + probe.getTraceProbability());
+    }
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/GossipInfo.java b/src/java/org/apache/cassandra/tools/nodetool/GossipInfo.java
new file mode 100644
index 0000000..2acfcf1
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/GossipInfo.java
@@ -0,0 +1,33 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Command;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "gossipinfo", description = "Shows the gossip information for the cluster")
+public class GossipInfo extends NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        System.out.println(probe.getGossipInfo());
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/HostStat.java b/src/java/org/apache/cassandra/tools/nodetool/HostStat.java
new file mode 100644
index 0000000..19c0448
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/HostStat.java
@@ -0,0 +1,41 @@
+/*
+ * 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.tools.nodetool;
+
+import java.net.InetAddress;
+
+public class HostStat
+{
+    public final InetAddress endpoint;
+    public final boolean resolveIp;
+    public final Float owns;
+    public final String token;
+
+    public HostStat(String token, InetAddress endpoint, boolean resolveIp, Float owns)
+    {
+        this.token = token;
+        this.endpoint = endpoint;
+        this.resolveIp = resolveIp;
+        this.owns = owns;
+    }
+
+    public String ipOrDns()
+    {
+        return resolveIp ? endpoint.getHostName() : endpoint.getHostAddress();
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/Info.java b/src/java/org/apache/cassandra/tools/nodetool/Info.java
new file mode 100644
index 0000000..0d9bd73
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/Info.java
@@ -0,0 +1,162 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Command;
+import io.airlift.command.Option;
+
+import java.lang.management.MemoryUsage;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import javax.management.InstanceNotFoundException;
+
+import org.apache.cassandra.db.ColumnFamilyStoreMBean;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.service.CacheServiceMBean;
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "info", description = "Print node information (uptime, load, ...)")
+public class Info extends NodeToolCmd
+{
+    @Option(name = {"-T", "--tokens"}, description = "Display all tokens")
+    private boolean tokens = false;
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        boolean gossipInitialized = probe.isInitialized();
+
+        System.out.printf("%-23s: %s%n", "ID", probe.getLocalHostId());
+        System.out.printf("%-23s: %s%n", "Gossip active", gossipInitialized);
+        System.out.printf("%-23s: %s%n", "Thrift active", probe.isThriftServerRunning());
+        System.out.printf("%-23s: %s%n", "Native Transport active", probe.isNativeTransportRunning());
+        System.out.printf("%-23s: %s%n", "Load", probe.getLoadString());
+        if (gossipInitialized)
+            System.out.printf("%-23s: %s%n", "Generation No", probe.getCurrentGenerationNumber());
+        else
+            System.out.printf("%-23s: %s%n", "Generation No", 0);
+
+        // Uptime
+        long secondsUp = probe.getUptime() / 1000;
+        System.out.printf("%-23s: %d%n", "Uptime (seconds)", secondsUp);
+
+        // Memory usage
+        MemoryUsage heapUsage = probe.getHeapMemoryUsage();
+        double memUsed = (double) heapUsage.getUsed() / (1024 * 1024);
+        double memMax = (double) heapUsage.getMax() / (1024 * 1024);
+        System.out.printf("%-23s: %.2f / %.2f%n", "Heap Memory (MB)", memUsed, memMax);
+        try
+        {
+            System.out.printf("%-23s: %.2f%n", "Off Heap Memory (MB)", getOffHeapMemoryUsed(probe));
+        }
+        catch (RuntimeException e)
+        {
+            // offheap-metrics introduced in 2.1.3 - older versions do not have the appropriate mbeans
+            if (!(e.getCause() instanceof InstanceNotFoundException))
+                throw e;
+        }
+
+        // Data Center/Rack
+        System.out.printf("%-23s: %s%n", "Data Center", probe.getDataCenter());
+        System.out.printf("%-23s: %s%n", "Rack", probe.getRack());
+
+        // Exceptions
+        System.out.printf("%-23s: %s%n", "Exceptions", probe.getStorageMetric("Exceptions"));
+
+        CacheServiceMBean cacheService = probe.getCacheServiceMBean();
+
+        // Key Cache: Hits, Requests, RecentHitRate, SavePeriodInSeconds
+        System.out.printf("%-23s: entries %d, size %s, capacity %s, %d hits, %d requests, %.3f recent hit rate, %d save period in seconds%n",
+                "Key Cache",
+                probe.getCacheMetric("KeyCache", "Entries"),
+                FileUtils.stringifyFileSize((long) probe.getCacheMetric("KeyCache", "Size")),
+                FileUtils.stringifyFileSize((long) probe.getCacheMetric("KeyCache", "Capacity")),
+                probe.getCacheMetric("KeyCache", "Hits"),
+                probe.getCacheMetric("KeyCache", "Requests"),
+                probe.getCacheMetric("KeyCache", "HitRate"),
+                cacheService.getKeyCacheSavePeriodInSeconds());
+
+        // Row Cache: Hits, Requests, RecentHitRate, SavePeriodInSeconds
+        System.out.printf("%-23s: entries %d, size %s, capacity %s, %d hits, %d requests, %.3f recent hit rate, %d save period in seconds%n",
+                "Row Cache",
+                probe.getCacheMetric("RowCache", "Entries"),
+                FileUtils.stringifyFileSize((long) probe.getCacheMetric("RowCache", "Size")),
+                FileUtils.stringifyFileSize((long) probe.getCacheMetric("RowCache", "Capacity")),
+                probe.getCacheMetric("RowCache", "Hits"),
+                probe.getCacheMetric("RowCache", "Requests"),
+                probe.getCacheMetric("RowCache", "HitRate"),
+                cacheService.getRowCacheSavePeriodInSeconds());
+
+        // Counter Cache: Hits, Requests, RecentHitRate, SavePeriodInSeconds
+        System.out.printf("%-23s: entries %d, size %s, capacity %s, %d hits, %d requests, %.3f recent hit rate, %d save period in seconds%n",
+                "Counter Cache",
+                probe.getCacheMetric("CounterCache", "Entries"),
+                FileUtils.stringifyFileSize((long) probe.getCacheMetric("CounterCache", "Size")),
+                FileUtils.stringifyFileSize((long) probe.getCacheMetric("CounterCache", "Capacity")),
+                probe.getCacheMetric("CounterCache", "Hits"),
+                probe.getCacheMetric("CounterCache", "Requests"),
+                probe.getCacheMetric("CounterCache", "HitRate"),
+                cacheService.getCounterCacheSavePeriodInSeconds());
+
+        // check if node is already joined, before getting tokens, since it throws exception if not.
+        if (probe.isJoined())
+        {
+            // Tokens
+            List<String> tokens = probe.getTokens();
+            if (tokens.size() == 1 || this.tokens)
+                for (String token : tokens)
+                    System.out.printf("%-23s: %s%n", "Token", token);
+            else
+                System.out.printf("%-23s: (invoke with -T/--tokens to see all %d tokens)%n", "Token",
+                                  tokens.size());
+        }
+        else
+        {
+            System.out.printf("%-23s: (node is not joined to the cluster)%n", "Token");
+        }
+    }
+
+    /**
+     * Returns the total off heap memory used in MB.
+     * @return the total off heap memory used in MB.
+     */
+    private static double getOffHeapMemoryUsed(NodeProbe probe)
+    {
+        long offHeapMemUsedInBytes = 0;
+        // get a list of column family stores
+        Iterator<Map.Entry<String, ColumnFamilyStoreMBean>> cfamilies = probe.getColumnFamilyStoreMBeanProxies();
+
+        while (cfamilies.hasNext())
+        {
+            Entry<String, ColumnFamilyStoreMBean> entry = cfamilies.next();
+            String keyspaceName = entry.getKey();
+            String cfName = entry.getValue().getColumnFamilyName();
+
+            offHeapMemUsedInBytes += (Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "MemtableOffHeapSize");
+            offHeapMemUsedInBytes += (Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "BloomFilterOffHeapMemoryUsed");
+            offHeapMemUsedInBytes += (Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "IndexSummaryOffHeapMemoryUsed");
+            offHeapMemUsedInBytes += (Long) probe.getColumnFamilyMetric(keyspaceName, cfName, "CompressionMetadataOffHeapMemoryUsed");
+        }
+
+        return offHeapMemUsedInBytes / (1024d * 1024);
+    }
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/InvalidateCounterCache.java b/src/java/org/apache/cassandra/tools/nodetool/InvalidateCounterCache.java
new file mode 100644
index 0000000..a5f0ebc
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/InvalidateCounterCache.java
@@ -0,0 +1,33 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Command;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "invalidatecountercache", description = "Invalidate the counter cache")
+public class InvalidateCounterCache extends NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        probe.invalidateCounterCache();
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/InvalidateKeyCache.java b/src/java/org/apache/cassandra/tools/nodetool/InvalidateKeyCache.java
new file mode 100644
index 0000000..70abd53
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/InvalidateKeyCache.java
@@ -0,0 +1,33 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Command;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "invalidatekeycache", description = "Invalidate the key cache")
+public class InvalidateKeyCache extends NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        probe.invalidateKeyCache();
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/InvalidateRowCache.java b/src/java/org/apache/cassandra/tools/nodetool/InvalidateRowCache.java
new file mode 100644
index 0000000..149f80b
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/InvalidateRowCache.java
@@ -0,0 +1,33 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Command;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "invalidaterowcache", description = "Invalidate the row cache")
+public class InvalidateRowCache extends NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        probe.invalidateRowCache();
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/Join.java b/src/java/org/apache/cassandra/tools/nodetool/Join.java
new file mode 100644
index 0000000..5815591
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/Join.java
@@ -0,0 +1,44 @@
+/*
+ * 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.tools.nodetool;
+
+import static com.google.common.base.Preconditions.checkState;
+import io.airlift.command.Command;
+
+import java.io.IOException;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "join", description = "Join the ring")
+public class Join extends NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        checkState(!probe.isJoined(), "This node has already joined the ring.");
+
+        try
+        {
+            probe.joinRing();
+        } catch (IOException e)
+        {
+            throw new RuntimeException("Error during joining the ring", e);
+        }
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/ListSnapshots.java b/src/java/org/apache/cassandra/tools/nodetool/ListSnapshots.java
new file mode 100644
index 0000000..ee7bf34
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/ListSnapshots.java
@@ -0,0 +1,72 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Command;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import javax.management.openmbean.TabularData;
+
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "listsnapshots", description = "Lists all the snapshots along with the size on disk and true size.")
+public class ListSnapshots extends NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        try
+        {
+            System.out.println("Snapshot Details: ");
+
+            final Map<String,TabularData> snapshotDetails = probe.getSnapshotDetails();
+            if (snapshotDetails.isEmpty())
+            {
+                System.out.printf("There are no snapshots");
+                return;
+            }
+
+            final long trueSnapshotsSize = probe.trueSnapshotsSize();
+            final String format = "%-20s%-29s%-29s%-19s%-19s%n";
+            // display column names only once
+            final List<String> indexNames = snapshotDetails.entrySet().iterator().next().getValue().getTabularType().getIndexNames();
+            System.out.printf(format, (Object[]) indexNames.toArray(new String[indexNames.size()]));
+
+            for (final Map.Entry<String, TabularData> snapshotDetail : snapshotDetails.entrySet())
+            {
+                Set<?> values = snapshotDetail.getValue().keySet();
+                for (Object eachValue : values)
+                {
+                    final List<?> value = (List<?>) eachValue;
+                    System.out.printf(format, value.toArray(new Object[value.size()]));
+                }
+            }
+
+            System.out.println("\nTotal TrueDiskSpaceUsed: " + FileUtils.stringifyFileSize(trueSnapshotsSize) + "\n");
+        }
+        catch (Exception e)
+        {
+            throw new RuntimeException("Error during list snapshot", e);
+        }
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/Move.java b/src/java/org/apache/cassandra/tools/nodetool/Move.java
new file mode 100644
index 0000000..fc6b1bf
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/Move.java
@@ -0,0 +1,46 @@
+/*
+ * 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.tools.nodetool;
+
+import static org.apache.commons.lang3.StringUtils.EMPTY;
+import io.airlift.command.Arguments;
+import io.airlift.command.Command;
+
+import java.io.IOException;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "move", description = "Move node on the token ring to a new token")
+public class Move extends NodeToolCmd
+{
+    @Arguments(usage = "<new token>", description = "The new token.", required = true)
+    private String newToken = EMPTY;
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        try
+        {
+            probe.move(newToken);
+        } catch (IOException e)
+        {
+            throw new RuntimeException("Error during moving node", e);
+        }
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/NetStats.java b/src/java/org/apache/cassandra/tools/nodetool/NetStats.java
new file mode 100644
index 0000000..5b84dff
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/NetStats.java
@@ -0,0 +1,134 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Command;
+import io.airlift.command.Option;
+
+import java.util.Set;
+
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.net.MessagingServiceMBean;
+import org.apache.cassandra.streaming.ProgressInfo;
+import org.apache.cassandra.streaming.SessionInfo;
+import org.apache.cassandra.streaming.StreamState;
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "netstats", description = "Print network information on provided host (connecting node by default)")
+public class NetStats extends NodeToolCmd
+{
+    @Option(title = "human_readable",
+            name = {"-H", "--human-readable"},
+            description = "Display bytes in human readable form, i.e. KB, MB, GB, TB")
+    private boolean humanReadable = false;
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        System.out.printf("Mode: %s%n", probe.getOperationMode());
+        Set<StreamState> statuses = probe.getStreamStatus();
+        if (statuses.isEmpty())
+            System.out.println("Not sending any streams.");
+        for (StreamState status : statuses)
+        {
+            System.out.printf("%s %s%n", status.description, status.planId.toString());
+            for (SessionInfo info : status.sessions)
+            {
+                System.out.printf("    %s", info.peer.toString());
+                // print private IP when it is used
+                if (!info.peer.equals(info.connecting))
+                {
+                    System.out.printf(" (using %s)", info.connecting.toString());
+                }
+                System.out.printf("%n");
+                if (!info.receivingSummaries.isEmpty())
+                {
+                    if (humanReadable)
+                        System.out.printf("        Receiving %d files, %s total. Already received %d files, %s total%n", info.getTotalFilesToReceive(), FileUtils.stringifyFileSize(info.getTotalSizeToReceive()), info.getTotalFilesReceived(), FileUtils.stringifyFileSize(info.getTotalSizeReceived()));
+                    else
+                        System.out.printf("        Receiving %d files, %d bytes total. Already received %d files, %d bytes total%n", info.getTotalFilesToReceive(), info.getTotalSizeToReceive(), info.getTotalFilesReceived(), info.getTotalSizeReceived());
+                    for (ProgressInfo progress : info.getReceivingFiles())
+                    {
+                        System.out.printf("            %s%n", progress.toString());
+                    }
+                }
+                if (!info.sendingSummaries.isEmpty())
+                {
+                    if (humanReadable)
+                        System.out.printf("        Sending %d files, %s total. Already sent %d files, %s total%n", info.getTotalFilesToSend(), FileUtils.stringifyFileSize(info.getTotalSizeToSend()), info.getTotalFilesSent(), FileUtils.stringifyFileSize(info.getTotalSizeSent()));
+                    else
+                        System.out.printf("        Sending %d files, %d bytes total. Already sent %d files, %d bytes total%n", info.getTotalFilesToSend(), info.getTotalSizeToSend(), info.getTotalFilesSent(), info.getTotalSizeSent());
+                    for (ProgressInfo progress : info.getSendingFiles())
+                    {
+                        System.out.printf("            %s%n", progress.toString());
+                    }
+                }
+            }
+        }
+
+        if (!probe.isStarting())
+        {
+            System.out.printf("Read Repair Statistics:%nAttempted: %d%nMismatch (Blocking): %d%nMismatch (Background): %d%n", probe.getReadRepairAttempted(), probe.getReadRepairRepairedBlocking(), probe.getReadRepairRepairedBackground());
+
+            MessagingServiceMBean ms = probe.msProxy;
+            System.out.printf("%-25s", "Pool Name");
+            System.out.printf("%10s", "Active");
+            System.out.printf("%10s", "Pending");
+            System.out.printf("%15s", "Completed");
+            System.out.printf("%10s%n", "Dropped");
+
+            int pending;
+            long completed;
+            long dropped;
+
+            pending = 0;
+            for (int n : ms.getLargeMessagePendingTasks().values())
+                pending += n;
+            completed = 0;
+            for (long n : ms.getLargeMessageCompletedTasks().values())
+                completed += n;
+            dropped = 0;
+            for (long n : ms.getLargeMessageDroppedTasks().values())
+                dropped += n;
+            System.out.printf("%-25s%10s%10s%15s%10s%n", "Large messages", "n/a", pending, completed, dropped);
+
+            pending = 0;
+            for (int n : ms.getSmallMessagePendingTasks().values())
+                pending += n;
+            completed = 0;
+            for (long n : ms.getSmallMessageCompletedTasks().values())
+                completed += n;
+            dropped = 0;
+            for (long n : ms.getSmallMessageDroppedTasks().values())
+                dropped += n;
+            System.out.printf("%-25s%10s%10s%15s%10s%n", "Small messages", "n/a", pending, completed, dropped);
+
+            pending = 0;
+            for (int n : ms.getGossipMessagePendingTasks().values())
+                pending += n;
+            completed = 0;
+            for (long n : ms.getGossipMessageCompletedTasks().values())
+                completed += n;
+            dropped = 0;
+            for (long n : ms.getGossipMessageDroppedTasks().values())
+                dropped += n;
+            System.out.printf("%-25s%10s%10s%15s%10s%n", "Gossip messages", "n/a", pending, completed, dropped);
+        }
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/PauseHandoff.java b/src/java/org/apache/cassandra/tools/nodetool/PauseHandoff.java
new file mode 100644
index 0000000..ed1f655
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/PauseHandoff.java
@@ -0,0 +1,33 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Command;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "pausehandoff", description = "Pause hints delivery process")
+public class PauseHandoff extends NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        probe.pauseHintsDelivery();
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/ProxyHistograms.java b/src/java/org/apache/cassandra/tools/nodetool/ProxyHistograms.java
new file mode 100644
index 0000000..2a2851d
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/ProxyHistograms.java
@@ -0,0 +1,52 @@
+/*
+ * 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.tools.nodetool;
+
+import static java.lang.String.format;
+import io.airlift.command.Command;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "proxyhistograms", description = "Print statistic histograms for network operations")
+public class ProxyHistograms extends NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        String[] percentiles = new String[]{"50%", "75%", "95%", "98%", "99%", "Min", "Max"};
+        double[] readLatency = probe.metricPercentilesAsArray(probe.getProxyMetric("Read"));
+        double[] writeLatency = probe.metricPercentilesAsArray(probe.getProxyMetric("Write"));
+        double[] rangeLatency = probe.metricPercentilesAsArray(probe.getProxyMetric("RangeSlice"));
+
+        System.out.println("proxy histograms");
+        System.out.println(format("%-10s%18s%18s%18s",
+                "Percentile", "Read Latency", "Write Latency", "Range Latency"));
+        System.out.println(format("%-10s%18s%18s%18s",
+                "", "(micros)", "(micros)", "(micros)"));
+        for (int i = 0; i < percentiles.length; i++)
+        {
+            System.out.println(format("%-10s%18.2f%18.2f%18.2f",
+                    percentiles[i],
+                    readLatency[i],
+                    writeLatency[i],
+                    rangeLatency[i]));
+        }
+        System.out.println();
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/RangeKeySample.java b/src/java/org/apache/cassandra/tools/nodetool/RangeKeySample.java
new file mode 100644
index 0000000..e079a4b
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/RangeKeySample.java
@@ -0,0 +1,40 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Command;
+
+import java.util.List;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "rangekeysample", description = "Shows the sampled keys held across all keyspaces")
+public class RangeKeySample extends NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        System.out.println("RangeKeySample: ");
+        List<String> tokenStrings = probe.sampleKeyRange();
+        for (String tokenString : tokenStrings)
+        {
+            System.out.println("\t" + tokenString);
+        }
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/Rebuild.java b/src/java/org/apache/cassandra/tools/nodetool/Rebuild.java
new file mode 100644
index 0000000..8a6dbf1
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/Rebuild.java
@@ -0,0 +1,37 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Arguments;
+import io.airlift.command.Command;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "rebuild", description = "Rebuild data by streaming from other nodes (similarly to bootstrap)")
+public class Rebuild extends NodeToolCmd
+{
+    @Arguments(usage = "<src-dc-name>", description = "Name of DC from which to select sources for streaming. By default, pick any DC")
+    private String sourceDataCenterName = null;
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        probe.rebuild(sourceDataCenterName);
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/RebuildIndex.java b/src/java/org/apache/cassandra/tools/nodetool/RebuildIndex.java
new file mode 100644
index 0000000..9985b2b
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/RebuildIndex.java
@@ -0,0 +1,43 @@
+/*
+ * 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.tools.nodetool;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.collect.Iterables.toArray;
+import io.airlift.command.Arguments;
+import io.airlift.command.Command;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "rebuild_index", description = "A full rebuild of native secondary indexes for a given table")
+public class RebuildIndex extends NodeToolCmd
+{
+    @Arguments(usage = "<keyspace> <table> <indexName...>", description = "The keyspace and table name followed by a list of index names (IndexNameExample: Standard3.IdxName Standard3.IdxName1)")
+    List<String> args = new ArrayList<>();
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        checkArgument(args.size() >= 3, "rebuild_index requires ks, cf and idx args");
+        probe.rebuildIndex(args.get(0), args.get(1), toArray(args.subList(2, args.size()), String.class));
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/Refresh.java b/src/java/org/apache/cassandra/tools/nodetool/Refresh.java
new file mode 100644
index 0000000..153255c
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/Refresh.java
@@ -0,0 +1,42 @@
+/*
+ * 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.tools.nodetool;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import io.airlift.command.Arguments;
+import io.airlift.command.Command;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "refresh", description = "Load newly placed SSTables to the system without restart")
+public class Refresh extends NodeToolCmd
+{
+    @Arguments(usage = "<keyspace> <table>", description = "The keyspace and table name")
+    private List<String> args = new ArrayList<>();
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        checkArgument(args.size() == 2, "refresh requires ks and cf args");
+        probe.loadNewSSTables(args.get(0), args.get(1));
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/RefreshSizeEstimates.java b/src/java/org/apache/cassandra/tools/nodetool/RefreshSizeEstimates.java
new file mode 100644
index 0000000..870c7b4
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/RefreshSizeEstimates.java
@@ -0,0 +1,33 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Command;
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool;
+
+@Command(name = "refreshsizeestimates", description = "Refresh system.size_estimates")
+public class RefreshSizeEstimates extends NodeTool.NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        probe.refreshSizeEstimates();
+    }
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/ReloadTriggers.java b/src/java/org/apache/cassandra/tools/nodetool/ReloadTriggers.java
new file mode 100644
index 0000000..416aff0
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/ReloadTriggers.java
@@ -0,0 +1,33 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Command;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "reloadtriggers", description = "Reload trigger classes")
+public class ReloadTriggers extends NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        probe.reloadTriggers();
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/RemoveNode.java b/src/java/org/apache/cassandra/tools/nodetool/RemoveNode.java
new file mode 100644
index 0000000..848049e
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/RemoveNode.java
@@ -0,0 +1,50 @@
+/*
+ * 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.tools.nodetool;
+
+import static org.apache.commons.lang3.StringUtils.EMPTY;
+import io.airlift.command.Arguments;
+import io.airlift.command.Command;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "removenode", description = "Show status of current node removal, force completion of pending removal or remove provided ID")
+public class RemoveNode extends NodeToolCmd
+{
+    @Arguments(title = "remove_operation", usage = "<status>|<force>|<ID>", description = "Show status of current node removal, force completion of pending removal, or remove provided ID", required = true)
+    private String removeOperation = EMPTY;
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        switch (removeOperation)
+        {
+            case "status":
+                System.out.println("RemovalStatus: " + probe.getRemovalStatus());
+                break;
+            case "force":
+                System.out.println("RemovalStatus: " + probe.getRemovalStatus());
+                probe.forceRemoveCompletion();
+                break;
+            default:
+                probe.removeNode(removeOperation);
+                break;
+        }
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/Repair.java b/src/java/org/apache/cassandra/tools/nodetool/Repair.java
new file mode 100644
index 0000000..7d0e207
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/Repair.java
@@ -0,0 +1,133 @@
+/*
+ * 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.tools.nodetool;
+
+import static com.google.common.collect.Lists.newArrayList;
+import static org.apache.commons.lang3.StringUtils.EMPTY;
+import io.airlift.command.Arguments;
+import io.airlift.command.Command;
+import io.airlift.command.Option;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import com.google.common.collect.Sets;
+
+import org.apache.cassandra.repair.RepairParallelism;
+import org.apache.cassandra.repair.SystemDistributedKeyspace;
+import org.apache.cassandra.repair.messages.RepairOption;
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+import org.apache.commons.lang3.StringUtils;
+
+@Command(name = "repair", description = "Repair one or more tables")
+public class Repair extends NodeToolCmd
+{
+    public final static Set<String> ONLY_EXPLICITLY_REPAIRED = Sets.newHashSet(SystemDistributedKeyspace.NAME);
+
+    @Arguments(usage = "[<keyspace> <tables>...]", description = "The keyspace followed by one or many tables")
+    private List<String> args = new ArrayList<>();
+
+    @Option(title = "seqential", name = {"-seq", "--sequential"}, description = "Use -seq to carry out a sequential repair")
+    private boolean sequential = false;
+
+    @Option(title = "dc parallel", name = {"-dcpar", "--dc-parallel"}, description = "Use -dcpar to repair data centers in parallel.")
+    private boolean dcParallel = false;
+
+    @Option(title = "local_dc", name = {"-local", "--in-local-dc"}, description = "Use -local to only repair against nodes in the same datacenter")
+    private boolean localDC = false;
+
+    @Option(title = "specific_dc", name = {"-dc", "--in-dc"}, description = "Use -dc to repair specific datacenters")
+    private List<String> specificDataCenters = new ArrayList<>();
+
+    @Option(title = "specific_host", name = {"-hosts", "--in-hosts"}, description = "Use -hosts to repair specific hosts")
+    private List<String> specificHosts = new ArrayList<>();
+
+    @Option(title = "start_token", name = {"-st", "--start-token"}, description = "Use -st to specify a token at which the repair range starts")
+    private String startToken = EMPTY;
+
+    @Option(title = "end_token", name = {"-et", "--end-token"}, description = "Use -et to specify a token at which repair range ends")
+    private String endToken = EMPTY;
+
+    @Option(title = "primary_range", name = {"-pr", "--partitioner-range"}, description = "Use -pr to repair only the first range returned by the partitioner")
+    private boolean primaryRange = false;
+
+    @Option(title = "full", name = {"-full", "--full"}, description = "Use -full to issue a full repair.")
+    private boolean fullRepair = false;
+
+    @Option(title = "job_threads", name = {"-j", "--job-threads"}, description = "Number of threads to run repair jobs. " +
+                                                                                 "Usually this means number of CFs to repair concurrently. " +
+                                                                                 "WARNING: increasing this puts more load on repairing nodes, so be careful. (default: 1, max: 4)")
+    private int numJobThreads = 1;
+
+    @Option(title = "trace_repair", name = {"-tr", "--trace"}, description = "Use -tr to trace the repair. Traces are logged to system_traces.events.")
+    private boolean trace = false;
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        List<String> keyspaces = parseOptionalKeyspace(args, probe);
+        String[] cfnames = parseOptionalColumnFamilies(args);
+
+        if (primaryRange && (!specificDataCenters.isEmpty() || !specificHosts.isEmpty()))
+            throw new RuntimeException("Primary range repair should be performed on all nodes in the cluster.");
+
+        for (String keyspace : keyspaces)
+        {
+            // avoid repairing system_distributed by default (CASSANDRA-9621)
+            if ((args == null || args.isEmpty()) && ONLY_EXPLICITLY_REPAIRED.contains(keyspace))
+                continue;
+
+            Map<String, String> options = new HashMap<>();
+            RepairParallelism parallelismDegree = RepairParallelism.PARALLEL;
+            if (sequential)
+                parallelismDegree = RepairParallelism.SEQUENTIAL;
+            else if (dcParallel)
+                parallelismDegree = RepairParallelism.DATACENTER_AWARE;
+            options.put(RepairOption.PARALLELISM_KEY, parallelismDegree.getName());
+            options.put(RepairOption.PRIMARY_RANGE_KEY, Boolean.toString(primaryRange));
+            options.put(RepairOption.INCREMENTAL_KEY, Boolean.toString(!fullRepair));
+            options.put(RepairOption.JOB_THREADS_KEY, Integer.toString(numJobThreads));
+            options.put(RepairOption.TRACE_KEY, Boolean.toString(trace));
+            options.put(RepairOption.COLUMNFAMILIES_KEY, StringUtils.join(cfnames, ","));
+            if (!startToken.isEmpty() || !endToken.isEmpty())
+            {
+                options.put(RepairOption.RANGES_KEY, startToken + ":" + endToken);
+            }
+            if (localDC)
+            {
+                options.put(RepairOption.DATACENTERS_KEY, StringUtils.join(newArrayList(probe.getDataCenter()), ","));
+            }
+            else
+            {
+                options.put(RepairOption.DATACENTERS_KEY, StringUtils.join(specificDataCenters, ","));
+            }
+            options.put(RepairOption.HOSTS_KEY, StringUtils.join(specificHosts, ","));
+            try
+            {
+                probe.repairAsync(System.out, keyspace, options);
+            } catch (Exception e)
+            {
+                throw new RuntimeException("Error occurred during repair", e);
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/ResetLocalSchema.java b/src/java/org/apache/cassandra/tools/nodetool/ResetLocalSchema.java
new file mode 100644
index 0000000..43280ab
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/ResetLocalSchema.java
@@ -0,0 +1,41 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Command;
+
+import java.io.IOException;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "resetlocalschema", description = "Reset node's local schema and resync")
+public class ResetLocalSchema extends NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        try
+        {
+            probe.resetLocalSchema();
+        } catch (IOException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/ResumeHandoff.java b/src/java/org/apache/cassandra/tools/nodetool/ResumeHandoff.java
new file mode 100644
index 0000000..584ab64
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/ResumeHandoff.java
@@ -0,0 +1,33 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Command;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "resumehandoff", description = "Resume hints delivery process")
+public class ResumeHandoff extends NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        probe.resumeHintsDelivery();
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/Ring.java b/src/java/org/apache/cassandra/tools/nodetool/Ring.java
new file mode 100644
index 0000000..5102029
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/Ring.java
@@ -0,0 +1,177 @@
+/*
+ * 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.tools.nodetool;
+
+import static java.lang.String.format;
+import io.airlift.command.Arguments;
+import io.airlift.command.Command;
+import io.airlift.command.Option;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.text.DecimalFormat;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+import com.google.common.collect.LinkedHashMultimap;
+
+@Command(name = "ring", description = "Print information about the token ring")
+public class Ring extends NodeToolCmd
+{
+    @Arguments(description = "Specify a keyspace for accurate ownership information (topology awareness)")
+    private String keyspace = null;
+
+    @Option(title = "resolve_ip", name = {"-r", "--resolve-ip"}, description = "Show node domain names instead of IPs")
+    private boolean resolveIp = false;
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        Map<String, String> tokensToEndpoints = probe.getTokenToEndpointMap();
+        LinkedHashMultimap<String, String> endpointsToTokens = LinkedHashMultimap.create();
+        boolean haveVnodes = false;
+        for (Map.Entry<String, String> entry : tokensToEndpoints.entrySet())
+        {
+            haveVnodes |= endpointsToTokens.containsKey(entry.getValue());
+            endpointsToTokens.put(entry.getValue(), entry.getKey());
+        }
+
+        int maxAddressLength = Collections.max(endpointsToTokens.keys(), new Comparator<String>()
+        {
+            @Override
+            public int compare(String first, String second)
+            {
+                return ((Integer) first.length()).compareTo(second.length());
+            }
+        }).length();
+
+        String formatPlaceholder = "%%-%ds  %%-12s%%-7s%%-8s%%-16s%%-20s%%-44s%%n";
+        String format = format(formatPlaceholder, maxAddressLength);
+
+        StringBuffer errors = new StringBuffer();
+        boolean showEffectiveOwnership = true;
+        // Calculate per-token ownership of the ring
+        Map<InetAddress, Float> ownerships;
+        try
+        {
+            ownerships = probe.effectiveOwnership(keyspace);
+        }
+        catch (IllegalStateException ex)
+        {
+            ownerships = probe.getOwnership();
+            errors.append("Note: " + ex.getMessage() + "%n");
+            showEffectiveOwnership = false;
+        }
+        catch (IllegalArgumentException ex)
+        {
+            System.out.printf("%nError: " + ex.getMessage() + "%n");
+            return;
+        }
+
+
+        System.out.println();
+        for (Entry<String, SetHostStat> entry : NodeTool.getOwnershipByDc(probe, resolveIp, tokensToEndpoints, ownerships).entrySet())
+            printDc(probe, format, entry.getKey(), endpointsToTokens, entry.getValue(),showEffectiveOwnership);
+
+        if (haveVnodes)
+        {
+            System.out.println("  Warning: \"nodetool ring\" is used to output all the tokens of a node.");
+            System.out.println("  To view status related info of a node use \"nodetool status\" instead.\n");
+        }
+
+        System.out.printf("%n  " + errors.toString());
+    }
+
+    private void printDc(NodeProbe probe, String format,
+                         String dc,
+                         LinkedHashMultimap<String, String> endpointsToTokens,
+                         SetHostStat hoststats,boolean showEffectiveOwnership)
+    {
+        Collection<String> liveNodes = probe.getLiveNodes();
+        Collection<String> deadNodes = probe.getUnreachableNodes();
+        Collection<String> joiningNodes = probe.getJoiningNodes();
+        Collection<String> leavingNodes = probe.getLeavingNodes();
+        Collection<String> movingNodes = probe.getMovingNodes();
+        Map<String, String> loadMap = probe.getLoadMap();
+
+        System.out.println("Datacenter: " + dc);
+        System.out.println("==========");
+
+        // get the total amount of replicas for this dc and the last token in this dc's ring
+        List<String> tokens = new ArrayList<>();
+        String lastToken = "";
+
+        for (HostStat stat : hoststats)
+        {
+            tokens.addAll(endpointsToTokens.get(stat.endpoint.getHostAddress()));
+            lastToken = tokens.get(tokens.size() - 1);
+        }
+
+        System.out.printf(format, "Address", "Rack", "Status", "State", "Load", "Owns", "Token");
+
+        if (hoststats.size() > 1)
+            System.out.printf(format, "", "", "", "", "", "", lastToken);
+        else
+            System.out.println();
+
+        for (HostStat stat : hoststats)
+        {
+            String endpoint = stat.endpoint.getHostAddress();
+            String rack;
+            try
+            {
+                rack = probe.getEndpointSnitchInfoProxy().getRack(endpoint);
+            }
+            catch (UnknownHostException e)
+            {
+                rack = "Unknown";
+            }
+
+            String status = liveNodes.contains(endpoint)
+                    ? "Up"
+                    : deadNodes.contains(endpoint)
+                            ? "Down"
+                            : "?";
+
+            String state = "Normal";
+
+            if (joiningNodes.contains(endpoint))
+                state = "Joining";
+            else if (leavingNodes.contains(endpoint))
+                state = "Leaving";
+            else if (movingNodes.contains(endpoint))
+                state = "Moving";
+
+            String load = loadMap.containsKey(endpoint)
+                    ? loadMap.get(endpoint)
+                    : "?";
+            String owns = stat.owns != null && showEffectiveOwnership? new DecimalFormat("##0.00%").format(stat.owns) : "?";
+            System.out.printf(format, stat.ipOrDns(), rack, status, state, load, owns, stat.token);
+        }
+        System.out.println();
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/Scrub.java b/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
new file mode 100644
index 0000000..dafe8d1
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
@@ -0,0 +1,76 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Arguments;
+import io.airlift.command.Command;
+import io.airlift.command.Option;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "scrub", description = "Scrub (rebuild sstables for) one or more tables")
+public class Scrub extends NodeToolCmd
+{
+    @Arguments(usage = "[<keyspace> <tables>...]", description = "The keyspace followed by one or many tables")
+    private List<String> args = new ArrayList<>();
+
+    @Option(title = "disable_snapshot",
+            name = {"-ns", "--no-snapshot"},
+            description = "Scrubbed CFs will be snapshotted first, if disableSnapshot is false. (default false)")
+    private boolean disableSnapshot = false;
+
+    @Option(title = "skip_corrupted",
+            name = {"-s", "--skip-corrupted"},
+            description = "Skip corrupted partitions even when scrubbing counter tables. (default false)")
+    private boolean skipCorrupted = false;
+
+    @Option(title = "no_validate",
+                   name = {"-n", "--no-validate"},
+                   description = "Do not validate columns using column validator")
+    private boolean noValidation = false;
+
+    @Option(title = "jobs",
+            name = {"-j", "--jobs"},
+            description = "Number of sstables to scrub simultanously, set to 0 to use all available compaction threads")
+    private int jobs = 2;
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        List<String> keyspaces = parseOptionalKeyspace(args, probe);
+        String[] cfnames = parseOptionalColumnFamilies(args);
+
+        for (String keyspace : keyspaces)
+        {
+            try
+            {
+                probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, jobs, keyspace, cfnames);
+            } catch (IllegalArgumentException e)
+            {
+                throw e;
+            } catch (Exception e)
+            {
+                throw new RuntimeException("Error occurred during scrubbing", e);
+            }
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/SetCacheCapacity.java b/src/java/org/apache/cassandra/tools/nodetool/SetCacheCapacity.java
new file mode 100644
index 0000000..6c280d8
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/SetCacheCapacity.java
@@ -0,0 +1,45 @@
+/*
+ * 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.tools.nodetool;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import io.airlift.command.Arguments;
+import io.airlift.command.Command;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "setcachecapacity", description = "Set global key, row, and counter cache capacities (in MB units)")
+public class SetCacheCapacity extends NodeToolCmd
+{
+    @Arguments(title = "<key-cache-capacity> <row-cache-capacity> <counter-cache-capacity>",
+               usage = "<key-cache-capacity> <row-cache-capacity> <counter-cache-capacity>",
+               description = "Key cache, row cache, and counter cache (in MB)",
+               required = true)
+    private List<Integer> args = new ArrayList<>();
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        checkArgument(args.size() == 3, "setcachecapacity requires key-cache-capacity, row-cache-capacity, and counter-cache-capacity args.");
+        probe.setCacheCapacities(args.get(0), args.get(1), args.get(2));
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/SetCacheKeysToSave.java b/src/java/org/apache/cassandra/tools/nodetool/SetCacheKeysToSave.java
new file mode 100644
index 0000000..12a4570
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/SetCacheKeysToSave.java
@@ -0,0 +1,45 @@
+/*
+ * 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.tools.nodetool;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import io.airlift.command.Arguments;
+import io.airlift.command.Command;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "setcachekeystosave", description = "Set number of keys saved by each cache for faster post-restart warmup. 0 to disable")
+public class SetCacheKeysToSave extends NodeToolCmd
+{
+    @Arguments(title = "<key-cache-keys-to-save> <row-cache-keys-to-save> <counter-cache-keys-to-save>",
+               usage = "<key-cache-keys-to-save> <row-cache-keys-to-save> <counter-cache-keys-to-save>",
+               description = "The number of keys saved by each cache. 0 to disable",
+               required = true)
+    private List<Integer> args = new ArrayList<>();
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        checkArgument(args.size() == 3, "setcachekeystosave requires key-cache-keys-to-save, row-cache-keys-to-save, and counter-cache-keys-to-save args.");
+        probe.setCacheKeysToSave(args.get(0), args.get(1), args.get(2));
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/SetCompactionThreshold.java b/src/java/org/apache/cassandra/tools/nodetool/SetCompactionThreshold.java
new file mode 100644
index 0000000..304f2b7
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/SetCompactionThreshold.java
@@ -0,0 +1,50 @@
+/*
+ * 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.tools.nodetool;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static java.lang.Integer.parseInt;
+import io.airlift.command.Arguments;
+import io.airlift.command.Command;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "setcompactionthreshold", description = "Set min and max compaction thresholds for a given table")
+public class SetCompactionThreshold extends NodeToolCmd
+{
+    @Arguments(title = "<keyspace> <table> <minthreshold> <maxthreshold>", usage = "<keyspace> <table> <minthreshold> <maxthreshold>", description = "The keyspace, the table, min and max threshold", required = true)
+    private List<String> args = new ArrayList<>();
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        checkArgument(args.size() == 4, "setcompactionthreshold requires ks, cf, min, and max threshold args.");
+
+        int minthreshold = parseInt(args.get(2));
+        int maxthreshold = parseInt(args.get(3));
+        checkArgument(minthreshold >= 0 && maxthreshold >= 0, "Thresholds must be positive integers");
+        checkArgument(minthreshold <= maxthreshold, "Min threshold cannot be greater than max.");
+        checkArgument(minthreshold >= 2 || maxthreshold == 0, "Min threshold must be at least 2");
+
+        probe.setCompactionThreshold(args.get(0), args.get(1), minthreshold, maxthreshold);
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/SetCompactionThroughput.java b/src/java/org/apache/cassandra/tools/nodetool/SetCompactionThroughput.java
new file mode 100644
index 0000000..0111a20
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/SetCompactionThroughput.java
@@ -0,0 +1,37 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Arguments;
+import io.airlift.command.Command;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "setcompactionthroughput", description = "Set the MB/s throughput cap for compaction in the system, or 0 to disable throttling")
+public class SetCompactionThroughput extends NodeToolCmd
+{
+    @Arguments(title = "compaction_throughput", usage = "<value_in_mb>", description = "Value in MB, 0 to disable throttling", required = true)
+    private Integer compactionThroughput = null;
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        probe.setCompactionThroughput(compactionThroughput);
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/SetHintedHandoffThrottleInKB.java b/src/java/org/apache/cassandra/tools/nodetool/SetHintedHandoffThrottleInKB.java
new file mode 100644
index 0000000..d20ff3f
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/SetHintedHandoffThrottleInKB.java
@@ -0,0 +1,37 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Arguments;
+import io.airlift.command.Command;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "sethintedhandoffthrottlekb", description =  "Set hinted handoff throttle in kb per second, per delivery thread.")
+public class SetHintedHandoffThrottleInKB extends NodeToolCmd
+{
+    @Arguments(title = "throttle_in_kb", usage = "<value_in_kb_per_sec>", description = "Value in KB per second", required = true)
+    private Integer throttleInKB = null;
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        probe.setHintedHandoffThrottleInKB(throttleInKB);
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/SetHostStat.java b/src/java/org/apache/cassandra/tools/nodetool/SetHostStat.java
new file mode 100644
index 0000000..c43abe1
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/SetHostStat.java
@@ -0,0 +1,54 @@
+/*
+ * 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.tools.nodetool;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+public class SetHostStat implements Iterable<HostStat>
+{
+    final List<HostStat> hostStats = new ArrayList<HostStat>();
+    final boolean resolveIp;
+
+    public SetHostStat(boolean resolveIp)
+    {
+        this.resolveIp = resolveIp;
+    }
+
+    public int size()
+    {
+        return hostStats.size();
+    }
+
+    @Override
+    public Iterator<HostStat> iterator()
+    {
+        return hostStats.iterator();
+    }
+
+    public void add(String token, String host, Map<InetAddress, Float> ownerships) throws UnknownHostException
+    {
+        InetAddress endpoint = InetAddress.getByName(host);
+        Float owns = ownerships.get(endpoint);
+        hostStats.add(new HostStat(token, endpoint, resolveIp, owns));
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/SetInterDCStreamThroughput.java b/src/java/org/apache/cassandra/tools/nodetool/SetInterDCStreamThroughput.java
new file mode 100644
index 0000000..41ce43a
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/SetInterDCStreamThroughput.java
@@ -0,0 +1,37 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Arguments;
+import io.airlift.command.Command;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "setinterdcstreamthroughput", description = "Set the Mb/s throughput cap for inter-datacenter streaming in the system, or 0 to disable throttling")
+public class SetInterDCStreamThroughput extends NodeToolCmd
+{
+    @Arguments(title = "inter_dc_stream_throughput", usage = "<value_in_mb>", description = "Value in Mb, 0 to disable throttling", required = true)
+    private Integer interDCStreamThroughput = null;
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        probe.setInterDCStreamThroughput(interDCStreamThroughput);
+    }
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/SetLoggingLevel.java b/src/java/org/apache/cassandra/tools/nodetool/SetLoggingLevel.java
new file mode 100644
index 0000000..d11d48f
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/SetLoggingLevel.java
@@ -0,0 +1,43 @@
+/*
+ * 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.tools.nodetool;
+
+import static org.apache.commons.lang3.StringUtils.EMPTY;
+import io.airlift.command.Arguments;
+import io.airlift.command.Command;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "setlogginglevel", description = "Set the log level threshold for a given class. If both class and level are empty/null, it will reset to the initial configuration")
+public class SetLoggingLevel extends NodeToolCmd
+{
+    @Arguments(usage = "<class> <level>", description = "The class to change the level for and the log level threshold to set (can be empty)")
+    private List<String> args = new ArrayList<>();
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        String classQualifier = args.size() >= 1 ? args.get(0) : EMPTY;
+        String level = args.size() == 2 ? args.get(1) : EMPTY;
+        probe.setLoggingLevel(classQualifier, level);
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/SetStreamThroughput.java b/src/java/org/apache/cassandra/tools/nodetool/SetStreamThroughput.java
new file mode 100644
index 0000000..8055872
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/SetStreamThroughput.java
@@ -0,0 +1,37 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Arguments;
+import io.airlift.command.Command;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "setstreamthroughput", description = "Set the Mb/s throughput cap for streaming in the system, or 0 to disable throttling")
+public class SetStreamThroughput extends NodeToolCmd
+{
+    @Arguments(title = "stream_throughput", usage = "<value_in_mb>", description = "Value in Mb, 0 to disable throttling", required = true)
+    private Integer streamThroughput = null;
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        probe.setStreamThroughput(streamThroughput);
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/SetTraceProbability.java b/src/java/org/apache/cassandra/tools/nodetool/SetTraceProbability.java
new file mode 100644
index 0000000..ebef1a4
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/SetTraceProbability.java
@@ -0,0 +1,39 @@
+/*
+ * 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.tools.nodetool;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import io.airlift.command.Arguments;
+import io.airlift.command.Command;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "settraceprobability", description = "Sets the probability for tracing any given request to value. 0 disables, 1 enables for all requests, 0 is the default")
+public class SetTraceProbability extends NodeToolCmd
+{
+    @Arguments(title = "trace_probability", usage = "<value>", description = "Trace probability between 0 and 1 (ex: 0.2)", required = true)
+    private Double traceProbability = null;
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        checkArgument(traceProbability >= 0 && traceProbability <= 1, "Trace probability must be between 0 and 1");
+        probe.setTraceProbability(traceProbability);
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/Snapshot.java b/src/java/org/apache/cassandra/tools/nodetool/Snapshot.java
new file mode 100644
index 0000000..2318620
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/Snapshot.java
@@ -0,0 +1,95 @@
+/*
+ * 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.tools.nodetool;
+
+import static com.google.common.collect.Iterables.toArray;
+import static org.apache.commons.lang3.StringUtils.join;
+import io.airlift.command.Arguments;
+import io.airlift.command.Command;
+import io.airlift.command.Option;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "snapshot", description = "Take a snapshot of specified keyspaces or a snapshot of the specified table")
+public class Snapshot extends NodeToolCmd
+{
+    @Arguments(usage = "[<keyspaces...>]", description = "List of keyspaces. By default, all keyspaces")
+    private List<String> keyspaces = new ArrayList<>();
+
+    @Option(title = "table", name = {"-cf", "--column-family", "--table"}, description = "The table name (you must specify one and only one keyspace for using this option)")
+    private String table = null;
+
+    @Option(title = "tag", name = {"-t", "--tag"}, description = "The name of the snapshot")
+    private String snapshotName = Long.toString(System.currentTimeMillis());
+
+    @Option(title = "ktlist", name = { "-kt", "--kt-list", "-kc", "--kc.list" }, description = "The list of Keyspace.table to take snapshot.(you must not specify only keyspace)")
+    private String ktList = null;
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        try
+        {
+            StringBuilder sb = new StringBuilder();
+
+            sb.append("Requested creating snapshot(s) for ");
+
+            // Create a separate path for kclist to avoid breaking of already existing scripts
+            if (null != ktList && !ktList.isEmpty())
+            {
+                ktList = ktList.replace(" ", "");
+                if (keyspaces.isEmpty() && null == table)
+                    sb.append("[").append(ktList).append("]");
+                else
+                {
+                    throw new IOException(
+                            "When specifying the Keyspace columfamily list for a snapshot, you should not specify columnfamily");
+                }
+                if (!snapshotName.isEmpty())
+                    sb.append(" with snapshot name [").append(snapshotName).append("]");
+                System.out.println(sb.toString());
+                probe.takeMultipleColumnFamilySnapshot(snapshotName, ktList.split(","));
+                System.out.println("Snapshot directory: " + snapshotName);
+            }
+            else
+            {
+                if (keyspaces.isEmpty())
+                    sb.append("[all keyspaces]");
+                else
+                    sb.append("[").append(join(keyspaces, ", ")).append("]");
+
+                if (!snapshotName.isEmpty())
+                    sb.append(" with snapshot name [").append(snapshotName).append("]");
+
+                System.out.println(sb.toString());
+
+                probe.takeSnapshot(snapshotName, table, toArray(keyspaces, String.class));
+                System.out.println("Snapshot directory: " + snapshotName);
+            }
+        }
+        catch (IOException e)
+        {
+            throw new RuntimeException("Error during taking a snapshot", e);
+        }
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/Status.java b/src/java/org/apache/cassandra/tools/nodetool/Status.java
new file mode 100644
index 0000000..99f745d
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/Status.java
@@ -0,0 +1,207 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Arguments;
+import io.airlift.command.Command;
+import io.airlift.command.Option;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.text.DecimalFormat;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+
+import org.apache.cassandra.locator.EndpointSnitchInfoMBean;
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+import com.google.common.collect.ArrayListMultimap;
+
+@Command(name = "status", description = "Print cluster information (state, load, IDs, ...)")
+public class Status extends NodeToolCmd
+{
+    @Arguments(usage = "[<keyspace>]", description = "The keyspace name")
+    private String keyspace = null;
+
+    @Option(title = "resolve_ip", name = {"-r", "--resolve-ip"}, description = "Show node domain names instead of IPs")
+    private boolean resolveIp = false;
+
+    private boolean isTokenPerNode = true;
+    private int maxAddressLength = 0;
+    private String format = null;
+    private Collection<String> joiningNodes, leavingNodes, movingNodes, liveNodes, unreachableNodes;
+    private Map<String, String> loadMap, hostIDMap;
+    private EndpointSnitchInfoMBean epSnitchInfo;
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        joiningNodes = probe.getJoiningNodes();
+        leavingNodes = probe.getLeavingNodes();
+        movingNodes = probe.getMovingNodes();
+        loadMap = probe.getLoadMap();
+        Map<String, String> tokensToEndpoints = probe.getTokenToEndpointMap();
+        liveNodes = probe.getLiveNodes();
+        unreachableNodes = probe.getUnreachableNodes();
+        hostIDMap = probe.getHostIdMap();
+        epSnitchInfo = probe.getEndpointSnitchInfoProxy();
+
+        StringBuffer errors = new StringBuffer();
+
+        Map<InetAddress, Float> ownerships = null;
+        boolean hasEffectiveOwns = false;
+        try
+        {
+            ownerships = probe.effectiveOwnership(keyspace);
+            hasEffectiveOwns = true;
+        }
+        catch (IllegalStateException e)
+        {
+            ownerships = probe.getOwnership();
+            errors.append("Note: " + e.getMessage() + "%n");
+        }
+        catch (IllegalArgumentException ex)
+        {
+            System.out.printf("%nError: " + ex.getMessage() + "%n");
+            System.exit(1);
+        }
+
+        SortedMap<String, SetHostStat> dcs = NodeTool.getOwnershipByDc(probe, resolveIp, tokensToEndpoints, ownerships);
+
+        // More tokens than nodes (aka vnodes)?
+        if (dcs.values().size() < tokensToEndpoints.keySet().size())
+            isTokenPerNode = false;
+
+        findMaxAddressLength(dcs);
+
+        // Datacenters
+        for (Map.Entry<String, SetHostStat> dc : dcs.entrySet())
+        {
+            String dcHeader = String.format("Datacenter: %s%n", dc.getKey());
+            System.out.printf(dcHeader);
+            for (int i = 0; i < (dcHeader.length() - 1); i++) System.out.print('=');
+            System.out.println();
+
+            // Legend
+            System.out.println("Status=Up/Down");
+            System.out.println("|/ State=Normal/Leaving/Joining/Moving");
+
+            printNodesHeader(hasEffectiveOwns, isTokenPerNode);
+
+            ArrayListMultimap<InetAddress, HostStat> hostToTokens = ArrayListMultimap.create();
+            for (HostStat stat : dc.getValue())
+                hostToTokens.put(stat.endpoint, stat);
+
+            for (InetAddress endpoint : hostToTokens.keySet())
+            {
+                Float owns = ownerships.get(endpoint);
+                List<HostStat> tokens = hostToTokens.get(endpoint);
+                printNode(endpoint.getHostAddress(), owns, tokens, hasEffectiveOwns, isTokenPerNode);
+            }
+        }
+
+        System.out.printf("%n" + errors.toString());
+
+    }
+
+    private void findMaxAddressLength(Map<String, SetHostStat> dcs)
+    {
+        maxAddressLength = 0;
+        for (Map.Entry<String, SetHostStat> dc : dcs.entrySet())
+        {
+            for (HostStat stat : dc.getValue())
+            {
+                maxAddressLength = Math.max(maxAddressLength, stat.ipOrDns().length());
+            }
+        }
+    }
+
+    private void printNodesHeader(boolean hasEffectiveOwns, boolean isTokenPerNode)
+    {
+        String fmt = getFormat(hasEffectiveOwns, isTokenPerNode);
+        String owns = hasEffectiveOwns ? "Owns (effective)" : "Owns";
+
+        if (isTokenPerNode)
+            System.out.printf(fmt, "-", "-", "Address", "Load", owns, "Host ID", "Token", "Rack");
+        else
+            System.out.printf(fmt, "-", "-", "Address", "Load", "Tokens", owns, "Host ID", "Rack");
+    }
+
+    private void printNode(String endpoint, Float owns, List<HostStat> tokens, boolean hasEffectiveOwns, boolean isTokenPerNode)
+    {
+        String status, state, load, strOwns, hostID, rack, fmt;
+        fmt = getFormat(hasEffectiveOwns, isTokenPerNode);
+        if (liveNodes.contains(endpoint)) status = "U";
+        else if (unreachableNodes.contains(endpoint)) status = "D";
+        else status = "?";
+        if (joiningNodes.contains(endpoint)) state = "J";
+        else if (leavingNodes.contains(endpoint)) state = "L";
+        else if (movingNodes.contains(endpoint)) state = "M";
+        else state = "N";
+
+        load = loadMap.containsKey(endpoint) ? loadMap.get(endpoint) : "?";
+        strOwns = owns != null && hasEffectiveOwns ? new DecimalFormat("##0.0%").format(owns) : "?";
+        hostID = hostIDMap.get(endpoint);
+
+        try
+        {
+            rack = epSnitchInfo.getRack(endpoint);
+        } catch (UnknownHostException e)
+        {
+            throw new RuntimeException(e);
+        }
+
+        String endpointDns = tokens.get(0).ipOrDns();
+        if (isTokenPerNode)
+            System.out.printf(fmt, status, state, endpointDns, load, strOwns, hostID, tokens.get(0).token, rack);
+        else
+            System.out.printf(fmt, status, state, endpointDns, load, tokens.size(), strOwns, hostID, rack);
+    }
+
+    private String getFormat(
+            boolean hasEffectiveOwns,
+            boolean isTokenPerNode)
+    {
+        if (format == null)
+        {
+            StringBuilder buf = new StringBuilder();
+            String addressPlaceholder = String.format("%%-%ds  ", maxAddressLength);
+            buf.append("%s%s  ");                         // status
+            buf.append(addressPlaceholder);               // address
+            buf.append("%-9s  ");                         // load
+            if (!isTokenPerNode)
+                buf.append("%-11s  ");                     // "Tokens"
+            if (hasEffectiveOwns)
+                buf.append("%-16s  ");                    // "Owns (effective)"
+            else
+                buf.append("%-6s  ");                     // "Owns
+            buf.append("%-36s  ");                        // Host ID
+            if (isTokenPerNode)
+                buf.append("%-39s  ");                    // token
+            buf.append("%s%n");                           // "Rack"
+
+            format = buf.toString();
+        }
+
+        return format;
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/StatusBackup.java b/src/java/org/apache/cassandra/tools/nodetool/StatusBackup.java
new file mode 100644
index 0000000..49a6750
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/StatusBackup.java
@@ -0,0 +1,36 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Command;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "statusbackup", description = "Status of incremental backup")
+public class StatusBackup extends NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        System.out.println(
+                probe.isIncrementalBackupsEnabled()
+                ? "running"
+                : "not running");
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/StatusBinary.java b/src/java/org/apache/cassandra/tools/nodetool/StatusBinary.java
new file mode 100644
index 0000000..d4fae14
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/StatusBinary.java
@@ -0,0 +1,36 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Command;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "statusbinary", description = "Status of native transport (binary protocol)")
+public class StatusBinary extends NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        System.out.println(
+                probe.isNativeTransportRunning()
+                ? "running"
+                : "not running");
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/StatusGossip.java b/src/java/org/apache/cassandra/tools/nodetool/StatusGossip.java
new file mode 100644
index 0000000..e40df8d
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/StatusGossip.java
@@ -0,0 +1,36 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Command;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "statusgossip", description = "Status of gossip")
+public class StatusGossip extends NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        System.out.println(
+                probe.isGossipRunning()
+                ? "running"
+                : "not running");
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/StatusHandoff.java b/src/java/org/apache/cassandra/tools/nodetool/StatusHandoff.java
new file mode 100644
index 0000000..5a00069
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/StatusHandoff.java
@@ -0,0 +1,36 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Command;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "statushandoff", description = "Status of storing future hints on the current node")
+public class StatusHandoff extends NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        System.out.println(
+                probe.isHandoffEnabled()
+                ? "running"
+                : "not running");
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/StatusThrift.java b/src/java/org/apache/cassandra/tools/nodetool/StatusThrift.java
new file mode 100644
index 0000000..0cb17d2
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/StatusThrift.java
@@ -0,0 +1,36 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Command;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "statusthrift", description = "Status of thrift server")
+public class StatusThrift extends NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        System.out.println(
+                probe.isThriftServerRunning()
+                ? "running"
+                : "not running");
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/Stop.java b/src/java/org/apache/cassandra/tools/nodetool/Stop.java
new file mode 100644
index 0000000..ad1fc27
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/Stop.java
@@ -0,0 +1,51 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Arguments;
+import io.airlift.command.Command;
+import io.airlift.command.Option;
+
+import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "stop", description = "Stop compaction")
+public class Stop extends NodeToolCmd
+{
+    @Arguments(title = "compaction_type",
+              usage = "<compaction type>",
+              description = "Supported types are COMPACTION, VALIDATION, CLEANUP, SCRUB, VERIFY, INDEX_BUILD",
+              required = false)
+    private OperationType compactionType = OperationType.UNKNOWN;
+
+    @Option(title = "compactionId",
+           name = {"-id", "--compaction-id"},
+           description = "Use -id to stop a compaction by the specified id. Ids can be found in the system.compactions_in_progress table.",
+           required = false)
+    private String compactionId = "";
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        if (!compactionId.isEmpty())
+            probe.stopById(compactionId);
+        else
+            probe.stop(compactionType.name());
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/StopDaemon.java b/src/java/org/apache/cassandra/tools/nodetool/StopDaemon.java
new file mode 100644
index 0000000..a0af89f
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/StopDaemon.java
@@ -0,0 +1,42 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Command;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+
+@Command(name = "stopdaemon", description = "Stop cassandra daemon")
+public class StopDaemon extends NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        try
+        {
+            probe.stopCassandraDaemon();
+        } catch (Exception e)
+        {
+            JVMStabilityInspector.inspectThrowable(e);
+            // ignored
+        }
+        System.out.println("Cassandra has shutdown.");
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/TableHistograms.java b/src/java/org/apache/cassandra/tools/nodetool/TableHistograms.java
new file mode 100644
index 0000000..207a74e
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/TableHistograms.java
@@ -0,0 +1,127 @@
+/*
+ * 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.tools.nodetool;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static java.lang.String.format;
+import io.airlift.command.Arguments;
+import io.airlift.command.Command;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.metrics.CassandraMetricsRegistry;
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+import org.apache.cassandra.utils.EstimatedHistogram;
+import org.apache.commons.lang3.ArrayUtils;
+
+@Command(name = "tablehistograms", description = "Print statistic histograms for a given table")
+public class TableHistograms extends NodeToolCmd
+{
+    @Arguments(usage = "<keyspace> <table>", description = "The keyspace and table name")
+    private List<String> args = new ArrayList<>();
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        checkArgument(args.size() == 2, "tablehistograms requires keyspace and table name arguments");
+
+        String keyspace = args.get(0);
+        String table = args.get(1);
+
+        // calculate percentile of row size and column count
+        long[] estimatedRowSize = (long[]) probe.getColumnFamilyMetric(keyspace, table, "EstimatedRowSizeHistogram");
+        long[] estimatedColumnCount = (long[]) probe.getColumnFamilyMetric(keyspace, table, "EstimatedColumnCountHistogram");
+
+        // build arrays to store percentile values
+        double[] estimatedRowSizePercentiles = new double[7];
+        double[] estimatedColumnCountPercentiles = new double[7];
+        double[] offsetPercentiles = new double[]{0.5, 0.75, 0.95, 0.98, 0.99};
+
+        if (ArrayUtils.isEmpty(estimatedRowSize) || ArrayUtils.isEmpty(estimatedColumnCount))
+        {
+            System.err.println("No SSTables exists, unable to calculate 'Partition Size' and 'Cell Count' percentiles");
+
+            for (int i = 0; i < 7; i++)
+            {
+                estimatedRowSizePercentiles[i] = Double.NaN;
+                estimatedColumnCountPercentiles[i] = Double.NaN;
+            }
+        }
+        else
+        {
+            EstimatedHistogram rowSizeHist = new EstimatedHistogram(estimatedRowSize);
+            EstimatedHistogram columnCountHist = new EstimatedHistogram(estimatedColumnCount);
+
+            if (rowSizeHist.isOverflowed())
+            {
+                System.err.println(String.format("Row sizes are larger than %s, unable to calculate percentiles", rowSizeHist.getLargestBucketOffset()));
+                for (int i = 0; i < offsetPercentiles.length; i++)
+                        estimatedRowSizePercentiles[i] = Double.NaN;
+            }
+            else
+            {
+                for (int i = 0; i < offsetPercentiles.length; i++)
+                    estimatedRowSizePercentiles[i] = rowSizeHist.percentile(offsetPercentiles[i]);
+            }
+
+            if (columnCountHist.isOverflowed())
+            {
+                System.err.println(String.format("Column counts are larger than %s, unable to calculate percentiles", columnCountHist.getLargestBucketOffset()));
+                for (int i = 0; i < estimatedColumnCountPercentiles.length; i++)
+                    estimatedColumnCountPercentiles[i] = Double.NaN;
+            }
+            else
+            {
+                for (int i = 0; i < offsetPercentiles.length; i++)
+                    estimatedColumnCountPercentiles[i] = columnCountHist.percentile(offsetPercentiles[i]);
+            }
+
+            // min value
+            estimatedRowSizePercentiles[5] = rowSizeHist.min();
+            estimatedColumnCountPercentiles[5] = columnCountHist.min();
+            // max value
+            estimatedRowSizePercentiles[6] = rowSizeHist.max();
+            estimatedColumnCountPercentiles[6] = columnCountHist.max();
+        }
+
+        String[] percentiles = new String[]{"50%", "75%", "95%", "98%", "99%", "Min", "Max"};
+        double[] readLatency = probe.metricPercentilesAsArray((CassandraMetricsRegistry.JmxTimerMBean) probe.getColumnFamilyMetric(keyspace, table, "ReadLatency"));
+        double[] writeLatency = probe.metricPercentilesAsArray((CassandraMetricsRegistry.JmxTimerMBean) probe.getColumnFamilyMetric(keyspace, table, "WriteLatency"));
+        double[] sstablesPerRead = probe.metricPercentilesAsArray((CassandraMetricsRegistry.JmxHistogramMBean) probe.getColumnFamilyMetric(keyspace, table, "SSTablesPerReadHistogram"));
+
+        System.out.println(format("%s/%s histograms", keyspace, table));
+        System.out.println(format("%-10s%10s%18s%18s%18s%18s",
+                "Percentile", "SSTables", "Write Latency", "Read Latency", "Partition Size", "Cell Count"));
+        System.out.println(format("%-10s%10s%18s%18s%18s%18s",
+                "", "", "(micros)", "(micros)", "(bytes)", ""));
+
+        for (int i = 0; i < percentiles.length; i++)
+        {
+            System.out.println(format("%-10s%10.2f%18.2f%18.2f%18.0f%18.0f",
+                    percentiles[i],
+                    sstablesPerRead[i],
+                    writeLatency[i],
+                    readLatency[i],
+                    estimatedRowSizePercentiles[i],
+                    estimatedColumnCountPercentiles[i]));
+        }
+        System.out.println();
+    }
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/TableStats.java b/src/java/org/apache/cassandra/tools/nodetool/TableStats.java
new file mode 100644
index 0000000..a1d2038
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/TableStats.java
@@ -0,0 +1,317 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Arguments;
+import io.airlift.command.Command;
+import io.airlift.command.Option;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import javax.management.InstanceNotFoundException;
+
+import org.apache.cassandra.db.ColumnFamilyStoreMBean;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.metrics.CassandraMetricsRegistry;
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "tablestats", description = "Print statistics on tables")
+public class TableStats extends NodeToolCmd
+{
+    @Arguments(usage = "[<keyspace.table>...]", description = "List of tables (or keyspace) names")
+    private List<String> tableNames = new ArrayList<>();
+
+    @Option(name = "-i", description = "Ignore the list of tables and display the remaining tables")
+    private boolean ignore = false;
+
+    @Option(title = "human_readable",
+            name = {"-H", "--human-readable"},
+            description = "Display bytes in human readable form, i.e. KB, MB, GB, TB")
+    private boolean humanReadable = false;
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        TableStats.OptionFilter filter = new OptionFilter(ignore, tableNames);
+        Map<String, List<ColumnFamilyStoreMBean>> tableStoreMap = new HashMap<>();
+
+        // get a list of column family stores
+        Iterator<Map.Entry<String, ColumnFamilyStoreMBean>> tables = probe.getColumnFamilyStoreMBeanProxies();
+
+        while (tables.hasNext())
+        {
+            Map.Entry<String, ColumnFamilyStoreMBean> entry = tables.next();
+            String keyspaceName = entry.getKey();
+            ColumnFamilyStoreMBean tableProxy = entry.getValue();
+
+            if (!tableStoreMap.containsKey(keyspaceName) && filter.isColumnFamilyIncluded(entry.getKey(), tableProxy.getColumnFamilyName()))
+            {
+                List<ColumnFamilyStoreMBean> columnFamilies = new ArrayList<>();
+                columnFamilies.add(tableProxy);
+                tableStoreMap.put(keyspaceName, columnFamilies);
+            } else if (filter.isColumnFamilyIncluded(entry.getKey(), tableProxy.getColumnFamilyName()))
+            {
+                tableStoreMap.get(keyspaceName).add(tableProxy);
+            }
+        }
+
+        // make sure all specified keyspace and tables exist
+        filter.verifyKeyspaces(probe.getKeyspaces());
+        filter.verifyColumnFamilies();
+
+        // print out the table statistics
+        for (Map.Entry<String, List<ColumnFamilyStoreMBean>> entry : tableStoreMap.entrySet())
+        {
+            String keyspaceName = entry.getKey();
+            List<ColumnFamilyStoreMBean> columnFamilies = entry.getValue();
+            long keyspaceReadCount = 0;
+            long keyspaceWriteCount = 0;
+            int keyspacePendingFlushes = 0;
+            double keyspaceTotalReadTime = 0.0f;
+            double keyspaceTotalWriteTime = 0.0f;
+
+            System.out.println("Keyspace: " + keyspaceName);
+            for (ColumnFamilyStoreMBean table : columnFamilies)
+            {
+                String tableName = table.getColumnFamilyName();
+                long writeCount = ((CassandraMetricsRegistry.JmxTimerMBean) probe.getColumnFamilyMetric(keyspaceName, tableName, "WriteLatency")).getCount();
+                long readCount = ((CassandraMetricsRegistry.JmxTimerMBean) probe.getColumnFamilyMetric(keyspaceName, tableName, "ReadLatency")).getCount();
+
+                if (readCount > 0)
+                {
+                    keyspaceReadCount += readCount;
+                    keyspaceTotalReadTime += (long) probe.getColumnFamilyMetric(keyspaceName, tableName, "ReadTotalLatency");
+                }
+                if (writeCount > 0)
+                {
+                    keyspaceWriteCount += writeCount;
+                    keyspaceTotalWriteTime += (long) probe.getColumnFamilyMetric(keyspaceName, tableName, "WriteTotalLatency");
+                }
+                keyspacePendingFlushes += (long) probe.getColumnFamilyMetric(keyspaceName, tableName, "PendingFlushes");
+            }
+
+            double keyspaceReadLatency = keyspaceReadCount > 0
+                                         ? keyspaceTotalReadTime / keyspaceReadCount / 1000
+                                         : Double.NaN;
+            double keyspaceWriteLatency = keyspaceWriteCount > 0
+                                          ? keyspaceTotalWriteTime / keyspaceWriteCount / 1000
+                                          : Double.NaN;
+
+            System.out.println("\tRead Count: " + keyspaceReadCount);
+            System.out.println("\tRead Latency: " + String.format("%s", keyspaceReadLatency) + " ms.");
+            System.out.println("\tWrite Count: " + keyspaceWriteCount);
+            System.out.println("\tWrite Latency: " + String.format("%s", keyspaceWriteLatency) + " ms.");
+            System.out.println("\tPending Flushes: " + keyspacePendingFlushes);
+
+            // print out column family statistics for this keyspace
+            for (ColumnFamilyStoreMBean table : columnFamilies)
+            {
+                String tableName = table.getColumnFamilyName();
+                if (tableName.contains("."))
+                    System.out.println("\t\tTable (index): " + tableName);
+                else
+                    System.out.println("\t\tTable: " + tableName);
+
+                System.out.println("\t\tSSTable count: " + probe.getColumnFamilyMetric(keyspaceName, tableName, "LiveSSTableCount"));
+
+                int[] leveledSStables = table.getSSTableCountPerLevel();
+                if (leveledSStables != null)
+                {
+                    System.out.print("\t\tSSTables in each level: [");
+                    for (int level = 0; level < leveledSStables.length; level++)
+                    {
+                        int count = leveledSStables[level];
+                        System.out.print(count);
+                        long maxCount = 4L; // for L0
+                        if (level > 0)
+                            maxCount = (long) Math.pow(10, level);
+                        //  show max threshold for level when exceeded
+                        if (count > maxCount)
+                            System.out.print("/" + maxCount);
+
+                        if (level < leveledSStables.length - 1)
+                            System.out.print(", ");
+                        else
+                            System.out.println("]");
+                    }
+                }
+
+                Long memtableOffHeapSize = null;
+                Long bloomFilterOffHeapSize = null;
+                Long indexSummaryOffHeapSize = null;
+                Long compressionMetadataOffHeapSize = null;
+
+                Long offHeapSize = null;
+
+                try
+                {
+                    memtableOffHeapSize = (Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "MemtableOffHeapSize");
+                    bloomFilterOffHeapSize = (Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "BloomFilterOffHeapMemoryUsed");
+                    indexSummaryOffHeapSize = (Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "IndexSummaryOffHeapMemoryUsed");
+                    compressionMetadataOffHeapSize = (Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "CompressionMetadataOffHeapMemoryUsed");
+
+                    offHeapSize = memtableOffHeapSize + bloomFilterOffHeapSize + indexSummaryOffHeapSize + compressionMetadataOffHeapSize;
+                }
+                catch (RuntimeException e)
+                {
+                    // offheap-metrics introduced in 2.1.3 - older versions do not have the appropriate mbeans
+                    if (!(e.getCause() instanceof InstanceNotFoundException))
+                        throw e;
+                }
+
+                System.out.println("\t\tSpace used (live): " + format((Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "LiveDiskSpaceUsed"), humanReadable));
+                System.out.println("\t\tSpace used (total): " + format((Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "TotalDiskSpaceUsed"), humanReadable));
+                System.out.println("\t\tSpace used by snapshots (total): " + format((Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "SnapshotsSize"), humanReadable));
+                if (offHeapSize != null)
+                    System.out.println("\t\tOff heap memory used (total): " + format(offHeapSize, humanReadable));
+                System.out.println("\t\tSSTable Compression Ratio: " + probe.getColumnFamilyMetric(keyspaceName, tableName, "CompressionRatio"));
+
+                Object estimatedRowCount = probe.getColumnFamilyMetric(keyspaceName, tableName, "EstimatedRowCount");
+                if (Long.valueOf(-1L).equals(estimatedRowCount))
+                {
+                    estimatedRowCount = 0L;
+                }
+
+                System.out.println("\t\tNumber of keys (estimate): " + estimatedRowCount);
+
+                System.out.println("\t\tMemtable cell count: " + probe.getColumnFamilyMetric(keyspaceName, tableName, "MemtableColumnsCount"));
+                System.out.println("\t\tMemtable data size: " + format((Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "MemtableLiveDataSize"), humanReadable));
+                if (memtableOffHeapSize != null)
+                    System.out.println("\t\tMemtable off heap memory used: " + format(memtableOffHeapSize, humanReadable));
+                System.out.println("\t\tMemtable switch count: " + probe.getColumnFamilyMetric(keyspaceName, tableName, "MemtableSwitchCount"));
+                System.out.println("\t\tLocal read count: " + ((CassandraMetricsRegistry.JmxTimerMBean) probe.getColumnFamilyMetric(keyspaceName, tableName, "ReadLatency")).getCount());
+                double localReadLatency = ((CassandraMetricsRegistry.JmxTimerMBean) probe.getColumnFamilyMetric(keyspaceName, tableName, "ReadLatency")).getMean() / 1000;
+                double localRLatency = localReadLatency > 0 ? localReadLatency : Double.NaN;
+                System.out.printf("\t\tLocal read latency: %01.3f ms%n", localRLatency);
+                System.out.println("\t\tLocal write count: " + ((CassandraMetricsRegistry.JmxTimerMBean) probe.getColumnFamilyMetric(keyspaceName, tableName, "WriteLatency")).getCount());
+                double localWriteLatency = ((CassandraMetricsRegistry.JmxTimerMBean) probe.getColumnFamilyMetric(keyspaceName, tableName, "WriteLatency")).getMean() / 1000;
+                double localWLatency = localWriteLatency > 0 ? localWriteLatency : Double.NaN;
+                System.out.printf("\t\tLocal write latency: %01.3f ms%n", localWLatency);
+                System.out.println("\t\tPending flushes: " + probe.getColumnFamilyMetric(keyspaceName, tableName, "PendingFlushes"));
+                System.out.println("\t\tBloom filter false positives: " + probe.getColumnFamilyMetric(keyspaceName, tableName, "BloomFilterFalsePositives"));
+                System.out.printf("\t\tBloom filter false ratio: %s%n", String.format("%01.5f", probe.getColumnFamilyMetric(keyspaceName, tableName, "RecentBloomFilterFalseRatio")));
+                System.out.println("\t\tBloom filter space used: " + format((Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "BloomFilterDiskSpaceUsed"), humanReadable));
+                if (bloomFilterOffHeapSize != null)
+                    System.out.println("\t\tBloom filter off heap memory used: " + format(bloomFilterOffHeapSize, humanReadable));
+                if (indexSummaryOffHeapSize != null)
+                    System.out.println("\t\tIndex summary off heap memory used: " + format(indexSummaryOffHeapSize, humanReadable));
+                if (compressionMetadataOffHeapSize != null)
+                    System.out.println("\t\tCompression metadata off heap memory used: " + format(compressionMetadataOffHeapSize, humanReadable));
+
+                System.out.println("\t\tCompacted partition minimum bytes: " + format((Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "MinRowSize"), humanReadable));
+                System.out.println("\t\tCompacted partition maximum bytes: " + format((Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "MaxRowSize"), humanReadable));
+                System.out.println("\t\tCompacted partition mean bytes: " + format((Long) probe.getColumnFamilyMetric(keyspaceName, tableName, "MeanRowSize"), humanReadable));
+                CassandraMetricsRegistry.JmxHistogramMBean histogram = (CassandraMetricsRegistry.JmxHistogramMBean) probe.getColumnFamilyMetric(keyspaceName, tableName, "LiveScannedHistogram");
+                System.out.println("\t\tAverage live cells per slice (last five minutes): " + histogram.getMean());
+                System.out.println("\t\tMaximum live cells per slice (last five minutes): " + histogram.getMax());
+                histogram = (CassandraMetricsRegistry.JmxHistogramMBean) probe.getColumnFamilyMetric(keyspaceName, tableName, "TombstoneScannedHistogram");
+                System.out.println("\t\tAverage tombstones per slice (last five minutes): " + histogram.getMean());
+                System.out.println("\t\tMaximum tombstones per slice (last five minutes): " + histogram.getMax());
+
+                System.out.println("");
+            }
+            System.out.println("----------------");
+        }
+    }
+
+    private String format(long bytes, boolean humanReadable) {
+        return humanReadable ? FileUtils.stringifyFileSize(bytes) : Long.toString(bytes);
+    }
+
+    /**
+     * Used for filtering keyspaces and tables to be displayed using the tablestats command.
+     */
+    private static class OptionFilter
+    {
+        private Map<String, List<String>> filter = new HashMap<>();
+        private Map<String, List<String>> verifier = new HashMap<>();
+        private List<String> filterList = new ArrayList<>();
+        private boolean ignoreMode;
+
+        public OptionFilter(boolean ignoreMode, List<String> filterList)
+        {
+            this.filterList.addAll(filterList);
+            this.ignoreMode = ignoreMode;
+
+            for (String s : filterList)
+            {
+                String[] keyValues = s.split("\\.", 2);
+
+                // build the map that stores the keyspaces and tables to use
+                if (!filter.containsKey(keyValues[0]))
+                {
+                    filter.put(keyValues[0], new ArrayList<String>());
+                    verifier.put(keyValues[0], new ArrayList<String>());
+
+                    if (keyValues.length == 2)
+                    {
+                        filter.get(keyValues[0]).add(keyValues[1]);
+                        verifier.get(keyValues[0]).add(keyValues[1]);
+                    }
+                } else
+                {
+                    if (keyValues.length == 2)
+                    {
+                        filter.get(keyValues[0]).add(keyValues[1]);
+                        verifier.get(keyValues[0]).add(keyValues[1]);
+                    }
+                }
+            }
+        }
+
+        public boolean isColumnFamilyIncluded(String keyspace, String columnFamily)
+        {
+            // supplying empty params list is treated as wanting to display all keyspaces and tables
+            if (filterList.isEmpty())
+                return !ignoreMode;
+
+            List<String> tables = filter.get(keyspace);
+
+            // no such keyspace is in the map
+            if (tables == null)
+                return ignoreMode;
+                // only a keyspace with no tables was supplied
+                // so ignore or include (based on the flag) every column family in specified keyspace
+            else if (tables.size() == 0)
+                return !ignoreMode;
+
+            // keyspace exists, and it contains specific table
+            verifier.get(keyspace).remove(columnFamily);
+            return ignoreMode ^ tables.contains(columnFamily);
+        }
+
+        public void verifyKeyspaces(List<String> keyspaces)
+        {
+            for (String ks : verifier.keySet())
+                if (!keyspaces.contains(ks))
+                    throw new IllegalArgumentException("Unknown keyspace: " + ks);
+        }
+
+        public void verifyColumnFamilies()
+        {
+            for (String ks : filter.keySet())
+                if (verifier.get(ks).size() > 0)
+                    throw new IllegalArgumentException("Unknown tables: " + verifier.get(ks) + " in keyspace: " + ks);
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/TopPartitions.java b/src/java/org/apache/cassandra/tools/nodetool/TopPartitions.java
new file mode 100644
index 0000000..35e13ce
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/TopPartitions.java
@@ -0,0 +1,117 @@
+/*
+ * 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.tools.nodetool;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.commons.lang3.StringUtils.join;
+import io.airlift.command.Arguments;
+import io.airlift.command.Command;
+import io.airlift.command.Option;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import javax.management.openmbean.CompositeData;
+import javax.management.openmbean.OpenDataException;
+import javax.management.openmbean.TabularDataSupport;
+
+import org.apache.cassandra.metrics.ColumnFamilyMetrics;
+import org.apache.cassandra.metrics.ColumnFamilyMetrics.Sampler;
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Ordering;
+
+@Command(name = "toppartitions", description = "Sample and print the most active partitions for a given column family")
+public class TopPartitions extends NodeToolCmd
+{
+    @Arguments(usage = "<keyspace> <cfname> <duration>", description = "The keyspace, column family name, and duration in milliseconds")
+    private List<String> args = new ArrayList<>();
+    @Option(name = "-s", description = "Capacity of stream summary, closer to the actual cardinality of partitions will yield more accurate results (Default: 256)")
+    private int size = 256;
+    @Option(name = "-k", description = "Number of the top partitions to list (Default: 10)")
+    private int topCount = 10;
+    @Option(name = "-a", description = "Comma separated list of samplers to use (Default: all)")
+    private String samplers = join(ColumnFamilyMetrics.Sampler.values(), ',');
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        checkArgument(args.size() == 3, "toppartitions requires keyspace, column family name, and duration");
+        checkArgument(topCount < size, "TopK count (-k) option must be smaller then the summary capacity (-s)");
+        String keyspace = args.get(0);
+        String cfname = args.get(1);
+        Integer duration = Integer.parseInt(args.get(2));
+        // generate the list of samplers
+        List<Sampler> targets = Lists.newArrayList();
+        for (String s : samplers.split(","))
+        {
+            try
+            {
+                targets.add(Sampler.valueOf(s.toUpperCase()));
+            } catch (Exception e)
+            {
+                throw new IllegalArgumentException(s + " is not a valid sampler, choose one of: " + join(Sampler.values(), ", "));
+            }
+        }
+
+        Map<Sampler, CompositeData> results;
+        try
+        {
+            results = probe.getPartitionSample(keyspace, cfname, size, duration, topCount, targets);
+        } catch (OpenDataException e)
+        {
+            throw new RuntimeException(e);
+        }
+        boolean first = true;
+        for(Entry<Sampler, CompositeData> result : results.entrySet())
+        {
+            CompositeData sampling = result.getValue();
+            // weird casting for http://bugs.sun.com/view_bug.do?bug_id=6548436
+            List<CompositeData> topk = (List<CompositeData>) (Object) Lists.newArrayList(((TabularDataSupport) sampling.get("partitions")).values());
+            Collections.sort(topk, new Ordering<CompositeData>()
+            {
+                public int compare(CompositeData left, CompositeData right)
+                {
+                    return Long.compare((long) right.get("count"), (long) left.get("count"));
+                }
+            });
+            if(!first)
+                System.out.println();
+            System.out.println(result.getKey().toString()+ " Sampler:");
+            System.out.printf("  Cardinality: ~%d (%d capacity)%n", (long) sampling.get("cardinality"), size);
+            System.out.printf("  Top %d partitions:%n", topCount);
+            if (topk.size() == 0)
+            {
+                System.out.println("\tNothing recorded during sampling period...");
+            } else
+            {
+                int offset = 0;
+                for (CompositeData entry : topk)
+                    offset = Math.max(offset, entry.get("string").toString().length());
+                System.out.printf("\t%-" + offset + "s%10s%10s%n", "Partition", "Count", "+/-");
+                for (CompositeData entry : topk)
+                    System.out.printf("\t%-" + offset + "s%10d%10d%n", entry.get("string").toString(), entry.get("count"), entry.get("error"));
+            }
+            first = false;
+        }
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/TpStats.java b/src/java/org/apache/cassandra/tools/nodetool/TpStats.java
new file mode 100644
index 0000000..f3448ab
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/TpStats.java
@@ -0,0 +1,54 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Command;
+
+import java.util.Map;
+
+import com.google.common.collect.Multimap;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "tpstats", description = "Print usage statistics of thread pools")
+public class TpStats extends NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        System.out.printf("%-25s%10s%10s%15s%10s%18s%n", "Pool Name", "Active", "Pending", "Completed", "Blocked", "All time blocked");
+
+        Multimap<String, String> threadPools = probe.getThreadPools();
+
+        for (Map.Entry<String, String> tpool : threadPools.entries())
+        {
+            System.out.printf("%-25s%10s%10s%15s%10s%18s%n",
+                              tpool.getValue(),
+                              probe.getThreadPoolMetric(tpool.getKey(), tpool.getValue(), "ActiveTasks"),
+                              probe.getThreadPoolMetric(tpool.getKey(), tpool.getValue(), "PendingTasks"),
+                              probe.getThreadPoolMetric(tpool.getKey(), tpool.getValue(), "CompletedTasks"),
+                              probe.getThreadPoolMetric(tpool.getKey(), tpool.getValue(), "CurrentlyBlockedTasks"),
+                              probe.getThreadPoolMetric(tpool.getKey(), tpool.getValue(), "TotalBlockedTasks"));
+        }
+
+        System.out.printf("%n%-20s%10s%n", "Message type", "Dropped");
+        for (Map.Entry<String, Integer> entry : probe.getDroppedMessages().entrySet())
+            System.out.printf("%-20s%10s%n", entry.getKey(), entry.getValue());
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/TruncateHints.java b/src/java/org/apache/cassandra/tools/nodetool/TruncateHints.java
new file mode 100644
index 0000000..bcd554f
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/TruncateHints.java
@@ -0,0 +1,41 @@
+/*
+ * 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.tools.nodetool;
+
+import static org.apache.commons.lang3.StringUtils.EMPTY;
+import io.airlift.command.Arguments;
+import io.airlift.command.Command;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "truncatehints", description = "Truncate all hints on the local node, or truncate hints for the endpoint(s) specified.")
+public class TruncateHints extends NodeToolCmd
+{
+    @Arguments(usage = "[endpoint ... ]", description = "Endpoint address(es) to delete hints for, either ip address (\"127.0.0.1\") or hostname")
+    private String endpoint = EMPTY;
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        if (endpoint.isEmpty())
+            probe.truncateHints();
+        else
+            probe.truncateHints(endpoint);
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/UpgradeSSTable.java b/src/java/org/apache/cassandra/tools/nodetool/UpgradeSSTable.java
new file mode 100644
index 0000000..fcb1ab2
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/UpgradeSSTable.java
@@ -0,0 +1,61 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Arguments;
+import io.airlift.command.Command;
+import io.airlift.command.Option;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "upgradesstables", description = "Rewrite sstables (for the requested tables) that are not on the current version (thus upgrading them to said current version)")
+public class UpgradeSSTable extends NodeToolCmd
+{
+    @Arguments(usage = "[<keyspace> <tables>...]", description = "The keyspace followed by one or many tables")
+    private List<String> args = new ArrayList<>();
+
+    @Option(title = "include_all", name = {"-a", "--include-all-sstables"}, description = "Use -a to include all sstables, even those already on the current version")
+    private boolean includeAll = false;
+
+    @Option(title = "jobs",
+            name = {"-j", "--jobs"},
+            description = "Number of sstables to upgrade simultanously, set to 0 to use all available compaction threads")
+    private int jobs = 2;
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        List<String> keyspaces = parseOptionalKeyspace(args, probe, true);
+        String[] cfnames = parseOptionalColumnFamilies(args);
+
+        for (String keyspace : keyspaces)
+        {
+            try
+            {
+                probe.upgradeSSTables(System.out, keyspace, !includeAll, jobs, cfnames);
+            } catch (Exception e)
+            {
+                throw new RuntimeException("Error occurred during enabling auto-compaction", e);
+            }
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/tools/nodetool/Verify.java b/src/java/org/apache/cassandra/tools/nodetool/Verify.java
new file mode 100644
index 0000000..813b761
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/Verify.java
@@ -0,0 +1,58 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Arguments;
+import io.airlift.command.Command;
+import io.airlift.command.Option;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "verify", description = "Verify (check data checksum for) one or more tables")
+public class Verify extends NodeToolCmd
+{
+    @Arguments(usage = "[<keyspace> <tables>...]", description = "The keyspace followed by one or many tables")
+    private List<String> args = new ArrayList<>();
+
+    @Option(title = "extended_verify",
+        name = {"-e", "--extended-verify"},
+        description = "Verify each cell data, beyond simply checking sstable checksums")
+    private boolean extendedVerify = false;
+
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        List<String> keyspaces = parseOptionalKeyspace(args, probe);
+        String[] cfnames = parseOptionalColumnFamilies(args);
+
+        for (String keyspace : keyspaces)
+        {
+            try
+            {
+                probe.verify(System.out, extendedVerify, keyspace, cfnames);
+            } catch (Exception e)
+            {
+                throw new RuntimeException("Error occurred during verifying", e);
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tools/nodetool/Version.java b/src/java/org/apache/cassandra/tools/nodetool/Version.java
new file mode 100644
index 0000000..2495508
--- /dev/null
+++ b/src/java/org/apache/cassandra/tools/nodetool/Version.java
@@ -0,0 +1,33 @@
+/*
+ * 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.tools.nodetool;
+
+import io.airlift.command.Command;
+
+import org.apache.cassandra.tools.NodeProbe;
+import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
+
+@Command(name = "version", description = "Print cassandra version")
+public class Version extends NodeToolCmd
+{
+    @Override
+    public void execute(NodeProbe probe)
+    {
+        System.out.println("ReleaseVersion: " + probe.getReleaseVersion());
+    }
+}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/tracing/ExpiredTraceState.java b/src/java/org/apache/cassandra/tracing/ExpiredTraceState.java
index 37a013b..5cc3c21 100644
--- a/src/java/org/apache/cassandra/tracing/ExpiredTraceState.java
+++ b/src/java/org/apache/cassandra/tracing/ExpiredTraceState.java
@@ -27,9 +27,9 @@
 
 public class ExpiredTraceState extends TraceState
 {
-    public ExpiredTraceState(UUID sessionId)
+    public ExpiredTraceState(UUID sessionId, Tracing.TraceType traceType)
     {
-        super(FBUtilities.getBroadcastAddress(), sessionId);
+        super(FBUtilities.getBroadcastAddress(), sessionId, traceType);
     }
 
     public int elapsed()
diff --git a/src/java/org/apache/cassandra/tracing/TraceKeyspace.java b/src/java/org/apache/cassandra/tracing/TraceKeyspace.java
new file mode 100644
index 0000000..f66269d
--- /dev/null
+++ b/src/java/org/apache/cassandra/tracing/TraceKeyspace.java
@@ -0,0 +1,128 @@
+/*
+ * 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.tracing;
+
+import java.net.InetAddress;
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.db.CFRowAdder;
+import org.apache.cassandra.db.ColumnFamily;
+import org.apache.cassandra.db.Mutation;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.UUIDGen;
+
+public final class TraceKeyspace
+{
+    public static final String NAME = "system_traces";
+
+    public static final String SESSIONS = "sessions";
+    public static final String EVENTS = "events";
+
+    private static final CFMetaData Sessions =
+        compile(SESSIONS,
+                "tracing sessions",
+                "CREATE TABLE %s ("
+                + "session_id uuid,"
+                + "command text,"
+                + "client inet,"
+                + "coordinator inet,"
+                + "duration int,"
+                + "parameters map<text, text>,"
+                + "request text,"
+                + "started_at timestamp,"
+                + "PRIMARY KEY ((session_id)))");
+
+    private static final CFMetaData Events =
+        compile(EVENTS,
+                "tracing events",
+                "CREATE TABLE %s ("
+                + "session_id uuid,"
+                + "event_id timeuuid,"
+                + "activity text,"
+                + "source inet,"
+                + "source_elapsed int,"
+                + "thread text,"
+                + "PRIMARY KEY ((session_id), event_id))");
+
+    private static CFMetaData compile(String name, String description, String schema)
+    {
+        return CFMetaData.compile(String.format(schema, name), NAME)
+                         .comment(description);
+    }
+
+    public static KSMetaData definition()
+    {
+        List<CFMetaData> tables = Arrays.asList(Sessions, Events);
+        return new KSMetaData(NAME, SimpleStrategy.class, ImmutableMap.of("replication_factor", "2"), true, tables);
+    }
+
+    static Mutation makeStartSessionMutation(ByteBuffer sessionId,
+                                             InetAddress client,
+                                             Map<String, String> parameters,
+                                             String request,
+                                             long startedAt,
+                                             String command,
+                                             int ttl)
+    {
+        Mutation mutation = new Mutation(NAME, sessionId);
+        ColumnFamily cells = mutation.addOrGet(TraceKeyspace.Sessions);
+
+        CFRowAdder adder = new CFRowAdder(cells, cells.metadata().comparator.builder().build(), FBUtilities.timestampMicros(), ttl);
+        adder.add("client", client)
+             .add("coordinator", FBUtilities.getBroadcastAddress())
+             .add("request", request)
+             .add("started_at", new Date(startedAt))
+             .add("command", command);
+        for (Map.Entry<String, String> entry : parameters.entrySet())
+            adder.addMapEntry("parameters", entry.getKey(), entry.getValue());
+
+        return mutation;
+    }
+
+    static Mutation makeStopSessionMutation(ByteBuffer sessionId, int elapsed, int ttl)
+    {
+        Mutation mutation = new Mutation(NAME, sessionId);
+        ColumnFamily cells = mutation.addOrGet(Sessions);
+
+        CFRowAdder adder = new CFRowAdder(cells, cells.metadata().comparator.builder().build(), FBUtilities.timestampMicros(), ttl);
+        adder.add("duration", elapsed);
+
+        return mutation;
+    }
+
+    static Mutation makeEventMutation(ByteBuffer sessionId, String message, int elapsed, String threadName, int ttl)
+    {
+        Mutation mutation = new Mutation(NAME, sessionId);
+        ColumnFamily cells = mutation.addOrGet(Events);
+
+        CFRowAdder adder = new CFRowAdder(cells, cells.metadata().comparator.make(UUIDGen.getTimeUUID()), FBUtilities.timestampMicros(), ttl);
+        adder.add("activity", message)
+             .add("source", FBUtilities.getBroadcastAddress())
+             .add("thread", threadName);
+        if (elapsed >= 0)
+            adder.add("source_elapsed", elapsed);
+
+        return mutation;
+    }
+}
diff --git a/src/java/org/apache/cassandra/tracing/TraceState.java b/src/java/org/apache/cassandra/tracing/TraceState.java
index 399b6e9..88f36d8 100644
--- a/src/java/org/apache/cassandra/tracing/TraceState.java
+++ b/src/java/org/apache/cassandra/tracing/TraceState.java
@@ -19,40 +19,66 @@
 
 import java.net.InetAddress;
 import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
 import java.util.UUID;
+import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import com.google.common.base.Stopwatch;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.slf4j.helpers.MessageFormatter;
 
 import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.concurrent.StageManager;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.db.CFRowAdder;
-import org.apache.cassandra.db.ColumnFamily;
+import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.Mutation;
+import org.apache.cassandra.exceptions.OverloadedException;
+import org.apache.cassandra.service.StorageProxy;
 import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.UUIDGen;
+import org.apache.cassandra.utils.JVMStabilityInspector;
 import org.apache.cassandra.utils.WrappedRunnable;
+import org.apache.cassandra.utils.progress.ProgressEvent;
+import org.apache.cassandra.utils.progress.ProgressEventNotifier;
+import org.apache.cassandra.utils.progress.ProgressListener;
 
 /**
  * ThreadLocal state for a tracing session. The presence of an instance of this class as a ThreadLocal denotes that an
  * operation is being traced.
  */
-public class TraceState
+public class TraceState implements ProgressEventNotifier
 {
+    private static final Logger logger = LoggerFactory.getLogger(TraceState.class);
+    private static final int WAIT_FOR_PENDING_EVENTS_TIMEOUT_SECS =
+    Integer.valueOf(System.getProperty("cassandra.wait_for_tracing_events_timeout_secs", "0"));
+
     public final UUID sessionId;
     public final InetAddress coordinator;
     public final Stopwatch watch;
     public final ByteBuffer sessionIdBytes;
+    public final Tracing.TraceType traceType;
+    public final int ttl;
+
+    private boolean notify;
+    private final List<ProgressListener> listeners = new CopyOnWriteArrayList<>();
+    private String tag;
+
+    public enum Status
+    {
+        IDLE,
+        ACTIVE,
+        STOPPED
+    }
+
+    private volatile Status status;
 
     // Multiple requests can use the same TraceState at a time, so we need to reference count.
     // See CASSANDRA-7626 for more details.
     private final AtomicInteger references = new AtomicInteger(1);
 
-    public TraceState(InetAddress coordinator, UUID sessionId)
+    public TraceState(InetAddress coordinator, UUID sessionId, Tracing.TraceType traceType)
     {
         assert coordinator != null;
         assert sessionId != null;
@@ -60,7 +86,36 @@
         this.coordinator = coordinator;
         this.sessionId = sessionId;
         sessionIdBytes = ByteBufferUtil.bytes(sessionId);
+        this.traceType = traceType;
+        this.ttl = traceType.getTTL();
         watch = Stopwatch.createStarted();
+        this.status = Status.IDLE;
+}
+
+    /**
+     * Activate notification with provided {@code tag} name.
+     *
+     * @param tag Tag name to add when emitting notification
+     */
+    public void enableActivityNotification(String tag)
+    {
+        assert traceType == Tracing.TraceType.REPAIR;
+        notify = true;
+        this.tag = tag;
+    }
+
+    @Override
+    public void addProgressListener(ProgressListener listener)
+    {
+        assert traceType == Tracing.TraceType.REPAIR;
+        listeners.add(listener);
+    }
+
+    @Override
+    public void removeProgressListener(ProgressListener listener)
+    {
+        assert traceType == Tracing.TraceType.REPAIR;
+        listeners.remove(listener);
     }
 
     public int elapsed()
@@ -69,6 +124,48 @@
         return elapsed < Integer.MAX_VALUE ? (int) elapsed : Integer.MAX_VALUE;
     }
 
+    public synchronized void stop()
+    {
+        waitForPendingEvents();
+
+        status = Status.STOPPED;
+        notifyAll();
+    }
+
+    /*
+     * Returns immediately if there has been trace activity since the last
+     * call, otherwise waits until there is trace activity, or until the
+     * timeout expires.
+     * @param timeout timeout in milliseconds
+     * @return activity status
+     */
+    public synchronized Status waitActivity(long timeout)
+    {
+        if (status == Status.IDLE)
+        {
+            try
+            {
+                wait(timeout);
+            }
+            catch (InterruptedException e)
+            {
+                throw new RuntimeException();
+            }
+        }
+        if (status == Status.ACTIVE)
+        {
+            status = Status.IDLE;
+            return Status.ACTIVE;
+        }
+        return status;
+    }
+
+    private synchronized void notifyActivity()
+    {
+        status = Status.ACTIVE;
+        notifyAll();
+    }
+
     public void trace(String format, Object arg)
     {
         trace(MessageFormatter.format(format, arg).getMessage());
@@ -86,10 +183,38 @@
 
     public void trace(String message)
     {
-        TraceState.trace(sessionIdBytes, message, elapsed());
+        if (notify)
+            notifyActivity();
+
+        final String threadName = Thread.currentThread().getName();
+        final int elapsed = elapsed();
+
+        executeMutation(TraceKeyspace.makeEventMutation(sessionIdBytes, message, elapsed, threadName, ttl));
+        if (logger.isTraceEnabled())
+            logger.trace("Adding <{}> to trace events", message);
+
+        for (ProgressListener listener : listeners)
+        {
+            listener.progress(tag, ProgressEvent.createNotification(message));
+        }
     }
 
-    public static void trace(final ByteBuffer sessionIdBytes, final String message, final int elapsed)
+    static void executeMutation(final Mutation mutation)
+    {
+        StageManager.getStage(Stage.TRACING).execute(new WrappedRunnable()
+        {
+            protected void runMayThrow() throws Exception
+            {
+                mutateWithCatch(mutation);
+            }
+        });
+    }
+
+    /**
+     * Called from {@link org.apache.cassandra.net.OutboundTcpConnection} for non-local traces (traces
+     * that are not initiated by local node == coordinator).
+     */
+    public static void mutateWithTracing(final ByteBuffer sessionId, final String message, final int elapsed, final int ttl)
     {
         final String threadName = Thread.currentThread().getName();
 
@@ -97,21 +222,50 @@
         {
             public void runMayThrow()
             {
-                Mutation mutation = new Mutation(Tracing.TRACE_KS, sessionIdBytes);
-                ColumnFamily cells = mutation.addOrGet(CFMetaData.TraceEventsCf);
-
-                CFRowAdder adder = new CFRowAdder(cells, cells.metadata().comparator.make(UUIDGen.getTimeUUID()), FBUtilities.timestampMicros());
-                adder.add("activity", message);
-                adder.add("source", FBUtilities.getBroadcastAddress());
-                if (elapsed >= 0)
-                    adder.add("source_elapsed", elapsed);
-                adder.add("thread", threadName);
-
-                Tracing.mutateWithCatch(mutation);
+                mutateWithCatch(TraceKeyspace.makeEventMutation(sessionId, message, elapsed, threadName, ttl));
             }
         });
     }
 
+    static void mutateWithCatch(Mutation mutation)
+    {
+        try
+        {
+            StorageProxy.mutate(Collections.singletonList(mutation), ConsistencyLevel.ANY);
+        }
+        catch (OverloadedException e)
+        {
+            Tracing.logger.warn("Too many nodes are overloaded to save trace events");
+        }
+    }
+
+    /**
+     * Post a no-op event to the TRACING stage, so that we can be sure that any previous mutations
+     * have at least been applied to one replica. This works because the tracking executor only
+     * has one thread in its pool, see {@link StageManager#tracingExecutor()}.
+     */
+    protected void waitForPendingEvents()
+    {
+        if (WAIT_FOR_PENDING_EVENTS_TIMEOUT_SECS <= 0)
+            return;
+
+        try
+        {
+            if (logger.isTraceEnabled())
+                logger.trace("Waiting for up to {} seconds for trace events to complete",
+                             +WAIT_FOR_PENDING_EVENTS_TIMEOUT_SECS);
+
+            StageManager.getStage(Stage.TRACING).submit(StageManager.NO_OP_TASK)
+                        .get(WAIT_FOR_PENDING_EVENTS_TIMEOUT_SECS, TimeUnit.SECONDS);
+        }
+        catch (Throwable t)
+        {
+            JVMStabilityInspector.inspectThrowable(t);
+            logger.debug("Failed to wait for tracing events to complete: {}", t);
+        }
+    }
+
+
     public boolean acquireReference()
     {
         while (true)
@@ -126,6 +280,7 @@
 
     public int releaseReference()
     {
+        waitForPendingEvents();
         return references.decrementAndGet();
     }
 }
diff --git a/src/java/org/apache/cassandra/tracing/Tracing.java b/src/java/org/apache/cassandra/tracing/Tracing.java
index d74859a..bf9cee7 100644
--- a/src/java/org/apache/cassandra/tracing/Tracing.java
+++ b/src/java/org/apache/cassandra/tracing/Tracing.java
@@ -21,8 +21,6 @@
 
 import java.net.InetAddress;
 import java.nio.ByteBuffer;
-import java.util.Arrays;
-import java.util.Date;
 import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
@@ -31,32 +29,55 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.concurrent.Stage;
-import org.apache.cassandra.concurrent.StageManager;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.db.*;
+import org.apache.cassandra.concurrent.ExecutorLocal;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.marshal.TimeUUIDType;
-import org.apache.cassandra.exceptions.OverloadedException;
-import org.apache.cassandra.exceptions.UnavailableException;
-import org.apache.cassandra.exceptions.WriteTimeoutException;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.service.StorageProxy;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.UUIDGen;
 
+
 /**
  * A trace session context. Able to track and store trace sessions. A session is usually a user initiated query, and may
  * have multiple local and remote events before it is completed. All events and sessions are stored at keyspace.
  */
-public class Tracing
+public class Tracing implements ExecutorLocal<TraceState>
 {
-    public static final String TRACE_KS = "system_traces";
-    public static final String EVENTS_CF = "events";
-    public static final String SESSIONS_CF = "sessions";
     public static final String TRACE_HEADER = "TraceSession";
+    public static final String TRACE_TYPE = "TraceType";
 
-    private static final Logger logger = LoggerFactory.getLogger(Tracing.class);
+    public enum TraceType
+    {
+        NONE,
+        QUERY,
+        REPAIR;
+
+        private static final TraceType[] ALL_VALUES = values();
+
+        public static TraceType deserialize(byte b)
+        {
+            if (b < 0 || ALL_VALUES.length <= b)
+                return NONE;
+            return ALL_VALUES[b];
+        }
+
+        public static byte serialize(TraceType value)
+        {
+            return (byte) value.ordinal();
+        }
+
+        private static final int[] TTLS = { DatabaseDescriptor.getTracetypeQueryTTL(),
+                                            DatabaseDescriptor.getTracetypeQueryTTL(),
+                                            DatabaseDescriptor.getTracetypeRepairTTL() };
+
+        public int getTTL()
+        {
+            return TTLS[ordinal()];
+        }
+    }
+
+    static final Logger logger = LoggerFactory.getLogger(Tracing.class);
 
     private final InetAddress localAddress = FBUtilities.getLocalAddress();
 
@@ -72,6 +93,18 @@
         return state.get().sessionId;
     }
 
+    public TraceType getTraceType()
+    {
+        assert isTracing();
+        return state.get().traceType;
+    }
+
+    public int getTTL()
+    {
+        assert isTracing();
+        return state.get().ttl;
+    }
+
     /**
      * Indicates if the current thread's execution is being traced.
      */
@@ -82,14 +115,24 @@
 
     public UUID newSession()
     {
-        return newSession(TimeUUIDType.instance.compose(ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes())));
+        return newSession(TraceType.QUERY);
+    }
+
+    public UUID newSession(TraceType traceType)
+    {
+        return newSession(TimeUUIDType.instance.compose(ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes())), traceType);
     }
 
     public UUID newSession(UUID sessionId)
     {
+        return newSession(sessionId, TraceType.QUERY);
+    }
+
+    private UUID newSession(UUID sessionId, TraceType traceType)
+    {
         assert state.get() == null;
 
-        TraceState ts = new TraceState(localAddress, sessionId);
+        TraceState ts = new TraceState(localAddress, sessionId, traceType);
         state.set(ts);
         sessions.put(sessionId, ts);
 
@@ -110,27 +153,17 @@
         TraceState state = this.state.get();
         if (state == null) // inline isTracing to avoid implicit two calls to state.get()
         {
-            logger.debug("request complete");
+            logger.trace("request complete");
         }
         else
         {
             final int elapsed = state.elapsed();
-            final ByteBuffer sessionIdBytes = state.sessionIdBytes;
+            final ByteBuffer sessionId = state.sessionIdBytes;
+            final int ttl = state.ttl;
 
-            StageManager.getStage(Stage.TRACING).execute(new Runnable()
-            {
-                public void run()
-                {
-                    Mutation mutation = new Mutation(TRACE_KS, sessionIdBytes);
-                    ColumnFamily cells = mutation.addOrGet(CFMetaData.TraceSessionsCf);
+            TraceState.executeMutation(TraceKeyspace.makeStopSessionMutation(sessionId, elapsed, ttl));
 
-                    CFRowAdder adder = new CFRowAdder(cells, cells.metadata().comparator.builder().build(), FBUtilities.timestampMicros());
-                    adder.add("duration", elapsed);
-
-                    mutateWithCatch(mutation);
-                }
-            });
-
+            state.stop();
             sessions.remove(state.sessionId);
             this.state.set(null);
         }
@@ -151,30 +184,24 @@
         state.set(tls);
     }
 
-    public void begin(final String request, final Map<String, String> parameters)
+    public TraceState begin(final String request, final Map<String, String> parameters)
+    {
+        return begin(request, null, parameters);
+    }
+
+    public TraceState begin(final String request, final InetAddress client, final Map<String, String> parameters)
     {
         assert isTracing();
 
-        final long started_at = System.currentTimeMillis();
-        final ByteBuffer sessionIdBytes = state.get().sessionIdBytes;
+        final TraceState state = this.state.get();
+        final long startedAt = System.currentTimeMillis();
+        final ByteBuffer sessionId = state.sessionIdBytes;
+        final String command = state.traceType.toString();
+        final int ttl = state.ttl;
 
-        StageManager.getStage(Stage.TRACING).execute(new Runnable()
-        {
-            public void run()
-            {
-                Mutation mutation = new Mutation(TRACE_KS, sessionIdBytes);
-                ColumnFamily cells = mutation.addOrGet(CFMetaData.TraceSessionsCf);
+        TraceState.executeMutation(TraceKeyspace.makeStartSessionMutation(sessionId, client, parameters, request, startedAt, command, ttl));
 
-                CFRowAdder adder = new CFRowAdder(cells, cells.metadata().comparator.builder().build(), FBUtilities.timestampMicros());
-                adder.add("coordinator", FBUtilities.getBroadcastAddress());
-                for (Map.Entry<String, String> entry : parameters.entrySet())
-                    adder.addMapEntry("parameters", entry.getKey(), entry.getValue());
-                adder.add("request", request);
-                adder.add("started_at", new Date(started_at));
-
-                mutateWithCatch(mutation);
-            }
-        });
+        return state;
     }
 
     /**
@@ -184,7 +211,7 @@
      */
     public TraceState initializeFromMessage(final MessageIn<?> message)
     {
-        final byte[] sessionBytes = message.parameters.get(Tracing.TRACE_HEADER);
+        final byte[] sessionBytes = message.parameters.get(TRACE_HEADER);
 
         if (sessionBytes == null)
             return null;
@@ -195,19 +222,36 @@
         if (ts != null && ts.acquireReference())
             return ts;
 
+        byte[] tmpBytes;
+        TraceType traceType = TraceType.QUERY;
+        if ((tmpBytes = message.parameters.get(TRACE_TYPE)) != null)
+            traceType = TraceType.deserialize(tmpBytes[0]);
+
         if (message.verb == MessagingService.Verb.REQUEST_RESPONSE)
         {
             // received a message for a session we've already closed out.  see CASSANDRA-5668
-            return new ExpiredTraceState(sessionId);
+            return new ExpiredTraceState(sessionId, traceType);
         }
         else
         {
-            ts = new TraceState(message.from, sessionId);
+            ts = new TraceState(message.from, sessionId, traceType);
             sessions.put(sessionId, ts);
             return ts;
         }
     }
 
+
+    // repair just gets a varargs method since it's so heavyweight anyway
+    public static void traceRepair(String format, Object... args)
+    {
+        final TraceState state = instance.get();
+        if (state == null) // inline isTracing to avoid implicit two calls to state.get()
+            return;
+
+        state.trace(format, args);
+    }
+
+    // normal traces get zero-, one-, and two-argument overloads so common case doesn't need to create varargs array
     public static void trace(String message)
     {
         final TraceState state = instance.get();
@@ -235,7 +279,7 @@
         state.trace(format, arg1, arg2);
     }
 
-    public static void trace(String format, Object[] args)
+    public static void trace(String format, Object... args)
     {
         final TraceState state = instance.get();
         if (state == null) // inline isTracing to avoid implicit two calls to state.get()
@@ -243,21 +287,4 @@
 
         state.trace(format, args);
     }
-
-    static void mutateWithCatch(Mutation mutation)
-    {
-        try
-        {
-            StorageProxy.mutate(Arrays.asList(mutation), ConsistencyLevel.ANY);
-        }
-        catch (UnavailableException | WriteTimeoutException e)
-        {
-            // should never happen; ANY does not throw UAE or WTE
-            throw new AssertionError(e);
-        }
-        catch (OverloadedException e)
-        {
-            logger.warn("Too many nodes are overloaded to save trace events");
-        }
-    }
 }
diff --git a/src/java/org/apache/cassandra/transport/CBUtil.java b/src/java/org/apache/cassandra/transport/CBUtil.java
index f9425c3..92e2891 100644
--- a/src/java/org/apache/cassandra/transport/CBUtil.java
+++ b/src/java/org/apache/cassandra/transport/CBUtil.java
@@ -36,6 +36,7 @@
 import io.netty.buffer.*;
 import io.netty.util.CharsetUtil;
 
+import org.apache.cassandra.config.Config;
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.utils.Pair;
@@ -45,13 +46,14 @@
 /**
  * ByteBuf utility methods.
  * Note that contrarily to ByteBufferUtil, these method do "read" the
- * ByteBuf advancing it's (read) position. They also write by
+ * ByteBuf advancing its (read) position. They also write by
  * advancing the write position. Functions are also provided to create
  * ByteBuf while avoiding copies.
  */
 public abstract class CBUtil
 {
-    public static final ByteBufAllocator allocator = new PooledByteBufAllocator(true);
+    public static final boolean USE_HEAP_ALLOCATOR = Boolean.getBoolean(Config.PROPERTY_PREFIX + "netty_use_heap_allocator");
+    public static final ByteBufAllocator allocator = USE_HEAP_ALLOCATOR ? new UnpooledByteBufAllocator(false) : new PooledByteBufAllocator(true);
 
     private CBUtil() {}
 
@@ -91,7 +93,7 @@
         }
         catch (IndexOutOfBoundsException e)
         {
-            throw new ProtocolException("Not enough bytes to read an UTF8 serialized string preceded by it's 2 bytes length");
+            throw new ProtocolException("Not enough bytes to read an UTF8 serialized string preceded by its 2 bytes length");
         }
     }
 
@@ -139,7 +141,7 @@
         }
         catch (IndexOutOfBoundsException e)
         {
-            throw new ProtocolException("Not enough bytes to read an UTF8 serialized string preceded by it's 4 bytes length");
+            throw new ProtocolException("Not enough bytes to read an UTF8 serialized string preceded by its 4 bytes length");
         }
     }
 
@@ -166,7 +168,7 @@
         }
         catch (IndexOutOfBoundsException e)
         {
-            throw new ProtocolException("Not enough bytes to read a byte array preceded by it's 2 bytes length");
+            throw new ProtocolException("Not enough bytes to read a byte array preceded by its 2 bytes length");
         }
     }
 
@@ -181,6 +183,40 @@
         return 2 + bytes.length;
     }
 
+    public static Map<String, ByteBuffer> readBytesMap(ByteBuf cb)
+    {
+        int length = cb.readUnsignedShort();
+        Map<String, ByteBuffer> m = new HashMap<>(length);
+        for (int i = 0; i < length; i++)
+        {
+            String k = readString(cb);
+            ByteBuffer v = readValue(cb);
+            m.put(k, v);
+        }
+        return m;
+    }
+
+    public static void writeBytesMap(Map<String, ByteBuffer> m, ByteBuf cb)
+    {
+        cb.writeShort(m.size());
+        for (Map.Entry<String, ByteBuffer> entry : m.entrySet())
+        {
+            writeString(entry.getKey(), cb);
+            writeValue(entry.getValue(), cb);
+        }
+    }
+
+    public static int sizeOfBytesMap(Map<String, ByteBuffer> m)
+    {
+        int size = 2;
+        for (Map.Entry<String, ByteBuffer> entry : m.entrySet())
+        {
+            size += sizeOfString(entry.getKey());
+            size += sizeOfValue(entry.getValue());
+        }
+        return size;
+    }
+
     public static ConsistencyLevel readConsistencyLevel(ByteBuf cb)
     {
         return ConsistencyLevel.fromCode(cb.readUnsignedShort());
@@ -338,6 +374,25 @@
         return ByteBuffer.wrap(readRawBytes(slice));
     }
 
+    public static ByteBuffer readBoundValue(ByteBuf cb, int protocolVersion)
+    {
+        int length = cb.readInt();
+        if (length < 0)
+        {
+            if (protocolVersion < 4) // backward compatibility for pre-version 4
+                return null;
+            if (length == -1)
+                return null;
+            else if (length == -2)
+                return ByteBufferUtil.UNSET_BYTE_BUFFER;
+            else
+                throw new ProtocolException("Invalid ByteBuf length " + length);
+        }
+        ByteBuf slice = cb.readSlice(length);
+
+        return ByteBuffer.wrap(readRawBytes(slice));
+    }
+
     public static void writeValue(byte[] bytes, ByteBuf cb)
     {
         if (bytes == null)
@@ -375,7 +430,7 @@
         return 4 + (bytes == null ? 0 : bytes.remaining());
     }
 
-    public static List<ByteBuffer> readValueList(ByteBuf cb)
+    public static List<ByteBuffer> readValueList(ByteBuf cb, int protocolVersion)
     {
         int size = cb.readUnsignedShort();
         if (size == 0)
@@ -383,7 +438,7 @@
 
         List<ByteBuffer> l = new ArrayList<ByteBuffer>(size);
         for (int i = 0; i < size; i++)
-            l.add(readValue(cb));
+            l.add(readBoundValue(cb, protocolVersion));
         return l;
     }
 
@@ -402,7 +457,7 @@
         return size;
     }
 
-    public static Pair<List<String>, List<ByteBuffer>> readNameAndValueList(ByteBuf cb)
+    public static Pair<List<String>, List<ByteBuffer>> readNameAndValueList(ByteBuf cb, int protocolVersion)
     {
         int size = cb.readUnsignedShort();
         if (size == 0)
@@ -413,7 +468,7 @@
         for (int i = 0; i < size; i++)
         {
             s.add(readString(cb));
-            l.add(readValue(cb));
+            l.add(readBoundValue(cb, protocolVersion));
         }
         return Pair.create(s, l);
     }
diff --git a/src/java/org/apache/cassandra/transport/Client.java b/src/java/org/apache/cassandra/transport/Client.java
index 8535dbd..92466d2 100644
--- a/src/java/org/apache/cassandra/transport/Client.java
+++ b/src/java/org/apache/cassandra/transport/Client.java
@@ -22,16 +22,11 @@
 import java.io.InputStreamReader;
 import java.nio.ByteBuffer;
 import java.nio.charset.StandardCharsets;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
+import java.util.*;
 
 import com.google.common.base.Splitter;
 
-import org.apache.cassandra.auth.IAuthenticator;
+import org.apache.cassandra.auth.PasswordAuthenticator;
 import org.apache.cassandra.config.Config;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.db.ConsistencyLevel;
@@ -46,9 +41,12 @@
 
 public class Client extends SimpleClient
 {
-    public Client(String host, int port, ClientEncryptionOptions encryptionOptions)
+    private final SimpleEventHandler eventHandler = new SimpleEventHandler();
+
+    public Client(String host, int port, int version, ClientEncryptionOptions encryptionOptions)
     {
-        super(host, port, encryptionOptions);
+        super(host, port, version, encryptionOptions);
+        setEventHandler(eventHandler);
     }
 
     public void run() throws IOException
@@ -62,6 +60,12 @@
         BufferedReader in = new BufferedReader(new InputStreamReader(System.in));
         for (;;)
         {
+            Event event;
+            while ((event = eventHandler.queue.poll()) != null)
+            {
+                System.out.println("<< " + event);
+            }
+
             System.out.print(">> ");
             System.out.flush();
             String line = in.readLine();
@@ -180,7 +184,7 @@
         else if (msgType.equals("AUTHENTICATE"))
         {
             Map<String, String> credentials = readCredentials(iter);
-            if(!credentials.containsKey(IAuthenticator.USERNAME_KEY) || !credentials.containsKey(IAuthenticator.PASSWORD_KEY))
+            if(!credentials.containsKey(PasswordAuthenticator.USERNAME_KEY) || !credentials.containsKey(PasswordAuthenticator.PASSWORD_KEY))
             {
                 System.err.println("[ERROR] Authentication requires both 'username' and 'password'");
                 return null;
@@ -222,8 +226,8 @@
 
     private byte[] encodeCredentialsForSasl(Map<String, String> credentials)
     {
-        byte[] username = credentials.get(IAuthenticator.USERNAME_KEY).getBytes(StandardCharsets.UTF_8);
-        byte[] password = credentials.get(IAuthenticator.PASSWORD_KEY).getBytes(StandardCharsets.UTF_8);
+        byte[] username = credentials.get(PasswordAuthenticator.USERNAME_KEY).getBytes(StandardCharsets.UTF_8);
+        byte[] password = credentials.get(PasswordAuthenticator.PASSWORD_KEY).getBytes(StandardCharsets.UTF_8);
         byte[] initialResponse = new byte[username.length + password.length + 2];
         initialResponse[0] = 0;
         System.arraycopy(username, 0, initialResponse, 1, username.length);
@@ -237,20 +241,21 @@
         Config.setClientMode(true);
 
         // Print usage if no argument is specified.
-        if (args.length != 2)
+        if (args.length < 2 || args.length > 3)
         {
-            System.err.println("Usage: " + Client.class.getSimpleName() + " <host> <port>");
+            System.err.println("Usage: " + Client.class.getSimpleName() + " <host> <port> [<version>]");
             return;
         }
 
         // Parse options.
         String host = args[0];
         int port = Integer.parseInt(args[1]);
+        int version = args.length == 3 ? Integer.parseInt(args[2]) : Server.CURRENT_VERSION;
 
         ClientEncryptionOptions encryptionOptions = new ClientEncryptionOptions();
-        System.out.println("CQL binary protocol console " + host + "@" + port);
+        System.out.println("CQL binary protocol console " + host + "@" + port + " using native protocol version " + version);
 
-        new Client(host, port, encryptionOptions).run();
+        new Client(host, port, version, encryptionOptions).run();
         System.exit(0);
     }
 }
diff --git a/src/java/org/apache/cassandra/transport/Connection.java b/src/java/org/apache/cassandra/transport/Connection.java
index aa571a7..af26557 100644
--- a/src/java/org/apache/cassandra/transport/Connection.java
+++ b/src/java/org/apache/cassandra/transport/Connection.java
@@ -66,12 +66,11 @@
 
     public interface Factory
     {
-        public Connection newConnection(Channel channel, int version);
+        Connection newConnection(Channel channel, int version);
     }
 
     public interface Tracker
     {
-        public void addConnection(Channel ch, Connection connection);
-        public void closeAll();
+        void addConnection(Channel ch, Connection connection);
     }
 }
diff --git a/src/java/org/apache/cassandra/transport/DataType.java b/src/java/org/apache/cassandra/transport/DataType.java
index 0ea353a..e3eaf32 100644
--- a/src/java/org/apache/cassandra/transport/DataType.java
+++ b/src/java/org/apache/cassandra/transport/DataType.java
@@ -24,6 +24,8 @@
 import java.util.Map;
 import java.util.List;
 
+import com.google.common.annotations.VisibleForTesting;
+
 import io.netty.buffer.ByteBuf;
 
 import org.apache.cassandra.exceptions.RequestValidationException;
@@ -32,33 +34,37 @@
 
 public enum DataType implements OptionCodec.Codecable<DataType>
 {
-    CUSTOM   (0,  null),
-    ASCII    (1,  AsciiType.instance),
-    BIGINT   (2,  LongType.instance),
-    BLOB     (3,  BytesType.instance),
-    BOOLEAN  (4,  BooleanType.instance),
-    COUNTER  (5,  CounterColumnType.instance),
-    DECIMAL  (6,  DecimalType.instance),
-    DOUBLE   (7,  DoubleType.instance),
-    FLOAT    (8,  FloatType.instance),
-    INT      (9,  Int32Type.instance),
-    TEXT     (10, UTF8Type.instance),
-    TIMESTAMP(11, TimestampType.instance),
-    UUID     (12, UUIDType.instance),
-    VARCHAR  (13, UTF8Type.instance),
-    VARINT   (14, IntegerType.instance),
-    TIMEUUID (15, TimeUUIDType.instance),
-    INET     (16, InetAddressType.instance),
-    LIST     (32, null),
-    MAP      (33, null),
-    SET      (34, null),
-    UDT      (48, null),
-    TUPLE    (49, null);
-
+    CUSTOM   (0,  null, 1),
+    ASCII    (1,  AsciiType.instance, 1),
+    BIGINT   (2,  LongType.instance, 1),
+    BLOB     (3,  BytesType.instance, 1),
+    BOOLEAN  (4,  BooleanType.instance, 1),
+    COUNTER  (5,  CounterColumnType.instance, 1),
+    DECIMAL  (6,  DecimalType.instance, 1),
+    DOUBLE   (7,  DoubleType.instance, 1),
+    FLOAT    (8,  FloatType.instance, 1),
+    INT      (9,  Int32Type.instance, 1),
+    TEXT     (10, UTF8Type.instance, 1),
+    TIMESTAMP(11, TimestampType.instance, 1),
+    UUID     (12, UUIDType.instance, 1),
+    VARCHAR  (13, UTF8Type.instance, 1),
+    VARINT   (14, IntegerType.instance, 1),
+    TIMEUUID (15, TimeUUIDType.instance, 1),
+    INET     (16, InetAddressType.instance, 1),
+    DATE     (17, SimpleDateType.instance, 4),
+    TIME     (18, TimeType.instance, 4),
+    SMALLINT (19, ShortType.instance, 4),
+    BYTE     (20, ByteType.instance, 4),
+    LIST     (32, null, 1),
+    MAP      (33, null, 1),
+    SET      (34, null, 1),
+    UDT      (48, null, 3),
+    TUPLE    (49, null, 3);
 
     public static final OptionCodec<DataType> codec = new OptionCodec<DataType>(DataType.class);
 
     private final int id;
+    private final int protocolVersion;
     private final AbstractType type;
     private static final Map<AbstractType, DataType> dataTypeMap = new HashMap<AbstractType, DataType>();
     static
@@ -70,14 +76,17 @@
         }
     }
 
-    private DataType(int id, AbstractType type)
+    DataType(int id, AbstractType type, int protocolVersion)
     {
         this.id = id;
         this.type = type;
+        this.protocolVersion = protocolVersion;
     }
 
-    public int getId()
+    public int getId(int version)
     {
+        if (version < protocolVersion)
+            return DataType.CUSTOM.getId(version);
         return id;
     }
 
@@ -121,6 +130,13 @@
 
     public void writeValue(Object value, ByteBuf cb, int version)
     {
+        // Serialize as CUSTOM if client on the other side's version is < required for type
+        if (version < protocolVersion)
+        {
+            CBUtil.writeString(value.toString(), cb);
+            return;
+        }
+
         switch (this)
         {
             case CUSTOM:
@@ -160,6 +176,10 @@
 
     public int serializedValueSize(Object value, int version)
     {
+        // Serialize as CUSTOM if client on the other side's version is < required for type
+        if (version < protocolVersion)
+            return CBUtil.sizeOfString(value.toString());
+
         switch (this)
         {
             case CUSTOM:
@@ -228,16 +248,19 @@
                 throw new AssertionError();
             }
 
-            if (type instanceof UserType && version >= 3)
+            if (type instanceof UserType && version >= UDT.protocolVersion)
                 return Pair.<DataType, Object>create(UDT, type);
 
-            if (type instanceof TupleType && version >= 3)
+            if (type instanceof TupleType && version >= TUPLE.protocolVersion)
                 return Pair.<DataType, Object>create(TUPLE, type);
 
             return Pair.<DataType, Object>create(CUSTOM, type.toString());
         }
         else
         {
+            // Fall back to CUSTOM if target doesn't know this data type
+            if (version < dt.protocolVersion)
+                return Pair.<DataType, Object>create(CUSTOM, type.toString());
             return Pair.create(dt, null);
         }
     }
@@ -270,4 +293,10 @@
             throw new ProtocolException(e.getMessage());
         }
     }
+
+    @VisibleForTesting
+    public int getProtocolVersion()
+    {
+        return protocolVersion;
+    }
 }
diff --git a/src/java/org/apache/cassandra/transport/Event.java b/src/java/org/apache/cassandra/transport/Event.java
index 12ad6e9..a3e0888 100644
--- a/src/java/org/apache/cassandra/transport/Event.java
+++ b/src/java/org/apache/cassandra/transport/Event.java
@@ -19,13 +19,28 @@
 
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
+import java.util.Iterator;
+import java.util.List;
+import java.util.UUID;
 
 import com.google.common.base.Objects;
 import io.netty.buffer.ByteBuf;
 
 public abstract class Event
 {
-    public enum Type { TOPOLOGY_CHANGE, STATUS_CHANGE, SCHEMA_CHANGE }
+    public enum Type {
+        TOPOLOGY_CHANGE(Server.VERSION_1),
+        STATUS_CHANGE(Server.VERSION_1),
+        SCHEMA_CHANGE(Server.VERSION_1),
+        TRACE_COMPLETE(Server.VERSION_4);
+
+        public final int minimumVersion;
+
+        Type(int minimumVersion)
+        {
+            this.minimumVersion = minimumVersion;
+        }
+    }
 
     public final Type type;
 
@@ -36,7 +51,10 @@
 
     public static Event deserialize(ByteBuf cb, int version)
     {
-        switch (CBUtil.readEnumValue(Type.class, cb))
+        Type eventType = CBUtil.readEnumValue(Type.class, cb);
+        if (eventType.minimumVersion > version)
+            throw new ProtocolException("Event " + eventType.name() + " not valid for protocol version " + version);
+        switch (eventType)
         {
             case TOPOLOGY_CHANGE:
                 return TopologyChange.deserializeEvent(cb, version);
@@ -50,6 +68,8 @@
 
     public void serialize(ByteBuf dest, int version)
     {
+        if (type.minimumVersion > version)
+            throw new ProtocolException("Event " + type.name() + " not valid for protocol version " + version);
         CBUtil.writeEnumValue(type, dest);
         serializeEvent(dest, version);
     }
@@ -217,22 +237,29 @@
     public static class SchemaChange extends Event
     {
         public enum Change { CREATED, UPDATED, DROPPED }
-        public enum Target { KEYSPACE, TABLE, TYPE }
+        public enum Target { KEYSPACE, TABLE, TYPE, FUNCTION, AGGREGATE }
 
         public final Change change;
         public final Target target;
         public final String keyspace;
-        public final String tableOrType;
+        public final String name;
+        public final List<String> argTypes;
 
-        public SchemaChange(Change change, Target target, String keyspace, String tableOrType)
+        public SchemaChange(Change change, Target target, String keyspace, String name, List<String> argTypes)
         {
             super(Type.SCHEMA_CHANGE);
             this.change = change;
             this.target = target;
             this.keyspace = keyspace;
-            this.tableOrType = tableOrType;
+            this.name = name;
             if (target != Target.KEYSPACE)
-                assert this.tableOrType != null : "Table or type should be set for non-keyspace schema change events";
+                assert this.name != null : "Table, type, function or aggregate name should be set for non-keyspace schema change events";
+            this.argTypes = argTypes;
+        }
+
+        public SchemaChange(Change change, Target target, String keyspace, String name)
+        {
+            this(change, target, keyspace, name, null);
         }
 
         public SchemaChange(Change change, String keyspace)
@@ -249,7 +276,11 @@
                 Target target = CBUtil.readEnumValue(Target.class, cb);
                 String keyspace = CBUtil.readString(cb);
                 String tableOrType = target == Target.KEYSPACE ? null : CBUtil.readString(cb);
-                return new SchemaChange(change, target, keyspace, tableOrType);
+                List<String> argTypes = null;
+                if (target == Target.FUNCTION || target == Target.AGGREGATE)
+                    argTypes = CBUtil.readStringList(cb);
+
+                return new SchemaChange(change, target, keyspace, tableOrType, argTypes);
             }
             else
             {
@@ -261,13 +292,36 @@
 
         public void serializeEvent(ByteBuf dest, int version)
         {
+            if (target == Target.FUNCTION || target == Target.AGGREGATE)
+            {
+                if (version >= 4)
+                {
+                    // available since protocol version 4
+                    CBUtil.writeEnumValue(change, dest);
+                    CBUtil.writeEnumValue(target, dest);
+                    CBUtil.writeString(keyspace, dest);
+                    CBUtil.writeString(name, dest);
+                    CBUtil.writeStringList(argTypes, dest);
+                }
+                else
+                {
+                    // not available in protocol versions < 4 - just say the keyspace was updated.
+                    CBUtil.writeEnumValue(Change.UPDATED, dest);
+                    if (version >= 3)
+                        CBUtil.writeEnumValue(Target.KEYSPACE, dest);
+                    CBUtil.writeString(keyspace, dest);
+                    CBUtil.writeString("", dest);
+                }
+                return;
+            }
+
             if (version >= 3)
             {
                 CBUtil.writeEnumValue(change, dest);
                 CBUtil.writeEnumValue(target, dest);
                 CBUtil.writeString(keyspace, dest);
                 if (target != Target.KEYSPACE)
-                    CBUtil.writeString(tableOrType, dest);
+                    CBUtil.writeString(name, dest);
             }
             else
             {
@@ -283,13 +337,30 @@
                 {
                     CBUtil.writeEnumValue(change, dest);
                     CBUtil.writeString(keyspace, dest);
-                    CBUtil.writeString(target == Target.KEYSPACE ? "" : tableOrType, dest);
+                    CBUtil.writeString(target == Target.KEYSPACE ? "" : name, dest);
                 }
             }
         }
 
         public int eventSerializedSize(int version)
         {
+            if (target == Target.FUNCTION || target == Target.AGGREGATE)
+            {
+                if (version >= 4)
+                    return CBUtil.sizeOfEnumValue(change)
+                               + CBUtil.sizeOfEnumValue(target)
+                               + CBUtil.sizeOfString(keyspace)
+                               + CBUtil.sizeOfString(name)
+                               + CBUtil.sizeOfStringList(argTypes);
+                if (version >= 3)
+                    return CBUtil.sizeOfEnumValue(Change.UPDATED)
+                           + CBUtil.sizeOfEnumValue(Target.KEYSPACE)
+                           + CBUtil.sizeOfString(keyspace);
+                return CBUtil.sizeOfEnumValue(Change.UPDATED)
+                       + CBUtil.sizeOfString(keyspace)
+                       + CBUtil.sizeOfString("");
+            }
+
             if (version >= 3)
             {
                 int size = CBUtil.sizeOfEnumValue(change)
@@ -297,7 +368,7 @@
                          + CBUtil.sizeOfString(keyspace);
 
                 if (target != Target.KEYSPACE)
-                    size += CBUtil.sizeOfString(tableOrType);
+                    size += CBUtil.sizeOfString(name);
 
                 return size;
             }
@@ -311,20 +382,36 @@
                 }
                 return CBUtil.sizeOfEnumValue(change)
                      + CBUtil.sizeOfString(keyspace)
-                     + CBUtil.sizeOfString(target == Target.KEYSPACE ? "" : tableOrType);
+                     + CBUtil.sizeOfString(target == Target.KEYSPACE ? "" : name);
             }
         }
 
         @Override
         public String toString()
         {
-            return change + " " + target + " " + keyspace + (tableOrType == null ? "" : "." + tableOrType);
+            StringBuilder sb = new StringBuilder().append(change)
+                                                  .append(' ').append(target)
+                                                  .append(' ').append(keyspace);
+            if (name != null)
+                sb.append('.').append(name);
+            if (argTypes != null)
+            {
+                sb.append(" (");
+                for (Iterator<String> iter = argTypes.iterator(); iter.hasNext(); )
+                {
+                    sb.append(iter.next());
+                    if (iter.hasNext())
+                        sb.append(',');
+                }
+                sb.append(')');
+            }
+            return sb.toString();
         }
 
         @Override
         public int hashCode()
         {
-            return Objects.hashCode(change, target, keyspace, tableOrType);
+            return Objects.hashCode(change, target, keyspace, name, argTypes);
         }
 
         @Override
@@ -337,7 +424,8 @@
             return Objects.equal(change, scc.change)
                 && Objects.equal(target, scc.target)
                 && Objects.equal(keyspace, scc.keyspace)
-                && Objects.equal(tableOrType, scc.tableOrType);
+                && Objects.equal(name, scc.name)
+                && Objects.equal(argTypes, scc.argTypes);
         }
     }
 }
diff --git a/src/java/org/apache/cassandra/transport/Frame.java b/src/java/org/apache/cassandra/transport/Frame.java
index 021143e..66df3e7 100644
--- a/src/java/org/apache/cassandra/transport/Frame.java
+++ b/src/java/org/apache/cassandra/transport/Frame.java
@@ -22,24 +22,33 @@
 import java.util.EnumSet;
 import java.util.List;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import io.netty.buffer.ByteBuf;
 import io.netty.channel.*;
 import io.netty.handler.codec.ByteToMessageDecoder;
 import io.netty.handler.codec.MessageToMessageDecoder;
 import io.netty.handler.codec.MessageToMessageEncoder;
-
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.transport.messages.ErrorMessage;
 
 public class Frame
 {
+    private static final Logger logger = LoggerFactory.getLogger(Frame.class);
+
     public static final byte PROTOCOL_VERSION_MASK = 0x7f;
 
     public final Header header;
     public final ByteBuf body;
 
     /**
+     * <code>true</code> if the deprecation warning for protocol versions 1 and 2 has been logged.
+     */
+    private static boolean hasLoggedDeprecationWarning;
+
+    /**
      * An on-wire frame consists of a header and a body.
      *
      * The header is defined the following way in native protocol version 3 and later:
@@ -113,7 +122,9 @@
         {
             // The order of that enum matters!!
             COMPRESSED,
-            TRACING;
+            TRACING,
+            CUSTOM_PAYLOAD,
+            WARNING;
 
             private static final Flag[] ALL_VALUES = values();
 
@@ -183,7 +194,17 @@
             int version = firstByte & PROTOCOL_VERSION_MASK;
 
             if (version > Server.CURRENT_VERSION)
-                throw new ProtocolException("Invalid or unsupported protocol version: " + version);
+                throw new ProtocolException(String.format("Invalid or unsupported protocol version (%d); highest supported is %d ",
+                                                          version, Server.CURRENT_VERSION));
+
+            if (version < Server.VERSION_3 && !hasLoggedDeprecationWarning)
+            {
+                hasLoggedDeprecationWarning = true;
+                logger.warn("Detected connection using native protocol version {}. Both version 1 and 2"
+                          + " of the native protocol are now deprecated and support will be removed in Cassandra 3.0."
+                          + " You are encouraged to upgrade to a client driver using version 3 of the native protocol",
+                            version);
+            }
 
             // Wait until we have the complete V3+ header
             if (version >= Server.VERSION_3 && buffer.readableBytes() < Header.MODERN_LENGTH)
@@ -218,7 +239,6 @@
 
             long bodyLength = buffer.getUnsignedInt(idx);
             idx += Header.BODY_LENGTH_SIZE;
-
             long frameLength = bodyLength + headerLength;
             if (frameLength > MAX_FRAME_LENGTH)
             {
diff --git a/src/java/org/apache/cassandra/transport/FrameCompressor.java b/src/java/org/apache/cassandra/transport/FrameCompressor.java
index fa337ec..01c0c31 100644
--- a/src/java/org/apache/cassandra/transport/FrameCompressor.java
+++ b/src/java/org/apache/cassandra/transport/FrameCompressor.java
@@ -20,11 +20,9 @@
 import java.io.IOException;
 
 import io.netty.buffer.ByteBuf;
-import io.netty.buffer.Unpooled;
 import org.xerial.snappy.Snappy;
 import org.xerial.snappy.SnappyError;
 
-import net.jpountz.lz4.LZ4Exception;
 import net.jpountz.lz4.LZ4Factory;
 
 import org.apache.cassandra.utils.JVMStabilityInspector;
@@ -53,15 +51,7 @@
                 JVMStabilityInspector.inspectThrowable(e);
                 i = null;
             }
-            catch (NoClassDefFoundError e)
-            {
-                i = null;
-            }
-            catch (SnappyError e)
-            {
-                i = null;
-            }
-            catch (UnsatisfiedLinkError e)
+            catch (NoClassDefFoundError | SnappyError | UnsatisfiedLinkError e)
             {
                 i = null;
             }
diff --git a/src/java/org/apache/cassandra/transport/Message.java b/src/java/org/apache/cassandra/transport/Message.java
index 1cf56fc..e4f5cac 100644
--- a/src/java/org/apache/cassandra/transport/Message.java
+++ b/src/java/org/apache/cassandra/transport/Message.java
@@ -19,9 +19,11 @@
 
 import java.util.ArrayList;
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.EnumSet;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
@@ -40,6 +42,7 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.service.ClientWarn;
 import org.apache.cassandra.transport.messages.*;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.utils.JVMStabilityInspector;
@@ -146,6 +149,7 @@
     protected Connection connection;
     private int streamId;
     private Frame sourceFrame;
+    private Map<String, ByteBuffer> customPayload;
 
     protected Message(Type type)
     {
@@ -183,6 +187,16 @@
         return sourceFrame;
     }
 
+    public Map<String, ByteBuffer> getCustomPayload()
+    {
+        return customPayload;
+    }
+
+    public void setCustomPayload(Map<String, ByteBuffer> customPayload)
+    {
+        this.customPayload = customPayload;
+    }
+
     public static abstract class Request extends Message
     {
         protected boolean tracingRequested;
@@ -211,6 +225,7 @@
     public static abstract class Response extends Message
     {
         protected UUID tracingId;
+        protected List<String> warnings;
 
         protected Response(Type type)
         {
@@ -230,6 +245,17 @@
         {
             return tracingId;
         }
+
+        public Message setWarnings(List<String> warnings)
+        {
+            this.warnings = warnings;
+            return this;
+        }
+
+        public List<String> getWarnings()
+        {
+            return warnings;
+        }
     }
 
     @ChannelHandler.Sharable
@@ -239,14 +265,22 @@
         {
             boolean isRequest = frame.header.type.direction == Direction.REQUEST;
             boolean isTracing = frame.header.flags.contains(Frame.Header.Flag.TRACING);
+            boolean isCustomPayload = frame.header.flags.contains(Frame.Header.Flag.CUSTOM_PAYLOAD);
+            boolean hasWarning = frame.header.flags.contains(Frame.Header.Flag.WARNING);
 
             UUID tracingId = isRequest || !isTracing ? null : CBUtil.readUUID(frame.body);
+            List<String> warnings = isRequest || !hasWarning ? null : CBUtil.readStringList(frame.body);
+            Map<String, ByteBuffer> customPayload = !isCustomPayload ? null : CBUtil.readBytesMap(frame.body);
 
             try
             {
+                if (isCustomPayload && frame.header.version < Server.VERSION_4)
+                    throw new ProtocolException("Received frame with CUSTOM_PAYLOAD flag for native protocol version < 4");
+
                 Message message = frame.header.type.codec.decode(frame.body, frame.header.version);
                 message.setStreamId(frame.header.streamId);
                 message.setSourceFrame(frame);
+                message.setCustomPayload(customPayload);
 
                 if (isRequest)
                 {
@@ -262,6 +296,8 @@
                     assert message instanceof Response;
                     if (isTracing)
                         ((Response)message).setTracingId(tracingId);
+                    if (hasWarning)
+                        ((Response)message).setWarnings(warnings);
                 }
 
                 results.add(message);
@@ -294,23 +330,53 @@
                 if (message instanceof Response)
                 {
                     UUID tracingId = ((Response)message).getTracingId();
+                    Map<String, ByteBuffer> customPayload = message.getCustomPayload();
+                    if (tracingId != null)
+                        messageSize += CBUtil.sizeOfUUID(tracingId);
+                    List<String> warnings = ((Response)message).getWarnings();
+                    if (warnings != null)
+                    {
+                        if (version < Server.VERSION_4)
+                            throw new ProtocolException("Must not send frame with WARNING flag for native protocol version < 4");
+                        messageSize += CBUtil.sizeOfStringList(warnings);
+                    }
+                    if (customPayload != null)
+                    {
+                        if (version < Server.VERSION_4)
+                            throw new ProtocolException("Must not send frame with CUSTOM_PAYLOAD flag for native protocol version < 4");
+                        messageSize += CBUtil.sizeOfBytesMap(customPayload);
+                    }
+                    body = CBUtil.allocator.buffer(messageSize);
                     if (tracingId != null)
                     {
-                        body = CBUtil.allocator.buffer(CBUtil.sizeOfUUID(tracingId) + messageSize);
                         CBUtil.writeUUID(tracingId, body);
                         flags.add(Frame.Header.Flag.TRACING);
                     }
-                    else
+                    if (warnings != null)
                     {
-                        body = CBUtil.allocator.buffer(messageSize);
+                        CBUtil.writeStringList(warnings, body);
+                        flags.add(Frame.Header.Flag.WARNING);
+                    }
+                    if (customPayload != null)
+                    {
+                        CBUtil.writeBytesMap(customPayload, body);
+                        flags.add(Frame.Header.Flag.CUSTOM_PAYLOAD);
                     }
                 }
                 else
                 {
                     assert message instanceof Request;
-                    body = CBUtil.allocator.buffer(messageSize);
                     if (((Request)message).isTracingRequested())
                         flags.add(Frame.Header.Flag.TRACING);
+                    Map<String, ByteBuffer> payload = message.getCustomPayload();
+                    if (payload != null)
+                        messageSize += CBUtil.sizeOfBytesMap(payload);
+                    body = CBUtil.allocator.buffer(messageSize);
+                    if (payload != null)
+                    {
+                        CBUtil.writeBytesMap(payload, body);
+                        flags.add(Frame.Header.Flag.CUSTOM_PAYLOAD);
+                    }
                 }
 
                 try
@@ -348,7 +414,7 @@
             }
         }
 
-        private final class Flusher implements Runnable
+        private static final class Flusher implements Runnable
         {
             final EventLoop eventLoop;
             final ConcurrentLinkedQueue<FlushItem> queued = new ConcurrentLinkedQueue<>();
@@ -432,12 +498,15 @@
             {
                 assert request.connection() instanceof ServerConnection;
                 connection = (ServerConnection)request.connection();
+                if (connection.getVersion() >= Server.VERSION_4)
+                    ClientWarn.instance.captureWarnings();
+
                 QueryState qstate = connection.validateNewMessage(request.type, connection.getVersion(), request.getStreamId());
 
-                logger.debug("Received: {}, v={}", request, connection.getVersion());
-
+                logger.trace("Received: {}, v={}", request, connection.getVersion());
                 response = request.execute(qstate);
                 response.setStreamId(request.getStreamId());
+                response.setWarnings(ClientWarn.instance.getWarnings());
                 response.attach(connection);
                 connection.applyStateTransition(request.type, response.type);
             }
@@ -448,8 +517,12 @@
                 flush(new FlushItem(ctx, ErrorMessage.fromException(t, handler).setStreamId(request.getStreamId()), request.getSourceFrame()));
                 return;
             }
+            finally
+            {
+                ClientWarn.instance.resetWarnings();
+            }
 
-            logger.debug("Responding: {}, v={}", response, connection.getVersion());
+            logger.trace("Responding: {}, v={}", response, connection.getVersion());
             flush(new FlushItem(ctx, response, request.getSourceFrame()));
         }
 
@@ -528,7 +601,7 @@
                 if (ioExceptionsAtDebugLevel.contains(exception.getMessage()))
                 {
                     // Likely unclean client disconnects
-                    logger.debug(message, exception);
+                    logger.trace(message, exception);
                 }
                 else
                 {
diff --git a/src/java/org/apache/cassandra/transport/OptionCodec.java b/src/java/org/apache/cassandra/transport/OptionCodec.java
index ec2a1fa..3a8b813 100644
--- a/src/java/org/apache/cassandra/transport/OptionCodec.java
+++ b/src/java/org/apache/cassandra/transport/OptionCodec.java
@@ -30,7 +30,7 @@
 {
     public interface Codecable<T extends Enum<T>>
     {
-        public int getId();
+        public int getId(int version);
 
         public Object readValue(ByteBuf cb, int version);
         public void writeValue(Object value, ByteBuf cb, int version);
@@ -48,13 +48,13 @@
         T[] values = klass.getEnumConstants();
         int maxId = -1;
         for (T opt : values)
-            maxId = Math.max(maxId, opt.getId());
+            maxId = Math.max(maxId, opt.getId(Server.CURRENT_VERSION));
         ids = (T[])Array.newInstance(klass, maxId + 1);
         for (T opt : values)
         {
-            if (ids[opt.getId()] != null)
-                throw new IllegalStateException(String.format("Duplicate option id %d", opt.getId()));
-            ids[opt.getId()] = opt;
+            if (ids[opt.getId(Server.CURRENT_VERSION)] != null)
+                throw new IllegalStateException(String.format("Duplicate option id %d", opt.getId(Server.CURRENT_VERSION)));
+            ids[opt.getId(Server.CURRENT_VERSION)] = opt;
         }
     }
 
@@ -91,7 +91,7 @@
         for (Map.Entry<T, Object> entry : options.entrySet())
         {
             T opt = entry.getKey();
-            cb.writeShort(opt.getId());
+            cb.writeShort(opt.getId(version));
             opt.writeValue(entry.getValue(), cb, version);
         }
         return cb;
@@ -108,7 +108,7 @@
     {
         T opt = option.left;
         Object obj = option.right;
-        dest.writeShort(opt.getId());
+        dest.writeShort(opt.getId(version));
         opt.writeValue(obj, dest, version);
     }
 
diff --git a/src/java/org/apache/cassandra/transport/RequestThreadPoolExecutor.java b/src/java/org/apache/cassandra/transport/RequestThreadPoolExecutor.java
index 75b983a..75dd05d 100644
--- a/src/java/org/apache/cassandra/transport/RequestThreadPoolExecutor.java
+++ b/src/java/org/apache/cassandra/transport/RequestThreadPoolExecutor.java
@@ -23,19 +23,19 @@
 import io.netty.util.concurrent.AbstractEventExecutor;
 import io.netty.util.concurrent.EventExecutorGroup;
 import io.netty.util.concurrent.Future;
-import org.apache.cassandra.concurrent.TracingAwareExecutorService;
+import org.apache.cassandra.concurrent.LocalAwareExecutorService;
 import org.apache.cassandra.config.DatabaseDescriptor;
 
-import static org.apache.cassandra.concurrent.JMXEnabledSharedExecutorPool.SHARED;
+import static org.apache.cassandra.concurrent.SharedExecutorPool.SHARED;
 
 public class RequestThreadPoolExecutor extends AbstractEventExecutor
 {
     private final static int MAX_QUEUED_REQUESTS = Integer.getInteger("cassandra.max_queued_native_transport_requests", 128);
     private final static String THREAD_FACTORY_ID = "Native-Transport-Requests";
-    private final TracingAwareExecutorService wrapped = SHARED.newExecutor(DatabaseDescriptor.getNativeTransportMaxThreads(),
+    private final LocalAwareExecutorService wrapped = SHARED.newExecutor(DatabaseDescriptor.getNativeTransportMaxThreads(),
                                                                            MAX_QUEUED_REQUESTS,
-                                                                           THREAD_FACTORY_ID,
-                                                                           "transport");
+                                                                           "transport",
+                                                                           THREAD_FACTORY_ID);
 
     public boolean isShuttingDown()
     {
diff --git a/src/java/org/apache/cassandra/transport/Server.java b/src/java/org/apache/cassandra/transport/Server.java
index 02f17b0..d1047f9 100644
--- a/src/java/org/apache/cassandra/transport/Server.java
+++ b/src/java/org/apache/cassandra/transport/Server.java
@@ -21,43 +21,40 @@
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
-import java.util.EnumMap;
-import java.util.List;
-import java.util.Map;
+import java.util.*;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicBoolean;
 import javax.net.ssl.SSLContext;
 import javax.net.ssl.SSLEngine;
 
-import io.netty.buffer.ByteBuf;
-import io.netty.channel.epoll.Epoll;
-import io.netty.channel.epoll.EpollEventLoopGroup;
-import io.netty.channel.epoll.EpollServerSocketChannel;
-import io.netty.handler.codec.ByteToMessageDecoder;
-import io.netty.util.Version;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import io.netty.bootstrap.ServerBootstrap;
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.*;
+import io.netty.channel.epoll.Epoll;
+import io.netty.channel.epoll.EpollEventLoopGroup;
+import io.netty.channel.epoll.EpollServerSocketChannel;
+import io.netty.channel.group.ChannelGroup;
+import io.netty.channel.group.DefaultChannelGroup;
 import io.netty.channel.nio.NioEventLoopGroup;
 import io.netty.channel.socket.nio.NioServerSocketChannel;
+import io.netty.handler.codec.ByteToMessageDecoder;
+import io.netty.handler.ssl.SslHandler;
+import io.netty.util.Version;
 import io.netty.util.concurrent.EventExecutor;
 import io.netty.util.concurrent.GlobalEventExecutor;
 import io.netty.util.internal.logging.InternalLoggerFactory;
 import io.netty.util.internal.logging.Slf4JLoggerFactory;
-import org.apache.cassandra.auth.IAuthenticator;
-import org.apache.cassandra.auth.ISaslAwareAuthenticator;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.EncryptionOptions;
+import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.metrics.ClientMetrics;
 import org.apache.cassandra.security.SSLFactory;
 import org.apache.cassandra.service.*;
 import org.apache.cassandra.transport.messages.EventMessage;
-import io.netty.bootstrap.ServerBootstrap;
-import io.netty.channel.*;
-import io.netty.channel.group.ChannelGroup;
-import io.netty.channel.group.DefaultChannelGroup;
-import io.netty.handler.ssl.SslHandler;
 import org.apache.cassandra.utils.FBUtilities;
 
 public class Server implements CassandraDaemon.Server
@@ -70,8 +67,11 @@
     private static final Logger logger = LoggerFactory.getLogger(Server.class);
     private static final boolean enableEpoll = Boolean.valueOf(System.getProperty("cassandra.native.epoll.enabled", "true"));
 
+    public static final int VERSION_1 = 1;
+    public static final int VERSION_2 = 2;
     public static final int VERSION_3 = 3;
-    public static final int CURRENT_VERSION = VERSION_3;
+    public static final int VERSION_4 = 4;
+    public static final int CURRENT_VERSION = VERSION_4;
 
     private final ConnectionTracker connectionTracker = new ConnectionTracker();
 
@@ -134,20 +134,9 @@
 
     private void run()
     {
-        // Check that a SaslAuthenticator can be provided by the configured
-        // IAuthenticator. If not, don't start the server.
-        IAuthenticator authenticator = DatabaseDescriptor.getAuthenticator();
-        if (authenticator.requireAuthentication() && !(authenticator instanceof ISaslAwareAuthenticator))
-        {
-            logger.error("Not starting native transport as the configured IAuthenticator is not capable of SASL authentication");
-            isRunning.compareAndSet(true, false);
-            return;
-        }
-
         // Configure the server.
         eventExecutorGroup = new RequestThreadPoolExecutor();
 
-
         boolean hasEpoll = enableEpoll ? Epoll.isAvailable() : false;
         if (hasEpoll)
         {
@@ -199,6 +188,8 @@
 
         connectionTracker.allChannels.add(bindFuture.channel());
         isRunning.set(true);
+
+        StorageService.instance.setRpcReady(true);
     }
 
     private void registerMetrics()
@@ -223,6 +214,8 @@
         eventExecutorGroup.shutdown();
         eventExecutorGroup = null;
         logger.info("Stop listening for CQL clients");
+
+        StorageService.instance.setRpcReady(false);
     }
 
 
@@ -230,7 +223,7 @@
     {
         // TODO: should we be using the GlobalEventExecutor or defining our own?
         public final ChannelGroup allChannels = new DefaultChannelGroup(GlobalEventExecutor.INSTANCE);
-        private final EnumMap<Event.Type, ChannelGroup> groups = new EnumMap<Event.Type, ChannelGroup>(Event.Type.class);
+        private final EnumMap<Event.Type, ChannelGroup> groups = new EnumMap<>(Event.Type.class);
 
         public ConnectionTracker()
         {
@@ -248,12 +241,6 @@
             groups.get(type).add(ch);
         }
 
-        public void unregister(Channel ch)
-        {
-            for (ChannelGroup group : groups.values())
-                group.remove(ch);
-        }
-
         public void send(Event event)
         {
             groups.get(event.type).writeAndFlush(new EventMessage(event));
@@ -342,7 +329,8 @@
         protected final SslHandler createSslHandler() {
             SSLEngine sslEngine = sslContext.createSSLEngine();
             sslEngine.setUseClientMode(false);
-            sslEngine.setEnabledCipherSuites(encryptionOptions.cipher_suites);
+            String[] suites = SSLFactory.filterCipherSuites(sslEngine.getSupportedCipherSuites(), encryptionOptions.cipher_suites);
+            sslEngine.setEnabledCipherSuites(suites);
             sslEngine.setNeedClientAuth(encryptionOptions.require_client_auth);
             sslEngine.setEnabledProtocols(SSLFactory.ACCEPTED_PROTOCOLS);
             return new SslHandler(sslEngine);
@@ -403,10 +391,53 @@
         }
     }
 
+    private static class LatestEvent
+    {
+        public final Event.StatusChange.Status status;
+        public final Event.TopologyChange.Change topology;
+
+        private LatestEvent(Event.StatusChange.Status status, Event.TopologyChange.Change topology)
+        {
+            this.status = status;
+            this.topology = topology;
+        }
+
+        @Override
+        public String toString()
+        {
+            return String.format("Status %s, Topology %s", status, topology);
+        }
+
+        public static LatestEvent forStatusChange(Event.StatusChange.Status status, LatestEvent prev)
+        {
+            return new LatestEvent(status,
+                                   prev == null ?
+                                           null :
+                                           prev.topology);
+        }
+
+        public static LatestEvent forTopologyChange(Event.TopologyChange.Change change, LatestEvent prev)
+        {
+            return new LatestEvent(prev == null ?
+                                           null :
+                                           prev.status,
+                                           change);
+        }
+    }
+
     private static class EventNotifier extends MigrationListener implements IEndpointLifecycleSubscriber
     {
         private final Server server;
-        private final Map<InetAddress, Event.StatusChange.Status> lastStatusChange = new ConcurrentHashMap<>();
+
+        // We keep track of the latest status change events we have sent to avoid sending duplicates
+        // since StorageService may send duplicate notifications (CASSANDRA-7816, CASSANDRA-8236, CASSANDRA-9156)
+        private final Map<InetAddress, LatestEvent> latestEvents = new ConcurrentHashMap<>();
+        // We also want to delay delivering a NEW_NODE notification until the new node has set its RPC ready
+        // state. This tracks the endpoints which have joined, but not yet signalled they're ready for clients
+        private final Set<InetAddress> endpointsPendingJoinedNotification =
+            Collections.newSetFromMap(new ConcurrentHashMap<InetAddress, Boolean>());
+
+
         private static final InetAddress bindAll;
         static {
             try
@@ -446,13 +477,16 @@
 
         private void send(InetAddress endpoint, Event.NodeEvent event)
         {
+            if (logger.isTraceEnabled())
+                logger.trace("Sending event for endpoint {}, rpc address {}", endpoint, event.nodeAddress());
+
             // If the endpoint is not the local node, extract the node address
             // and if it is the same as our own RPC broadcast address (which defaults to the rcp address)
             // then don't send the notification. This covers the case of rpc_address set to "localhost",
             // which is not useful to any driver and in fact may cauase serious problems to some drivers,
             // see CASSANDRA-10052
             if (!endpoint.equals(FBUtilities.getBroadcastAddress()) &&
-                event.nodeAddress().equals(DatabaseDescriptor.getBroadcastRpcAddress()))
+                event.nodeAddress().equals(FBUtilities.getBroadcastRpcAddress()))
                 return;
 
             send(event);
@@ -465,31 +499,61 @@
 
         public void onJoinCluster(InetAddress endpoint)
         {
-            send(endpoint, Event.TopologyChange.newNode(getRpcAddress(endpoint), server.socket.getPort()));
+            if (!StorageService.instance.isRpcReady(endpoint))
+                endpointsPendingJoinedNotification.add(endpoint);
+            else
+                onTopologyChange(endpoint, Event.TopologyChange.newNode(getRpcAddress(endpoint), server.socket.getPort()));
         }
 
         public void onLeaveCluster(InetAddress endpoint)
         {
-            send(endpoint, Event.TopologyChange.removedNode(getRpcAddress(endpoint), server.socket.getPort()));
+            onTopologyChange(endpoint, Event.TopologyChange.removedNode(getRpcAddress(endpoint), server.socket.getPort()));
         }
 
         public void onMove(InetAddress endpoint)
         {
-            send(endpoint, Event.TopologyChange.movedNode(getRpcAddress(endpoint), server.socket.getPort()));
+            onTopologyChange(endpoint, Event.TopologyChange.movedNode(getRpcAddress(endpoint), server.socket.getPort()));
         }
 
         public void onUp(InetAddress endpoint)
         {
-            Event.StatusChange.Status prev = lastStatusChange.put(endpoint, Event.StatusChange.Status.UP);
-            if (prev == null || prev != Event.StatusChange.Status.UP)
-                send(endpoint, Event.StatusChange.nodeUp(getRpcAddress(endpoint), server.socket.getPort()));
+            if (endpointsPendingJoinedNotification.remove(endpoint))
+                onJoinCluster(endpoint);
+            
+            onStatusChange(endpoint, Event.StatusChange.nodeUp(getRpcAddress(endpoint), server.socket.getPort()));
         }
 
         public void onDown(InetAddress endpoint)
         {
-            Event.StatusChange.Status prev = lastStatusChange.put(endpoint, Event.StatusChange.Status.DOWN);
-            if (prev == null || prev != Event.StatusChange.Status.DOWN)
-                send(endpoint, Event.StatusChange.nodeDown(getRpcAddress(endpoint), server.socket.getPort()));
+            onStatusChange(endpoint, Event.StatusChange.nodeDown(getRpcAddress(endpoint), server.socket.getPort()));
+        }
+
+        private void onTopologyChange(InetAddress endpoint, Event.TopologyChange event)
+        {
+            if (logger.isTraceEnabled())
+                logger.trace("Topology changed event : {}, {}", endpoint, event.change);
+
+            LatestEvent prev = latestEvents.get(endpoint);
+            if (prev == null || prev.topology != event.change)
+            {
+                LatestEvent ret = latestEvents.put(endpoint, LatestEvent.forTopologyChange(event.change, prev));
+                if (ret == prev)
+                    send(endpoint, event);
+            }
+        }
+
+        private void onStatusChange(InetAddress endpoint, Event.StatusChange event)
+        {
+            if (logger.isTraceEnabled())
+                logger.trace("Status changed event : {}, {}", endpoint, event.status);
+
+            LatestEvent prev = latestEvents.get(endpoint);
+            if (prev == null || prev.status != event.status)
+            {
+                LatestEvent ret = latestEvents.put(endpoint, LatestEvent.forStatusChange(event.status, null));
+                if (ret == prev)
+                    send(endpoint, event);
+            }
         }
 
         public void onCreateKeyspace(String ksName)
@@ -507,6 +571,18 @@
             send(new Event.SchemaChange(Event.SchemaChange.Change.CREATED, Event.SchemaChange.Target.TYPE, ksName, typeName));
         }
 
+        public void onCreateFunction(String ksName, String functionName, List<AbstractType<?>> argTypes)
+        {
+            send(new Event.SchemaChange(Event.SchemaChange.Change.CREATED, Event.SchemaChange.Target.FUNCTION,
+                                        ksName, functionName, AbstractType.asCQLTypeStringList(argTypes)));
+        }
+
+        public void onCreateAggregate(String ksName, String aggregateName, List<AbstractType<?>> argTypes)
+        {
+            send(new Event.SchemaChange(Event.SchemaChange.Change.CREATED, Event.SchemaChange.Target.AGGREGATE,
+                                        ksName, aggregateName, AbstractType.asCQLTypeStringList(argTypes)));
+        }
+
         public void onUpdateKeyspace(String ksName)
         {
             send(new Event.SchemaChange(Event.SchemaChange.Change.UPDATED, ksName));
@@ -522,6 +598,18 @@
             send(new Event.SchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.TYPE, ksName, typeName));
         }
 
+        public void onUpdateFunction(String ksName, String functionName, List<AbstractType<?>> argTypes)
+        {
+            send(new Event.SchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.FUNCTION,
+                                        ksName, functionName, AbstractType.asCQLTypeStringList(argTypes)));
+        }
+
+        public void onUpdateAggregate(String ksName, String aggregateName, List<AbstractType<?>> argTypes)
+        {
+            send(new Event.SchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.AGGREGATE,
+                                        ksName, aggregateName, AbstractType.asCQLTypeStringList(argTypes)));
+        }
+
         public void onDropKeyspace(String ksName)
         {
             send(new Event.SchemaChange(Event.SchemaChange.Change.DROPPED, ksName));
@@ -536,5 +624,17 @@
         {
             send(new Event.SchemaChange(Event.SchemaChange.Change.DROPPED, Event.SchemaChange.Target.TYPE, ksName, typeName));
         }
+
+        public void onDropFunction(String ksName, String functionName, List<AbstractType<?>> argTypes)
+        {
+            send(new Event.SchemaChange(Event.SchemaChange.Change.DROPPED, Event.SchemaChange.Target.FUNCTION,
+                                        ksName, functionName, AbstractType.asCQLTypeStringList(argTypes)));
+        }
+
+        public void onDropAggregate(String ksName, String aggregateName, List<AbstractType<?>> argTypes)
+        {
+            send(new Event.SchemaChange(Event.SchemaChange.Change.DROPPED, Event.SchemaChange.Target.AGGREGATE,
+                                        ksName, aggregateName, AbstractType.asCQLTypeStringList(argTypes)));
+        }
     }
 }
diff --git a/src/java/org/apache/cassandra/transport/ServerConnection.java b/src/java/org/apache/cassandra/transport/ServerConnection.java
index ce4d164..29691f5 100644
--- a/src/java/org/apache/cassandra/transport/ServerConnection.java
+++ b/src/java/org/apache/cassandra/transport/ServerConnection.java
@@ -21,10 +21,7 @@
 import java.util.concurrent.ConcurrentMap;
 
 import io.netty.channel.Channel;
-
 import org.apache.cassandra.auth.IAuthenticator;
-import org.apache.cassandra.auth.ISaslAwareAuthenticator;
-import org.apache.cassandra.auth.ISaslAwareAuthenticator.SaslAuthenticator;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
@@ -33,7 +30,7 @@
 {
     private enum State { UNINITIALIZED, AUTHENTICATION, READY }
 
-    private volatile SaslAuthenticator saslAuthenticator;
+    private volatile IAuthenticator.SaslNegotiator saslNegotiator;
     private final ClientState clientState;
     private volatile State state;
 
@@ -103,7 +100,7 @@
                 {
                     state = State.READY;
                     // we won't use the authenticator again, null it so that it can be GC'd
-                    saslAuthenticator = null;
+                    saslNegotiator = null;
                 }
                 break;
             case READY:
@@ -113,14 +110,10 @@
         }
     }
 
-    public SaslAuthenticator getAuthenticator()
+    public IAuthenticator.SaslNegotiator getSaslNegotiator()
     {
-        if (saslAuthenticator == null)
-        {
-            IAuthenticator authenticator = DatabaseDescriptor.getAuthenticator();
-            assert authenticator instanceof ISaslAwareAuthenticator : "Configured IAuthenticator does not support SASL authentication";
-            saslAuthenticator = ((ISaslAwareAuthenticator)authenticator).newAuthenticator();
-        }
-        return saslAuthenticator;
+        if (saslNegotiator == null)
+            saslNegotiator = DatabaseDescriptor.getAuthenticator().newSaslNegotiator();
+        return saslNegotiator;
     }
 }
diff --git a/src/java/org/apache/cassandra/transport/SimpleClient.java b/src/java/org/apache/cassandra/transport/SimpleClient.java
index 3a6ecf7..4759c2a 100644
--- a/src/java/org/apache/cassandra/transport/SimpleClient.java
+++ b/src/java/org/apache/cassandra/transport/SimpleClient.java
@@ -17,6 +17,7 @@
  */
 package org.apache.cassandra.transport;
 
+import java.io.Closeable;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
@@ -25,7 +26,7 @@
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.SynchronousQueue;
 import javax.net.ssl.SSLContext;
 import javax.net.ssl.SSLEngine;
@@ -39,7 +40,6 @@
 import io.netty.channel.ChannelOption;
 import io.netty.channel.SimpleChannelInboundHandler;
 import io.netty.channel.nio.NioEventLoopGroup;
-import io.netty.channel.socket.nio.NioServerSocketChannel;
 import io.netty.util.internal.logging.InternalLoggerFactory;
 import io.netty.util.internal.logging.Slf4JLoggerFactory;
 import org.apache.cassandra.cql3.QueryOptions;
@@ -47,6 +47,7 @@
 import org.apache.cassandra.security.SSLFactory;
 import org.apache.cassandra.transport.messages.CredentialsMessage;
 import org.apache.cassandra.transport.messages.ErrorMessage;
+import org.apache.cassandra.transport.messages.EventMessage;
 import org.apache.cassandra.transport.messages.ExecuteMessage;
 import org.apache.cassandra.transport.messages.PrepareMessage;
 import org.apache.cassandra.transport.messages.QueryMessage;
@@ -60,7 +61,7 @@
 import io.netty.handler.ssl.SslHandler;
 import static org.apache.cassandra.config.EncryptionOptions.ClientEncryptionOptions;
 
-public class SimpleClient
+public class SimpleClient implements Closeable
 {
     static
     {
@@ -74,8 +75,9 @@
 
     protected final ResponseHandler responseHandler = new ResponseHandler();
     protected final Connection.Tracker tracker = new ConnectionTracker();
+    protected final int version;
     // We don't track connection really, so we don't need one Connection per channel
-    protected final Connection connection = new Connection(null, Server.CURRENT_VERSION, tracker);
+    protected Connection connection;
     protected Bootstrap bootstrap;
     protected Channel channel;
     protected ChannelFuture lastWriteFuture;
@@ -84,18 +86,28 @@
     {
         public Connection newConnection(Channel channel, int version)
         {
-            assert version == Server.CURRENT_VERSION;
             return connection;
         }
     };
 
-    public SimpleClient(String host, int port, ClientEncryptionOptions encryptionOptions)
+    public SimpleClient(String host, int port, int version, ClientEncryptionOptions encryptionOptions)
     {
         this.host = host;
         this.port = port;
+        this.version = version;
         this.encryptionOptions = encryptionOptions;
     }
 
+    public SimpleClient(String host, int port, ClientEncryptionOptions encryptionOptions)
+    {
+        this(host, port, Server.CURRENT_VERSION, encryptionOptions);
+    }
+
+    public SimpleClient(String host, int port, int version)
+    {
+        this(host, port, version, new ClientEncryptionOptions());
+    }
+
     public SimpleClient(String host, int port)
     {
         this(host, port, new ClientEncryptionOptions());
@@ -105,7 +117,7 @@
     {
         establishConnection();
 
-        Map<String, String> options = new HashMap<String, String>();
+        Map<String, String> options = new HashMap<>();
         options.put(StartupMessage.CQL_VERSION, "3.0.0");
         if (useCompression)
         {
@@ -115,6 +127,11 @@
         execute(new StartupMessage(options));
     }
 
+    public void setEventHandler(EventHandler eventHandler)
+    {
+        responseHandler.eventHandler = eventHandler;
+    }
+
     protected void establishConnection() throws IOException
     {
         // Configure the client.
@@ -190,7 +207,7 @@
         bootstrap.group().shutdownGracefully();
     }
 
-    protected Message.Response execute(Message.Request request)
+    public Message.Response execute(Message.Request request)
     {
         try
         {
@@ -207,6 +224,21 @@
         }
     }
 
+    public interface EventHandler
+    {
+        void onEvent(Event event);
+    }
+
+    public static class SimpleEventHandler implements EventHandler
+    {
+        public final LinkedBlockingQueue<Event> queue = new LinkedBlockingQueue<>();
+
+        public void onEvent(Event event)
+        {
+            queue.add(event);
+        }
+    }
+
     // Stateless handlers
     private static final Message.ProtocolDecoder messageDecoder = new Message.ProtocolDecoder();
     private static final Message.ProtocolEncoder messageEncoder = new Message.ProtocolEncoder();
@@ -217,13 +249,20 @@
     private static class ConnectionTracker implements Connection.Tracker
     {
         public void addConnection(Channel ch, Connection connection) {}
-        public void closeAll() {}
+
+        public boolean isRegistered(Event.Type type, Channel ch)
+        {
+            return false;
+        }
     }
 
     private class Initializer extends ChannelInitializer<Channel>
     {
         protected void initChannel(Channel channel) throws Exception
         {
+            connection = new Connection(channel, version, tracker);
+            channel.attr(Connection.attributeKey).set(connection);
+
             ChannelPipeline pipeline = channel.pipeline();
             pipeline.addLast("frameDecoder", new Frame.Decoder(connectionFactory));
             pipeline.addLast("frameEncoder", frameEncoder);
@@ -252,7 +291,8 @@
             super.initChannel(channel);
             SSLEngine sslEngine = sslContext.createSSLEngine();
             sslEngine.setUseClientMode(true);
-            sslEngine.setEnabledCipherSuites(encryptionOptions.cipher_suites);
+            String[] suites = SSLFactory.filterCipherSuites(sslEngine.getSupportedCipherSuites(), encryptionOptions.cipher_suites);
+            sslEngine.setEnabledCipherSuites(suites);
             sslEngine.setEnabledProtocols(SSLFactory.ACCEPTED_PROTOCOLS);
             channel.pipeline().addFirst("ssl", new SslHandler(sslEngine));
         }
@@ -261,14 +301,21 @@
     @ChannelHandler.Sharable
     private static class ResponseHandler extends SimpleChannelInboundHandler<Message.Response>
     {
-        public final BlockingQueue<Message.Response> responses = new SynchronousQueue<Message.Response>(true);
+        public final BlockingQueue<Message.Response> responses = new SynchronousQueue<>(true);
+        public EventHandler eventHandler;
 
         @Override
         public void channelRead0(ChannelHandlerContext ctx, Message.Response r)
         {
             try
             {
-                responses.put(r);
+                if (r instanceof EventMessage)
+                {
+                    if (eventHandler != null)
+                        eventHandler.onEvent(((EventMessage) r).event);
+                }
+                else
+                    responses.put(r);
             }
             catch (InterruptedException ie)
             {
diff --git a/src/java/org/apache/cassandra/transport/messages/AuthResponse.java b/src/java/org/apache/cassandra/transport/messages/AuthResponse.java
index 3f3f774..257a26a 100644
--- a/src/java/org/apache/cassandra/transport/messages/AuthResponse.java
+++ b/src/java/org/apache/cassandra/transport/messages/AuthResponse.java
@@ -17,22 +17,18 @@
  */
 package org.apache.cassandra.transport.messages;
 
-import org.apache.cassandra.auth.AuthenticatedUser;
-import org.apache.cassandra.auth.ISaslAwareAuthenticator.SaslAuthenticator;
-import org.apache.cassandra.exceptions.AuthenticationException;
-import org.apache.cassandra.service.QueryState;
-import org.apache.cassandra.transport.CBUtil;
-import org.apache.cassandra.transport.Message;
-import org.apache.cassandra.transport.ProtocolException;
-import org.apache.cassandra.transport.ServerConnection;
+import java.nio.ByteBuffer;
 
 import io.netty.buffer.ByteBuf;
-
-import java.nio.ByteBuffer;
+import org.apache.cassandra.auth.AuthenticatedUser;
+import org.apache.cassandra.auth.IAuthenticator;
+import org.apache.cassandra.exceptions.AuthenticationException;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.transport.*;
 
 /**
  * A SASL token message sent from client to server. Some SASL
- * mechanisms & clients may send an initial token before
+ * mechanisms and clients may send an initial token before
  * receiving a challenge from the server.
  */
 public class AuthResponse extends Message.Request
@@ -61,11 +57,12 @@
         }
     };
 
-    private byte[] token;
+    private final byte[] token;
 
     public AuthResponse(byte[] token)
     {
         super(Message.Type.AUTH_RESPONSE);
+        assert token != null;
         this.token = token;
     }
 
@@ -74,11 +71,11 @@
     {
         try
         {
-            SaslAuthenticator authenticator = ((ServerConnection) connection).getAuthenticator();
-            byte[] challenge = authenticator.evaluateResponse(token == null ? new byte[0] : token);
-            if (authenticator.isComplete())
+            IAuthenticator.SaslNegotiator negotiator = ((ServerConnection) connection).getSaslNegotiator();
+            byte[] challenge = negotiator.evaluateResponse(token);
+            if (negotiator.isComplete())
             {
-                AuthenticatedUser user = authenticator.getAuthenticatedUser();
+                AuthenticatedUser user = negotiator.getAuthenticatedUser();
                 queryState.getClientState().login(user);
                 // authentication is complete, send a ready message to the client
                 return new AuthSuccess(challenge);
diff --git a/src/java/org/apache/cassandra/transport/messages/BatchMessage.java b/src/java/org/apache/cassandra/transport/messages/BatchMessage.java
index b34b156..d86bb1a 100644
--- a/src/java/org/apache/cassandra/transport/messages/BatchMessage.java
+++ b/src/java/org/apache/cassandra/transport/messages/BatchMessage.java
@@ -50,8 +50,8 @@
 
             byte type = body.readByte();
             int n = body.readUnsignedShort();
-            List<Object> queryOrIds = new ArrayList<Object>(n);
-            List<List<ByteBuffer>> variables = new ArrayList<List<ByteBuffer>>(n);
+            List<Object> queryOrIds = new ArrayList<>(n);
+            List<List<ByteBuffer>> variables = new ArrayList<>(n);
             for (int i = 0; i < n; i++)
             {
                 byte kind = body.readByte();
@@ -61,7 +61,7 @@
                     queryOrIds.add(MD5Digest.wrap(CBUtil.readBytes(body)));
                 else
                     throw new ProtocolException("Invalid query kind in BATCH messages. Must be 0 or 1 but got " + kind);
-                variables.add(CBUtil.readValueList(body));
+                variables.add(CBUtil.readValueList(body, version));
             }
             QueryOptions options = version < 3
                                  ? QueryOptions.fromPreV3Batch(CBUtil.readConsistencyLevel(body))
@@ -74,7 +74,7 @@
         {
             int queries = msg.queryOrIdList.size();
 
-            dest.writeByte(fromType(msg.type));
+            dest.writeByte(fromType(msg.batchType));
             dest.writeShort(queries);
 
             for (int i = 0; i < queries; i++)
@@ -138,7 +138,7 @@
         }
     };
 
-    public final BatchStatement.Type type;
+    public final BatchStatement.Type batchType;
     public final List<Object> queryOrIdList;
     public final List<List<ByteBuffer>> values;
     public final QueryOptions options;
@@ -146,7 +146,7 @@
     public BatchMessage(BatchStatement.Type type, List<Object> queryOrIdList, List<List<ByteBuffer>> values, QueryOptions options)
     {
         super(Message.Type.BATCH);
-        this.type = type;
+        this.batchType = type;
         this.queryOrIdList = queryOrIdList;
         this.values = values;
         this.options = options;
@@ -174,7 +174,7 @@
                     builder.put("serial_consistency_level", options.getSerialConsistency().name());
 
                 // TODO we don't have [typed] access to CQL bind variables here.  CASSANDRA-4560 is open to add support.
-                Tracing.instance.begin("Execute batch of CQL3 queries", builder.build());
+                Tracing.instance.begin("Execute batch of CQL3 queries", state.getClientAddress(), builder.build());
             }
 
             QueryHandler handler = ClientState.getCQLQueryHandler();
@@ -208,7 +208,7 @@
             for (int i = 0; i < prepared.size(); i++)
             {
                 ParsedStatement.Prepared p = prepared.get(i);
-                batchOptions.forStatement(i).prepare(p.boundNames);
+                batchOptions.prepareStatement(i, p.boundNames);
 
                 if (!(p.statement instanceof ModificationStatement))
                     throw new InvalidRequestException("Invalid statement in batch: only UPDATE, INSERT and DELETE statements are allowed.");
@@ -218,8 +218,8 @@
 
             // Note: It's ok at this point to pass a bogus value for the number of bound terms in the BatchState ctor
             // (and no value would be really correct, so we prefer passing a clearly wrong one).
-            BatchStatement batch = new BatchStatement(-1, type, statements, Attributes.none());
-            Message.Response response = handler.processBatch(batch, state, batchOptions);
+            BatchStatement batch = new BatchStatement(-1, batchType, statements, Attributes.none());
+            Message.Response response = handler.processBatch(batch, state, batchOptions, getCustomPayload());
 
             if (tracingId != null)
                 response.setTracingId(tracingId);
diff --git a/src/java/org/apache/cassandra/transport/messages/CredentialsMessage.java b/src/java/org/apache/cassandra/transport/messages/CredentialsMessage.java
index a9e098f..fc959ab 100644
--- a/src/java/org/apache/cassandra/transport/messages/CredentialsMessage.java
+++ b/src/java/org/apache/cassandra/transport/messages/CredentialsMessage.java
@@ -20,15 +20,14 @@
 import java.util.HashMap;
 import java.util.Map;
 
+import io.netty.buffer.ByteBuf;
 import org.apache.cassandra.auth.AuthenticatedUser;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.transport.ProtocolException;
-import io.netty.buffer.ByteBuf;
-
 import org.apache.cassandra.exceptions.AuthenticationException;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.transport.CBUtil;
 import org.apache.cassandra.transport.Message;
+import org.apache.cassandra.transport.ProtocolException;
 
 /**
  * Message to indicate that the server is ready to receive requests.
@@ -75,14 +74,15 @@
     {
         try
         {
-            AuthenticatedUser user = DatabaseDescriptor.getAuthenticator().authenticate(credentials);
+            AuthenticatedUser user = DatabaseDescriptor.getAuthenticator().legacyAuthenticate(credentials);
             state.getClientState().login(user);
-            return new ReadyMessage();
         }
         catch (AuthenticationException e)
         {
             return ErrorMessage.fromException(e);
         }
+
+        return new ReadyMessage();
     }
 
     @Override
diff --git a/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java b/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java
index 01c0e2e..021db5a 100644
--- a/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java
+++ b/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java
@@ -17,20 +17,20 @@
  */
 package org.apache.cassandra.transport.messages;
 
-import com.google.common.annotations.VisibleForTesting;
+import java.util.List;
+
 import io.netty.buffer.ByteBuf;
 import io.netty.handler.codec.CodecException;
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Predicate;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.cassandra.cql3.functions.FunctionName;
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.WriteType;
 import org.apache.cassandra.exceptions.*;
-import org.apache.cassandra.transport.CBUtil;
-import org.apache.cassandra.transport.Message;
-import org.apache.cassandra.transport.ProtocolException;
-import org.apache.cassandra.transport.ServerError;
+import org.apache.cassandra.transport.*;
 import org.apache.cassandra.utils.MD5Digest;
 
 /**
@@ -76,6 +76,25 @@
                 case TRUNCATE_ERROR:
                     te = new TruncateException(msg);
                     break;
+                case WRITE_FAILURE: 
+                case READ_FAILURE:
+                    {
+                        ConsistencyLevel cl = CBUtil.readConsistencyLevel(body);
+                        int received = body.readInt();
+                        int blockFor = body.readInt();
+                        int failure = body.readInt();
+                        if (code == ExceptionCode.WRITE_FAILURE)
+                        {
+                            WriteType writeType = Enum.valueOf(WriteType.class, CBUtil.readString(body));
+                            te = new WriteFailureException(cl, received, failure, blockFor, writeType);
+                        }
+                        else
+                        {
+                            byte dataPresent = body.readByte();
+                            te = new ReadFailureException(cl, received, failure, blockFor, dataPresent != 0);   
+                        }
+                    }
+                    break;
                 case WRITE_TIMEOUT:
                 case READ_TIMEOUT:
                     ConsistencyLevel cl = CBUtil.readConsistencyLevel(body);
@@ -92,6 +111,12 @@
                         te = new ReadTimeoutException(cl, received, blockFor, dataPresent != 0);
                     }
                     break;
+                case FUNCTION_FAILURE:
+                    String fKeyspace = CBUtil.readString(body);
+                    String fName = CBUtil.readString(body);
+                    List<String> argTypes = CBUtil.readStringList(body);
+                    te = new FunctionExecutionException(new FunctionName(fKeyspace, fName), argTypes, msg);
+                    break;
                 case UNPREPARED:
                     {
                         MD5Digest id = MD5Digest.wrap(CBUtil.readBytes(body));
@@ -124,21 +149,40 @@
 
         public void encode(ErrorMessage msg, ByteBuf dest, int version)
         {
-            dest.writeInt(msg.error.code().value);
-            CBUtil.writeString(msg.error.getMessage(), dest);
+            final TransportException err = getBackwardsCompatibleException(msg, version);
+            dest.writeInt(err.code().value);
+            String errorString = err.getMessage() == null ? "" : err.getMessage();
+            CBUtil.writeString(errorString, dest);
 
-            switch (msg.error.code())
+            switch (err.code())
             {
                 case UNAVAILABLE:
-                    UnavailableException ue = (UnavailableException)msg.error;
+                    UnavailableException ue = (UnavailableException)err;
                     CBUtil.writeConsistencyLevel(ue.consistency, dest);
                     dest.writeInt(ue.required);
                     dest.writeInt(ue.alive);
                     break;
+                case WRITE_FAILURE:
+                case READ_FAILURE:
+                    {
+                        RequestFailureException rfe = (RequestFailureException)err;
+                        boolean isWrite = err.code() == ExceptionCode.WRITE_FAILURE;
+
+                        CBUtil.writeConsistencyLevel(rfe.consistency, dest);
+                        dest.writeInt(rfe.received);
+                        dest.writeInt(rfe.blockFor);
+                        dest.writeInt(rfe.failures);
+
+                        if (isWrite)
+                            CBUtil.writeString(((WriteFailureException)rfe).writeType.toString(), dest);
+                        else
+                            dest.writeByte((byte)(((ReadFailureException)rfe).dataPresent ? 1 : 0));
+                    }
+                    break;
                 case WRITE_TIMEOUT:
                 case READ_TIMEOUT:
-                    RequestTimeoutException rte = (RequestTimeoutException)msg.error;
-                    boolean isWrite = msg.error.code() == ExceptionCode.WRITE_TIMEOUT;
+                    RequestTimeoutException rte = (RequestTimeoutException)err;
+                    boolean isWrite = err.code() == ExceptionCode.WRITE_TIMEOUT;
 
                     CBUtil.writeConsistencyLevel(rte.consistency, dest);
                     dest.writeInt(rte.received);
@@ -148,12 +192,18 @@
                     else
                         dest.writeByte((byte)(((ReadTimeoutException)rte).dataPresent ? 1 : 0));
                     break;
+                case FUNCTION_FAILURE:
+                    FunctionExecutionException fee = (FunctionExecutionException)msg.error;
+                    CBUtil.writeString(fee.functionName.keyspace, dest);
+                    CBUtil.writeString(fee.functionName.name, dest);
+                    CBUtil.writeStringList(fee.argTypes, dest);
+                    break;
                 case UNPREPARED:
-                    PreparedQueryNotFoundException pqnfe = (PreparedQueryNotFoundException)msg.error;
+                    PreparedQueryNotFoundException pqnfe = (PreparedQueryNotFoundException)err;
                     CBUtil.writeBytes(pqnfe.id.bytes, dest);
                     break;
                 case ALREADY_EXISTS:
-                    AlreadyExistsException aee = (AlreadyExistsException)msg.error;
+                    AlreadyExistsException aee = (AlreadyExistsException)err;
                     CBUtil.writeString(aee.ksName, dest);
                     CBUtil.writeString(aee.cfName, dest);
                     break;
@@ -162,26 +212,43 @@
 
         public int encodedSize(ErrorMessage msg, int version)
         {
-            int size = 4 + CBUtil.sizeOfString(msg.error.getMessage());
-            switch (msg.error.code())
+            final TransportException err = getBackwardsCompatibleException(msg, version);
+            String errorString = err.getMessage() == null ? "" : err.getMessage();
+            int size = 4 + CBUtil.sizeOfString(errorString);
+            switch (err.code())
             {
                 case UNAVAILABLE:
-                    UnavailableException ue = (UnavailableException)msg.error;
+                    UnavailableException ue = (UnavailableException)err;
                     size += CBUtil.sizeOfConsistencyLevel(ue.consistency) + 8;
                     break;
+                case WRITE_FAILURE:
+                case READ_FAILURE:
+                    {
+                        RequestFailureException rfe = (RequestFailureException)err;
+                        boolean isWrite = err.code() == ExceptionCode.WRITE_FAILURE;
+                        size += CBUtil.sizeOfConsistencyLevel(rfe.consistency) + 4 + 4 + 4;
+                        size += isWrite ? CBUtil.sizeOfString(((WriteFailureException)rfe).writeType.toString()) : 1;
+                    }
+                    break;
                 case WRITE_TIMEOUT:
                 case READ_TIMEOUT:
-                    RequestTimeoutException rte = (RequestTimeoutException)msg.error;
-                    boolean isWrite = msg.error.code() == ExceptionCode.WRITE_TIMEOUT;
+                    RequestTimeoutException rte = (RequestTimeoutException)err;
+                    boolean isWrite = err.code() == ExceptionCode.WRITE_TIMEOUT;
                     size += CBUtil.sizeOfConsistencyLevel(rte.consistency) + 8;
                     size += isWrite ? CBUtil.sizeOfString(((WriteTimeoutException)rte).writeType.toString()) : 1;
                     break;
+                case FUNCTION_FAILURE:
+                    FunctionExecutionException fee = (FunctionExecutionException)msg.error;
+                    size += CBUtil.sizeOfString(fee.functionName.keyspace);
+                    size += CBUtil.sizeOfString(fee.functionName.name);
+                    size += CBUtil.sizeOfStringList(fee.argTypes);
+                    break;
                 case UNPREPARED:
-                    PreparedQueryNotFoundException pqnfe = (PreparedQueryNotFoundException)msg.error;
+                    PreparedQueryNotFoundException pqnfe = (PreparedQueryNotFoundException)err;
                     size += CBUtil.sizeOfBytes(pqnfe.id.bytes);
                     break;
                 case ALREADY_EXISTS:
-                    AlreadyExistsException aee = (AlreadyExistsException)msg.error;
+                    AlreadyExistsException aee = (AlreadyExistsException)err;
                     size += CBUtil.sizeOfString(aee.ksName);
                     size += CBUtil.sizeOfString(aee.cfName);
                     break;
@@ -190,6 +257,26 @@
         }
     };
 
+    private static TransportException getBackwardsCompatibleException(ErrorMessage msg, int version)
+    {
+        if (version < Server.VERSION_4)
+        {
+            switch (msg.error.code())
+            {
+                case READ_FAILURE:
+                    ReadFailureException rfe = (ReadFailureException) msg.error;
+                    return new ReadTimeoutException(rfe.consistency, rfe.received, rfe.blockFor, rfe.dataPresent);
+                case WRITE_FAILURE:
+                    WriteFailureException wfe = (WriteFailureException) msg.error;
+                    return new WriteTimeoutException(wfe.writeType, wfe.consistency, wfe.received, wfe.blockFor);
+                case FUNCTION_FAILURE:
+                    return new InvalidRequestException(msg.toString());
+            }
+        }
+
+        return msg.error;
+    }
+
     // We need to figure error codes out (#3979)
     public final TransportException error;
 
diff --git a/src/java/org/apache/cassandra/transport/messages/ExecuteMessage.java b/src/java/org/apache/cassandra/transport/messages/ExecuteMessage.java
index 7dd5f83..11a227c 100644
--- a/src/java/org/apache/cassandra/transport/messages/ExecuteMessage.java
+++ b/src/java/org/apache/cassandra/transport/messages/ExecuteMessage.java
@@ -30,6 +30,7 @@
 import org.apache.cassandra.cql3.statements.ParsedStatement;
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.exceptions.PreparedQueryNotFoundException;
+import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.transport.*;
@@ -46,7 +47,7 @@
             byte[] id = CBUtil.readBytes(body);
             if (version == 1)
             {
-                List<ByteBuffer> values = CBUtil.readValueList(body);
+                List<ByteBuffer> values = CBUtil.readValueList(body, version);
                 ConsistencyLevel consistency = CBUtil.readConsistencyLevel(body);
                 return new ExecuteMessage(MD5Digest.wrap(id), QueryOptions.fromProtocolV1(consistency, values));
             }
@@ -101,7 +102,7 @@
     {
         try
         {
-            QueryHandler handler = state.getClientState().getCQLQueryHandler();
+            QueryHandler handler = ClientState.getCQLQueryHandler();
             ParsedStatement.Prepared prepared = handler.getPrepared(statementId);
             if (prepared == null)
                 throw new PreparedQueryNotFoundException(statementId);
@@ -132,10 +133,13 @@
                     builder.put("serial_consistency_level", options.getSerialConsistency().name());
 
                 // TODO we don't have [typed] access to CQL bind variables here.  CASSANDRA-4560 is open to add support.
-                Tracing.instance.begin("Execute CQL3 prepared query", builder.build());
+                Tracing.instance.begin("Execute CQL3 prepared query", state.getClientAddress(), builder.build());
             }
 
-            Message.Response response = handler.processPrepared(statement, state, options);
+            // Some custom QueryHandlers are interested by the bound names. We provide them this information
+            // by wrapping the QueryOptions.
+            QueryOptions queryOptions = QueryOptions.addColumnSpecifications(options, prepared.boundNames);
+            Message.Response response = handler.processPrepared(statement, state, queryOptions, getCustomPayload());
             if (options.skipMetadata() && response instanceof ResultMessage.Rows)
                 ((ResultMessage.Rows)response).result.metadata.setSkipMetadata();
 
diff --git a/src/java/org/apache/cassandra/transport/messages/PrepareMessage.java b/src/java/org/apache/cassandra/transport/messages/PrepareMessage.java
index 221b9ca..f54d1d9 100644
--- a/src/java/org/apache/cassandra/transport/messages/PrepareMessage.java
+++ b/src/java/org/apache/cassandra/transport/messages/PrepareMessage.java
@@ -22,7 +22,7 @@
 import com.google.common.collect.ImmutableMap;
 import io.netty.buffer.ByteBuf;
 
-import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.transport.*;
@@ -72,10 +72,10 @@
             if (state.traceNextQuery())
             {
                 state.createTracingSession();
-                Tracing.instance.begin("Preparing CQL3 query", ImmutableMap.of("query", query));
+                Tracing.instance.begin("Preparing CQL3 query", state.getClientAddress(), ImmutableMap.of("query", query));
             }
 
-            Message.Response response = state.getClientState().getCQLQueryHandler().prepare(query, state);
+            Message.Response response = ClientState.getCQLQueryHandler().prepare(query, state, getCustomPayload());
 
             if (tracingId != null)
                 response.setTracingId(tracingId);
diff --git a/src/java/org/apache/cassandra/transport/messages/QueryMessage.java b/src/java/org/apache/cassandra/transport/messages/QueryMessage.java
index 2dd98cc..6d22dd1 100644
--- a/src/java/org/apache/cassandra/transport/messages/QueryMessage.java
+++ b/src/java/org/apache/cassandra/transport/messages/QueryMessage.java
@@ -27,6 +27,7 @@
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.exceptions.*;
+import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.transport.*;
@@ -116,10 +117,10 @@
                 if(options.getSerialConsistency() != null)
                     builder.put("serial_consistency_level", options.getSerialConsistency().name());
 
-                Tracing.instance.begin("Execute CQL3 query", builder.build());
+                Tracing.instance.begin("Execute CQL3 query", state.getClientAddress(), builder.build());
             }
 
-            Message.Response response = state.getClientState().getCQLQueryHandler().process(query, state, options);
+            Message.Response response = ClientState.getCQLQueryHandler().process(query, state, options, getCustomPayload());
             if (options.skipMetadata() && response instanceof ResultMessage.Rows)
                 ((ResultMessage.Rows)response).result.metadata.setSkipMetadata();
 
diff --git a/src/java/org/apache/cassandra/transport/messages/RegisterMessage.java b/src/java/org/apache/cassandra/transport/messages/RegisterMessage.java
index ee410bb..928e676 100644
--- a/src/java/org/apache/cassandra/transport/messages/RegisterMessage.java
+++ b/src/java/org/apache/cassandra/transport/messages/RegisterMessage.java
@@ -32,7 +32,7 @@
         public RegisterMessage decode(ByteBuf body, int version)
         {
             int length = body.readUnsignedShort();
-            List<Event.Type> eventTypes = new ArrayList<Event.Type>(length);
+            List<Event.Type> eventTypes = new ArrayList<>(length);
             for (int i = 0; i < length; ++i)
                 eventTypes.add(CBUtil.readEnumValue(Event.Type.class, body));
             return new RegisterMessage(eventTypes);
@@ -65,10 +65,14 @@
     public Response execute(QueryState state)
     {
         assert connection instanceof ServerConnection;
-        Connection.Tracker tracker = ((ServerConnection)connection).getTracker();
+        Connection.Tracker tracker = connection.getTracker();
         assert tracker instanceof Server.ConnectionTracker;
         for (Event.Type type : eventTypes)
-            ((Server.ConnectionTracker)tracker).register(type, connection().channel());
+        {
+            if (type.minimumVersion > connection.getVersion())
+                throw new ProtocolException("Event " + type.name() + " not valid for protocol version " + connection.getVersion());
+            ((Server.ConnectionTracker) tracker).register(type, connection().channel());
+        }
         return new ReadyMessage();
     }
 
diff --git a/src/java/org/apache/cassandra/transport/messages/ResultMessage.java b/src/java/org/apache/cassandra/transport/messages/ResultMessage.java
index 723beed..b76243f 100644
--- a/src/java/org/apache/cassandra/transport/messages/ResultMessage.java
+++ b/src/java/org/apache/cassandra/transport/messages/ResultMessage.java
@@ -26,7 +26,6 @@
 import org.apache.cassandra.cql3.ResultSet;
 import org.apache.cassandra.cql3.statements.SelectStatement;
 import org.apache.cassandra.cql3.statements.ParsedStatement;
-import org.apache.cassandra.service.pager.PagingState;
 import org.apache.cassandra.transport.*;
 import org.apache.cassandra.thrift.CqlPreparedResult;
 import org.apache.cassandra.thrift.CqlResult;
@@ -239,11 +238,11 @@
             public ResultMessage decode(ByteBuf body, int version)
             {
                 MD5Digest id = MD5Digest.wrap(CBUtil.readBytes(body));
-                ResultSet.Metadata metadata = ResultSet.Metadata.codec.decode(body, version);
+                ResultSet.PreparedMetadata metadata = ResultSet.PreparedMetadata.codec.decode(body, version);
 
-                ResultSet.Metadata resultMetadata = ResultSet.Metadata.EMPTY;
+                ResultSet.ResultMetadata resultMetadata = ResultSet.ResultMetadata.EMPTY;
                 if (version > 1)
-                    resultMetadata = ResultSet.Metadata.codec.decode(body, version);
+                    resultMetadata = ResultSet.ResultMetadata.codec.decode(body, version);
 
                 return new Prepared(id, -1, metadata, resultMetadata);
             }
@@ -255,9 +254,9 @@
                 assert prepared.statementId != null;
 
                 CBUtil.writeBytes(prepared.statementId.bytes, dest);
-                ResultSet.Metadata.codec.encode(prepared.metadata, dest, version);
+                ResultSet.PreparedMetadata.codec.encode(prepared.metadata, dest, version);
                 if (version > 1)
-                    ResultSet.Metadata.codec.encode(prepared.resultMetadata, dest, version);
+                    ResultSet.ResultMetadata.codec.encode(prepared.resultMetadata, dest, version);
             }
 
             public int encodedSize(ResultMessage msg, int version)
@@ -268,9 +267,9 @@
 
                 int size = 0;
                 size += CBUtil.sizeOfBytes(prepared.statementId.bytes);
-                size += ResultSet.Metadata.codec.encodedSize(prepared.metadata, version);
+                size += ResultSet.PreparedMetadata.codec.encodedSize(prepared.metadata, version);
                 if (version > 1)
-                    size += ResultSet.Metadata.codec.encodedSize(prepared.resultMetadata, version);
+                    size += ResultSet.ResultMetadata.codec.encodedSize(prepared.resultMetadata, version);
                 return size;
             }
         };
@@ -278,25 +277,25 @@
         public final MD5Digest statementId;
 
         /** Describes the variables to be bound in the prepared statement */
-        public final ResultSet.Metadata metadata;
+        public final ResultSet.PreparedMetadata metadata;
 
         /** Describes the results of executing this prepared statement */
-        public final ResultSet.Metadata resultMetadata;
+        public final ResultSet.ResultMetadata resultMetadata;
 
         // statement id for CQL-over-thrift compatibility. The binary protocol ignore that.
         private final int thriftStatementId;
 
         public Prepared(MD5Digest statementId, ParsedStatement.Prepared prepared)
         {
-            this(statementId, -1, new ResultSet.Metadata(prepared.boundNames), extractResultMetadata(prepared.statement));
+            this(statementId, -1, new ResultSet.PreparedMetadata(prepared.boundNames, prepared.partitionKeyBindIndexes), extractResultMetadata(prepared.statement));
         }
 
         public static Prepared forThrift(int statementId, List<ColumnSpecification> names)
         {
-            return new Prepared(null, statementId, new ResultSet.Metadata(names), ResultSet.Metadata.EMPTY);
+            return new Prepared(null, statementId, new ResultSet.PreparedMetadata(names, null), ResultSet.ResultMetadata.EMPTY);
         }
 
-        private Prepared(MD5Digest statementId, int thriftStatementId, ResultSet.Metadata metadata, ResultSet.Metadata resultMetadata)
+        private Prepared(MD5Digest statementId, int thriftStatementId, ResultSet.PreparedMetadata metadata, ResultSet.ResultMetadata resultMetadata)
         {
             super(Kind.PREPARED);
             this.statementId = statementId;
@@ -305,10 +304,10 @@
             this.resultMetadata = resultMetadata;
         }
 
-        private static ResultSet.Metadata extractResultMetadata(CQLStatement statement)
+        private static ResultSet.ResultMetadata extractResultMetadata(CQLStatement statement)
         {
             if (!(statement instanceof SelectStatement))
-                return ResultSet.Metadata.EMPTY;
+                return ResultSet.ResultMetadata.EMPTY;
 
             return ((SelectStatement)statement).getResultMetadata();
         }
diff --git a/src/java/org/apache/cassandra/transport/messages/StartupMessage.java b/src/java/org/apache/cassandra/transport/messages/StartupMessage.java
index 1a5071f..04d8e62 100644
--- a/src/java/org/apache/cassandra/transport/messages/StartupMessage.java
+++ b/src/java/org/apache/cassandra/transport/messages/StartupMessage.java
@@ -23,11 +23,9 @@
 import io.netty.buffer.ByteBuf;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.transport.*;
-import org.apache.cassandra.utils.SemanticVersion;
+import org.apache.cassandra.utils.CassandraVersion;
 
 /**
  * The initial message of the protocol.
@@ -66,23 +64,20 @@
 
     public Message.Response execute(QueryState state)
     {
-        ClientState cState = state.getClientState();
         String cqlVersion = options.get(CQL_VERSION);
         if (cqlVersion == null)
             throw new ProtocolException("Missing value CQL_VERSION in STARTUP message");
 
         try 
         {
-            cState.setCQLVersion(cqlVersion);
+            if (new CassandraVersion(cqlVersion).compareTo(new CassandraVersion("2.99.0")) < 0)
+                throw new ProtocolException(String.format("CQL version %s is not supported by the binary protocol (supported version are >= 3.0.0)", cqlVersion));
         }
-        catch (InvalidRequestException e)
+        catch (IllegalArgumentException e)
         {
             throw new ProtocolException(e.getMessage());
         }
 
-        if (cState.getCQLVersion().compareTo(new SemanticVersion("2.99.0")) < 0)
-            throw new ProtocolException(String.format("CQL version %s is not supported by the binary protocol (supported version are >= 3.0.0)", cqlVersion));
-
         if (options.containsKey(COMPRESSION))
         {
             String compression = options.get(COMPRESSION).toLowerCase();
diff --git a/src/java/org/apache/cassandra/triggers/CustomClassLoader.java b/src/java/org/apache/cassandra/triggers/CustomClassLoader.java
index ecb34a2..965da4b 100644
--- a/src/java/org/apache/cassandra/triggers/CustomClassLoader.java
+++ b/src/java/org/apache/cassandra/triggers/CustomClassLoader.java
@@ -112,7 +112,7 @@
         }
         catch (ClassNotFoundException ex)
         {
-            logger.debug("Class not found using parent class loader,", ex);
+            logger.trace("Class not found using parent class loader,", ex);
             // Don't throw the exception here, try triggers directory.
         }
         Class<?> clazz = this.findClass(name);
diff --git a/src/java/org/apache/cassandra/triggers/TriggerExecutor.java b/src/java/org/apache/cassandra/triggers/TriggerExecutor.java
index bc6f46c..973ad8b 100644
--- a/src/java/org/apache/cassandra/triggers/TriggerExecutor.java
+++ b/src/java/org/apache/cassandra/triggers/TriggerExecutor.java
@@ -182,7 +182,7 @@
             for (ColumnFamily cf : mutation.getColumnFamilies())
             {
                 if (! cf.id().equals(cfId))
-                    throw new InvalidRequestException("Column family of additional mutation does not match primary update cf");
+                    throw new InvalidRequestException("table of additional mutation does not match primary update table");
             }
         }
         validate(tmutations);
@@ -228,7 +228,7 @@
         }
         catch (Exception ex)
         {
-            throw new RuntimeException(String.format("Exception while creating trigger on CF with ID: %s", columnFamily.id()), ex);
+            throw new RuntimeException(String.format("Exception while creating trigger on table with ID: %s", columnFamily.id()), ex);
         }
         finally
         {
diff --git a/src/java/org/apache/cassandra/utils/AlwaysPresentFilter.java b/src/java/org/apache/cassandra/utils/AlwaysPresentFilter.java
index 1a029e5..44d8f24 100644
--- a/src/java/org/apache/cassandra/utils/AlwaysPresentFilter.java
+++ b/src/java/org/apache/cassandra/utils/AlwaysPresentFilter.java
@@ -17,16 +17,14 @@
  */
 package org.apache.cassandra.utils;
 
-import java.nio.ByteBuffer;
-
 public class AlwaysPresentFilter implements IFilter
 {
-    public boolean isPresent(ByteBuffer key)
+    public boolean isPresent(FilterKey key)
     {
         return true;
     }
 
-    public void add(ByteBuffer key) { }
+    public void add(FilterKey key) { }
 
     public void clear() { }
 
@@ -37,6 +35,11 @@
         return this;
     }
 
+    public Throwable close(Throwable accumulate)
+    {
+        return accumulate;
+    }
+
     public long serializedSize() { return 0; }
 
     @Override
diff --git a/src/java/org/apache/cassandra/utils/AsymmetricOrdering.java b/src/java/org/apache/cassandra/utils/AsymmetricOrdering.java
new file mode 100644
index 0000000..74597f5
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/AsymmetricOrdering.java
@@ -0,0 +1,140 @@
+/*
+* 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.utils;
+
+import java.util.List;
+
+import com.google.common.collect.Ordering;
+
+import net.nicoulaj.compilecommand.annotations.Inline;
+
+public abstract class AsymmetricOrdering<T1, T2> extends Ordering<T1>
+{
+
+    public abstract int compareAsymmetric(T1 left, T2 right);
+
+    public static enum Op
+    {
+        // maximum index < key; -1 if no such key. == CEIL - 1
+        LOWER,
+
+        // maximum index <= key; -1 if no such key. == HIGHER + 1
+        FLOOR,
+
+        // minimum index >= key; size() if no such key.  == LOWER + 1
+        CEIL,
+
+        // minimum index > key; size() if no such key. == FLOOR - 1
+        HIGHER
+    }
+
+    /**
+     * @param searchIn sorted list to look in
+     * @param searchFor key to find
+     */
+    public int binarySearchAsymmetric(List<? extends T1> searchIn, T2 searchFor, Op op)
+    {
+        final int strictnessOfLessThan = strictnessOfLessThan(op);
+        int lb = -1;
+        int ub = searchIn.size();
+        // a[-1]            ^= -infinity
+        // a[search.size()] ^= +infinity
+
+        while (lb + 1 < ub)
+        {
+            int m = (lb + ub) / 2;
+            int c = compareAsymmetric(searchIn.get(m), searchFor);
+
+            if (c < strictnessOfLessThan) lb = m;
+            else ub = m;
+        }
+
+        return selectBoundary(op, lb, ub);
+    }
+
+    @Inline
+    // this value, used as the right operand to a less than operator for the result
+    // of a compare() makes its behaviour either strict (<) or not strict (<=).
+    // a value of 1 is not strict, whereas 0 is strict
+    private static int strictnessOfLessThan(Op op)
+    {
+        switch (op)
+        {
+            case FLOOR: case HIGHER:
+
+            // { a[lb] <= v ^ a[ub] > v }
+            return 1;
+
+            // { a[m] >  v   ==>   a[ub] >  v   ==>   a[lb] <= v ^ a[ub] > v }
+            // { a[m] <= v   ==>   a[lb] <= v   ==>   a[lb] <= v ^ a[ub] > v }
+
+            case CEIL: case LOWER:
+
+            // { a[lb] < v ^ a[ub] >= v }
+
+            return 0;
+
+            // { a[m] >= v   ==>   a[ub] >= v   ==>   a[lb] < v ^ a[ub] >= v }
+            // { a[m] <  v   ==>   a[lb] <  v   ==>   a[lb] < v ^ a[ub] >= v }
+        }
+        throw new IllegalStateException();
+    }
+
+    @Inline
+    private static int selectBoundary(Op op, int lb, int ub)
+    {
+        switch (op)
+        {
+            case CEIL:
+                // { a[lb] < v ^ a[ub] >= v }
+            case HIGHER:
+                // { a[lb] <= v ^ a[ub] > v }
+                return ub;
+            case FLOOR:
+                // { a[lb] <= v ^ a[ub] > v }
+            case LOWER:
+                // { a[lb] < v ^ a[ub] >= v }
+                return lb;
+        }
+        throw new IllegalStateException();
+    }
+
+    private class Reversed extends AsymmetricOrdering<T1, T2>
+    {
+        public int compareAsymmetric(T1 left, T2 right)
+        {
+            return -AsymmetricOrdering.this.compareAsymmetric(left, right);
+        }
+
+        public int compare(T1 left, T1 right)
+        {
+            return AsymmetricOrdering.this.compare(right, left);
+        }
+
+        public AsymmetricOrdering<T1, T2> reverse()
+        {
+            return AsymmetricOrdering.this;
+        }
+    }
+
+    public AsymmetricOrdering<T1, T2> reverse()
+    {
+        return new Reversed();
+    }
+}
diff --git a/src/java/org/apache/cassandra/utils/BloomCalculations.java b/src/java/org/apache/cassandra/utils/BloomCalculations.java
index b73f531..7ba5452 100644
--- a/src/java/org/apache/cassandra/utils/BloomCalculations.java
+++ b/src/java/org/apache/cassandra/utils/BloomCalculations.java
@@ -26,8 +26,8 @@
  * Filter class by helping to choose correct values of 'bits per element' and
  * 'number of hash functions, k'.
  */
-class BloomCalculations {
-
+public class BloomCalculations
+{
     private static final int minBuckets = 2;
     private static final int minK = 1;
 
@@ -182,4 +182,15 @@
         }
         return Math.min(BloomCalculations.probs.length - 1, (int)v);
     }
+
+    /**
+     * Retrieves the minimum supported BloomFilterFpChance value
+     * @return Minimum supported value for BloomFilterFpChance
+     */
+    public static double minSupportedBloomFilterFpChance()
+    {
+        int maxBuckets = probs.length - 1;
+        int maxK = probs[maxBuckets].length - 1;
+        return probs[maxBuckets][maxK];
+    }
 }
diff --git a/src/java/org/apache/cassandra/utils/BloomFilter.java b/src/java/org/apache/cassandra/utils/BloomFilter.java
index 77b2d44..9de202c 100644
--- a/src/java/org/apache/cassandra/utils/BloomFilter.java
+++ b/src/java/org/apache/cassandra/utils/BloomFilter.java
@@ -17,14 +17,13 @@
  */
 package org.apache.cassandra.utils;
 
-import java.nio.ByteBuffer;
-
 import com.google.common.annotations.VisibleForTesting;
 
 import org.apache.cassandra.utils.concurrent.WrappedSharedCloseable;
 import org.apache.cassandra.utils.obs.IBitSet;
+import org.apache.cassandra.db.TypeSizes;
 
-public abstract class BloomFilter extends WrappedSharedCloseable implements IFilter
+public class BloomFilter extends WrappedSharedCloseable implements IFilter
 {
     private static final ThreadLocal<long[]> reusableIndexes = new ThreadLocal<long[]>()
     {
@@ -51,20 +50,26 @@
         this.bitset = copy.bitset;
     }
 
+    public static final BloomFilterSerializer serializer = new BloomFilterSerializer();
+
+    public long serializedSize()
+    {
+        return serializer.serializedSize(this, TypeSizes.NATIVE);
+    }
+
     // Murmur is faster than an SHA-based approach and provides as-good collision
     // resistance.  The combinatorial generation approach described in
     // http://www.eecs.harvard.edu/~kirsch/pubs/bbbf/esa06.pdf
     // does prove to work in actual tests, and is obviously faster
     // than performing further iterations of murmur.
-    protected abstract void hash(ByteBuffer b, int position, int remaining, long seed, long[] result);
 
     // tests ask for ridiculous numbers of hashes so here is a special case for them
     // rather than using the threadLocal like we do in production
     @VisibleForTesting
-    public long[] getHashBuckets(ByteBuffer key, int hashCount, long max)
+    public long[] getHashBuckets(FilterKey key, int hashCount, long max)
     {
         long[] hash = new long[2];
-        hash(key, key.position(), key.remaining(), 0L, hash);
+        key.filterHash(hash);
         long[] indexes = new long[hashCount];
         setIndexes(hash[0], hash[1], hashCount, max, indexes);
         return indexes;
@@ -74,12 +79,12 @@
     // to avoid generating a lot of garbage since stack allocation currently does not support stores
     // (CASSANDRA-6609).  it returns the array so that the caller does not need to perform
     // a second threadlocal lookup.
-    private long[] indexes(ByteBuffer key)
+    private long[] indexes(FilterKey key)
     {
         // we use the same array both for storing the hash result, and for storing the indexes we return,
         // so that we do not need to allocate two arrays.
         long[] indexes = reusableIndexes.get();
-        hash(key, key.position(), key.remaining(), 0L, indexes);
+        key.filterHash(indexes);
         setIndexes(indexes[0], indexes[1], hashCount, bitset.capacity(), indexes);
         return indexes;
     }
@@ -93,7 +98,7 @@
         }
     }
 
-    public void add(ByteBuffer key)
+    public void add(FilterKey key)
     {
         long[] indexes = indexes(key);
         for (int i = 0; i < hashCount; i++)
@@ -102,7 +107,7 @@
         }
     }
 
-    public final boolean isPresent(ByteBuffer key)
+    public final boolean isPresent(FilterKey key)
     {
         long[] indexes = indexes(key);
         for (int i = 0; i < hashCount; i++)
@@ -119,4 +124,15 @@
     {
         bitset.clear();
     }
+
+    public IFilter sharedCopy()
+    {
+        return new BloomFilter(this);
+    }
+
+    @Override
+    public long offHeapSize()
+    {
+        return bitset.offHeapSize();
+    }
 }
diff --git a/src/java/org/apache/cassandra/utils/BloomFilterSerializer.java b/src/java/org/apache/cassandra/utils/BloomFilterSerializer.java
index b95544c..6f80ac0 100644
--- a/src/java/org/apache/cassandra/utils/BloomFilterSerializer.java
+++ b/src/java/org/apache/cassandra/utils/BloomFilterSerializer.java
@@ -27,7 +27,7 @@
 import org.apache.cassandra.utils.obs.OffHeapBitSet;
 import org.apache.cassandra.utils.obs.OpenBitSet;
 
-abstract class BloomFilterSerializer implements ISerializer<BloomFilter>
+class BloomFilterSerializer implements ISerializer<BloomFilter>
 {
     public void serialize(BloomFilter bf, DataOutputPlus out) throws IOException
     {
@@ -40,6 +40,7 @@
         return deserialize(in, false);
     }
 
+    @SuppressWarnings("resource")
     public BloomFilter deserialize(DataInput in, boolean offheap) throws IOException
     {
         int hashes = in.readInt();
@@ -47,7 +48,10 @@
         return createFilter(hashes, bs);
     }
 
-    protected abstract BloomFilter createFilter(int hashes, IBitSet bs);
+    BloomFilter createFilter(int hashes, IBitSet bs)
+    {
+        return new BloomFilter(hashes, bs);
+    }
 
     /**
      * Calculates a serialized size of the given Bloom Filter
diff --git a/src/java/org/apache/cassandra/utils/BooleanSerializer.java b/src/java/org/apache/cassandra/utils/BooleanSerializer.java
index f1707c3..8f3abde 100644
--- a/src/java/org/apache/cassandra/utils/BooleanSerializer.java
+++ b/src/java/org/apache/cassandra/utils/BooleanSerializer.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.utils;
 
 import java.io.DataInput;
-import java.io.DataOutput;
 import java.io.IOException;
 
 import org.apache.cassandra.io.IVersionedSerializer;
diff --git a/src/java/org/apache/cassandra/utils/ByteBufferUtil.java b/src/java/org/apache/cassandra/utils/ByteBufferUtil.java
index b78b8eb..6c676e0 100644
--- a/src/java/org/apache/cassandra/utils/ByteBufferUtil.java
+++ b/src/java/org/apache/cassandra/utils/ByteBufferUtil.java
@@ -78,6 +78,8 @@
 public class ByteBufferUtil
 {
     public static final ByteBuffer EMPTY_BYTE_BUFFER = ByteBuffer.wrap(new byte[0]);
+    /** Represents an unset value in bound variables */
+    public static final ByteBuffer UNSET_BYTE_BUFFER = ByteBuffer.wrap(new byte[]{});
 
     @Inline
     public static int compareUnsigned(ByteBuffer o1, ByteBuffer o2)
@@ -272,6 +274,15 @@
         FastByteOperations.copy(src, srcPos, dst, dstPos, length);
     }
 
+    public static int put(ByteBuffer src, ByteBuffer trg)
+    {
+        int length = Math.min(src.remaining(), trg.remaining());
+        arrayCopy(src, src.position(), trg, trg.position(), length);
+        trg.position(trg.position() + length);
+        src.position(src.position() + length);
+        return length;
+    }
+
     public static void writeWithLength(ByteBuffer bytes, DataOutputPlus out) throws IOException
     {
         out.writeInt(bytes.remaining());
@@ -288,8 +299,7 @@
     {
         int length = buffer.remaining();
         assert 0 <= length && length <= FBUtilities.MAX_UNSIGNED_SHORT :
-            String.format("Attempted serializing to buffer exceeded maximum of %s bytes: %s", FBUtilities.MAX_UNSIGNED_SHORT, length);
-
+        String.format("Attempted serializing to buffer exceeded maximum of %s bytes: %s", FBUtilities.MAX_UNSIGNED_SHORT, length);
         out.writeShort(length);
         out.write(buffer);
     }
@@ -298,8 +308,7 @@
     {
         int length = buffer.length;
         assert 0 <= length && length <= FBUtilities.MAX_UNSIGNED_SHORT :
-            String.format("Attempted serializing to buffer exceeded maximum of %s bytes: %s", FBUtilities.MAX_UNSIGNED_SHORT, length);
-
+        String.format("Attempted serializing to buffer exceeded maximum of %s bytes: %s", FBUtilities.MAX_UNSIGNED_SHORT, length);
         out.writeShort(length);
         out.write(buffer);
     }
@@ -376,6 +385,18 @@
         return bytes.getInt(bytes.position());
     }
 
+    /**
+     * Convert a byte buffer to a short.
+     * Does not change the byte buffer position.
+     *
+     * @param bytes byte buffer to convert to short
+     * @return short representation of the byte buffer
+     */
+    public static short toShort(ByteBuffer bytes)
+    {
+        return bytes.getShort(bytes.position());
+    }
+
     public static long toLong(ByteBuffer bytes)
     {
         return bytes.getLong(bytes.position());
@@ -391,6 +412,11 @@
         return bytes.getDouble(bytes.position());
     }
 
+    public static ByteBuffer bytes(short s)
+    {
+        return ByteBuffer.allocate(2).putShort(0, s);
+    }
+
     public static ByteBuffer bytes(int i)
     {
         return ByteBuffer.allocate(4).putInt(0, i);
@@ -555,4 +581,5 @@
         int length = readShortLength(bb);
         return readBytes(bb, length);
     }
+
 }
diff --git a/src/java/org/apache/cassandra/utils/CLibrary.java b/src/java/org/apache/cassandra/utils/CLibrary.java
index 48cb67c..b6598ec 100644
--- a/src/java/org/apache/cassandra/utils/CLibrary.java
+++ b/src/java/org/apache/cassandra/utils/CLibrary.java
@@ -18,10 +18,12 @@
 package org.apache.cassandra.utils;
 
 import java.io.FileDescriptor;
-import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.RandomAccessFile;
 import java.lang.reflect.Field;
+import java.nio.channels.FileChannel;
+import java.nio.file.Paths;
+import java.nio.file.StandardOpenOption;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -68,7 +70,7 @@
         catch (UnsatisfiedLinkError e)
         {
             logger.warn("JNA link failure, one or more native method will be unavailable.");
-            logger.debug("JNA link failure details: {}", e.getMessage());
+            logger.trace("JNA link failure details: {}", e.getMessage());
         }
         catch (NoSuchMethodError e)
         {
@@ -302,6 +304,21 @@
         }
     }
 
+    public static int getfd(FileChannel channel)
+    {
+        Field field = FBUtilities.getProtectedField(channel.getClass(), "fd");
+
+        try
+        {
+            return getfd((FileDescriptor)field.get(channel));
+        }
+        catch (IllegalArgumentException|IllegalAccessException e)
+        {
+            logger.warn("Unable to read fd field from FileChannel");
+        }
+        return -1;
+    }
+
     /**
      * Get system file descriptor from FileDescriptor object.
      * @param descriptor - FileDescriptor objec to get fd from
@@ -311,9 +328,6 @@
     {
         Field field = FBUtilities.getProtectedField(descriptor.getClass(), "fd");
 
-        if (field == null)
-            return -1;
-
         try
         {
             return field.getInt(descriptor);
@@ -321,7 +335,7 @@
         catch (Exception e)
         {
             JVMStabilityInspector.inspectThrowable(e);
-            logger.warn("unable to read fd field from FileDescriptor");
+            logger.warn("Unable to read fd field from FileDescriptor");
         }
 
         return -1;
@@ -329,29 +343,15 @@
 
     public static int getfd(String path)
     {
-        RandomAccessFile file = null;
-        try
+        try(FileChannel channel = FileChannel.open(Paths.get(path), StandardOpenOption.READ))
         {
-            file = new RandomAccessFile(path, "r");
-            return getfd(file.getFD());
+            return getfd(channel);
         }
-        catch (Throwable t)
+        catch (IOException e)
         {
-            JVMStabilityInspector.inspectThrowable(t);
+            JVMStabilityInspector.inspectThrowable(e);
             // ignore
             return -1;
         }
-        finally
-        {
-            try
-            {
-                if (file != null)
-                    file.close();
-            }
-            catch (Throwable t)
-            {
-                // ignore
-            }
-        }
     }
 }
diff --git a/src/java/org/apache/cassandra/utils/CRC32Factory.java b/src/java/org/apache/cassandra/utils/CRC32Factory.java
new file mode 100644
index 0000000..a031f09
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/CRC32Factory.java
@@ -0,0 +1,35 @@
+/*
+ *
+ * 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.utils;
+
+
+/**
+ * CRC Factory that uses our pure java crc for default
+ */
+public class CRC32Factory extends com.github.tjake.CRC32Factory
+{
+    public static final CRC32Factory instance = new CRC32Factory();
+
+    public CRC32Factory()
+    {
+        super(PureJavaCrc32.class);
+    }
+}
diff --git a/src/java/org/apache/cassandra/utils/CassandraVersion.java b/src/java/org/apache/cassandra/utils/CassandraVersion.java
new file mode 100644
index 0000000..62d68be
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/CassandraVersion.java
@@ -0,0 +1,231 @@
+/*
+ * 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.utils;
+
+import java.util.Arrays;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import com.google.common.base.Objects;
+import org.apache.commons.lang3.StringUtils;
+
+/**
+ * Implements versioning used in Cassandra and CQL.
+ * <p/>
+ * Note: The following code uses a slight variation from the semver document (http://semver.org).
+ */
+public class CassandraVersion implements Comparable<CassandraVersion>
+{
+    private static final String VERSION_REGEXP = "(\\d+)\\.(\\d+)\\.(\\d+)(\\-[.\\w]+)?([.+][.\\w]+)?";
+    private static final Pattern pattern = Pattern.compile(VERSION_REGEXP);
+    private static final Pattern SNAPSHOT = Pattern.compile("-SNAPSHOT");
+
+    public final int major;
+    public final int minor;
+    public final int patch;
+
+    private final String[] preRelease;
+    private final String[] build;
+
+    private CassandraVersion(int major, int minor, int patch, String[] preRelease, String[] build)
+    {
+        this.major = major;
+        this.minor = minor;
+        this.patch = patch;
+        this.preRelease = preRelease;
+        this.build = build;
+    }
+
+    /**
+     * Parse a version from a string.
+     *
+     * @param version the string to parse
+     * @throws IllegalArgumentException if the provided string does not
+     *                                  represent a version
+     */
+    public CassandraVersion(String version)
+    {
+        String stripped = SNAPSHOT.matcher(version).replaceFirst("");
+        Matcher matcher = pattern.matcher(stripped);
+        if (!matcher.matches())
+            throw new IllegalArgumentException("Invalid version value: " + version);
+
+        try
+        {
+            this.major = Integer.parseInt(matcher.group(1));
+            this.minor = Integer.parseInt(matcher.group(2));
+            this.patch = Integer.parseInt(matcher.group(3));
+
+            String pr = matcher.group(4);
+            String bld = matcher.group(5);
+
+            this.preRelease = pr == null || pr.isEmpty() ? null : parseIdentifiers(stripped, pr);
+            this.build = bld == null || bld.isEmpty() ? null : parseIdentifiers(stripped, bld);
+        }
+        catch (NumberFormatException e)
+        {
+            throw new IllegalArgumentException("Invalid version value: " + version);
+        }
+    }
+
+    private static String[] parseIdentifiers(String version, String str)
+    {
+        // Drop initial - or +
+        str = str.substring(1);
+        String[] parts = str.split("\\.");
+        for (String part : parts)
+        {
+            if (!part.matches("\\w+"))
+                throw new IllegalArgumentException("Invalid version value: " + version);
+        }
+        return parts;
+    }
+
+    public int compareTo(CassandraVersion other)
+    {
+        if (major < other.major)
+            return -1;
+        if (major > other.major)
+            return 1;
+
+        if (minor < other.minor)
+            return -1;
+        if (minor > other.minor)
+            return 1;
+
+        if (patch < other.patch)
+            return -1;
+        if (patch > other.patch)
+            return 1;
+
+        int c = compareIdentifiers(preRelease, other.preRelease, 1);
+        if (c != 0)
+            return c;
+
+        return compareIdentifiers(build, other.build, -1);
+    }
+
+    /**
+     * Returns a version that is backward compatible with this version amongst a list
+     * of provided version, or null if none can be found.
+     * <p/>
+     * For instance:
+     * "2.0.0".findSupportingVersion("2.0.0", "3.0.0") == "2.0.0"
+     * "2.0.0".findSupportingVersion("2.1.3", "3.0.0") == "2.1.3"
+     * "2.0.0".findSupportingVersion("3.0.0") == null
+     * "2.0.3".findSupportingVersion("2.0.0") == "2.0.0"
+     * "2.1.0".findSupportingVersion("2.0.0") == null
+     */
+    public CassandraVersion findSupportingVersion(CassandraVersion... versions)
+    {
+        for (CassandraVersion version : versions)
+        {
+            if (isSupportedBy(version))
+                return version;
+        }
+        return null;
+    }
+
+    public boolean isSupportedBy(CassandraVersion version)
+    {
+        return major == version.major && this.compareTo(version) <= 0;
+    }
+
+    private static int compareIdentifiers(String[] ids1, String[] ids2, int defaultPred)
+    {
+        if (ids1 == null)
+            return ids2 == null ? 0 : defaultPred;
+        else if (ids2 == null)
+            return -defaultPred;
+
+        int min = Math.min(ids1.length, ids2.length);
+        for (int i = 0; i < min; i++)
+        {
+            Integer i1 = tryParseInt(ids1[i]);
+            Integer i2 = tryParseInt(ids2[i]);
+
+            if (i1 != null)
+            {
+                // integer have precedence
+                if (i2 == null || i1 < i2)
+                    return -1;
+                else if (i1 > i2)
+                    return 1;
+            }
+            else
+            {
+                // integer have precedence
+                if (i2 != null)
+                    return 1;
+
+                int c = ids1[i].compareTo(ids2[i]);
+                if (c != 0)
+                    return c;
+            }
+        }
+
+        if (ids1.length < ids2.length)
+            return -1;
+        if (ids1.length > ids2.length)
+            return 1;
+        return 0;
+    }
+
+    private static Integer tryParseInt(String str)
+    {
+        try
+        {
+            return Integer.valueOf(str);
+        }
+        catch (NumberFormatException e)
+        {
+            return null;
+        }
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (!(o instanceof CassandraVersion))
+            return false;
+        CassandraVersion that = (CassandraVersion) o;
+        return major == that.major
+               && minor == that.minor
+               && patch == that.patch
+               && Arrays.equals(preRelease, that.preRelease)
+               && Arrays.equals(build, that.build);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hashCode(major, minor, patch, preRelease, build);
+    }
+
+    @Override
+    public String toString()
+    {
+        StringBuilder sb = new StringBuilder();
+        sb.append(major).append('.').append(minor).append('.').append(patch);
+        if (preRelease != null)
+            sb.append('-').append(StringUtils.join(preRelease, "."));
+        if (build != null)
+            sb.append('+').append(StringUtils.join(build, "."));
+        return sb.toString();
+    }
+}
diff --git a/src/java/org/apache/cassandra/utils/CoalescingStrategies.java b/src/java/org/apache/cassandra/utils/CoalescingStrategies.java
index 6b69066..52d4240 100644
--- a/src/java/org/apache/cassandra/utils/CoalescingStrategies.java
+++ b/src/java/org/apache/cassandra/utils/CoalescingStrategies.java
@@ -18,8 +18,11 @@
 package org.apache.cassandra.utils;
 
 import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.io.util.FileUtils;
+
 import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.File;
 import java.io.RandomAccessFile;
@@ -38,6 +41,7 @@
 
 public class CoalescingStrategies
 {
+    static protected final Logger logger = LoggerFactory.getLogger(CoalescingStrategies.class);
 
     /*
      * Log debug information at info level about what the average is and when coalescing is enabled/disabled
@@ -85,15 +89,23 @@
     {
         long now = System.nanoTime();
         final long timer = now + nanos;
+        // We shouldn't loop if it's within a few % of the target sleep time if on a second iteration.
+        // See CASSANDRA-8692.
+        final long limit = timer - nanos / 16;
         do
         {
             LockSupport.parkNanos(timer - now);
+            now = System.nanoTime();
         }
-        while (timer - (now = System.nanoTime()) > nanos / 16);
+        while (now < limit);
     }
 
     private static boolean maybeSleep(int messages, long averageGap, long maxCoalesceWindow, Parker parker)
     {
+        // Do not sleep if there are still items in the backlog (CASSANDRA-13090).
+        if (messages >= DatabaseDescriptor.getOtcCoalescingEnoughCoalescedMessages())
+            return false;
+
         // only sleep if we can expect to double the number of messages we're sending in the time interval
         long sleep = messages * averageGap;
         if (sleep <= 0 || sleep > maxCoalesceWindow)
@@ -169,7 +181,7 @@
             if (DEBUG_COALESCING && shouldLogAverage)
             {
                 shouldLogAverage = false;
-                logger.info(toString() + " gap " + TimeUnit.NANOSECONDS.toMicros(averageGap) + "μs");
+                logger.info("{} gap {}μs", this, TimeUnit.NANOSECONDS.toMicros(averageGap));
             }
         }
 
@@ -329,7 +341,7 @@
             if (input.drainTo(out, maxItems) == 0)
             {
                 out.add(input.take());
-                input.drainTo(out, maxItems - 1);
+                input.drainTo(out, maxItems - out.size());
             }
 
             for (Coalescable qm : out)
@@ -411,15 +423,16 @@
             if (input.drainTo(out, maxItems) == 0)
             {
                 out.add(input.take());
+                input.drainTo(out, maxItems - out.size());
             }
 
             long average = notifyOfSample(out.get(0).timestampNanos());
-
             debugGap(average);
 
-            maybeSleep(out.size(), average, maxCoalesceWindow, parker);
+            if (maybeSleep(out.size(), average, maxCoalesceWindow, parker)) {
+                input.drainTo(out, maxItems - out.size());
+            }
 
-            input.drainTo(out, maxItems - out.size());
             for (int ii = 1; ii < out.size(); ii++)
                 notifyOfSample(out.get(ii).timestampNanos());
         }
@@ -447,11 +460,16 @@
         @Override
         protected <C extends Coalescable> void coalesceInternal(BlockingQueue<C> input, List<C> out,  int maxItems) throws InterruptedException
         {
+            int enough = DatabaseDescriptor.getOtcCoalescingEnoughCoalescedMessages();
+
             if (input.drainTo(out, maxItems) == 0)
             {
                 out.add(input.take());
-                parker.park(coalesceWindow);
-                input.drainTo(out, maxItems - 1);
+                input.drainTo(out, maxItems - out.size());
+                if (out.size() < enough) {
+                    parker.park(coalesceWindow);
+                    input.drainTo(out, maxItems - out.size());
+                }
             }
             debugTimestamps(out);
         }
diff --git a/src/java/org/apache/cassandra/utils/EstimatedHistogram.java b/src/java/org/apache/cassandra/utils/EstimatedHistogram.java
index fd22f6f..1a48039 100644
--- a/src/java/org/apache/cassandra/utils/EstimatedHistogram.java
+++ b/src/java/org/apache/cassandra/utils/EstimatedHistogram.java
@@ -57,15 +57,13 @@
 
     public EstimatedHistogram(int bucketCount)
     {
-        bucketOffsets = newOffsets(bucketCount);
-        buckets = new AtomicLongArray(bucketOffsets.length + 1);
+        this(bucketCount, false);
     }
 
-    public EstimatedHistogram(long[] offsets, long[] bucketData)
+    public EstimatedHistogram(int bucketCount, boolean considerZeroes)
     {
-        assert bucketData.length == offsets.length +1;
-        bucketOffsets = offsets;
-        buckets = new AtomicLongArray(bucketData);
+        bucketOffsets = newOffsets(bucketCount, considerZeroes);
+        buckets = new AtomicLongArray(bucketOffsets.length + 1);
     }
 
     /**
@@ -76,16 +74,26 @@
     public EstimatedHistogram(long[] bucketData)
     {
         assert bucketData != null && bucketData.length > 0 : "Bucket data must be an array of size more than 0";
-        bucketOffsets = newOffsets(bucketData.length - 1);
+        bucketOffsets = newOffsets(bucketData.length - 1, false);
         buckets = new AtomicLongArray(bucketData);
     }
 
-    private static long[] newOffsets(int size)
+    public EstimatedHistogram(long[] offsets, long[] bucketData)
     {
-        long[] result = new long[size];
+        assert bucketData.length == offsets.length +1;
+        bucketOffsets = offsets;
+        buckets = new AtomicLongArray(bucketData);
+    }
+
+    public static long[] newOffsets(int size, boolean considerZeroes)
+    {
+        long[] result = new long[size + (considerZeroes ? 1 : 0)];
+        int i = 0;
+        if (considerZeroes)
+            result[i++] = 0;
         long last = 1;
-        result[0] = last;
-        for (int i = 1; i < size; i++)
+        result[i++] = last;
+        for (; i < result.length; i++)
         {
             long next = Math.round(last * 1.2);
             if (next == last)
@@ -205,11 +213,20 @@
     }
 
     /**
-     * @return the mean histogram value (average of bucket offsets, weighted by count)
+     * @return the ceil of mean histogram value (average of bucket offsets, weighted by count)
      * @throws IllegalStateException if any values were greater than the largest bucket threshold
      */
     public long mean()
     {
+        return (long) Math.ceil(rawMean());
+    }
+
+    /**
+     * @return the mean histogram value (average of bucket offsets, weighted by count)
+     * @throws IllegalStateException if any values were greater than the largest bucket threshold
+     */
+    public double rawMean()
+    {
         int lastBucket = buckets.length() - 1;
         if (buckets.get(lastBucket) > 0)
             throw new IllegalStateException("Unable to compute ceiling for max when histogram overflowed");
@@ -223,7 +240,7 @@
             sum += bCount * bucketOffsets[i];
         }
 
-        return (long) Math.ceil((double) sum / elements);
+        return (double) sum / elements;
     }
 
     /**
@@ -254,7 +271,7 @@
     }
 
     /**
-     * log.debug() every record in the histogram
+     * log.trace() every record in the histogram
      *
      * @param log
      */
@@ -285,7 +302,7 @@
             // calculation, and accept the unnecessary whitespace prefixes that will occasionally occur
             if (i == 0 && count == 0)
                 continue;
-            log.debug(String.format(formatstr, names[i], count));
+            log.trace(String.format(formatstr, names[i], count));
         }
     }
 
diff --git a/src/java/org/apache/cassandra/utils/ExpiringMap.java b/src/java/org/apache/cassandra/utils/ExpiringMap.java
index e7b626c..8359918 100644
--- a/src/java/org/apache/cassandra/utils/ExpiringMap.java
+++ b/src/java/org/apache/cassandra/utils/ExpiringMap.java
@@ -105,12 +105,12 @@
         service.scheduleWithFixedDelay(runnable, defaultExpiration / 2, defaultExpiration / 2, TimeUnit.MILLISECONDS);
     }
 
-    public void shutdownBlocking()
+    public boolean shutdownBlocking()
     {
         service.shutdown();
         try
         {
-            service.awaitTermination(defaultExpiration * 2, TimeUnit.MILLISECONDS);
+            return service.awaitTermination(defaultExpiration * 2, TimeUnit.MILLISECONDS);
         }
         catch (InterruptedException e)
         {
diff --git a/src/java/org/apache/cassandra/utils/FBUtilities.java b/src/java/org/apache/cassandra/utils/FBUtilities.java
index 8d8dd22..23a2c2e 100644
--- a/src/java/org/apache/cassandra/utils/FBUtilities.java
+++ b/src/java/org/apache/cassandra/utils/FBUtilities.java
@@ -19,34 +19,31 @@
 
 import java.io.*;
 import java.lang.reflect.Field;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
 import java.math.BigInteger;
-import java.net.InetAddress;
-import java.net.NetworkInterface;
-import java.net.SocketException;
-import java.net.URL;
-import java.net.UnknownHostException;
+import java.net.*;
 import java.nio.ByteBuffer;
 import java.security.MessageDigest;
 import java.security.NoSuchAlgorithmException;
 import java.util.*;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
+import java.util.concurrent.*;
+import java.util.zip.Adler32;
 import java.util.zip.Checksum;
 
 import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Joiner;
 import com.google.common.collect.AbstractIterator;
-import org.apache.cassandra.io.util.FileUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.auth.IAuthenticator;
 import org.apache.cassandra.auth.IAuthorizer;
+import org.apache.cassandra.auth.IRoleManager;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.dht.IPartitioner;
@@ -54,8 +51,10 @@
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.compress.CompressionParameters;
 import org.apache.cassandra.io.util.DataOutputBuffer;
-import org.apache.cassandra.io.util.IAllocator;
+import org.apache.cassandra.io.util.DataOutputBufferFixed;
+import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.net.AsyncOneResponse;
 
 import org.codehaus.jackson.JsonFactory;
@@ -65,19 +64,18 @@
 {
     private static final Logger logger = LoggerFactory.getLogger(FBUtilities.class);
 
-    private static ObjectMapper jsonMapper = new ObjectMapper(new JsonFactory());
+    private static final ObjectMapper jsonMapper = new ObjectMapper(new JsonFactory());
 
     public static final BigInteger TWO = new BigInteger("2");
     private static final String DEFAULT_TRIGGER_DIR = "triggers";
 
     private static final String OPERATING_SYSTEM = System.getProperty("os.name").toLowerCase();
-
     private static final boolean IS_WINDOWS = OPERATING_SYSTEM.contains("windows");
-
     private static final boolean HAS_PROCFS = !IS_WINDOWS && (new File(File.separator + "proc")).exists();
 
     private static volatile InetAddress localInetAddress;
     private static volatile InetAddress broadcastInetAddress;
+    private static volatile InetAddress broadcastRpcAddress;
 
     public static int getAvailableProcessors()
     {
@@ -151,6 +149,16 @@
         return broadcastInetAddress;
     }
 
+
+    public static InetAddress getBroadcastRpcAddress()
+    {
+        if (broadcastRpcAddress == null)
+            broadcastRpcAddress = DatabaseDescriptor.getBroadcastRpcAddress() == null
+                                   ? DatabaseDescriptor.getRpcAddress()
+                                   : DatabaseDescriptor.getBroadcastRpcAddress();
+        return broadcastRpcAddress;
+    }
+
     public static Collection<InetAddress> getAllLocalAddresses()
     {
         Set<InetAddress> localAddresses = new HashSet<InetAddress>();
@@ -170,6 +178,22 @@
         return localAddresses;
     }
 
+    public static String getNetworkInterface(InetAddress localAddress)
+    {
+        try {
+            for(NetworkInterface ifc : Collections.list(NetworkInterface.getNetworkInterfaces())) {
+                if(ifc.isUp()) {
+                    for(InetAddress addr : Collections.list(ifc.getInetAddresses())) {
+                        if (addr.equals(localAddress))
+                            return ifc.getDisplayName();
+                    }
+                }
+            }
+        }
+        catch (SocketException e) {}
+        return null;
+    }
+
     /**
      * Given two bit arrays represented as BigIntegers, containing the given
      * number of significant bits, calculate a midpoint.
@@ -315,10 +339,8 @@
 
     public static String getReleaseVersionString()
     {
-        InputStream in = null;
-        try
+        try (InputStream in = FBUtilities.class.getClassLoader().getResourceAsStream("org/apache/cassandra/config/version.properties"))
         {
-            in = FBUtilities.class.getClassLoader().getResourceAsStream("org/apache/cassandra/config/version.properties");
             if (in == null)
             {
                 return System.getProperty("cassandra.releaseVersion", "Unknown");
@@ -333,10 +355,6 @@
             logger.warn("Unable to load version.properties", e);
             return "debug version";
         }
-        finally
-        {
-            FileUtils.closeQuietly(in);
-        }
     }
 
     public static long timestampMicros()
@@ -391,14 +409,7 @@
     {
         if (!partitionerClassName.contains("."))
             partitionerClassName = "org.apache.cassandra.dht." + partitionerClassName;
-        return FBUtilities.construct(partitionerClassName, "partitioner");
-    }
-
-    public static IAllocator newOffHeapAllocator(String offheap_allocator) throws ConfigurationException
-    {
-        if (!offheap_allocator.contains("."))
-            offheap_allocator = "org.apache.cassandra.io.util." + offheap_allocator;
-        return FBUtilities.construct(offheap_allocator, "off-heap allocator");
+        return FBUtilities.instanceOrConstruct(partitionerClassName, "partitioner");
     }
 
     public static IAuthorizer newAuthorizer(String className) throws ConfigurationException
@@ -415,6 +426,13 @@
         return FBUtilities.construct(className, "authenticator");
     }
 
+    public static IRoleManager newRoleManager(String className) throws ConfigurationException
+    {
+        if (!className.contains("."))
+            className = "org.apache.cassandra.auth." + className;
+        return FBUtilities.construct(className, "role manager");
+    }
+
     /**
      * @return The Class for the given name.
      * @param classname Fully qualified classname.
@@ -427,13 +445,30 @@
         {
             return (Class<T>)Class.forName(classname);
         }
-        catch (ClassNotFoundException e)
+        catch (ClassNotFoundException | NoClassDefFoundError e)
         {
             throw new ConfigurationException(String.format("Unable to find %s class '%s'", readable, classname), e);
         }
-        catch (NoClassDefFoundError e)
+    }
+
+    /**
+     * Constructs an instance of the given class, which must have a no-arg or default constructor.
+     * @param classname Fully qualified classname.
+     * @param readable Descriptive noun for the role the class plays.
+     * @throws ConfigurationException If the class cannot be found.
+     */
+    public static <T> T instanceOrConstruct(String classname, String readable) throws ConfigurationException
+    {
+        Class<T> cls = FBUtilities.classForName(classname, readable);
+        try
         {
-            throw new ConfigurationException(String.format("Unable to find %s class '%s'", readable, classname), e);
+            Field instance = cls.getField("instance");
+            return cls.cast(instance.get(null));
+        }
+        catch (NoSuchFieldException | SecurityException | IllegalArgumentException | IllegalAccessException e)
+        {
+            // Could not get instance field. Try instantiating.
+            return construct(cls, classname, readable);
         }
     }
 
@@ -446,6 +481,11 @@
     public static <T> T construct(String classname, String readable) throws ConfigurationException
     {
         Class<T> cls = FBUtilities.classForName(classname, readable);
+        return construct(cls, classname, readable);
+    }
+
+    private static <T> T construct(Class<T> cls, String classname, String readable) throws ConfigurationException
+    {
         try
         {
             return cls.newInstance();
@@ -498,19 +538,16 @@
      */
     public static Field getProtectedField(Class klass, String fieldName)
     {
-        Field field;
-
         try
         {
-            field = klass.getDeclaredField(fieldName);
+            Field field = klass.getDeclaredField(fieldName);
             field.setAccessible(true);
+            return field;
         }
         catch (Exception e)
         {
             throw new AssertionError(e);
         }
-
-        return field;
     }
 
     public static <T> CloseableIterator<T> closeableIterator(Iterator<T> iterator)
@@ -575,17 +612,20 @@
             int errCode = p.waitFor();
             if (errCode != 0)
             {
-                BufferedReader in = new BufferedReader(new InputStreamReader(p.getInputStream()));
-                BufferedReader err = new BufferedReader(new InputStreamReader(p.getErrorStream()));
-                StringBuilder sb = new StringBuilder();
-                String str;
-                while ((str = in.readLine()) != null)
-                    sb.append(str).append(System.getProperty("line.separator"));
-                while ((str = err.readLine()) != null)
-                    sb.append(str).append(System.getProperty("line.separator"));
-                throw new IOException("Exception while executing the command: "+ StringUtils.join(pb.command(), " ") +
-                                      ", command error Code: " + errCode +
-                                      ", command output: "+ sb.toString());
+            	try (BufferedReader in = new BufferedReader(new InputStreamReader(p.getInputStream()));
+                     BufferedReader err = new BufferedReader(new InputStreamReader(p.getErrorStream())))
+                {
+            		String lineSep = System.getProperty("line.separator");
+	                StringBuilder sb = new StringBuilder();
+	                String str;
+	                while ((str = in.readLine()) != null)
+	                    sb.append(str).append(lineSep);
+	                while ((str = err.readLine()) != null)
+	                    sb.append(str).append(lineSep);
+	                throw new IOException("Exception while executing the command: "+ StringUtils.join(pb.command(), " ") +
+	                                      ", command error Code: " + errCode +
+	                                      ", command output: "+ sb.toString());
+                }
             }
         }
         catch (InterruptedException e)
@@ -602,6 +642,73 @@
         checksum.update((v >>> 0) & 0xFF);
     }
 
+    private static Method directUpdate;
+    static
+    {
+        try
+        {
+            directUpdate = Adler32.class.getDeclaredMethod("update", new Class[]{ByteBuffer.class});
+            directUpdate.setAccessible(true);
+        } catch (NoSuchMethodException e)
+        {
+            logger.warn("JVM doesn't support Adler32 byte buffer access");
+            directUpdate = null;
+        }
+    }
+
+    private static final ThreadLocal<byte[]> threadLocalScratchBuffer = new ThreadLocal<byte[]>()
+    {
+        @Override
+        protected byte[] initialValue()
+        {
+            return new byte[CompressionParameters.DEFAULT_CHUNK_LENGTH];
+        }
+    };
+
+    public static byte[] getThreadLocalScratchBuffer()
+    {
+        return threadLocalScratchBuffer.get();
+    }
+
+    //Java 7 has this method but it's private till Java 8. Thanks JDK!
+    public static boolean supportsDirectChecksum()
+    {
+        return directUpdate != null;
+    }
+
+    public static void directCheckSum(Adler32 checksum, ByteBuffer bb)
+    {
+        if (directUpdate != null)
+        {
+            try
+            {
+                directUpdate.invoke(checksum, bb);
+                return;
+            }
+            catch (IllegalAccessException e)
+            {
+                directUpdate = null;
+                logger.warn("JVM doesn't support Adler32 byte buffer access");
+            }
+            catch (InvocationTargetException e)
+            {
+                throw new RuntimeException(e);
+            }
+        }
+
+        //Fallback
+        byte[] buffer = getThreadLocalScratchBuffer();
+
+        int remaining;
+        while ((remaining = bb.remaining()) > 0)
+        {
+            remaining = Math.min(remaining, buffer.length);
+            ByteBufferUtil.arrayCopy(bb, bb.position(), buffer, 0, remaining);
+            bb.position(bb.position() + remaining);
+            checksum.update(buffer, 0, remaining);
+        }
+    }
+
     public static long abs(long index)
     {
         long negbit = index >> 63;
@@ -629,10 +736,10 @@
 
     public static <T> byte[] serialize(T object, IVersionedSerializer<T> serializer, int version)
     {
-        try
+        int size = (int) serializer.serializedSize(object, version);
+
+        try (DataOutputBuffer buffer = new DataOutputBufferFixed(size))
         {
-            int size = (int) serializer.serializedSize(object, version);
-            DataOutputBuffer buffer = new DataOutputBuffer(size);
             serializer.serialize(object, buffer, version);
             assert buffer.getLength() == size && buffer.getData().length == size
                 : String.format("Final buffer length %s to accommodate data size of %s (predicted %s) for %s",
@@ -729,4 +836,12 @@
             throw new RuntimeException(e);
         }
     }
+
+    @VisibleForTesting
+    protected static void reset()
+    {
+        localInetAddress = null;
+        broadcastInetAddress = null;
+        broadcastRpcAddress = null;
+    }
 }
diff --git a/src/java/org/apache/cassandra/utils/FastByteOperations.java b/src/java/org/apache/cassandra/utils/FastByteOperations.java
index 6e25492..f35d2e2 100644
--- a/src/java/org/apache/cassandra/utils/FastByteOperations.java
+++ b/src/java/org/apache/cassandra/utils/FastByteOperations.java
@@ -349,8 +349,8 @@
             int wordComparisons = minLength & ~7;
             for (int i = 0; i < wordComparisons ; i += Longs.BYTES)
             {
-                long lw = theUnsafe.getLong(buffer1, memoryOffset1 + (long) i);
-                long rw = theUnsafe.getLong(buffer2, memoryOffset2 + (long) i);
+                long lw = theUnsafe.getLong(buffer1, memoryOffset1 + i);
+                long rw = theUnsafe.getLong(buffer2, memoryOffset2 + i);
 
                 if (lw != rw)
                 {
diff --git a/src/java/org/apache/cassandra/utils/FilterFactory.java b/src/java/org/apache/cassandra/utils/FilterFactory.java
index 757e8dd..d77500c 100644
--- a/src/java/org/apache/cassandra/utils/FilterFactory.java
+++ b/src/java/org/apache/cassandra/utils/FilterFactory.java
@@ -37,12 +37,12 @@
 
     public static void serialize(IFilter bf, DataOutputPlus output) throws IOException
     {
-        Murmur3BloomFilter.serializer.serialize((Murmur3BloomFilter) bf, output);
+        BloomFilter.serializer.serialize((BloomFilter) bf, output);
     }
 
     public static IFilter deserialize(DataInput input, boolean offheap) throws IOException
     {
-        return Murmur3BloomFilter.serializer.deserialize(input, offheap);
+        return BloomFilter.serializer.deserialize(input, offheap);
     }
 
     /**
@@ -78,10 +78,11 @@
         return createFilter(spec.K, numElements, spec.bucketsPerElement, offheap);
     }
 
+    @SuppressWarnings("resource")
     private static IFilter createFilter(int hash, long numElements, int bucketsPer, boolean offheap)
     {
         long numBits = (numElements * bucketsPer) + BITSET_EXCESS;
         IBitSet bitset = offheap ? new OffHeapBitSet(numBits) : new OpenBitSet(numBits);
-        return new Murmur3BloomFilter(hash, bitset);
+        return new BloomFilter(hash, bitset);
     }
 }
diff --git a/src/java/org/apache/cassandra/utils/HeapUtils.java b/src/java/org/apache/cassandra/utils/HeapUtils.java
new file mode 100644
index 0000000..bfc8a0b
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/HeapUtils.java
@@ -0,0 +1,205 @@
+/*
+ * 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.utils;
+
+import java.io.*;
+import java.lang.management.ManagementFactory;
+import java.lang.management.RuntimeMXBean;
+import java.nio.file.FileSystems;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.List;
+
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.commons.lang3.text.StrBuilder;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Utility to generate heap dumps.
+ *
+ */
+public final class HeapUtils
+{
+    private static final Logger logger = LoggerFactory.getLogger(HeapUtils.class);
+
+    /**
+     * Generates a HEAP dump in the directory specified by the <code>HeapDumpPath</code> JVM option
+     * or in the <code>CASSANDRA_HOME</code> directory.
+     */
+    public static void generateHeapDump()
+    {
+        Long processId = getProcessId();
+        if (processId == null)
+        {
+            logger.error("The process ID could not be retrieved. Skipping heap dump generation.");
+            return;
+        }
+
+        String heapDumpPath = getHeapDumpPathOption();
+        if (heapDumpPath == null)
+        {
+            String cassandraHome = System.getenv("CASSANDRA_HOME");
+            if (cassandraHome == null)
+            {
+                return;
+            }
+
+            heapDumpPath = cassandraHome;
+        }
+
+        Path dumpPath = FileSystems.getDefault().getPath(heapDumpPath);
+        if (Files.isDirectory(dumpPath))
+        {
+            dumpPath = dumpPath.resolve("java_pid" + processId + ".hprof");
+        }
+
+        String jmapPath = getJmapPath();
+
+        // The jmap file could not be found. In this case let's default to jmap in the hope that it is in the path.
+        String jmapCommand = jmapPath == null ? "jmap" : jmapPath;
+
+        String[] dumpCommands = new String[] {jmapCommand,
+                                              "-dump:format=b,file=" + dumpPath,
+                                              processId.toString()};
+
+        // Lets also log the Heap histogram
+        String[] histoCommands = new String[] {jmapCommand,
+                                               "-histo",
+                                               processId.toString()};
+        try
+        {
+            logProcessOutput(Runtime.getRuntime().exec(dumpCommands));
+            logProcessOutput(Runtime.getRuntime().exec(histoCommands));
+        }
+        catch (IOException e)
+        {
+            logger.error("The heap dump could not be generated due to the following error: ", e);
+        }
+    }
+
+    /**
+     * Retrieve the path to the JMAP executable.
+     * @return the path to the JMAP executable or null if it cannot be found.
+     */
+    private static String getJmapPath()
+    {
+        // Searching in the JAVA_HOME is safer than searching into System.getProperty("java.home") as the Oracle
+        // JVM might use the JRE which do not contains jmap.
+        String javaHome = System.getenv("JAVA_HOME");
+        if (javaHome == null)
+            return null;
+
+        File javaBinDirectory = new File(javaHome, "bin");
+        File[] files = javaBinDirectory.listFiles(new FilenameFilter()
+        {
+            public boolean accept(File dir, String name)
+            {
+                return name.startsWith("jmap");
+            }
+        });
+        return ArrayUtils.isEmpty(files) ? null : files[0].getPath();
+    }
+
+    /**
+     * Logs the output of the specified process.
+     *
+     * @param p the process
+     * @throws IOException if an I/O problem occurs
+     */
+    private static void logProcessOutput(Process p) throws IOException
+    {
+        BufferedReader input = new BufferedReader(new InputStreamReader(p.getInputStream()));
+
+        StrBuilder builder = new StrBuilder();
+        String line;
+        while ((line = input.readLine()) != null)
+        {
+            builder.appendln(line);
+        }
+        logger.info(builder.toString());
+    }
+
+    /**
+     * Retrieves the value of the <code>HeapDumpPath</code> JVM option.
+     * @return the value of the <code>HeapDumpPath</code> JVM option or <code>null</code> if the value has not been
+     * specified.
+     */
+    private static String getHeapDumpPathOption()
+    {
+        RuntimeMXBean runtimeMxBean = ManagementFactory.getRuntimeMXBean();
+        List<String> inputArguments = runtimeMxBean.getInputArguments();
+        String heapDumpPathOption = null;
+        for (String argument : inputArguments)
+        {
+            if (argument.startsWith("-XX:HeapDumpPath="))
+            {
+                heapDumpPathOption = argument;
+                // We do not break in case the option has been specified several times.
+                // In general it seems that JVMs use the right-most argument as the winner.
+            }
+        }
+
+        if (heapDumpPathOption == null)
+            return null;
+
+        return heapDumpPathOption.substring(17, heapDumpPathOption.length());
+    }
+
+    /**
+     * Retrieves the process ID or <code>null</code> if the process ID cannot be retrieved.
+     * @return the process ID or <code>null</code> if the process ID cannot be retrieved.
+     */
+    private static Long getProcessId()
+    {
+        // Once Java 9 is ready the process API should provide a better way to get the process ID.
+        long pid = SigarLibrary.instance.getPid();
+
+        if (pid >= 0)
+            return Long.valueOf(pid);
+
+        return getProcessIdFromJvmName();
+    }
+
+    /**
+     * Retrieves the process ID from the JVM name.
+     * @return the process ID or <code>null</code> if the process ID cannot be retrieved.
+     */
+    private static Long getProcessIdFromJvmName()
+    {
+        // the JVM name in Oracle JVMs is: '<pid>@<hostname>' but this might not be the case on all JVMs
+        String jvmName = ManagementFactory.getRuntimeMXBean().getName();
+        try
+        {
+            return Long.parseLong(jvmName.split("@")[0]);
+        }
+        catch (NumberFormatException e)
+        {
+            // ignore
+        }
+        return null;
+    }
+
+    /**
+     * The class must not be instantiated.
+     */
+    private HeapUtils()
+    {
+    }
+}
diff --git a/src/java/org/apache/cassandra/utils/Hex.java b/src/java/org/apache/cassandra/utils/Hex.java
index f018f26..0883c34 100644
--- a/src/java/org/apache/cassandra/utils/Hex.java
+++ b/src/java/org/apache/cassandra/utils/Hex.java
@@ -119,9 +119,9 @@
      * @return Constructor if successful, null if the constructor cannot be
      * accessed
      */
-    public static Constructor getProtectedConstructor(Class klass, Class... paramTypes)
+    public static <T> Constructor<T> getProtectedConstructor(Class<T> klass, Class<?>... paramTypes)
     {
-        Constructor c;
+        Constructor<T> c;
         try
         {
             c = klass.getDeclaredConstructor(paramTypes);
diff --git a/src/java/org/apache/cassandra/utils/HistogramBuilder.java b/src/java/org/apache/cassandra/utils/HistogramBuilder.java
index c2accb2..5d22352 100644
--- a/src/java/org/apache/cassandra/utils/HistogramBuilder.java
+++ b/src/java/org/apache/cassandra/utils/HistogramBuilder.java
@@ -88,7 +88,7 @@
                 max = value;
         }
 
-        final long mean = (long)Math.round(sum / count);
+        final long mean = Math.round(sum / count);
         final double stdev =
                 Math.sqrt((sumsq / count) - (mean * (double) mean));
 
diff --git a/src/java/org/apache/cassandra/utils/IFilter.java b/src/java/org/apache/cassandra/utils/IFilter.java
index bde6333..2f59864 100644
--- a/src/java/org/apache/cassandra/utils/IFilter.java
+++ b/src/java/org/apache/cassandra/utils/IFilter.java
@@ -17,15 +17,19 @@
  */
 package org.apache.cassandra.utils;
 
-import java.nio.ByteBuffer;
-
 import org.apache.cassandra.utils.concurrent.SharedCloseable;
 
 public interface IFilter extends SharedCloseable
 {
-    void add(ByteBuffer key);
+    public interface FilterKey
+    {
+        /** Places the murmur3 hash of the key in the given long array of size at least two. */
+        void filterHash(long[] dest);
+    }
 
-    boolean isPresent(ByteBuffer key);
+    void add(FilterKey key);
+
+    boolean isPresent(FilterKey key);
 
     void clear();
 
diff --git a/src/java/org/apache/cassandra/utils/Interval.java b/src/java/org/apache/cassandra/utils/Interval.java
index c74f153..335ef27 100644
--- a/src/java/org/apache/cassandra/utils/Interval.java
+++ b/src/java/org/apache/cassandra/utils/Interval.java
@@ -17,6 +17,8 @@
  */
 package org.apache.cassandra.utils;
 
+import java.util.Comparator;
+
 import com.google.common.base.Objects;
 
 public class Interval<C, D>
@@ -64,4 +66,44 @@
         // handles nulls properly
         return Objects.equal(min, that.min) && Objects.equal(max, that.max) && Objects.equal(data, that.data);
     }
+
+    private static final AsymmetricOrdering<Interval<Comparable, Object>, Comparable> minOrdering
+    = new AsymmetricOrdering<Interval<Comparable, Object>, Comparable>()
+    {
+        public int compareAsymmetric(Interval<Comparable, Object> left, Comparable right)
+        {
+            return left.min.compareTo(right);
+        }
+
+        public int compare(Interval<Comparable, Object> i1, Interval<Comparable, Object> i2)
+        {
+            return i1.min.compareTo(i2.min);
+        }
+    };
+
+    private static final AsymmetricOrdering<Interval<Comparable, Object>, Comparable> maxOrdering
+    = new AsymmetricOrdering<Interval<Comparable, Object>, Comparable>()
+    {
+        public int compareAsymmetric(Interval<Comparable, Object> left, Comparable right)
+        {
+            return left.max.compareTo(right);
+        }
+
+        public int compare(Interval<Comparable, Object> i1, Interval<Comparable, Object> i2)
+        {
+            return i1.max.compareTo(i2.max);
+        }
+    };
+
+    private static final AsymmetricOrdering<Interval<Comparable, Object>, Comparable> reverseMaxOrdering = maxOrdering.reverse();
+
+    public static <C extends Comparable<? super C>, V> AsymmetricOrdering<Interval<C, V>, C> minOrdering()
+    {
+        return (AsymmetricOrdering) minOrdering;
+    }
+
+    public static <C extends Comparable<? super C>, V> AsymmetricOrdering<Interval<C, V>, C> maxOrdering()
+    {
+        return (AsymmetricOrdering) maxOrdering;
+    }
 }
diff --git a/src/java/org/apache/cassandra/utils/IntervalTree.java b/src/java/org/apache/cassandra/utils/IntervalTree.java
index 3755c54..e857ee7 100644
--- a/src/java/org/apache/cassandra/utils/IntervalTree.java
+++ b/src/java/org/apache/cassandra/utils/IntervalTree.java
@@ -26,7 +26,6 @@
 import com.google.common.base.Joiner;
 import com.google.common.collect.AbstractIterator;
 import com.google.common.collect.Iterators;
-import com.google.common.collect.Ordering;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -34,75 +33,41 @@
 import org.apache.cassandra.io.ISerializer;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.utils.AsymmetricOrdering.Op;
 
-public class IntervalTree<C, D, I extends Interval<C, D>> implements Iterable<I>
+public class IntervalTree<C extends Comparable<? super C>, D, I extends Interval<C, D>> implements Iterable<I>
 {
     private static final Logger logger = LoggerFactory.getLogger(IntervalTree.class);
 
     @SuppressWarnings("unchecked")
-    private static final IntervalTree EMPTY_TREE = new IntervalTree(null, null);
+    private static final IntervalTree EMPTY_TREE = new IntervalTree(null);
 
     private final IntervalNode head;
     private final int count;
-    private final Comparator<C> comparator;
 
-    final Ordering<I> minOrdering;
-    final Ordering<I> maxOrdering;
-
-    protected IntervalTree(Collection<I> intervals, Comparator<C> comparator)
+    protected IntervalTree(Collection<I> intervals)
     {
-        this.comparator = comparator;
-
-        final IntervalTree it = this;
-        this.minOrdering = new Ordering<I>()
-        {
-            public int compare(I interval1, I interval2)
-            {
-                return it.comparePoints(interval1.min, interval2.min);
-            }
-        };
-        this.maxOrdering = new Ordering<I>()
-        {
-            public int compare(I interval1, I interval2)
-            {
-                return it.comparePoints(interval1.max, interval2.max);
-            }
-        };
-
         this.head = intervals == null || intervals.isEmpty() ? null : new IntervalNode(intervals);
         this.count = intervals == null ? 0 : intervals.size();
     }
 
-    public static <C, D, I extends Interval<C, D>> IntervalTree<C, D, I> build(Collection<I> intervals, Comparator<C> comparator)
+    public static <C extends Comparable<? super C>, D, I extends Interval<C, D>> IntervalTree<C, D, I> build(Collection<I> intervals)
     {
         if (intervals == null || intervals.isEmpty())
             return emptyTree();
 
-        return new IntervalTree<C, D, I>(intervals, comparator);
+        return new IntervalTree<C, D, I>(intervals);
     }
 
-    public static <C extends Comparable<C>, D, I extends Interval<C, D>> IntervalTree<C, D, I> build(Collection<I> intervals)
-    {
-        if (intervals == null || intervals.isEmpty())
-            return emptyTree();
-
-        return new IntervalTree<C, D, I>(intervals, null);
-    }
-
-    public static <C, D, I extends Interval<C, D>> Serializer<C, D, I> serializer(ISerializer<C> pointSerializer, ISerializer<D> dataSerializer, Constructor<I> constructor)
+    public static <C extends Comparable<? super C>, D, I extends Interval<C, D>> Serializer<C, D, I> serializer(ISerializer<C> pointSerializer, ISerializer<D> dataSerializer, Constructor<I> constructor)
     {
         return new Serializer<>(pointSerializer, dataSerializer, constructor);
     }
 
     @SuppressWarnings("unchecked")
-    public static <C, D, I extends Interval<C, D>> IntervalTree<C, D, I> emptyTree()
+    public static <C extends Comparable<? super C>, D, I extends Interval<C, D>> IntervalTree<C, D, I> emptyTree()
     {
-        return (IntervalTree<C, D, I>)EMPTY_TREE;
-    }
-
-    public Comparator<C> comparator()
-    {
-        return comparator;
+        return EMPTY_TREE;
     }
 
     public int intervalCount()
@@ -172,43 +137,12 @@
     @Override
     public final int hashCode()
     {
-        int result = comparator.hashCode();
+        int result = 0;
         for (Interval<C, D> interval : this)
             result = 31 * result + interval.hashCode();
         return result;
     }
 
-    private int comparePoints(C point1, C point2)
-    {
-        if (comparator != null)
-        {
-            return comparator.compare(point1, point2);
-        }
-        else
-        {
-            assert point1 instanceof Comparable;
-            assert point2 instanceof Comparable;
-            return ((Comparable<C>)point1).compareTo(point2);
-        }
-    }
-
-    private boolean encloses(Interval<C, D> enclosing, Interval<C, D> enclosed)
-    {
-        return comparePoints(enclosing.min, enclosed.min) <= 0
-            && comparePoints(enclosing.max, enclosed.max) >= 0;
-    }
-
-    private boolean contains(Interval<C, D> interval, C point)
-    {
-        return comparePoints(interval.min, point) <= 0
-            && comparePoints(interval.max, point) >= 0;
-    }
-
-    private boolean intersects(Interval<C, D> interval1, Interval<C, D> interval2)
-    {
-        return contains(interval1, interval2.min) || contains(interval1, interval2.max);
-    }
-
     private class IntervalNode
     {
         final C center;
@@ -246,15 +180,11 @@
                 List<C> allEndpoints = new ArrayList<C>(toBisect.size() * 2);
                 for (I interval : toBisect)
                 {
-                    assert (comparator == null ? ((Comparable)interval.min).compareTo(interval.max)
-                                               : comparator.compare(interval.min, interval.max)) <= 0 : "Interval min > max";
                     allEndpoints.add(interval.min);
                     allEndpoints.add(interval.max);
                 }
-                if (comparator != null)
-                    Collections.sort(allEndpoints, comparator);
-                else
-                    Collections.sort((List<Comparable>)allEndpoints);
+
+                Collections.sort(allEndpoints);
 
                 low = allEndpoints.get(0);
                 center = allEndpoints.get(toBisect.size());
@@ -267,16 +197,16 @@
 
                 for (I candidate : toBisect)
                 {
-                    if (comparePoints(candidate.max, center) < 0)
+                    if (candidate.max.compareTo(center) < 0)
                         leftSegment.add(candidate);
-                    else if (comparePoints(candidate.min, center) > 0)
+                    else if (candidate.min.compareTo(center) > 0)
                         rightSegment.add(candidate);
                     else
                         intersects.add(candidate);
                 }
 
-                intersectsLeft = minOrdering.sortedCopy(intersects);
-                intersectsRight = maxOrdering.reverse().sortedCopy(intersects);
+                intersectsLeft = Interval.<C, D>minOrdering().sortedCopy(intersects);
+                intersectsRight = Interval.<C, D>maxOrdering().sortedCopy(intersects);
                 left = leftSegment.isEmpty() ? null : new IntervalNode(leftSegment);
                 right = rightSegment.isEmpty() ? null : new IntervalNode(rightSegment);
 
@@ -290,10 +220,31 @@
 
         void searchInternal(Interval<C, D> searchInterval, List<D> results)
         {
-            if (comparePoints(searchInterval.max, low) < 0 || comparePoints(searchInterval.min, high) > 0)
-                return;
+            if (center.compareTo(searchInterval.min) < 0)
+            {
+                int i = Interval.<C, D>maxOrdering().binarySearchAsymmetric(intersectsRight, searchInterval.min, Op.CEIL);
+                if (i == intersectsRight.size() && high.compareTo(searchInterval.min) < 0)
+                    return;
 
-            if (contains(searchInterval, center))
+                while (i < intersectsRight.size())
+                    results.add(intersectsRight.get(i++).data);
+
+                if (right != null)
+                    right.searchInternal(searchInterval, results);
+            }
+            else if (center.compareTo(searchInterval.max) > 0)
+            {
+                int j = Interval.<C, D>minOrdering().binarySearchAsymmetric(intersectsLeft, searchInterval.max, Op.HIGHER);
+                if (j == 0 && low.compareTo(searchInterval.max) > 0)
+                    return;
+
+                for (int i = 0 ; i < j ; i++)
+                    results.add(intersectsLeft.get(i).data);
+
+                if (left != null)
+                    left.searchInternal(searchInterval, results);
+            }
+            else
             {
                 // Adds every interval contained in this node to the result set then search left and right for further
                 // overlapping intervals
@@ -305,35 +256,6 @@
                 if (right != null)
                     right.searchInternal(searchInterval, results);
             }
-            else if (comparePoints(center, searchInterval.min) < 0)
-            {
-                // Adds intervals i in intersects right as long as i.max >= searchInterval.min
-                // then search right
-                for (Interval<C, D> interval : intersectsRight)
-                {
-                    if (comparePoints(interval.max, searchInterval.min) >= 0)
-                        results.add(interval.data);
-                    else
-                        break;
-                }
-                if (right != null)
-                    right.searchInternal(searchInterval, results);
-            }
-            else
-            {
-                assert comparePoints(center, searchInterval.max) > 0;
-                // Adds intervals i in intersects left as long as i.min >= searchInterval.max
-                // then search left
-                for (Interval<C, D> interval : intersectsLeft)
-                {
-                    if (comparePoints(interval.min, searchInterval.max) <= 0)
-                        results.add(interval.data);
-                    else
-                        break;
-                }
-                if (left != null)
-                    left.searchInternal(searchInterval, results);
-            }
         }
     }
 
@@ -377,7 +299,7 @@
         }
     }
 
-    public static class Serializer<C, D, I extends Interval<C, D>> implements IVersionedSerializer<IntervalTree<C, D, I>>
+    public static class Serializer<C extends Comparable<? super C>, D, I extends Interval<C, D>> implements IVersionedSerializer<IntervalTree<C, D, I>>
     {
         private final ISerializer<C> pointSerializer;
         private final ISerializer<D> dataSerializer;
@@ -417,7 +339,7 @@
             try
             {
                 int count = in.readInt();
-                List<Interval<C, D>> intervals = new ArrayList<Interval<C, D>>(count);
+                List<I> intervals = new ArrayList<I>(count);
                 for (int i = 0; i < count; i++)
                 {
                     C min = pointSerializer.deserialize(in);
@@ -425,17 +347,9 @@
                     D data = dataSerializer.deserialize(in);
                     intervals.add(constructor.newInstance(min, max, data));
                 }
-                return new IntervalTree(intervals, comparator);
+                return new IntervalTree<C, D, I>(intervals);
             }
-            catch (InstantiationException e)
-            {
-                throw new RuntimeException(e);
-            }
-            catch (InvocationTargetException e)
-            {
-                throw new RuntimeException(e);
-            }
-            catch (IllegalAccessException e)
+            catch (InstantiationException | InvocationTargetException | IllegalAccessException e)
             {
                 throw new RuntimeException(e);
             }
diff --git a/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java b/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
index 2884bc2..f8cb775 100644
--- a/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
+++ b/src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
@@ -28,9 +28,7 @@
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.io.FSError;
 import org.apache.cassandra.io.sstable.CorruptSSTableException;
-import org.apache.cassandra.service.CassandraDaemon;
 import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.thrift.Cassandra;
 
 /**
  * Responsible for deciding whether to kill the JVM if it gets in an "unstable" state (think OOM).
@@ -53,7 +51,10 @@
     {
         boolean isUnstable = false;
         if (t instanceof OutOfMemoryError)
+        {
             isUnstable = true;
+            HeapUtils.generateHeapDump();
+        }
 
         if (DatabaseDescriptor.getDiskFailurePolicy() == Config.DiskFailurePolicy.die)
             if (t instanceof FSError || t instanceof CorruptSSTableException)
diff --git a/src/java/org/apache/cassandra/utils/MerkleTree.java b/src/java/org/apache/cassandra/utils/MerkleTree.java
index 8e6d5c0..1e0f505 100644
--- a/src/java/org/apache/cassandra/utils/MerkleTree.java
+++ b/src/java/org/apache/cassandra/utils/MerkleTree.java
@@ -28,6 +28,7 @@
 
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.IPartitionerDependentSerializer;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.exceptions.ConfigurationException;
@@ -85,8 +86,8 @@
             out.writeLong(mt.size);
             out.writeUTF(mt.partitioner.getClass().getCanonicalName());
             // full range
-            Token.serializer.serialize(mt.fullRange.left, out);
-            Token.serializer.serialize(mt.fullRange.right, out);
+            Token.serializer.serialize(mt.fullRange.left, out, version);
+            Token.serializer.serialize(mt.fullRange.right, out, version);
             Hashable.serializer.serialize(mt.root, out, version);
         }
 
@@ -106,13 +107,13 @@
             }
 
             // full range
-            Token left = Token.serializer.deserialize(in);
-            Token right = Token.serializer.deserialize(in);
-            Range<Token> fullRange = new Range<>(left, right, partitioner);
+            Token left = Token.serializer.deserialize(in, partitioner, version);
+            Token right = Token.serializer.deserialize(in, partitioner, version);
+            Range<Token> fullRange = new Range<>(left, right);
 
             MerkleTree mt = new MerkleTree(partitioner, fullRange, hashdepth, maxsize);
             mt.size = size;
-            mt.root = Hashable.serializer.deserialize(in, version);
+            mt.root = Hashable.serializer.deserialize(in, partitioner, version);
             return mt;
         }
 
@@ -124,8 +125,8 @@
                  + TypeSizes.NATIVE.sizeof(mt.partitioner.getClass().getCanonicalName());
 
             // full range
-            size += Token.serializer.serializedSize(mt.fullRange.left, TypeSizes.NATIVE);
-            size += Token.serializer.serializedSize(mt.fullRange.right, TypeSizes.NATIVE);
+            size += Token.serializer.serializedSize(mt.fullRange.left, version);
+            size += Token.serializer.serializedSize(mt.fullRange.right, version);
 
             size += Hashable.serializer.serializedSize(mt.root, version);
             return size;
@@ -515,6 +516,16 @@
         return histbuild.buildWithStdevRangesAroundMean();
     }
 
+    public long rowCount()
+    {
+        long count = 0;
+        for (TreeRange range : new TreeRangeIterator(this))
+        {
+            count += range.hashable.rowsInRange;
+        }
+        return count;
+    }
+
     @Override
     public String toString()
     {
@@ -811,7 +822,7 @@
             return buff.toString();
         }
 
-        private static class InnerSerializer implements IVersionedSerializer<Inner>
+        private static class InnerSerializer implements IPartitionerDependentSerializer<Inner>
         {
             public void serialize(Inner inner, DataOutputPlus out, int version) throws IOException
             {
@@ -822,20 +833,20 @@
                     out.writeInt(inner.hash.length);
                     out.write(inner.hash);
                 }
-                Token.serializer.serialize(inner.token, out);
+                Token.serializer.serialize(inner.token, out, version);
                 Hashable.serializer.serialize(inner.lchild, out, version);
                 Hashable.serializer.serialize(inner.rchild, out, version);
             }
 
-            public Inner deserialize(DataInput in, int version) throws IOException
+            public Inner deserialize(DataInput in, IPartitioner p, int version) throws IOException
             {
                 int hashLen = in.readInt();
                 byte[] hash = hashLen >= 0 ? new byte[hashLen] : null;
                 if (hash != null)
                     in.readFully(hash);
-                Token token = Token.serializer.deserialize(in);
-                Hashable lchild = Hashable.serializer.deserialize(in, version);
-                Hashable rchild = Hashable.serializer.deserialize(in, version);
+                Token token = Token.serializer.deserialize(in, p, version);
+                Hashable lchild = Hashable.serializer.deserialize(in, p, version);
+                Hashable rchild = Hashable.serializer.deserialize(in, p, version);
                 return new Inner(token, lchild, rchild);
             }
 
@@ -845,7 +856,7 @@
                 ? TypeSizes.NATIVE.sizeof(-1)
                         : TypeSizes.NATIVE.sizeof(inner.hash().length) + inner.hash().length;
 
-                size += Token.serializer.serializedSize(inner.token, TypeSizes.NATIVE)
+                size += Token.serializer.serializedSize(inner.token, version)
                 + Hashable.serializer.serializedSize(inner.lchild, version)
                 + Hashable.serializer.serializedSize(inner.rchild, version);
                 return size;
@@ -892,7 +903,7 @@
             return "#<Leaf " + Hashable.toString(hash()) + ">";
         }
 
-        private static class LeafSerializer implements IVersionedSerializer<Leaf>
+        private static class LeafSerializer implements IPartitionerDependentSerializer<Leaf>
         {
             public void serialize(Leaf leaf, DataOutputPlus out, int version) throws IOException
             {
@@ -907,7 +918,7 @@
                 }
             }
 
-            public Leaf deserialize(DataInput in, int version) throws IOException
+            public Leaf deserialize(DataInput in, IPartitioner p, int version) throws IOException
             {
                 int hashLen = in.readInt();
                 byte[] hash = hashLen < 0 ? null : new byte[hashLen];
@@ -955,7 +966,7 @@
     static abstract class Hashable implements Serializable
     {
         private static final long serialVersionUID = 1L;
-        private static final IVersionedSerializer<Hashable> serializer = new HashableSerializer();
+        private static final IPartitionerDependentSerializer<Hashable> serializer = new HashableSerializer();
 
         protected byte[] hash;
         protected long sizeOfRange;
@@ -1033,7 +1044,7 @@
             return "[" + Hex.bytesToHex(hash) + "]";
         }
 
-        private static class HashableSerializer implements IVersionedSerializer<Hashable>
+        private static class HashableSerializer implements IPartitionerDependentSerializer<Hashable>
         {
             public void serialize(Hashable h, DataOutputPlus out, int version) throws IOException
             {
@@ -1051,13 +1062,13 @@
                     throw new IOException("Unexpected Hashable: " + h.getClass().getCanonicalName());
             }
 
-            public Hashable deserialize(DataInput in, int version) throws IOException
+            public Hashable deserialize(DataInput in, IPartitioner p, int version) throws IOException
             {
                 byte ident = in.readByte();
                 if (Inner.IDENT == ident)
-                    return Inner.serializer.deserialize(in, version);
+                    return Inner.serializer.deserialize(in, p, version);
                 else if (Leaf.IDENT == ident)
-                    return Leaf.serializer.deserialize(in, version);
+                    return Leaf.serializer.deserialize(in, p, version);
                 else
                     throw new IOException("Unexpected Hashable: " + ident);
             }
diff --git a/src/java/org/apache/cassandra/utils/Murmur3BloomFilter.java b/src/java/org/apache/cassandra/utils/Murmur3BloomFilter.java
deleted file mode 100644
index 431ca5b..0000000
--- a/src/java/org/apache/cassandra/utils/Murmur3BloomFilter.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * 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.utils;
-
-import java.nio.ByteBuffer;
-
-import org.apache.cassandra.db.TypeSizes;
-import org.apache.cassandra.utils.obs.IBitSet;
-
-public class Murmur3BloomFilter extends BloomFilter
-{
-    public static final Murmur3BloomFilterSerializer serializer = new Murmur3BloomFilterSerializer();
-
-    public Murmur3BloomFilter(int hashCount, IBitSet bs)
-    {
-        super(hashCount, bs);
-    }
-
-    protected Murmur3BloomFilter(Murmur3BloomFilter copy)
-    {
-        super(copy);
-    }
-
-    public long serializedSize()
-    {
-        return serializer.serializedSize(this, TypeSizes.NATIVE);
-    }
-
-    public IFilter sharedCopy()
-    {
-        return new Murmur3BloomFilter(this);
-    }
-
-    @Override
-    public long offHeapSize()
-    {
-        return bitset.offHeapSize();
-    }
-
-    protected void hash(ByteBuffer b, int position, int remaining, long seed, long[] result)
-    {
-        MurmurHash.hash3_x64_128(b, b.position(), b.remaining(), seed, result);
-    }
-
-    public static class Murmur3BloomFilterSerializer extends BloomFilterSerializer
-    {
-        protected BloomFilter createFilter(int hashes, IBitSet bs)
-        {
-            return new Murmur3BloomFilter(hashes, bs);
-        }
-    }
-}
\ No newline at end of file
diff --git a/src/java/org/apache/cassandra/utils/MurmurHash.java b/src/java/org/apache/cassandra/utils/MurmurHash.java
index e1fc80e..c02fdcc 100644
--- a/src/java/org/apache/cassandra/utils/MurmurHash.java
+++ b/src/java/org/apache/cassandra/utils/MurmurHash.java
@@ -17,8 +17,6 @@
  */
 package org.apache.cassandra.utils;
 
-import net.nicoulaj.compilecommand.annotations.Inline;
-
 import java.nio.ByteBuffer;
 
 /**
diff --git a/src/java/org/apache/cassandra/utils/Mx4jTool.java b/src/java/org/apache/cassandra/utils/Mx4jTool.java
index e8fdb29..41a5b33 100644
--- a/src/java/org/apache/cassandra/utils/Mx4jTool.java
+++ b/src/java/org/apache/cassandra/utils/Mx4jTool.java
@@ -42,7 +42,7 @@
     {
         try
         {
-            logger.debug("Will try to load mx4j now, if it's in the classpath");
+            logger.trace("Will try to load mx4j now, if it's in the classpath");
             MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
             ObjectName processorName = new ObjectName("Server:name=XSLTProcessor");
 
@@ -65,7 +65,7 @@
         }
         catch (ClassNotFoundException e)
         {
-            logger.debug("Will not load MX4J, mx4j-tools.jar is not in the classpath");
+            logger.trace("Will not load MX4J, mx4j-tools.jar is not in the classpath");
         }
         catch(Exception e)
         {
diff --git a/src/java/org/apache/cassandra/utils/NanoTimeToCurrentTimeMillis.java b/src/java/org/apache/cassandra/utils/NanoTimeToCurrentTimeMillis.java
index a6c5d287..f124383 100644
--- a/src/java/org/apache/cassandra/utils/NanoTimeToCurrentTimeMillis.java
+++ b/src/java/org/apache/cassandra/utils/NanoTimeToCurrentTimeMillis.java
@@ -46,7 +46,7 @@
      * There is also the issue of how scalable nanoTime() and currentTimeMillis() are which is a moving target.
      *
      * These timestamps don't order with System.currentTimeMillis() because currentTimeMillis() can tick over
-     * before this one does. I have seen it behind by as much as 2 milliseconds.
+     * before this one does. I have seen it behind by as much as 2ms on Linux and 25ms on Windows.
      */
     public static final long convert(long nanoTime)
     {
diff --git a/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java b/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java
new file mode 100644
index 0000000..3ec6965
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java
@@ -0,0 +1,126 @@
+/*
+ * 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.utils;
+
+import java.net.InetAddress;
+import java.util.*;
+
+import com.datastax.driver.core.*;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.ColumnFamilyType;
+import org.apache.cassandra.db.SystemKeyspace;
+import org.apache.cassandra.db.composites.CellNameType;
+import org.apache.cassandra.db.composites.CellNames;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.TypeParser;
+import org.apache.cassandra.dht.*;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.SSTableLoader;
+import org.apache.cassandra.schema.LegacySchemaTables;
+
+public class NativeSSTableLoaderClient extends SSTableLoader.Client
+{
+    protected final Map<String, CFMetaData> tables;
+    private final Collection<InetAddress> hosts;
+    private final int port;
+    private final String username;
+    private final String password;
+    private final SSLOptions sslOptions;
+
+    public NativeSSTableLoaderClient(Collection<InetAddress> hosts, int port, String username, String password, SSLOptions sslOptions)
+    {
+        super();
+        this.tables = new HashMap<>();
+        this.hosts = hosts;
+        this.port = port;
+        this.username = username;
+        this.password = password;
+        this.sslOptions = sslOptions;
+    }
+
+    public void init(String keyspace)
+    {
+        Cluster.Builder builder = Cluster.builder().addContactPoints(hosts).withPort(port);
+        if (sslOptions != null)
+            builder.withSSL(sslOptions);
+        if (username != null && password != null)
+            builder = builder.withCredentials(username, password);
+
+        try (Cluster cluster = builder.build(); Session session = cluster.connect())
+        {
+
+            Metadata metadata = cluster.getMetadata();
+
+            setPartitioner(metadata.getPartitioner());
+
+            Set<TokenRange> tokenRanges = metadata.getTokenRanges();
+
+            Token.TokenFactory tokenFactory = getPartitioner().getTokenFactory();
+
+            for (TokenRange tokenRange : tokenRanges)
+            {
+                Set<Host> endpoints = metadata.getReplicas(Metadata.quote(keyspace), tokenRange);
+                Range<Token> range = new Range<>(tokenFactory.fromString(tokenRange.getStart().getValue().toString()),
+                                                 tokenFactory.fromString(tokenRange.getEnd().getValue().toString()));
+                for (Host endpoint : endpoints)
+                    addRangeForEndpoint(range, endpoint.getAddress());
+            }
+
+            tables.putAll(fetchTablesMetadata(keyspace, session));
+        }
+    }
+
+    public CFMetaData getTableMetadata(String tableName)
+    {
+        return tables.get(tableName);
+    }
+
+    @Override
+    public void setTableMetadata(CFMetaData cfm)
+    {
+        tables.put(cfm.cfName, cfm);
+    }
+
+    private static Map<String, CFMetaData> fetchTablesMetadata(String keyspace, Session session)
+    {
+        Map<String, CFMetaData> tables = new HashMap<>();
+
+        String query = String.format("SELECT columnfamily_name, cf_id, type, comparator, subcomparator, is_dense FROM %s.%s WHERE keyspace_name = '%s'",
+                                     SystemKeyspace.NAME,
+                                     LegacySchemaTables.COLUMNFAMILIES,
+                                     keyspace);
+
+        for (Row row : session.execute(query))
+        {
+            String name = row.getString("columnfamily_name");
+            UUID id = row.getUUID("cf_id");
+            ColumnFamilyType type = ColumnFamilyType.valueOf(row.getString("type"));
+            AbstractType rawComparator = TypeParser.parse(row.getString("comparator"));
+            AbstractType subComparator = row.isNull("subcomparator")
+                                       ? null
+                                       : TypeParser.parse(row.getString("subcomparator"));
+            boolean isDense = row.getBool("is_dense");
+            CellNameType comparator = CellNames.fromAbstractType(CFMetaData.makeRawAbstractType(rawComparator, subComparator),
+                                                                 isDense);
+
+            tables.put(name, new CFMetaData(keyspace, name, type, comparator, id));
+        }
+
+        return tables;
+    }
+}
diff --git a/src/java/org/apache/cassandra/utils/OutputHandler.java b/src/java/org/apache/cassandra/utils/OutputHandler.java
index b203663..88c54ce 100644
--- a/src/java/org/apache/cassandra/utils/OutputHandler.java
+++ b/src/java/org/apache/cassandra/utils/OutputHandler.java
@@ -44,7 +44,7 @@
 
         public void debug(String msg)
         {
-            logger.debug(msg);
+            logger.trace(msg);
         }
 
         public void warn(String msg)
diff --git a/src/java/org/apache/cassandra/utils/OverlapIterator.java b/src/java/org/apache/cassandra/utils/OverlapIterator.java
new file mode 100644
index 0000000..7c1544a
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/OverlapIterator.java
@@ -0,0 +1,74 @@
+/*
+ *
+ * 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.utils;
+
+import java.util.*;
+
+/**
+ * A class for iterating sequentially through an ordered collection and efficiently
+ * finding the overlapping set of matching intervals.
+ *
+ * The algorithm is quite simple: the intervals are sorted ascending by both min and max
+ * in two separate lists. These lists are walked forwards each time we visit a new point,
+ * with the set of intervals in the min-ordered list being added to our set of overlaps,
+ * and those in the max-ordered list being removed.
+ */
+public class OverlapIterator<I extends Comparable<? super I>, V>
+{
+    // indexing into sortedByMin, tracks the next interval to include
+    int nextToInclude;
+    final List<Interval<I, V>> sortedByMin;
+    // indexing into sortedByMax, tracks the next interval to exclude
+    int nextToExclude;
+    final List<Interval<I, V>> sortedByMax;
+    final Set<V> overlaps = new HashSet<>();
+    final Set<V> accessible = Collections.unmodifiableSet(overlaps);
+
+    public OverlapIterator(Collection<Interval<I, V>> intervals)
+    {
+        sortedByMax = new ArrayList<>(intervals);
+        Collections.sort(sortedByMax, Interval.<I, V>maxOrdering());
+        // we clone after first sorting by max;  this is quite likely to make sort cheaper, since a.max < b.max
+        // generally increases the likelihood that a.min < b.min, so the list may be partially sorted already.
+        // this also means if (in future) we sort either collection (or a subset thereof) by the other's comparator
+        // all items, including equal, will occur in the same order, including
+        sortedByMin = new ArrayList<>(sortedByMax);
+        Collections.sort(sortedByMin, Interval.<I, V>minOrdering());
+    }
+
+    // move the iterator forwards to the overlaps matching point
+    public void update(I point)
+    {
+        // we don't use binary search here since we expect points to be a superset of the min/max values
+
+        // add those we are now after the start of
+        while (nextToInclude < sortedByMin.size() && sortedByMin.get(nextToInclude).min.compareTo(point) <= 0)
+            overlaps.add(sortedByMin.get(nextToInclude++).data);
+        // remove those we are now after the end of
+        while (nextToExclude < sortedByMax.size() && sortedByMax.get(nextToExclude).max.compareTo(point) < 0)
+            overlaps.remove(sortedByMax.get(nextToExclude++).data);
+    }
+
+    public Set<V> overlaps()
+    {
+        return accessible;
+    }
+}
diff --git a/src/java/org/apache/cassandra/utils/PureJavaCrc32.java b/src/java/org/apache/cassandra/utils/PureJavaCrc32.java
index 9a1ac02..17e6235 100644
--- a/src/java/org/apache/cassandra/utils/PureJavaCrc32.java
+++ b/src/java/org/apache/cassandra/utils/PureJavaCrc32.java
@@ -17,9 +17,13 @@
  */
 package org.apache.cassandra.utils;
 import java.nio.ByteBuffer;
-import java.util.zip.Checksum;
+
+import com.github.tjake.ICRC32;
 
 /**
+ * NOTE: You should be using CRCFactory class because it will pick a better
+ * version based on your JDK version
+ *
  * A pure-java implementation of the CRC32 checksum that uses
  * the same polynomial as the built-in native CRC32.
  *
@@ -34,7 +38,8 @@
  * This class is copied from hadoop-commons project and retains that formatting.
  * (The initial patch added PureJavaCrc32 was HADOOP-6148)
  */
-public class PureJavaCrc32 implements Checksum {
+public class PureJavaCrc32 implements ICRC32
+{
 
   /** the current CRC value, bit-flipped */
   private int crc;
diff --git a/src/java/org/apache/cassandra/utils/SearchIterator.java b/src/java/org/apache/cassandra/utils/SearchIterator.java
new file mode 100644
index 0000000..004b02a
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/SearchIterator.java
@@ -0,0 +1,26 @@
+/*
+ * 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.utils;
+
+public interface SearchIterator<K, V>
+{
+
+    public boolean hasNext();
+    public V next(K key);
+
+}
diff --git a/src/java/org/apache/cassandra/utils/SemanticVersion.java b/src/java/org/apache/cassandra/utils/SemanticVersion.java
deleted file mode 100644
index 858029d..0000000
--- a/src/java/org/apache/cassandra/utils/SemanticVersion.java
+++ /dev/null
@@ -1,231 +0,0 @@
-/*
- * 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.utils;
-
-import java.util.regex.Pattern;
-import java.util.regex.Matcher;
-import java.util.Arrays;
-
-import org.apache.commons.lang3.StringUtils;
-import com.google.common.base.Objects;
-
-/**
- * Implements semantic versioning as defined at http://semver.org/.
- *
- * Note: The following code uses a slight variation from the document above in
- * that it doesn't allow dashes in pre-release and build identifier.
- */
-public class SemanticVersion implements Comparable<SemanticVersion>
-{
-    private static final String VERSION_REGEXP = "(\\d+)\\.(\\d+)\\.(\\d+)(\\-[.\\w]+)?([.+][.\\w]+)?";
-    private static final Pattern pattern = Pattern.compile(VERSION_REGEXP);
-
-    public final int major;
-    public final int minor;
-    public final int patch;
-
-    private final String[] preRelease;
-    private final String[] build;
-
-    private SemanticVersion(int major, int minor, int patch, String[] preRelease, String[] build)
-    {
-        this.major = major;
-        this.minor = minor;
-        this.patch = patch;
-        this.preRelease = preRelease;
-        this.build = build;
-    }
-
-    /**
-     * Parse a semantic version from a string.
-     *
-     * @param version the string to parse
-     * @throws IllegalArgumentException if the provided string does not
-     * represent a semantic version
-     */
-    public SemanticVersion(String version)
-    {
-        Matcher matcher = pattern.matcher(version);
-        if (!matcher.matches())
-            throw new IllegalArgumentException("Invalid version value: " + version + " (see http://semver.org/ for details)");
-
-        try
-        {
-            this.major = Integer.parseInt(matcher.group(1));
-            this.minor = Integer.parseInt(matcher.group(2));
-            this.patch = Integer.parseInt(matcher.group(3));
-
-            String pr = matcher.group(4);
-            String bld = matcher.group(5);
-
-            this.preRelease = pr == null || pr.isEmpty() ? null : parseIdentifiers(version, pr);
-            this.build = bld == null || bld.isEmpty() ? null : parseIdentifiers(version, bld);
-
-        }
-        catch (NumberFormatException e)
-        {
-            throw new IllegalArgumentException("Invalid version value: " + version + " (see http://semver.org/ for details)");
-        }
-    }
-
-    private static String[] parseIdentifiers(String version, String str)
-    {
-        // Drop initial - or +
-        str = str.substring(1);
-        String[] parts = str.split("\\.");
-        for (String part : parts)
-        {
-            if (!part.matches("\\w+"))
-                throw new IllegalArgumentException("Invalid version value: " + version + " (see http://semver.org/ for details)");
-        }
-        return parts;
-    }
-
-    public int compareTo(SemanticVersion other)
-    {
-        if (major < other.major)
-            return -1;
-        if (major > other.major)
-            return 1;
-
-        if (minor < other.minor)
-            return -1;
-        if (minor > other.minor)
-            return 1;
-
-        if (patch < other.patch)
-            return -1;
-        if (patch > other.patch)
-            return 1;
-
-        int c = compareIdentifiers(preRelease, other.preRelease, 1);
-        if (c != 0)
-            return c;
-
-        return compareIdentifiers(build, other.build, -1);
-    }
-
-    /**
-     * Returns a version that is backward compatible with this version amongst a list
-     * of provided version, or null if none can be found.
-     *
-     * For instance:
-     *   "2.0.0".findSupportingVersion("2.0.0", "3.0.0") == "2.0.0"
-     *   "2.0.0".findSupportingVersion("2.1.3", "3.0.0") == "2.1.3"
-     *   "2.0.0".findSupportingVersion("3.0.0") == null
-     *   "2.0.3".findSupportingVersion("2.0.0") == "2.0.0"
-     *   "2.1.0".findSupportingVersion("2.0.0") == null
-     */
-    public SemanticVersion findSupportingVersion(SemanticVersion... versions)
-    {
-        for (SemanticVersion version : versions)
-        {
-            if (isSupportedBy(version))
-                return version;
-        }
-        return null;
-    }
-
-    public boolean isSupportedBy(SemanticVersion version)
-    {
-        return major == version.major && this.compareTo(version) <= 0;
-    }
-
-    private static int compareIdentifiers(String[] ids1, String[] ids2, int defaultPred)
-    {
-        if (ids1 == null)
-            return ids2 == null ? 0 : defaultPred;
-        else if (ids2 == null)
-            return -defaultPred;
-
-        int min = Math.min(ids1.length, ids2.length);
-        for (int i = 0; i < min; i++)
-        {
-            Integer i1 = tryParseInt(ids1[i]);
-            Integer i2 = tryParseInt(ids2[i]);
-
-            if (i1 != null)
-            {
-                // integer have precedence
-                if (i2 == null || i1 < i2)
-                    return -1;
-                else if (i1 > i2)
-                    return 1;
-            }
-            else
-            {
-                // integer have precedence
-                if (i2 != null)
-                    return 1;
-
-                int c = ids1[i].compareTo(ids2[i]);
-                if (c != 0)
-                    return c;
-            }
-        }
-
-        if (ids1.length < ids2.length)
-            return -1;
-        if (ids1.length > ids2.length)
-            return 1;
-        return 0;
-    }
-
-    private static Integer tryParseInt(String str)
-    {
-        try
-        {
-            return Integer.valueOf(str);
-        }
-        catch (NumberFormatException e)
-        {
-            return null;
-        }
-    }
-
-    @Override
-    public boolean equals(Object o)
-    {
-        if(!(o instanceof SemanticVersion))
-            return false;
-        SemanticVersion that = (SemanticVersion)o;
-        return major == that.major
-            && minor == that.minor
-            && patch == that.patch
-            && Arrays.equals(preRelease, that.preRelease)
-            && Arrays.equals(build, that.build);
-    }
-
-    @Override
-    public int hashCode()
-    {
-        return Objects.hashCode(major, minor, patch, preRelease, build);
-    }
-
-    @Override
-    public String toString()
-    {
-        StringBuilder sb = new StringBuilder();
-        sb.append(major).append('.').append(minor).append('.').append(patch);
-        if (preRelease != null)
-            sb.append('-').append(StringUtils.join(preRelease, "."));
-        if (build != null)
-            sb.append('+').append(StringUtils.join(build, "."));
-        return sb.toString();
-    }
-}
diff --git a/src/java/org/apache/cassandra/utils/SigarLibrary.java b/src/java/org/apache/cassandra/utils/SigarLibrary.java
new file mode 100644
index 0000000..0312204
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/SigarLibrary.java
@@ -0,0 +1,188 @@
+/*
+ * 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.utils;
+
+import org.hyperic.sigar.*;
+import org.slf4j.LoggerFactory;
+import org.slf4j.Logger;
+
+public class SigarLibrary
+{
+    private Logger logger = LoggerFactory.getLogger(SigarLibrary.class);
+
+    public static final SigarLibrary instance = new SigarLibrary();
+
+    private Sigar sigar;
+    private FileSystemMap mounts = null;
+    private boolean initialized = false;
+    private long INFINITY = -1;
+    private long EXPECTED_MIN_NOFILE = 10000l; // number of files that can be opened
+    private long EXPECTED_NPROC = 32768l; // number of processes
+    private long EXPECTED_AS = INFINITY; // address space
+
+    // TODO: Determine memlock limits if possible
+    // TODO: Determine if file system is remote or local
+    // TODO: Determine if disk latency is within acceptable limits
+
+    private SigarLibrary()
+    {
+        logger.info("Initializing SIGAR library");
+        try
+        {
+            sigar = new Sigar();
+            mounts = sigar.getFileSystemMap();
+            initialized = true;
+        }
+        catch (SigarException e)
+        {
+            logger.info("Could not initialize SIGAR library {} ", e.getMessage());
+        }
+        catch (UnsatisfiedLinkError linkError)
+        {
+            logger.info("Could not initialize SIGAR library {} ", linkError.getMessage());
+        }
+    }
+
+    /**
+     *
+     * @return true or false indicating if sigar was successfully initialized
+     */
+    public boolean initialized()
+    {
+        return initialized;
+    }
+
+    private boolean hasAcceptableProcNumber()
+    {
+        try
+        {
+            long fileMax = sigar.getResourceLimit().getProcessesMax();
+            if (fileMax >= EXPECTED_NPROC || fileMax == INFINITY)
+            {
+                return true;
+            }
+            else
+            {
+                return false;
+            }
+        }
+        catch (SigarException sigarException)
+        {
+            logger.warn("Could not determine if max processes was acceptable. Error message: {}", sigarException);
+            return false;
+        }
+    }
+
+    private boolean hasAcceptableFileLimits()
+    {
+        try
+        {
+            long fileMax = sigar.getResourceLimit().getOpenFilesMax();
+            if (fileMax >= EXPECTED_MIN_NOFILE || fileMax == INFINITY)
+            {
+                return true;
+            }
+            else
+            {
+                return false;
+            }
+        }
+        catch (SigarException sigarException)
+        {
+            logger.warn("Could not determine if max open file handle limit is correctly configured. Error message: {}", sigarException);
+            return false;
+        }
+    }
+
+    private boolean hasAcceptableAddressSpace()
+    {
+        // Check is invalid on Windows
+        if (FBUtilities.isWindows())
+            return true;
+
+        try
+        {
+            long fileMax = sigar.getResourceLimit().getVirtualMemoryMax();
+            if (fileMax == EXPECTED_AS)
+            {
+                return true;
+            }
+            else
+            {
+                return false;
+            }
+        }
+        catch (SigarException sigarException)
+        {
+            logger.warn("Could not determine if VirtualMemoryMax was acceptable. Error message: {}", sigarException);
+            return false;
+        }
+    }
+
+    private boolean isSwapEnabled()
+    {
+        try
+        {
+            Swap swap = sigar.getSwap();
+            long swapSize = swap.getTotal();
+            if (swapSize > 0)
+            {
+                return true;
+            }
+            else
+            {
+                return false;
+            }
+        }
+        catch (SigarException sigarException)
+        {
+            logger.warn("Could not determine if swap configuration is acceptable. Error message: {}", sigarException);
+            return false;
+        }
+    }
+
+    public long getPid()
+    {
+        return initialized ? sigar.getPid() : -1;
+    }
+
+    public void warnIfRunningInDegradedMode()
+    {
+        if (initialized)
+        {
+            boolean swapEnabled = isSwapEnabled();
+            boolean goodAddressSpace = hasAcceptableAddressSpace();
+            boolean goodFileLimits = hasAcceptableFileLimits();
+            boolean goodProcNumber = hasAcceptableProcNumber();
+            if (swapEnabled || !goodAddressSpace || !goodFileLimits || !goodProcNumber)
+            {
+                logger.warn("Cassandra server running in degraded mode. Is swap disabled? : {},  Address space adequate? : {}, " +
+                            " nofile limit adequate? : {}, nproc limit adequate? : {} ", !swapEnabled, goodAddressSpace,
+                            goodFileLimits, goodProcNumber );
+            }
+            else
+            {
+                logger.info("Checked OS settings and found them configured for optimal performance.");
+            }
+        }
+        else
+        {
+            logger.info("Sigar could not be initialized, test for checking degraded mode omitted.");
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/utils/StatusLogger.java b/src/java/org/apache/cassandra/utils/StatusLogger.java
index 66ff985..712e8f7 100644
--- a/src/java/org/apache/cassandra/utils/StatusLogger.java
+++ b/src/java/org/apache/cassandra/utils/StatusLogger.java
@@ -18,16 +18,16 @@
 package org.apache.cassandra.utils;
 
 import java.lang.management.ManagementFactory;
+import java.util.Map;
 import java.util.Set;
-import javax.management.JMX;
-import javax.management.MBeanServer;
-import javax.management.MalformedObjectNameException;
-import javax.management.ObjectName;
+import javax.management.*;
 
 import com.google.common.collect.Iterables;
 
 import org.apache.cassandra.cache.*;
 
+import org.apache.cassandra.concurrent.Stage;
+import org.apache.cassandra.metrics.ThreadPoolMetrics;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -43,49 +43,40 @@
 {
     private static final Logger logger = LoggerFactory.getLogger(StatusLogger.class);
 
+
     public static void log()
     {
         MBeanServer server = ManagementFactory.getPlatformMBeanServer();
 
         // everything from o.a.c.concurrent
         logger.info(String.format("%-25s%10s%10s%15s%10s%18s", "Pool Name", "Active", "Pending", "Completed", "Blocked", "All Time Blocked"));
-        Set<ObjectName> request, internal;
-        try
+
+        for (Map.Entry<String, String> tpool : ThreadPoolMetrics.getJmxThreadPools(server).entries())
         {
-            request = server.queryNames(new ObjectName("org.apache.cassandra.request:type=*"), null);
-            internal = server.queryNames(new ObjectName("org.apache.cassandra.internal:type=*"), null);
+            logger.info(String.format("%-25s%10s%10s%15s%10s%18s%n",
+                                      tpool.getValue(),
+                                      ThreadPoolMetrics.getJmxMetric(server, tpool.getKey(), tpool.getValue(), "ActiveTasks"),
+                                      ThreadPoolMetrics.getJmxMetric(server, tpool.getKey(), tpool.getValue(), "PendingTasks"),
+                                      ThreadPoolMetrics.getJmxMetric(server, tpool.getKey(), tpool.getValue(), "CompletedTasks"),
+                                      ThreadPoolMetrics.getJmxMetric(server, tpool.getKey(), tpool.getValue(), "CurrentlyBlockedTasks"),
+                                      ThreadPoolMetrics.getJmxMetric(server, tpool.getKey(), tpool.getValue(), "TotalBlockedTasks")));
         }
-        catch (MalformedObjectNameException e)
-        {
-            throw new RuntimeException(e);
-        }
-        for (ObjectName objectName : Iterables.concat(request, internal))
-        {
-            String poolName = objectName.getKeyProperty("type");
-            JMXEnabledThreadPoolExecutorMBean threadPoolProxy = JMX.newMBeanProxy(server, objectName, JMXEnabledThreadPoolExecutorMBean.class);
-            logger.info(String.format("%-25s%10s%10s%15s%10s%18s",
-                                      poolName,
-                                      threadPoolProxy.getActiveCount(),
-                                      threadPoolProxy.getPendingTasks(),
-                                      threadPoolProxy.getCompletedTasks(),
-                                      threadPoolProxy.getCurrentlyBlockedTasks(),
-                                      threadPoolProxy.getTotalBlockedTasks()));
-        }
+
         // one offs
         logger.info(String.format("%-25s%10s%10s",
                                   "CompactionManager", CompactionManager.instance.getActiveCompactions(), CompactionManager.instance.getPendingTasks()));
-        int pendingCommands = 0;
-        for (int n : MessagingService.instance().getCommandPendingTasks().values())
+        int pendingLargeMessages = 0;
+        for (int n : MessagingService.instance().getLargeMessagePendingTasks().values())
         {
-            pendingCommands += n;
+            pendingLargeMessages += n;
         }
-        int pendingResponses = 0;
-        for (int n : MessagingService.instance().getResponsePendingTasks().values())
+        int pendingSmallMessages = 0;
+        for (int n : MessagingService.instance().getSmallMessagePendingTasks().values())
         {
-            pendingResponses += n;
+            pendingSmallMessages += n;
         }
         logger.info(String.format("%-25s%10s%10s",
-                                  "MessagingService", "n/a", pendingCommands + "/" + pendingResponses));
+                                  "MessagingService", "n/a", pendingLargeMessages + "/" + pendingSmallMessages));
 
         // Global key/row cache information
         AutoSavingCache<KeyCacheKey, RowIndexEntry> keyCache = CacheService.instance.keyCache;
@@ -109,12 +100,12 @@
                                   rowCacheKeysToSave == Integer.MAX_VALUE ? "all" : rowCacheKeysToSave));
 
         // per-CF stats
-        logger.info(String.format("%-25s%20s", "ColumnFamily", "Memtable ops,data"));
+        logger.info(String.format("%-25s%20s", "Table", "Memtable ops,data"));
         for (ColumnFamilyStore cfs : ColumnFamilyStore.all())
         {
             logger.info(String.format("%-25s%20s",
                                       cfs.keyspace.getName() + "." + cfs.name,
-                                      cfs.getMemtableColumnsCount() + "," + cfs.getMemtableDataSize()));
+                                      cfs.metric.memtableColumnsCount.getValue() + "," + cfs.metric.memtableLiveDataSize.getValue()));
         }
     }
 }
diff --git a/src/java/org/apache/cassandra/utils/StreamingHistogram.java b/src/java/org/apache/cassandra/utils/StreamingHistogram.java
index 3f5a715..eb884be 100644
--- a/src/java/org/apache/cassandra/utils/StreamingHistogram.java
+++ b/src/java/org/apache/cassandra/utils/StreamingHistogram.java
@@ -201,7 +201,7 @@
             Map<Double, Long> entries = histogram.getAsMap();
             size += typeSizes.sizeof(entries.size());
             // size of entries = size * (8(double) + 8(long))
-            size += entries.size() * (8 + 8);
+            size += entries.size() * (8L + 8L);
             return size;
         }
     }
diff --git a/src/java/org/apache/cassandra/utils/SyncUtil.java b/src/java/org/apache/cassandra/utils/SyncUtil.java
new file mode 100644
index 0000000..0d293aa
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/SyncUtil.java
@@ -0,0 +1,185 @@
+/*
+ *
+ * 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.utils;
+
+import java.io.FileDescriptor;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.io.SyncFailedException;
+import java.lang.reflect.Field;
+import java.nio.MappedByteBuffer;
+import java.nio.channels.ClosedChannelException;
+import java.nio.channels.FileChannel;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.cassandra.config.Config;
+
+import com.google.common.base.Preconditions;
+
+/*
+ * A wrapper around various mechanisms for syncing files that makes it possible it intercept
+ * and skip syncing. Useful for unit tests in certain environments where syncs can have outliers
+ * bad enough to causes tests to run 10s of seconds longer.
+ */
+public class SyncUtil
+{
+    public static boolean SKIP_SYNC = Boolean.getBoolean(Config.PROPERTY_PREFIX + "skip_sync");
+
+    private static final Field mbbFDField;
+    private static final Field fdClosedField;
+    private static final Field fdUseCountField;
+
+    static
+    {
+        Field mbbFDFieldTemp = null;
+        try
+        {
+            mbbFDFieldTemp = MappedByteBuffer.class.getDeclaredField("fd");
+            mbbFDFieldTemp.setAccessible(true);
+        }
+        catch (NoSuchFieldException e)
+        {
+        }
+        mbbFDField = mbbFDFieldTemp;
+
+        //Java 8
+        Field fdClosedFieldTemp = null;
+        try
+        {
+            fdClosedFieldTemp = FileDescriptor.class.getDeclaredField("closed");
+            fdClosedFieldTemp.setAccessible(true);
+        }
+        catch (NoSuchFieldException e)
+        {
+        }
+        fdClosedField = fdClosedFieldTemp;
+
+        //Java 7
+        Field fdUseCountTemp = null;
+        try
+        {
+            fdUseCountTemp = FileDescriptor.class.getDeclaredField("useCount");
+            fdUseCountTemp.setAccessible(true);
+        }
+        catch (NoSuchFieldException e)
+        {
+        }
+        fdUseCountField = fdUseCountTemp;
+    }
+
+    public static MappedByteBuffer force(MappedByteBuffer buf)
+    {
+        Preconditions.checkNotNull(buf);
+        if (SKIP_SYNC)
+        {
+            Object fd = null;
+            try
+            {
+                if (mbbFDField != null)
+                {
+                    fd = mbbFDField.get(buf);
+                }
+            }
+            catch (Exception e)
+            {
+                throw new RuntimeException(e);
+            }
+            //This is what MappedByteBuffer.force() throws if a you call force() on an umapped buffer
+            if (mbbFDField != null && fd == null)
+                throw new UnsupportedOperationException();
+            return buf;
+        }
+        else
+        {
+            return buf.force();
+        }
+    }
+
+    public static void sync(FileDescriptor fd) throws SyncFailedException
+    {
+        Preconditions.checkNotNull(fd);
+        if (SKIP_SYNC)
+        {
+            boolean closed = false;
+            try
+            {
+                if (fdClosedField != null)
+                    closed = fdClosedField.getBoolean(fd);
+            }
+            catch (Exception e)
+            {
+                throw new RuntimeException(e);
+            }
+
+            int useCount = 1;
+            try
+            {
+                if (fdUseCountField != null)
+                    useCount = ((AtomicInteger)fdUseCountField.get(fd)).get();
+            }
+            catch (Exception e)
+            {
+                throw new RuntimeException(e);
+            }
+            if (closed || !fd.valid() || useCount < 0)
+                throw new SyncFailedException("Closed " + closed + " valid " + fd.valid() + " useCount " + useCount);
+        }
+        else
+        {
+            fd.sync();
+        }
+    }
+
+    public static void force(FileChannel fc, boolean metaData) throws IOException
+    {
+        Preconditions.checkNotNull(fc);
+        if (SKIP_SYNC)
+        {
+            if (!fc.isOpen())
+                throw new ClosedChannelException();
+        }
+        else
+        {
+            fc.force(metaData);
+        }
+    }
+
+    public static void sync(RandomAccessFile ras) throws IOException
+    {
+        Preconditions.checkNotNull(ras);
+        sync(ras.getFD());
+    }
+
+    public static void sync(FileOutputStream fos) throws IOException
+    {
+        Preconditions.checkNotNull(fos);
+        sync(fos.getFD());
+    }
+
+    public static void trySync(int fd)
+    {
+        if (SKIP_SYNC)
+            return;
+        else
+            CLibrary.trySync(fd);
+    }
+}
diff --git a/src/java/org/apache/cassandra/utils/Throwables.java b/src/java/org/apache/cassandra/utils/Throwables.java
index 0a2bd28..877f388 100644
--- a/src/java/org/apache/cassandra/utils/Throwables.java
+++ b/src/java/org/apache/cassandra/utils/Throwables.java
@@ -18,6 +18,10 @@
 */
 package org.apache.cassandra.utils;
 
+import java.io.IOException;
+
+import com.google.common.base.Optional;
+
 public class Throwables
 {
 
@@ -34,4 +38,33 @@
         if (fail != null)
             com.google.common.base.Throwables.propagate(fail);
     }
+
+    public static Throwable close(Throwable accumulate, Iterable<? extends AutoCloseable> closeables)
+    {
+        for (AutoCloseable closeable : closeables)
+        {
+            try
+            {
+                closeable.close();
+            }
+            catch (Throwable t)
+            {
+                accumulate = merge(accumulate, t);
+            }
+        }
+        return accumulate;
+    }
+
+    public static Optional<IOException> extractIOExceptionCause(Throwable t)
+    {
+        if (t instanceof IOException)
+            return Optional.of((IOException) t);
+        Throwable cause = t;
+        while ((cause = cause.getCause()) != null)
+        {
+            if (cause instanceof IOException)
+                return Optional.of((IOException) cause);
+        }
+        return Optional.absent();
+    }
 }
diff --git a/src/java/org/apache/cassandra/utils/TopKSampler.java b/src/java/org/apache/cassandra/utils/TopKSampler.java
index a8bd602..37fcb60 100644
--- a/src/java/org/apache/cassandra/utils/TopKSampler.java
+++ b/src/java/org/apache/cassandra/utils/TopKSampler.java
@@ -110,7 +110,7 @@
                                 hll.offerHashed(hash);
                             } catch (Exception e)
                             {
-                                logger.debug("Failure to offer sample", e);
+                                logger.trace("Failure to offer sample", e);
                             }
                         }
                     }
diff --git a/src/java/org/apache/cassandra/utils/UUIDGen.java b/src/java/org/apache/cassandra/utils/UUIDGen.java
index 2c5b605..78b8b57 100644
--- a/src/java/org/apache/cassandra/utils/UUIDGen.java
+++ b/src/java/org/apache/cassandra/utils/UUIDGen.java
@@ -27,6 +27,7 @@
 import java.util.UUID;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Charsets;
 
 
 /**
@@ -223,27 +224,6 @@
     }
 
     /**
-     * Converts a milliseconds-since-epoch timestamp into the 16 byte representation
-     * of a type 1 UUID (a time-based UUID).
-     *
-     * <p><i><b>Deprecated:</b> This method goes again the principle of a time
-     * UUID and should not be used. For queries based on timestamp, minTimeUUID() and
-     * maxTimeUUID() can be used but this method has questionable usefulness. This is
-     * only kept because CQL2 uses it (see TimeUUID.fromStringCQL2) and we
-     * don't want to break compatibility.</i></p>
-     *
-     * <p><i><b>Warning:</b> This method is not guaranteed to return unique UUIDs; Multiple
-     * invocations using identical timestamps will result in identical UUIDs.</i></p>
-     *
-     * @param timeMillis
-     * @return a type 1 UUID represented as a byte[]
-     */
-    public static byte[] getTimeUUIDBytes(long timeMillis)
-    {
-        return createTimeUUIDBytes(instance.createTimeUnsafe(timeMillis));
-    }
-
-    /**
      * Converts a 100-nanoseconds precision timestamp into the 16 byte representation
      * of a type 1 UUID (a time-based UUID).
      *
@@ -314,12 +294,6 @@
         return createTime(nanosSince);
     }
 
-    /** @param when time in milliseconds */
-    private long createTimeUnsafe(long when)
-    {
-        return createTimeUnsafe(when, 0);
-    }
-
     private long createTimeUnsafe(long when, int nanos)
     {
         long nanosSince = ((when - START_EPOCH) * 10000) + nanos;
@@ -369,10 +343,21 @@
     {
         try
         {
+            // Identify the host.
             MessageDigest messageDigest = MessageDigest.getInstance("MD5");
             for(InetAddress addr : data)
                 messageDigest.update(addr.getAddress());
 
+            // Identify the process on the load: we use both the PID and class loader hash.
+            long pid = SigarLibrary.instance.getPid();
+            if (pid < 0)
+                pid = new Random(System.currentTimeMillis()).nextLong();
+            FBUtilities.updateWithLong(messageDigest, pid);
+
+            ClassLoader loader = UUIDGen.class.getClassLoader();
+            int loaderId = loader != null ? System.identityHashCode(loader) : 0;
+            FBUtilities.updateWithInt(messageDigest, loaderId);
+
             return messageDigest.digest();
         }
         catch (NoSuchAlgorithmException nsae)
diff --git a/src/java/org/apache/cassandra/utils/WindowsTimer.java b/src/java/org/apache/cassandra/utils/WindowsTimer.java
new file mode 100644
index 0000000..9db8559
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/WindowsTimer.java
@@ -0,0 +1,65 @@
+/*
+ * 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.utils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.sun.jna.LastErrorException;
+import com.sun.jna.Native;
+
+public final class WindowsTimer
+{
+    private static final Logger logger = LoggerFactory.getLogger(WindowsTimer.class);
+
+    static
+    {
+        try
+        {
+            Native.register("winmm");
+        }
+        catch (Exception e)
+        {
+            logger.error("Failed to register winmm.dll. Performance will be negatively impacted on this node.");
+        }
+    }
+
+    private static native int timeBeginPeriod(int period) throws LastErrorException;
+    private static native int timeEndPeriod(int period) throws LastErrorException;
+
+    private WindowsTimer() {}
+
+    public static void startTimerPeriod(int period)
+    {
+        if (period == 0)
+            return;
+        assert(period > 0);
+        if (timeBeginPeriod(period) != 0)
+            logger.warn("Failed to set timer to : " + period + ". Performance will be degraded.");
+    }
+
+    public static void endTimerPeriod(int period)
+    {
+        if (period == 0)
+            return;
+        assert(period > 0);
+        if (timeEndPeriod(period) != 0)
+            logger.warn("Failed to end accelerated timer period. System timer will remain set to: " + period + " ms.");
+    }
+}
diff --git a/src/java/org/apache/cassandra/utils/btree/BTreeSearchIterator.java b/src/java/org/apache/cassandra/utils/btree/BTreeSearchIterator.java
new file mode 100644
index 0000000..7a83238
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/btree/BTreeSearchIterator.java
@@ -0,0 +1,67 @@
+/*
+ * 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.utils.btree;
+
+import java.util.Comparator;
+
+import org.apache.cassandra.utils.SearchIterator;
+
+import static org.apache.cassandra.utils.btree.BTree.getKeyEnd;
+
+public class BTreeSearchIterator<CK, K extends CK, V> extends Path implements SearchIterator<K, V>
+{
+
+    final Comparator<CK> comparator;
+    public BTreeSearchIterator(Object[] btree, Comparator<CK> comparator)
+    {
+        init(btree);
+        this.comparator = comparator;
+    }
+
+    public V next(K target)
+    {
+        while (depth > 0)
+        {
+            byte successorParentDepth = findSuccessorParentDepth();
+            if (successorParentDepth < 0)
+                break; // we're in last section of tree, so can only search down
+            int successorParentIndex = indexes[successorParentDepth] + 1;
+            Object[] successParentNode = path[successorParentDepth];
+            Object successorParentKey = successParentNode[successorParentIndex];
+            int c = BTree.compare(comparator, target, successorParentKey);
+            if (c < 0)
+                break;
+            if (c == 0)
+            {
+                depth = successorParentDepth;
+                indexes[successorParentDepth]++;
+                return (V) successorParentKey;
+            }
+            depth = successorParentDepth;
+            indexes[successorParentDepth]++;
+        }
+        if (find(comparator, target, Op.CEIL, true))
+            return (V) currentKey();
+        return null;
+    }
+
+    public boolean hasNext()
+    {
+        return depth != 0 || indexes[0] != getKeyEnd(path[0]);
+    }
+}
diff --git a/src/java/org/apache/cassandra/utils/btree/Builder.java b/src/java/org/apache/cassandra/utils/btree/Builder.java
index f6677d4..aa4c5dc 100644
--- a/src/java/org/apache/cassandra/utils/btree/Builder.java
+++ b/src/java/org/apache/cassandra/utils/btree/Builder.java
@@ -18,7 +18,6 @@
  */
 package org.apache.cassandra.utils.btree;
 
-import java.util.Collection;
 import java.util.Comparator;
 
 import static org.apache.cassandra.utils.btree.BTree.EMPTY_LEAF;
diff --git a/src/java/org/apache/cassandra/utils/btree/Cursor.java b/src/java/org/apache/cassandra/utils/btree/Cursor.java
index 02e047a..6814d26 100644
--- a/src/java/org/apache/cassandra/utils/btree/Cursor.java
+++ b/src/java/org/apache/cassandra/utils/btree/Cursor.java
@@ -93,7 +93,7 @@
 
     private void _reset(Object[] btree, Comparator<K> comparator, Object lowerBound, boolean inclusiveLowerBound, Object upperBound, boolean inclusiveUpperBound, boolean forwards)
     {
-        ensureDepth(btree);
+        init(btree);
         if (lowerBound == null)
             lowerBound = NEGATIVE_INFINITY;
         if (upperBound == null)
@@ -101,16 +101,16 @@
 
         this.forwards = forwards;
 
-        Path findLast = new Path(this.path.length);
+        Path findLast = new Path(this.path.length, btree);
         if (forwards)
         {
-            findLast.find(btree, comparator, upperBound, inclusiveUpperBound ? Op.HIGHER : Op.CEIL, true);
-            find(btree, comparator, lowerBound, inclusiveLowerBound ? Op.CEIL : Op.HIGHER, true);
+            findLast.find(comparator, upperBound, inclusiveUpperBound ? Op.HIGHER : Op.CEIL, true);
+            find(comparator, lowerBound, inclusiveLowerBound ? Op.CEIL : Op.HIGHER, true);
         }
         else
         {
-            findLast.find(btree, comparator, lowerBound, inclusiveLowerBound ? Op.LOWER : Op.FLOOR, false);
-            find(btree, comparator, upperBound, inclusiveUpperBound ? Op.FLOOR : Op.LOWER, false);
+            findLast.find(comparator, lowerBound, inclusiveLowerBound ? Op.LOWER : Op.FLOOR, false);
+            find(comparator, upperBound, inclusiveUpperBound ? Op.FLOOR : Op.LOWER, false);
         }
         int c = this.compareTo(findLast, forwards);
         if (forwards ? c > 0 : c < 0)
diff --git a/src/java/org/apache/cassandra/utils/btree/NodeBuilder.java b/src/java/org/apache/cassandra/utils/btree/NodeBuilder.java
index 0e304e5..f683ec4 100644
--- a/src/java/org/apache/cassandra/utils/btree/NodeBuilder.java
+++ b/src/java/org/apache/cassandra/utils/btree/NodeBuilder.java
@@ -133,7 +133,7 @@
 
         int i = copyFromKeyPosition;
         boolean found; // exact key match?
-        boolean owns = true; // true iff this node (or a child) should contain the key
+        boolean owns = true; // true if this node (or a child) should contain the key
         if (i == copyFromKeyEnd)
         {
             found = false;
diff --git a/src/java/org/apache/cassandra/utils/btree/Path.java b/src/java/org/apache/cassandra/utils/btree/Path.java
index 51207ba..b1b0e03 100644
--- a/src/java/org/apache/cassandra/utils/btree/Path.java
+++ b/src/java/org/apache/cassandra/utils/btree/Path.java
@@ -36,7 +36,7 @@
  *
  * Path is only intended to be used via Cursor.
  */
-class Path
+public class Path<V>
 {
     // operations corresponding to the ones in NavigableSet
     static enum Op
@@ -55,13 +55,14 @@
     byte depth;
 
     Path() { }
-    Path(int depth)
+    Path(int depth, Object[] btree)
     {
         this.path = new Object[depth][];
         this.indexes = new byte[depth];
+        this.path[0] = btree;
     }
 
-    void ensureDepth(Object[] btree)
+    void init(Object[] btree)
     {
         int depth = BTree.depth(btree);
         if (path == null || path.length < depth)
@@ -69,6 +70,7 @@
             path = new Object[depth][];
             indexes = new byte[depth];
         }
+        path[0] = btree;
     }
 
     void moveEnd(Object[] node, boolean forwards)
@@ -88,21 +90,24 @@
     /**
      * Find the provided key in the tree rooted at node, and store the root to it in the path
      *
-     * @param node       the tree to search in
      * @param comparator the comparator defining the order on the tree
      * @param target     the key to search for
      * @param mode       the type of search to perform
      * @param forwards   if the path should be setup for forward or backward iteration
-     * @param <V>
+     * @param <K>
      */
-    <V> void find(Object[] node, Comparator<V> comparator, Object target, Op mode, boolean forwards)
+    <K> boolean find(Comparator<K> comparator, Object target, Op mode, boolean forwards)
     {
         // TODO : should not require parameter 'forwards' - consider modifying index to represent both
         // child and key position, as opposed to just key position (which necessitates a different value depending
         // on which direction you're moving in. Prerequisite for making Path public and using to implement general
         // search
 
-        depth = -1;
+        Object[] node = path[depth];
+        int lb = indexes[depth];
+        assert lb == 0 || forwards;
+        pop();
+
         if (target instanceof BTree.Special)
         {
             if (target == POSITIVE_INFINITY)
@@ -111,7 +116,7 @@
                 moveStart(node, forwards);
             else
                 throw new AssertionError();
-            return;
+            return false;
         }
 
         while (true)
@@ -119,7 +124,8 @@
             int keyEnd = getKeyEnd(node);
 
             // search for the target in the current node
-            int i = BTree.find(comparator, target, node, 0, keyEnd);
+            int i = BTree.find(comparator, target, node, lb, keyEnd);
+            lb = 0;
             if (i >= 0)
             {
                 // exact match. transform exclusive bounds into the correct index by moving back or forwards one
@@ -132,7 +138,7 @@
                     case LOWER:
                         predecessor();
                 }
-                return;
+                return true;
             }
             i = -i - 1;
 
@@ -167,16 +173,16 @@
                 push(node, i);
             }
 
-            return;
+            return false;
         }
     }
 
-    private boolean isRoot()
+    boolean isRoot()
     {
         return depth == 0;
     }
 
-    private void pop()
+    void pop()
     {
         depth--;
     }
@@ -191,7 +197,7 @@
         return indexes[depth];
     }
 
-    private void push(Object[] node, int index)
+    void push(Object[] node, int index)
     {
         path[++depth] = node;
         indexes[depth] = (byte) index;
@@ -202,6 +208,21 @@
         indexes[depth] = (byte) index;
     }
 
+    byte findSuccessorParentDepth()
+    {
+        byte depth = this.depth;
+        depth--;
+        while (depth >= 0)
+        {
+            int ub = indexes[depth] + 1;
+            Object[] node = path[depth];
+            if (ub < getBranchKeyEnd(node))
+                return depth;
+            depth--;
+        }
+        return -1;
+    }
+
     // move to the next key in the tree
     void successor()
     {
@@ -300,7 +321,7 @@
         return currentNode()[currentIndex()];
     }
 
-    int compareTo(Path that, boolean forwards)
+    int compareTo(Path<V> that, boolean forwards)
     {
         int d = Math.min(this.depth, that.depth);
         for (int i = 0; i <= d; i++)
diff --git a/src/java/org/apache/cassandra/utils/concurrent/Accumulator.java b/src/java/org/apache/cassandra/utils/concurrent/Accumulator.java
new file mode 100644
index 0000000..baecb34
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/concurrent/Accumulator.java
@@ -0,0 +1,133 @@
+/*
+* 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.utils.concurrent;
+
+import java.util.Iterator;
+import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
+
+/**
+ * A simple append-only collection supporting an unbounded number of concurrent readers/writers,
+ * but a bounded number of items.
+ *
+ * @param <E>
+ */
+public class Accumulator<E> implements Iterable<E>
+{
+    private volatile int nextIndex;
+    private volatile int presentCount;
+    private final Object[] values;
+    private static final AtomicIntegerFieldUpdater<Accumulator> nextIndexUpdater = AtomicIntegerFieldUpdater.newUpdater(Accumulator.class, "nextIndex");
+    private static final AtomicIntegerFieldUpdater<Accumulator> presentCountUpdater = AtomicIntegerFieldUpdater.newUpdater(Accumulator.class, "presentCount");
+
+    public Accumulator(int size)
+    {
+        values = new Object[size];
+    }
+
+    /**
+     * Adds an item to the collection.
+     *
+     * Note it is not guaranteed to be visible on exiting the method, if another add was happening concurrently;
+     * it will be visible once all concurrent adds (which are non-blocking) complete, but it is not guaranteed
+     * that any size change occurs during the execution of any specific call.
+     *
+     * @param item add to collection
+     */
+    public void add(E item)
+    {
+        int insertPos;
+        while (true)
+        {
+            insertPos = nextIndex;
+            if (insertPos >= values.length)
+                throw new IllegalStateException();
+            if (nextIndexUpdater.compareAndSet(this, insertPos, insertPos + 1))
+                break;
+        }
+        values[insertPos] = item;
+        // we then try to increase presentCount for each consecutive value that is visible after the current size;
+        // this should hopefully extend past us, but if it doesn't this behaviour means the lagging write will fix up
+        // our state for us.
+        //
+        // we piggyback off presentCountUpdater to get volatile write semantics for our update to values
+        boolean volatileWrite = false;
+        while (true)
+        {
+            int cur = presentCount;
+            if (cur != insertPos && (cur == values.length || values[cur] == null))
+            {
+                // ensure our item has been made visible before aborting
+                if (!volatileWrite && cur < insertPos && !presentCountUpdater.compareAndSet(this, cur, cur))
+                {
+                    // if we fail to CAS it means an older write has completed, and may have not fixed us up
+                    // due to our write not being visible
+                    volatileWrite = true;
+                    continue;
+                }
+                return;
+            }
+            presentCountUpdater.compareAndSet(this, cur, cur + 1);
+            volatileWrite = true;
+        }
+    }
+
+    public boolean isEmpty()
+    {
+        return presentCount == 0;
+    }
+
+    /**
+     * @return the size of guaranteed-to-be-visible portion of the list
+     */
+    public int size()
+    {
+        return presentCount;
+    }
+
+    public Iterator<E> iterator()
+    {
+        return new Iterator<E>()
+        {
+            int p = 0;
+
+            public boolean hasNext()
+            {
+                return p < presentCount;
+            }
+
+            public E next()
+            {
+                return (E) values[p++];
+            }
+
+            public void remove()
+            {
+                throw new UnsupportedOperationException();
+            }
+        };
+    }
+
+    public E get(int i)
+    {
+        // we read presentCount to guarantee a volatile read of values
+        if (i >= presentCount)
+            throw new IndexOutOfBoundsException();
+        return (E) values[i];
+    }
+}
diff --git a/src/java/org/apache/cassandra/utils/concurrent/Blocker.java b/src/java/org/apache/cassandra/utils/concurrent/Blocker.java
new file mode 100644
index 0000000..5192e98
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/concurrent/Blocker.java
@@ -0,0 +1,63 @@
+/*
+* 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.utils.concurrent;
+
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+public class Blocker
+{
+    private final ReentrantLock lock = new ReentrantLock();
+    private final Condition unblocked = lock.newCondition();
+    private volatile boolean block = false;
+
+    public void block(boolean block)
+    {
+        this.block = block;
+        if (!block)
+        {
+            lock.lock();
+            try
+            {
+                unblocked.signalAll();
+            }
+            finally
+            {
+                lock.unlock();
+            }
+        }
+    }
+
+    public void ask()
+    {
+        if (block)
+        {
+            lock.lock();
+            try
+            {
+                while (block)
+                    unblocked.awaitUninterruptibly();
+            }
+            finally
+            {
+                lock.unlock();
+            }
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/utils/concurrent/OpOrder.java b/src/java/org/apache/cassandra/utils/concurrent/OpOrder.java
index 5cebf44..b80fe99 100644
--- a/src/java/org/apache/cassandra/utils/concurrent/OpOrder.java
+++ b/src/java/org/apache/cassandra/utils/concurrent/OpOrder.java
@@ -30,6 +30,7 @@
  *
  * <p>The typical usage is something like:
  * <pre>
+ * {@code
      public final class ExampleShared
      {
         final OpOrder order = new OpOrder();
@@ -73,6 +74,7 @@
             }
         }
     }
+ * }
  * </pre>
  */
 public class OpOrder
diff --git a/src/java/org/apache/cassandra/utils/concurrent/Ref.java b/src/java/org/apache/cassandra/utils/concurrent/Ref.java
index 1983d26..65fe9a8 100644
--- a/src/java/org/apache/cassandra/utils/concurrent/Ref.java
+++ b/src/java/org/apache/cassandra/utils/concurrent/Ref.java
@@ -29,12 +29,14 @@
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
 
-import com.google.common.base.Throwables;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.concurrent.NamedThreadFactory;
 
+import static org.apache.cassandra.utils.Throwables.maybeFail;
+import static org.apache.cassandra.utils.Throwables.merge;
+
 /**
  * An object that needs ref counting does the two following:
  *   - defines a Tidy object that will cleanup once it's gone,
@@ -65,7 +67,7 @@
  * Once the Ref.GlobalState has been completely released, the Tidy method is called and it removes the global reference
  * to itself so it may also be collected.
  */
-public final class Ref<T> implements RefCounted<T>, AutoCloseable
+public final class Ref<T> implements RefCounted<T>
 {
     static final Logger logger = LoggerFactory.getLogger(Ref.class);
     public static final boolean DEBUG_ENABLED = System.getProperty("cassandra.debugrefcount", "false").equalsIgnoreCase("true");
@@ -95,14 +97,19 @@
         state.release(false);
     }
 
+    public Throwable ensureReleased(Throwable accumulate)
+    {
+        return state.ensureReleased(accumulate);
+    }
+
     public void ensureReleased()
     {
-        state.ensureReleased();
+        maybeFail(state.ensureReleased(null));
     }
 
     public void close()
     {
-        state.ensureReleased();
+        ensureReleased();
     }
 
     public T get()
@@ -168,14 +175,15 @@
             assert released == 0;
         }
 
-        void ensureReleased()
+        Throwable ensureReleased(Throwable accumulate)
         {
             if (releasedUpdater.getAndSet(this, 1) == 0)
             {
-                globalState.release(this);
+                accumulate = globalState.release(this, accumulate);
                 if (DEBUG_ENABLED)
                     debug.deallocate();
             }
+            return accumulate;
         }
 
         void release(boolean leak)
@@ -192,7 +200,7 @@
                 }
                 return;
             }
-            globalState.release(this);
+            Throwable fail = globalState.release(this, null);
             if (leak)
             {
                 String id = this.toString();
@@ -204,6 +212,8 @@
             {
                 debug.deallocate();
             }
+            if (fail != null)
+                logger.error("Error when closing {}", globalState, fail);
         }
     }
 
@@ -232,7 +242,7 @@
         String print(String thread, StackTraceElement[] trace)
         {
             StringBuilder sb = new StringBuilder();
-            sb.append(thread.toString());
+            sb.append(thread);
             sb.append("\n");
             for (StackTraceElement element : trace)
             {
@@ -282,7 +292,7 @@
         }
 
         // release a single reference, and cleanup if no more are extant
-        void release(Ref.State ref)
+        Throwable release(Ref.State ref, Throwable accumulate)
         {
             locallyExtant.remove(ref);
             if (-1 == counts.decrementAndGet())
@@ -294,10 +304,10 @@
                 }
                 catch (Throwable t)
                 {
-                    logger.error("Error when closing {}", this, t);
-                    Throwables.propagate(t);
+                    accumulate = merge(accumulate, t);
                 }
             }
+            return accumulate;
         }
 
         int count()
diff --git a/src/java/org/apache/cassandra/utils/concurrent/Refs.java b/src/java/org/apache/cassandra/utils/concurrent/Refs.java
index 0cd07c4..ec8d6c0 100644
--- a/src/java/org/apache/cassandra/utils/concurrent/Refs.java
+++ b/src/java/org/apache/cassandra/utils/concurrent/Refs.java
@@ -29,6 +29,7 @@
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Iterators;
 
+import static org.apache.cassandra.utils.Throwables.maybeFail;
 import static org.apache.cassandra.utils.Throwables.merge;
 
 /**
@@ -108,6 +109,12 @@
         return ref != null;
     }
 
+    public void relaseAllExcept(Collection<T> keep)
+    {
+        Collection<T> release = new ArrayList<>(references.keySet());
+        release.retainAll(keep);
+        release(release);
+    }
     /**
      * Release a retained Ref to all of the provided objects; if any is not held, an exception will be thrown
      * @param release
@@ -224,7 +231,10 @@
 
     public static void release(Iterable<? extends Ref<?>> refs)
     {
-        Throwable fail = null;
+        maybeFail(release(refs, null));
+    }
+    public static Throwable release(Iterable<? extends Ref<?>> refs, Throwable accumulate)
+    {
         for (Ref ref : refs)
         {
             try
@@ -233,11 +243,10 @@
             }
             catch (Throwable t)
             {
-                fail = merge(fail, t);
+                accumulate = merge(accumulate, t);
             }
         }
-        if (fail != null)
-            throw Throwables.propagate(fail);
+        return accumulate;
     }
 
     public static <T extends SelfRefCounted<T>> Iterable<Ref<T>> selfRefs(Iterable<T> refs)
diff --git a/src/java/org/apache/cassandra/utils/concurrent/SharedCloseable.java b/src/java/org/apache/cassandra/utils/concurrent/SharedCloseable.java
index 1e5a026..a3a1863 100644
--- a/src/java/org/apache/cassandra/utils/concurrent/SharedCloseable.java
+++ b/src/java/org/apache/cassandra/utils/concurrent/SharedCloseable.java
@@ -31,5 +31,6 @@
      * Throws an exception if the shared resource has already been closed.
      */
     public SharedCloseable sharedCopy();
+    public Throwable close(Throwable accumulate);
 
 }
diff --git a/src/java/org/apache/cassandra/utils/concurrent/SharedCloseableImpl.java b/src/java/org/apache/cassandra/utils/concurrent/SharedCloseableImpl.java
index 0d3a843..d85fd54 100644
--- a/src/java/org/apache/cassandra/utils/concurrent/SharedCloseableImpl.java
+++ b/src/java/org/apache/cassandra/utils/concurrent/SharedCloseableImpl.java
@@ -44,4 +44,9 @@
     {
         ref.ensureReleased();
     }
+
+    public Throwable close(Throwable accumulate)
+    {
+        return ref.ensureReleased(accumulate);
+    }
 }
diff --git a/src/java/org/apache/cassandra/utils/concurrent/Transactional.java b/src/java/org/apache/cassandra/utils/concurrent/Transactional.java
new file mode 100644
index 0000000..85c3de5
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/concurrent/Transactional.java
@@ -0,0 +1,219 @@
+/*
+* 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.utils.concurrent;
+
+import static org.apache.cassandra.utils.Throwables.maybeFail;
+import static org.apache.cassandra.utils.Throwables.merge;
+
+/**
+ * An abstraction for Transactional behaviour. An object implementing this interface has a lifetime
+ * of the following pattern:
+ *
+ * Throwable failure = null;
+ * try (Transactional t1, t2 = ...)
+ * {
+ *     // do work with t1 and t2
+ *     t1.prepareToCommit();
+ *     t2.prepareToCommit();
+ *     failure = t1.commit(failure);
+ *     failure = t2.commit(failure);
+ * }
+ * logger.error(failure);
+ *
+ * If something goes wrong before commit() is called on any transaction, then on exiting the try block
+ * the auto close method should invoke cleanup() and then abort() to reset any state.
+ * If everything completes normally, then on exiting the try block the auto close method will invoke cleanup
+ * to release any temporary state/resources
+ *
+ * No exceptions should be thrown during commit; if they are, it is not at all clear what the correct behaviour
+ * of the system should be, and so simply logging the exception is likely best (since it may have been an issue
+ * during cleanup, say), and rollback cannot now occur. As such all exceptions and assertions that may be thrown
+ * should be checked and ruled out during commit preparation.
+ *
+ * Since Transactional implementations will abort any changes they've made if calls to prepareToCommit() and commit()
+ * aren't made prior to calling close(), the semantics of its close() method differ significantly from
+ * most AutoCloseable implementations.
+ */
+public interface Transactional extends AutoCloseable
+{
+
+    /**
+     * A simple abstract implementation of Transactional behaviour.
+     * In general this should be used as the base class for any transactional implementations.
+     *
+     * If the implementation wraps any internal Transactional objects, it must proxy every
+     * commit() and abort() call onto each internal object to ensure correct behaviour
+     */
+    public static abstract class AbstractTransactional implements Transactional
+    {
+        public static enum State
+        {
+            IN_PROGRESS,
+            READY_TO_COMMIT,
+            COMMITTED,
+            ABORTED;
+        }
+
+        private boolean permitRedundantTransitions;
+        private State state = State.IN_PROGRESS;
+
+        // the methods for actually performing the necessary behaviours, that are themselves protected against
+        // improper use by the external implementations provided by this class. empty default implementations
+        // could be provided, but we consider it safer to force implementers to consider explicitly their presence
+
+        protected abstract Throwable doCommit(Throwable accumulate);
+        protected abstract Throwable doAbort(Throwable accumulate);
+
+        // these only needs to perform cleanup of state unique to this instance; any internal
+        // Transactional objects will perform cleanup in the commit() or abort() calls
+
+        /**
+         * perform an exception-safe pre-abort cleanup; this will still be run *after* commit
+         */
+        protected Throwable doPreCleanup(Throwable accumulate){ return accumulate; }
+
+        /**
+         * perform an exception-safe post-abort cleanup
+         */
+        protected Throwable doPostCleanup(Throwable accumulate){ return accumulate; }
+
+        /**
+         * Do any preparatory work prior to commit. This method should throw any exceptions that can be encountered
+         * during the finalization of the behaviour.
+         */
+        protected abstract void doPrepare();
+
+        /**
+         * commit any effects of this transaction object graph, then cleanup; delegates first to doCommit, then to doCleanup
+         */
+        public final Throwable commit(Throwable accumulate)
+        {
+            if (permitRedundantTransitions && state == State.COMMITTED)
+                return accumulate;
+            if (state != State.READY_TO_COMMIT)
+                throw new IllegalStateException("Cannot commit unless READY_TO_COMMIT; state is " + state);
+            accumulate = doCommit(accumulate);
+            accumulate = doPreCleanup(accumulate);
+            accumulate = doPostCleanup(accumulate);
+            state = State.COMMITTED;
+            return accumulate;
+        }
+
+        /**
+         * rollback any effects of this transaction object graph; delegates first to doCleanup, then to doAbort
+         */
+        public final Throwable abort(Throwable accumulate)
+        {
+            if (state == State.ABORTED)
+                return accumulate;
+            if (state == State.COMMITTED)
+            {
+                try
+                {
+                    throw new IllegalStateException("Attempted to abort a committed operation");
+                }
+                catch (Throwable t)
+                {
+                    accumulate = merge(accumulate, t);
+                }
+                return accumulate;
+            }
+            state = State.ABORTED;
+            // we cleanup first so that, e.g., file handles can be released prior to deletion
+            accumulate = doPreCleanup(accumulate);
+            accumulate = doAbort(accumulate);
+            accumulate = doPostCleanup(accumulate);
+            return accumulate;
+        }
+
+        // if we are committed or aborted, then we are done; otherwise abort
+        public final void close()
+        {
+            switch (state)
+            {
+                case COMMITTED:
+                case ABORTED:
+                    break;
+                default:
+                    abort();
+            }
+        }
+
+        /**
+         * The first phase of commit: delegates to doPrepare(), with valid state transition enforcement.
+         * This call should be propagated onto any child objects participating in the transaction
+         */
+        public final void prepareToCommit()
+        {
+            if (permitRedundantTransitions && state == State.READY_TO_COMMIT)
+                return;
+            if (state != State.IN_PROGRESS)
+                throw new IllegalStateException("Cannot prepare to commit unless IN_PROGRESS; state is " + state);
+
+            doPrepare();
+            state = State.READY_TO_COMMIT;
+        }
+
+        /**
+         * convenience method to both prepareToCommit() and commit() in one operation;
+         * only of use to outer-most transactional object of an object graph
+         */
+        public Object finish()
+        {
+            prepareToCommit();
+            commit();
+            return this;
+        }
+
+        // convenience method wrapping abort, and throwing any exception encountered
+        // only of use to (and to be used by) outer-most object in a transactional graph
+        public final void abort()
+        {
+            maybeFail(abort(null));
+        }
+
+        // convenience method wrapping commit, and throwing any exception encountered
+        // only of use to (and to be used by) outer-most object in a transactional graph
+        public final void commit()
+        {
+            maybeFail(commit(null));
+        }
+
+        public final State state()
+        {
+            return state;
+        }
+
+        protected void permitRedundantTransitions()
+        {
+            permitRedundantTransitions = true;
+        }
+    }
+
+    // commit should generally never throw an exception, and preferably never generate one,
+    // but if it does generate one it should accumulate it in the parameter and return the result
+    // IF a commit implementation has a real correctness affecting exception that cannot be moved to
+    // prepareToCommit, it MUST be executed before any other commit methods in the object graph
+    public Throwable commit(Throwable accumulate);
+
+    // release any resources, then rollback all state changes (unless commit() has already been invoked)
+    public Throwable abort(Throwable accumulate);
+
+    public void prepareToCommit();
+}
diff --git a/src/java/org/apache/cassandra/utils/concurrent/WaitQueue.java b/src/java/org/apache/cassandra/utils/concurrent/WaitQueue.java
index 2322210..be271b6 100644
--- a/src/java/org/apache/cassandra/utils/concurrent/WaitQueue.java
+++ b/src/java/org/apache/cassandra/utils/concurrent/WaitQueue.java
@@ -18,17 +18,13 @@
  */
 package org.apache.cassandra.utils.concurrent;
 
-import com.yammer.metrics.core.TimerContext;
-import org.slf4j.*;
-
-import java.util.ArrayList;
 import java.util.Iterator;
-import java.util.List;
-import java.util.concurrent.ConcurrentLinkedDeque;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
 import java.util.concurrent.locks.LockSupport;
 
+import com.codahale.metrics.Timer;
+
 /**
  * <p>A relatively easy to use utility for general purpose thread signalling.</p>
  * <p>Usage on a thread awaiting a state change using a WaitQueue q is:</p>
@@ -100,7 +96,7 @@
      * or the waiting thread is interrupted.
      * @return
      */
-    public Signal register(TimerContext context)
+    public Signal register(Timer.Context context)
     {
         assert context != null;
         RegisteredSignal signal = new TimedSignal(context);
@@ -393,9 +389,9 @@
      */
     private final class TimedSignal extends RegisteredSignal
     {
-        private final TimerContext context;
+        private final Timer.Context context;
 
-        private TimedSignal(TimerContext context)
+        private TimedSignal(Timer.Context context)
         {
             this.context = context;
         }
diff --git a/src/java/org/apache/cassandra/utils/memory/HeapPool.java b/src/java/org/apache/cassandra/utils/memory/HeapPool.java
index 2a19d9c..d28dbf7 100644
--- a/src/java/org/apache/cassandra/utils/memory/HeapPool.java
+++ b/src/java/org/apache/cassandra/utils/memory/HeapPool.java
@@ -43,7 +43,7 @@
         return new Allocator(this);
     }
 
-    public class Allocator extends MemtableBufferAllocator
+    public static class Allocator extends MemtableBufferAllocator
     {
         Allocator(HeapPool pool)
         {
diff --git a/src/java/org/apache/cassandra/utils/memory/MemoryUtil.java b/src/java/org/apache/cassandra/utils/memory/MemoryUtil.java
index f96957d..25d2dd8 100644
--- a/src/java/org/apache/cassandra/utils/memory/MemoryUtil.java
+++ b/src/java/org/apache/cassandra/utils/memory/MemoryUtil.java
@@ -22,8 +22,9 @@
 import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
 
-import com.google.common.primitives.*;
+import com.sun.jna.Native;
 import sun.misc.Unsafe;
+import sun.nio.ch.DirectBuffer;
 
 public abstract class MemoryUtil
 {
@@ -34,6 +35,10 @@
     private static final long DIRECT_BYTE_BUFFER_ADDRESS_OFFSET;
     private static final long DIRECT_BYTE_BUFFER_CAPACITY_OFFSET;
     private static final long DIRECT_BYTE_BUFFER_LIMIT_OFFSET;
+    private static final long DIRECT_BYTE_BUFFER_POSITION_OFFSET;
+    private static final Class<?> BYTE_BUFFER_CLASS;
+    private static final long BYTE_BUFFER_OFFSET_OFFSET;
+    private static final long BYTE_BUFFER_HB_OFFSET;
     private static final long BYTE_ARRAY_BASE_OFFSET;
 
     private static final boolean BIG_ENDIAN = ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN);
@@ -56,7 +61,14 @@
             DIRECT_BYTE_BUFFER_ADDRESS_OFFSET = unsafe.objectFieldOffset(Buffer.class.getDeclaredField("address"));
             DIRECT_BYTE_BUFFER_CAPACITY_OFFSET = unsafe.objectFieldOffset(Buffer.class.getDeclaredField("capacity"));
             DIRECT_BYTE_BUFFER_LIMIT_OFFSET = unsafe.objectFieldOffset(Buffer.class.getDeclaredField("limit"));
+            DIRECT_BYTE_BUFFER_POSITION_OFFSET = unsafe.objectFieldOffset(Buffer.class.getDeclaredField("position"));
             DIRECT_BYTE_BUFFER_CLASS = clazz;
+
+            clazz = ByteBuffer.allocate(0).getClass();
+            BYTE_BUFFER_OFFSET_OFFSET = unsafe.objectFieldOffset(ByteBuffer.class.getDeclaredField("offset"));
+            BYTE_BUFFER_HB_OFFSET = unsafe.objectFieldOffset(ByteBuffer.class.getDeclaredField("hb"));
+            BYTE_BUFFER_CLASS = clazz;
+
             BYTE_ARRAY_BASE_OFFSET = unsafe.arrayBaseOffset(byte[].class);
         }
         catch (Exception e)
@@ -65,6 +77,16 @@
         }
     }
 
+    public static long allocate(long size)
+    {
+        return Native.malloc(size);
+    }
+
+    public static void free(long peer)
+    {
+        Native.free(peer);
+    }
+
     public static void setByte(long address, byte b)
     {
         unsafe.putByte(address, b);
@@ -98,7 +120,7 @@
 
     public static int getShort(long address)
     {
-        return UNALIGNED ? unsafe.getShort(address) & 0xffff : getShortByByte(address);
+        return (UNALIGNED ? unsafe.getShort(address) : getShortByByte(address)) & 0xffff;
     }
 
     public static int getInt(long address)
@@ -113,6 +135,13 @@
 
     public static ByteBuffer getByteBuffer(long address, int length)
     {
+        ByteBuffer instance = getHollowDirectByteBuffer();
+        setByteBuffer(instance, address, length);
+        return instance;
+    }
+
+    public static ByteBuffer getHollowDirectByteBuffer()
+    {
         ByteBuffer instance;
         try
         {
@@ -122,12 +151,51 @@
         {
             throw new AssertionError(e);
         }
+        instance.order(ByteOrder.nativeOrder());
+        return instance;
+    }
 
+    public static ByteBuffer getHollowByteBuffer()
+    {
+        ByteBuffer instance;
+        try
+        {
+            instance = (ByteBuffer) unsafe.allocateInstance(BYTE_BUFFER_CLASS);
+        }
+        catch (InstantiationException e)
+        {
+            throw new AssertionError(e);
+        }
+        instance.order(ByteOrder.nativeOrder());
+        return instance;
+    }
+
+    public static void setByteBuffer(ByteBuffer instance, long address, int length)
+    {
         unsafe.putLong(instance, DIRECT_BYTE_BUFFER_ADDRESS_OFFSET, address);
         unsafe.putInt(instance, DIRECT_BYTE_BUFFER_CAPACITY_OFFSET, length);
         unsafe.putInt(instance, DIRECT_BYTE_BUFFER_LIMIT_OFFSET, length);
-        instance.order(ByteOrder.nativeOrder());
-        return instance;
+    }
+
+    public static ByteBuffer duplicateDirectByteBuffer(ByteBuffer source, ByteBuffer hollowBuffer)
+    {
+        assert(source.isDirect());
+        unsafe.putLong(hollowBuffer, DIRECT_BYTE_BUFFER_ADDRESS_OFFSET, unsafe.getLong(source, DIRECT_BYTE_BUFFER_ADDRESS_OFFSET));
+        unsafe.putInt(hollowBuffer, DIRECT_BYTE_BUFFER_POSITION_OFFSET, unsafe.getInt(source, DIRECT_BYTE_BUFFER_POSITION_OFFSET));
+        unsafe.putInt(hollowBuffer, DIRECT_BYTE_BUFFER_LIMIT_OFFSET, unsafe.getInt(source, DIRECT_BYTE_BUFFER_LIMIT_OFFSET));
+        unsafe.putInt(hollowBuffer, DIRECT_BYTE_BUFFER_CAPACITY_OFFSET, unsafe.getInt(source, DIRECT_BYTE_BUFFER_CAPACITY_OFFSET));
+        return hollowBuffer;
+    }
+
+    public static ByteBuffer duplicateByteBuffer(ByteBuffer source, ByteBuffer hollowBuffer)
+    {
+        assert(!source.isDirect());
+        unsafe.putInt(hollowBuffer, DIRECT_BYTE_BUFFER_POSITION_OFFSET, unsafe.getInt(source, DIRECT_BYTE_BUFFER_POSITION_OFFSET));
+        unsafe.putInt(hollowBuffer, DIRECT_BYTE_BUFFER_LIMIT_OFFSET, unsafe.getInt(source, DIRECT_BYTE_BUFFER_LIMIT_OFFSET));
+        unsafe.putInt(hollowBuffer, DIRECT_BYTE_BUFFER_CAPACITY_OFFSET, unsafe.getInt(source, DIRECT_BYTE_BUFFER_CAPACITY_OFFSET));
+        unsafe.putInt(hollowBuffer, BYTE_BUFFER_OFFSET_OFFSET, unsafe.getInt(source, BYTE_BUFFER_OFFSET_OFFSET));
+        unsafe.putObject(hollowBuffer, BYTE_BUFFER_HB_OFFSET, unsafe.getObject(source, BYTE_BUFFER_HB_OFFSET));
+        return hollowBuffer;
     }
 
     public static long getLongByByte(long address)
@@ -241,7 +309,7 @@
             return;
 
         if (buffer.isDirect())
-            setBytes(unsafe.getLong(buffer, DIRECT_BYTE_BUFFER_ADDRESS_OFFSET) + start, address, count);
+            setBytes(((DirectBuffer)buffer).address() + start, address, count);
         else
             setBytes(address, buffer.array(), buffer.arrayOffset() + start, count);
     }
diff --git a/src/java/org/apache/cassandra/utils/memory/NativeAllocator.java b/src/java/org/apache/cassandra/utils/memory/NativeAllocator.java
index 272a4c2..98878c0 100644
--- a/src/java/org/apache/cassandra/utils/memory/NativeAllocator.java
+++ b/src/java/org/apache/cassandra/utils/memory/NativeAllocator.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.utils.memory;
 
-import java.lang.reflect.Field;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.concurrent.ConcurrentLinkedQueue;
@@ -26,7 +25,6 @@
 import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.Cell;
 import org.apache.cassandra.db.CounterCell;
 import org.apache.cassandra.db.DecoratedKey;
@@ -37,18 +35,24 @@
 import org.apache.cassandra.db.NativeDecoratedKey;
 import org.apache.cassandra.db.NativeDeletedCell;
 import org.apache.cassandra.db.NativeExpiringCell;
-import org.apache.cassandra.io.util.IAllocator;
 import org.apache.cassandra.utils.concurrent.OpOrder;
-import sun.misc.Unsafe;
 
+/**
+ * This NativeAllocator uses global slab allocation strategy
+ * with slab size that scales exponentially from 8kb to 1Mb to
+ * serve allocation of up to 128kb.
+ * <p>
+ * </p>
+ * The slab allocation reduces heap fragmentation from small
+ * long-lived objects.
+ *
+ */
 public class NativeAllocator extends MemtableAllocator
 {
     private final static int MAX_REGION_SIZE = 1 * 1024 * 1024;
     private final static int MAX_CLONED_SIZE = 128 * 1024; // bigger than this don't go in the region
     private final static int MIN_REGION_SIZE = 8 * 1024;
 
-    private static final IAllocator allocator = DatabaseDescriptor.getoffHeapMemoryAllocator();
-
     // globally stash any Regions we allocate but are beaten to using, and use these up before allocating any more
     private static final Map<Integer, RaceAllocated> RACE_ALLOCATED = new HashMap<>();
 
@@ -108,7 +112,7 @@
         // satisfy large allocations directly from JVM since they don't cause fragmentation
         // as badly, and fill up our regions quickly
         if (size > MAX_CLONED_SIZE)
-            return allocateOversize(size, opGroup);
+            return allocateOversize(size);
 
         while (true)
         {
@@ -139,21 +143,21 @@
 
         // if there are none, we allocate one
         if (next == null)
-            next = new Region(allocator.allocate(size), size);
+            next = new Region(MemoryUtil.allocate(size), size);
 
         // we try to swap in the region we've obtained;
         // if we fail to swap the region, we try to stash it for repurposing later; if we're out of stash room, we free it
         if (currentRegion.compareAndSet(current, next))
             regions.add(next);
         else if (!raceAllocated.stash(next))
-            allocator.free(next.peer);
+            MemoryUtil.free(next.peer);
     }
 
-    private long allocateOversize(int size, OpOrder.Group opGroup)
+    private long allocateOversize(int size)
     {
         // satisfy large allocations directly from JVM since they don't cause fragmentation
         // as badly, and fill up our regions quickly
-        Region region = new Region(allocator.allocate(size), size);
+        Region region = new Region(MemoryUtil.allocate(size), size);
         regions.add(region);
 
         long peer;
@@ -166,7 +170,7 @@
     public void setDiscarded()
     {
         for (Region region : regions)
-            allocator.free(region.peer);
+            MemoryUtil.free(region.peer);
         super.setDiscarded();
     }
 
diff --git a/src/java/org/apache/cassandra/utils/memory/SlabAllocator.java b/src/java/org/apache/cassandra/utils/memory/SlabAllocator.java
index 19334ee..8ffead1 100644
--- a/src/java/org/apache/cassandra/utils/memory/SlabAllocator.java
+++ b/src/java/org/apache/cassandra/utils/memory/SlabAllocator.java
@@ -31,15 +31,15 @@
 import sun.nio.ch.DirectBuffer;
 
 /**
- * The SlabAllocator is a bump-the-pointer allocator that allocates
- * large (2MB by default) regions and then doles them out to threads that request
- * slices into the array.
- * <p/>
++ * The SlabAllocator is a bump-the-pointer allocator that allocates
++ * large (1MB) global regions and then doles them out to threads that
++ * request smaller sized (up to 128kb) slices into the array.
+ * <p></p>
  * The purpose of this class is to combat heap fragmentation in long lived
  * objects: by ensuring that all allocations with similar lifetimes
  * only to large regions of contiguous memory, we ensure that large blocks
  * get freed up at the same time.
- * <p/>
+ * <p></p>
  * Otherwise, variable length byte arrays allocated end up
  * interleaved throughout the heap, and the old generation gets progressively
  * more fragmented until a stop-the-world compacting collection occurs.
diff --git a/src/java/org/apache/cassandra/utils/obs/OffHeapBitSet.java b/src/java/org/apache/cassandra/utils/obs/OffHeapBitSet.java
index 02efa65..46c1bd0 100644
--- a/src/java/org/apache/cassandra/utils/obs/OffHeapBitSet.java
+++ b/src/java/org/apache/cassandra/utils/obs/OffHeapBitSet.java
@@ -125,6 +125,7 @@
         return type.sizeof((int) bytes.size()) + bytes.size();
     }
 
+    @SuppressWarnings("resource")
     public static OffHeapBitSet deserialize(DataInput in) throws IOException
     {
         long byteCount = in.readInt() * 8L;
diff --git a/src/java/org/apache/cassandra/utils/obs/OpenBitSet.java b/src/java/org/apache/cassandra/utils/obs/OpenBitSet.java
index 7edf06f..e793f6c 100644
--- a/src/java/org/apache/cassandra/utils/obs/OpenBitSet.java
+++ b/src/java/org/apache/cassandra/utils/obs/OpenBitSet.java
@@ -25,25 +25,30 @@
 import org.apache.cassandra.db.TypeSizes;
 
 /**
+ * <p>
  * An "open" BitSet implementation that allows direct access to the arrays of words
  * storing the bits.  Derived from Lucene's OpenBitSet, but with a paged backing array
  * (see bits delaration, below).
- * <p/>
+ * </p>
+ * <p>
  * Unlike java.util.bitset, the fact that bits are packed into an array of longs
  * is part of the interface.  This allows efficient implementation of other algorithms
  * by someone other than the author.  It also allows one to efficiently implement
  * alternate serialization or interchange formats.
- * <p/>
+ * </p>
+ * <p>
  * <code>OpenBitSet</code> is faster than <code>java.util.BitSet</code> in most operations
  * and *much* faster at calculating cardinality of sets and results of set operations.
  * It can also handle sets of larger cardinality (up to 64 * 2**32-1)
- * <p/>
+ * </p>
+ * <p>
  * The goals of <code>OpenBitSet</code> are the fastest implementation possible, and
  * maximum code reuse.  Extra safety and encapsulation
  * may always be built on top, but if that's built in, the cost can never be removed (and
  * hence people re-implement their own version in order to get better performance).
  * If you want a "safe", totally encapsulated (and slower and limited) BitSet
  * class, use <code>java.util.BitSet</code>.
+ * </p>
  */
 
 public class OpenBitSet implements IBitSet
diff --git a/src/java/org/apache/cassandra/utils/progress/ProgressEvent.java b/src/java/org/apache/cassandra/utils/progress/ProgressEvent.java
new file mode 100644
index 0000000..31d3120
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/progress/ProgressEvent.java
@@ -0,0 +1,75 @@
+/*
+ * 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.utils.progress;
+
+/**
+ * Progress event
+ */
+public class ProgressEvent
+{
+    private final ProgressEventType type;
+    private final int progressCount;
+    private final int total;
+    private final String message;
+
+    public static ProgressEvent createNotification(String message)
+    {
+        return new ProgressEvent(ProgressEventType.NOTIFICATION, 0, 0, message);
+    }
+
+    public ProgressEvent(ProgressEventType type, int progressCount, int total)
+    {
+        this(type, progressCount, total, null);
+    }
+
+    public ProgressEvent(ProgressEventType type, int progressCount, int total, String message)
+    {
+        this.type = type;
+        this.progressCount = progressCount;
+        this.total = total;
+        this.message = message;
+    }
+
+    public ProgressEventType getType()
+    {
+        return type;
+    }
+
+    public int getProgressCount()
+    {
+        return progressCount;
+    }
+
+    public int getTotal()
+    {
+        return total;
+    }
+
+    public double getProgressPercentage()
+    {
+        return total != 0 ? progressCount * 100 / (double) total : 0;
+    }
+
+    /**
+     * @return Message attached to this event. Can be null.
+     */
+    public String getMessage()
+    {
+        return message;
+    }
+}
diff --git a/src/java/org/apache/cassandra/utils/progress/ProgressEventNotifier.java b/src/java/org/apache/cassandra/utils/progress/ProgressEventNotifier.java
new file mode 100644
index 0000000..07a6618
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/progress/ProgressEventNotifier.java
@@ -0,0 +1,38 @@
+/*
+ * 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.utils.progress;
+
+/**
+ * Interface for {@link ProgressEvent} publisher.
+ */
+public interface ProgressEventNotifier
+{
+    /**
+     * Register progress listener to this publisher.
+     *
+     * @param listener listener to register.
+     */
+    void addProgressListener(ProgressListener listener);
+
+    /**
+     * Remove progress listener from this publisher.
+     *
+     * @param listener listener to remove
+     */
+    void removeProgressListener(ProgressListener listener);
+}
diff --git a/src/java/org/apache/cassandra/utils/progress/ProgressEventNotifierSupport.java b/src/java/org/apache/cassandra/utils/progress/ProgressEventNotifierSupport.java
new file mode 100644
index 0000000..f5fe795
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/progress/ProgressEventNotifierSupport.java
@@ -0,0 +1,49 @@
+/*
+ * 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.utils.progress;
+
+import java.util.List;
+import java.util.concurrent.CopyOnWriteArrayList;
+
+/**
+ * Provides basic, thread safe ProgressEvent notification support
+ */
+public abstract class ProgressEventNotifierSupport implements ProgressEventNotifier
+{
+    private List<ProgressListener> listeners = new CopyOnWriteArrayList<>();
+
+    @Override
+    public void addProgressListener(ProgressListener listener)
+    {
+        listeners.add(listener);
+    }
+
+    @Override
+    public void removeProgressListener(ProgressListener listener)
+    {
+        listeners.remove(listener);
+    }
+
+    protected void fireProgressEvent(String tag, ProgressEvent event)
+    {
+        for (ProgressListener listener : listeners)
+        {
+            listener.progress(tag, event);
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/utils/progress/ProgressEventType.java b/src/java/org/apache/cassandra/utils/progress/ProgressEventType.java
new file mode 100644
index 0000000..8d7daee
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/progress/ProgressEventType.java
@@ -0,0 +1,72 @@
+/*
+ * 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.utils.progress;
+
+/**
+ * Progress event type.
+ *
+ * <p>
+ * Progress starts by emitting {@link #START}, followed by emitting zero or more {@link #PROGRESS} events,
+ * then it emits either one of {@link #ERROR}/{@link #ABORT}/{@link #SUCCESS}.
+ * Progress indicates its completion by emitting {@link #COMPLETE} at the end of process.
+ * </p>
+ * <p>
+ * {@link #NOTIFICATION} event type is used to just notify message without progress.
+ * </p>
+ */
+public enum ProgressEventType
+{
+    /**
+     * Fired first when progress starts.
+     * Happens only once.
+     */
+    START,
+
+    /**
+     * Fire when progress happens.
+     * This can be zero or more time after START.
+     */
+    PROGRESS,
+
+    /**
+     * When observing process completes with error, this is sent once before COMPLETE.
+     */
+    ERROR,
+
+    /**
+     * When observing process is aborted by user, this is sent once before COMPLETE.
+     */
+    ABORT,
+
+    /**
+     * When observing process completes successfully, this is sent once before COMPLETE.
+     */
+    SUCCESS,
+
+    /**
+     * Fire when progress complete.
+     * This is fired once, after ERROR/ABORT/SUCCESS is fired.
+     * After this, no more ProgressEvent should be fired for the same event.
+     */
+    COMPLETE,
+
+    /**
+     * Used when sending message without progress.
+     */
+    NOTIFICATION
+}
diff --git a/src/java/org/apache/cassandra/utils/progress/ProgressListener.java b/src/java/org/apache/cassandra/utils/progress/ProgressListener.java
new file mode 100644
index 0000000..48342a8
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/progress/ProgressListener.java
@@ -0,0 +1,32 @@
+/*
+ * 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.utils.progress;
+
+/**
+ * Listener interface to handle {@link org.apache.cassandra.utils.progress.ProgressEvent}
+ */
+public interface ProgressListener
+{
+    /**
+     * Called when some progress is made by progress publisher.
+     *
+     * @param tag String that identifies progress event.
+     * @param event Current progress
+     */
+    void progress(String tag, ProgressEvent event);
+}
diff --git a/src/java/org/apache/cassandra/utils/progress/jmx/JMXNotificationProgressListener.java b/src/java/org/apache/cassandra/utils/progress/jmx/JMXNotificationProgressListener.java
new file mode 100644
index 0000000..3461487
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/progress/jmx/JMXNotificationProgressListener.java
@@ -0,0 +1,94 @@
+/*
+ * 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.utils.progress.jmx;
+
+import java.util.Map;
+import javax.management.Notification;
+import javax.management.NotificationListener;
+import javax.management.remote.JMXConnectionNotification;
+
+import org.apache.cassandra.utils.progress.ProgressEvent;
+import org.apache.cassandra.utils.progress.ProgressEventType;
+import org.apache.cassandra.utils.progress.ProgressListener;
+
+/**
+ * JMXNotificationProgressListener uses JMX Notification API to convert JMX Notification message to progress event
+ * and notifies its {@link ProgressListener}s.
+ *
+ * This is to be implemented in client tools side.
+ */
+public abstract class JMXNotificationProgressListener implements ProgressListener, NotificationListener
+{
+    /**
+     * @param tag tag name to be checked
+     * @return true if given tag for ProgressEvent is a target to consume. If this returns false, then
+     *         {@link #progress} is not called for that event.
+     */
+    public abstract boolean isInterestedIn(String tag);
+
+    /**
+     * Called when receiving {@link JMXConnectionNotification#NOTIFS_LOST} message.
+     */
+    public void handleNotificationLost(long timestamp, String message) {}
+
+    /**
+     * Called when JMX connection is closed.
+     * Specifically when {@link JMXConnectionNotification#CLOSED} message is received.
+     */
+    public void handleConnectionClosed(long timestamp, String message) {}
+
+    /**
+     * Called when JMX connection is failed.
+     * Specifically when {@link JMXConnectionNotification#FAILED} message is received.
+     */
+    public void handleConnectionFailed(long timestamp, String message) {}
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void handleNotification(Notification notification, Object handback)
+    {
+        switch (notification.getType())
+        {
+            case "progress":
+                String tag = (String) notification.getSource();
+                if (this.isInterestedIn(tag))
+                {
+                    Map<String, Integer> progress = (Map<String, Integer>) notification.getUserData();
+                    String message = notification.getMessage();
+                    ProgressEvent event = new ProgressEvent(ProgressEventType.values()[progress.get("type")],
+                                                            progress.get("progressCount"),
+                                                            progress.get("total"),
+                                                            message);
+                    this.progress(tag, event);
+                }
+                break;
+
+            case JMXConnectionNotification.NOTIFS_LOST:
+                handleNotificationLost(notification.getTimeStamp(), notification.getMessage());
+                break;
+
+            case JMXConnectionNotification.FAILED:
+                handleConnectionFailed(notification.getTimeStamp(), notification.getMessage());
+                break;
+
+            case JMXConnectionNotification.CLOSED:
+                handleConnectionClosed(notification.getTimeStamp(), notification.getMessage());
+                break;
+        }
+    }
+}
diff --git a/src/java/org/apache/cassandra/utils/progress/jmx/JMXProgressSupport.java b/src/java/org/apache/cassandra/utils/progress/jmx/JMXProgressSupport.java
new file mode 100644
index 0000000..12efd0d
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/progress/jmx/JMXProgressSupport.java
@@ -0,0 +1,58 @@
+/*
+ * 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.utils.progress.jmx;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
+import javax.management.Notification;
+import javax.management.NotificationBroadcasterSupport;
+
+import org.apache.cassandra.utils.progress.ProgressEvent;
+import org.apache.cassandra.utils.progress.ProgressListener;
+
+/**
+ * ProgressListener that translates ProgressEvent to JMX Notification message.
+ */
+public class JMXProgressSupport implements ProgressListener
+{
+    private final AtomicLong notificationSerialNumber = new AtomicLong();
+
+    private final NotificationBroadcasterSupport broadcaster;
+
+    public JMXProgressSupport(NotificationBroadcasterSupport broadcaster)
+    {
+        this.broadcaster = broadcaster;
+    }
+
+    @Override
+    public void progress(String tag, ProgressEvent event)
+    {
+        Notification notification = new Notification("progress",
+                                                     tag,
+                                                     notificationSerialNumber.getAndIncrement(),
+                                                     System.currentTimeMillis(),
+                                                     event.getMessage());
+        Map<String, Integer> userData = new HashMap<>();
+        userData.put("type", event.getType().ordinal());
+        userData.put("progressCount", event.getProgressCount());
+        userData.put("total", event.getTotal());
+        notification.setUserData(userData);
+        broadcaster.sendNotification(notification);
+    }
+}
diff --git a/src/java/org/apache/cassandra/utils/progress/jmx/LegacyJMXProgressSupport.java b/src/java/org/apache/cassandra/utils/progress/jmx/LegacyJMXProgressSupport.java
new file mode 100644
index 0000000..fae6f2a
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/progress/jmx/LegacyJMXProgressSupport.java
@@ -0,0 +1,108 @@
+/*
+ * 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.utils.progress.jmx;
+
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.regex.Pattern;
+import javax.management.Notification;
+import javax.management.NotificationBroadcasterSupport;
+import javax.management.ObjectName;
+
+import com.google.common.base.Optional;
+
+import org.apache.cassandra.utils.progress.ProgressEvent;
+import org.apache.cassandra.utils.progress.ProgressListener;
+
+import static org.apache.cassandra.service.ActiveRepairService.Status;
+
+/**
+ * ProgressListener that translates ProgressEvent to legacy JMX Notification message (backward compatibility support)
+ */
+public class LegacyJMXProgressSupport implements ProgressListener
+{
+    protected static final Pattern SESSION_FAILED_MATCHER = Pattern.compile("Repair session .* for range .* failed with error .*");
+    protected static final Pattern SESSION_SUCCESS_MATCHER = Pattern.compile("Repair session .* for range .* finished");
+
+    private final AtomicLong notificationSerialNumber = new AtomicLong();
+    private final ObjectName jmxObjectName;
+
+    private final NotificationBroadcasterSupport broadcaster;
+
+    public LegacyJMXProgressSupport(NotificationBroadcasterSupport broadcaster,
+                                    ObjectName jmxObjectName)
+    {
+        this.broadcaster = broadcaster;
+        this.jmxObjectName = jmxObjectName;
+    }
+
+    @Override
+    public void progress(String tag, ProgressEvent event)
+    {
+        if (tag.startsWith("repair:"))
+        {
+            Optional<int[]> legacyUserData = getLegacyUserdata(tag, event);
+            if (legacyUserData.isPresent())
+            {
+                Notification jmxNotification = new Notification("repair", jmxObjectName, notificationSerialNumber.incrementAndGet(), event.getMessage());
+                jmxNotification.setUserData(legacyUserData.get());
+                broadcaster.sendNotification(jmxNotification);
+            }
+        }
+    }
+
+    protected static Optional<int[]> getLegacyUserdata(String tag, ProgressEvent event)
+    {
+        Optional<Status> status = getStatus(event);
+        if (status.isPresent())
+        {
+            int[] result = new int[2];
+            result[0] = getCmd(tag);
+            result[1] = status.get().ordinal();
+            return Optional.of(result);
+        }
+        return Optional.absent();
+    }
+
+    protected static Optional<Status> getStatus(ProgressEvent event)
+    {
+        switch (event.getType())
+        {
+            case START:
+                return Optional.of(Status.STARTED);
+            case COMPLETE:
+                return Optional.of(Status.FINISHED);
+            case PROGRESS:
+                if (SESSION_FAILED_MATCHER.matcher(event.getMessage()).matches())
+                {
+                    return Optional.of(Status.SESSION_FAILED);
+                }
+                else if (SESSION_SUCCESS_MATCHER.matcher(event.getMessage()).matches())
+                {
+                    return Optional.of(Status.SESSION_SUCCESS);
+                }
+        }
+
+        return Optional.absent();
+    }
+
+    protected static int getCmd(String tag)
+    {
+        return Integer.valueOf(tag.split(":")[1]);
+    }
+}
diff --git a/src/java/org/apache/cassandra/utils/vint/EncodedDataInputStream.java b/src/java/org/apache/cassandra/utils/vint/EncodedDataInputStream.java
index b35d180..bee8ab0 100644
--- a/src/java/org/apache/cassandra/utils/vint/EncodedDataInputStream.java
+++ b/src/java/org/apache/cassandra/utils/vint/EncodedDataInputStream.java
@@ -25,10 +25,10 @@
 /**
  * Borrows idea from
  * https://developers.google.com/protocol-buffers/docs/encoding#varints
- * 
+ *
  * Should be used with EncodedDataOutputStream
  */
-public class EncodedDataInputStream extends AbstractDataInput
+public class EncodedDataInputStream extends AbstractDataInput implements DataInput
 {
     private DataInput input;
 
@@ -47,12 +47,22 @@
         return input.readByte() & 0xFF;
     }
 
-    protected void seekInternal(int position)
+    public void seek(long position)
     {
         throw new UnsupportedOperationException();
     }
 
-    protected int getPosition()
+    public long getPosition()
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    public long getPositionLimit()
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    protected long length()
     {
         throw new UnsupportedOperationException();
     }
diff --git a/src/java/org/apache/cassandra/utils/vint/EncodedDataOutputStream.java b/src/java/org/apache/cassandra/utils/vint/EncodedDataOutputStream.java
index 92612b6..fe43ff2 100644
--- a/src/java/org/apache/cassandra/utils/vint/EncodedDataOutputStream.java
+++ b/src/java/org/apache/cassandra/utils/vint/EncodedDataOutputStream.java
@@ -20,13 +20,13 @@
 import java.io.IOException;
 import java.io.OutputStream;
 
-import org.apache.cassandra.io.util.AbstractDataOutput;
+import org.apache.cassandra.io.util.UnbufferedDataOutputStreamPlus;
 
 /**
  * Borrows idea from
  * https://developers.google.com/protocol-buffers/docs/encoding#varints
  */
-public class EncodedDataOutputStream extends AbstractDataOutput
+public class EncodedDataOutputStream extends UnbufferedDataOutputStreamPlus
 {
     private OutputStream out;
 
diff --git a/src/jdkoverride/java/util/zip/CRC32.class b/src/jdkoverride/java/util/zip/CRC32.class
new file mode 100644
index 0000000..546199e
--- /dev/null
+++ b/src/jdkoverride/java/util/zip/CRC32.class
Binary files differ
diff --git a/src/resources/org/apache/cassandra/cli/CliHelp.yaml b/src/resources/org/apache/cassandra/cli/CliHelp.yaml
deleted file mode 100644
index cc838f8..0000000
--- a/src/resources/org/apache/cassandra/cli/CliHelp.yaml
+++ /dev/null
@@ -1,1262 +0,0 @@
-
-# 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.
-
-# Help file for online commands in Yaml.
-
-banner: |
-    Type 'help;' or '?' for help.
-    Type 'quit;' or 'exit;' to quit.
-
-help: |
-    Getting around:
-    ?                       Display this help.
-    help;                   Display this help.
-    help <command>;         Display command-specific help.
-    exit;                   Exit this utility.
-    quit;                   Exit this utility.
-
-    Commands:
-    assume                  Apply client side validation.
-    connect                 Connect to a Cassandra node.
-    consistencylevel        Sets consisteny level for the client to use.
-    count                   Count columns or super columns.
-    create column family    Add a column family to an existing keyspace.
-    create keyspace         Add a keyspace to the cluster.
-    del                     Delete a column, super column or row.
-    decr                    Decrements a counter column.
-    describe cluster        Describe the cluster configuration.
-    describe                Describe a keyspace and its column families or column family in current keyspace.
-    drop column family      Remove a column family and its data.
-    drop keyspace           Remove a keyspace and its data.
-    drop index              Remove an existing index from specific column.
-    get                     Get rows and columns.
-    incr                    Increments a counter column.
-    list                    List rows in a column family.
-    set                     Set columns.
-    show api version        Show the server API version.
-    show cluster name       Show the cluster name.
-    show keyspaces          Show all keyspaces and their column families.
-    show schema             Show a cli script to create keyspaces and column families.
-    truncate                Drop the data in a column family.
-    update column family    Update the settings for a column family.
-    update keyspace         Update the settings for a keyspace.
-    use                     Switch to a keyspace.
-
-commands:
-    - name: NODE_HELP
-      help: |
-        help <command>;
-
-        Display the general help page with a list of available commands.;
-    - name: NODE_CONNECT
-      help: |
-        connect <hostname>/<port> (<username> '<password>')?;
-
-        Connect to the a Cassandra node on the specified port.
-
-        If a username and password are supplied the login will occur when the
-        'use' statement is executed. If the server does not support authentication
-        it will silently ignore credentials.
-
-        For information on configuring authentication and authorisation see the
-        conf/cassandra.yaml file or the project documentation.
-
-        Required Parameters:
-        - hostname: Machine name or IP address of the node to connect to.
-
-        - port: rpc_port to connect to the node on, as defined in
-        conf/Cassandra.yaml for the node. The default port is 9160.
-
-        Optional Parameters:
-        - password: Password for the supplied username.
-
-        - username: Username to authenticate to the node as.
-
-        Examples:
-        connect localhost/9160;
-        connect localhost/9160 user 'badpasswd';
-        connect 127.0.0.1/9160 user 'badpasswd';
-    - name: NODE_USE_TABLE
-      help: |
-        use <keyspace>;
-        use <keyspace> <username> '<password>';
-
-        Use the specified keyspace.
-
-        If a username and password are supplied they will be used to authorize
-        against the keyspace. Otherwise the credentials supplied to the 'connect'
-        statement will be used to authorize the user . If the server does not
-        support authentication it will silently ignore credentials.
-
-        Required Parameters:
-        - keyspace: Name of the keyspace to use. The keyspace must exist.
-
-        Optional Parameters:
-        - password: Password for the supplied username.
-
-        - username: Username to login to the node as.
-
-        Examples:
-        use Keyspace1;
-        use Keyspace1 user 'badpasswd';
-    - name: NODE_DESCRIBE
-      help: |
-        describe;
-        describe <keyspace>;
-        describe <column_family>;
-
-        Describes the settings for the current or named keyspace, or the settings
-        of the column family in the current authenticated keyspace.
-
-        Optional Parameters:
-        - keyspace: Name of the keyspace to describe.
-        - column_family: Name of the column family to describe.
-
-        Examples:
-        describe; - Describes current authenticated keyspace
-        describe <keyspace>; - Describe this keyspace
-        describe <column_family>; - Describe the colum family in the current authenticated keyspace
-    - name: NODE_DESCRIBE_CLUSTER
-      help: |
-        describe cluster;
-
-        Describes the snitch, partitioner and schema versions for the currently
-        connected cluster.
-
-        NOTE: The cluster should only report one schema version. Multiple versions
-        may indicate a failed schema modification, consult the project documentation.
-
-        Examples:
-        describe cluster;
-    - name: NODE_EXIT
-      help: |
-        exit;
-        quit;
-
-        Exit this utility.
-
-        Examples:
-        exit;
-        quit;
-    - name: NODE_SHOW_CLUSTER_NAME
-      help: |
-        show cluster name;
-
-        Displays the name of the currently connected cluster.
-
-        Examples:
-        show cluster name;
-    - name: NODE_SHOW_VERSION
-      help: |
-        show api version;
-
-        Displays the API version number.
-
-        This version number is used by high level clients and is not the same as
-        the server release version.
-
-        Examples:
-        show api version;
-    - name: NODE_SHOW_KEYSPACES
-      help: |
-        show keyspaces;
-
-        Describes the settings and the column families for all keyspaces on the
-        currently connected cluster.
-
-        Examples:
-        show keyspaces;
-    - name: NODE_SHOW_SCHEMA
-      help: |
-        show schema;
-        show schema <keyspace>;
-
-        Creates a CLI script to create the current, specified or all keyspaces
-        and their column families.
-
-        Optional Parameters:
-        - keyspace: Name of the keyspace to create the script for. If omitted
-        the current keyspace is used, if there is no current keyspace all
-        keyspaces are considered.
-
-        Examples:
-        show schema;
-        show schema Keyspace1;
-    - name: NODE_ADD_KEYSPACE
-      help: |
-        create keyspace <keyspace>;
-        create keyspace <keyspace> with <att1>=<value1>;
-        create keyspace <keyspace> with <att1>=<value1> and <att2>=<value2> ...;
-
-        Create a keyspace with the specified attributes.
-
-        Required Parameters:
-        - keyspace: Name of the new keyspace, "system" is reserved for
-          Cassandra internals. Names may only contain letters, numbers and
-          underscores.
-
-        Keyspace Attributes (all are optional):
-        - placement_strategy: Class used to determine how replicas
-          are distributed among nodes. Defaults to NetworkTopologyStrategy with
-          one datacenter defined with a replication factor of 1 ("[datacenter1:1]").
-
-          Supported values are:
-            - org.apache.cassandra.locator.SimpleStrategy
-            - org.apache.cassandra.locator.NetworkTopologyStrategy
-            - org.apache.cassandra.locator.OldNetworkTopologyStrategy
-
-          SimpleStrategy merely places the first replica at the node whose
-          token is closest to the key (as determined by the Partitioner), and
-          additional replicas on subsequent nodes along the ring in increasing
-          Token order.
-
-          Supports a single strategy option 'replication_factor' that
-          specifies the replication factor for the cluster.
-
-          With NetworkTopologyStrategy, for each datacenter, you can specify
-          how many replicas you want on a per-keyspace basis. Replicas are
-          placed on different racks within each DC, if possible.
-
-          Supports strategy options which specify the replication factor for
-          each datacenter. The replication factor for the entire cluster is the
-          sum of all per datacenter values. Note that the datacenter names
-          must match those used in conf/cassandra-topology.properties.
-
-          OldNetworkToplogyStrategy [formerly RackAwareStrategy]
-          places one replica in each of two datacenters, and the third on a
-          different rack in in the first.  Additional datacenters are not
-          guaranteed to get a replica.  Additional replicas after three are
-          placed in ring order after the third without regard to rack or
-          datacenter.
-
-          Supports a single strategy option 'replication_factor' that
-          specifies the replication factor for the cluster.
-
-        - strategy_options: Optional additional options for placement_strategy.
-          Options have the form {key:value}, see the information on each
-          strategy and the examples.
-
-        - durable_writes: When set to false all Mutations on keyspace will by-pass CommitLog.
-          Set to true by default.
-
-        Examples:
-        create keyspace Keyspace2
-            with placement_strategy = 'org.apache.cassandra.locator.SimpleStrategy'
-            and strategy_options = {replication_factor:4};
-        create keyspace Keyspace3
-            with placement_strategy = 'org.apache.cassandra.locator.NetworkTopologyStrategy'
-            and strategy_options={DC1:2, DC2:2};
-        create keyspace Keyspace4
-            with placement_strategy = 'org.apache.cassandra.locator.OldNetworkTopologyStrategy'
-            and strategy_options = {replication_factor:1};
-    - name: NODE_UPDATE_KEYSPACE
-      help: |
-        update keyspace <keyspace>;
-        update keyspace <keyspace> with <att1>=<value1>;
-        update keyspace <keyspace> with <att1>=<value1> and <att2>=<value2> ...;
-
-        Update a keyspace with the specified attributes.
-
-        Note: updating some keyspace properties may require additional maintenance
-        actions. Consult project documentation for more details.
-
-        Required Parameters:
-        - keyspace: Name of the keyspace to update.
-
-        Keyspace Attributes (all are optional):
-        - placement_strategy: Class used to determine how replicas
-          are distributed among nodes. Defaults to NetworkTopologyStrategy with
-          one datacenter defined with a replication factor of 1 ("[datacenter1:1]").
-
-          Supported values are:
-            - org.apache.cassandra.locator.SimpleStrategy
-            - org.apache.cassandra.locator.NetworkTopologyStrategy
-            - org.apache.cassandra.locator.OldNetworkTopologyStrategy
-
-          SimpleStrategy merely places the first replica at the node whose
-          token is closest to the key (as determined by the Partitioner), and
-          additional replicas on subsequent nodes along the ring in increasing
-          Token order.
-
-          Supports a single strategy option 'replication_factor' that
-          specifies the replication factor for the cluster.
-
-          With NetworkTopologyStrategy, for each datacenter, you can specify
-          how many replicas you want on a per-keyspace basis. Replicas are
-          placed on different racks within each DC, if possible.
-
-          Supports strategy options which specify the replication factor for
-          each datacenter. The replication factor for the entire cluster is the
-          sum of all per datacenter values. Note that the datacenter names
-          must match those used in conf/cassandra-topology.properties.
-
-          OldNetworkToplogyStrategy [formerly RackAwareStrategy]
-          places one replica in each of two datacenters, and the third on a
-          different rack in in the first.  Additional datacenters are not
-          guaranteed to get a replica.  Additional replicas after three are
-          placed in ring order after the third without regard to rack or
-          datacenter.
-
-          Supports a single strategy option 'replication_factor' that
-          specifies the replication factor for the cluster.
-
-        - strategy_options: Optional additional options for placement_strategy.
-          Options have the form {key:value}, see the information on each
-          strategy and the examples.
-
-        - durable_writes: When set to false all Mutations on keyspace will by-pass CommitLog.
-          Set to true by default.
-
-        Examples:
-        update keyspace Keyspace2
-            with placement_strategy = 'org.apache.cassandra.locator.SimpleStrategy'
-            and strategy_options = {replication_factor:4};
-        update keyspace Keyspace3
-            with placement_strategy = 'org.apache.cassandra.locator.NetworkTopologyStrategy'
-            and strategy_options={DC1:2, DC2:2};
-        update keyspace Keyspace4
-            with placement_strategy = 'org.apache.cassandra.locator.OldNetworkTopologyStrategy'
-            and strategy_options = {replication_factor:1};
-    - name: NODE_ADD_COLUMN_FAMILY
-      help: |
-        create column family <name>;
-        create column family <name> with <att1>=<value1>;
-        create column family <name> with <att1>=<value1> and <att2>=<value2>...;
-
-        Create a column family in the current keyspace with the specified
-        attributes.
-
-        Required Parameters:
-        - name: Name of the new column family. Names may only contain letters,
-          numbers and underscores.
-
-        column family Attributes (all are optional):
-        - column_metadata: Defines the validation and indexes for known columns in
-          this column family.
-
-          Columns not listed in the column_metadata section will use the
-          default_validator to validate their values.
-
-          Column Required parameters:
-            - name: Binds a validator (and optionally an indexer) to columns
-              with this name in any row of the enclosing column family.
-
-            - validator: Validator to use for values for this column.
-
-              Supported values are:
-                - AsciiType
-                - BooleanType
-                - BytesType
-                - CounterColumnType (distributed counter column)
-                - DateType
-                - DoubleType
-                - FloatType
-                - Int32Type
-                - IntegerType (a generic variable-length integer type)
-                - LexicalUUIDType
-                - LongType
-                - UTF8Type
-                - CompositeType (should be used with sub-types specified e.g. 'CompositeType(UTF8Type, Int32Type)'
-                  quotes are important (!) in this case)
-
-              It is also valid to specify the fully-qualified class name to a class
-              that extends org.apache.cassandra.db.marshal.AbstractType.
-
-          Column Optional parameters:
-            - index_name: Name for the index. Both an index name and
-              type must be specified.
-
-            - index_type: The type of index to be created.
-
-              Suported values are:
-                - KEYS: a ColumnFamily backed index
-                - CUSTOM: a user supplied index implementaion. You must supply a
-                  'class_name' field in the index_options with the full classname 
-                  of the implementation.
-            
-            - index_options: Optional additional options for index_type.
-              Options have the form {key:value}.
-                   
-        - bloom_filter_fp_chance: Desired false positive probability for
-          sstable row bloom filters.  Default is 0.000744.
-
-        - index_interval: controls the sampling of entries from the primrary
-          row index in terms of space versus time.  The larger the interval,
-          the smaller and less effective the sampling will be. All the sampled
-          entries must fit in memory.  Default value is 128.
-
-        - column_type: Type of columns this column family holds, valid values are
-          Standard and Super. Default is Standard.
-
-        - comment: Human readable column family description.
-
-        - comparator: Validator to use to validate and compare column names in
-          this column family. For Standard column families it applies to columns, for
-          Super column families applied to  super columns. Also see the subcomparator
-          attribute. Default is BytesType, which is a straight forward lexical
-          comparison of the bytes in each column.
-
-          Supported values are:
-            - AsciiType
-            - BooleanType
-            - BytesType
-            - CounterColumnType (distributed counter column)
-            - DateType
-            - DoubleType
-            - FloatType
-            - Int32Type
-            - IntegerType (a generic variable-length integer type)
-            - LexicalUUIDType
-            - LongType
-            - UTF8Type
-            - CompositeType (should be used with sub-types specified e.g. 'CompositeType(UTF8Type, Int32Type)'
-              quotes are important (!) in this case)
-
-          It is also valid to specify the fully-qualified class name to a class that
-          extends org.apache.cassandra.db.marshal.AbstractType.
-
-        - default_validation_class: Validator to use for values in columns which are
-          not listed in the column_metadata. Default is BytesType which applies
-          no validation.
-
-          Supported values are:
-            - AsciiType
-            - BooleanType
-            - BytesType
-            - CounterColumnType (distributed counter column)
-            - DateType
-            - DoubleType
-            - FloatType
-            - Int32Type
-            - IntegerType (a generic variable-length integer type)
-            - LexicalUUIDType
-            - LongType
-            - UTF8Type
-            - CompositeType (should be used with sub-types specified e.g. 'CompositeType(UTF8Type, Int32Type)'
-              quotes are important (!) in this case)
-
-          It is also valid to specify the fully-qualified class name to a class that
-          extends org.apache.cassandra.db.marshal.AbstractType.
-
-        - key_validation_class: Validator to use for keys.
-          Default is BytesType which applies no validation.
-
-          Supported values are:
-            - AsciiType
-            - BooleanType
-            - BytesType
-            - DateType
-            - DoubleType
-            - FloatType
-            - Int32Type
-            - IntegerType (a generic variable-length integer type)
-            - LexicalUUIDType
-            - LongType
-            - UTF8Type
-
-          It is also valid to specify the fully-qualified class name to a class that
-          extends org.apache.cassandra.db.marshal.AbstractType.
-
-        - gc_grace: Time to wait in seconds before garbage collecting tombstone
-          deletion markers. Default value is 864000 or 10 days.
-
-          Set this to a large enough value that you are confident that the deletion
-          markers will be propagated to all replicas by the time this many seconds
-          has elapsed, even in the face of hardware failures.
-
-          See http://wiki.apache.org/Cassandra/DistributedDeletes
-
-        - read_repair_chance: Probability (0.0-1.0) with which to perform read
-          repairs for any read operation. Default is 0.1.
-
-          Note that disabling read repair entirely means that the dynamic snitch
-          will not have any latency information from all the replicas to recognize
-          when one is performing worse than usual.
-
-        - dclocal_read_repair_chance: Probability (0.0-1.0) with which to
-          perform read repairs against the node from the local data-center. If
-          this is lower than read_repair_chance, this will be ignored.
-
-          Example:
-            update column family Standard2
-                 with read_repair_chance=0.1
-                 and dclocal_read_repair_chance=0.5;
-
-            For 10 read queries, 1 will do read repair on all replicas (and
-            thus in particular on all replica of the local DC), 4 will only do
-            read repair on replica of the local DC and 5 will not do any read
-            repair.
-
-        - subcomparator:  Validator to use to validate and compare sub column names
-          in this column family. Only applied to Super column families. Default is
-          BytesType, which is a straight forward lexical comparison of the bytes in
-          each column.
-
-          Supported values are:
-            - AsciiType
-            - BooleanType
-            - BytesType
-            - CounterColumnType (distributed counter column)
-            - DateType
-            - DoubleType
-            - FloatType
-            - Int32Type
-            - IntegerType (a generic variable-length integer type)
-            - LexicalUUIDType
-            - LongType
-            - UTF8Type
-
-          It is also valid to specify the fully-qualified class name to a class that
-          extends org.apache.cassandra.db.marshal.AbstractType.
-
-        - max_compaction_threshold: The maximum number of SSTables allowed before a
-        minor compaction is forced. Default is 32, setting to 0 disables minor
-        compactions.
-
-        Decreasing this will cause minor compactions to start more frequently and
-        be less intensive. The min_compaction_threshold and max_compaction_threshold
-        boundaries are the number of tables Cassandra attempts to merge together at
-        once.
-
-        - min_compaction_threshold: The minimum number of SSTables needed
-        to start a minor compaction. Default is 4, setting to 0 disables minor
-        compactions.
-
-        Increasing this will cause minor compactions to start less frequently and
-        be more intensive. The min_compaction_threshold and max_compaction_threshold
-        boundaries are the number of tables Cassandra attempts to merge together at
-        once.
-
-        - compression_options: Options related to compression.
-          Options have the form {key:value}.
-          The main recognized options are:
-            - sstable_compression: the algorithm to use to compress sstables for
-              this column family. If none is provided, compression will not be
-              enabled. Supported values are SnappyCompressor, DeflateCompressor or
-              any custom compressor. It is also valid to specify the fully-qualified
-              class name to a class that implements org.apache.cassandra.io.ICompressor.
-
-            - chunk_length_kb: specify the size of the chunk used by sstable
-              compression (default to 64, must be a power of 2).
-
-          To disable compression just set compression_options to null like this
-          `compression_options = null`.
-
-        Examples:
-        create column family Super4
-            with column_type = 'Super'
-            and comparator = 'AsciiType'
-            and caching='ALL';
-        create column family Standard3
-            with comparator = 'LongType'
-            and caching='ALL';
-        create column family Standard4
-            with comparator = AsciiType
-            and column_metadata =
-            [{
-                column_name : Test,
-                validation_class : IntegerType,
-                index_type : 0,
-                index_name : IdxName},
-            {
-                column_name : 'other name',
-                validation_class : LongType
-            }];
-        create column family UseComposites
-           with comparator = 'CompositeType(UTF8Type, Int32Type)'
-           and caching='ALL';
-    - name: NODE_UPDATE_COLUMN_FAMILY
-      help: |
-        update column family <name>;
-        update column family <name> with <att1>=<value1>;
-        update column family <name> with <att1>=<value1> and <att2>=<value2>...;
-
-        Updates the settings for a column family in the current keyspace.
-
-        Required Parameters:
-        - name: Name of the column family to update.
-
-        column family Attributes (all are optional):
-        - column_metadata: Defines the validation and indexes for known columns in
-          this column family.
-
-          Columns not listed in the column_metadata section will use the
-          default_validator to validate their values.
-
-          Column Required parameters:
-            - name: Binds a validator (and optionally an indexer) to columns
-              with this name in any row of the enclosing column family.
-
-            - validator: Validator to use for values for this column.
-
-              Supported values are:
-                - AsciiType
-                - BooleanType
-                - BytesType
-                - CounterColumnType (distributed counter column)
-                - DateType
-                - DoubleType
-                - FloatType
-                - Int32Type
-                - IntegerType (a generic variable-length integer type)
-                - LexicalUUIDType
-                - LongType
-                - UTF8Type
-                - CompositeType (should be used with sub-types specified e.g. 'CompositeType(UTF8Type, Int32Type)'
-                  quotes are important (!) in this case)
-
-              It is also valid to specify the fully-qualified class name to a class
-              that extends org.apache.cassandra.db.marshal.AbstractType.
-
-          Column Optional parameters:
-            - index_name: Name for the index. Both an index name and
-              type must be specified.
-
-              NOTE: After the update has completed the column family will only
-              contain the secondary indexes listed in the update statement. Existing
-              indexes will be dropped if they are not present in the update.
-
-            - index_type: The type of index to be created.
-
-              Suported values are:
-                - KEYS: a ColumnFamily backed index
-                - CUSTOM: a user supplied index implementaion. You must supply a
-                  'class_name' field in the index_options with the full classname 
-                  of the implementation.
-            
-            - index_options: Optional additional options for index_type.
-              Options have the form {key:value}.
-
-        - bloom_filter_fp_chance: Desired false positive probability for
-          sstable row bloom filters.  Default is 0.000744.
-
-        - column_type: Type of columns this column family holds, valid values are
-          Standard and Super. Default is Standard.
-
-        - comment: Column family description.
-
-        - comparator: Validator to use to validate and compare column names in
-          this column family. For Standard column families it applies to columns, for
-          Super column families applied to  super columns. Also see the subcomparator
-          attribute. Default is BytesType, which is a straight forward lexical
-          comparison of the bytes in each column.
-
-          Supported values are:
-            - AsciiType
-            - BooleanType
-            - BytesType
-            - CounterColumnType (distributed counter column)
-            - DateType
-            - DoubleType
-            - FloatType
-            - Int32Type
-            - IntegerType (a generic variable-length integer type)
-            - LexicalUUIDType
-            - LongType
-            - UTF8Type
-            - CompositeType (should be used with sub-types specified e.g. 'CompositeType(UTF8Type, Int32Type)'
-              quotes are important (!) in this case)
-
-          It is also valid to specify the fully-qualified class name to a class that
-          extends org.apache.cassandra.db.marshal.AbstractType.
-
-        - default_validation_class: Validator to use for values in columns which are
-          not listed in the column_metadata. Default is BytesType which applies
-          no validation.
-
-          Supported values are:
-            - AsciiType
-            - BooleanType
-            - BytesType
-            - CounterColumnType (distributed counter column)
-            - DateType
-            - DoubleType
-            - FloatType
-            - Int32Type
-            - IntegerType (a generic variable-length integer type)
-            - LexicalUUIDType
-            - LongType
-            - UTF8Type
-            - CompositeType (should be used with sub-types specified e.g. 'CompositeType(UTF8Type, Int32Type)'
-              quotes are important (!) in this case)
-
-          It is also valid to specify the fully-qualified class name to a class that
-          extends org.apache.cassandra.db.marshal.AbstractType.
-
-        - key_validation_class: Validator to use for keys.
-          Default is BytesType which applies no validation.
-
-          Supported values are:
-            - AsciiType
-            - BooleanType
-            - BytesType
-            - DateType
-            - DoubleType
-            - FloatType
-            - Int32Type
-            - IntegerType (a generic variable-length integer type)
-            - LexicalUUIDType
-            - LongType
-            - UTF8Type
-
-          It is also valid to specify the fully-qualified class name to a class that
-          extends org.apache.cassandra.db.marshal.AbstractType.
-
-        - gc_grace: Time to wait in seconds before garbage collecting tombstone
-          deletion markers. Default value is 864000 or 10 days.
-
-          Set this to a large enough value that you are confident that the deletion
-          markers will be propagated to all replicas by the time this many seconds
-          has elapsed, even in the face of hardware failures.
-
-          See http://wiki.apache.org/Cassandra/DistributedDeletes
-
-        - read_repair_chance: Probability (0.0-1.0) with which to perform read
-          repairs for any read operation. Default is 0.1.
-
-          Note that disabling read repair entirely means that the dynamic snitch
-          will not have any latency information from all the replicas to recognize
-          when one is performing worse than usual.
-
-        - dclocal_read_repair_chance: Probability (0.0-1.0) with which to
-          perform read repairs against the node from the local data-center. If
-          this is lower than read_repair_chance, this will be ignored.
-
-          Example:
-            update column family Standard2
-                 with read_repair_chance=0.1
-                 and dclocal_read_repair_chance=0.5;
-
-            For 10 read queries, 1 will do read repair on all replicas (and
-            thus in particular on all replica of the local DC), 4 will only do
-            read repair on replica of the local DC and 5 will not do any read
-            repair.
-
-        - subcomparator:  Validator to use to validate and compare sub column names
-          in this column family. Only applied to Super column families. Default is
-          BytesType, which is a straight forward lexical comparison of the bytes in
-          each column.
-
-          Supported values are:
-            - AsciiType
-            - BooleanType
-            - BytesType
-            - CounterColumnType (distributed counter column)
-            - DateType
-            - DoubleType
-            - FloatType
-            - Int32Type
-            - IntegerType (a generic variable-length integer type)
-            - LexicalUUIDType
-            - LongType
-            - UTF8Type
-
-          It is also valid to specify the fully-qualified class name to a class that
-          extends org.apache.cassandra.db.marshal.AbstractType.
-
-        - caching: Enable/Disable caching for the column family.
-
-          There is 2 kind of caches: key cache and row cache. A column family
-          can be set to use no cache, only one of them or both.
-
-          A key cache hit saves 1 seek while a row cache hit avoids disk
-          completely (saving at least 2 seeks). However the row cache holds
-          entire rows and is thus much more space-intensive. Also, since full
-          rows are stored, row cache should be used only for column families
-          whose rows are small.
-
-          Note: the global memory size of both cache can be set in the
-          configuration file (yaml) or through JMX (though it isn't persisted
-          across restarts in that latter case).
-
-          Supported values are:
-            - ALL (Enable row cache and key Cache)
-            - KEYS_ONLY
-            - ROWS_ONLY
-            - NONE;
-        - cells_per_row_to_cache: State the number of cells per row to cache.
-
-          Defaults to 100. Set to "ALL" if you want the old cache behaviour.
-
-          Will not be used if row caching is not enabled.
-        - speculative_retry: Speculative retry is used to speculate a read failure.
-
-          Speculative retry will execute additional read on a different nodes when
-          the read request doesn't complete within the x milliseconds.
-
-          Xpercentile will execute additional read requests to a different replicas
-          when read request was not completed within X percentile of the
-          normal/earlier recorded latencies.
-
-          Xms will execute additional read request to a diffrent replica when read
-          request was not completed in X milliseconds.
-
-          ALWAYS will execute data read request to 2 (If available) of the replicas
-          expecting a node to fail read.
-
-          Supported values are:
-            - ALWAYS
-            - Xpercentile
-            - Xms
-            - NONE;
-
-        - max_compaction_threshold: The maximum number of SSTables allowed before a
-        minor compaction is forced. Default is 32, setting to 0 disables minor
-        compactions.
-
-        Decreasing this will cause minor compactions to start more frequently and
-        be less intensive. The min_compaction_threshold and max_compaction_threshold
-        boundaries are the number of tables Cassandra attempts to merge together at
-        once.
-
-        - min_compaction_threshold: The minimum number of SSTables needed
-        to start a minor compaction. Default is 4, setting to 0 disables minor
-        compactions.
-
-        Increasing this will cause minor compactions to start less frequently and
-        be more intensive. The min_compaction_threshold and max_compaction_threshold
-        boundaries are the number of tables Cassandra attempts to merge together at
-        once.
-
-        - compression_options: Options related to compression.
-          Options have the form {key:value}.
-          The main recognized options are:
-            - sstable_compression: the algorithm to use to compress sstables for
-              this column family. If none is provided, compression will not be
-              enabled. Supported values are SnappyCompressor, DeflateCompressor or
-              any custom compressor. It is also valid to specify the fully-qualified
-              class name to a class that implements org.apache.cassandra.io.ICompressor.
-
-            - chunk_length_kb: specify the size of the chunk used by sstable
-              compression (default to 64, must be a power of 2).
-
-          To disable compression just set compression_options to null like this
-          `compression_options = null`.
-
-        Examples:
-        update column family Super4
-            with column_type = 'Super'
-            and caching='ALL';
-        update column family Standard3
-            and caching='ALL';
-        update column family Standard4
-            with column_metadata =
-            [{
-                column_name : Test,
-                validation_class : IntegerType,
-                index_type : 0,
-                index_name : IdxName},
-            {
-                column_name : 'other name',
-                validation_class : LongType
-            }];
-    - name: NODE_DEL_KEYSPACE
-      help: |
-        drop keyspace <keyspace>;
-
-        Drops the specified keyspace.
-
-        A snapshot of the data is created in a sub directory of the Keyspace data directory. The files
-        must be manually deleted using either "nodetool clearsnapshot" or the command line.
-
-        Required Parameters:
-        - keyspace: Name of the keyspace to delete.
-
-        Example:
-        drop keyspace Keyspace1;
-    - name: NODE_DEL_COLUMN_FAMILY
-      help: |
-        drop column family <cf>;
-
-        Drops the specified column family.
-
-        A snapshot of the data is created in a sub directory of the Keyspace data directory. The files
-        must be manually deleted using either "nodetool clearsnapshot" or the command line.
-
-        Required Parameters:
-        - cf: Name of the column family to delete.
-
-        Example:
-        drop column family Standard2;
-    - name: NODE_DROP_INDEX
-      help: |
-        drop index on <cf>.<column>;
-
-        Drops index on specified column of the column family.
-
-        Required Parameters:
-          - cf: Name of the column family.
-          - column: Name of the column to delete index on.
-
-        Example:
-        drop index on Users.name;
-    - name: NODE_THRIFT_GET
-      help: |
-        get <cf>['<key>'];
-        get <cf>['<key>']['<col>'] (as <type>)*;
-        get <cf>['<key>']['<super>']['<col>'] (as <type>)*;
-        get <cf>['<key>']['<super>'];
-        get <cf>['<key>'][<function>];
-        get <cf>[function(<key>)][<function>(<super>)][<function>(<col>)];
-        get <cf> where <col> <operator> <value> [
-            and <col> <operator> <value> and ...] [limit <limit>];
-        get <cf> where <col> <operator> <function>(<value>) [
-            and <col> <operator> <function> and ...] [limit <limit>];
-
-        Gets columns or super columns for the specified column family and key. Or
-        returns all columns from rows which meet the specified criteria when using
-        the 'where' form.
-
-        Note: The implementation of secondary indexes in Cassandra 0.7 has some
-        restrictions, see
-        http://www.datastax.com/dev/blog/whats-new-Cassandra-07-secondary-indexes
-
-        Required Parameters:
-        - cf: Name of the column family to read from.
-
-        Optional Parameters:
-        - col: Name of the column to read. Or in the 'where' form name of the column
-        to test the value of.
-
-        - function: Name of a function to call to parse the supplied argument to the
-          specified type. Some functions will generate values without needing an
-          argument.
-
-          Valid options are:
-            - ascii
-            - bytes: if used without arguments generates a zero length byte array
-            - int
-            - integer
-            - lexicaluuid: if used without arguments generates a new random uuid
-            - long
-            - timeuuid: if used without arguments generates a new time uuid
-            - utf8
-
-        - key: Key for the row to read columns from. This parameter is
-          required in all cases except when the 'where' clause is used.
-
-        - limit: Number of rows to return. Default is 100.
-
-        - operator: Operator to test the column value with. Supported operators are
-          =, >, >=, <, <= .
-
-          In Cassandra 0.7 at least one = operator must be present.
-
-        - super: Name of the super column to read from. If super is supplied without
-          col then all columns from the super column are returned.
-
-        - type: Data type to interpret the the columns value as for display.
-
-          Valid options are:
-            - AsciiType
-            - BooleanType
-            - BytesType
-            - CounterColumnType (distributed counter column)
-            - DateType
-            - DoubleType
-            - FloatType
-            - Int32Type
-            - IntegerType (a generic variable-length integer type)
-            - LexicalUUIDType
-            - LongType
-            - UTF8Type
-
-        - value: The value to test the column for, if a function is provided the
-        value is parsed by the function. Otherwise the meta data for the target
-        column is used to determine the correct type.
-
-        Examples:
-        get Standard1[ascii('testkey')];
-        #tell cli to convert keys from ascii to bytes
-        assume Standard1 keys as ascii;
-        get Standard1[testkey][test_column] as IntegerType;
-        get Standard1[testkey][utf8(hello)];
-        get Indexed1 where birthdate=19750403;
-    - name: NODE_THRIFT_SET
-      help: |
-        set <cf>['<key>']['<col>'] = <value>;
-        set <cf>['<key>']['<super>']['<col>'] = <value>;
-        set <cf>['<key>']['<col>'] = <function>(<argument>);
-        set <cf>['<key>']['<super>']['<col>'] = <function>(<argument>);
-        set <cf>[<key>][<function>(<col>)] = <value> || <function>;
-        set <cf>[<function>(<key>)][<function>(<col>) || <col>] =
-            <value> || <function> with ttl = <secs>;
-
-        Sets the column value for the specified column family and key.
-
-        Required Parameters:
-        - cf: Name of the column family to set columns in.
-
-        - col: Name of the column to set.
-
-        - key: Key for the row to set columns in.
-
-        Optional Parameters:
-        - function: Name of a function to call to parse the supplied argument to the
-          specified type. Some functions will generate values without needing an
-          argument.
-
-          Valid options are:
-            - ascii
-            - bytes: if used without arguments generates a zero length byte array
-            - int
-            - integer
-            - lexicaluuid: if used without arguments generates a new random uuid
-            - long
-            - timeuuid: if used without arguments generates a new time uuid
-            - utf8
-
-        - secs: Time To Live for the column in seconds. Defaults to no ttl.
-
-        - super: Name of the super column to contain the column.
-
-        - value: The value to set the column to.
-
-        Examples:
-        set Super1[ascii('testkey')][ascii('my super')][ascii('test col')]='this is a test';
-        set Standard1['testkey']['test col']='this is also a test';
-        set Standard1[testkey][testcol] = utf8('this is utf8 string.');
-        set Standard1[testkey][timeuuid()] = utf8('hello world');
-        set Standard1[testkey][timeuuid()] = utf8('hello world') with ttl = 30;
-        set UseComposites[utf8('testkey')]['CompositeType(utf8(first),int(4))'] = utf8('inserts this string into a column with name first:4');
-    - name: NODE_THRIFT_DEL
-      help: |
-        del <cf>['<key>'];
-        del <cf>['<key>']['<col>'];
-        del <cf>['<key>']['<super>'];
-        del <cf>['<key>']['<super>']['<col>'];
-        del <cf>[<function>(<key>)][<function>(<super>)][<function>(<col>)];
-
-        Deletes a row, a column, or a subcolumn.
-
-        Required Parameters:
-        - cf: Name of the column family to delete from.
-
-        - key: Key for the row delete from.
-
-        Optional Parameters:
-        - col: Name of the column to delete.
-
-        - function: Name of a function to call to parse the supplied argument to the
-          specified type. Some functions will generate values without needing an
-          argument.
-
-           Supported values are:
-            - ascii
-            - bytes: if used without arguments generates a zero length byte array
-            - int
-            - integer
-            - lexicaluuid: if used without arguments generates a new random uuid
-            - long
-            - double
-            - timeuuid: if used without arguments generates a new time uuid
-            - utf8
-
-        - super: Name of the super column to delete from. If col is not specified
-        the super column and all sub columns will be deleted.
-
-        Examples:
-        del Super1[ascii('testkey')][ascii('my_super')][ascii('test_col')];
-        del Standard1['testkey'][ascii('test col')];
-        del Standard1['testkey'];
-        del Standard1[utf8('testkey')];
-    - name: NODE_THRIFT_COUNT
-      help: |
-        count <cf>['<key>'];
-        count <cf>['<key>']['<super>'];
-
-        Count the number of columns in the row with the specified key, or
-        subcolumns in the specified super column.
-
-        Required Parameters:
-        - cf: Name of the column family to read from..
-
-        - key: Key for the row to count.
-
-        Optional Parameters:
-        - super: Name of the super column to count subcolumns in.
-
-        Examples:
-        count Super1['testkey']['my super'];
-        count Standard1['testkey'];
-    - name: NODE_LIST
-      help: |
-        list <cf>;
-        list <cf>[<startKey>:];
-        list <cf>[<startKey>:<endKey>];
-        list <cf>[<startKey>:<endKey>] limit <limit>;
-        list <cf>[<startKey>:<endKey>] ?(limit <limit>) ?(columns <col_limit> ?(reversed));
-
-        List a range of rows, and all of their columns, in the specified column
-        family.
-
-        The order of rows returned is dependant on the Partitioner in use.
-
-        Required Parameters:
-        - cf: Name of the column family to list rows from.
-
-        Optional Parameters:
-        - endKey: Key to end the range at. The end key will be included
-        in the result. Defaults to an empty byte array.
-
-        - limit: Number of rows to return. Default is 100.
-
-        - columns: Number of columns to return per row, Default to MAX_INTEGER. reversed -
-        columns in reverse order.
-
-        - startKey: Key start the range from. The start key will be
-        included in the result. Defaults to an empty byte array.
-
-        Examples:
-        list Standard1;
-        list Super1[j:];
-        list Standard1[j:k] limit 40;
-        list Standard1 columns 2;
-        list Standard1 limit 40 columns 0;
-        list Standard1 columns 3 reversed;
-    - name: NODE_TRUNCATE
-      help: |
-        truncate <cf>;
-
-        Truncate specified column family.
-
-        Note: All nodes in the cluster must be up to truncate command to execute.
-
-        A snapshot of the data is created, which is deleted asyncronously during a
-        'graveyard' compaction.
-
-        Required Parameters:
-        - cf: Name of the column family to truncate.
-
-        Examples:
-        truncate Standard1;
-    - name: NODE_ASSUME
-      help: |
-        assume <cf> comparator as <type>;
-        assume <cf> sub_comparator as <type>;
-        assume <cf> validator as <type>;
-        assume <cf> keys as <type>;
-
-        Assume one of the attributes (comparator, sub_comparator, validator or keys)
-        of the given column family match specified type. The specified type will
-        be used when displaying data returned from the column family.
-
-        This statement does not change the column family definition stored in
-        Cassandra. It only affects the cli and how it will transform values
-        to be sent to and interprets results from Cassandra.
-
-        If results from Cassandra do not validate according to the assumptions an
-        error is displayed in the cli.
-
-        Required Parameters:
-        - cf: Name of the column family to make the assumption about.
-
-        - type: Validator type to use when processing values.
-
-          Supported values are:
-            - ascii
-            - bytes
-            - counterColumn (distributed counter column)
-            - int
-            - integer (a generic variable-length integer type)
-            - lexicalUUID
-            - long
-            - utf8
-
-          It is also valid to specify the fully-qualified class name to a class that
-          extends org.apache.cassandra.db.marshal.AbstractType.
-
-        Examples:
-        assume Standard1 comparator as lexicaluuid;
-        assume Standard1 keys as ascii;
-    - name: NODE_THRIFT_INCR
-      help: |
-        incr <cf>['<key>']['<col>'] [by <value>];
-        incr <cf>['<key>']['<super>']['<col>'] [by <value>];
-
-        Increment the specified counter column by the supplied value.
-
-        Note: Counter columns must be defined using a 'create column family' or
-        'update column family' statement in the column_metadata as using the
-        ColumnCounterType validator.
-
-        Required Parameters:
-        - cf: Name of the column family to increment the column in.
-
-        - col: Name of the counter column to increment.
-
-        - key: Key for the row to increment the counter in.
-
-        Optional Parameters:
-        - super: Name of the super column that contains the counter column.
-
-        - value: Signed integer value to increment the column by. If not supplied
-        1 is used.
-
-        Examples:
-        incr Counter1[ascii('testkey')][ascii('test col')];
-        incr SuperCounter1[ascii('testkey')][ascii('my super')][ascii('test col')] by 42;
-        incr Counter1[ascii('testkey')][ascii('test col')] by -4;
-    - name: NODE_THRIFT_DECR
-      help: |
-        decr <cf>['<key>']['<col>'] [by <value>];
-        decr <cf>['<key>']['<super>']['<col>'] [by <value>];
-
-        Decrement the specified column by the supplied value.
-
-        Note: Counter columns must be defined using a 'create column family' or
-        'update column family' statement in the column_metadata as using the
-        ColumnCounterType validator.
-
-        Required Parameters:
-        - cf: Name of the column family to decrement the column in.
-
-        - col: Name of the counter column to increment.
-
-        - key: Key for the row to decrement the counter in.
-
-        Optional Parameters:
-        - super: Name of the super column that contains the counter column.
-
-        - value: Signed integer value to decrement the column by. If not supplied
-        1 is used.
-
-        Examples:
-        decr Counter1[ascii('testkey')][ascii('test col')];
-        decr SuperCounter1[ascii('testkey')][ascii('my super')][ascii('test col')] by 42;
-        decr Counter1[ascii('testkey')][ascii('test col')] by 10;
-    - name: NODE_CONSISTENCY_LEVEL
-      help: |
-        consistencylevel as <level>
-
-        Sets the consistency level for the client to use. Defaults to One.
-
-        Required Parameters:
-        - level: Consistency level the client should use. Value is case
-          insensitive.
-
-          Supported values are:
-            - ONE
-            - TWO
-            - THREE
-            - QUORUM
-            - ALL
-            - LOCAL_QUORUM
-            - EACH_QUORUM
-            - ANY
-
-          Note: Consistency level ANY can only be used for write operations.
diff --git a/src/resources/org/apache/cassandra/cql3/functions/JavaSourceUDF.txt b/src/resources/org/apache/cassandra/cql3/functions/JavaSourceUDF.txt
new file mode 100644
index 0000000..f57b01e
--- /dev/null
+++ b/src/resources/org/apache/cassandra/cql3/functions/JavaSourceUDF.txt
@@ -0,0 +1,43 @@
+package org.apache.cassandra.cql3.udf.gen;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import com.datastax.driver.core.DataType;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.functions.FunctionName;
+import org.apache.cassandra.cql3.functions.JavaSourceUDFFactory;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.exceptions.FunctionExecutionException;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+
+public final class #class_name# extends org.apache.cassandra.cql3.functions.UDFunction
+{
+    public #class_name#(FunctionName name, List<ColumnIdentifier> argNames, List<AbstractType<?>> argTypes,
+                        DataType[] argDataTypes, AbstractType<?> returnType, DataType returnDataType, boolean calledOnNullInput, String body)
+    {
+        super(name, argNames, argTypes, argDataTypes, returnType, returnDataType, calledOnNullInput, "java", body);
+    }
+
+    protected ByteBuffer executeUserDefined(int protocolVersion, List<ByteBuffer> params) throws InvalidRequestException
+    {
+        try
+        {
+            #return_type# result = executeInternal(
+#arguments#
+            );
+            return decompose(protocolVersion, result);
+        }
+        catch (Throwable t)
+        {
+            logger.debug("Invocation of function '{}' failed", this, t);
+            if (t instanceof VirtualMachineError)
+                throw (VirtualMachineError)t;
+            throw FunctionExecutionException.create(this, t);
+        }
+    }
+
+    private #return_type# executeInternal(#argument_list#)
+    {
+#body#
+    }
+}
diff --git a/test/burn/org/apache/cassandra/concurrent/LongSharedExecutorPoolTest.java b/test/burn/org/apache/cassandra/concurrent/LongSharedExecutorPoolTest.java
index 3453261..fe464c7 100644
--- a/test/burn/org/apache/cassandra/concurrent/LongSharedExecutorPoolTest.java
+++ b/test/burn/org/apache/cassandra/concurrent/LongSharedExecutorPoolTest.java
@@ -116,7 +116,7 @@
         final ExecutorService[] executors = new ExecutorService[executorCount];
         for (int i = 0 ; i < executors.length ; i++)
         {
-            executors[i] = JMXEnabledSharedExecutorPool.SHARED.newExecutor(threadCount, maxQueued, "test" + i, "test" + i);
+            executors[i] = SharedExecutorPool.SHARED.newExecutor(threadCount, maxQueued, "test" + i, "test" + i);
             threadCounts[i] = threadCount;
             workCount[i] = new WeibullDistribution(2, maxQueued);
             threadCount *= 2;
diff --git a/test/burn/org/apache/cassandra/io/sstable/LongSegmentedFileBoundaryTest.java b/test/burn/org/apache/cassandra/io/sstable/LongSegmentedFileBoundaryTest.java
index e17c6a7..4913b32 100644
--- a/test/burn/org/apache/cassandra/io/sstable/LongSegmentedFileBoundaryTest.java
+++ b/test/burn/org/apache/cassandra/io/sstable/LongSegmentedFileBoundaryTest.java
@@ -36,10 +36,12 @@
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.dht.ByteOrderedPartitioner;
 import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.util.DataOutputStreamPlus;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.io.util.MmappedSegmentedFile;
 import org.apache.cassandra.io.util.MmappedSegmentedFile.Builder.Boundaries;
+import org.apache.cassandra.io.util.WrappedDataOutputStreamPlus;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
@@ -296,7 +298,7 @@
         assertBoundaries(descriptor.filenameFor(Component.PRIMARY_INDEX), true, ibuilder.boundaries());
         assertBoundaries(descriptor.filenameFor(Component.DATA), expectDataMmappable, dbuilder.boundaries());
 
-        DataOutputStreamPlus oStream = new DataOutputStreamPlus(new FileOutputStream(summaryFile));
+        DataOutputStreamPlus oStream = new WrappedDataOutputStreamPlus(new FileOutputStream(summaryFile));
         IndexSummary.serializer.serialize(indexSummary, oStream, true);
         ByteBufferUtil.writeWithLength(first, oStream);
         ByteBufferUtil.writeWithLength(last, oStream);
diff --git a/test/burn/org/apache/cassandra/utils/LongBTreeTest.java b/test/burn/org/apache/cassandra/utils/LongBTreeTest.java
index 2227dd3..9641930 100644
--- a/test/burn/org/apache/cassandra/utils/LongBTreeTest.java
+++ b/test/burn/org/apache/cassandra/utils/LongBTreeTest.java
@@ -52,6 +52,7 @@
 import com.codahale.metrics.Timer;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
 import org.apache.cassandra.utils.btree.BTree;
+import org.apache.cassandra.utils.btree.BTreeSearchIterator;
 import org.apache.cassandra.utils.btree.BTreeSet;
 import org.apache.cassandra.utils.btree.UpdateFunction;
 
@@ -121,6 +122,52 @@
         testInsertions(10000, 50, 10, 10, false);
     }
 
+    @Test
+    public void testSearchIterator() throws InterruptedException
+    {
+        int threads = Runtime.getRuntime().availableProcessors();
+        final CountDownLatch latch = new CountDownLatch(threads);
+        final AtomicLong errors = new AtomicLong();
+        final AtomicLong count = new AtomicLong();
+        final int perThreadTrees = 100;
+        final int perTreeSelections = 100;
+        final long totalCount = threads * perThreadTrees * perTreeSelections;
+        for (int t = 0 ; t < threads ; t++)
+        {
+            MODIFY.execute(new Runnable()
+            {
+                public void run()
+                {
+                    ThreadLocalRandom random = ThreadLocalRandom.current();
+                    for (int i = 0 ; i < perThreadTrees ; i++)
+                    {
+                        Object[] tree = randomTree(10000, random);
+                        for (int j = 0 ; j < perTreeSelections ; j++)
+                        {
+                            BTreeSearchIterator<Integer, Integer, Integer> searchIterator = new BTreeSearchIterator<>(tree, ICMP);
+                            for (Integer key : randomSelection(tree, random))
+                                if (key != searchIterator.next(key))
+                                    errors.incrementAndGet();
+                            searchIterator = new BTreeSearchIterator<Integer, Integer, Integer>(tree, ICMP);
+                            for (Integer key : randomMix(tree, random))
+                                if (key != searchIterator.next(key))
+                                    if (BTree.find(tree, ICMP, key) == key)
+                                        errors.incrementAndGet();
+                            count.incrementAndGet();
+                        }
+                    }
+                    latch.countDown();
+                }
+            });
+        }
+        while (latch.getCount() > 0)
+        {
+            latch.await(10L, TimeUnit.SECONDS);
+            System.out.println(String.format("%.0f%% complete %s", 100 * count.get() / (double) totalCount, errors.get() > 0 ? ("Errors: " + errors.get()) : ""));
+            assert errors.get() == 0;
+        }
+    }
+
     private static void testInsertions(int totalCount, int perTestCount, int testKeyRatio, int modificationBatchSize, boolean quickEquality) throws ExecutionException, InterruptedException
     {
         int batchesPerTest = perTestCount / modificationBatchSize;
diff --git a/test/conf/cassandra-topology.yaml b/test/conf/cassandra-topology.yaml
deleted file mode 100644
index 51542da..0000000
--- a/test/conf/cassandra-topology.yaml
+++ /dev/null
@@ -1,74 +0,0 @@
-# Test YAML topology configuration file for Cassandra,
-# to be used with YamlFileNetworkTopologySnitch.
-
-# The topology, as a list of data centers.
-topology:
-    # Each data center has a name and a list of racks.
-    - dc_name: DC1
-      racks:
-          # Each rack has a name and a list of nodes.
-          - rack_name: RAC1
-            nodes:
-                # Each node has a broadcast address (required)
-                # and a data-center-local address (optional).
-                # If dc_local_address is specified, its peers
-                # in the same data center will attempt to
-                # reconnect over that address instead.
-                - broadcast_address: 127.0.0.1
-                  dc_local_address: 9.0.0.1
-                - broadcast_address: 192.168.1.100
-                  dc_local_address: 9.0.0.2
-                - broadcast_address: 10.0.0.10
-                  dc_local_address: 9.0.0.3
-                - broadcast_address: 10.0.0.11
-                  dc_local_address: 9.0.0.4
-          - rack_name: RAC2
-            nodes:
-                - broadcast_address: 192.168.2.200
-                - broadcast_address: 10.0.0.12
-                - broadcast_address: 10.0.0.13
-                - broadcast_address: 127.0.0.2
-          - rack_name: RAC3
-            nodes:
-                - broadcast_address: 127.0.0.3
-    - dc_name: DC2
-      racks:
-          - rack_name: RAC1
-            nodes:
-                - broadcast_address: 10.20.114.10
-                - broadcast_address: 10.20.114.11
-          - rack_name: RAC2
-            nodes:
-                - broadcast_address: 10.20.114.15
-          - rack_name: RAC4
-            nodes:
-                - broadcast_address: 127.0.0.4
-          - rack_name: RAC5
-            nodes:
-                - broadcast_address: 127.0.0.5
-    - dc_name: DC3
-      racks:
-          - rack_name: RAC1
-            nodes:
-                - broadcast_address: 10.21.119.13
-                - broadcast_address: 10.21.119.10
-          - rack_name: RAC2
-            nodes:
-                - broadcast_address: 10.21.119.14
-          - rack_name: RAC6
-            nodes:
-                - broadcast_address: 127.0.0.6
-          - rack_name: RAC7
-            nodes:
-                - broadcast_address: 127.0.0.7
-          - rack_name: RAC8
-            nodes:
-                - broadcast_address: 127.0.0.8
-
-
-# Default data center name for unknown nodes
-default_dc_name: DC1
-
-# Default rack name for unknown nodes
-default_rack_name: r1
-
diff --git a/test/conf/cassandra.yaml b/test/conf/cassandra.yaml
index 7be72dd..3d3de84 100644
--- a/test/conf/cassandra.yaml
+++ b/test/conf/cassandra.yaml
@@ -7,15 +7,14 @@
 commitlog_sync: batch
 commitlog_sync_batch_window_in_ms: 1.0
 commitlog_segment_size_in_mb: 5
+commitlog_directory: build/test/cassandra/commitlog
 partitioner: org.apache.cassandra.dht.ByteOrderedPartitioner
 listen_address: 127.0.0.1
 storage_port: 7010
-ssl_storage_port: 7011
 rpc_port: 9170
 start_native_transport: true
 native_transport_port: 9042
 column_index_size_in_kb: 4
-commitlog_directory: build/test/cassandra/commitlog
 saved_caches_directory: build/test/cassandra/saved_caches
 data_file_directories:
     - build/test/cassandra/data
@@ -37,3 +36,6 @@
 incremental_backups: true
 concurrent_compactors: 4
 compaction_throughput_mb_per_sec: 0
+row_cache_class_name: org.apache.cassandra.cache.OHCProvider
+row_cache_size_in_mb: 16
+enable_user_defined_functions: true
diff --git a/test/conf/cassandra_pig.yaml b/test/conf/cassandra_pig.yaml
new file mode 100644
index 0000000..68615cf
--- /dev/null
+++ b/test/conf/cassandra_pig.yaml
@@ -0,0 +1,41 @@
+#
+# Warning!
+# Consider the effects on 'o.a.c.i.s.LegacySSTableTest' before changing schemas in this file.
+#
+cluster_name: Test Cluster
+memtable_allocation_type: offheap_objects
+commitlog_sync: batch
+commitlog_sync_batch_window_in_ms: 1.0
+commitlog_segment_size_in_mb: 5
+commitlog_directory: build/test/cassandra/commitlog
+partitioner: org.apache.cassandra.dht.Murmur3Partitioner
+listen_address: 127.0.0.1
+storage_port: 7010
+rpc_port: 9170
+start_native_transport: true
+native_transport_port: 9042
+column_index_size_in_kb: 4
+saved_caches_directory: build/test/cassandra/saved_caches
+data_file_directories:
+    - build/test/cassandra/data
+disk_access_mode: mmap
+seed_provider:
+    - class_name: org.apache.cassandra.locator.SimpleSeedProvider
+      parameters:
+          - seeds: "127.0.0.1"
+endpoint_snitch: org.apache.cassandra.locator.SimpleSnitch
+dynamic_snitch: true
+request_scheduler: org.apache.cassandra.scheduler.RoundRobinScheduler
+request_scheduler_id: keyspace
+server_encryption_options:
+    internode_encryption: none
+    keystore: conf/.keystore
+    keystore_password: cassandra
+    truststore: conf/.truststore
+    truststore_password: cassandra
+incremental_backups: true
+concurrent_compactors: 4
+compaction_throughput_mb_per_sec: 0
+row_cache_class_name: org.apache.cassandra.cache.OHCProvider
+row_cache_size_in_mb: 16
+enable_user_defined_functions: true
diff --git a/test/conf/commitlog_compression.yaml b/test/conf/commitlog_compression.yaml
new file mode 100644
index 0000000..9849d7b
--- /dev/null
+++ b/test/conf/commitlog_compression.yaml
@@ -0,0 +1,2 @@
+commitlog_compression:
+    - class_name: LZ4Compressor
diff --git a/test/conf/keystore.jks b/test/conf/keystore.jks
new file mode 100644
index 0000000..334025d
--- /dev/null
+++ b/test/conf/keystore.jks
Binary files differ
diff --git a/test/conf/logback-test.xml b/test/conf/logback-test.xml
index faffac4..6d75aaf 100644
--- a/test/conf/logback-test.xml
+++ b/test/conf/logback-test.xml
@@ -17,7 +17,9 @@
  under the License.
 -->
 
-<configuration>
+<configuration debug="false">
+  <shutdownHook class="ch.qos.logback.core.hook.DelayingShutdownHook"/>
+
   <appender name="FILE" class="ch.qos.logback.core.rolling.RollingFileAppender">
     <file>./build/test/logs/system.log</file>
     <rollingPolicy class="ch.qos.logback.core.rolling.FixedWindowRollingPolicy">
@@ -31,9 +33,17 @@
     </triggeringPolicy>
     <encoder>
       <pattern>%-5level [%thread] %date{ISO8601} %msg%n</pattern>
+      <immediateFlush>false</immediateFlush>
     </encoder>
   </appender>
   
+  <appender name="ASYNCFILE" class="ch.qos.logback.classic.AsyncAppender">
+      <discardingThreshold>0</discardingThreshold>
+      <maxFlushTime>0</maxFlushTime>
+      <queueSize>1024</queueSize>
+      <appender-ref ref="FILE"/>
+  </appender>
+
     <appender name="STDERR" target="System.err" class="ch.qos.logback.core.ConsoleAppender">
     <encoder>
       <pattern>%-5level %date{HH:mm:ss,SSS} %msg%n</pattern>
@@ -51,9 +61,11 @@
       <level>WARN</level>
     </filter>
   </appender>
-        
+
+  <logger name="org.apache.hadoop" level="WARN"/>
+
   <root level="DEBUG">
-    <appender-ref ref="FILE" />
+    <appender-ref ref="ASYNCFILE" />
     <appender-ref ref="STDERR" />
     <appender-ref ref="STDOUT" />
   </root>
diff --git a/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Data.db b/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Data.db
new file mode 100644
index 0000000..98d3f41
--- /dev/null
+++ b/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Data.db
Binary files differ
diff --git a/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Digest.sha1 b/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Digest.sha1
new file mode 100644
index 0000000..470b056
--- /dev/null
+++ b/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Digest.sha1
@@ -0,0 +1 @@
+9ee805b905aa147afe14d4f37f5ed3be3af53c72  Keyspace1-legacyleveled-ic-0-Data.db
\ No newline at end of file
diff --git a/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Filter.db b/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Filter.db
new file mode 100644
index 0000000..c63729b
--- /dev/null
+++ b/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Filter.db
Binary files differ
diff --git a/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Index.db b/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Index.db
new file mode 100644
index 0000000..6603018
--- /dev/null
+++ b/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Index.db
Binary files differ
diff --git a/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Statistics.db b/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Statistics.db
new file mode 100644
index 0000000..5ed9ce0
--- /dev/null
+++ b/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Statistics.db
Binary files differ
diff --git a/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Summary.db b/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Summary.db
new file mode 100644
index 0000000..c1c8fd8
--- /dev/null
+++ b/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-Summary.db
Binary files differ
diff --git a/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-TOC.txt b/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-TOC.txt
new file mode 100644
index 0000000..6baaf14
--- /dev/null
+++ b/test/data/invalid-legacy-sstables/Keyspace1/Standard1/Keyspace1-Standard1-ic-0-TOC.txt
@@ -0,0 +1,7 @@
+Filter.db
+Summary.db
+Data.db
+Digest.sha1
+Index.db
+TOC.txt
+Statistics.db
diff --git a/test/data/invalid-legacy-sstables/Keyspace1/Standard1/legacyleveled.json b/test/data/invalid-legacy-sstables/Keyspace1/Standard1/legacyleveled.json
new file mode 100644
index 0000000..1fc9c01
--- /dev/null
+++ b/test/data/invalid-legacy-sstables/Keyspace1/Standard1/legacyleveled.json
@@ -0,0 +1,27 @@
+{
+  "generations" : [ {
+    "generation" : 0,
+    "members" : [ 0 ]
+  }, {
+    "generation" : 1,
+    "members" : [ 1 ]
+  }, {
+    "generation" : 2,
+    "members" : [ 2 ]
+  }, {
+    "generation" : 3,
+    "members" : [ ]
+  }, {
+    "generation" : 4,
+    "members" : [ ]
+  }, {
+    "generation" : 5,
+    "members" : [ ]
+  }, {
+    "generation" : 6,
+    "members" : [ ]
+  }, {
+    "generation" : 7,
+    "members" : [ ]
+  } ]
+}
\ No newline at end of file
diff --git a/test/data/legacy-commitlog/2.0/CommitLog-3-1431528750790.log b/test/data/legacy-commitlog/2.0/CommitLog-3-1431528750790.log
new file mode 100644
index 0000000..3301331
--- /dev/null
+++ b/test/data/legacy-commitlog/2.0/CommitLog-3-1431528750790.log
Binary files differ
diff --git a/test/data/legacy-commitlog/2.0/CommitLog-3-1431528750791.log b/test/data/legacy-commitlog/2.0/CommitLog-3-1431528750791.log
new file mode 100644
index 0000000..04314d6
--- /dev/null
+++ b/test/data/legacy-commitlog/2.0/CommitLog-3-1431528750791.log
Binary files differ
diff --git a/test/data/legacy-commitlog/2.0/CommitLog-3-1431528750792.log b/test/data/legacy-commitlog/2.0/CommitLog-3-1431528750792.log
new file mode 100644
index 0000000..a9af9e4
--- /dev/null
+++ b/test/data/legacy-commitlog/2.0/CommitLog-3-1431528750792.log
Binary files differ
diff --git a/test/data/legacy-commitlog/2.0/CommitLog-3-1431528750793.log b/test/data/legacy-commitlog/2.0/CommitLog-3-1431528750793.log
new file mode 100644
index 0000000..3301331
--- /dev/null
+++ b/test/data/legacy-commitlog/2.0/CommitLog-3-1431528750793.log
Binary files differ
diff --git a/test/data/legacy-commitlog/2.0/hash.txt b/test/data/legacy-commitlog/2.0/hash.txt
new file mode 100644
index 0000000..4bbec02
--- /dev/null
+++ b/test/data/legacy-commitlog/2.0/hash.txt
@@ -0,0 +1,3 @@
+cfid = 4d331c44-f018-302b-91c2-2dcf94c4bfad

+cells = 9724

+hash = -682777064

diff --git a/test/data/legacy-commitlog/2.1/CommitLog-4-1431529069529.log b/test/data/legacy-commitlog/2.1/CommitLog-4-1431529069529.log
new file mode 100644
index 0000000..60064ee
--- /dev/null
+++ b/test/data/legacy-commitlog/2.1/CommitLog-4-1431529069529.log
Binary files differ
diff --git a/test/data/legacy-commitlog/2.1/CommitLog-4-1431529069530.log b/test/data/legacy-commitlog/2.1/CommitLog-4-1431529069530.log
new file mode 100644
index 0000000..fdf7071
--- /dev/null
+++ b/test/data/legacy-commitlog/2.1/CommitLog-4-1431529069530.log
Binary files differ
diff --git a/test/data/legacy-commitlog/2.1/hash.txt b/test/data/legacy-commitlog/2.1/hash.txt
new file mode 100644
index 0000000..f05cf97
--- /dev/null
+++ b/test/data/legacy-commitlog/2.1/hash.txt
@@ -0,0 +1,3 @@
+cfid = 6c622920-f980-11e4-b8a0-e7d448d5e26d

+cells = 5165

+hash = -1915888171

diff --git a/test/data/legacy-commitlog/2.2-lz4-bitrot/CommitLog-5-1438186885380.log b/test/data/legacy-commitlog/2.2-lz4-bitrot/CommitLog-5-1438186885380.log
new file mode 100644
index 0000000..d248d59
--- /dev/null
+++ b/test/data/legacy-commitlog/2.2-lz4-bitrot/CommitLog-5-1438186885380.log
Binary files differ
diff --git a/test/data/legacy-commitlog/2.2-lz4-bitrot/hash.txt b/test/data/legacy-commitlog/2.2-lz4-bitrot/hash.txt
new file mode 100644
index 0000000..c4d8fe7
--- /dev/null
+++ b/test/data/legacy-commitlog/2.2-lz4-bitrot/hash.txt
@@ -0,0 +1,6 @@
+#CommitLog bitrot test, version 2.2.0-SNAPSHOT
+#This is a copy of 2.2-lz4 with some overwritten bytes.
+#Replaying this should result in an error which can be overridden.
+cells=6051
+hash=-170208326
+cfid=dc32ce20-360d-11e5-826c-afadad37221d
diff --git a/test/data/legacy-commitlog/2.2-lz4-bitrot2/CommitLog-5-1438186885380.log b/test/data/legacy-commitlog/2.2-lz4-bitrot2/CommitLog-5-1438186885380.log
new file mode 100644
index 0000000..083d65c
--- /dev/null
+++ b/test/data/legacy-commitlog/2.2-lz4-bitrot2/CommitLog-5-1438186885380.log
Binary files differ
diff --git a/test/data/legacy-commitlog/2.2-lz4-bitrot2/hash.txt b/test/data/legacy-commitlog/2.2-lz4-bitrot2/hash.txt
new file mode 100644
index 0000000..c49dda0
--- /dev/null
+++ b/test/data/legacy-commitlog/2.2-lz4-bitrot2/hash.txt
@@ -0,0 +1,6 @@
+#CommitLog upgrade test, version 2.2.0-SNAPSHOT
+#This is a copy of 2.2-lz4 with some overwritten bytes.
+#Replaying this should result in an error which can be overridden.
+cells=6037
+hash=-1312748407
+cfid=dc32ce20-360d-11e5-826c-afadad37221d
diff --git a/test/data/legacy-commitlog/2.2-lz4-truncated/CommitLog-5-1438186885380.log b/test/data/legacy-commitlog/2.2-lz4-truncated/CommitLog-5-1438186885380.log
new file mode 100644
index 0000000..939d408
--- /dev/null
+++ b/test/data/legacy-commitlog/2.2-lz4-truncated/CommitLog-5-1438186885380.log
Binary files differ
diff --git a/test/data/legacy-commitlog/2.2-lz4-truncated/hash.txt b/test/data/legacy-commitlog/2.2-lz4-truncated/hash.txt
new file mode 100644
index 0000000..ce7f600
--- /dev/null
+++ b/test/data/legacy-commitlog/2.2-lz4-truncated/hash.txt
@@ -0,0 +1,5 @@
+#Truncated CommitLog test.
+#This is a copy of 2.2-lz4 with the last 50 bytes deleted.
+cells=6037
+hash=-889057729
+cfid=dc32ce20-360d-11e5-826c-afadad37221d
diff --git a/test/long/org/apache/cassandra/cql3/CorruptionTest.java b/test/long/org/apache/cassandra/cql3/CorruptionTest.java
index 1a42112..43cf5e0 100644
--- a/test/long/org/apache/cassandra/cql3/CorruptionTest.java
+++ b/test/long/org/apache/cassandra/cql3/CorruptionTest.java
@@ -28,7 +28,6 @@
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 
-import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
diff --git a/test/long/org/apache/cassandra/db/LongFlushMemtableTest.java b/test/long/org/apache/cassandra/db/LongFlushMemtableTest.java
index 4bb8fdd..24993c8 100644
--- a/test/long/org/apache/cassandra/db/LongFlushMemtableTest.java
+++ b/test/long/org/apache/cassandra/db/LongFlushMemtableTest.java
@@ -24,22 +24,36 @@
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.service.MigrationManager;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
-public class LongFlushMemtableTest extends SchemaLoader
+public class LongFlushMemtableTest
 {
-    @Test
-    public void testFlushMemtables() throws IOException, ConfigurationException
+    public static final String KEYSPACE1 = "LongFlushMemtableTest";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        SchemaLoader.loadSchema();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1));
+    }
+
+    @Test
+    public void testFlushMemtables() throws ConfigurationException
+    {
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         for (int i = 0; i < 100; i++)
         {
             CFMetaData metadata = CFMetaData.denseCFMetaData(keyspace.getName(), "_CF" + i, UTF8Type.instance);
@@ -50,8 +64,8 @@
         {
             for (int i = 0; i < 100; i++)
             {
-                Mutation rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("key" + j));
-                ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "_CF" + i);
+                Mutation rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("key" + j));
+                ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "_CF" + i);
                 // don't cheat by allocating this outside of the loop; that defeats the purpose of deliberately using lots of memory
                 ByteBuffer value = ByteBuffer.allocate(100000);
                 cf.addColumn(new BufferCell(Util.cellname("c"), value));
@@ -64,7 +78,7 @@
         for (ColumnFamilyStore cfs : ColumnFamilyStore.all())
         {
             if (cfs.name.startsWith("_CF"))
-                flushes += cfs.getMemtableSwitchCount();
+                flushes += cfs.metric.memtableSwitchCount.getCount();
         }
         assert flushes > 0;
     }
diff --git a/test/long/org/apache/cassandra/db/LongKeyspaceTest.java b/test/long/org/apache/cassandra/db/LongKeyspaceTest.java
index 7a5b837..fe22da8 100644
--- a/test/long/org/apache/cassandra/db/LongKeyspaceTest.java
+++ b/test/long/org/apache/cassandra/db/LongKeyspaceTest.java
@@ -18,27 +18,43 @@
 
 package org.apache.cassandra.db;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.WrappedRunnable;
 import static org.apache.cassandra.Util.column;
 
 import org.apache.cassandra.Util;
 
-
-public class LongKeyspaceTest extends SchemaLoader
+public class LongKeyspaceTest
 {
+    public static final String KEYSPACE1 = "LongKeyspaceTest";
+    public static final String CF_STANDARD = "Standard1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD));
+    }
+
     @Test
     public void testGetRowMultiColumn() throws Throwable
     {
-        final Keyspace keyspace = Keyspace.open("Keyspace1");
+        final Keyspace keyspace = Keyspace.open(KEYSPACE1);
         final ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore("Standard1");
 
         for (int i = 1; i < 5000; i += 100)
         {
-            Mutation rm = new Mutation("Keyspace1", Util.dk("key" + i).getKey());
-            ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+            Mutation rm = new Mutation(KEYSPACE1, Util.dk("key" + i).getKey());
+            ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
             for (int j = 0; j < i; j++)
                 cf.addColumn(column("c" + j, "v" + j, 1L));
             rm.add(cf);
diff --git a/test/long/org/apache/cassandra/db/commitlog/ComitLogStress.java b/test/long/org/apache/cassandra/db/commitlog/ComitLogStress.java
index 5b334cc..b4efd49 100644
--- a/test/long/org/apache/cassandra/db/commitlog/ComitLogStress.java
+++ b/test/long/org/apache/cassandra/db/commitlog/ComitLogStress.java
@@ -48,10 +48,11 @@
             System.out.println("Setting num threads to: " + NUM_THREADS);
         }
         ExecutorService executor = new JMXEnabledThreadPoolExecutor(NUM_THREADS, NUM_THREADS, 60,
-                TimeUnit.SECONDS, new ArrayBlockingQueue<Runnable>(10 * NUM_THREADS), new NamedThreadFactory(""), "");
+                TimeUnit.SECONDS, new ArrayBlockingQueue<Runnable>(10 * NUM_THREADS), new NamedThreadFactory("Stress"), "");
         ScheduledExecutorService scheduled = Executors.newScheduledThreadPool(1);
 
         org.apache.cassandra.SchemaLoader.loadSchema();
+        org.apache.cassandra.SchemaLoader.schemaDefinition(""); // leave def. blank to maintain old behaviour
         final AtomicLong count = new AtomicLong();
         final long start = System.currentTimeMillis();
         System.out.println(String.format(format, "seconds", "max_mb", "allocated_mb", "free_mb", "diffrence", "count"));
@@ -85,10 +86,12 @@
         public void run() {
             String ks = "Keyspace1";
             ByteBuffer key = ByteBufferUtil.bytes(keyString);
-            Mutation mutation = new Mutation(ks, key);
-            mutation.add("Standard1", Util.cellname("name"), ByteBufferUtil.bytes("value"),
-                    System.currentTimeMillis());
-            CommitLog.instance.add(mutation);
+            for (int i=0; i<100; ++i) {
+                Mutation mutation = new Mutation(ks, key);
+                mutation.add("Standard1", Util.cellname("name"), ByteBufferUtil.bytes("value" + i),
+                        System.currentTimeMillis());
+                CommitLog.instance.add(mutation);
+            }
         }
     }
 }
diff --git a/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java b/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
new file mode 100644
index 0000000..4604c49
--- /dev/null
+++ b/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
@@ -0,0 +1,494 @@
+package org.apache.cassandra.db.commitlog;
+
+/*
+ *
+ * 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.
+ *
+ */
+
+import java.io.DataInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import junit.framework.Assert;
+
+import com.google.common.util.concurrent.RateLimiter;
+
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.Util;
+import org.apache.cassandra.config.Config.CommitLogSync;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.ParameterizedClass;
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.db.Cell;
+import org.apache.cassandra.db.ColumnFamily;
+import org.apache.cassandra.db.ColumnSerializer;
+import org.apache.cassandra.db.Mutation;
+import org.apache.cassandra.io.util.FastByteArrayInputStream;
+
+public class CommitLogStressTest
+{
+    public static ByteBuffer dataSource;
+
+    public static int NUM_THREADS = 4 * Runtime.getRuntime().availableProcessors() - 1;
+    public static int numCells = 1;
+    public static int cellSize = 1024;
+    public static int rateLimit = 0;
+    public static int runTimeMs = 10000;
+
+    public static String location = DatabaseDescriptor.getCommitLogLocation() + "/stress";
+
+    public static int hash(int hash, ByteBuffer bytes)
+    {
+        int shift = 0;
+        for (int i = 0; i < bytes.limit(); i++)
+        {
+            hash += (bytes.get(i) & 0xFF) << shift;
+            shift = (shift + 8) & 0x1F;
+        }
+        return hash;
+    }
+
+    public static void main(String[] args) throws Exception
+    {
+        try
+        {
+            if (args.length >= 1)
+            {
+                NUM_THREADS = Integer.parseInt(args[0]);
+                System.out.println("Setting num threads to: " + NUM_THREADS);
+            }
+
+            if (args.length >= 2)
+            {
+                numCells = Integer.parseInt(args[1]);
+                System.out.println("Setting num cells to: " + numCells);
+            }
+
+            if (args.length >= 3)
+            {
+                cellSize = Integer.parseInt(args[1]);
+                System.out.println("Setting cell size to: " + cellSize + " be aware the source corpus may be small");
+            }
+
+            if (args.length >= 4)
+            {
+                rateLimit = Integer.parseInt(args[1]);
+                System.out.println("Setting per thread rate limit to: " + rateLimit);
+            }
+            initialize();
+
+            CommitLogStressTest tester = new CommitLogStressTest();
+            tester.cleanDir();
+            tester.testFixedSize();
+        }
+        catch (Throwable e)
+        {
+            e.printStackTrace(System.err);
+        }
+        finally
+        {
+            System.exit(0);
+        }
+    }
+
+    boolean failed = false;
+    volatile boolean stop = false;
+    boolean randomSize = false;
+    boolean discardedRun = false;
+    ReplayPosition discardedPos;
+
+    @BeforeClass
+    static public void initialize() throws IOException
+    {
+        try (FileInputStream fis = new FileInputStream("CHANGES.txt"))
+        {
+            dataSource = ByteBuffer.allocateDirect((int) fis.getChannel().size());
+            while (dataSource.hasRemaining())
+            {
+                fis.getChannel().read(dataSource);
+            }
+            dataSource.flip();
+        }
+
+        SchemaLoader.loadSchema();
+        SchemaLoader.schemaDefinition(""); // leave def. blank to maintain old behaviour
+    }
+
+    @Before
+    public void cleanDir()
+    {
+        File dir = new File(location);
+        if (dir.isDirectory())
+        {
+            File[] files = dir.listFiles();
+
+            for (File f : files)
+                if (!f.delete())
+                    Assert.fail("Failed to delete " + f);
+        }
+        else
+        {
+            dir.mkdir();
+        }
+    }
+
+    @Test
+    public void testRandomSize() throws Exception
+    {
+        randomSize = true;
+        discardedRun = false;
+        testAllLogConfigs();
+    }
+
+    @Test
+    public void testFixedSize() throws Exception
+    {
+        randomSize = false;
+        discardedRun = false;
+
+        testAllLogConfigs();
+    }
+
+    @Test
+    public void testDiscardedRun() throws Exception
+    {
+        discardedRun = true;
+        randomSize = true;
+
+        testAllLogConfigs();
+    }
+
+    public void testAllLogConfigs() throws IOException, InterruptedException
+    {
+        failed = false;
+        DatabaseDescriptor.setCommitLogSyncBatchWindow(1);
+        DatabaseDescriptor.setCommitLogSyncPeriod(30);
+        DatabaseDescriptor.setCommitLogSegmentSize(32);
+        for (ParameterizedClass compressor : new ParameterizedClass[] {
+                null,
+                new ParameterizedClass("LZ4Compressor", null),
+                new ParameterizedClass("SnappyCompressor", null),
+                new ParameterizedClass("DeflateCompressor", null) })
+        {
+            DatabaseDescriptor.setCommitLogCompression(compressor);
+            for (CommitLogSync sync : CommitLogSync.values())
+            {
+                DatabaseDescriptor.setCommitLogSync(sync);
+                CommitLog commitLog = new CommitLog(location, CommitLogArchiver.disabled()).start();
+                testLog(commitLog);
+            }
+        }
+        assert !failed;
+    }
+
+    public void testLog(CommitLog commitLog) throws IOException, InterruptedException
+    {
+        System.out.format("\nTesting commit log size %.0fmb, compressor %s, sync %s%s%s\n",
+                          mb(DatabaseDescriptor.getCommitLogSegmentSize()),
+                          commitLog.configuration.getCompressorName(),
+                          commitLog.executor.getClass().getSimpleName(),
+                          randomSize ? " random size" : "",
+                          discardedRun ? " with discarded run" : "");
+        commitLog.allocator.enableReserveSegmentCreation();
+
+        final List<CommitlogExecutor> threads = new ArrayList<>();
+        ScheduledExecutorService scheduled = startThreads(commitLog, threads);
+
+        discardedPos = ReplayPosition.NONE;
+        if (discardedRun)
+        {
+            // Makes sure post-break data is not deleted, and that replayer correctly rejects earlier mutations.
+            Thread.sleep(runTimeMs / 3);
+            stop = true;
+            scheduled.shutdown();
+            scheduled.awaitTermination(2, TimeUnit.SECONDS);
+
+            for (CommitlogExecutor t : threads)
+            {
+                t.join();
+                if (t.rp.compareTo(discardedPos) > 0)
+                    discardedPos = t.rp;
+            }
+            verifySizes(commitLog);
+
+            commitLog.discardCompletedSegments(Schema.instance.getCFMetaData("Keyspace1", "Standard1").cfId,
+                                               discardedPos);
+            threads.clear();
+            System.out.format("Discarded at %s\n", discardedPos);
+            verifySizes(commitLog);
+
+            scheduled = startThreads(commitLog, threads);
+        }
+
+        Thread.sleep(runTimeMs);
+        stop = true;
+        scheduled.shutdown();
+        scheduled.awaitTermination(2, TimeUnit.SECONDS);
+
+        int hash = 0;
+        int cells = 0;
+        for (CommitlogExecutor t : threads)
+        {
+            t.join();
+            hash += t.hash;
+            cells += t.cells;
+        }
+        verifySizes(commitLog);
+
+        commitLog.shutdownBlocking();
+
+        System.out.print("Stopped. Replaying... ");
+        System.out.flush();
+        Replayer repl = new Replayer(commitLog);
+        File[] files = new File(location).listFiles();
+        repl.recover(files);
+
+        for (File f : files)
+            if (!f.delete())
+                Assert.fail("Failed to delete " + f);
+
+        if (hash == repl.hash && cells == repl.cells)
+            System.out.println("Test success.");
+        else
+        {
+            System.out.format("Test failed. Cells %d expected %d, hash %d expected %d.\n",
+                              repl.cells,
+                              cells,
+                              repl.hash,
+                              hash);
+            failed = true;
+        }
+    }
+
+    private void verifySizes(CommitLog commitLog)
+    {
+        // Complete anything that's still left to write.
+        commitLog.executor.requestExtraSync().awaitUninterruptibly();
+        // One await() does not suffice as we may be signalled when an ongoing sync finished. Request another
+        // (which shouldn't write anything) to make sure the first we triggered completes.
+        // FIXME: The executor should give us a chance to await completion of the sync we requested.
+        commitLog.executor.requestExtraSync().awaitUninterruptibly();
+        // Wait for any pending deletes or segment allocations to complete.
+        commitLog.allocator.awaitManagementTasksCompletion();
+
+        long combinedSize = 0;
+        for (File f : new File(commitLog.location).listFiles())
+            combinedSize += f.length();
+        Assert.assertEquals(combinedSize, commitLog.getActiveOnDiskSize());
+
+        List<String> logFileNames = commitLog.getActiveSegmentNames();
+        Map<String, Double> ratios = commitLog.getActiveSegmentCompressionRatios();
+        Collection<CommitLogSegment> segments = commitLog.allocator.getActiveSegments();
+
+        for (CommitLogSegment segment : segments)
+        {
+            Assert.assertTrue(logFileNames.remove(segment.getName()));
+            Double ratio = ratios.remove(segment.getName());
+
+            Assert.assertEquals(segment.logFile.length(), segment.onDiskSize());
+            Assert.assertEquals(segment.onDiskSize() * 1.0 / segment.contentSize(), ratio, 0.01);
+        }
+        Assert.assertTrue(logFileNames.isEmpty());
+        Assert.assertTrue(ratios.isEmpty());
+    }
+
+    public ScheduledExecutorService startThreads(final CommitLog commitLog, final List<CommitlogExecutor> threads)
+    {
+        stop = false;
+        for (int ii = 0; ii < NUM_THREADS; ii++)
+        {
+            final CommitlogExecutor t = new CommitlogExecutor(commitLog, new Random(ii));
+            threads.add(t);
+            t.start();
+        }
+
+        final long start = System.currentTimeMillis();
+        Runnable printRunnable = new Runnable()
+        {
+            long lastUpdate = 0;
+
+            public void run()
+            {
+                Runtime runtime = Runtime.getRuntime();
+                long maxMemory = runtime.maxMemory();
+                long allocatedMemory = runtime.totalMemory();
+                long freeMemory = runtime.freeMemory();
+                long temp = 0;
+                long sz = 0;
+                for (CommitlogExecutor cle : threads)
+                {
+                    temp += cle.counter.get();
+                    sz += cle.dataSize;
+                }
+                double time = (System.currentTimeMillis() - start) / 1000.0;
+                double avg = (temp / time);
+                System.out
+                        .println(
+                        String.format("second %d mem max %.0fmb allocated %.0fmb free %.0fmb mutations %d since start %d avg %.3f content %.1fmb ondisk %.1fmb transfer %.3fmb",
+                                      ((System.currentTimeMillis() - start) / 1000),
+                                      mb(maxMemory),
+                                      mb(allocatedMemory),
+                                      mb(freeMemory),
+                                      (temp - lastUpdate),
+                                      lastUpdate,
+                                      avg,
+                                      mb(commitLog.getActiveContentSize()),
+                                      mb(commitLog.getActiveOnDiskSize()),
+                                      mb(sz / time)));
+                lastUpdate = temp;
+            }
+        };
+        ScheduledExecutorService scheduled = Executors.newScheduledThreadPool(1);
+        scheduled.scheduleAtFixedRate(printRunnable, 1, 1, TimeUnit.SECONDS);
+        return scheduled;
+    }
+
+    private static double mb(long maxMemory)
+    {
+        return maxMemory / (1024.0 * 1024);
+    }
+
+    private static double mb(double maxMemory)
+    {
+        return maxMemory / (1024 * 1024);
+    }
+
+    public static ByteBuffer randomBytes(int quantity, Random tlr)
+    {
+        ByteBuffer slice = ByteBuffer.allocate(quantity);
+        ByteBuffer source = dataSource.duplicate();
+        source.position(tlr.nextInt(source.capacity() - quantity));
+        source.limit(source.position() + quantity);
+        slice.put(source);
+        slice.flip();
+        return slice;
+    }
+
+    public class CommitlogExecutor extends Thread
+    {
+        final AtomicLong counter = new AtomicLong();
+        int hash = 0;
+        int cells = 0;
+        int dataSize = 0;
+        final CommitLog commitLog;
+        final Random random;
+
+        volatile ReplayPosition rp;
+
+        public CommitlogExecutor(CommitLog commitLog, Random rand)
+        {
+            this.commitLog = commitLog;
+            this.random = rand;
+        }
+
+        public void run()
+        {
+            RateLimiter rl = rateLimit != 0 ? RateLimiter.create(rateLimit) : null;
+            final Random rand = random != null ? random : ThreadLocalRandom.current();
+            while (!stop)
+            {
+                if (rl != null)
+                    rl.acquire();
+                String ks = "Keyspace1";
+                ByteBuffer key = randomBytes(16, rand);
+                Mutation mutation = new Mutation(ks, key);
+
+                for (int ii = 0; ii < numCells; ii++)
+                {
+                    int sz = randomSize ? rand.nextInt(cellSize) : cellSize;
+                    ByteBuffer bytes = randomBytes(sz, rand);
+                    mutation.add("Standard1", Util.cellname("name" + ii), bytes, System.currentTimeMillis());
+                    hash = hash(hash, bytes);
+                    ++cells;
+                    dataSize += sz;
+                }
+                rp = commitLog.add(mutation);
+                counter.incrementAndGet();
+            }
+        }
+    }
+
+    class Replayer extends CommitLogReplayer
+    {
+        Replayer(CommitLog log)
+        {
+            super(log, discardedPos, null, ReplayFilter.create());
+        }
+
+        int hash = 0;
+        int cells = 0;
+
+        @Override
+        void replayMutation(byte[] inputBuffer, int size, final int entryLocation, final CommitLogDescriptor desc)
+        {
+            if (desc.id < discardedPos.segment)
+            {
+                System.out.format("Mutation from discarded segment, segment %d pos %d\n", desc.id, entryLocation);
+                return;
+            }
+            else if (desc.id == discardedPos.segment && entryLocation <= discardedPos.position)
+                // Skip over this mutation.
+                return;
+
+            FastByteArrayInputStream bufIn = new FastByteArrayInputStream(inputBuffer, 0, size);
+            Mutation mutation;
+            try
+            {
+                mutation = Mutation.serializer.deserialize(new DataInputStream(bufIn),
+                                                           desc.getMessagingVersion(),
+                                                           ColumnSerializer.Flag.LOCAL);
+            }
+            catch (IOException e)
+            {
+                // Test fails.
+                throw new AssertionError(e);
+            }
+
+            for (ColumnFamily cf : mutation.getColumnFamilies())
+            {
+                for (Cell c : cf.getSortedColumns())
+                {
+                    if (new String(c.name().toByteBuffer().array(), StandardCharsets.UTF_8).startsWith("name"))
+                    {
+                        hash = hash(hash, c.value());
+                        ++cells;
+                    }
+                }
+            }
+        }
+
+    }
+}
diff --git a/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java b/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java
index e87e336..e6c8f56 100644
--- a/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java
+++ b/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java
@@ -18,28 +18,45 @@
 */
 package org.apache.cassandra.db.compaction;
 
-import java.io.IOException;
 import java.util.*;
-import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 
+import org.junit.BeforeClass;
 import org.junit.Before;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.Util;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.io.sstable.SSTableUtils;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import static org.junit.Assert.assertEquals;
 
-public class LongCompactionsTest extends SchemaLoader
+public class LongCompactionsTest
 {
     public static final String KEYSPACE1 = "Keyspace1";
+    public static final String CF_STANDARD = "Standard1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        Map<String, String> compactionOptions = new HashMap<>();
+        compactionOptions.put("tombstone_compaction_interval", "1");
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD)
+                                                .compactionStrategyOptions(compactionOptions));
+    }
 
     @Before
     public void cleanupFiles()
@@ -83,7 +100,7 @@
         Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
 
-        ArrayList<SSTableReader> sstables = new ArrayList<SSTableReader>();
+        ArrayList<SSTableReader> sstables = new ArrayList<>();
         for (int k = 0; k < sstableCount; k++)
         {
             SortedMap<String,ColumnFamily> rows = new TreeMap<String,ColumnFamily>();
@@ -96,7 +113,7 @@
                     // last sstable has highest timestamps
                     cols[i] = Util.column(String.valueOf(i), String.valueOf(i), k);
                 }
-                rows.put(key, SSTableUtils.createCF(Long.MIN_VALUE, Integer.MIN_VALUE, cols));
+                rows.put(key, SSTableUtils.createCF(KEYSPACE1, CF_STANDARD, Long.MIN_VALUE, Integer.MIN_VALUE, cols));
             }
             SSTableReader sstable = SSTableUtils.prepare().write(rows);
             sstables.add(sstable);
@@ -108,8 +125,11 @@
 
         long start = System.nanoTime();
         final int gcBefore = (int) (System.currentTimeMillis() / 1000) - Schema.instance.getCFMetaData(KEYSPACE1, "Standard1").getGcGraceSeconds();
-        assert store.getDataTracker().markCompacting(sstables): "Cannot markCompacting all sstables";
-        new CompactionTask(store, sstables, gcBefore, false).execute(null);
+        try (LifecycleTransaction txn = store.getTracker().tryModify(sstables, OperationType.COMPACTION))
+        {
+            assert txn != null : "Cannot markCompacting all sstables";
+            new CompactionTask(store, txn, gcBefore, false).execute(null);
+        }
         System.out.println(String.format("%s: sstables=%d rowsper=%d colsper=%d: %d ms",
                                          this.getClass().getName(),
                                          sstableCount,
@@ -119,7 +139,7 @@
     }
 
     @Test
-    public void testStandardColumnCompactions() throws IOException, ExecutionException, InterruptedException
+    public void testStandardColumnCompactions()
     {
         // this test does enough rows to force multiple block indexes to be used
         Keyspace keyspace = Keyspace.open(KEYSPACE1);
@@ -160,7 +180,7 @@
         cfs.truncateBlocking();
     }
 
-    private void forceCompactions(ColumnFamilyStore cfs) throws ExecutionException, InterruptedException
+    private void forceCompactions(ColumnFamilyStore cfs)
     {
         // re-enable compaction with thresholds low enough to force a few rounds
         cfs.setCompactionThresholds(2, 4);
@@ -178,7 +198,7 @@
 
         if (cfs.getSSTables().size() > 1)
         {
-            CompactionManager.instance.performMaximal(cfs);
+            CompactionManager.instance.performMaximal(cfs, false);
         }
     }
 }
diff --git a/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java b/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java
index b3dfb3c..8e63006 100644
--- a/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java
+++ b/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java
@@ -18,42 +18,53 @@
 package org.apache.cassandra.db.compaction;
 
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
+import java.util.*;
 import java.util.concurrent.*;
 
+import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.dht.BytesToken;
-import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
-import org.apache.cassandra.io.sstable.ISSTableScanner;
-import org.apache.cassandra.io.sstable.SSTableReader;
-import org.apache.cassandra.notifications.INotification;
-import org.apache.cassandra.notifications.INotificationConsumer;
-import org.apache.cassandra.notifications.SSTableCompactingNotification;
-import org.apache.cassandra.notifications.SSTableListChangedNotification;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.concurrent.Refs;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-public class LongLeveledCompactionStrategyTest extends SchemaLoader
+public class LongLeveledCompactionStrategyTest
 {
+    public static final String KEYSPACE1 = "LongLeveledCompactionStrategyTest";
+    public static final String CF_STANDARDLVL = "StandardLeveled";
+    public static final String CF_STANDARDLVL2 = "StandardLeveled2";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        Map<String, String> leveledOptions = new HashMap<>();
+        leveledOptions.put("sstable_size_in_mb", "1");
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARDLVL)
+                                                .compactionStrategyClass(LeveledCompactionStrategy.class)
+                                                .compactionStrategyOptions(leveledOptions),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARDLVL2)
+                                                .compactionStrategyClass(LeveledCompactionStrategy.class)
+                                                .compactionStrategyOptions(leveledOptions));
+    }
+
     @Test
     public void testParallelLeveledCompaction() throws Exception
     {
-        String ksname = "Keyspace1";
+        String ksname = KEYSPACE1;
         String cfname = "StandardLeveled";
         Keyspace keyspace = Keyspace.open(ksname);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore(cfname);
@@ -81,12 +92,10 @@
             store.forceBlockingFlush();
         }
 
-
         // Execute LCS in parallel
         ExecutorService executor = new ThreadPoolExecutor(4, 4,
                                                           Long.MAX_VALUE, TimeUnit.SECONDS,
                                                           new LinkedBlockingDeque<Runnable>());
-
         List<Runnable> tasks = new ArrayList<Runnable>();
         while (true)
         {
@@ -95,7 +104,6 @@
                 final AbstractCompactionTask nextTask = lcs.getNextBackgroundTask(Integer.MIN_VALUE);
                 if (nextTask == null)
                     break;
-
                 tasks.add(new Runnable()
                 {
                     public void run()
@@ -104,7 +112,6 @@
                     }
                 });
             }
-
             if (tasks.isEmpty())
                 break;
 
@@ -116,17 +123,15 @@
             tasks.clear();
         }
 
-
         // Assert all SSTables are lined up correctly.
         LeveledManifest manifest = lcs.manifest;
         int levels = manifest.getLevelCount();
         for (int level = 0; level < levels; level++)
         {
             List<SSTableReader> sstables = manifest.getLevel(level);
-
             // score check
-            assert (double) SSTableReader.getTotalBytes(sstables) / manifest.maxBytesForLevel(level) < 1.00;
-
+            assert (double) SSTableReader.getTotalBytes(sstables) / LeveledManifest.maxBytesForLevel(level, 1 * 1024 * 1024) < 1.00;
+            // overlap check for levels greater than 0
             for (SSTableReader sstable : sstables)
             {
                 // level check
@@ -141,117 +146,45 @@
         }
     }
 
-    class CheckThatSSTableIsReleasedOnlyAfterCompactionFinishes implements INotificationConsumer
-    {
-        public final Set<SSTableReader> finishedCompaction = new HashSet<>();
-
-        boolean failed = false;
-
-        public void handleNotification(INotification received, Object sender)
-        {
-            if (received instanceof SSTableCompactingNotification)
-            {
-                SSTableCompactingNotification notification = (SSTableCompactingNotification) received;
-                if (!notification.compacting)
-                {
-                    for (SSTableReader reader : notification.sstables)
-                    {
-                        finishedCompaction.add(reader);
-                    }
-                }
-            }
-            if (received instanceof SSTableListChangedNotification)
-            {
-                SSTableListChangedNotification notification = (SSTableListChangedNotification) received;
-                for (SSTableReader reader : notification.removed)
-                {
-                    if (finishedCompaction.contains(reader))
-                        failed = true;
-                }
-            }
-        }
-
-        boolean isFailed()
-        {
-            return failed;
-        }
-    }
-
-    @Test
-    public void testAntiCompactionAfterLCS() throws Exception
-    {
-        testParallelLeveledCompaction();
-
-        String ksname = "Keyspace1";
-        String cfname = "StandardLeveled";
-        Keyspace keyspace = Keyspace.open(ksname);
-        ColumnFamilyStore store = keyspace.getColumnFamilyStore(cfname);
-        WrappingCompactionStrategy strategy = ((WrappingCompactionStrategy) store.getCompactionStrategy());
-
-        Collection<SSTableReader> initialSSTables = store.getUnrepairedSSTables();
-        assertEquals(store.getSSTables().size(), initialSSTables.size());
-
-        CheckThatSSTableIsReleasedOnlyAfterCompactionFinishes checker = new CheckThatSSTableIsReleasedOnlyAfterCompactionFinishes();
-        store.getDataTracker().subscribe(checker);
-
-        //anti-compact a subset of sstables
-        Range<Token> range = new Range<Token>(new BytesToken("110".getBytes()), new BytesToken("111".getBytes()), store.partitioner);
-        List<Range<Token>> ranges = Arrays.asList(range);
-        Refs<SSTableReader> refs = Refs.tryRef(initialSSTables);
-        if (refs == null)
-            throw new IllegalStateException();
-        long repairedAt = 1000;
-        CompactionManager.instance.performAnticompaction(store, ranges, refs, repairedAt);
-
-        //check that sstables were released only after compaction finished
-        assertFalse("Anti-compaction released sstable from compacting set before compaction was finished",
-                    checker.isFailed());
-
-        //check there is only one global ref count
-        for (SSTableReader sstable : store.getSSTables())
-        {
-            assertFalse(sstable.isMarkedCompacted());
-            assertEquals(1, sstable.selfRef().globalCount());
-        }
-
-        //check that compacting status was clearedd in all sstables
-        assertEquals(0, store.getDataTracker().getCompacting().size());
-
-        //make sure readers were replaced correctly on unrepaired leveled manifest after anti-compaction
-        LeveledCompactionStrategy lcs = (LeveledCompactionStrategy) strategy.getWrappedStrategies().get(1);
-        for (SSTableReader reader : initialSSTables)
-        {
-            Range<Token> sstableRange = new Range<Token>(reader.first.getToken(), reader.last.getToken());
-            if (sstableRange.intersects(range))
-            {
-                assertFalse(lcs.manifest.generations[reader.getSSTableLevel()].contains(reader));
-            }
-        }
-    }
-
     @Test
     public void testLeveledScanner() throws Exception
     {
-        testParallelLeveledCompaction();
-        String ksname = "Keyspace1";
-        String cfname = "StandardLeveled";
-        Keyspace keyspace = Keyspace.open(ksname);
-        ColumnFamilyStore store = keyspace.getColumnFamilyStore(cfname);
-        store.disableAutoCompaction();
-
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        final ColumnFamilyStore store = keyspace.getColumnFamilyStore(CF_STANDARDLVL2);
         WrappingCompactionStrategy strategy = ((WrappingCompactionStrategy) store.getCompactionStrategy());
-        LeveledCompactionStrategy lcs = (LeveledCompactionStrategy) strategy.getWrappedStrategies().get(1);
+        final LeveledCompactionStrategy lcs = (LeveledCompactionStrategy) strategy.getWrappedStrategies().get(1);
 
-        ByteBuffer value = ByteBuffer.wrap(new byte[10 * 1024]); // 10 KB value
+        ByteBuffer value = ByteBuffer.wrap(new byte[100 * 1024]); // 100 KB value, make it easy to have multiple files
 
+        // Enough data to have a level 1 and 2
+        int rows = 128;
+        int columns = 10;
+
+        // Adds enough data to trigger multiple sstable per level
+        for (int r = 0; r < rows; r++)
+        {
+            DecoratedKey key = Util.dk(String.valueOf(r));
+            Mutation rm = new Mutation(KEYSPACE1, key.getKey());
+            for (int c = 0; c < columns; c++)
+            {
+                rm.add(CF_STANDARDLVL2, Util.cellname("column" + c), value, 0);
+            }
+            rm.apply();
+            store.forceBlockingFlush();
+        }
+
+        value = ByteBuffer.wrap(new byte[10 * 1024]); // 10 KB value
+        LeveledCompactionStrategyTest.waitForLeveling(store);
+        // wait for higher-level compactions to finish
+        store.disableAutoCompaction();
         // Adds 10 partitions
         for (int r = 0; r < 10; r++)
         {
             DecoratedKey key = Util.dk(String.valueOf(r));
-            Mutation rm = new Mutation(ksname, key.getKey());
+            Mutation rm = new Mutation(KEYSPACE1, key.getKey());
             for (int c = 0; c < 10; c++)
             {
-                rm.add(cfname, Util.cellname("column" + c), value, 0);
+                rm.add(CF_STANDARDLVL2, Util.cellname("column" + c), value, 0);
             }
             rm.apply();
         }
@@ -259,33 +192,40 @@
         //Flush sstable
         store.forceBlockingFlush();
 
-        Collection<SSTableReader> allSSTables = store.getSSTables();
-        for (SSTableReader sstable : allSSTables)
+        store.runWithCompactionsDisabled(new Callable<Void>()
         {
-            if (sstable.getSSTableLevel() == 0)
+            public Void call() throws Exception
             {
-                System.out.println("Mutating L0-SSTABLE level to L1 to simulate a bug: " + sstable.getFilename());
-                sstable.descriptor.getMetadataSerializer().mutateLevel(sstable.descriptor, 1);
-                sstable.reloadSSTableMetadata();
-            }
-        }
-
-        try (AbstractCompactionStrategy.ScannerList scannerList = lcs.getScanners(allSSTables))
-        {
-            //Verify that leveled scanners will always iterate in ascending order (CASSANDRA-9935)
-            for (ISSTableScanner scanner : scannerList.scanners)
-            {
-                DecoratedKey lastKey = null;
-                while (scanner.hasNext())
+                Collection<SSTableReader> allSSTables = store.getSSTables();
+                for (SSTableReader sstable : allSSTables)
                 {
-                    OnDiskAtomIterator row = scanner.next();
-                    if (lastKey != null)
+                    if (sstable.getSSTableLevel() == 0)
                     {
-                        assertTrue("row " + row.getKey() + " received out of order wrt " + lastKey, row.getKey().compareTo(lastKey) >= 0);
+                        System.out.println("Mutating L0-SSTABLE level to L1 to simulate a bug: " + sstable.getFilename());
+                        sstable.descriptor.getMetadataSerializer().mutateLevel(sstable.descriptor, 1);
+                        sstable.reloadSSTableMetadata();
                     }
-                    lastKey = row.getKey();
                 }
+
+                try (AbstractCompactionStrategy.ScannerList scannerList = lcs.getScanners(allSSTables))
+                {
+                    //Verify that leveled scanners will always iterate in ascending order (CASSANDRA-9935)
+                    for (ISSTableScanner scanner : scannerList.scanners)
+                    {
+                        DecoratedKey lastKey = null;
+                        while (scanner.hasNext())
+                        {
+                            OnDiskAtomIterator row = scanner.next();
+                            if (lastKey != null)
+                            {
+                                assertTrue("row " + row.getKey() + " received out of order wrt " + lastKey, row.getKey().compareTo(lastKey) >= 0);
+                            }
+                            lastKey = row.getKey();
+                        }
+                    }
+                }
+                return null;
             }
-        }
+        }, true);
     }
 }
diff --git a/test/long/org/apache/cassandra/io/compress/CompressorPerformance.java b/test/long/org/apache/cassandra/io/compress/CompressorPerformance.java
new file mode 100644
index 0000000..3612412
--- /dev/null
+++ b/test/long/org/apache/cassandra/io/compress/CompressorPerformance.java
@@ -0,0 +1,119 @@
+/*
+ *
+ * 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.io.compress;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ThreadLocalRandom;
+
+public class CompressorPerformance
+{
+
+    static public void testPerformances() throws IOException
+    {
+        for (ICompressor compressor: new ICompressor[] {
+                SnappyCompressor.instance,  // warm up
+                DeflateCompressor.instance,
+                LZ4Compressor.instance,
+                SnappyCompressor.instance
+        })
+        {
+            for (BufferType in: BufferType.values())
+            {
+                if (compressor.supports(in))
+                {
+                    for (BufferType out: BufferType.values())
+                    {
+                        if (compressor.supports(out))
+                        {
+                            for (int i=0; i<10; ++i)
+                                testPerformance(compressor, in, out);
+                            System.out.println();
+                        }
+                    }
+                }
+            }
+        }
+    }
+
+    static ByteBuffer dataSource;
+    static int bufLen;
+
+    static private void testPerformance(ICompressor compressor, BufferType in, BufferType out) throws IOException
+    {
+        int len = dataSource.capacity();
+        int bufLen = compressor.initialCompressedBufferLength(len);
+        ByteBuffer input = in.allocate(bufLen);
+        ByteBuffer output = out.allocate(bufLen);
+
+        int checksum = 0;
+        int count = 100;
+
+        long time = System.nanoTime();
+        for (int i=0; i<count; ++i)
+        {
+            output.clear();
+            compressor.compress(dataSource, output);
+            // Make sure not optimized away.
+            checksum += output.get(ThreadLocalRandom.current().nextInt(output.position()));
+            dataSource.rewind();
+        }
+        long timec = System.nanoTime() - time;
+        output.flip();
+        input.put(output);
+        input.flip();
+
+        time = System.nanoTime();
+        for (int i=0; i<count; ++i)
+        {
+            output.clear();
+            compressor.uncompress(input, output);
+            // Make sure not optimized away.
+            checksum += output.get(ThreadLocalRandom.current().nextInt(output.position()));
+            input.rewind();
+        }
+        long timed = System.nanoTime() - time;
+        System.out.format("Compressor %s %s->%s compress %.3f ns/b %.3f mb/s uncompress %.3f ns/b %.3f mb/s.%s\n",
+                          compressor.getClass().getSimpleName(),
+                          in,
+                          out,
+                          1.0 * timec / (count * len),
+                          Math.scalb(1.0e9, -20) * count * len / timec,
+                          1.0 * timed / (count * len),
+                          Math.scalb(1.0e9, -20) * count * len / timed,
+                          checksum == 0 ? " " : "");
+    }
+
+    public static void main(String[] args) throws IOException
+    {
+        try (FileInputStream fis = new FileInputStream("CHANGES.txt"))
+        {
+            int len = (int)fis.getChannel().size();
+            dataSource = ByteBuffer.allocateDirect(len);
+            while (dataSource.hasRemaining()) {
+                fis.getChannel().read(dataSource);
+            }
+            dataSource.flip();
+        }
+        testPerformances();
+    }
+}
diff --git a/test/long/org/apache/cassandra/io/sstable/CQLSSTableWriterLongTest.java b/test/long/org/apache/cassandra/io/sstable/CQLSSTableWriterLongTest.java
index fcec40d..ee719d1 100644
--- a/test/long/org/apache/cassandra/io/sstable/CQLSSTableWriterLongTest.java
+++ b/test/long/org/apache/cassandra/io/sstable/CQLSSTableWriterLongTest.java
@@ -30,6 +30,7 @@
 import org.junit.Test;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.service.StorageService;
@@ -39,6 +40,7 @@
     @BeforeClass
     public static void setup() throws Exception
     {
+        DatabaseDescriptor.setDaemonInitialized();
         SchemaLoader.cleanupAndLeaveDirs();
         Keyspace.setInitialized();
         StorageService.instance.initServer();
diff --git a/test/long/org/apache/cassandra/locator/DynamicEndpointSnitchLongTest.java b/test/long/org/apache/cassandra/locator/DynamicEndpointSnitchLongTest.java
index 1c628fa..841f73e 100644
--- a/test/long/org/apache/cassandra/locator/DynamicEndpointSnitchLongTest.java
+++ b/test/long/org/apache/cassandra/locator/DynamicEndpointSnitchLongTest.java
@@ -46,7 +46,7 @@
             final int ITERATIONS = 10000;
 
             // do this because SS needs to be initialized before DES can work properly.
-            StorageService.instance.initClient(0);
+            StorageService.instance.unsafeInitialize();
             SimpleSnitch ss = new SimpleSnitch();
             DynamicEndpointSnitch dsnitch = new DynamicEndpointSnitch(ss, String.valueOf(ss.hashCode()));
             InetAddress self = FBUtilities.getBroadcastAddress();
diff --git a/test/microbench/org/apache/cassandra/test/microbench/OutputStreamBench.java b/test/microbench/org/apache/cassandra/test/microbench/OutputStreamBench.java
new file mode 100644
index 0000000..7a159a2
--- /dev/null
+++ b/test/microbench/org/apache/cassandra/test/microbench/OutputStreamBench.java
@@ -0,0 +1,291 @@
+/**
+ * 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.test.microbench;
+
+import org.apache.cassandra.io.util.BufferedDataOutputStreamPlus;
+import org.apache.cassandra.io.util.BufferedDataOutputStreamTest;
+import org.apache.cassandra.io.util.WrappedDataOutputStreamPlus;
+import org.openjdk.jmh.annotations.*;
+import org.openjdk.jmh.infra.Blackhole;
+
+import java.io.BufferedOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.nio.channels.WritableByteChannel;
+import java.util.concurrent.TimeUnit;
+
+@BenchmarkMode(Mode.AverageTime)
+@OutputTimeUnit(TimeUnit.NANOSECONDS)
+@Warmup(iterations = 5, time = 1, timeUnit = TimeUnit.SECONDS)
+@Measurement(iterations = 5, time = 1, timeUnit = TimeUnit.SECONDS)
+@Fork(value = 3,jvmArgsAppend = "-Xmx512M")
+@Threads(1)
+@State(Scope.Benchmark)
+public class OutputStreamBench
+{
+
+    BufferedOutputStream hole;
+
+    WrappedDataOutputStreamPlus streamA;
+
+    BufferedDataOutputStreamPlus streamB;
+
+    byte foo;
+
+    int foo1;
+
+    long foo2;
+
+    double foo3;
+
+    float foo4;
+
+    short foo5;
+
+    char foo6;
+
+
+    String tinyM = BufferedDataOutputStreamTest.fourByte;
+    String smallM;
+    String largeM;
+    String tiny = "a";
+    String small = "adsjglhnafsjk;gujfakyhgukafshgjkahfsgjkhafs;jkhausjkgaksfj;gafskdghajfsk;g";
+    String large;
+
+    @Setup
+    public void setUp(final Blackhole bh) {
+        StringBuilder sb = new StringBuilder();
+        for (int ii = 0; ii < 11; ii++) {
+            sb.append(BufferedDataOutputStreamTest.fourByte);
+            sb.append(BufferedDataOutputStreamTest.threeByte);
+            sb.append(BufferedDataOutputStreamTest.twoByte);
+        }
+        smallM = sb.toString();
+            
+        sb = new StringBuilder();
+        while (sb.length() < 1024 * 12) {
+            sb.append(small);
+        }
+        large = sb.toString();
+
+        sb = new StringBuilder();
+        while (sb.length() < 1024 * 12) {
+            sb.append(smallM);
+        }
+        largeM = sb.toString();
+
+        hole = new BufferedOutputStream(new OutputStream() {
+
+            @Override
+            public void write(int b) throws IOException
+            {
+                bh.consume(b);
+            }
+
+            @Override
+            public void write(byte b[]) throws IOException {
+                bh.consume(b);
+            }
+
+            @Override
+            public void write(byte b[], int a, int c) throws IOException {
+                bh.consume(b);
+                bh.consume(a);
+                bh.consume(c);
+            }
+            });
+
+        streamA = new WrappedDataOutputStreamPlus(hole);
+
+        streamB = new BufferedDataOutputStreamPlus(new WritableByteChannel() {
+
+            @Override
+            public boolean isOpen()
+            {
+                return true;
+            }
+
+            @Override
+            public void close() throws IOException
+            {
+            }
+
+            @Override
+            public int write(ByteBuffer src) throws IOException
+            {
+                bh.consume(src);
+                int remaining = src.remaining();
+                src.position(src.limit());
+                return remaining;
+            }
+
+        }, 8192);
+    }
+
+    @Benchmark
+    public void testBOSByte() throws IOException
+    {
+        streamA.write(foo);
+    }
+
+    @Benchmark
+    public void testBDOSPByte() throws IOException
+    {
+        streamB.write(foo);
+    }
+
+    @Benchmark
+    public void testBOSInt() throws IOException
+    {
+        streamA.writeInt(foo1);
+    }
+
+    @Benchmark
+    public void testBDOSPInt() throws IOException
+    {
+        streamB.writeInt(foo1);
+    }
+
+    @Benchmark
+    public void testBOSLong() throws IOException
+    {
+        streamA.writeLong(foo2);
+    }
+
+    @Benchmark
+    public void testBDOSPLong() throws IOException
+    {
+        streamB.writeLong(foo2);
+    }
+
+    @Benchmark
+    public void testBOSMixed() throws IOException
+    {
+        streamA.write(foo);
+        streamA.writeInt(foo1);
+        streamA.writeLong(foo2);
+        streamA.writeDouble(foo3);
+        streamA.writeFloat(foo4);
+        streamA.writeShort(foo5);
+        streamA.writeChar(foo6);
+    }
+
+    @Benchmark
+    public void testBDOSPMixed() throws IOException
+    {
+        streamB.write(foo);
+        streamB.writeInt(foo1);
+        streamB.writeLong(foo2);
+        streamB.writeDouble(foo3);
+        streamB.writeFloat(foo4);
+        streamB.writeShort(foo5);
+        streamB.writeChar(foo6);
+    }
+
+    @Benchmark
+    public void testMTinyStringBOS() throws IOException {
+        streamA.writeUTF(tinyM);
+    }
+
+    @Benchmark
+    public void testMTinyStringBDOSP() throws IOException {
+        streamB.writeUTF(tinyM);
+    }
+
+    @Benchmark
+    public void testMTinyLegacyWriteUTF() throws IOException {
+        BufferedDataOutputStreamTest.writeUTFLegacy(tinyM, hole);
+    }
+
+    @Benchmark
+    public void testMSmallStringBOS() throws IOException {
+        streamA.writeUTF(smallM);
+    }
+
+    @Benchmark
+    public void testMSmallStringBDOSP() throws IOException {
+        streamB.writeUTF(smallM);
+    }
+
+    @Benchmark
+    public void testMSmallLegacyWriteUTF() throws IOException {
+        BufferedDataOutputStreamTest.writeUTFLegacy(smallM, hole);
+    }
+
+    @Benchmark
+    public void testMLargeStringBOS() throws IOException {
+        streamA.writeUTF(largeM);
+    }
+
+    @Benchmark
+    public void testMLargeStringBDOSP() throws IOException {
+        streamB.writeUTF(largeM);
+    }
+
+    @Benchmark
+    public void testMLargeLegacyWriteUTF() throws IOException {
+        BufferedDataOutputStreamTest.writeUTFLegacy(largeM, hole);
+    }
+
+    @Benchmark
+    public void testTinyStringBOS() throws IOException {
+        streamA.writeUTF(tiny);
+    }
+
+    @Benchmark
+    public void testTinyStringBDOSP() throws IOException {
+        streamB.writeUTF(tiny);
+    }
+
+    @Benchmark
+    public void testTinyLegacyWriteUTF() throws IOException {
+        BufferedDataOutputStreamTest.writeUTFLegacy(tiny, hole);
+    }
+
+    @Benchmark
+    public void testSmallStringBOS() throws IOException {
+        streamA.writeUTF(small);
+    }
+
+    @Benchmark
+    public void testSmallStringBDOSP() throws IOException {
+        streamB.writeUTF(small);
+    }
+
+    @Benchmark
+    public void testSmallLegacyWriteUTF() throws IOException {
+        BufferedDataOutputStreamTest.writeUTFLegacy(small, hole);
+    }
+
+    @Benchmark
+    public void testRLargeStringBOS() throws IOException {
+        streamA.writeUTF(large);
+    }
+
+    @Benchmark
+    public void testRLargeStringBDOSP() throws IOException {
+        streamB.writeUTF(large);
+    }
+
+    @Benchmark
+    public void testRLargeLegacyWriteUTF() throws IOException {
+        BufferedDataOutputStreamTest.writeUTFLegacy(large, hole);
+    }
+}
\ No newline at end of file
diff --git a/test/microbench/org/apache/cassandra/test/microbench/PendingRangesBench.java b/test/microbench/org/apache/cassandra/test/microbench/PendingRangesBench.java
new file mode 100644
index 0000000..9ec1aa6
--- /dev/null
+++ b/test/microbench/org/apache/cassandra/test/microbench/PendingRangesBench.java
@@ -0,0 +1,109 @@
+/*
+ *
+ * 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.test.microbench;
+
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimap;
+import org.apache.cassandra.dht.RandomPartitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.PendingRangeMaps;
+import org.openjdk.jmh.annotations.*;
+import org.openjdk.jmh.infra.Blackhole;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+
+@BenchmarkMode(Mode.AverageTime)
+@OutputTimeUnit(TimeUnit.NANOSECONDS)
+@Warmup(iterations = 1, time = 1, timeUnit = TimeUnit.SECONDS)
+@Measurement(iterations = 50, time = 1, timeUnit = TimeUnit.SECONDS)
+@Fork(value = 3,jvmArgsAppend = "-Xmx512M")
+@Threads(1)
+@State(Scope.Benchmark)
+public class PendingRangesBench
+{
+    PendingRangeMaps pendingRangeMaps;
+    int maxToken = 256 * 100;
+
+    Multimap<Range<Token>, InetAddress> oldPendingRanges;
+
+    private Range<Token> genRange(String left, String right)
+    {
+        return new Range<Token>(new RandomPartitioner.BigIntegerToken(left), new RandomPartitioner.BigIntegerToken(right));
+    }
+
+    @Setup
+    public void setUp() throws UnknownHostException
+    {
+        pendingRangeMaps = new PendingRangeMaps();
+        oldPendingRanges = HashMultimap.create();
+
+        InetAddress[] addresses = {InetAddress.getByName("127.0.0.1"), InetAddress.getByName("127.0.0.2")};
+
+        for (int i = 0; i < maxToken; i++)
+        {
+            for (int j = 0; j < ThreadLocalRandom.current().nextInt(2); j ++)
+            {
+                Range<Token> range = genRange(Integer.toString(i * 10 + 5), Integer.toString(i * 10 + 15));
+                pendingRangeMaps.addPendingRange(range, addresses[j]);
+                oldPendingRanges.put(range, addresses[j]);
+            }
+        }
+
+        // add the wrap around range
+        for (int j = 0; j < ThreadLocalRandom.current().nextInt(2); j ++)
+        {
+            Range<Token> range = genRange(Integer.toString(maxToken * 10 + 5), Integer.toString(5));
+            pendingRangeMaps.addPendingRange(range, addresses[j]);
+            oldPendingRanges.put(range, addresses[j]);
+        }
+    }
+
+    @Benchmark
+    public void searchToken(final Blackhole bh)
+    {
+        int randomToken = ThreadLocalRandom.current().nextInt(maxToken * 10 + 5);
+        Token searchToken = new RandomPartitioner.BigIntegerToken(Integer.toString(randomToken));
+        bh.consume(pendingRangeMaps.pendingEndpointsFor(searchToken));
+    }
+
+    @Benchmark
+    public void searchTokenForOldPendingRanges(final Blackhole bh)
+    {
+        int randomToken = ThreadLocalRandom.current().nextInt(maxToken * 10 + 5);
+        Token searchToken = new RandomPartitioner.BigIntegerToken(Integer.toString(randomToken));
+        Set<InetAddress> endpoints = new HashSet<>();
+        for (Map.Entry<Range<Token>, Collection<InetAddress>> entry : oldPendingRanges.asMap().entrySet())
+        {
+            if (entry.getKey().contains(searchToken))
+                endpoints.addAll(entry.getValue());
+        }
+        bh.consume(endpoints);
+    }
+
+}
diff --git a/test/microbench/org/apache/cassandra/test/microbench/Sample.java b/test/microbench/org/apache/cassandra/test/microbench/Sample.java
new file mode 100644
index 0000000..1f149c0
--- /dev/null
+++ b/test/microbench/org/apache/cassandra/test/microbench/Sample.java
@@ -0,0 +1,130 @@
+/**
+ * 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.test.microbench;
+
+import net.jpountz.lz4.LZ4Compressor;
+import net.jpountz.lz4.LZ4Factory;
+import net.jpountz.lz4.LZ4FastDecompressor;
+import org.openjdk.jmh.annotations.*;
+import org.xerial.snappy.Snappy;
+
+import java.io.IOException;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+
+@BenchmarkMode(Mode.Throughput)
+@OutputTimeUnit(TimeUnit.MILLISECONDS)
+@Warmup(iterations = 5, time = 1, timeUnit = TimeUnit.SECONDS)
+@Measurement(iterations = 5, time = 2, timeUnit = TimeUnit.SECONDS)
+@Fork(value = 1,jvmArgsAppend = "-Xmx512M")
+@Threads(1)
+@State(Scope.Benchmark)
+public class Sample
+{
+    @Param({"65536"})
+    private int pageSize;
+
+    @Param({"1024"})
+    private int uniquePages;
+
+    @Param({"0.1"})
+    private double randomRatio;
+
+    @Param({"4..16"})
+    private String randomRunLength;
+
+    @Param({"4..128"})
+    private String duplicateLookback;
+
+    private byte[][] lz4Bytes;
+    private byte[][] snappyBytes;
+    private byte[][] rawBytes;
+
+    private LZ4FastDecompressor lz4Decompressor = LZ4Factory.fastestInstance().fastDecompressor();
+
+    private LZ4Compressor lz4Compressor = LZ4Factory.fastestInstance().fastCompressor();
+
+    @State(Scope.Thread)
+    public static class ThreadState
+    {
+        byte[] bytes;
+    }
+
+    @Setup
+    public void setup() throws IOException
+    {
+        ThreadLocalRandom random = ThreadLocalRandom.current();
+        int[] randomRunLength = range(this.randomRunLength);
+        int[] duplicateLookback = range(this.duplicateLookback);
+        rawBytes = new byte[uniquePages][pageSize];
+        lz4Bytes = new byte[uniquePages][];
+        snappyBytes = new byte[uniquePages][];
+        byte[][] runs = new byte[duplicateLookback[1] - duplicateLookback[0]][];
+        for (int i = 0 ; i < rawBytes.length ; i++)
+        {
+            byte[] trg = rawBytes[0];
+            int runCount = 0;
+            int byteCount = 0;
+            while (byteCount < trg.length)
+            {
+                byte[] nextRun;
+                if (runCount == 0 || random.nextDouble() < this.randomRatio)
+                {
+                    nextRun = new byte[random.nextInt(randomRunLength[0], randomRunLength[1])];
+                    random.nextBytes(nextRun );
+                    runs[runCount % runs.length] = nextRun;
+                    runCount++;
+                }
+                else
+                {
+                    int index = runCount < duplicateLookback[1]
+                            ? random.nextInt(runCount)
+                            : (runCount - random.nextInt(duplicateLookback[0], duplicateLookback[1]));
+                    nextRun = runs[index % runs.length];
+                }
+                System.arraycopy(nextRun, 0, trg, byteCount, Math.min(nextRun.length, trg.length - byteCount));
+                byteCount += nextRun.length;
+            }
+            lz4Bytes[i] = lz4Compressor.compress(trg);
+            snappyBytes[i] = Snappy.compress(trg);
+        }
+    }
+
+    static int[] range(String spec)
+    {
+        String[] split = spec.split("\\.\\.");
+        return new int[] { Integer.parseInt(split[0]), Integer.parseInt(split[1]) };
+    }
+
+    @Benchmark
+    public void lz4(ThreadState state)
+    {
+        if (state.bytes == null)
+            state.bytes = new byte[this.pageSize];
+        byte[] in = lz4Bytes[ThreadLocalRandom.current().nextInt(lz4Bytes.length)];
+        lz4Decompressor.decompress(in, state.bytes);
+    }
+
+    @Benchmark
+    public void snappy(ThreadState state) throws IOException
+    {
+        byte[] in = snappyBytes[ThreadLocalRandom.current().nextInt(snappyBytes.length)];
+        state.bytes = Snappy.uncompress(in);
+    }
+}
diff --git a/test/pig/org/apache/cassandra/pig/CqlRecordReaderTest.java b/test/pig/org/apache/cassandra/pig/CqlRecordReaderTest.java
index 877d30b..b2a74b6 100644
--- a/test/pig/org/apache/cassandra/pig/CqlRecordReaderTest.java
+++ b/test/pig/org/apache/cassandra/pig/CqlRecordReaderTest.java
@@ -18,32 +18,20 @@
 package org.apache.cassandra.pig;
 
 import java.io.IOException;
-import java.nio.charset.CharacterCodingException;
 import java.util.Iterator;
 
 import org.junit.Assert;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Test;
 
-import org.apache.cassandra.db.marshal.TimeUUIDType;
-import org.apache.cassandra.db.marshal.UUIDType;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.thrift.AuthenticationException;
-import org.apache.cassandra.thrift.AuthorizationException;
-import org.apache.cassandra.thrift.InvalidRequestException;
-import org.apache.cassandra.thrift.NotFoundException;
-import org.apache.cassandra.thrift.TimedOutException;
-import org.apache.cassandra.thrift.UnavailableException;
-import org.apache.cassandra.utils.Hex;
-import org.apache.pig.data.DataByteArray;
 import org.apache.pig.data.Tuple;
 import org.apache.thrift.TException;
-import org.apache.thrift.transport.TTransportException;
 
 public class CqlRecordReaderTest extends PigTestBase
 {
     private static String[] statements = {
+        "DROP KEYSPACE IF EXISTS cql3ks",
         "CREATE KEYSPACE cql3ks WITH replication = {'class': 'SimpleStrategy', 'replication_factor' : 1};",
         "USE cql3ks;",
 
@@ -69,11 +57,10 @@
     };
 
     @BeforeClass
-    public static void setup() throws TTransportException, IOException, InterruptedException, ConfigurationException,
-    AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, CharacterCodingException, ClassNotFoundException, NoSuchFieldException, IllegalAccessException, InstantiationException
+    public static void setup() throws IOException, ConfigurationException, TException
     {
         startCassandra();
-        setupDataByCql(statements);
+        executeCQLStatements(statements);
         startHadoopCluster();
     }
 
diff --git a/test/pig/org/apache/cassandra/pig/CqlTableDataTypeTest.java b/test/pig/org/apache/cassandra/pig/CqlTableDataTypeTest.java
index bbd5a87..ca01901 100644
--- a/test/pig/org/apache/cassandra/pig/CqlTableDataTypeTest.java
+++ b/test/pig/org/apache/cassandra/pig/CqlTableDataTypeTest.java
@@ -19,24 +19,16 @@
 package org.apache.cassandra.pig;
 
 import java.io.IOException;
-import java.nio.charset.CharacterCodingException;
 import java.util.Iterator;
 
 import org.apache.cassandra.db.marshal.TimeUUIDType;
 import org.apache.cassandra.db.marshal.UUIDType;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.thrift.AuthenticationException;
-import org.apache.cassandra.thrift.AuthorizationException;
-import org.apache.cassandra.thrift.InvalidRequestException;
-import org.apache.cassandra.thrift.NotFoundException;
-import org.apache.cassandra.thrift.SchemaDisagreementException;
-import org.apache.cassandra.thrift.TimedOutException;
-import org.apache.cassandra.thrift.UnavailableException;
 import org.apache.cassandra.utils.Hex;
 import org.apache.pig.data.DataByteArray;
 import org.apache.pig.data.Tuple;
 import org.apache.thrift.TException;
-import org.apache.thrift.transport.TTransportException;
+
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -64,6 +56,7 @@
     //MAP
     //Create table to test the above data types
     private static String[] statements = {
+            "DROP KEYSPACE IF EXISTS cql3ks",
             "CREATE KEYSPACE cql3ks WITH replication = {'class': 'SimpleStrategy', 'replication_factor' : 1}",
             "USE cql3ks;",
 
@@ -208,17 +201,15 @@
     };
 
     @BeforeClass
-    public static void setup() throws TTransportException, IOException, InterruptedException, ConfigurationException,
-                                      AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, CharacterCodingException, ClassNotFoundException, NoSuchFieldException, IllegalAccessException, InstantiationException
+    public static void setup() throws IOException, ConfigurationException, TException
     {
         startCassandra();
-        setupDataByCql(statements);
+        executeCQLStatements(statements);
         startHadoopCluster();
     }
 
     @Test
-    public void testCqlNativeStorageRegularType()
-    throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, SchemaDisagreementException, IOException
+    public void testCqlNativeStorageRegularType() throws IOException
     {
         //input_cql=select * from cqltable where token(key) > ? and token(key) <= ?
         cqlTableTest("rows = LOAD 'cql://cql3ks/cqltable?" + defaultParameters + nativeParameters + "&input_cql=select%20*%20from%20cqltable%20where%20token(key)%20%3E%20%3F%20and%20token(key)%20%3C%3D%20%3F' USING CqlNativeStorage();");
@@ -288,8 +279,7 @@
     }
 
     @Test
-    public void testCqlNativeStorageSetType()
-    throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, SchemaDisagreementException, IOException
+    public void testCqlNativeStorageSetType() throws IOException
     {
         //input_cql=select * from settable where token(key) > ? and token(key) <= ?
         settableTest("set_rows = LOAD 'cql://cql3ks/settable?" + defaultParameters + nativeParameters + "&input_cql=select%20*%20from%20settable%20where%20token(key)%20%3E%20%3F%20and%20token(key)%20%3C%3D%20%3F' USING CqlNativeStorage();");
@@ -355,8 +345,7 @@
     }
 
     @Test
-    public void testCqlNativeStorageListType()
-    throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, SchemaDisagreementException, IOException
+    public void testCqlNativeStorageListType() throws IOException
     {
         //input_cql=select * from listtable where token(key) > ? and token(key) <= ?
         listtableTest("list_rows = LOAD 'cql://cql3ks/listtable?" + defaultParameters + nativeParameters + "&input_cql=select%20*%20from%20listtable%20where%20token(key)%20%3E%20%3F%20and%20token(key)%20%3C%3D%20%3F' USING CqlNativeStorage();");
@@ -422,8 +411,7 @@
     }
 
     @Test
-    public void testCqlNativeStorageMapType()
-    throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, SchemaDisagreementException, IOException
+    public void testCqlNativeStorageMapType() throws IOException
     {
         //input_cql=select * from maptable where token(key) > ? and token(key) <= ?
         maptableTest("map_rows = LOAD 'cql://cql3ks/maptable?" + defaultParameters + nativeParameters + "&input_cql=select%20*%20from%20maptable%20where%20token(key)%20%3E%20%3F%20and%20token(key)%20%3C%3D%20%3F' USING CqlNativeStorage();");
diff --git a/test/pig/org/apache/cassandra/pig/CqlTableTest.java b/test/pig/org/apache/cassandra/pig/CqlTableTest.java
index 2e1758e..3902fce 100644
--- a/test/pig/org/apache/cassandra/pig/CqlTableTest.java
+++ b/test/pig/org/apache/cassandra/pig/CqlTableTest.java
@@ -19,21 +19,11 @@
 package org.apache.cassandra.pig;
 
 import java.io.IOException;
-import java.nio.charset.CharacterCodingException;
 import java.util.Iterator;
 
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.thrift.AuthenticationException;
-import org.apache.cassandra.thrift.AuthorizationException;
-import org.apache.cassandra.thrift.InvalidRequestException;
-import org.apache.cassandra.thrift.NotFoundException;
-import org.apache.cassandra.thrift.SchemaDisagreementException;
-import org.apache.cassandra.thrift.TimedOutException;
-import org.apache.cassandra.thrift.UnavailableException;
-import org.apache.pig.data.DataBag;
 import org.apache.pig.data.Tuple;
 import org.apache.thrift.TException;
-import org.apache.thrift.transport.TTransportException;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -41,6 +31,7 @@
 public class CqlTableTest extends PigTestBase
 {    
     private static String[] statements = {
+            "DROP KEYSPACE IF EXISTS cql3ks",
             "CREATE KEYSPACE cql3ks WITH replication = {'class': 'SimpleStrategy', 'replication_factor' : 1}",
             "USE cql3ks;",
 
@@ -82,20 +73,23 @@
             "UPDATE collectiontable SET n['key2'] = 'value2' WHERE m = 'book2';",
             "UPDATE collectiontable SET n['key3'] = 'value3' WHERE m = 'book3';",
             "UPDATE collectiontable SET n['key4'] = 'value4' WHERE m = 'book4';",
+            "CREATE TABLE nulltable(m text PRIMARY KEY, n map<text, text>);",
+            "UPDATE nulltable SET n['key1'] = 'value1' WHERE m = 'book1';",
+            "UPDATE nulltable SET n['key2'] = 'value2' WHERE m = 'book2';",
+            "UPDATE nulltable SET n['key3'] = 'value3' WHERE m = 'book3';",
+            "UPDATE nulltable SET n['key4'] = 'value4' WHERE m = 'book4';",
     };
 
     @BeforeClass
-    public static void setup() throws TTransportException, IOException, InterruptedException, ConfigurationException,
-                                      AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, CharacterCodingException, ClassNotFoundException, NoSuchFieldException, IllegalAccessException, InstantiationException
+    public static void setup() throws IOException, ConfigurationException, TException
     {
         startCassandra();
-        setupDataByCql(statements);
+        executeCQLStatements(statements);
         startHadoopCluster();
     }
 
     @Test
-    public void testCqlNativeStorageSchema()
-    throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, SchemaDisagreementException, IOException
+    public void testCqlNativeStorageSchema() throws IOException
     {
         //input_cql=select * from cqltable where token(key1) > ? and token(key1) <= ?
         cqlTableSchemaTest("rows = LOAD 'cql://cql3ks/cqltable?" + defaultParameters + nativeParameters +  "&input_cql=select%20*%20from%20cqltable%20where%20token(key1)%20%3E%20%3F%20and%20token(key1)%20%3C%3D%20%3F' USING CqlNativeStorage();");
@@ -140,15 +134,13 @@
     }
 
     @Test
-    public void testCqlNativeStorageSingleKeyTable()
-    throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, SchemaDisagreementException, IOException
+    public void testCqlNativeStorageSingleKeyTable() throws IOException
     {
         //input_cql=select * from moredata where token(x) > ? and token(x) <= ?
-        SingleKeyTableTest("moretestvalues= LOAD 'cql://cql3ks/moredata?" + defaultParameters + nativeParameters + "&input_cql=select%20*%20from%20moredata%20where%20token(x)%20%3E%20%3F%20and%20token(x)%20%3C%3D%20%3F' USING CqlNativeStorage();");
+        singleKeyTableTest("moretestvalues= LOAD 'cql://cql3ks/moredata?" + defaultParameters + nativeParameters + "&input_cql=select%20*%20from%20moredata%20where%20token(x)%20%3E%20%3F%20and%20token(x)%20%3C%3D%20%3F' USING CqlNativeStorage();");
     }
 
-    private void SingleKeyTableTest(String initialQuery)
-    throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, SchemaDisagreementException, IOException
+    private void singleKeyTableTest(String initialQuery) throws IOException
     {
         pig.setBatchOn();
         pig.registerQuery(initialQuery);
@@ -174,15 +166,13 @@
     }
 
     @Test
-    public void testCqlNativeStorageCompositeKeyTable()
-    throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, SchemaDisagreementException, IOException
+    public void testCqlNativeStorageCompositeKeyTable() throws IOException
     {
         //input_cql=select * from compmore where token(id) > ? and token(id) <= ?
-        CompositeKeyTableTest("moredata= LOAD 'cql://cql3ks/compmore?" + defaultParameters + nativeParameters + "&input_cql=select%20*%20from%20compmore%20where%20token(id)%20%3E%20%3F%20and%20token(id)%20%3C%3D%20%3F' USING CqlNativeStorage();");
+        compositeKeyTableTest("moredata= LOAD 'cql://cql3ks/compmore?" + defaultParameters + nativeParameters + "&input_cql=select%20*%20from%20compmore%20where%20token(id)%20%3E%20%3F%20and%20token(id)%20%3C%3D%20%3F' USING CqlNativeStorage();");
     }
 
-    private void CompositeKeyTableTest(String initialQuery)
-    throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, SchemaDisagreementException, IOException
+    private void compositeKeyTableTest(String initialQuery) throws IOException
     {
         pig.setBatchOn();
         pig.registerQuery(initialQuery);
@@ -211,15 +201,13 @@
     }
 
     @Test
-    public void testCqlNativeStorageCollectionColumnTable()
-    throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, SchemaDisagreementException, IOException
+    public void testCqlNativeStorageCollectionColumnTable() throws IOException
     {
         //input_cql=select * from collectiontable where token(m) > ? and token(m) <= ?
         CollectionColumnTableTest("collectiontable= LOAD 'cql://cql3ks/collectiontable?" + defaultParameters + nativeParameters + "&input_cql=select%20*%20from%20collectiontable%20where%20token(m)%20%3E%20%3F%20and%20token(m)%20%3C%3D%20%3F' USING CqlNativeStorage();");
     }
 
-    private void CollectionColumnTableTest(String initialQuery)
-    throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, SchemaDisagreementException, IOException
+    private void CollectionColumnTableTest(String initialQuery) throws IOException
     {
         pig.setBatchOn();
         pig.registerQuery(initialQuery);
@@ -252,65 +240,32 @@
     }
 
     @Test
-    public void testCassandraStorageSchema() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException
+    public void testCqlNativeStorageNullTuples() throws IOException
     {
-        //results: (key1,{((111,),),((111,column1),100),((111,column2),10.1)})
-        pig.registerQuery("rows = LOAD 'cassandra://cql3ks/cqltable?" + defaultParameters + "' USING CassandraStorage();");
+        //input_cql=select * from collectiontable where token(m) > ? and token(m) <= ?
+        NullTupleTest("nulltable= LOAD 'cql://cql3ks/collectiontable?" + defaultParameters + nativeParameters + "&input_cql=select%20*%20from%20nulltable%20where%20token(m)%20%3E%20%3F%20and%20token(m)%20%3C%3D%20%3F' USING CqlNativeStorage();");
+    }
 
-        //schema: {key: chararray,columns: {(name: (),value: bytearray)}}
-        Iterator<Tuple> it = pig.openIterator("rows");
+    private void NullTupleTest(String initialQuery) throws IOException
+    {
+        pig.setBatchOn();
+        pig.registerQuery(initialQuery);
+        pig.registerQuery("recs= FOREACH nulltable GENERATE TOTUPLE(TOTUPLE('m', m) ), TOTUPLE(TOTUPLE('map', TOTUPLE('m', null), TOTUPLE('n', null)));");
+        pig.registerQuery("STORE recs INTO 'cql://cql3ks/nulltable?" + defaultParameters + nativeParameters + "&output_query=update+cql3ks.nulltable+set+n+%3D+%3F' USING CqlNativeStorage();");
+        pig.executeBatch();
+
+        pig.registerQuery("result= LOAD 'cql://cql3ks/nulltable?" + defaultParameters + nativeParameters + "&input_cql=select%20*%20from%20nulltable%20where%20token(m)%20%3E%20%3F%20and%20token(m)%20%3C%3D%20%3F' USING CqlNativeStorage();");
+        Iterator<Tuple> it = pig.openIterator("result");
         if (it.hasNext()) {
             Tuple t = it.next();
-            String rowKey =  t.get(0).toString();
-            Assert.assertEquals(rowKey, "key1");
-            DataBag columns = (DataBag) t.get(1);
-            Iterator<Tuple> iter = columns.iterator();
-            int i = 0;
-            while(iter.hasNext())
-            {
-                i++;
-                Tuple column = (Tuple) iter.next();
-                if (i==1)
-                {
-                    Assert.assertEquals(((Tuple) column.get(0)).get(0), 111);
-                    Assert.assertEquals(((Tuple) column.get(0)).get(1), "");
-                    Assert.assertEquals(column.get(1).toString(), "");
-                }
-                if (i==2)
-                {
-                    Assert.assertEquals(((Tuple) column.get(0)).get(0), 111);
-                    Assert.assertEquals(((Tuple) column.get(0)).get(1), "column1");
-                    Assert.assertEquals(column.get(1), 100);
-                }
-                if (i==3)
-                {
-                    Assert.assertEquals(((Tuple) column.get(0)).get(0), 111);
-                    Assert.assertEquals(((Tuple) column.get(0)).get(1), "column2");
-                    Assert.assertEquals(column.get(1), 10.1f);
-                }
-            }
-            Assert.assertEquals(3, columns.size());
-        }
-        else
-        {
-            Assert.fail("Can't fetch any data");
-        }
-
-        //results: (key1,(column1,100),(column2,10.1))
-        pig.registerQuery("compact_rows = LOAD 'cassandra://cql3ks/compactcqltable?" + defaultParameters + "' USING CassandraStorage();");
-
-        //schema: {key: chararray,column1: (name: chararray,value: int),column2: (name: chararray,value: float)}
-        it = pig.openIterator("compact_rows");
-        if (it.hasNext()) {
-            Tuple t = it.next();
-            String rowKey =  t.get(0).toString();
-            Assert.assertEquals(rowKey, "key1");
-            Tuple column = (Tuple) t.get(1);
-            Assert.assertEquals(column.get(0), "column1");
-            Assert.assertEquals(column.get(1), 100);
-            column = (Tuple) t.get(2);
-            Assert.assertEquals(column.get(0), "column2");
-            Assert.assertEquals(column.get(1), 10.1f);
+            Tuple t1 = (Tuple) t.get(1);
+            Assert.assertEquals(t1.size(), 2);
+            Tuple element1 = (Tuple) t1.get(0);
+            Tuple element2 = (Tuple) t1.get(1);
+            Assert.assertEquals(element1.get(0), "m");
+            Assert.assertEquals(element1.get(1), "");
+            Assert.assertEquals(element2.get(0), "n");
+            Assert.assertEquals(element2.get(1), "");
         }
         else
         {
@@ -319,8 +274,7 @@
     }
 
     @Test
-    public void testCqlStorageSingleKeyTableBulkLoad()
-    throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, SchemaDisagreementException, IOException
+    public void testCqlStorageSingleKeyTableBulkLoad() throws TException, IOException
     {
         pig.setBatchOn();
         //input_cql=select * from moredata where token(x) > ? and token(x) <= ?
diff --git a/test/pig/org/apache/cassandra/pig/PigTestBase.java b/test/pig/org/apache/cassandra/pig/PigTestBase.java
index e6964f8..a8a9de5 100644
--- a/test/pig/org/apache/cassandra/pig/PigTestBase.java
+++ b/test/pig/org/apache/cassandra/pig/PigTestBase.java
@@ -18,14 +18,9 @@
  */
 package org.apache.cassandra.pig;
 
-import java.io.ByteArrayOutputStream;
 import java.io.IOException;
-import java.io.PrintStream;
-import java.nio.charset.CharacterCodingException;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.cli.CliMain;
-import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.TypeParser;
@@ -35,11 +30,6 @@
 import org.apache.cassandra.thrift.Cassandra;
 import org.apache.cassandra.thrift.Compression;
 import org.apache.cassandra.thrift.ConsistencyLevel;
-import org.apache.cassandra.thrift.InvalidRequestException;
-import org.apache.cassandra.thrift.NotFoundException;
-import org.apache.cassandra.thrift.SchemaDisagreementException;
-import org.apache.cassandra.thrift.TimedOutException;
-import org.apache.cassandra.thrift.UnavailableException;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.pig.ExecType;
@@ -56,7 +46,6 @@
 import org.apache.thrift.transport.TTransportException;
 import org.junit.After;
 import org.junit.AfterClass;
-import org.junit.Assert;
 import org.junit.Before;
 
 public class PigTestBase extends SchemaLoader
@@ -65,8 +54,7 @@
     protected static Configuration conf;
     protected static MiniCluster cluster; 
     protected static PigServer pig;
-    protected static String defaultParameters= "init_address=localhost&rpc_port=9170&partitioner=org.apache.cassandra.dht.ByteOrderedPartitioner" +
-                                               "&storage_port=7010&ssl_storage_port=7011&internode_encrypt=NONE";
+    protected static String defaultParameters= "init_address=localhost&rpc_port=9170&partitioner=org.apache.cassandra.dht.Murmur3Partitioner";
     protected static String nativeParameters = "&core_conns=2&max_conns=10&min_simult_reqs=3&max_simult_reqs=10&native_timeout=10000000"  +
                                                "&native_read_timeout=10000000&send_buff_size=4096&receive_buff_size=4096&solinger=3" +
                                                "&tcp_nodelay=true&reuse_address=true&keep_alive=true&native_port=9042";
@@ -74,6 +62,7 @@
     static
     {
         System.setProperty("logback.configurationFile", "logback-test.xml");
+        System.setProperty("cassandra.config", "cassandra_pig.yaml");
     }
 
     @AfterClass
@@ -120,75 +109,20 @@
         {
             return TypeParser.parse(type);
         }
-        catch (ConfigurationException e)
-        {
-            throw new IOException(e);
-        }
-        catch (SyntaxException e)
+        catch (ConfigurationException | SyntaxException e)
         {
             throw new IOException(e);
         }
     }
 
-    protected static void setupDataByCli(String[] statements) throws CharacterCodingException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException
-    {
-        // new error/output streams for CliSessionState
-        ByteArrayOutputStream errStream = new ByteArrayOutputStream();
-        ByteArrayOutputStream outStream = new ByteArrayOutputStream();
-
-        // checking if we can connect to the running cassandra node on localhost
-        CliMain.connect("127.0.0.1", 9170);
-
-        // setting new output stream
-        CliMain.sessionState.setOut(new PrintStream(outStream));
-        CliMain.sessionState.setErr(new PrintStream(errStream));
-
-        // re-creating keyspace for tests
-        try
-        {
-            // dropping in case it exists e.g. could be left from previous run
-            CliMain.processStatement("drop keyspace thriftKs;");
-        }
-        catch (Exception e)
-        {
-        }
-
-        for (String statement : statements)
-        {
-            errStream.reset();
-            System.out.println("Executing statement: " + statement);
-            CliMain.processStatement(statement);
-            String result = outStream.toString();
-            System.out.println("result: " + result);
-            outStream.reset(); // reset stream so we have only output from next statement all the time
-            errStream.reset(); // no errors to the end user.
-        }
-    }
-    
-    protected static void setupDataByCql(String[] statements) throws InvalidRequestException, UnavailableException, TimedOutException, TException
+    protected static void executeCQLStatements(String[] statements) throws TException
     {
         Cassandra.Client client = getClient();
-        // re-creating keyspace for tests
-        try
-        {
-            // dropping in case it exists e.g. could be left from previous run
-            client.execute_cql3_query(ByteBufferUtil.bytes("DROP KEYSPACE cql3ks"), Compression.NONE, ConsistencyLevel.ONE);
-        }
-        catch (Exception e)
-        {
-        }
 
         for (String statement : statements)
         {
-            try
-            {
-                System.out.println("Executing statement: " + statement);
-                client.execute_cql3_query(ByteBufferUtil.bytes(statement), Compression.NONE, ConsistencyLevel.ONE);
-            }
-            catch (SchemaDisagreementException e)
-            {
-                Assert.fail();
-            }
+            System.out.println("Executing statement: " + statement);
+            client.execute_cql3_query(ByteBufferUtil.bytes(statement), Compression.NONE, ConsistencyLevel.ONE);
         }
     }
 }
diff --git a/test/pig/org/apache/cassandra/pig/ThriftColumnFamilyDataTypeTest.java b/test/pig/org/apache/cassandra/pig/ThriftColumnFamilyDataTypeTest.java
index cc54620..3ddb94e 100644
--- a/test/pig/org/apache/cassandra/pig/ThriftColumnFamilyDataTypeTest.java
+++ b/test/pig/org/apache/cassandra/pig/ThriftColumnFamilyDataTypeTest.java
@@ -19,181 +19,136 @@
 package org.apache.cassandra.pig;
 
 import java.io.IOException;
-import java.nio.charset.CharacterCodingException;
-import java.util.Iterator;
 
 import org.apache.cassandra.db.marshal.TimeUUIDType;
 import org.apache.cassandra.db.marshal.UUIDType;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.thrift.AuthenticationException;
-import org.apache.cassandra.thrift.AuthorizationException;
-import org.apache.cassandra.thrift.InvalidRequestException;
-import org.apache.cassandra.thrift.NotFoundException;
-import org.apache.cassandra.thrift.TimedOutException;
-import org.apache.cassandra.thrift.UnavailableException;
 import org.apache.cassandra.utils.Hex;
 import org.apache.pig.data.DataBag;
 import org.apache.pig.data.DataByteArray;
 import org.apache.pig.data.Tuple;
 import org.apache.thrift.TException;
-import org.apache.thrift.transport.TTransportException;
-import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+import static junit.framework.Assert.assertEquals;
+
 public class ThriftColumnFamilyDataTypeTest extends PigTestBase
 {
-    //AsciiType
-    //LongType
-    //BytesType
-    //BooleanType
-    //CounterColumnType
-    //DecimalType
-    //DoubleType
-    //FloatType
-    //InetAddressType
-    //Int32Type
-    //UTF8Type
-    //DateType
-    //UUIDType
-    //IntegerType
-    //TimeUUIDType
-    //IntegerType
-    //LexicalUUIDType
     private static String[] statements = {
-            "create keyspace thriftKs with placement_strategy = 'org.apache.cassandra.locator.SimpleStrategy' and" +
-            " strategy_options={replication_factor:1};",
-            "use thriftKs;",
+            "DROP KEYSPACE IF EXISTS thrift_ks",
+            "CREATE KEYSPACE thrift_ks WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1};",
+            "USE thrift_ks;",
 
-            "create column family SomeApp " +
-                    " with comparator = UTF8Type " +
-                    " and default_validation_class = UTF8Type " +
-                    " and key_validation_class = UTF8Type " +
-                    " and column_metadata = [" +
-                    "{column_name: col_ascii, validation_class: AsciiType}, " +
-                    "{column_name: col_long, validation_class: LongType}, " +
-                    "{column_name: col_bytes, validation_class: BytesType}, " +
-                    "{column_name: col_boolean, validation_class: BooleanType}, " +
-                    "{column_name: col_decimal, validation_class: DecimalType}, " +
-                    "{column_name: col_double, validation_class: DoubleType}, " +
-                    "{column_name: col_float, validation_class: FloatType}," +
-                    "{column_name: col_inetaddress, validation_class: InetAddressType}, " +
-                    "{column_name: col_int32, validation_class: Int32Type}, " +
-                    "{column_name: col_uft8, validation_class: UTF8Type}, " +
-                    "{column_name: col_date, validation_class: DateType}, " +
-                    "{column_name: col_uuid, validation_class: UUIDType}, " +
-                    "{column_name: col_integer, validation_class: IntegerType}, " +
-                    "{column_name: col_timeuuid, validation_class: TimeUUIDType}, " +
-                    "{column_name: col_lexical_uuid, validation_class: LexicalUUIDType}, " +
-                    "]; ",
+            "CREATE TABLE some_app (" +
+            "key text PRIMARY KEY," +
+            "col_ascii ascii," +
+            "col_bigint bigint," +
+            "col_blob blob," +
+            "col_boolean boolean," +
+            "col_decimal decimal," +
+            "col_double double," +
+            "col_float float," +
+            "col_inet inet," +
+            "col_int int," +
+            "col_text text," +
+            "col_timestamp timestamp," +
+            "col_timeuuid timeuuid," +
+            "col_uuid uuid," +
+            "col_varint varint)" +
+            " WITH COMPACT STORAGE;",
 
-             "set SomeApp['foo']['col_ascii'] = 'ascii';",
-             "set SomeApp['foo']['col_boolean'] = false;",
-             "set SomeApp['foo']['col_bytes'] = 'DEADBEEF';",
-             "set SomeApp['foo']['col_date'] = '2011-02-03T04:05:00+0000';",
-             "set SomeApp['foo']['col_decimal'] = '23.345';",
-             "set SomeApp['foo']['col_double'] = '2.7182818284590451';",
-             "set SomeApp['foo']['col_float'] = '23.45';",
-             "set SomeApp['foo']['col_inetaddress'] = '127.0.0.1';",          
-             "set SomeApp['foo']['col_int32'] = 23;",
-             "set SomeApp['foo']['col_integer'] = 12345;",
-             "set SomeApp['foo']['col_long'] = 12345678;",
-             "set SomeApp['foo']['col_lexical_uuid'] = 'e23f450f-53a6-11e2-7f7f-7f7f7f7f7f77';",
-             "set SomeApp['foo']['col_timeuuid'] = 'e23f450f-53a6-11e2-7f7f-7f7f7f7f7f7f';",
-             "set SomeApp['foo']['col_uft8'] = 'hello';",
-             "set SomeApp['foo']['col_uuid'] = '550e8400-e29b-41d4-a716-446655440000';",
+            "INSERT INTO some_app (key, col_ascii, col_bigint, col_blob, col_boolean, col_decimal, col_double, col_float," +
+                "col_inet, col_int, col_text, col_timestamp, col_uuid, col_varint, col_timeuuid) " +
+                    "VALUES ('foo', 'ascii', 12345678, 0xDEADBEEF, false, 23.345, 2.7182818284590451, 23.45, '127.0.0.1', 23, 'hello', " +
+                        "'2011-02-03T04:05:00+0000', 550e8400-e29b-41d4-a716-446655440000, 12345, e23f450f-53a6-11e2-7f7f-7f7f7f7f7f7f);",
 
-             "create column family CC with " +
-                       "key_validation_class = UTF8Type and " +
-                       "default_validation_class=CounterColumnType " +
-                       "and comparator=UTF8Type;",
+            "CREATE TABLE cc (key text, name text, value counter, PRIMARY KEY (key, name)) WITH COMPACT STORAGE",
 
-             "incr CC['chuck']['kick'];",
-             "incr CC['chuck']['kick'];",
-             "incr CC['chuck']['kick'];"
+            "UPDATE cc SET value = value + 3 WHERE key = 'chuck' AND name = 'kick'",
     };
 
     @BeforeClass
-    public static void setup() throws TTransportException, IOException, InterruptedException, ConfigurationException,
-                                      AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, CharacterCodingException, ClassNotFoundException, NoSuchFieldException, IllegalAccessException, InstantiationException
+    public static void setup() throws IOException, ConfigurationException, TException
     {
         startCassandra();
-        setupDataByCli(statements);
+        executeCQLStatements(statements);
         startHadoopCluster();
     }
 
     @Test
-    public void testCassandraStorageDataType() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException
+    public void testCassandraStorageDataType() throws IOException
     {
-        pig.registerQuery("rows = LOAD 'cassandra://thriftKs/SomeApp?" + defaultParameters + "' USING CassandraStorage();");
+        pig.registerQuery("rows = LOAD 'cassandra://thrift_ks/some_app?" + defaultParameters + "' USING CassandraStorage();");
+        Tuple t = pig.openIterator("rows").next();
 
-        //{key: chararray, col_ascii: (name: chararray,value: chararray),
-        //col_boolean: (name: chararray,value: bytearray),
-        //col_bytes: (name: chararray,value: bytearray),
-        //col_date: (name: chararray,value: long),
-        //col_decimal: (name: chararray,value: chararray),
-        //col_double: (name: chararray,value: double),
-        //col_float: (name: chararray,value: float),
-        //col_inetaddress: (name: chararray,value: chararray),
-        //col_int32: (name: chararray,value: int),
-        //col_integer: (name: chararray,value: int),
-        //col_lexical_uuid: (name: chararray,value: chararray),
-        //col_long: (name: chararray,value: long),
-        //col_timeuuid: (name: chararray,value: bytearray),
-        //col_uft8: (name: chararray,value: chararray),
-        //col_uuid: (name: chararray,value: chararray),
-        //columns: {(name: chararray,value: chararray)}}
-        Iterator<Tuple> it = pig.openIterator("rows");
-        if (it.hasNext()) {
-            Tuple t = it.next();
-            Assert.assertEquals(t.get(0), "foo");
-            Tuple column = (Tuple) t.get(1);
-            Assert.assertEquals(column.get(1), "ascii");
-            column = (Tuple) t.get(2);
-            Assert.assertEquals(column.get(1), false);
-            column = (Tuple) t.get(3);
-            Assert.assertEquals(column.get(1), new DataByteArray(Hex.hexToBytes("DEADBEEF")));
-            column = (Tuple) t.get(4);
-            Assert.assertEquals(column.get(1), 1296705900000L);
-            column = (Tuple) t.get(5);
-            Assert.assertEquals(column.get(1), "23.345");
-            column = (Tuple) t.get(6);
-            Assert.assertEquals(column.get(1), 2.7182818284590451d);
-            column = (Tuple) t.get(7);
-            Assert.assertEquals(column.get(1), 23.45f);
-            column = (Tuple) t.get(8);
-            Assert.assertEquals(column.get(1), "127.0.0.1");
-            column = (Tuple) t.get(9);
-            Assert.assertEquals(column.get(1), 23);
-            column = (Tuple) t.get(10);
-            Assert.assertEquals(column.get(1), 12345);
-            column = (Tuple) t.get(11);
-            Assert.assertEquals(column.get(1), new DataByteArray((TimeUUIDType.instance.fromString("e23f450f-53a6-11e2-7f7f-7f7f7f7f7f77").array())));
-            column = (Tuple) t.get(12);
-            Assert.assertEquals(column.get(1), 12345678L);
-            column = (Tuple) t.get(13);
-            Assert.assertEquals(column.get(1), new DataByteArray((TimeUUIDType.instance.fromString("e23f450f-53a6-11e2-7f7f-7f7f7f7f7f7f").array())));
-            column = (Tuple) t.get(14);
-            Assert.assertEquals(column.get(1), "hello");
-            column = (Tuple) t.get(15);
-            Assert.assertEquals(column.get(1), new DataByteArray((UUIDType.instance.fromString("550e8400-e29b-41d4-a716-446655440000").array())));
-        }
+        // key
+        assertEquals("foo", t.get(0));
 
-        pig.registerQuery("cc_rows = LOAD 'cassandra://thriftKs/CC?" + defaultParameters + "' USING CassandraStorage();");
+        // col_ascii
+        Tuple column = (Tuple) t.get(1);
+        assertEquals("ascii", column.get(1));
 
-        //(chuck,{(kick,3)})
-        it = pig.openIterator("cc_rows");
-        if (it.hasNext()) {
-            Tuple t = it.next();
-            Assert.assertEquals(t.get(0), "chuck");           
-            DataBag columns = (DataBag) t.get(1);
-            Iterator<Tuple> iter = columns.iterator();
-            if(iter.hasNext())
-            {
-                Tuple column = iter.next();
-                Assert.assertEquals(column.get(0), "kick");
-                Assert.assertEquals(column.get(1), 3L);
-            }
-         }
+        // col_bigint
+        column = (Tuple) t.get(2);
+        assertEquals(12345678L, column.get(1));
+
+        // col_blob
+        column = (Tuple) t.get(3);
+        assertEquals(new DataByteArray(Hex.hexToBytes("DEADBEEF")), column.get(1));
+
+        // col_boolean
+        column = (Tuple) t.get(4);
+        assertEquals(false, column.get(1));
+
+        // col_decimal
+        column = (Tuple) t.get(5);
+        assertEquals("23.345", column.get(1));
+
+        // col_double
+        column = (Tuple) t.get(6);
+        assertEquals(2.7182818284590451d, column.get(1));
+
+        // col_float
+        column = (Tuple) t.get(7);
+        assertEquals(23.45f, column.get(1));
+
+        // col_inet
+        column = (Tuple) t.get(8);
+        assertEquals("127.0.0.1", column.get(1));
+
+        // col_int
+        column = (Tuple) t.get(9);
+        assertEquals(23, column.get(1));
+
+        // col_text
+        column = (Tuple) t.get(10);
+        assertEquals("hello", column.get(1));
+
+        // col_timestamp
+        column = (Tuple) t.get(11);
+        assertEquals(1296705900000L, column.get(1));
+
+        // col_timeuuid
+        column = (Tuple) t.get(12);
+        assertEquals(new DataByteArray((TimeUUIDType.instance.fromString("e23f450f-53a6-11e2-7f7f-7f7f7f7f7f7f").array())), column.get(1));
+
+        // col_uuid
+        column = (Tuple) t.get(13);
+        assertEquals(new DataByteArray((UUIDType.instance.fromString("550e8400-e29b-41d4-a716-446655440000").array())), column.get(1));
+
+        // col_varint
+        column = (Tuple) t.get(14);
+        assertEquals(12345, column.get(1));
+
+        pig.registerQuery("cc_rows = LOAD 'cassandra://thrift_ks/cc?" + defaultParameters + "' USING CassandraStorage();");
+        t = pig.openIterator("cc_rows").next();
+
+        assertEquals("chuck", t.get(0));
+
+        DataBag columns = (DataBag) t.get(1);
+        column = columns.iterator().next();
+        assertEquals("kick", column.get(0));
+        assertEquals(3L, column.get(1));
     }
 }
diff --git a/test/pig/org/apache/cassandra/pig/ThriftColumnFamilyTest.java b/test/pig/org/apache/cassandra/pig/ThriftColumnFamilyTest.java
index 8903297..60d04d3 100644
--- a/test/pig/org/apache/cassandra/pig/ThriftColumnFamilyTest.java
+++ b/test/pig/org/apache/cassandra/pig/ThriftColumnFamilyTest.java
@@ -20,27 +20,20 @@
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.nio.charset.CharacterCodingException;
 import java.util.Iterator;
 
-import org.apache.cassandra.cli.CliMain;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.thrift.AuthenticationException;
-import org.apache.cassandra.thrift.AuthorizationException;
 import org.apache.cassandra.thrift.Cassandra;
 import org.apache.cassandra.thrift.ColumnOrSuperColumn;
 import org.apache.cassandra.thrift.ColumnPath;
 import org.apache.cassandra.thrift.ConsistencyLevel;
-import org.apache.cassandra.thrift.InvalidRequestException;
 import org.apache.cassandra.thrift.NotFoundException;
-import org.apache.cassandra.thrift.TimedOutException;
-import org.apache.cassandra.thrift.UnavailableException;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.pig.data.DataBag;
 import org.apache.pig.data.DataByteArray;
 import org.apache.pig.data.Tuple;
 import org.apache.thrift.TException;
-import org.apache.thrift.transport.TTransportException;
+
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -48,171 +41,157 @@
 public class ThriftColumnFamilyTest extends PigTestBase
 {    
     private static String[] statements = {
-            "create keyspace thriftKs with placement_strategy = 'org.apache.cassandra.locator.SimpleStrategy' and" +
-            " strategy_options={replication_factor:1};",
-            "use thriftKs;",
+            "DROP KEYSPACE IF EXISTS thrift_ks",
+            "CREATE KEYSPACE thrift_ks WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1};",
+            "USE thrift_ks;",
 
-            "create column family SomeApp " +
-                    " with comparator = UTF8Type " +
-                    " and default_validation_class = UTF8Type " +
-                    " and key_validation_class = UTF8Type " +
-                    " and column_metadata = [{column_name: name, validation_class: UTF8Type, index_type: KEYS}, " +
-                    "{column_name: vote_type, validation_class: UTF8Type}, " +
-                    "{column_name: rating, validation_class: Int32Type}, " +
-                    "{column_name: score, validation_class: LongType}, " +
-                    "{column_name: percent, validation_class: FloatType}, " +
-                    "{column_name: atomic_weight, validation_class: DoubleType}, " +
-                    "{column_name: created, validation_class: DateType},]; ",
+            "CREATE TABLE some_app (" +
+            "key text PRIMARY KEY," +
+            "name text," +
+            "vote_type text," +
+            "rating int," +
+            "score bigint," +
+            "percent float," +
+            "atomic_weight double," +
+            "created timestamp)" +
+            " WITH COMPACT STORAGE;",
 
-             "create column family CopyOfSomeApp " +
-                    "with key_validation_class = UTF8Type " +
-                    "and default_validation_class = UTF8Type " +
-                    "and comparator = UTF8Type " +
-                    "and column_metadata = " +
-                    "[ " +
-                        "{column_name: name, validation_class: UTF8Type, index_type: KEYS}, " +
-                        "{column_name: vote_type, validation_class: UTF8Type}, " +
-                        "{column_name: rating, validation_class: Int32Type}, " +
-                        "{column_name: score, validation_class: LongType}, " +
-                        "{column_name: percent, validation_class: FloatType}, " +
-                        "{column_name: atomic_weight, validation_class: DoubleType}, " +
-                        "{column_name: created, validation_class: DateType}, " +
-                    "];",
+            "CREATE INDEX ON some_app(name);",
 
-             "set SomeApp['foo']['name'] = 'User Foo';",
-             "set SomeApp['foo']['vote_type'] = 'like';",
-             "set SomeApp['foo']['rating'] = 8;",
-             "set SomeApp['foo']['score'] = 125000;",
-             "set SomeApp['foo']['percent'] = '85.0';",
-             "set SomeApp['foo']['atomic_weight'] = '2.7182818284590451';",
-             "set SomeApp['foo']['created'] = 1335890877;",
+            "INSERT INTO some_app (key, name, vote_type, rating, score, percent, atomic_weight, created) " +
+                    "VALUES ('foo', 'User Foo', 'like', 8, 125000, 85.0, 2.7182818284590451, 1335890877);",
 
-             "set SomeApp['bar']['name'] = 'User Bar';",
-             "set SomeApp['bar']['vote_type'] = 'like';",
-             "set SomeApp['bar']['rating'] = 9;",
-             "set SomeApp['bar']['score'] = 15000;",
-             "set SomeApp['bar']['percent'] = '35.0';",
-             "set SomeApp['bar']['atomic_weight'] = '3.1415926535897931';",
-             "set SomeApp['bar']['created'] = 1335890877;",
+            "INSERT INTO some_app (key, name, vote_type, rating, score, percent, atomic_weight, created) " +
+                    "VALUES ('bar', 'User Bar', 'like', 9, 15000, 35.0, 3.1415926535897931, 1335890877);",
 
-             "set SomeApp['baz']['name'] = 'User Baz';",
-             "set SomeApp['baz']['vote_type'] = 'dislike';",
-             "set SomeApp['baz']['rating'] = 3;",
-             "set SomeApp['baz']['score'] = 512000;",
-             "set SomeApp['baz']['percent'] = '95.3';",
-             "set SomeApp['baz']['atomic_weight'] = '1.61803399';",
-             "set SomeApp['baz']['created'] = 1335890877;",
-             "set SomeApp['baz']['extra1'] = 'extra1';",
-             "set SomeApp['baz']['extra2'] = 'extra2';",
-             "set SomeApp['baz']['extra3'] = 'extra3';",
+            "INSERT INTO some_app (key, name, vote_type, rating, score, percent, atomic_weight, created) " +
+                    "VALUES ('baz', 'User Baz', 'dislike', 3, 512000, 95.3, 1.61803399, 1335890877);",
 
-             "set SomeApp['qux']['name'] = 'User Qux';",
-             "set SomeApp['qux']['vote_type'] = 'dislike';",
-             "set SomeApp['qux']['rating'] = 2;",
-             "set SomeApp['qux']['score'] = 12000;",
-             "set SomeApp['qux']['percent'] = '64.7';",
-             "set SomeApp['qux']['atomic_weight'] = '0.660161815846869';",
-             "set SomeApp['qux']['created'] = 1335890877;",
-             "set SomeApp['qux']['extra1'] = 'extra1';",
-             "set SomeApp['qux']['extra2'] = 'extra2';",
-             "set SomeApp['qux']['extra3'] = 'extra3';",
-             "set SomeApp['qux']['extra4'] = 'extra4';",
-             "set SomeApp['qux']['extra5'] = 'extra5';",
-             "set SomeApp['qux']['extra6'] = 'extra6';",
-             "set SomeApp['qux']['extra7'] = 'extra7';",
+            "INSERT INTO some_app (key, name, vote_type, rating, score, percent, atomic_weight, created) " +
+                    "VALUES ('qux', 'User Qux', 'dislike', 2, 12000, 64.7, 0.660161815846869, 1335890877);",
 
-             "create column family U8 with " +
-                     "key_validation_class = UTF8Type and " +
-                     "comparator = UTF8Type;",
-                     
-             "create column family Bytes with " +
-                      "key_validation_class = BytesType and " +
-                      "comparator = UTF8Type;",
+            "CREATE TABLE copy_of_some_app (" +
+            "key text PRIMARY KEY," +
+            "name text," +
+            "vote_type text," +
+            "rating int," +
+            "score bigint," +
+            "percent float," +
+            "atomic_weight double," +
+            "created timestamp)" +
+            " WITH COMPACT STORAGE;",
 
-             "set U8['foo']['x'] = ascii('Z');",
-             "set Bytes[ascii('foo')]['x'] = ascii('Z');",
+            "CREATE INDEX ON copy_of_some_app(name);",
 
-             "create column family CC with " +
-                       "key_validation_class = UTF8Type and " +
-                       "default_validation_class=CounterColumnType " +
-                       "and comparator=UTF8Type;",
+            "CREATE TABLE u8 (" +
+            "key text," +
+            "column1 text," +
+            "value blob," +
+            "PRIMARY KEY (key, column1))" +
+            " WITH COMPACT STORAGE",
 
-             "incr CC['chuck']['kick'];",
-             "incr CC['chuck']['kick'];",
-             "incr CC['chuck']['kick'];",
-             "incr CC['chuck']['fist'];",
+            "INSERT INTO u8 (key, column1, value) VALUES ('foo', 'x', asciiAsBlob('Z'))",
 
-             "create column family Compo " +
-                       "with key_validation_class = UTF8Type " +
-                       "and default_validation_class = UTF8Type " +
-                       "and comparator = 'CompositeType(UTF8Type,UTF8Type)';",
+            "CREATE TABLE bytes (" +
+            "key blob," +
+            "column1 text," +
+            "value blob," +
+            "PRIMARY KEY (key, column1))" +
+            " WITH COMPACT STORAGE",
 
-             "set Compo['punch']['bruce:lee'] = 'ouch';",
-             "set Compo['punch']['bruce:bruce'] = 'hunh?';",
-             "set Compo['kick']['bruce:lee'] = 'oww';",
-             "set Compo['kick']['bruce:bruce'] = 'watch it, mate';",
+            "INSERT INTO bytes (key, column1, value) VALUES (asciiAsBlob('foo'), 'x', asciiAsBlob('Z'))",
 
-             "create column family CompoInt " +
-                       "with key_validation_class = UTF8Type " +
-                       "and default_validation_class = UTF8Type " +
-                       "and comparator = 'CompositeType(LongType,LongType)';",
+            "CREATE TABLE cc (key text, name text, value counter, PRIMARY KEY (key, name)) WITH COMPACT STORAGE",
 
-            "set CompoInt['clock']['1:0'] = 'z';",
-            "set CompoInt['clock']['1:30'] = 'zzzz';",
-            "set CompoInt['clock']['2:30'] = 'daddy?';",
-            "set CompoInt['clock']['6:30'] = 'coffee...';",
+            "UPDATE cc SET value = value + 3 WHERE key = 'chuck' AND name = 'kick'",
+            "UPDATE cc SET value = value + 1 WHERE key = 'chuck' AND name = 'fist'",
 
-            "create column family CompoIntCopy " +
-                        "with key_validation_class = UTF8Type " +
-                        "and default_validation_class = UTF8Type " +
-                        "and comparator = 'CompositeType(LongType,LongType)';",
+            "CREATE TABLE compo (" +
+            "key text," +
+            "column1 text," +
+            "column2 text," +
+            "value text," +
+            "PRIMARY KEY (key, column1, column2))" +
+            " WITH COMPACT STORAGE",
 
-            "create column family CompoKey " +
-                        "with key_validation_class = 'CompositeType(UTF8Type,LongType)' " +
-                        "and default_validation_class = UTF8Type " +
-                        "and comparator = LongType;",
+            "INSERT INTO compo (key, column1, column2, value) VALUES ('punch', 'bruce', 'lee', 'ouch');",
+            "INSERT INTO compo (key, column1, column2, value) VALUES ('punch', 'bruce', 'bruce', 'hunh?');",
+            "INSERT INTO compo (key, column1, column2, value) VALUES ('kick', 'bruce', 'lee', 'oww');",
+            "INSERT INTO compo (key, column1, column2, value) VALUES ('kick', 'bruce', 'bruce', 'watch it, mate');",
 
-            "set CompoKey['clock:10']['1'] = 'z';",
-            "set CompoKey['clock:20']['1'] = 'zzzz';",
-            "set CompoKey['clock:30']['2'] = 'daddy?';",
-            "set CompoKey['clock:40']['6'] = 'coffee...';",
+            "CREATE TABLE compo_int (" +
+            "key text," +
+            "column1 bigint," +
+            "column2 bigint," +
+            "value text," +
+            "PRIMARY KEY (key, column1, column2))" +
+            " WITH COMPACT STORAGE",
 
-            "create column family CompoKeyCopy " +
-                        "with key_validation_class = 'CompositeType(UTF8Type,LongType)' " +
-                        "and default_validation_class = UTF8Type " +
-                        "and comparator = LongType;"
+            "INSERT INTO compo_int (key, column1, column2, value) VALUES ('clock', 1, 0, 'z');",
+            "INSERT INTO compo_int (key, column1, column2, value) VALUES ('clock', 1, 30, 'zzzz');",
+            "INSERT INTO compo_int (key, column1, column2, value) VALUES ('clock', 2, 30, 'daddy?');",
+            "INSERT INTO compo_int (key, column1, column2, value) VALUES ('clock', 6, 30, 'coffee...');",
+
+            "CREATE TABLE compo_int_copy (" +
+            "key text," +
+            "column1 bigint," +
+            "column2 bigint," +
+            "value text," +
+            "PRIMARY KEY (key, column1, column2))" +
+            " WITH COMPACT STORAGE",
+
+            "CREATE TABLE compo_key (" +
+            "key text," +
+            "column1 bigint," +
+            "column2 bigint," +
+            "value text," +
+            "PRIMARY KEY ((key, column1), column2))" +
+            " WITH COMPACT STORAGE",
+
+            "INSERT INTO compo_key (key, column1, column2, value) VALUES ('clock', 10, 1, 'z');",
+            "INSERT INTO compo_key (key, column1, column2, value) VALUES ('clock', 20, 1, 'zzzz');",
+            "INSERT INTO compo_key (key, column1, column2, value) VALUES ('clock', 30, 2, 'daddy?');",
+            "INSERT INTO compo_key (key, column1, column2, value) VALUES ('clock', 40, 6, 'coffee...');",
+
+            "CREATE TABLE compo_key_copy (" +
+            "key text," +
+            "column1 bigint," +
+            "column2 bigint," +
+            "value text," +
+            "PRIMARY KEY ((key, column1), column2))" +
+            " WITH COMPACT STORAGE",
     };
 
-    private static String[] deleteCopyOfSomeAppTableData = { "use thriftKs;",
-            "DEL CopyOfSomeApp ['foo']",
-            "DEL CopyOfSomeApp ['bar']",
-            "DEL CopyOfSomeApp ['baz']",
-            "DEL CopyOfSomeApp ['qux']"
+    private static String[] deleteCopyOfSomeAppTableData = {
+            "use thrift_ks;",
+            "DELETE FROM copy_of_some_app WHERE key = 'foo';",
+            "DELETE FROM copy_of_some_app WHERE key = 'bar';",
+            "DELETE FROM copy_of_some_app WHERE key = 'baz';",
+            "DELETE FROM copy_of_some_app WHERE key = 'qux';",
     };
 
     @BeforeClass
-    public static void setup() throws TTransportException, IOException, InterruptedException, ConfigurationException,
-                                      AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, CharacterCodingException, ClassNotFoundException, NoSuchFieldException, IllegalAccessException, InstantiationException
+    public static void setup() throws IOException, ConfigurationException, TException
     {
         startCassandra();
-        setupDataByCli(statements);
+        executeCQLStatements(statements);
         startHadoopCluster();
     }
 
     @Test
-    public void testCqlNativeStorage() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException, AuthenticationException, AuthorizationException
+    public void testCqlNativeStorage() throws IOException
     {
         //regular thrift column families
-        //input_cql=select * from "SomeApp" where token(key) > ? and token(key) <= ?
-        cqlStorageTest("data = load 'cql://thriftKs/SomeApp?" + defaultParameters + nativeParameters + "&input_cql=select%20*%20from%20%22SomeApp%22%20where%20token(key)%20%3E%20%3F%20and%20token(key)%20%3C%3D%20%3F' using CqlNativeStorage();");
+        //input_cql=select * from "some_app" where token(key) > ? and token(key) <= ?
+        cqlStorageTest("data = load 'cql://thrift_ks/some_app?" + defaultParameters + nativeParameters + "&input_cql=select%20*%20from%20%22some_app%22%20where%20token(key)%20%3E%20%3F%20and%20token(key)%20%3C%3D%20%3F' using CqlNativeStorage();");
 
-        //Test counter colun family
-        //input_cql=select * from "CC" where token(key) > ? and token(key) <= ?
-        cqlStorageCounterTableTest("cc_data = load 'cql://thriftKs/CC?" + defaultParameters + nativeParameters + "&input_cql=select%20*%20from%20%22CC%22%20where%20token(key)%20%3E%20%3F%20and%20token(key)%20%3C%3D%20%3F' using CqlNativeStorage();");
+        //Test counter column family
+        //input_cql=select * from "cc" where token(key) > ? and token(key) <= ?
+        cqlStorageCounterTableTest("cc_data = load 'cql://thrift_ks/cc?" + defaultParameters + nativeParameters + "&input_cql=select%20*%20from%20%22cc%22%20where%20token(key)%20%3E%20%3F%20and%20token(key)%20%3C%3D%20%3F' using CqlNativeStorage();");
 
         //Test composite column family
-        //input_cql=select * from "Compo" where token(key) > ? and token(key) <= ?
-        cqlStorageCompositeTableTest("compo_data = load 'cql://thriftKs/Compo?" + defaultParameters + nativeParameters + "&input_cql=select%20*%20from%20%22Compo%22%20where%20token(key)%20%3E%20%3F%20and%20token(key)%20%3C%3D%20%3F' using CqlNativeStorage();");
+        //input_cql=select * from "compo" where token(key) > ? and token(key) <= ?
+        cqlStorageCompositeTableTest("compo_data = load 'cql://thrift_ks/compo?" + defaultParameters + nativeParameters + "&input_cql=select%20*%20from%20%22compo%22%20where%20token(key)%20%3E%20%3F%20and%20token(key)%20%3C%3D%20%3F' using CqlNativeStorage();");
     }
 
     private void cqlStorageTest(String initialQuery) throws IOException
@@ -321,14 +300,11 @@
     }
 
     @Test
-    public void testCassandraStorageSchema() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException
+    public void testCassandraStorageSchema() throws IOException
     {
         //results: (qux,(atomic_weight,0.660161815846869),(created,1335890877),(name,User Qux),(percent,64.7),
-        //(rating,2),(score,12000),(vote_type,dislike),{(extra1,extra1),
-        //(extra2,extra2),(extra3,extra3),
-        //(extra4,extra4),(extra5,extra5),
-        //(extra6,extra6),(extra7,extra7)})
-        pig.registerQuery("rows = LOAD 'cassandra://thriftKs/SomeApp?" + defaultParameters + "' USING CassandraStorage();");
+        //(rating,2),(score,12000),(vote_type,dislike))
+        pig.registerQuery("rows = LOAD 'cassandra://thrift_ks/some_app?" + defaultParameters + "' USING CassandraStorage();");
 
         //schema: {key: chararray,atomic_weight: (name: chararray,value: double),created: (name: chararray,value: long),
         //name: (name: chararray,value: chararray),percent: (name: chararray,value: float),
@@ -358,48 +334,37 @@
                 column = (Tuple) t.get(7);
                 Assert.assertEquals(column.get(0), "vote_type");
                 Assert.assertEquals(column.get(1), "dislike");
-                DataBag columns = (DataBag) t.get(8);
-                Iterator<Tuple> iter = columns.iterator();
-                int i = 0;
-                while(iter.hasNext())
-                {
-                    i++;
-                    column = iter.next();
-                    Assert.assertEquals(column.get(0), "extra"+i);
-                }
-                Assert.assertEquals(7, columns.size());
             }
-
         }
     }
 
     @Test
-    public void testCassandraStorageFullCopy() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException, AuthenticationException, AuthorizationException
+    public void testCassandraStorageFullCopy() throws IOException, TException
     {
         pig.setBatchOn();
-        pig.registerQuery("rows = LOAD 'cassandra://thriftKs/SomeApp?" + defaultParameters + "' USING CassandraStorage();");
+        pig.registerQuery("rows = LOAD 'cassandra://thrift_ks/some_app?" + defaultParameters + "' USING CassandraStorage();");
         //full copy
-        pig.registerQuery("STORE rows INTO 'cassandra://thriftKs/CopyOfSomeApp?" + defaultParameters + "' USING CassandraStorage();");
+        pig.registerQuery("STORE rows INTO 'cassandra://thrift_ks/copy_of_some_app?" + defaultParameters + "' USING CassandraStorage();");
         pig.executeBatch();
-        Assert.assertEquals("User Qux", getColumnValue("thriftKs", "CopyOfSomeApp", "name", "qux", "UTF8Type"));
-        Assert.assertEquals("dislike", getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "qux", "UTF8Type"));
-        Assert.assertEquals("64.7", getColumnValue("thriftKs", "CopyOfSomeApp", "percent", "qux", "FloatType"));
+        Assert.assertEquals("User Qux", getColumnValue("thrift_ks", "copy_of_some_app", "name", "qux", "UTF8Type"));
+        Assert.assertEquals("dislike", getColumnValue("thrift_ks", "copy_of_some_app", "vote_type", "qux", "UTF8Type"));
+        Assert.assertEquals("64.7", getColumnValue("thrift_ks", "copy_of_some_app", "percent", "qux", "FloatType"));
     }
 
     @Test
-    public void testCassandraStorageSigleTupleCopy() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException, AuthenticationException, AuthorizationException
+    public void testCassandraStorageSingleTupleCopy() throws IOException, TException
     {
-        executeCliStatements(deleteCopyOfSomeAppTableData);
+        executeCQLStatements(deleteCopyOfSomeAppTableData);
         pig.setBatchOn();
-        pig.registerQuery("rows = LOAD 'cassandra://thriftKs/SomeApp?" + defaultParameters + "' USING CassandraStorage();");
-        //sigle tuple
+        pig.registerQuery("rows = LOAD 'cassandra://thrift_ks/some_app?" + defaultParameters + "' USING CassandraStorage();");
+        //single tuple
         pig.registerQuery("onecol = FOREACH rows GENERATE key, percent;");
-        pig.registerQuery("STORE onecol INTO 'cassandra://thriftKs/CopyOfSomeApp?" + defaultParameters + "' USING CassandraStorage();");
+        pig.registerQuery("STORE onecol INTO 'cassandra://thrift_ks/copy_of_some_app?" + defaultParameters + "' USING CassandraStorage();");
         pig.executeBatch();
         String value = null;
         try
         {
-            value = getColumnValue("thriftKs", "CopyOfSomeApp", "name", "qux", "UTF8Type");
+            value = getColumnValue("thrift_ks", "copy_of_some_app", "name", "qux", "UTF8Type");
         }
         catch (NotFoundException e)
         {
@@ -409,7 +374,7 @@
             Assert.fail();
         try
         {
-            value = getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "qux", "UTF8Type");
+            value = getColumnValue("thrift_ks", "copy_of_some_app", "vote_type", "qux", "UTF8Type");
         }
         catch (NotFoundException e)
         {
@@ -417,23 +382,23 @@
         }
         if (value != null)
             Assert.fail();
-        Assert.assertEquals("64.7", getColumnValue("thriftKs", "CopyOfSomeApp", "percent", "qux", "FloatType"));
+        Assert.assertEquals("64.7", getColumnValue("thrift_ks", "copy_of_some_app", "percent", "qux", "FloatType"));
     }
 
     @Test
-    public void testCassandraStorageBagOnlyCopy() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException, AuthenticationException, AuthorizationException
+    public void testCassandraStorageBagOnlyCopy() throws IOException, TException
     {
-        executeCliStatements(deleteCopyOfSomeAppTableData);
+        executeCQLStatements(deleteCopyOfSomeAppTableData);
         pig.setBatchOn();
-        pig.registerQuery("rows = LOAD 'cassandra://thriftKs/SomeApp?" + defaultParameters + "' USING CassandraStorage();");
+        pig.registerQuery("rows = LOAD 'cassandra://thrift_ks/some_app?" + defaultParameters + "' USING CassandraStorage();");
         //bag only
         pig.registerQuery("other = FOREACH rows GENERATE key, columns;");
-        pig.registerQuery("STORE other INTO 'cassandra://thriftKs/CopyOfSomeApp?" + defaultParameters + "' USING CassandraStorage();");
+        pig.registerQuery("STORE other INTO 'cassandra://thrift_ks/copy_of_some_app?" + defaultParameters + "' USING CassandraStorage();");
         pig.executeBatch();
         String value = null;
         try
         {
-            value = getColumnValue("thriftKs", "CopyOfSomeApp", "name", "qux", "UTF8Type");
+            value = getColumnValue("thrift_ks", "copy_of_some_app", "name", "qux", "UTF8Type");
         }
         catch (NotFoundException e)
         {
@@ -443,7 +408,7 @@
             Assert.fail();
         try
         {
-            value = getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "qux", "UTF8Type");
+            value = getColumnValue("thrift_ks", "copy_of_some_app", "vote_type", "qux", "UTF8Type");
         }
         catch (NotFoundException e)
         {
@@ -453,7 +418,7 @@
             Assert.fail();
         try
         {
-            value = getColumnValue("thriftKs", "CopyOfSomeApp", "percent", "qux", "FloatType");
+            value = getColumnValue("thrift_ks", "copy_of_some_app", "percent", "qux", "FloatType");
         }
         catch (NotFoundException e)
         {
@@ -461,27 +426,26 @@
         }
         if (value != null)
             Assert.fail();
-        Assert.assertEquals("extra1", getColumnValue("thriftKs", "CopyOfSomeApp", "extra1", "qux", "UTF8Type"));
     }
 
     @Test
-    public void testCassandraStorageFilter() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException, AuthenticationException, AuthorizationException
+    public void testCassandraStorageFilter() throws IOException, TException
     {
-        executeCliStatements(deleteCopyOfSomeAppTableData);
+        executeCQLStatements(deleteCopyOfSomeAppTableData);
         pig.setBatchOn();
-        pig.registerQuery("rows = LOAD 'cassandra://thriftKs/SomeApp?" + defaultParameters + "' USING CassandraStorage();");
+        pig.registerQuery("rows = LOAD 'cassandra://thrift_ks/some_app?" + defaultParameters + "' USING CassandraStorage();");
 
         //filter
         pig.registerQuery("likes = FILTER rows by vote_type.value eq 'like' and rating.value > 5;");
-        pig.registerQuery("STORE likes INTO 'cassandra://thriftKs/CopyOfSomeApp?" + defaultParameters + "' USING CassandraStorage();");
+        pig.registerQuery("STORE likes INTO 'cassandra://thrift_ks/copy_of_some_app?" + defaultParameters + "' USING CassandraStorage();");
         pig.executeBatch();
 
-        Assert.assertEquals("like", getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "bar", "UTF8Type"));
-        Assert.assertEquals("like", getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "foo", "UTF8Type"));
+        Assert.assertEquals("like", getColumnValue("thrift_ks", "copy_of_some_app", "vote_type", "bar", "UTF8Type"));
+        Assert.assertEquals("like", getColumnValue("thrift_ks", "copy_of_some_app", "vote_type", "foo", "UTF8Type"));
         String value = null;
         try
         {
-            value = getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "qux", "UTF8Type");
+            value = getColumnValue("thrift_ks", "copy_of_some_app", "vote_type", "qux", "UTF8Type");
         }
         catch (NotFoundException e)
         {
@@ -491,7 +455,7 @@
             Assert.fail();
         try
         {
-            value = getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "baz", "UTF8Type");
+            value = getColumnValue("thrift_ks", "copy_of_some_app", "vote_type", "baz", "UTF8Type");
         }
         catch (NotFoundException e)
         {
@@ -500,19 +464,18 @@
         if (value != null)
             Assert.fail();
 
-        executeCliStatements(deleteCopyOfSomeAppTableData);
+        executeCQLStatements(deleteCopyOfSomeAppTableData);
         pig.setBatchOn();
-        pig.registerQuery("rows = LOAD 'cassandra://thriftKs/SomeApp?" + defaultParameters + "' USING CassandraStorage();");
-        pig.registerQuery("dislikes_extras = FILTER rows by vote_type.value eq 'dislike' AND COUNT(columns) > 0;");
-        pig.registerQuery("STORE dislikes_extras INTO 'cassandra://thriftKs/CopyOfSomeApp?" + defaultParameters + "' USING CassandraStorage();");
-        pig.registerQuery("visible = FILTER rows BY COUNT(columns) == 0;");
+        pig.registerQuery("rows = LOAD 'cassandra://thrift_ks/some_app?" + defaultParameters + "' USING CassandraStorage();");
+        pig.registerQuery("dislikes_extras = FILTER rows by vote_type.value eq 'dislike';");
+        pig.registerQuery("STORE dislikes_extras INTO 'cassandra://thrift_ks/copy_of_some_app?" + defaultParameters + "' USING CassandraStorage();");
         pig.executeBatch();
-        Assert.assertEquals("dislike", getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "baz", "UTF8Type"));
-        Assert.assertEquals("dislike", getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "qux", "UTF8Type"));
+        Assert.assertEquals("dislike", getColumnValue("thrift_ks", "copy_of_some_app", "vote_type", "baz", "UTF8Type"));
+        Assert.assertEquals("dislike", getColumnValue("thrift_ks", "copy_of_some_app", "vote_type", "qux", "UTF8Type"));
         value = null;
         try
         {
-            value = getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "bar", "UTF8Type");
+            value = getColumnValue("thrift_ks", "copy_of_some_app", "vote_type", "bar", "UTF8Type");
         }
         catch (NotFoundException e)
         {
@@ -522,7 +485,7 @@
             Assert.fail();
         try
         {
-            value = getColumnValue("thriftKs", "CopyOfSomeApp", "vote_type", "foo", "UTF8Type");
+            value = getColumnValue("thrift_ks", "copy_of_some_app", "vote_type", "foo", "UTF8Type");
         }
         catch (NotFoundException e)
         {
@@ -533,11 +496,11 @@
     }
 
     @Test
-    public void testCassandraStorageJoin() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException, AuthenticationException, AuthorizationException
+    public void testCassandraStorageJoin() throws IOException
     {
         //test key types with a join
-        pig.registerQuery("U8 = load 'cassandra://thriftKs/U8?" + defaultParameters + "' using CassandraStorage();");
-        pig.registerQuery("Bytes = load 'cassandra://thriftKs/Bytes?" + defaultParameters + "' using CassandraStorage();");
+        pig.registerQuery("U8 = load 'cassandra://thrift_ks/u8?" + defaultParameters + "' using CassandraStorage();");
+        pig.registerQuery("Bytes = load 'cassandra://thrift_ks/bytes?" + defaultParameters + "' using CassandraStorage();");
 
         //cast key to chararray
         pig.registerQuery("b = foreach Bytes generate (chararray)key, columns;");
@@ -585,28 +548,22 @@
     }
 
     @Test
-    public void testCassandraStorageCounterCF() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException, AuthenticationException, AuthorizationException
+    public void testCassandraStorageCounterCF() throws IOException
     {
-        pig.registerQuery("rows = LOAD 'cassandra://thriftKs/SomeApp?" + defaultParameters + "' USING CassandraStorage();");
-
         //Test counter column family support
-        pig.registerQuery("CC = load 'cassandra://thriftKs/CC?" + defaultParameters + "' using CassandraStorage();");
+        pig.registerQuery("CC = load 'cassandra://thrift_ks/cc?" + defaultParameters + "' using CassandraStorage();");
         pig.registerQuery("total_hits = foreach CC generate key, SUM(columns.value);");
         //(chuck,4)
-        Iterator<Tuple> it = pig.openIterator("total_hits");
-        if (it.hasNext()) {
-            Tuple t = it.next();
-            Assert.assertEquals(t.get(0), "chuck");
-            Assert.assertEquals(t.get(1), 4l);
-        }
+        Tuple t = pig.openIterator("total_hits").next();
+        Assert.assertEquals(t.get(0), "chuck");
+        Assert.assertEquals(t.get(1), 4l);
     }
 
-    /** This test case fails due to antlr lib conflicts, Cassandra2.1 uses 3.2, Hive1.2 uses 3.4 */
-    //@Test
-    public void testCassandraStorageCompositeColumnCF() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException, AuthenticationException, AuthorizationException
+    @Test
+    public void testCassandraStorageCompositeColumnCF() throws IOException
     {
         //Test CompositeType
-        pig.registerQuery("compo = load 'cassandra://thriftKs/Compo?" + defaultParameters + "' using CassandraStorage();");
+        pig.registerQuery("compo = load 'cassandra://thrift_ks/compo?" + defaultParameters + "' using CassandraStorage();");
         pig.registerQuery("compo = foreach compo generate key as method, flatten(columns);");
         pig.registerQuery("lee = filter compo by columns::name == ('bruce','lee');");
 
@@ -626,7 +583,7 @@
                 Assert.assertEquals(t.get(2), "ouch");
         }
         Assert.assertEquals(count, 2);
-        pig.registerQuery("night = load 'cassandra://thriftKs/CompoInt?" + defaultParameters + "' using CassandraStorage();");
+        pig.registerQuery("night = load 'cassandra://thrift_ks/compo_int?" + defaultParameters + "' using CassandraStorage();");
         pig.registerQuery("night = foreach night generate flatten(columns);");
         pig.registerQuery("night = foreach night generate (int)columns::name.$0+(double)columns::name.$1/60 as hour, columns::value as noise;");
 
@@ -641,10 +598,10 @@
             Assert.assertEquals(t.get(1), "daddy?");
         }
         pig.setBatchOn();
-        pig.registerQuery("compo_int_rows = LOAD 'cassandra://thriftKs/CompoInt?" + defaultParameters + "' using CassandraStorage();");
-        pig.registerQuery("STORE compo_int_rows INTO 'cassandra://thriftKs/CompoIntCopy?" + defaultParameters + "' using CassandraStorage();");
+        pig.registerQuery("compo_int_rows = LOAD 'cassandra://thrift_ks/compo_int?" + defaultParameters + "' using CassandraStorage();");
+        pig.registerQuery("STORE compo_int_rows INTO 'cassandra://thrift_ks/compo_int_copy?" + defaultParameters + "' using CassandraStorage();");
         pig.executeBatch();
-        pig.registerQuery("compocopy_int_rows = LOAD 'cassandra://thriftKs/CompoIntCopy?" + defaultParameters + "' using CassandraStorage();");
+        pig.registerQuery("compocopy_int_rows = LOAD 'cassandra://thrift_ks/compo_int_copy?" + defaultParameters + "' using CassandraStorage();");
         //(clock,{((1,0),z),((1,30),zzzz),((2,30),daddy?),((6,30),coffee...)})
         it = pig.openIterator("compocopy_int_rows");
         count = 0;
@@ -652,11 +609,9 @@
             Tuple t = it.next();
             Assert.assertEquals(t.get(0), "clock");
             DataBag columns = (DataBag) t.get(1);
-            Iterator<Tuple> iter = columns.iterator();
-            while (iter.hasNext())
+            for (Tuple t1 : columns)
             {
-                count ++;
-                Tuple t1 = iter.next();
+                count++;
                 Tuple inner = (Tuple) t1.get(0);
                 if ((Long) inner.get(0) == 1L && (Long) inner.get(1) == 0L)
                     Assert.assertEquals(t1.get(1), "z");
@@ -672,10 +627,10 @@
     }
 
     @Test
-    public void testCassandraStorageCompositeKeyCF() throws IOException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException, AuthenticationException, AuthorizationException
+    public void testCassandraStorageCompositeKeyCF() throws IOException
     {
         //Test CompositeKey
-        pig.registerQuery("compokeys = load 'cassandra://thriftKs/CompoKey?" + defaultParameters + "' using CassandraStorage();");
+        pig.registerQuery("compokeys = load 'cassandra://thrift_ks/compo_key?" + defaultParameters + "' using CassandraStorage();");
         pig.registerQuery("compokeys = filter compokeys by key.$1 == 40;");
         //((clock,40),{(6,coffee...)})
         Iterator<Tuple> it = pig.openIterator("compokeys");
@@ -694,10 +649,10 @@
             }
         }
         pig.setBatchOn();
-        pig.registerQuery("compo_key_rows = LOAD 'cassandra://thriftKs/CompoKey?" + defaultParameters + "' using CassandraStorage();");
-        pig.registerQuery("STORE compo_key_rows INTO 'cassandra://thriftKs/CompoKeyCopy?" + defaultParameters + "' using CassandraStorage();");
+        pig.registerQuery("compo_key_rows = LOAD 'cassandra://thrift_ks/compo_key?" + defaultParameters + "' using CassandraStorage();");
+        pig.registerQuery("STORE compo_key_rows INTO 'cassandra://thrift_ks/compo_key_copy?" + defaultParameters + "' using CassandraStorage();");
         pig.executeBatch();
-        pig.registerQuery("compo_key_copy_rows = LOAD 'cassandra://thriftKs/CompoKeyCopy?" + defaultParameters + "' using CassandraStorage();");
+        pig.registerQuery("compo_key_copy_rows = LOAD 'cassandra://thrift_ks/compo_key_copy?" + defaultParameters + "' using CassandraStorage();");
         //((clock,10),{(1,z)})
         //((clock,20),{(1,zzzz)})
         //((clock,30),{(2,daddy?)})
@@ -753,11 +708,10 @@
                 }
             }
         }
-        Assert.assertEquals(count, 4);
+        Assert.assertEquals(4, count);
     }
 
-    private String getColumnValue(String ks, String cf, String colName, String key, String validator)
-    throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, IOException
+    private String getColumnValue(String ks, String cf, String colName, String key, String validator) throws TException, IOException
     {
         Cassandra.Client client = getClient();
         client.set_keyspace(ks);
@@ -770,17 +724,4 @@
         ColumnOrSuperColumn got = client.get(key_user_id, cp, ConsistencyLevel.ONE);
         return parseType(validator).getString(got.getColumn().value);
     }
-
-    private void executeCliStatements(String[] statements) throws CharacterCodingException, ClassNotFoundException, TException, TimedOutException, NotFoundException, InvalidRequestException, NoSuchFieldException, UnavailableException, IllegalAccessException, InstantiationException
-    {
-        CliMain.connect("127.0.0.1", 9170);
-        try
-        {
-            for (String stmt : statements)
-                CliMain.processStatement(stmt);
-        }
-        catch (Exception e)
-        {
-        }
-    }
 }
diff --git a/test/pig/org/apache/pig/test/MiniCluster.java b/test/pig/org/apache/pig/test/MiniCluster.java
index e8f1f6e..95acb84 100644
--- a/test/pig/org/apache/pig/test/MiniCluster.java
+++ b/test/pig/org/apache/pig/test/MiniCluster.java
@@ -22,6 +22,7 @@
 import java.io.IOException;
 import java.io.OutputStream;
 
+import org.apache.cassandra.utils.FBUtilities;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.mapred.MiniMRCluster;
@@ -48,6 +49,8 @@
 
             // Builds and starts the mini dfs and mapreduce clusters
             Configuration config = new Configuration();
+            if (FBUtilities.isWindows())
+                config.set("fs.file.impl", WindowsLocalFileSystem.class.getName());
             m_dfs = new MiniDFSCluster(config, dataNodes, true, null);
             m_fileSys = m_dfs.getFileSystem();
             m_mr = new MiniMRCluster(taskTrackers, m_fileSys.getUri().toString(), 1);
@@ -76,7 +79,8 @@
 
     @Override
     protected void shutdownMiniMrClusters() {
-        if (m_mr != null) { m_mr.shutdown(); }
-            m_mr = null;
+        if (m_mr != null)
+            m_mr.shutdown();
+        m_mr = null;
     }
 }
diff --git a/test/pig/org/apache/pig/test/WindowsLocalFileSystem.java b/test/pig/org/apache/pig/test/WindowsLocalFileSystem.java
new file mode 100644
index 0000000..9193341
--- /dev/null
+++ b/test/pig/org/apache/pig/test/WindowsLocalFileSystem.java
@@ -0,0 +1,62 @@
+/*
+ * 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.pig.test;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Workaround for https://issues.apache.org/jira/browse/HADOOP-7682 used to allow the Pig-tests to run on Cygwin on
+ * a Windows box. This workaround was suggested by Joshua Caplan in the comments of HADOOP-7682.
+ */
+public final class WindowsLocalFileSystem extends LocalFileSystem
+{
+    private final Logger logger = LoggerFactory.getLogger(getClass());
+
+    public WindowsLocalFileSystem()
+    {
+        logger.warn("Using {} instead of org.apache.hadoop.fs.LocalFileSystem to avoid the problem linked to HADOOP-7682. " +
+                    "IOException thrown when setting permissions will be swallowed.", getClass().getName());
+    }
+
+    @Override
+    public boolean mkdirs(Path path, FsPermission permission) throws IOException
+    {
+        boolean result = super.mkdirs(path);
+        setPermission(path, permission);
+        return result;
+    }
+
+    @Override
+    public void setPermission(Path p, FsPermission permission) throws IOException
+    {
+        try
+        {
+            super.setPermission(p, permission);
+        }
+        catch (IOException e)
+        {
+            // Just swallow the Exception as logging it produces too much output.
+        }
+    }
+}
diff --git a/test/unit/org/apache/cassandra/AbstractSerializationsTester.java b/test/unit/org/apache/cassandra/AbstractSerializationsTester.java
index 1c97eae..ebfa79d 100644
--- a/test/unit/org/apache/cassandra/AbstractSerializationsTester.java
+++ b/test/unit/org/apache/cassandra/AbstractSerializationsTester.java
@@ -21,7 +21,8 @@
 
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataOutputBuffer;
-import org.apache.cassandra.io.util.DataOutputStreamAndChannel;
+import org.apache.cassandra.io.util.DataOutputStreamPlus;
+import org.apache.cassandra.io.util.BufferedDataOutputStreamPlus;
 import org.apache.cassandra.net.MessagingService;
 
 import java.io.DataInputStream;
@@ -32,7 +33,7 @@
 import java.util.HashMap;
 import java.util.Map;
 
-public class AbstractSerializationsTester extends SchemaLoader
+public class AbstractSerializationsTester
 {
     protected static final String CUR_VER = System.getProperty("cassandra.version", "2.1");
     protected static final Map<String, Integer> VERSION_MAP = new HashMap<String, Integer> ()
@@ -65,10 +66,11 @@
         return new DataInputStream(new FileInputStream(f));
     }
 
-    protected static DataOutputStreamAndChannel getOutput(String name) throws IOException
+    @SuppressWarnings("resource")
+    protected static DataOutputStreamPlus getOutput(String name) throws IOException
     {
         File f = new File("test/data/serialization/" + CUR_VER + "/" + name);
         f.getParentFile().mkdirs();
-        return new DataOutputStreamAndChannel(new FileOutputStream(f));
+        return new BufferedDataOutputStreamPlus(new FileOutputStream(f).getChannel());
     }
 }
diff --git a/test/unit/org/apache/cassandra/CassandraBriefJUnitResultFormatter.java b/test/unit/org/apache/cassandra/CassandraBriefJUnitResultFormatter.java
new file mode 100644
index 0000000..084858f
--- /dev/null
+++ b/test/unit/org/apache/cassandra/CassandraBriefJUnitResultFormatter.java
@@ -0,0 +1,312 @@
+/*
+ *  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;
+
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.StringWriter;
+import java.text.NumberFormat;
+
+import junit.framework.AssertionFailedError;
+import junit.framework.Test;
+
+import org.apache.tools.ant.BuildException;
+import org.apache.tools.ant.taskdefs.optional.junit.IgnoredTestListener;
+import org.apache.tools.ant.taskdefs.optional.junit.JUnitResultFormatter;
+import org.apache.tools.ant.taskdefs.optional.junit.JUnitTest;
+import org.apache.tools.ant.taskdefs.optional.junit.JUnitTestRunner;
+import org.apache.tools.ant.taskdefs.optional.junit.JUnitVersionHelper;
+import org.apache.tools.ant.util.FileUtils;
+import org.apache.tools.ant.util.StringUtils;
+
+/**
+ * Prints plain text output of the test to a specified Writer.
+ * Inspired by the PlainJUnitResultFormatter.
+ *
+ * @see FormatterElement
+ * @see PlainJUnitResultFormatter
+ */
+public class CassandraBriefJUnitResultFormatter implements JUnitResultFormatter, IgnoredTestListener {
+
+    private static final double ONE_SECOND = 1000.0;
+
+    private static final String tag = System.getProperty("cassandra.testtag", "");
+
+    /**
+     * Where to write the log to.
+     */
+    private OutputStream out;
+
+    /**
+     * Used for writing the results.
+     */
+    private BufferedWriter output;
+
+    /**
+     * Used as part of formatting the results.
+     */
+    private StringWriter results;
+
+    /**
+     * Used for writing formatted results to.
+     */
+    private BufferedWriter resultWriter;
+
+    /**
+     * Formatter for timings.
+     */
+    private NumberFormat numberFormat = NumberFormat.getInstance();
+
+    /**
+     * Output suite has written to System.out
+     */
+    private String systemOutput = null;
+
+    /**
+     * Output suite has written to System.err
+     */
+    private String systemError = null;
+
+    /**
+     * Constructor for BriefJUnitResultFormatter.
+     */
+    public CassandraBriefJUnitResultFormatter() {
+        results = new StringWriter();
+        resultWriter = new BufferedWriter(results);
+    }
+
+    /**
+     * Sets the stream the formatter is supposed to write its results to.
+     * @param out the output stream to write to
+     */
+    public void setOutput(OutputStream out) {
+        this.out = out;
+        output = new BufferedWriter(new java.io.OutputStreamWriter(out));
+    }
+
+    /**
+     * @see JUnitResultFormatter#setSystemOutput(String)
+     */
+    /** {@inheritDoc}. */
+    public void setSystemOutput(String out) {
+        systemOutput = out;
+    }
+
+    /**
+     * @see JUnitResultFormatter#setSystemError(String)
+     */
+    /** {@inheritDoc}. */
+    public void setSystemError(String err) {
+        systemError = err;
+    }
+
+
+    /**
+     * The whole testsuite started.
+     * @param suite the test suite
+     */
+    public void startTestSuite(JUnitTest suite) {
+        if (output == null) {
+            return; // Quick return - no output do nothing.
+        }
+        StringBuffer sb = new StringBuffer("Testsuite: ");
+        String n = suite.getName();
+        if (n != null && !tag.isEmpty())
+            n = n + "-" + tag;
+        sb.append(n);
+        sb.append(StringUtils.LINE_SEP);
+        try {
+            output.write(sb.toString());
+            output.flush();
+        } catch (IOException ex) {
+            throw new BuildException(ex);
+        }
+    }
+
+    /**
+     * The whole testsuite ended.
+     * @param suite the test suite
+     */
+    public void endTestSuite(JUnitTest suite) {
+        StringBuffer sb = new StringBuffer("Tests run: ");
+        sb.append(suite.runCount());
+        sb.append(", Failures: ");
+        sb.append(suite.failureCount());
+        sb.append(", Errors: ");
+        sb.append(suite.errorCount());
+        sb.append(", Skipped: ");
+        sb.append(suite.skipCount());
+        sb.append(", Time elapsed: ");
+        sb.append(numberFormat.format(suite.getRunTime() / ONE_SECOND));
+        sb.append(" sec");
+        sb.append(StringUtils.LINE_SEP);
+        sb.append(StringUtils.LINE_SEP);
+
+        // append the err and output streams to the log
+        if (systemOutput != null && systemOutput.length() > 0) {
+            sb.append("------------- Standard Output ---------------")
+                    .append(StringUtils.LINE_SEP)
+                    .append(systemOutput)
+                    .append("------------- ---------------- ---------------")
+                    .append(StringUtils.LINE_SEP);
+        }
+
+        if (systemError != null && systemError.length() > 0) {
+            sb.append("------------- Standard Error -----------------")
+                    .append(StringUtils.LINE_SEP)
+                    .append(systemError)
+                    .append("------------- ---------------- ---------------")
+                    .append(StringUtils.LINE_SEP);
+        }
+
+        if (output != null) {
+            try {
+                output.write(sb.toString());
+                resultWriter.close();
+                output.write(results.toString());
+            } catch (IOException ex) {
+                throw new BuildException(ex);
+            } finally {
+                try {
+                    output.flush();
+                } catch (IOException ex) {
+                    // swallow, there has likely been an exception before this
+                }
+                if (out != System.out && out != System.err) {
+                    FileUtils.close(out);
+                }
+            }
+        }
+    }
+
+    /**
+     * A test started.
+     * @param test a test
+     */
+    public void startTest(Test test) {
+    }
+
+    /**
+     * A test ended.
+     * @param test a test
+     */
+    public void endTest(Test test) {
+    }
+
+    /**
+     * Interface TestListener for JUnit &lt;= 3.4.
+     *
+     * <p>A Test failed.
+     * @param test a test
+     * @param t    the exception thrown by the test
+     */
+    public void addFailure(Test test, Throwable t) {
+        formatError("\tFAILED", test, t);
+    }
+
+    /**
+     * Interface TestListener for JUnit &gt; 3.4.
+     *
+     * <p>A Test failed.
+     * @param test a test
+     * @param t    the assertion failed by the test
+     */
+    public void addFailure(Test test, AssertionFailedError t) {
+        addFailure(test, (Throwable) t);
+    }
+
+    /**
+     * A test caused an error.
+     * @param test  a test
+     * @param error the error thrown by the test
+     */
+    public void addError(Test test, Throwable error) {
+        formatError("\tCaused an ERROR", test, error);
+    }
+
+    /**
+     * Format the test for printing..
+     * @param test a test
+     * @return the formatted testname
+     */
+    protected String formatTest(Test test) {
+        if (test == null) {
+            return "Null Test: ";
+        } else {
+            if (!tag.isEmpty())
+                return "Testcase: " + test.toString() + "-" + tag + ":";
+            return "Testcase: " + test.toString() + ":";
+        }
+    }
+
+    /**
+     * Format an error and print it.
+     * @param type the type of error
+     * @param test the test that failed
+     * @param error the exception that the test threw
+     */
+    protected synchronized void formatError(String type, Test test,
+                                            Throwable error) {
+        if (test != null) {
+            endTest(test);
+        }
+
+        try {
+            resultWriter.write(formatTest(test) + type);
+            resultWriter.newLine();
+            resultWriter.write(String.valueOf(error.getMessage()));
+            resultWriter.newLine();
+            String strace = JUnitTestRunner.getFilteredTrace(error);
+            resultWriter.write(strace);
+            resultWriter.newLine();
+            resultWriter.newLine();
+        } catch (IOException ex) {
+            throw new BuildException(ex);
+        }
+    }
+
+
+    public void testIgnored(Test test) {
+        formatSkip(test, JUnitVersionHelper.getIgnoreMessage(test));
+    }
+
+
+    public void formatSkip(Test test, String message) {
+        if (test != null) {
+            endTest(test);
+        }
+
+        try {
+            resultWriter.write(formatTest(test) + "SKIPPED");
+            if (message != null) {
+                resultWriter.write(": ");
+                resultWriter.write(message);
+            }
+            resultWriter.newLine();
+        } catch (IOException ex) {
+            throw new BuildException(ex);
+        }
+
+    }
+
+    public void testAssumptionFailure(Test test, Throwable cause) {
+        formatSkip(test, cause.getMessage());
+    }
+}
\ No newline at end of file
diff --git a/test/unit/org/apache/cassandra/CassandraXMLJUnitResultFormatter.java b/test/unit/org/apache/cassandra/CassandraXMLJUnitResultFormatter.java
new file mode 100644
index 0000000..066315d
--- /dev/null
+++ b/test/unit/org/apache/cassandra/CassandraXMLJUnitResultFormatter.java
@@ -0,0 +1,380 @@
+/*
+ *  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;
+
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.Writer;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Date;
+import java.util.Enumeration;
+import java.util.Hashtable;
+import java.util.Properties;
+
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+
+import junit.framework.AssertionFailedError;
+import junit.framework.Test;
+
+import org.apache.tools.ant.BuildException;
+import org.apache.tools.ant.taskdefs.optional.junit.IgnoredTestListener;
+import org.apache.tools.ant.taskdefs.optional.junit.JUnitResultFormatter;
+import org.apache.tools.ant.taskdefs.optional.junit.JUnitTest;
+import org.apache.tools.ant.taskdefs.optional.junit.JUnitTestRunner;
+import org.apache.tools.ant.taskdefs.optional.junit.JUnitVersionHelper;
+import org.apache.tools.ant.taskdefs.optional.junit.XMLConstants;
+import org.apache.tools.ant.util.DOMElementWriter;
+import org.apache.tools.ant.util.DateUtils;
+import org.apache.tools.ant.util.FileUtils;
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.Text;
+
+
+/**
+ * Prints XML output of the test to a specified Writer.
+ *
+ * @see FormatterElement
+ */
+
+public class CassandraXMLJUnitResultFormatter implements JUnitResultFormatter, XMLConstants, IgnoredTestListener {
+
+    private static final double ONE_SECOND = 1000.0;
+
+    /** constant for unnnamed testsuites/cases */
+    private static final String UNKNOWN = "unknown";
+
+    private static DocumentBuilder getDocumentBuilder() {
+        try {
+            return DocumentBuilderFactory.newInstance().newDocumentBuilder();
+        } catch (final Exception exc) {
+            throw new ExceptionInInitializerError(exc);
+        }
+    }
+
+    private static final String tag = System.getProperty("cassandra.testtag", "");
+
+    /**
+     * The XML document.
+     */
+    private Document doc;
+
+    /**
+     * The wrapper for the whole testsuite.
+     */
+    private Element rootElement;
+
+    /**
+     * Element for the current test.
+     *
+     * The keying of this map is a bit of a hack: tests are keyed by caseName(className) since
+     * the Test we get for Test-start isn't the same as the Test we get during test-assumption-fail,
+     * so we can't easily match Test objects without manually iterating over all keys and checking
+     * individual fields.
+     */
+    private final Hashtable<String, Element> testElements = new Hashtable<String, Element>();
+
+    /**
+     * tests that failed.
+     */
+    private final Hashtable failedTests = new Hashtable();
+
+    /**
+     * Tests that were skipped.
+     */
+    private final Hashtable<String, Test> skippedTests = new Hashtable<String, Test>();
+    /**
+     * Tests that were ignored. See the note above about the key being a bit of a hack.
+     */
+    private final Hashtable<String, Test> ignoredTests = new Hashtable<String, Test>();
+    /**
+     * Timing helper.
+     */
+    private final Hashtable<String, Long> testStarts = new Hashtable<String, Long>();
+    /**
+     * Where to write the log to.
+     */
+    private OutputStream out;
+
+    /** No arg constructor. */
+    public CassandraXMLJUnitResultFormatter() {
+    }
+
+    /** {@inheritDoc}. */
+    public void setOutput(final OutputStream out) {
+        this.out = out;
+    }
+
+    /** {@inheritDoc}. */
+    public void setSystemOutput(final String out) {
+        formatOutput(SYSTEM_OUT, out);
+    }
+
+    /** {@inheritDoc}. */
+    public void setSystemError(final String out) {
+        formatOutput(SYSTEM_ERR, out);
+    }
+
+    /**
+     * The whole testsuite started.
+     * @param suite the testsuite.
+     */
+    public void startTestSuite(final JUnitTest suite) {
+        doc = getDocumentBuilder().newDocument();
+        rootElement = doc.createElement(TESTSUITE);
+        String n = suite.getName();
+//        if (n != null && !tag.isEmpty())
+//            n = n + "-" + tag;
+        rootElement.setAttribute(ATTR_NAME, n == null ? UNKNOWN : n);
+
+        //add the timestamp
+        final String timestamp = DateUtils.format(new Date(),
+                DateUtils.ISO8601_DATETIME_PATTERN);
+        rootElement.setAttribute(TIMESTAMP, timestamp);
+        //and the hostname.
+        rootElement.setAttribute(HOSTNAME, getHostname());
+
+        // Output properties
+        final Element propsElement = doc.createElement(PROPERTIES);
+        rootElement.appendChild(propsElement);
+        final Properties props = suite.getProperties();
+        if (props != null) {
+            final Enumeration e = props.propertyNames();
+            while (e.hasMoreElements()) {
+                final String name = (String) e.nextElement();
+                final Element propElement = doc.createElement(PROPERTY);
+                propElement.setAttribute(ATTR_NAME, name);
+                propElement.setAttribute(ATTR_VALUE, props.getProperty(name));
+                propsElement.appendChild(propElement);
+            }
+        }
+    }
+
+    /**
+     * get the local hostname
+     * @return the name of the local host, or "localhost" if we cannot work it out
+     */
+    private String getHostname()  {
+        String hostname = "localhost";
+        try {
+            final InetAddress localHost = InetAddress.getLocalHost();
+            if (localHost != null) {
+                hostname = localHost.getHostName();
+            }
+        } catch (final UnknownHostException e) {
+            // fall back to default 'localhost'
+        }
+        return hostname;
+    }
+
+    /**
+     * The whole testsuite ended.
+     * @param suite the testsuite.
+     * @throws BuildException on error.
+     */
+    public void endTestSuite(final JUnitTest suite) throws BuildException {
+        rootElement.setAttribute(ATTR_TESTS, "" + suite.runCount());
+        rootElement.setAttribute(ATTR_FAILURES, "" + suite.failureCount());
+        rootElement.setAttribute(ATTR_ERRORS, "" + suite.errorCount());
+        rootElement.setAttribute(ATTR_SKIPPED, "" + suite.skipCount());
+        rootElement.setAttribute(
+            ATTR_TIME, "" + (suite.getRunTime() / ONE_SECOND));
+        if (out != null) {
+            Writer wri = null;
+            try {
+                wri = new BufferedWriter(new OutputStreamWriter(out, "UTF8"));
+                wri.write("<?xml version=\"1.0\" encoding=\"UTF-8\" ?>\n");
+                (new DOMElementWriter()).write(rootElement, wri, 0, "  ");
+            } catch (final IOException exc) {
+                throw new BuildException("Unable to write log file", exc);
+            } finally {
+                if (wri != null) {
+                    try {
+                        wri.flush();
+                    } catch (final IOException ex) {
+                        // ignore
+                    }
+                }
+                if (out != System.out && out != System.err) {
+                    FileUtils.close(wri);
+                }
+            }
+        }
+    }
+
+    /**
+     * Interface TestListener.
+     *
+     * <p>A new Test is started.
+     * @param t the test.
+     */
+    public void startTest(final Test t) {
+        testStarts.put(createDescription(t), System.currentTimeMillis());
+    }
+
+    private static String createDescription(final Test test) throws BuildException {
+        if (!tag.isEmpty())
+            return JUnitVersionHelper.getTestCaseName(test) + "-" + tag +"(" + JUnitVersionHelper.getTestCaseClassName(test) + ")";
+        return JUnitVersionHelper.getTestCaseName(test) + "(" + JUnitVersionHelper.getTestCaseClassName(test) + ")";
+    }
+
+    /**
+     * Interface TestListener.
+     *
+     * <p>A Test is finished.
+     * @param test the test.
+     */
+    public void endTest(final Test test) {
+        final String testDescription = createDescription(test);
+
+        // Fix for bug #5637 - if a junit.extensions.TestSetup is
+        // used and throws an exception during setUp then startTest
+        // would never have been called
+        if (!testStarts.containsKey(testDescription)) {
+            startTest(test);
+        }
+        Element currentTest;
+        if (!failedTests.containsKey(test) && !skippedTests.containsKey(testDescription) && !ignoredTests.containsKey(testDescription)) {
+            currentTest = doc.createElement(TESTCASE);
+            String n = JUnitVersionHelper.getTestCaseName(test);
+            if (n != null && !tag.isEmpty())
+                n = n + "-" + tag;
+            currentTest.setAttribute(ATTR_NAME,
+                                     n == null ? UNKNOWN : n);
+            // a TestSuite can contain Tests from multiple classes,
+            // even tests with the same name - disambiguate them.
+            currentTest.setAttribute(ATTR_CLASSNAME,
+                    JUnitVersionHelper.getTestCaseClassName(test));
+            rootElement.appendChild(currentTest);
+            testElements.put(createDescription(test), currentTest);
+        } else {
+            currentTest = testElements.get(testDescription);
+        }
+
+        final Long l = testStarts.get(createDescription(test));
+        currentTest.setAttribute(ATTR_TIME,
+            "" + ((System.currentTimeMillis() - l) / ONE_SECOND));
+    }
+
+    /**
+     * Interface TestListener for JUnit &lt;= 3.4.
+     *
+     * <p>A Test failed.
+     * @param test the test.
+     * @param t the exception.
+     */
+    public void addFailure(final Test test, final Throwable t) {
+        formatError(FAILURE, test, t);
+    }
+
+    /**
+     * Interface TestListener for JUnit &gt; 3.4.
+     *
+     * <p>A Test failed.
+     * @param test the test.
+     * @param t the assertion.
+     */
+    public void addFailure(final Test test, final AssertionFailedError t) {
+        addFailure(test, (Throwable) t);
+    }
+
+    /**
+     * Interface TestListener.
+     *
+     * <p>An error occurred while running the test.
+     * @param test the test.
+     * @param t the error.
+     */
+    public void addError(final Test test, final Throwable t) {
+        formatError(ERROR, test, t);
+    }
+
+    private void formatError(final String type, final Test test, final Throwable t) {
+        if (test != null) {
+            endTest(test);
+            failedTests.put(test, test);
+        }
+
+        final Element nested = doc.createElement(type);
+        Element currentTest;
+        if (test != null) {
+            currentTest = testElements.get(createDescription(test));
+        } else {
+            currentTest = rootElement;
+        }
+
+        currentTest.appendChild(nested);
+
+        final String message = t.getMessage();
+        if (message != null && message.length() > 0) {
+            nested.setAttribute(ATTR_MESSAGE, t.getMessage());
+        }
+        nested.setAttribute(ATTR_TYPE, t.getClass().getName());
+
+        final String strace = JUnitTestRunner.getFilteredTrace(t);
+        final Text trace = doc.createTextNode(strace);
+        nested.appendChild(trace);
+    }
+
+    private void formatOutput(final String type, final String output) {
+        final Element nested = doc.createElement(type);
+        rootElement.appendChild(nested);
+        nested.appendChild(doc.createCDATASection(output));
+    }
+
+    public void testIgnored(final Test test) {
+        formatSkip(test, JUnitVersionHelper.getIgnoreMessage(test));
+        if (test != null) {
+            ignoredTests.put(createDescription(test), test);
+        }
+    }
+
+
+    public void formatSkip(final Test test, final String message) {
+        if (test != null) {
+            endTest(test);
+        }
+
+        final Element nested = doc.createElement("skipped");
+
+        if (message != null) {
+            nested.setAttribute("message", message);
+        }
+
+        Element currentTest;
+        if (test != null) {
+            currentTest = testElements.get(createDescription(test));
+        } else {
+            currentTest = rootElement;
+        }
+
+        currentTest.appendChild(nested);
+
+    }
+
+    public void testAssumptionFailure(final Test test, final Throwable failure) {
+        formatSkip(test, failure.getMessage());
+        skippedTests.put(createDescription(test), test);
+
+    }
+} // XMLJUnitResultFormatter
\ No newline at end of file
diff --git a/test/unit/org/apache/cassandra/MockSchema.java b/test/unit/org/apache/cassandra/MockSchema.java
new file mode 100644
index 0000000..c71c98b
--- /dev/null
+++ b/test/unit/org/apache/cassandra/MockSchema.java
@@ -0,0 +1,183 @@
+/*
+* 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;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+
+import org.apache.cassandra.cache.CachingOptions;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.composites.SimpleSparseCellNameType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.IndexSummary;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
+import org.apache.cassandra.io.sstable.metadata.MetadataType;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.io.util.BufferedSegmentedFile;
+import org.apache.cassandra.io.util.ChannelProxy;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.Memory;
+import org.apache.cassandra.io.util.SegmentedFile;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.utils.AlwaysPresentFilter;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class MockSchema
+{
+    static
+    {
+        Memory offsets = Memory.allocate(4);
+        offsets.setInt(0, 0);
+        indexSummary = new IndexSummary(Murmur3Partitioner.instance, offsets, 0, Memory.allocate(4), 0, 0, 0, 1);
+    }
+    private static final AtomicInteger id = new AtomicInteger();
+    public static final Keyspace ks = Keyspace.mockKS(new KSMetaData("mockks", SimpleStrategy.class, ImmutableMap.of("replication_factor", "1"), false));
+
+    private static final IndexSummary indexSummary;
+    private static final SegmentedFile segmentedFile = new BufferedSegmentedFile(new ChannelProxy(temp("mocksegmentedfile")), 0);
+
+    public static Memtable memtable(ColumnFamilyStore cfs)
+    {
+        return new Memtable(cfs.metadata);
+    }
+
+    public static SSTableReader sstable(int generation, ColumnFamilyStore cfs)
+    {
+        return sstable(generation, false, cfs);
+    }
+
+    public static SSTableReader sstable(int generation, boolean keepRef, ColumnFamilyStore cfs)
+    {
+        return sstable(generation, 0, keepRef, cfs);
+    }
+
+    public static SSTableReader sstable(int generation, int size, ColumnFamilyStore cfs)
+    {
+        return sstable(generation, size, false, cfs);
+    }
+
+    public static SSTableReader sstable(int generation, int size, boolean keepRef, ColumnFamilyStore cfs)
+    {
+        Descriptor descriptor = new Descriptor(cfs.directories.getDirectoryForNewSSTables(),
+                                               cfs.keyspace.getName(),
+                                               cfs.getColumnFamilyName(),
+                                               generation,
+                                               Descriptor.Type.FINAL);
+        Set<Component> components = ImmutableSet.of(Component.DATA, Component.PRIMARY_INDEX, Component.FILTER, Component.TOC);
+        for (Component component : components)
+        {
+            File file = new File(descriptor.filenameFor(component));
+            try
+            {
+                file.createNewFile();
+            }
+            catch (IOException e)
+            {
+            }
+        }
+        if (size > 0)
+        {
+            try
+            {
+                File file = new File(descriptor.filenameFor(Component.DATA));
+                try (RandomAccessFile raf = new RandomAccessFile(file, "rw"))
+                {
+                    raf.setLength(size);
+                }
+            }
+            catch (IOException e)
+            {
+                throw new RuntimeException(e);
+            }
+        }
+        StatsMetadata metadata = (StatsMetadata) new MetadataCollector(cfs.metadata.comparator)
+                                                 .finalizeMetadata(Murmur3Partitioner.instance.getClass().getCanonicalName(), 0.01f, -1)
+                                                 .get(MetadataType.STATS);
+        SSTableReader reader = SSTableReader.internalOpen(descriptor, components, cfs.metadata, Murmur3Partitioner.instance,
+                                                          segmentedFile.sharedCopy(), segmentedFile.sharedCopy(), indexSummary.sharedCopy(),
+                                                          new AlwaysPresentFilter(), 1L, metadata, SSTableReader.OpenReason.NORMAL);
+        reader.first = reader.last = readerBounds(generation);
+        if (!keepRef)
+            reader.selfRef().release();
+        return reader;
+    }
+
+    public static ColumnFamilyStore newCFS()
+    {
+        String cfname = "mockcf" + (id.incrementAndGet());
+        CFMetaData metadata = newCFMetaData(ks.getName(), cfname);
+        return new ColumnFamilyStore(ks, cfname, Murmur3Partitioner.instance, 0, metadata, new Directories(metadata), false, false);
+    }
+
+    private static CFMetaData newCFMetaData(String ksname, String cfname)
+    {
+        CFMetaData metadata = new CFMetaData(ksname,
+                                             cfname,
+                                             ColumnFamilyType.Standard,
+                                             new SimpleSparseCellNameType(UTF8Type.instance));
+        metadata.caching(CachingOptions.NONE);
+        return metadata;
+    }
+
+    public static BufferDecoratedKey readerBounds(int generation)
+    {
+        return new BufferDecoratedKey(new Murmur3Partitioner.LongToken(generation), ByteBufferUtil.EMPTY_BYTE_BUFFER);
+    }
+
+    private static File temp(String id)
+    {
+        try
+        {
+            File file = File.createTempFile(id, "tmp");
+            file.deleteOnExit();
+            return file;
+        }
+        catch (IOException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static void cleanup()
+    {
+        // clean up data directory which are stored as data directory/keyspace/data files
+        for (String dirName : DatabaseDescriptor.getAllDataFileLocations())
+        {
+            File dir = new File(dirName);
+            if (!dir.exists())
+                continue;
+            String[] children = dir.list();
+            for (String child : children)
+                FileUtils.deleteRecursive(new File(dir, child));
+        }
+    }
+}
diff --git a/test/unit/org/apache/cassandra/SchemaLoader.java b/test/unit/org/apache/cassandra/SchemaLoader.java
index c6a3855..45748a9 100644
--- a/test/unit/org/apache/cassandra/SchemaLoader.java
+++ b/test/unit/org/apache/cassandra/SchemaLoader.java
@@ -18,11 +18,11 @@
 package org.apache.cassandra;
 
 import java.io.File;
+import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.*;
 
 import org.junit.After;
-import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -53,21 +53,11 @@
     @BeforeClass
     public static void loadSchema() throws ConfigurationException
     {
-        loadSchema(null);
-    }
-
-    public static void loadSchema(Integer compressionChunkLength) throws ConfigurationException
-    {
         prepareServer();
 
         // Migrations aren't happy if gossiper is not started.  Even if we don't use migrations though,
         // some tests now expect us to start gossip for them.
         startGossiper();
-
-        // if you're messing with low-level sstable stuff, it can be useful to inject the schema directly
-        // Schema.instance.load(schemaDefinition());
-        for (KSMetaData ksm : schemaDefinition(compressionChunkLength))
-            MigrationManager.announceNewKeyspace(ksm);
     }
 
     @After
@@ -82,9 +72,15 @@
     public static void prepareServer()
     {
         // Cleanup first
-        cleanupAndLeaveDirs();
-
-        CommitLog.instance.allocator.enableReserveSegmentCreation();
+        try
+        {
+            cleanupAndLeaveDirs();
+        }
+        catch (IOException e)
+        {
+            logger.error("Failed to cleanup and recreate directories and files.");
+            throw new RuntimeException(e);
+        }
 
         Thread.setDefaultUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler()
         {
@@ -94,37 +90,33 @@
             }
         });
 
+        DatabaseDescriptor.setDaemonInitialized();
         Keyspace.setInitialized();
     }
 
     public static void startGossiper()
     {
-        Gossiper.instance.start((int) (System.currentTimeMillis() / 1000));
+        if (!Gossiper.instance.isEnabled())
+            Gossiper.instance.start((int) (System.currentTimeMillis() / 1000));
     }
 
-    @AfterClass
-    public static void stopGossiper()
-    {
-        Gossiper.instance.stop();
-    }
-
-    public static Collection<KSMetaData> schemaDefinition(Integer compressionChunkLength) throws ConfigurationException
+    public static void schemaDefinition(String testName) throws ConfigurationException
     {
         List<KSMetaData> schema = new ArrayList<KSMetaData>();
 
         // A whole bucket of shorthand
-        String ks1 = "Keyspace1";
-        String ks2 = "Keyspace2";
-        String ks3 = "Keyspace3";
-        String ks4 = "Keyspace4";
-        String ks5 = "Keyspace5";
-        String ks6 = "Keyspace6";
-        String ks_kcs = "KeyCacheSpace";
-        String ks_rcs = "RowCacheSpace";
-        String ks_ccs = "CounterCacheSpace";
-        String ks_nocommit = "NoCommitlogSpace";
-        String ks_prsi = "PerRowSecondaryIndex";
-        String ks_cql = "cql_keyspace";
+        String ks1 = testName + "Keyspace1";
+        String ks2 = testName + "Keyspace2";
+        String ks3 = testName + "Keyspace3";
+        String ks4 = testName + "Keyspace4";
+        String ks5 = testName + "Keyspace5";
+        String ks6 = testName + "Keyspace6";
+        String ks_kcs = testName + "KeyCacheSpace";
+        String ks_rcs = testName + "RowCacheSpace";
+        String ks_ccs = testName + "CounterCacheSpace";
+        String ks_nocommit = testName + "NoCommitlogSpace";
+        String ks_prsi = testName + "PerRowSecondaryIndex";
+        String ks_cql = testName + "cql_keyspace";
 
         Class<? extends AbstractReplicationStrategy> simple = SimpleStrategy.class;
 
@@ -188,15 +180,15 @@
                                            standardCFMD(ks1, "StandardLeveled")
                                                                                .compactionStrategyClass(LeveledCompactionStrategy.class)
                                                                                .compactionStrategyOptions(leveledOptions),
+                                           standardCFMD(ks1, "StandardLeveled2")
+                                                                               .compactionStrategyClass(LeveledCompactionStrategy.class)
+                                                                               .compactionStrategyOptions(leveledOptions),
                                            standardCFMD(ks1, "legacyleveled")
                                                                                .compactionStrategyClass(LeveledCompactionStrategy.class)
                                                                                .compactionStrategyOptions(leveledOptions),
                                            standardCFMD(ks1, "StandardLowIndexInterval").minIndexInterval(8)
                                                                                         .maxIndexInterval(256)
                                                                                         .caching(CachingOptions.NONE),
-                                           standardCFMD(ks1, "StandardRace").minIndexInterval(8)
-                                                                            .maxIndexInterval(256)
-                                                                            .caching(CachingOptions.NONE),
                                            standardCFMD(ks1, "UUIDKeys").keyValidator(UUIDType.instance),
                                            CFMetaData.denseCFMetaData(ks1, "MixedTypes", LongType.instance).keyValidator(UUIDType.instance).defaultValidator(BooleanType.instance),
                                            CFMetaData.denseCFMetaData(ks1, "MixedTypesComposite", composite).keyValidator(composite).defaultValidator(BooleanType.instance),
@@ -324,12 +316,35 @@
 
 
         if (Boolean.parseBoolean(System.getProperty("cassandra.test.compression", "false")))
-            useCompression(schema, compressionChunkLength);
+            useCompression(schema);
 
-        return schema;
+        // if you're messing with low-level sstable stuff, it can be useful to inject the schema directly
+        // Schema.instance.load(schemaDefinition());
+        for (KSMetaData ksm : schema)
+            MigrationManager.announceNewKeyspace(ksm, false);
     }
 
-    private static ColumnDefinition integerColumn(String ksName, String cfName)
+    public static void createKeyspace(String keyspaceName,
+                                      Class<? extends AbstractReplicationStrategy> strategy,
+                                      Map<String, String> options,
+                                      CFMetaData... cfmetas) throws ConfigurationException
+    {
+        createKeyspace(keyspaceName, true, true, strategy, options, cfmetas);
+    }
+
+    public static void createKeyspace(String keyspaceName,
+                                      boolean durable,
+                                      boolean announceLocally,
+                                      Class<? extends AbstractReplicationStrategy> strategy,
+                                      Map<String, String> options,
+                                      CFMetaData... cfmetas) throws ConfigurationException
+    {
+        KSMetaData ksm = durable ? KSMetaData.testMetadata(keyspaceName, strategy, options, cfmetas)
+                                 : KSMetaData.testMetadataNotDurable(keyspaceName, strategy, options, cfmetas);
+        MigrationManager.announceNewKeyspace(ksm, announceLocally);
+    }
+
+    public static ColumnDefinition integerColumn(String ksName, String cfName)
     {
         return new ColumnDefinition(ksName,
                                     cfName,
@@ -355,7 +370,7 @@
                                     ColumnDefinition.Kind.REGULAR);
     }
 
-    private static CFMetaData perRowIndexedCFMD(String ksName, String cfName)
+    public static CFMetaData perRowIndexedCFMD(String ksName, String cfName)
     {
         final Map<String, String> indexOptions = Collections.singletonMap(
                                                       SecondaryIndex.CUSTOM_INDEX_OPTION_NAME,
@@ -368,41 +383,49 @@
                                                                 .setIndex("indexe1", IndexType.CUSTOM, indexOptions));
     }
 
-    private static void useCompression(List<KSMetaData> schema, Integer chunkLength) throws ConfigurationException
+    private static void useCompression(List<KSMetaData> schema)
     {
         for (KSMetaData ksm : schema)
         {
             for (CFMetaData cfm : ksm.cfMetaData().values())
             {
-                cfm.compressionParameters(new CompressionParameters(SnappyCompressor.instance,
-                                                                    chunkLength,
-                                                                    Collections.<String, String>emptyMap()));
+                cfm.compressionParameters(new CompressionParameters(SnappyCompressor.instance));
             }
         }
     }
 
-    private static CFMetaData standardCFMD(String ksName, String cfName)
+    public static CFMetaData standardCFMD(String ksName, String cfName)
     {
-        return CFMetaData.denseCFMetaData(ksName, cfName, BytesType.instance);
+        return CFMetaData.denseCFMetaData(ksName, cfName, BytesType.instance).compressionParameters(getCompressionParameters());
     }
-    private static CFMetaData superCFMD(String ksName, String cfName, AbstractType subcc)
+
+    public static CFMetaData standardCFMD(String ksName, String cfName, AbstractType<?> comparator)
     {
-        return superCFMD(ksName, cfName, BytesType.instance, subcc);
+        return CFMetaData.denseCFMetaData(ksName, cfName, comparator).compressionParameters(getCompressionParameters());
     }
-    private static CFMetaData superCFMD(String ksName, String cfName, AbstractType cc, AbstractType subcc)
+
+    public static CFMetaData superCFMD(String ksName, String cfName, AbstractType subcc)
     {
-        return CFMetaData.denseCFMetaData(ksName, cfName, cc, subcc);
+        return superCFMD(ksName, cfName, BytesType.instance, subcc).compressionParameters(getCompressionParameters());
     }
-    private static CFMetaData indexCFMD(String ksName, String cfName, final Boolean withIdxType) throws ConfigurationException
+
+    public static CFMetaData superCFMD(String ksName, String cfName, AbstractType cc, AbstractType subcc)
+    {
+        return CFMetaData.denseCFMetaData(ksName, cfName, cc, subcc).compressionParameters(getCompressionParameters());
+    }
+
+    public static CFMetaData indexCFMD(String ksName, String cfName, final Boolean withIdxType) throws ConfigurationException
     {
         CFMetaData cfm = CFMetaData.sparseCFMetaData(ksName, cfName, BytesType.instance).keyValidator(AsciiType.instance);
 
         ByteBuffer cName = ByteBufferUtil.bytes("birthdate");
         IndexType keys = withIdxType ? IndexType.KEYS : null;
         return cfm.addColumnDefinition(ColumnDefinition.regularDef(cfm, cName, LongType.instance, null)
-                                                       .setIndex(withIdxType ? ByteBufferUtil.bytesToHex(cName) : null, keys, null));
+                                                       .setIndex(withIdxType ? ByteBufferUtil.bytesToHex(cName) : null, keys, null))
+                                      .compressionParameters(getCompressionParameters());
     }
-    private static CFMetaData compositeIndexCFMD(String ksName, String cfName, final Boolean withIdxType) throws ConfigurationException
+
+    public static CFMetaData compositeIndexCFMD(String ksName, String cfName, final Boolean withIdxType) throws ConfigurationException
     {
         final CompositeType composite = CompositeType.getInstance(Arrays.asList(new AbstractType<?>[]{UTF8Type.instance, UTF8Type.instance})); 
         CFMetaData cfm = CFMetaData.sparseCFMetaData(ksName, cfName, composite);
@@ -410,25 +433,41 @@
         ByteBuffer cName = ByteBufferUtil.bytes("col1");
         IndexType idxType = withIdxType ? IndexType.COMPOSITES : null;
         return cfm.addColumnDefinition(ColumnDefinition.regularDef(cfm, cName, UTF8Type.instance, 1)
-                                                       .setIndex(withIdxType ? "col1_idx" : null, idxType, Collections.<String, String>emptyMap()));
+                                                       .setIndex(withIdxType ? "col1_idx" : null, idxType, Collections.<String, String>emptyMap()))
+                                       .compressionParameters(getCompressionParameters());
     }
     
     private static CFMetaData jdbcCFMD(String ksName, String cfName, AbstractType comp)
     {
-        return CFMetaData.denseCFMetaData(ksName, cfName, comp).defaultValidator(comp);
+        return CFMetaData.denseCFMetaData(ksName, cfName, comp).defaultValidator(comp).compressionParameters(getCompressionParameters());
     }
 
-    private static CFMetaData jdbcSparseCFMD(String ksName, String cfName, AbstractType comp)
+    public static CFMetaData jdbcSparseCFMD(String ksName, String cfName, AbstractType comp)
     {
-        return CFMetaData.sparseCFMetaData(ksName, cfName, comp).defaultValidator(comp);
+        return CFMetaData.sparseCFMetaData(ksName, cfName, comp).defaultValidator(comp).compressionParameters(getCompressionParameters());
     }
 
-    public static void cleanupAndLeaveDirs()
+    public static CompressionParameters getCompressionParameters()
     {
+        return getCompressionParameters(null);
+    }
+
+    public static CompressionParameters getCompressionParameters(Integer chunkSize)
+    {
+        if (Boolean.parseBoolean(System.getProperty("cassandra.test.compression", "false")))
+            return new CompressionParameters(SnappyCompressor.instance, chunkSize, Collections.<String, String>emptyMap());
+        else
+            return new CompressionParameters(null);
+    }
+
+    public static void cleanupAndLeaveDirs() throws IOException
+    {
+        // We need to stop and unmap all CLS instances prior to cleanup() or we'll get failures on Windows.
+        CommitLog.instance.stopUnsafe(true);
         mkdirs();
         cleanup();
         mkdirs();
-        CommitLog.instance.resetUnsafe(); // cleanup screws w/ CommitLog, this brings it back to safe state
+        CommitLog.instance.restartUnsafe();
     }
 
     public static void cleanup()
@@ -440,7 +479,11 @@
             File dir = new File(dirName);
             if (!dir.exists())
                 throw new RuntimeException("No such directory: " + dir.getAbsolutePath());
-            FileUtils.deleteRecursive(dir);
+
+            // Leave the folder around as Windows will complain about directory deletion w/handles open to children files
+            String[] children = dir.list();
+            for (String child : children)
+                FileUtils.deleteRecursive(new File(dir, child));
         }
 
         cleanupSavedCaches();
@@ -451,7 +494,9 @@
             File dir = new File(dirName);
             if (!dir.exists())
                 throw new RuntimeException("No such directory: " + dir.getAbsolutePath());
-            FileUtils.deleteRecursive(dir);
+            String[] children = dir.list();
+            for (String child : children)
+                FileUtils.deleteRecursive(new File(dir, child));
         }
     }
 
@@ -460,7 +505,7 @@
         DatabaseDescriptor.createAllDirectories();
     }
 
-    protected void insertData(String keyspace, String columnFamily, int offset, int numberOfRows)
+    public static void insertData(String keyspace, String columnFamily, int offset, int numberOfRows)
     {
         for (int i = offset; i < offset + numberOfRows; i++)
         {
@@ -472,7 +517,7 @@
     }
 
     /* usually used to populate the cache */
-    protected void readData(String keyspace, String columnFamily, int offset, int numberOfRows)
+    public static void readData(String keyspace, String columnFamily, int offset, int numberOfRows)
     {
         ColumnFamilyStore store = Keyspace.open(keyspace).getColumnFamilyStore(columnFamily);
         for (int i = offset; i < offset + numberOfRows; i++)
@@ -482,7 +527,7 @@
         }
     }
 
-    protected static void cleanupSavedCaches()
+    public static void cleanupSavedCaches()
     {
         File cachesDir = new File(DatabaseDescriptor.getSavedCachesLocation());
 
diff --git a/test/unit/org/apache/cassandra/Util.java b/test/unit/org/apache/cassandra/Util.java
index 3c2d32c..f6b4771 100644
--- a/test/unit/org/apache/cassandra/Util.java
+++ b/test/unit/org/apache/cassandra/Util.java
@@ -24,16 +24,12 @@
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.SortedSet;
-import java.util.TreeSet;
-import java.util.UUID;
+import java.util.*;
 import java.util.concurrent.Callable;
 import java.util.concurrent.Future;
 
+import com.google.common.base.Supplier;
+
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.db.compaction.AbstractCompactionTask;
@@ -45,16 +41,17 @@
 import org.apache.cassandra.db.filter.NamesQueryFilter;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.dht.*;
+import org.apache.cassandra.dht.RandomPartitioner.BigIntegerToken;
 import org.apache.cassandra.gms.ApplicationState;
 import org.apache.cassandra.gms.Gossiper;
 import org.apache.cassandra.gms.VersionedValue;
 import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.sstable.SSTableReader;
-import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.CounterId;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 public class Util
@@ -66,7 +63,7 @@
         return StorageService.getPartitioner().decorateKey(ByteBufferUtil.bytes(key));
     }
 
-    public static DecoratedKey dk(String key, AbstractType type)
+    public static DecoratedKey dk(String key, AbstractType<?> type)
     {
         return StorageService.getPartitioner().decorateKey(type.fromString(key));
     }
@@ -117,6 +114,16 @@
         return new BufferCell(cellname(name), ByteBufferUtil.bytes(value), timestamp);
     }
 
+    public static Cell column(String name, long value, long timestamp)
+    {
+        return new BufferCell(cellname(name), ByteBufferUtil.bytes(value), timestamp);
+    }
+
+    public static Cell column(String clusterKey, String name, long value, long timestamp)
+    {
+        return new BufferCell(cellname(clusterKey, name), ByteBufferUtil.bytes(value), timestamp);
+    }
+
     public static Cell expiringColumn(String name, String value, long timestamp, int ttl)
     {
         return new BufferExpiringCell(cellname(name), ByteBufferUtil.bytes(value), timestamp, ttl);
@@ -168,6 +175,24 @@
         return bb;
     }
 
+    public static ByteBuffer getBytes(short v)
+    {
+        byte[] bytes = new byte[2];
+        ByteBuffer bb = ByteBuffer.wrap(bytes);
+        bb.putShort(v);
+        bb.rewind();
+        return bb;
+    }
+
+    public static ByteBuffer getBytes(byte v)
+    {
+        byte[] bytes = new byte[1];
+        ByteBuffer bb = ByteBuffer.wrap(bytes);
+        bb.put(v);
+        bb.rewind();
+        return bb;
+    }
+
     public static List<Row> getRangeSlice(ColumnFamilyStore cfs)
     {
         return getRangeSlice(cfs, null);
@@ -180,7 +205,7 @@
                                : new SliceQueryFilter(SuperColumns.startOf(superColumn), SuperColumns.endOf(superColumn), false, Integer.MAX_VALUE);
 
         Token min = StorageService.getPartitioner().getMinimumToken();
-        return cfs.getRangeSlice(new Bounds<Token>(min, min).toRowBounds(), null, filter, 10000);
+        return cfs.getRangeSlice(Bounds.makeRowBounds(min, min), null, filter, 10000);
     }
 
     /**
@@ -196,7 +221,7 @@
         UUID cfid = first.getColumnFamilyIds().iterator().next();
 
         for (Mutation rm : mutations)
-            rm.apply();
+            rm.applyUnsafe();
 
         ColumnFamilyStore store = Keyspace.open(keyspaceName).getColumnFamilyStore(cfid);
         store.forceBlockingFlush();
@@ -206,31 +231,10 @@
     public static ColumnFamily getColumnFamily(Keyspace keyspace, DecoratedKey key, String cfName)
     {
         ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore(cfName);
-        assert cfStore != null : "Column family " + cfName + " has not been defined";
+        assert cfStore != null : "Table " + cfName + " has not been defined";
         return cfStore.getColumnFamily(QueryFilter.getIdentityFilter(key, cfName, System.currentTimeMillis()));
     }
 
-    public static byte[] concatByteArrays(byte[] first, byte[]... remaining)
-    {
-        int length = first.length;
-        for (byte[] array : remaining)
-        {
-            length += array.length;
-        }
-
-        byte[] result = new byte[length];
-        System.arraycopy(first, 0, result, 0, first.length);
-        int offset = first.length;
-
-        for (byte[] array : remaining)
-        {
-            System.arraycopy(array, 0, result, offset, array.length);
-            offset += array.length;
-        }
-
-        return result;
-    }
-
     public static boolean equalsCounterId(CounterId n, ByteBuffer context, int offset)
     {
         return CounterId.wrap(context, context.position() + offset).equals(n);
@@ -314,22 +318,6 @@
         assert thrown : exception.getName() + " not received";
     }
 
-    public static ByteBuffer serializeForSSTable(ColumnFamily cf)
-    {
-        try
-        {
-            DataOutputBuffer out = new DataOutputBuffer();
-            DeletionTime.serializer.serialize(cf.deletionInfo().getTopLevelDeletion(), out);
-            out.writeInt(cf.getColumnCount());
-            new ColumnIndex.Builder(cf, ByteBufferUtil.EMPTY_BYTE_BUFFER, out).build(cf);
-            return ByteBuffer.wrap(out.toByteArray());
-        }
-        catch (IOException e)
-        {
-            throw new RuntimeException(e);
-        }
-    }
-
     public static QueryFilter namesQueryFilter(ColumnFamilyStore cfs, DecoratedKey key)
     {
         SortedSet<CellName> s = new TreeSet<CellName>(cfs.getComparator());
@@ -379,6 +367,18 @@
         return new RangeTombstone(startName, endName, timestamp , localtime);
     }
 
+    public static void spinAssertEquals(Object expected, Supplier<Object> s, int timeoutInSeconds)
+    {
+        long start = System.currentTimeMillis();
+        while (System.currentTimeMillis() < start + (1000 * timeoutInSeconds))
+        {
+            if (s.get().equals(expected))
+                break;
+            Thread.yield();
+        }
+        assertEquals(expected, s.get());
+    }
+
     public static void joinThread(Thread thread) throws InterruptedException
     {
         thread.join(10000);
diff --git a/test/unit/org/apache/cassandra/auth/RoleOptionsTest.java b/test/unit/org/apache/cassandra/auth/RoleOptionsTest.java
new file mode 100644
index 0000000..6dea2b5
--- /dev/null
+++ b/test/unit/org/apache/cassandra/auth/RoleOptionsTest.java
@@ -0,0 +1,226 @@
+/*
+ * 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.auth;
+
+import java.lang.reflect.Field;
+import java.util.*;
+
+import com.google.common.collect.ImmutableSet;
+import org.junit.Test;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.exceptions.*;
+import org.apache.cassandra.utils.FBUtilities;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class RoleOptionsTest
+{
+    @Test
+    public void validateValueTypes()
+    {
+        setupRoleManager(getRoleManager(IRoleManager.Option.values()));
+
+        RoleOptions opts = new RoleOptions();
+        opts.setOption(IRoleManager.Option.LOGIN, "test");
+        assertInvalidOptions(opts, "Invalid value for property 'LOGIN'. It must be a boolean");
+
+        opts = new RoleOptions();
+        opts.setOption(IRoleManager.Option.PASSWORD, 99);
+        assertInvalidOptions(opts, "Invalid value for property 'PASSWORD'. It must be a string");
+
+        opts = new RoleOptions();
+        opts.setOption(IRoleManager.Option.SUPERUSER, new HashSet<>());
+        assertInvalidOptions(opts, "Invalid value for property 'SUPERUSER'. It must be a boolean");
+
+        opts = new RoleOptions();
+        opts.setOption(IRoleManager.Option.OPTIONS, false);
+        assertInvalidOptions(opts, "Invalid value for property 'OPTIONS'. It must be a map");
+
+        opts = new RoleOptions();
+        opts.setOption(IRoleManager.Option.LOGIN, true);
+        opts.setOption(IRoleManager.Option.SUPERUSER, false);
+        opts.setOption(IRoleManager.Option.PASSWORD, "test");
+        opts.setOption(IRoleManager.Option.OPTIONS, Collections.singletonMap("key", "value"));
+        opts.validate();
+    }
+
+    @Test
+    public void rejectUnsupportedOptions()
+    {
+        // Our hypothetical IRoleManager only supports the LOGIN option
+        IRoleManager roleManager = getRoleManager(IRoleManager.Option.LOGIN);
+        setupRoleManager(roleManager);
+        RoleOptions opts = new RoleOptions();
+        opts.setOption(IRoleManager.Option.PASSWORD, "test");
+        assertInvalidOptions(opts, String.format("%s doesn't support PASSWORD", roleManager.getClass().getName()));
+    }
+
+    @Test
+    public void rejectSettingSameOptionMultipleTimes()
+    {
+        RoleOptions opts = new RoleOptions();
+        opts.setOption(IRoleManager.Option.LOGIN, true);
+        try
+        {
+            opts.setOption(IRoleManager.Option.LOGIN, false);
+        }
+        catch (SyntaxException e)
+        {
+            assertEquals("Multiple definition for property 'LOGIN'", e.getMessage());
+        }
+    }
+
+    @Test
+    public void emptyByDefault()
+    {
+        RoleOptions opts = new RoleOptions();
+        assertTrue(opts.isEmpty());
+        assertFalse(opts.getLogin().isPresent());
+
+        opts.setOption(IRoleManager.Option.LOGIN, true);
+        assertFalse(opts.isEmpty());
+        assertTrue(opts.getLogin().isPresent());
+        assertTrue(opts.getLogin().get());
+    }
+
+    private void assertInvalidOptions(RoleOptions opts, String message)
+    {
+        try
+        {
+            opts.validate();
+            fail("Expected error but didn't get one");
+        }
+        catch (InvalidRequestException e)
+        {
+            assertTrue(e.getMessage().equals(message));
+        }
+    }
+
+    private void setupRoleManager(IRoleManager manager)
+    {
+        Field field = FBUtilities.getProtectedField(DatabaseDescriptor.class, "roleManager");
+        try
+        {
+            field.set(null, manager);
+        }
+        catch (IllegalAccessException e)
+        {
+            fail("Error setting IRoleManager instance for test");
+        }
+    }
+
+    private IRoleManager getRoleManager(final IRoleManager.Option...supportedOptions)
+    {
+        return new IRoleManager()
+        {
+            public Set<Option> supportedOptions()
+            {
+                return ImmutableSet.copyOf(supportedOptions);
+            }
+
+            public Set<Option> alterableOptions()
+            {
+                return null;
+            }
+
+            public void createRole(AuthenticatedUser performer,
+                                   RoleResource role,
+                                   RoleOptions options) throws RequestValidationException, RequestExecutionException
+            {
+
+            }
+
+            public void dropRole(AuthenticatedUser performer,
+                                 RoleResource role) throws RequestValidationException, RequestExecutionException
+            {
+
+            }
+
+            public void alterRole(AuthenticatedUser performer,
+                                  RoleResource role,
+                                  RoleOptions options) throws RequestValidationException, RequestExecutionException
+            {
+
+            }
+
+            public void grantRole(AuthenticatedUser performer,
+                                  RoleResource role,
+                                  RoleResource grantee) throws RequestValidationException, RequestExecutionException
+            {
+
+            }
+
+            public void revokeRole(AuthenticatedUser performer,
+                                   RoleResource role,
+                                   RoleResource revokee) throws RequestValidationException, RequestExecutionException
+            {
+
+            }
+
+            public Set<RoleResource> getRoles(RoleResource grantee,
+                                              boolean includeInherited) throws RequestValidationException, RequestExecutionException
+            {
+                return null;
+            }
+
+            public Set<RoleResource> getAllRoles() throws RequestValidationException, RequestExecutionException
+            {
+                return null;
+            }
+
+            public boolean isSuper(RoleResource role)
+            {
+                return false;
+            }
+
+            public boolean canLogin(RoleResource role)
+            {
+                return false;
+            }
+
+            public Map<String, String> getCustomOptions(RoleResource role)
+            {
+                return Collections.EMPTY_MAP;
+            }
+
+            public boolean isExistingRole(RoleResource role)
+            {
+                return false;
+            }
+
+            public Set<? extends IResource> protectedResources()
+            {
+                return null;
+            }
+
+            public void validateConfiguration() throws ConfigurationException
+            {
+
+            }
+
+            public void setup()
+            {
+
+            }
+        };
+    }
+}
diff --git a/test/unit/org/apache/cassandra/cache/AutoSavingCacheTest.java b/test/unit/org/apache/cassandra/cache/AutoSavingCacheTest.java
index e6ef69e..475e436 100644
--- a/test/unit/org/apache/cassandra/cache/AutoSavingCacheTest.java
+++ b/test/unit/org/apache/cassandra/cache/AutoSavingCacheTest.java
@@ -17,30 +17,47 @@
  */
 package org.apache.cassandra.cache;
 
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.junit.Assert;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.RowIndexEntry;
 import org.apache.cassandra.db.Mutation;
-import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.service.CacheService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
-public class AutoSavingCacheTest extends SchemaLoader
+public class AutoSavingCacheTest
 {
+    private static final String KEYSPACE1 = "AutoSavingCacheTest1";
+    private static final String CF_STANDARD1 = "Standard1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1));
+    }
+
     @Test
     public void testSerializeAndLoadKeyCache() throws Exception
     {
-        ColumnFamilyStore cfs = Keyspace.open("Keyspace1").getColumnFamilyStore("Standard1");
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD1);
         for (int i = 0; i < 2; i++)
         {
-            Mutation rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("key1"));
-            rm.add("Standard1", Util.cellname("c1"), ByteBufferUtil.bytes(i), 0);
-            rm.apply();
+            Mutation rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("key1"));
+            rm.add(CF_STANDARD1, Util.cellname("c1"), ByteBufferUtil.bytes(i), 0);
+            rm.applyUnsafe();
             cfs.forceBlockingFlush();
         }
 
diff --git a/test/unit/org/apache/cassandra/cache/CacheProviderTest.java b/test/unit/org/apache/cassandra/cache/CacheProviderTest.java
index 63f89a4..bfcfa59 100644
--- a/test/unit/org/apache/cassandra/cache/CacheProviderTest.java
+++ b/test/unit/org/apache/cassandra/cache/CacheProviderTest.java
@@ -24,13 +24,16 @@
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.UUID;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.ArrayBackedSortedColumns;
 import org.apache.cassandra.db.ColumnFamily;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.Pair;
 
 import com.googlecode.concurrentlinkedhashmap.Weighers;
@@ -38,7 +41,7 @@
 import static org.apache.cassandra.Util.column;
 import static org.junit.Assert.*;
 
-public class CacheProviderTest extends SchemaLoader
+public class CacheProviderTest
 {
     MeasureableString key1 = new MeasureableString("key1");
     MeasureableString key2 = new MeasureableString("key2");
@@ -46,8 +49,18 @@
     MeasureableString key4 = new MeasureableString("key4");
     MeasureableString key5 = new MeasureableString("key5");
     private static final long CAPACITY = 4;
-    private String keyspaceName = "Keyspace1";
-    private String cfName = "Standard1";
+    private static final String KEYSPACE1 = "CacheProviderTest1";
+    private static final String CF_STANDARD1 = "Standard1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1));
+    }
 
     private void simpleCase(ColumnFamily cf, ICache<MeasureableString, IRowCacheEntry> cache)
     {
@@ -101,7 +114,7 @@
 
     private ColumnFamily createCF()
     {
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(keyspaceName, cfName);
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, CF_STANDARD1);
         cf.addColumn(column("vijay", "great", 1));
         cf.addColumn(column("awesome", "vijay", 1));
         return cf;
@@ -119,7 +132,7 @@
     @Test
     public void testKeys()
     {
-        Pair<String, String> ksAndCFName = Pair.create(keyspaceName, cfName);
+        Pair<String, String> ksAndCFName = Pair.create(KEYSPACE1, CF_STANDARD1);
         byte[] b1 = {1, 2, 3, 4};
         RowCacheKey key1 = new RowCacheKey(ksAndCFName, ByteBuffer.wrap(b1));
         byte[] b2 = {1, 2, 3, 4};
diff --git a/test/unit/org/apache/cassandra/cli/CliTest.java b/test/unit/org/apache/cassandra/cli/CliTest.java
deleted file mode 100644
index 38f567b..0000000
--- a/test/unit/org/apache/cassandra/cli/CliTest.java
+++ /dev/null
@@ -1,323 +0,0 @@
-/**
- * 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.cli;
-
-import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.service.EmbeddedCassandraService;
-import org.apache.cassandra.thrift.*;
-import org.apache.thrift.TException;
-import org.junit.Test;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.PrintStream;
-import java.util.regex.Pattern;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-public class CliTest extends SchemaLoader
-{
-    // please add new statements here so they could be auto-runned by this test.
-    private String[] statements = {
-        "use TestKeySpace;",
-        "create column family SecondaryIndicesWithoutIdxName" +
-                " with comparator = UTF8Type" +
-                " and default_validation_class = UTF8Type" +
-                " and column_metadata = [{column_name: profileId, validation_class: UTF8Type, index_type: KEYS}];",
-        "update column family SecondaryIndicesWithoutIdxName" +
-                " with column_metadata = " +
-                "[{column_name: profileId, validation_class: UTF8Type, index_type: KEYS}," +
-                 "{column_name: postedDate, validation_class: LongType}];",
-        "create column family 123 with comparator=UTF8Type and column_metadata=[{ column_name:world, validation_class:IntegerType, index_type:0, index_name:IdxName }, " +
-                                                                               "{ column_name:world2, validation_class:LongType, index_type:KEYS, index_name:LongIdxName}, " +
-                                                                               "{ column_name:617070, validation_class:UTF8Type, index_type:KEYS }, " +
-                                                                               "{ column_name:'-617071', validation_class:UTF8Type, index_type:KEYS }," +
-                                                                               "{ column_name:time_spent_uuid, validation_class:TimeUUIDType}] and default_validation_class=UTF8Type;",
-        "assume 123 keys as utf8;",
-        "set 123[hello][world] = 123848374878933948398384;",
-        "set 123[hello][test_quote] = 'value\\'';",
-        "set 123['k\\'ey'][VALUE] = 'VAL';",
-        "set 123['k\\'ey'][VALUE] = 'VAL\\'';",
-        "set 123[hello][-31337] = 'some string value';",
-        "list 123;",
-        "list 123[:];",
-        "list 123[456:];",
-        "list 123 limit 5;",
-        "list 123[12:15] limit 20;",
-        "list 123[12:15] columns 2;",
-        "list 123 columns 2 reversed;",
-        "list 123 limit 10 columns 2 reversed;",
-        "get 123[hello][-31337];",
-        "get 123[hello][world];",
-        "get 123[hello][test_quote];",
-        "get 123['k\\'ey'][VALUE]",
-        "set 123[hello][-31337] = -23876;",
-        "set 123[hello][world2] = 15;",
-        "get 123 where world2 = long(15);",
-        "get 123 where world2 = long(15);",
-        "get 123 where world2 = long(15);",
-        "del 123[utf8('hello')][utf8('world')];",
-        "del 123[hello][world2];",
-        "set 123['hello'][time_spent_uuid] = timeuuid(a8098c1a-f86e-11da-bd1a-00112444be1e);",
-        "create column family CF2 with comparator=IntegerType and default_validation_class=AsciiType;",
-        "assume CF2 keys as utf8;",
-        "set CF2['key'][98349387493847748398334] = 'some text';",
-        "get CF2['key'][98349387493847748398334];",
-        "set CF2['key'][98349387493] = 'some text other';",
-        "get CF2['key'][98349387493];",
-        "create column family CF3 with comparator=UTF8Type and column_metadata=[{column_name:'big world', validation_class:LongType, index_type:KEYS, index_name:WorldIdx}];",
-        "assume CF3 keys as utf8;",
-        "set CF3['hello']['big world'] = 3748;",
-        "get CF3['hello']['big world'];",
-        "list CF3;",
-        "list CF3[:];",
-        "list CF3[h:];",
-        "list CF3 limit 10;",
-        "list CF3[h:] limit 10;",
-        "create column family CF4 with comparator=IntegerType and column_metadata=[{column_name:9999, validation_class:LongType}];",
-        "assume CF4 keys as utf8;",
-        "set CF4['hello'][9999] = 1234;",
-        "get CF4['hello'][9999];",
-        "get CF4['hello'][9999] as Long;",
-        "get CF4['hello'][9999] as Bytes;",
-        "set CF4['hello'][9999] = Long(1234);",
-        "get CF4['hello'][9999];",
-        "get CF4['hello'][9999] as Long;",
-        "del CF4['hello'][9999];",
-        "get CF4['hello'][9999];",
-        "create column family sCf1 with column_type=Super and comparator=IntegerType and subcomparator=LongType and column_metadata=[{column_name:9999, validation_class:LongType}];",
-        "assume sCf1 keys as utf8;",
-        "set sCf1['hello'][1][9999] = 1234;",
-        "get sCf1['hello'][1][9999];",
-        "get sCf1['hello'][1][9999] as Long;",
-        "get sCf1['hello'][1][9999] as Bytes;",
-        "set sCf1['hello'][1][9999] = Long(1234);",
-        "set sCf1['hello'][-1][-12] = Long(5678);",
-        "get sCf1['hello'][-1][-12];",
-        "set sCf1['hello'][-1][-12] = -340897;",
-        "set sCf1['hello'][-1][-12] = integer(-340897);",
-        "get sCf1['hello'][1][9999];",
-        "get sCf1['hello'][1][9999] as Long;",
-        "del sCf1['hello'][1][9999];",
-        "get sCf1['hello'][1][9999];",
-        "set sCf1['hello'][1][9999] = Long(1234);",
-        "del sCf1['hello'][9999];",
-        "get sCf1['hello'][1][9999];",
-        "create column family 'Counter1' with comparator=UTF8Type and default_validation_class=CounterColumnType;",
-        "assume Counter1 keys as utf8;",
-        "incr Counter1['hello']['cassandra'];",
-        "incr Counter1['hello']['cassandra'] by 3;",
-        "incr Counter1['hello']['cassandra'] by -2;",
-        "decr Counter1['hello']['cassandra'];",
-        "decr Counter1['hello']['cassandra'] by 3;",
-        "decr Counter1['hello']['cassandra'] by -2;",
-        "get Counter1['hello']['cassandra'];",
-        "get Counter1['hello'];",
-        "truncate 123;",
-        "drop index on '123'.world2;",
-        "drop index on '123'.617070;",
-        "drop index on '123'.'-617071';",
-        "drop index on CF3.'big world';",
-        "update keyspace TestKeySpace with durable_writes = false;",
-        "assume 123 comparator as utf8;",
-        "assume 123 sub_comparator as integer;",
-        "assume 123 validator as lexicaluuid;",
-        "assume 123 keys as timeuuid;",
-        "create column family CF7;",
-        "assume CF7 keys as utf8;",
-        "set CF7[1][timeuuid()] = utf8(test1);",
-        "set CF7[2][lexicaluuid()] = utf8('hello world!');",
-        "set CF7[3][lexicaluuid(550e8400-e29b-41d4-a716-446655440000)] = utf8(test2);",
-        "set CF7[key2][timeuuid()] = utf8(test3);",
-        "assume CF7 comparator as lexicaluuid;",
-        "assume CF7 keys as utf8;",
-        "list CF7;",
-        "get CF7[3];",
-        "get CF7[3][lexicaluuid(550e8400-e29b-41d4-a716-446655440000)];",
-        "get sCf1['hello'][1][9999];",
-        "set sCf1['hello'][1][9999] = 938;",
-        "set sCf1['hello'][1][9999] = 938 with ttl = 30;",
-        "set sCf1['hello'][1][9999] = 938 with ttl = 560;",
-        "count sCf1[hello];",
-        "count sCf1[utf8('hello')];",
-        "count sCf1[utf8('hello')][integer(1)];",
-        "count sCf1[hello][1];",
-        "list sCf1;",
-        "del sCf1['hello'][1][9999];",
-        "assume sCf1 comparator as utf8;",
-        "create column family CF8;",
-        "drop column family cF8;",
-        "create keyspace TESTIN;",
-        "drop keyspace tesTIN;",
-        "update column family 123 with comparator=UTF8Type and column_metadata=[];",
-        "drop column family 123;",
-        "create column family myCF with column_type='Super' and comparator='UTF8Type' AND subcomparator='UTF8Type' AND default_validation_class=AsciiType;",
-        "assume myCF keys as utf8;",
-        "create column family Countries with comparator=UTF8Type and column_metadata=[ {column_name: name, validation_class: UTF8Type} ];",
-        "set Countries[11][name] = USA;",
-        "get Countries[11][name];",
-        "update column family Countries with compaction_strategy = 'org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy';",
-        "create column family Cities with compaction_strategy = 'org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy' and compaction_strategy_options = {min_sstable_size:1024};",
-        "set myCF['key']['scName']['firstname'] = 'John';",
-        "get myCF['key']['scName']",
-        "assume CF3 keys as utf8;",
-        "use TestKEYSpace;",
-        "update keyspace TestKeySpace with placement_strategy='org.apache.cassandra.locator.NetworkTopologyStrategy';",
-        "update keyspace TestKeySpace with strategy_options=[{DC1:3, DC2:4, DC5:1}];",
-        "describe cluster;",
-        "help describe cluster;",
-        "show cluster name",
-        "show api version",
-        "help help",
-        "help connect",
-        "help use",
-        "help describe",
-        "HELP exit",
-        "help QUIT",
-        "help show cluster name",
-        "help show keyspaces",
-        "help show schema",
-        "help show api version",
-        "help create keyspace",
-        "HELP update KEYSPACE",
-        "HELP CREATE column FAMILY",
-        "HELP UPDATE COLUMN family",
-        "HELP drop keyspace",
-        "help drop column family",
-        "HELP GET",
-        "HELP set",
-        "HELP DEL",
-        "HELP count",
-        "HELP list",
-        "HELP TRUNCATE",
-        "help assume",
-        "HELP",
-        "?",
-        "show schema",
-        "show schema TestKeySpace"
-    };
-
-    @Test
-    public void testCli() throws IOException, TException, TimedOutException, NotFoundException, SchemaDisagreementException, NoSuchFieldException, InvalidRequestException, UnavailableException, InstantiationException, IllegalAccessException
-    {
-        Schema.instance.clear(); // Schema are now written on disk and will be reloaded
-        new EmbeddedCassandraService().start();
-
-        // new error/output streams for CliSessionState
-        ByteArrayOutputStream errStream = new ByteArrayOutputStream();
-        ByteArrayOutputStream outStream = new ByteArrayOutputStream();
-
-        // checking if we can connect to the running cassandra node on localhost
-        CliMain.connect("127.0.0.1", DatabaseDescriptor.getRpcPort());
-
-        // setting new output stream
-        CliMain.sessionState.setOut(new PrintStream(outStream));
-        CliMain.sessionState.setErr(new PrintStream(errStream));
-
-        // re-creating keyspace for tests
-        try
-        {
-            // dropping in case it exists e.g. could be left from previous run
-            CliMain.processStatement("drop keyspace TestKeySpace;");
-        }
-        catch (Exception e)
-        {
-            // TODO check before drop so we don't have this fragile ignored exception block
-        }
-        CliMain.processStatement("create keyspace TestKeySpace;");
-
-        for (String statement : statements)
-        {
-            errStream.reset();
-            // System.out.println("Executing statement: " + statement);
-            CliMain.processStatement(statement);
-            String result = outStream.toString();
-            // System.out.println("Result:\n" + result);
-            if (statement.startsWith("show schema"))
-                assertEquals(errStream.toString() + "processing" + statement,
-                             "\nWARNING: CQL3 tables are intentionally omitted from 'show schema' output." + String.format("%n")
-                             + "See https://issues.apache.org/jira/browse/CASSANDRA-4377 for details.\n" + String.format("%n"),
-                             errStream.toString());
-            else
-                assertEquals(errStream.toString() + " processing " + statement, "", errStream.toString());
-
-            if (statement.startsWith("drop ") || statement.startsWith("create ") || statement.startsWith("update "))
-            {
-                assert Pattern.compile("(.{8})-(.{4})-(.{4})-(.{4})-(.{12}).*", Pattern.DOTALL).matcher(result).matches()
-                       : String.format("\"%s\" failed: %s", statement, result);
-            }
-            else if (statement.startsWith("set "))
-            {
-                 assertTrue(result.contains("Value inserted."));
-                 assertTrue(result.contains("Elapsed time:"));
-            }
-            else if (statement.startsWith("incr "))
-            {
-                 assertTrue(result.contains("Value incremented."));
-            }
-            else if (statement.startsWith("decr "))
-            {
-                 assertTrue(result.contains("Value decremented."));
-            }
-            else if (statement.startsWith("get "))
-            {
-                if (statement.contains("where"))
-                {
-                    assertTrue(result.startsWith("-------------------" + System.getProperty("line.separator") + "RowKey:"));
-                }
-                else if (statement.contains("Counter"))
-                {
-                    assertTrue(result.startsWith("=> (counter=") || result.startsWith("Value was not found"));
-                }
-                else
-                {
-                    assertTrue(result.startsWith("=> (name=") || result.startsWith("Value was not found"));
-                }
-                assertTrue(result.contains("Elapsed time:"));
-            }
-            else if (statement.startsWith("truncate "))
-            {
-                assertTrue(result.contains(" truncated."));
-            }
-            else if (statement.startsWith("assume "))
-            {
-                assertTrue(result.contains("successfully."));
-            }
-
-            outStream.reset(); // reset stream so we have only output from next statement all the time
-            errStream.reset(); // no errors to the end user.
-        }
-    }
-
-    @Test
-    public void testEscape()
-    {
-        //escaped is the string read from the cli.
-        String escaped = "backspace \\b tab \\t linefeed \\n form feed \\f carriage return \\r duble quote \\\" " +
-                "single quote \\' backslash \\\\";
-        String unescaped = "backspace \b tab \t linefeed \n form feed \f carriage return \r duble quote \" " +
-                "single quote ' backslash \\";
-        // when read from the cli may have single quotes around it
-        assertEquals(unescaped, CliUtils.unescapeSQLString("'" + escaped + "'"));
-        assertEquals(escaped, CliUtils.escapeSQLString(unescaped));
-    }
-}
diff --git a/test/unit/org/apache/cassandra/config/CFMetaDataTest.java b/test/unit/org/apache/cassandra/config/CFMetaDataTest.java
deleted file mode 100644
index 71f21a2..0000000
--- a/test/unit/org/apache/cassandra/config/CFMetaDataTest.java
+++ /dev/null
@@ -1,137 +0,0 @@
-/**
- * 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.config;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.HashMap;
-import java.util.HashSet;
-
-import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.cql3.QueryProcessor;
-import org.apache.cassandra.cql3.UntypedResultSet;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.marshal.AsciiType;
-import org.apache.cassandra.db.marshal.UTF8Type;
-import org.apache.cassandra.io.compress.*;
-import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.thrift.CfDef;
-import org.apache.cassandra.thrift.ColumnDef;
-import org.apache.cassandra.thrift.IndexType;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.junit.Test;
-
-import static org.hamcrest.CoreMatchers.is;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertThat;
-
-public class CFMetaDataTest extends SchemaLoader
-{
-    private static String KEYSPACE = "Keyspace1";
-    private static String COLUMN_FAMILY = "Standard1";
-
-    private static List<ColumnDef> columnDefs = new ArrayList<ColumnDef>();
-
-    static
-    {
-        columnDefs.add(new ColumnDef(ByteBufferUtil.bytes("col1"), AsciiType.class.getCanonicalName())
-                                    .setIndex_name("col1Index")
-                                    .setIndex_type(IndexType.KEYS));
-
-        columnDefs.add(new ColumnDef(ByteBufferUtil.bytes("col2"), UTF8Type.class.getCanonicalName())
-                                    .setIndex_name("col2Index")
-                                    .setIndex_type(IndexType.KEYS));
-    }
-
-    @Test
-    public void testThriftConversion() throws Exception
-    {
-        CfDef cfDef = new CfDef().setDefault_validation_class(AsciiType.class.getCanonicalName())
-                                 .setComment("Test comment")
-                                 .setColumn_metadata(columnDefs)
-                                 .setKeyspace(KEYSPACE)
-                                 .setName(COLUMN_FAMILY);
-
-        // convert Thrift to CFMetaData
-        CFMetaData cfMetaData = CFMetaData.fromThrift(cfDef);
-
-        CfDef thriftCfDef = new CfDef();
-        thriftCfDef.keyspace = KEYSPACE;
-        thriftCfDef.name = COLUMN_FAMILY;
-        thriftCfDef.default_validation_class = cfDef.default_validation_class;
-        thriftCfDef.comment = cfDef.comment;
-        thriftCfDef.column_metadata = new ArrayList<ColumnDef>();
-        for (ColumnDef columnDef : columnDefs)
-        {
-            ColumnDef c = new ColumnDef();
-            c.name = ByteBufferUtil.clone(columnDef.name);
-            c.validation_class = columnDef.getValidation_class();
-            c.index_name = columnDef.getIndex_name();
-            c.index_type = IndexType.KEYS;
-            thriftCfDef.column_metadata.add(c);
-        }
-
-        CfDef converted = cfMetaData.toThrift();
-
-        assertEquals(thriftCfDef.keyspace, converted.keyspace);
-        assertEquals(thriftCfDef.name, converted.name);
-        assertEquals(thriftCfDef.default_validation_class, converted.default_validation_class);
-        assertEquals(thriftCfDef.comment, converted.comment);
-        assertEquals(new HashSet<>(thriftCfDef.column_metadata), new HashSet<>(converted.column_metadata));
-    }
-
-    @Test
-    public void testConversionsInverses() throws Exception
-    {
-        for (String keyspaceName : Schema.instance.getNonSystemKeyspaces())
-        {
-            for (ColumnFamilyStore cfs : Keyspace.open(keyspaceName).getColumnFamilyStores())
-            {
-                CFMetaData cfm = cfs.metadata;
-                if (!cfm.isThriftCompatible())
-                    continue;
-
-                checkInverses(cfm);
-
-                // Testing with compression to catch #3558
-                CFMetaData withCompression = cfm.copy();
-                withCompression.compressionParameters(new CompressionParameters(SnappyCompressor.instance, 32768, new HashMap<String, String>()));
-                checkInverses(withCompression);
-            }
-        }
-    }
-
-    private void checkInverses(CFMetaData cfm) throws Exception
-    {
-        DecoratedKey k = StorageService.getPartitioner().decorateKey(ByteBufferUtil.bytes(cfm.ksName));
-
-        // Test thrift conversion
-        CFMetaData before = cfm;
-        CFMetaData after = CFMetaData.fromThriftForUpdate(before.toThrift(), before);
-        assert before.equals(after) : String.format("%n%s%n!=%n%s", before, after);
-
-        // Test schema conversion
-        Mutation rm = cfm.toSchema(System.currentTimeMillis());
-        ColumnFamily serializedCf = rm.getColumnFamily(Schema.instance.getId(Keyspace.SYSTEM_KS, SystemKeyspace.SCHEMA_COLUMNFAMILIES_CF));
-        ColumnFamily serializedCD = rm.getColumnFamily(Schema.instance.getId(Keyspace.SYSTEM_KS, SystemKeyspace.SCHEMA_COLUMNS_CF));
-        UntypedResultSet.Row result = QueryProcessor.resultify("SELECT * FROM system.schema_columnfamilies", new Row(k, serializedCf)).one();
-        CFMetaData newCfm = CFMetaData.fromSchemaNoTriggers(result, ColumnDefinition.resultify(new Row(k, serializedCD)));
-        assert cfm.equals(newCfm) : String.format("%n%s%n!=%n%s", cfm, newCfm);
-    }
-}
diff --git a/test/unit/org/apache/cassandra/config/ColumnDefinitionTest.java b/test/unit/org/apache/cassandra/config/ColumnDefinitionTest.java
index 890c46c..2bee0c3 100644
--- a/test/unit/org/apache/cassandra/config/ColumnDefinitionTest.java
+++ b/test/unit/org/apache/cassandra/config/ColumnDefinitionTest.java
@@ -25,6 +25,7 @@
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.thrift.ThriftConversion;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 public class ColumnDefinitionTest
@@ -45,7 +46,7 @@
 
     protected void testSerializeDeserialize(CFMetaData cfm, ColumnDefinition cd) throws Exception
     {
-        ColumnDefinition newCd = ColumnDefinition.fromThrift(cfm.ksName, cfm.cfName, cfm.comparator.asAbstractType(), null, cd.toThrift());
+        ColumnDefinition newCd = ThriftConversion.fromThrift(cfm.ksName, cfm.cfName, cfm.comparator.asAbstractType(), null, ThriftConversion.toThrift(cd));
         Assert.assertNotSame(cd, newCd);
         Assert.assertEquals(cd.hashCode(), newCd.hashCode());
         Assert.assertEquals(cd, newCd);
diff --git a/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java b/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
index 46522cc..7409535 100644
--- a/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
+++ b/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
@@ -18,6 +18,7 @@
 */
 package org.apache.cassandra.config;
 
+import java.io.IOException;
 import java.net.Inet4Address;
 import java.net.Inet6Address;
 import java.net.InetAddress;
@@ -37,6 +38,7 @@
 import org.apache.cassandra.gms.Gossiper;
 import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.service.MigrationManager;
+import org.apache.cassandra.thrift.ThriftConversion;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
@@ -45,6 +47,12 @@
 @RunWith(OrderedJUnit4ClassRunner.class)
 public class DatabaseDescriptorTest
 {
+    @BeforeClass
+    public static void setupDatabaseDescriptor()
+    {
+        DatabaseDescriptor.setDaemonInitialized();
+    }
+
     @Test
     public void testCFMetaDataSerialization() throws ConfigurationException, InvalidRequestException
     {
@@ -53,7 +61,7 @@
         {
             for (CFMetaData cfm : Schema.instance.getKeyspaceMetaData(keyspaceName).values())
             {
-                CFMetaData cfmDupe = CFMetaData.fromThrift(cfm.toThrift());
+                CFMetaData cfmDupe = ThriftConversion.fromThrift(ThriftConversion.toThrift(cfm));
                 assertNotNull(cfmDupe);
                 assertEquals(cfm, cfmDupe);
             }
@@ -63,10 +71,11 @@
     @Test
     public void testKSMetaDataSerialization() throws ConfigurationException
     {
-        for (KSMetaData ksm : Schema.instance.getKeyspaceDefinitions())
+        for (String ks : Schema.instance.getNonSystemKeyspaces())
         {
             // Not testing round-trip on the KsDef via serDe() because maps
-            KSMetaData ksmDupe = KSMetaData.fromThrift(ksm.toThrift());
+            KSMetaData ksm = Schema.instance.getKSMetaData(ks);
+            KSMetaData ksmDupe = ThriftConversion.fromThrift(ThriftConversion.toThrift(ksm));
             assertNotNull(ksmDupe);
             assertEquals(ksm, ksmDupe);
         }
@@ -74,10 +83,10 @@
 
     // this came as a result of CASSANDRA-995
     @Test
-    public void testTransKsMigration() throws ConfigurationException
+    public void testTransKsMigration() throws ConfigurationException, IOException
     {
         SchemaLoader.cleanupAndLeaveDirs();
-        DatabaseDescriptor.loadSchemas();
+        Schema.instance.loadFromDisk();
         assertEquals(0, Schema.instance.getNonSystemKeyspaces().size());
 
         Gossiper.instance.start((int)(System.currentTimeMillis() / 1000));
@@ -98,7 +107,7 @@
             assertNull(Schema.instance.getKSMetaData("ks0"));
             assertNull(Schema.instance.getKSMetaData("ks1"));
 
-            DatabaseDescriptor.loadSchemas();
+            Schema.instance.loadFromDisk();
 
             assertNotNull(Schema.instance.getKSMetaData("ks0"));
             assertNotNull(Schema.instance.getKSMetaData("ks1"));
@@ -158,7 +167,7 @@
                 boolean hasIPv6 = false;
                 Enumeration<InetAddress> addresses = suitableInterface.getInetAddresses();
                 while (addresses.hasMoreElements()) {
-                    if (addresses.nextElement().getClass() == Inet6Address.class)
+                    if (addresses.nextElement() instanceof Inet6Address)
                         hasIPv6 = true;
                     else
                         hasIPv4 = true;
diff --git a/test/unit/org/apache/cassandra/config/DefsTest.java b/test/unit/org/apache/cassandra/config/DefsTest.java
deleted file mode 100644
index 94738ac..0000000
--- a/test/unit/org/apache/cassandra/config/DefsTest.java
+++ /dev/null
@@ -1,533 +0,0 @@
-/**
- * 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.config;
-
-import java.io.File;
-import java.nio.ByteBuffer;
-
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
-import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.Util;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.composites.*;
-import org.apache.cassandra.db.marshal.BytesType;
-import org.apache.cassandra.db.marshal.UTF8Type;
-import org.apache.cassandra.db.marshal.TimeUUIDType;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.io.sstable.Component;
-import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.sstable.SSTableDeletingTask;
-import org.apache.cassandra.locator.OldNetworkTopologyStrategy;
-import org.apache.cassandra.locator.SimpleStrategy;
-import org.apache.cassandra.service.MigrationManager;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import static org.apache.cassandra.Util.cellname;
-
-import org.junit.Ignore;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-
-@RunWith(OrderedJUnit4ClassRunner.class)
-public class DefsTest extends SchemaLoader
-{
-    @Test
-    public void testCFMetaDataApply() throws ConfigurationException
-    {
-        CFMetaData cfm = new CFMetaData("Keyspace1",
-                                        "TestApplyCFM_CF",
-                                        ColumnFamilyType.Standard,
-                                        new SimpleDenseCellNameType(BytesType.instance));
-
-        for (int i = 0; i < 5; i++)
-        {
-            ByteBuffer name = ByteBuffer.wrap(new byte[] { (byte)i });
-            cfm.addColumnDefinition(ColumnDefinition.regularDef(cfm, name, BytesType.instance, null).setIndex(Integer.toString(i), IndexType.KEYS, null));
-        }
-
-        cfm.comment("No comment")
-           .readRepairChance(0.5)
-           .gcGraceSeconds(100000)
-           .minCompactionThreshold(500)
-           .maxCompactionThreshold(500);
-
-        // we'll be adding this one later. make sure it's not already there.
-        assert cfm.getColumnDefinition(ByteBuffer.wrap(new byte[] { 5 })) == null;
-
-        CFMetaData cfNew = cfm.copy();
-
-        // add one.
-        ColumnDefinition addIndexDef = ColumnDefinition.regularDef(cfm, ByteBuffer.wrap(new byte[] { 5 }), BytesType.instance, null)
-                                                       .setIndex("5", IndexType.KEYS, null);
-        cfNew.addColumnDefinition(addIndexDef);
-
-        // remove one.
-        ColumnDefinition removeIndexDef = ColumnDefinition.regularDef(cfm, ByteBuffer.wrap(new byte[] { 0 }), BytesType.instance, null)
-                                                          .setIndex("0", IndexType.KEYS, null);
-        assert cfNew.removeColumnDefinition(removeIndexDef);
-
-        cfm.apply(cfNew);
-
-        for (int i = 1; i < cfm.allColumns().size(); i++)
-            assert cfm.getColumnDefinition(ByteBuffer.wrap(new byte[] { 1 })) != null;
-        assert cfm.getColumnDefinition(ByteBuffer.wrap(new byte[] { 0 })) == null;
-        assert cfm.getColumnDefinition(ByteBuffer.wrap(new byte[] { 5 })) != null;
-    }
-
-    @Test
-    public void testInvalidNames()
-    {
-        String[] valid = {"1", "a", "_1", "b_", "__", "1_a"};
-        for (String s : valid)
-            assert CFMetaData.isNameValid(s);
-
-        String[] invalid = {"b@t", "dash-y", "", " ", "dot.s", ".hidden"};
-        for (String s : invalid)
-            assert !CFMetaData.isNameValid(s);
-    }
-
-    @Ignore
-    @Test
-    public void saveAndRestore()
-    {
-        /*
-        // verify dump and reload.
-        UUID first = UUIDGen.makeType1UUIDFromHost(FBUtilities.getBroadcastAddress());
-        DefsTables.dumpToStorage(first);
-        List<KSMetaData> defs = new ArrayList<KSMetaData>(DefsTables.loadFromStorage(first));
-
-        assert defs.size() > 0;
-        assert defs.size() == Schema.instance.getNonSystemKeyspaces().size();
-        for (KSMetaData loaded : defs)
-        {
-            KSMetaData defined = Schema.instance.getKeyspaceDefinition(loaded.name);
-            assert defined.equals(loaded) : String.format("%s != %s", loaded, defined);
-        }
-        */
-    }
-
-    @Test
-    public void addNewCfToBogusKeyspace()
-    {
-        CFMetaData newCf = addTestCF("MadeUpKeyspace", "NewCF", "new cf");
-        try
-        {
-            MigrationManager.announceNewColumnFamily(newCf);
-            throw new AssertionError("You shouldn't be able to do anything to a keyspace that doesn't exist.");
-        }
-        catch (ConfigurationException expected)
-        {
-        }
-    }
-
-    @Test
-    public void addNewCfWithNullComment() throws ConfigurationException
-    {
-        final String ks = "Keyspace1";
-        final String cf = "BrandNewCfWithNull";
-        KSMetaData original = Schema.instance.getKSMetaData(ks);
-
-        CFMetaData newCf = addTestCF(original.name, cf, null);
-
-        assert !Schema.instance.getKSMetaData(ks).cfMetaData().containsKey(newCf.cfName);
-        MigrationManager.announceNewColumnFamily(newCf);
-
-        assert Schema.instance.getKSMetaData(ks).cfMetaData().containsKey(newCf.cfName);
-        assert Schema.instance.getKSMetaData(ks).cfMetaData().get(newCf.cfName).equals(newCf);
-    }
-
-    @Test
-    public void addNewCF() throws ConfigurationException
-    {
-        final String ks = "Keyspace1";
-        final String cf = "BrandNewCf";
-        KSMetaData original = Schema.instance.getKSMetaData(ks);
-
-        CFMetaData newCf = addTestCF(original.name, cf, "A New Column Family");
-
-        assert !Schema.instance.getKSMetaData(ks).cfMetaData().containsKey(newCf.cfName);
-        MigrationManager.announceNewColumnFamily(newCf);
-
-        assert Schema.instance.getKSMetaData(ks).cfMetaData().containsKey(newCf.cfName);
-        assert Schema.instance.getKSMetaData(ks).cfMetaData().get(newCf.cfName).equals(newCf);
-
-        // now read and write to it.
-        CellName col0 = cellname("col0");
-        DecoratedKey dk = Util.dk("key0");
-        Mutation rm = new Mutation(ks, dk.getKey());
-        rm.add(cf, col0, ByteBufferUtil.bytes("value0"), 1L);
-        rm.apply();
-        ColumnFamilyStore store = Keyspace.open(ks).getColumnFamilyStore(cf);
-        assert store != null;
-        store.forceBlockingFlush();
-
-        ColumnFamily cfam = store.getColumnFamily(Util.namesQueryFilter(store, dk, col0));
-        assert cfam.getColumn(col0) != null;
-        Cell col = cfam.getColumn(col0);
-        assert ByteBufferUtil.bytes("value0").equals(col.value());
-    }
-
-    @Test
-    public void dropCf() throws ConfigurationException
-    {
-        DecoratedKey dk = Util.dk("dropCf");
-        // sanity
-        final KSMetaData ks = Schema.instance.getKSMetaData("Keyspace1");
-        assert ks != null;
-        final CFMetaData cfm = ks.cfMetaData().get("Standard1");
-        assert cfm != null;
-
-        // write some data, force a flush, then verify that files exist on disk.
-        Mutation rm = new Mutation(ks.name, dk.getKey());
-        for (int i = 0; i < 100; i++)
-            rm.add(cfm.cfName, cellname("col" + i), ByteBufferUtil.bytes("anyvalue"), 1L);
-        rm.apply();
-        ColumnFamilyStore store = Keyspace.open(cfm.ksName).getColumnFamilyStore(cfm.cfName);
-        assert store != null;
-        store.forceBlockingFlush();
-        assert store.directories.sstableLister().list().size() > 0;
-
-        MigrationManager.announceColumnFamilyDrop(ks.name, cfm.cfName);
-
-        assert !Schema.instance.getKSMetaData(ks.name).cfMetaData().containsKey(cfm.cfName);
-
-        // any write should fail.
-        rm = new Mutation(ks.name, dk.getKey());
-        boolean success = true;
-        try
-        {
-            rm.add("Standard1", cellname("col0"), ByteBufferUtil.bytes("value0"), 1L);
-            rm.apply();
-        }
-        catch (Throwable th)
-        {
-            success = false;
-        }
-        assert !success : "This mutation should have failed since the CF no longer exists.";
-
-        // verify that the files are gone.
-        for (File file : store.directories.sstableLister().listFiles())
-        {
-            if (file.getPath().endsWith("Data.db") && !new File(file.getPath().replace("Data.db", "Compacted")).exists())
-                throw new AssertionError("undeleted file " + file);
-        }
-    }
-
-    @Test
-    public void addNewKS() throws ConfigurationException
-    {
-        DecoratedKey dk = Util.dk("key0");
-        CFMetaData newCf = addTestCF("NewKeyspace1", "AddedStandard1", "A new cf for a new ks");
-
-        KSMetaData newKs = KSMetaData.testMetadata(newCf.ksName, SimpleStrategy.class, KSMetaData.optsWithRF(5), newCf);
-
-        MigrationManager.announceNewKeyspace(newKs);
-
-        assert Schema.instance.getKSMetaData(newCf.ksName) != null;
-        assert Schema.instance.getKSMetaData(newCf.ksName).equals(newKs);
-
-        // test reads and writes.
-        CellName col0 = cellname("col0");
-        Mutation rm = new Mutation(newCf.ksName, dk.getKey());
-        rm.add(newCf.cfName, col0, ByteBufferUtil.bytes("value0"), 1L);
-        rm.apply();
-        ColumnFamilyStore store = Keyspace.open(newCf.ksName).getColumnFamilyStore(newCf.cfName);
-        assert store != null;
-        store.forceBlockingFlush();
-
-        ColumnFamily cfam = store.getColumnFamily(Util.namesQueryFilter(store, dk, col0));
-        assert cfam.getColumn(col0) != null;
-        Cell col = cfam.getColumn(col0);
-        assert ByteBufferUtil.bytes("value0").equals(col.value());
-    }
-
-    @Test
-    public void dropKS() throws ConfigurationException
-    {
-        DecoratedKey dk = Util.dk("dropKs");
-        // sanity
-        final KSMetaData ks = Schema.instance.getKSMetaData("Keyspace1");
-        assert ks != null;
-        final CFMetaData cfm = ks.cfMetaData().get("Standard2");
-        assert cfm != null;
-
-        // write some data, force a flush, then verify that files exist on disk.
-        Mutation rm = new Mutation(ks.name, dk.getKey());
-        for (int i = 0; i < 100; i++)
-            rm.add(cfm.cfName, cellname("col" + i), ByteBufferUtil.bytes("anyvalue"), 1L);
-        rm.apply();
-        ColumnFamilyStore store = Keyspace.open(cfm.ksName).getColumnFamilyStore(cfm.cfName);
-        assert store != null;
-        store.forceBlockingFlush();
-        assert store.directories.sstableLister().list().size() > 0;
-
-        MigrationManager.announceKeyspaceDrop(ks.name);
-
-        assert Schema.instance.getKSMetaData(ks.name) == null;
-
-        // write should fail.
-        rm = new Mutation(ks.name, dk.getKey());
-        boolean success = true;
-        try
-        {
-            rm.add("Standard1", cellname("col0"), ByteBufferUtil.bytes("value0"), 1L);
-            rm.apply();
-        }
-        catch (Throwable th)
-        {
-            success = false;
-        }
-        assert !success : "This mutation should have failed since the CF no longer exists.";
-
-        // reads should fail too.
-        boolean threw = false;
-        try
-        {
-            Keyspace.open(ks.name);
-        }
-        catch (Throwable th)
-        {
-            threw = true;
-        }
-        assert threw;
-    }
-
-    @Test
-    public void dropKSUnflushed() throws ConfigurationException
-    {
-        DecoratedKey dk = Util.dk("dropKs");
-        // sanity
-        final KSMetaData ks = Schema.instance.getKSMetaData("Keyspace3");
-        assert ks != null;
-        final CFMetaData cfm = ks.cfMetaData().get("Standard1");
-        assert cfm != null;
-
-        // write some data
-        Mutation rm = new Mutation(ks.name, dk.getKey());
-        for (int i = 0; i < 100; i++)
-            rm.add(cfm.cfName, cellname("col" + i), ByteBufferUtil.bytes("anyvalue"), 1L);
-        rm.apply();
-
-        MigrationManager.announceKeyspaceDrop(ks.name);
-
-        assert Schema.instance.getKSMetaData(ks.name) == null;
-    }
-
-    @Test
-    public void createEmptyKsAddNewCf() throws ConfigurationException
-    {
-        assert Schema.instance.getKSMetaData("EmptyKeyspace") == null;
-
-        KSMetaData newKs = KSMetaData.testMetadata("EmptyKeyspace", SimpleStrategy.class, KSMetaData.optsWithRF(5));
-
-        MigrationManager.announceNewKeyspace(newKs);
-        assert Schema.instance.getKSMetaData("EmptyKeyspace") != null;
-
-        CFMetaData newCf = addTestCF("EmptyKeyspace", "AddedLater", "A new CF to add to an empty KS");
-
-        //should not exist until apply
-        assert !Schema.instance.getKSMetaData(newKs.name).cfMetaData().containsKey(newCf.cfName);
-
-        //add the new CF to the empty space
-        MigrationManager.announceNewColumnFamily(newCf);
-
-        assert Schema.instance.getKSMetaData(newKs.name).cfMetaData().containsKey(newCf.cfName);
-        assert Schema.instance.getKSMetaData(newKs.name).cfMetaData().get(newCf.cfName).equals(newCf);
-
-        // now read and write to it.
-        CellName col0 = cellname("col0");
-        DecoratedKey dk = Util.dk("key0");
-        Mutation rm = new Mutation(newKs.name, dk.getKey());
-        rm.add(newCf.cfName, col0, ByteBufferUtil.bytes("value0"), 1L);
-        rm.apply();
-        ColumnFamilyStore store = Keyspace.open(newKs.name).getColumnFamilyStore(newCf.cfName);
-        assert store != null;
-        store.forceBlockingFlush();
-
-        ColumnFamily cfam = store.getColumnFamily(Util.namesQueryFilter(store, dk, col0));
-        assert cfam.getColumn(col0) != null;
-        Cell col = cfam.getColumn(col0);
-        assert ByteBufferUtil.bytes("value0").equals(col.value());
-    }
-
-    @Test
-    public void testUpdateKeyspace() throws ConfigurationException
-    {
-        // create a keyspace to serve as existing.
-        CFMetaData cf = addTestCF("UpdatedKeyspace", "AddedStandard1", "A new cf for a new ks");
-        KSMetaData oldKs = KSMetaData.testMetadata(cf.ksName, SimpleStrategy.class, KSMetaData.optsWithRF(5), cf);
-
-        MigrationManager.announceNewKeyspace(oldKs);
-
-        assert Schema.instance.getKSMetaData(cf.ksName) != null;
-        assert Schema.instance.getKSMetaData(cf.ksName).equals(oldKs);
-
-        // names should match.
-        KSMetaData newBadKs2 = KSMetaData.testMetadata(cf.ksName + "trash", SimpleStrategy.class, KSMetaData.optsWithRF(4));
-        try
-        {
-            MigrationManager.announceKeyspaceUpdate(newBadKs2);
-            throw new AssertionError("Should not have been able to update a KS with an invalid KS name.");
-        }
-        catch (ConfigurationException ex)
-        {
-            // expected.
-        }
-
-        KSMetaData newKs = KSMetaData.testMetadata(cf.ksName, OldNetworkTopologyStrategy.class, KSMetaData.optsWithRF(1));
-        MigrationManager.announceKeyspaceUpdate(newKs);
-
-        KSMetaData newFetchedKs = Schema.instance.getKSMetaData(newKs.name);
-        assert newFetchedKs.strategyClass.equals(newKs.strategyClass);
-        assert !newFetchedKs.strategyClass.equals(oldKs.strategyClass);
-    }
-
-    @Test
-    public void testUpdateColumnFamilyNoIndexes() throws ConfigurationException
-    {
-        // create a keyspace with a cf to update.
-        CFMetaData cf = addTestCF("UpdatedCfKs", "Standard1added", "A new cf that will be updated");
-        KSMetaData ksm = KSMetaData.testMetadata(cf.ksName, SimpleStrategy.class, KSMetaData.optsWithRF(1), cf);
-        MigrationManager.announceNewKeyspace(ksm);
-
-        assert Schema.instance.getKSMetaData(cf.ksName) != null;
-        assert Schema.instance.getKSMetaData(cf.ksName).equals(ksm);
-        assert Schema.instance.getCFMetaData(cf.ksName, cf.cfName) != null;
-
-        // updating certain fields should fail.
-        CFMetaData newCfm = cf.copy();
-        newCfm.defaultValidator(BytesType.instance);
-        newCfm.minCompactionThreshold(5);
-        newCfm.maxCompactionThreshold(31);
-
-        // test valid operations.
-        newCfm.comment("Modified comment");
-        MigrationManager.announceColumnFamilyUpdate(newCfm, false); // doesn't get set back here.
-
-        newCfm.readRepairChance(0.23);
-        MigrationManager.announceColumnFamilyUpdate(newCfm, false);
-
-        newCfm.gcGraceSeconds(12);
-        MigrationManager.announceColumnFamilyUpdate(newCfm, false);
-
-        newCfm.defaultValidator(UTF8Type.instance);
-        MigrationManager.announceColumnFamilyUpdate(newCfm, false);
-
-        newCfm.minCompactionThreshold(3);
-        MigrationManager.announceColumnFamilyUpdate(newCfm, false);
-
-        newCfm.maxCompactionThreshold(33);
-        MigrationManager.announceColumnFamilyUpdate(newCfm, false);
-
-        // can't test changing the reconciler because there is only one impl.
-
-        // check the cumulative affect.
-        assert Schema.instance.getCFMetaData(cf.ksName, cf.cfName).getComment().equals(newCfm.getComment());
-        assert Schema.instance.getCFMetaData(cf.ksName, cf.cfName).getReadRepairChance() == newCfm.getReadRepairChance();
-        assert Schema.instance.getCFMetaData(cf.ksName, cf.cfName).getGcGraceSeconds() == newCfm.getGcGraceSeconds();
-        assert Schema.instance.getCFMetaData(cf.ksName, cf.cfName).getDefaultValidator() == UTF8Type.instance;
-
-        // Change cfId
-        newCfm = new CFMetaData(cf.ksName, cf.cfName, cf.cfType, cf.comparator);
-        CFMetaData.copyOpts(newCfm, cf);
-        try
-        {
-            cf.apply(newCfm);
-            throw new AssertionError("Should have blown up when you used a different id.");
-        }
-        catch (ConfigurationException expected) {}
-
-        // Change cfName
-        newCfm = new CFMetaData(cf.ksName, cf.cfName + "_renamed", cf.cfType, cf.comparator);
-        CFMetaData.copyOpts(newCfm, cf);
-        try
-        {
-            cf.apply(newCfm);
-            throw new AssertionError("Should have blown up when you used a different name.");
-        }
-        catch (ConfigurationException expected) {}
-
-        // Change ksName
-        newCfm = new CFMetaData(cf.ksName + "_renamed", cf.cfName, cf.cfType, cf.comparator);
-        CFMetaData.copyOpts(newCfm, cf);
-        try
-        {
-            cf.apply(newCfm);
-            throw new AssertionError("Should have blown up when you used a different keyspace.");
-        }
-        catch (ConfigurationException expected) {}
-
-        // Change cf type
-        newCfm = new CFMetaData(cf.ksName, cf.cfName, ColumnFamilyType.Super, cf.comparator);
-        CFMetaData.copyOpts(newCfm, cf);
-        try
-        {
-            cf.apply(newCfm);
-            throw new AssertionError("Should have blwon up when you used a different cf type.");
-        }
-        catch (ConfigurationException expected) {}
-
-        // Change comparator
-        newCfm = new CFMetaData(cf.ksName, cf.cfName, cf.cfType, new SimpleDenseCellNameType(TimeUUIDType.instance));
-        CFMetaData.copyOpts(newCfm, cf);
-        try
-        {
-            cf.apply(newCfm);
-            throw new AssertionError("Should have blown up when you used a different comparator.");
-        }
-        catch (ConfigurationException expected) {}
-    }
-
-    @Test
-    public void testDropIndex() throws ConfigurationException
-    {
-        // persist keyspace definition in the system keyspace
-        Schema.instance.getKSMetaData("Keyspace6").toSchema(System.currentTimeMillis()).apply();
-        ColumnFamilyStore cfs = Keyspace.open("Keyspace6").getColumnFamilyStore("Indexed1");
-
-        // insert some data.  save the sstable descriptor so we can make sure it's marked for delete after the drop
-        Mutation rm = new Mutation("Keyspace6", ByteBufferUtil.bytes("k1"));
-        rm.add("Indexed1", cellname("notbirthdate"), ByteBufferUtil.bytes(1L), 0);
-        rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(1L), 0);
-        rm.apply();
-        cfs.forceBlockingFlush();
-        ColumnFamilyStore indexedCfs = cfs.indexManager.getIndexForColumn(ByteBufferUtil.bytes("birthdate")).getIndexCfs();
-        Descriptor desc = indexedCfs.getSSTables().iterator().next().descriptor;
-
-        // drop the index
-        CFMetaData meta = cfs.metadata.copy();
-        ColumnDefinition cdOld = meta.regularColumns().iterator().next();
-        ColumnDefinition cdNew = ColumnDefinition.regularDef(meta, cdOld.name.bytes, cdOld.type, null);
-        meta.addOrReplaceColumnDefinition(cdNew);
-        MigrationManager.announceColumnFamilyUpdate(meta, false);
-
-        // check
-        assert cfs.indexManager.getIndexes().isEmpty();
-        SSTableDeletingTask.waitForDeletions();
-        assert !new File(desc.filenameFor(Component.DATA)).exists();
-    }
-
-    private CFMetaData addTestCF(String ks, String cf, String comment)
-    {
-        CFMetaData newCFMD = new CFMetaData(ks, cf, ColumnFamilyType.Standard, new SimpleDenseCellNameType(UTF8Type.instance));
-        newCFMD.comment(comment)
-               .readRepairChance(0.0);
-
-        return newCFMD;
-    }
-}
diff --git a/test/unit/org/apache/cassandra/config/KSMetaDataTest.java b/test/unit/org/apache/cassandra/config/KSMetaDataTest.java
index 9093086..39be913 100644
--- a/test/unit/org/apache/cassandra/config/KSMetaDataTest.java
+++ b/test/unit/org/apache/cassandra/config/KSMetaDataTest.java
@@ -19,7 +19,6 @@
 
 package org.apache.cassandra.config;
 
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -39,11 +38,7 @@
         options.put("key2", "value2");
         options.put("key3", "value3");
 
-        KSMetaData ksMeta = new KSMetaData("test",
-                                            SimpleStrategy.class,
-                                            options,
-                                            true,
-                                            Collections.<CFMetaData>emptyList());
+        KSMetaData ksMeta = new KSMetaData("test", SimpleStrategy.class, options, true);
 
         assertTrue(ksMeta.toString().contains(options.toString()));
     }
diff --git a/test/unit/org/apache/cassandra/config/LegacySchemaTablesTest.java b/test/unit/org/apache/cassandra/config/LegacySchemaTablesTest.java
new file mode 100644
index 0000000..f630c88
--- /dev/null
+++ b/test/unit/org/apache/cassandra/config/LegacySchemaTablesTest.java
@@ -0,0 +1,208 @@
+/*
+ * 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.config;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.HashMap;
+import java.util.HashSet;
+
+import com.google.common.collect.Iterables;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.marshal.AsciiType;
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.compress.*;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.schema.LegacySchemaTables;
+import org.apache.cassandra.service.MigrationManager;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.thrift.CfDef;
+import org.apache.cassandra.thrift.ColumnDef;
+import org.apache.cassandra.thrift.IndexType;
+import org.apache.cassandra.thrift.ThriftConversion;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+public class LegacySchemaTablesTest
+{
+    private static final String KEYSPACE1 = "CFMetaDataTest1";
+    private static final String CF_STANDARD1 = "Standard1";
+    private static final String CF_STANDARD2 = "Standard2";
+
+    private static List<ColumnDef> columnDefs = new ArrayList<ColumnDef>();
+
+    static
+    {
+        columnDefs.add(new ColumnDef(ByteBufferUtil.bytes("col1"), AsciiType.class.getCanonicalName())
+                                    .setIndex_name("col1Index")
+                                    .setIndex_type(IndexType.KEYS));
+
+        columnDefs.add(new ColumnDef(ByteBufferUtil.bytes("col2"), UTF8Type.class.getCanonicalName())
+                                    .setIndex_name("col2Index")
+                                    .setIndex_type(IndexType.KEYS));
+    }
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1));
+    }
+
+    @Test
+    public void testIsDenseRecalculation()
+    {
+        // 1.a start with a dense CF
+        CfDef cfDef0 = new CfDef().setDefault_validation_class(BytesType.class.getCanonicalName())
+                                  .setComparator_type(UTF8Type.class.getCanonicalName())
+                                  .setColumn_metadata(Collections.<ColumnDef>emptyList())
+                                  .setKeyspace(KEYSPACE1)
+                                  .setName(CF_STANDARD2);
+        CFMetaData cfm0 = ThriftConversion.fromThrift(cfDef0);
+        MigrationManager.announceNewColumnFamily(cfm0, true);
+
+        // 1.b validate that the cf is dense, has a single compact value and a clustering column, and no regulars
+        CFMetaData current = Schema.instance.getCFMetaData(KEYSPACE1, CF_STANDARD2);
+        assertTrue(current.getIsDense());
+        assertNotNull(current.compactValueColumn());
+        assertEquals(0, Iterables.size(current.regularAndStaticColumns()));
+        assertEquals(1, current.clusteringColumns().size());
+
+        // 2.a add a column to the table
+        CfDef cfDef1 = ThriftConversion.toThrift(current);
+        List<ColumnDef> colDefs =
+            Collections.singletonList(new ColumnDef(ByteBufferUtil.bytes("col1"), AsciiType.class.getCanonicalName()));
+        cfDef1.setColumn_metadata(colDefs);
+        CFMetaData cfm1 = ThriftConversion.fromThriftForUpdate(cfDef1, current);
+        MigrationManager.announceColumnFamilyUpdate(cfm1, true);
+
+        // 2.b validate that the cf is sparse now, had no compact value column or clustering column, and 1 regular
+        current = Schema.instance.getCFMetaData(KEYSPACE1, CF_STANDARD2);
+        assertFalse(current.getIsDense());
+        assertNull(current.compactValueColumn());
+        assertEquals(1, Iterables.size(current.regularAndStaticColumns()));
+        assertEquals(0, current.clusteringColumns().size());
+
+        // 3.a remove the column
+        CfDef cfDef2 = ThriftConversion.toThrift(current);
+        cfDef2.setColumn_metadata(Collections.<ColumnDef>emptyList());
+        CFMetaData cfm2 = ThriftConversion.fromThriftForUpdate(cfDef2, current);
+        MigrationManager.announceColumnFamilyUpdate(cfm2, true);
+
+        // 3.b validate that the cf is dense, has a single compact value and a clustering column, and no regulars
+        current = Schema.instance.getCFMetaData(KEYSPACE1, CF_STANDARD2);
+        assertTrue(current.getIsDense());
+        assertNotNull(current.compactValueColumn());
+        assertEquals(0, Iterables.size(current.regularAndStaticColumns()));
+        assertEquals(1, current.clusteringColumns().size());
+    }
+
+    @Test
+    public void testThriftConversion() throws Exception
+    {
+        CfDef cfDef = new CfDef().setDefault_validation_class(AsciiType.class.getCanonicalName())
+                                 .setComment("Test comment")
+                                 .setColumn_metadata(columnDefs)
+                                 .setKeyspace(KEYSPACE1)
+                                 .setName(CF_STANDARD1);
+
+        // convert Thrift to CFMetaData
+        CFMetaData cfMetaData = ThriftConversion.fromThrift(cfDef);
+
+        CfDef thriftCfDef = new CfDef();
+        thriftCfDef.keyspace = KEYSPACE1;
+        thriftCfDef.name = CF_STANDARD1;
+        thriftCfDef.default_validation_class = cfDef.default_validation_class;
+        thriftCfDef.comment = cfDef.comment;
+        thriftCfDef.column_metadata = new ArrayList<>();
+        for (ColumnDef columnDef : columnDefs)
+        {
+            ColumnDef c = new ColumnDef();
+            c.name = ByteBufferUtil.clone(columnDef.name);
+            c.validation_class = columnDef.getValidation_class();
+            c.index_name = columnDef.getIndex_name();
+            c.index_type = IndexType.KEYS;
+            thriftCfDef.column_metadata.add(c);
+        }
+
+        CfDef converted = ThriftConversion.toThrift(cfMetaData);
+
+        assertEquals(thriftCfDef.keyspace, converted.keyspace);
+        assertEquals(thriftCfDef.name, converted.name);
+        assertEquals(thriftCfDef.default_validation_class, converted.default_validation_class);
+        assertEquals(thriftCfDef.comment, converted.comment);
+        assertEquals(new HashSet<>(thriftCfDef.column_metadata), new HashSet<>(converted.column_metadata));
+    }
+
+    @Test
+    public void testConversionsInverses() throws Exception
+    {
+        for (String keyspaceName : Schema.instance.getNonSystemKeyspaces())
+        {
+            for (ColumnFamilyStore cfs : Keyspace.open(keyspaceName).getColumnFamilyStores())
+            {
+                CFMetaData cfm = cfs.metadata;
+                if (!cfm.isThriftCompatible())
+                    continue;
+
+                checkInverses(cfm);
+
+                // Testing with compression to catch #3558
+                CFMetaData withCompression = cfm.copy();
+                withCompression.compressionParameters(new CompressionParameters(SnappyCompressor.instance, 32768, new HashMap<String, String>()));
+                checkInverses(withCompression);
+            }
+        }
+    }
+
+    private void checkInverses(CFMetaData cfm) throws Exception
+    {
+        DecoratedKey k = StorageService.getPartitioner().decorateKey(ByteBufferUtil.bytes(cfm.ksName));
+        KSMetaData keyspace = Schema.instance.getKSMetaData(cfm.ksName);
+
+        // Test thrift conversion
+        CFMetaData before = cfm;
+        CFMetaData after = ThriftConversion.fromThriftForUpdate(ThriftConversion.toThrift(before), before);
+        assert before.equals(after) : String.format("%n%s%n!=%n%s", before, after);
+
+        // Test schema conversion
+        Mutation rm = LegacySchemaTables.makeCreateTableMutation(keyspace, cfm, FBUtilities.timestampMicros());
+        ColumnFamily serializedCf = rm.getColumnFamily(Schema.instance.getId(SystemKeyspace.NAME, LegacySchemaTables.COLUMNFAMILIES));
+        ColumnFamily serializedCD = rm.getColumnFamily(Schema.instance.getId(SystemKeyspace.NAME, LegacySchemaTables.COLUMNS));
+        CFMetaData newCfm = LegacySchemaTables.createTableFromTablePartitionAndColumnsPartition(new Row(k, serializedCf), new Row(k, serializedCD));
+        assert cfm.equals(newCfm) : String.format("%n%s%n!=%n%s", cfm, newCfm);
+    }
+}
diff --git a/test/unit/org/apache/cassandra/cql3/BatchTests.java b/test/unit/org/apache/cassandra/cql3/BatchTests.java
index 27d407e..95fe612 100644
--- a/test/unit/org/apache/cassandra/cql3/BatchTests.java
+++ b/test/unit/org/apache/cassandra/cql3/BatchTests.java
@@ -29,7 +29,6 @@
 import org.junit.Test;
 
 import java.io.IOException;
-import java.util.concurrent.TimeUnit;
 
 public class BatchTests
 {
@@ -122,6 +121,18 @@
         sendBatch(BatchStatement.Type.LOGGED, true, false);
     }
 
+    @Test(expected = InvalidQueryException.class)
+    public void testOversizedBatch()
+    {
+        int SIZE_FOR_FAILURE = 2500;
+        BatchStatement b = new BatchStatement(BatchStatement.Type.UNLOGGED);
+        for (int i = 0; i < SIZE_FOR_FAILURE; i++)
+        {
+            b.add(noncounter.bind(i, "foobar"));
+        }
+        session.execute(b);
+    }
+
 
 
     public void sendBatch(BatchStatement.Type type, boolean addCounter, boolean addNonCounter)
diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java
index 34c0980..98b8e23 100644
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@ -21,6 +21,7 @@
 import java.math.BigDecimal;
 import java.math.BigInteger;
 import java.net.InetAddress;
+import java.net.ServerSocket;
 import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.concurrent.CountDownLatch;
@@ -30,28 +31,43 @@
 
 import com.google.common.base.Objects;
 import com.google.common.collect.ImmutableSet;
-import org.junit.AfterClass;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
+import org.junit.*;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static junit.framework.Assert.assertNotNull;
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnDefinitions;
+import com.datastax.driver.core.DataType;
+import com.datastax.driver.core.ProtocolVersion;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.concurrent.ScheduledExecutors;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.cql3.functions.FunctionName;
+import org.apache.cassandra.cql3.statements.ParsedStatement;
 import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.db.marshal.TupleType;
+import org.apache.cassandra.exceptions.CassandraException;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.dht.Murmur3Partitioner;
-import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.serializers.TypeSerializer;
-import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.transport.Event;
+import org.apache.cassandra.transport.Server;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.ByteBufferUtil;
 
 /**
  * Base class for CQL tests.
@@ -61,19 +77,67 @@
     protected static final Logger logger = LoggerFactory.getLogger(CQLTester.class);
 
     public static final String KEYSPACE = "cql_test_keyspace";
-    private static final boolean USE_PREPARED_VALUES = Boolean.valueOf(System.getProperty("cassandra.test.use_prepared", "true"));
+    public static final String KEYSPACE_PER_TEST = "cql_test_keyspace_alt";
+    protected static final boolean USE_PREPARED_VALUES = Boolean.valueOf(System.getProperty("cassandra.test.use_prepared", "true"));
     protected static final long ROW_CACHE_SIZE_IN_MB = Integer.valueOf(System.getProperty("cassandra.test.row_cache_size_in_mb", "0"));
     private static final AtomicInteger seqNumber = new AtomicInteger();
-    protected static final ByteBuffer TOO_BIG = ByteBuffer.allocate(FBUtilities.MAX_UNSIGNED_SHORT + 1024);
+    protected static final ByteBuffer TOO_BIG = ByteBuffer.allocate(1024 * 65);
 
-    static
-    {
+    private static org.apache.cassandra.transport.Server server;
+    protected static final int nativePort;
+    protected static final InetAddress nativeAddr;
+    private static final Cluster[] cluster;
+    private static final Session[] session;
+
+    public static int maxProtocolVersion;
+    static {
+        int version;
+        for (version = 1; version <= Server.CURRENT_VERSION; )
+        {
+            try
+            {
+                ProtocolVersion.fromInt(++version);
+            }
+            catch (IllegalArgumentException e)
+            {
+                version--;
+                break;
+            }
+        }
+        maxProtocolVersion = version;
+        cluster = new Cluster[maxProtocolVersion];
+        session = new Session[maxProtocolVersion];
+
         // Once per-JVM is enough
         SchemaLoader.prepareServer();
+
+        nativeAddr = InetAddress.getLoopbackAddress();
+
+        try
+        {
+            try (ServerSocket serverSocket = new ServerSocket(0))
+            {
+                nativePort = serverSocket.getLocalPort();
+            }
+            Thread.sleep(250);
+        }
+        catch (Exception e)
+        {
+            throw new RuntimeException(e);
+        }
     }
 
+    public static ResultMessage lastSchemaChangeResult;
+
     private List<String> tables = new ArrayList<>();
     private List<String> types = new ArrayList<>();
+    private List<String> functions = new ArrayList<>();
+    private List<String> aggregates = new ArrayList<>();
+
+    // We don't use USE_PREPARED_VALUES in the code below so some test can foce value preparation (if the result
+    // is not expected to be the same without preparation)
+    private boolean usePrepared = USE_PREPARED_VALUES;
+    private static final boolean reusePrepared = Boolean.valueOf(System.getProperty("cassandra.test.reuse_prepared", "true"));
 
     @BeforeClass
     public static void setUpClass()
@@ -81,27 +145,51 @@
         if (ROW_CACHE_SIZE_IN_MB > 0)
             DatabaseDescriptor.setRowCacheSizeInMB(ROW_CACHE_SIZE_IN_MB);
 
-        DatabaseDescriptor.setPartitioner(new Murmur3Partitioner());
+        DatabaseDescriptor.setPartitioner(Murmur3Partitioner.instance);
     }
 
     @AfterClass
     public static void tearDownClass()
     {
+        for (Session sess : session)
+            if (sess != null)
+                sess.close();
+        for (Cluster cl : cluster)
+            if (cl != null)
+                cl.close();
+
+        if (server != null)
+            server.stop();
+
+        // We use queryInternal for CQLTester so prepared statement will populate our internal cache (if reusePrepared is used; otherwise prepared
+        // statements are not cached but re-prepared every time). So we clear the cache between test files to avoid accumulating too much.
+        if (reusePrepared)
+            QueryProcessor.clearInternalStatementsCache();
     }
 
     @Before
     public void beforeTest() throws Throwable
     {
         schemaChange(String.format("CREATE KEYSPACE IF NOT EXISTS %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}", KEYSPACE));
+        schemaChange(String.format("CREATE KEYSPACE IF NOT EXISTS %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}", KEYSPACE_PER_TEST));
     }
 
     @After
     public void afterTest() throws Throwable
     {
+        dropPerTestKeyspace();
+
+        // Restore standard behavior in case it was changed
+        usePrepared = USE_PREPARED_VALUES;
+
         final List<String> tablesToDrop = copy(tables);
         final List<String> typesToDrop = copy(types);
+        final List<String> functionsToDrop = copy(functions);
+        final List<String> aggregatesToDrop = copy(aggregates);
         tables = null;
         types = null;
+        functions = null;
+        aggregates = null;
 
         // We want to clean up after the test, but dropping a table is rather long so just do that asynchronously
         ScheduledExecutors.optionalTasks.execute(new Runnable()
@@ -113,6 +201,12 @@
                     for (int i = tablesToDrop.size() - 1; i >= 0; i--)
                         schemaChange(String.format("DROP TABLE IF EXISTS %s.%s", KEYSPACE, tablesToDrop.get(i)));
 
+                    for (int i = aggregatesToDrop.size() - 1; i >= 0; i--)
+                        schemaChange(String.format("DROP AGGREGATE IF EXISTS %s", aggregatesToDrop.get(i)));
+
+                    for (int i = functionsToDrop.size() - 1; i >= 0; i--)
+                        schemaChange(String.format("DROP FUNCTION IF EXISTS %s", functionsToDrop.get(i)));
+
                     for (int i = typesToDrop.size() - 1; i >= 0; i--)
                         schemaChange(String.format("DROP TYPE IF EXISTS %s.%s", KEYSPACE, typesToDrop.get(i)));
 
@@ -141,6 +235,40 @@
         });
     }
 
+    // lazy initialization for all tests that require Java Driver
+    protected static void requireNetwork() throws ConfigurationException
+    {
+        if (server != null)
+            return;
+
+        SystemKeyspace.finishStartup();
+        StorageService.instance.initServer();
+        SchemaLoader.startGossiper();
+
+        server = new org.apache.cassandra.transport.Server(nativeAddr, nativePort);
+        server.start();
+
+        for (int version = 1; version <= maxProtocolVersion; version++)
+        {
+            if (cluster[version-1] != null)
+                continue;
+
+            cluster[version-1] = Cluster.builder().addContactPoints(nativeAddr)
+                                  .withClusterName("Test Cluster")
+                                  .withPort(nativePort)
+                                  .withProtocolVersion(ProtocolVersion.fromInt(version))
+                                  .build();
+            session[version-1] = cluster[version-1].connect();
+
+            logger.info("Started Java Driver instance for protocol version {}", version);
+        }
+    }
+
+    protected void dropPerTestKeyspace() throws Throwable
+    {
+        execute(String.format("DROP KEYSPACE IF EXISTS %s", KEYSPACE_PER_TEST));
+    }
+
     /**
      * Returns a copy of the specified list.
      * @return a copy of the specified list.
@@ -158,11 +286,7 @@
             if (currentTable != null)
                 Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable).forceFlush().get();
         }
-        catch (InterruptedException e)
-        {
-            throw new RuntimeException(e);
-        }
-        catch (ExecutionException e)
+        catch (InterruptedException | ExecutionException e)
         {
             throw new RuntimeException(e);
         }
@@ -189,6 +313,19 @@
             Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable).cleanupCache();
     }
 
+    public static FunctionName parseFunctionName(String qualifiedName)
+    {
+        int i = qualifiedName.indexOf('.');
+        return i == -1
+               ? FunctionName.nativeFunction(qualifiedName)
+               : new FunctionName(qualifiedName.substring(0, i).trim(), qualifiedName.substring(i+1).trim());
+    }
+
+    public static String shortFunctionName(String f)
+    {
+        return parseFunctionName(f).name;
+    }
+
     private static void removeAllSSTables(String ks, List<String> tables)
     {
         // clean up data directory which are stored as data directory/keyspace/data files
@@ -202,7 +339,9 @@
     private static boolean containsAny(String filename, List<String> tables)
     {
         for (int i = 0, m = tables.size(); i < m; i++)
-            if (filename.contains(tables.get(i)))
+            // don't accidentally delete in-use directories with the
+            // same prefix as a table to delete, i.e. table_1 & table_11
+            if (filename.contains(tables.get(i) + "-"))
                 return true;
         return false;
     }
@@ -219,6 +358,21 @@
         return tables.get(tables.size() - 1);
     }
 
+    protected ByteBuffer unset()
+    {
+        return ByteBufferUtil.UNSET_BYTE_BUFFER;
+    }
+
+    protected void forcePreparedValues()
+    {
+        this.usePrepared = true;
+    }
+
+    protected void stopForcingPreparedValues()
+    {
+        this.usePrepared = USE_PREPARED_VALUES;
+    }
+
     protected String createType(String query)
     {
         String typeName = "type_" + seqNumber.getAndIncrement();
@@ -229,10 +383,40 @@
         return typeName;
     }
 
+    protected String createFunction(String keyspace, String argTypes, String query) throws Throwable
+    {
+        String functionName = keyspace + ".function_" + seqNumber.getAndIncrement();
+        createFunctionOverload(functionName, argTypes, query);
+        return functionName;
+    }
+
+    protected void createFunctionOverload(String functionName, String argTypes, String query) throws Throwable
+    {
+        String fullQuery = String.format(query, functionName);
+        functions.add(functionName + '(' + argTypes + ')');
+        logger.info(fullQuery);
+        schemaChange(fullQuery);
+    }
+
+    protected String createAggregate(String keyspace, String argTypes, String query) throws Throwable
+    {
+        String aggregateName = keyspace + "." + "aggregate_" + seqNumber.getAndIncrement();
+        createAggregateOverload(aggregateName, argTypes, query);
+        return aggregateName;
+    }
+
+    protected void createAggregateOverload(String aggregateName, String argTypes, String query) throws Throwable
+    {
+        String fullQuery = String.format(query, aggregateName);
+        aggregates.add(aggregateName + '(' + argTypes + ')');
+        logger.info(fullQuery);
+        schemaChange(fullQuery);
+    }
+
     protected String createTable(String query)
     {
         String currentTable = createTableName();
-        String fullQuery = String.format(query, KEYSPACE + "." + currentTable);
+        String fullQuery = formatQuery(query);
         logger.info(fullQuery);
         schemaChange(fullQuery);
         return currentTable;
@@ -249,37 +433,23 @@
     {
         String currentTable = "table_" + seqNumber.getAndIncrement();
         tables.add(currentTable);
-        String fullQuery = String.format(query, KEYSPACE + "." + currentTable);
+        String fullQuery = formatQuery(query);
         logger.info(fullQuery);
-        try
-        {
-            QueryProcessor.executeOnceInternal(fullQuery);
-        }
-        catch (RuntimeException ex)
-        {
-            throw ex.getCause();
-        }
+        QueryProcessor.executeOnceInternal(fullQuery);
     }
 
     protected void alterTable(String query)
     {
-        String fullQuery = String.format(query, KEYSPACE + "." + currentTable());
+        String fullQuery = formatQuery(query);
         logger.info(fullQuery);
         schemaChange(fullQuery);
     }
 
     protected void alterTableMayThrow(String query) throws Throwable
     {
-        String fullQuery = String.format(query, KEYSPACE + "." + currentTable());
+        String fullQuery = formatQuery(query);
         logger.info(fullQuery);
-        try
-        {
-            QueryProcessor.executeOnceInternal(fullQuery);
-        }
-        catch (RuntimeException ex)
-        {
-            throw ex.getCause();
-        }
+        QueryProcessor.executeOnceInternal(fullQuery);
     }
 
     protected void dropTable(String query)
@@ -291,7 +461,7 @@
 
     protected void createIndex(String query)
     {
-        String fullQuery = String.format(query, KEYSPACE + "." + currentTable());
+        String fullQuery = formatQuery(query);
         logger.info(fullQuery);
         schemaChange(fullQuery);
     }
@@ -329,16 +499,9 @@
 
     protected void createIndexMayThrow(String query) throws Throwable
     {
-        String fullQuery = String.format(query, KEYSPACE + "." + currentTable());
+        String fullQuery = formatQuery(query);
         logger.info(fullQuery);
-        try
-        {
-            QueryProcessor.executeOnceInternal(fullQuery);
-        }
-        catch (RuntimeException ex)
-        {
-            throw ex.getCause();
-        }
+        QueryProcessor.executeOnceInternal(fullQuery);
     }
 
     protected void dropIndex(String query) throws Throwable
@@ -348,12 +511,33 @@
         schemaChange(fullQuery);
     }
 
+    protected void assertLastSchemaChange(Event.SchemaChange.Change change, Event.SchemaChange.Target target,
+                                          String keyspace, String name,
+                                          String... argTypes)
+    {
+        Assert.assertTrue(lastSchemaChangeResult instanceof ResultMessage.SchemaChange);
+        ResultMessage.SchemaChange schemaChange = (ResultMessage.SchemaChange) lastSchemaChangeResult;
+        Assert.assertSame(change, schemaChange.change.change);
+        Assert.assertSame(target, schemaChange.change.target);
+        Assert.assertEquals(keyspace, schemaChange.change.keyspace);
+        Assert.assertEquals(name, schemaChange.change.name);
+        Assert.assertEquals(argTypes != null ? Arrays.asList(argTypes) : null, schemaChange.change.argTypes);
+    }
+
     protected static void schemaChange(String query)
     {
         try
         {
-            // executeOnceInternal don't work for schema changes
-            QueryProcessor.executeOnceInternal(query);
+            ClientState state = ClientState.forInternalCalls();
+            state.setKeyspace(SystemKeyspace.NAME);
+            QueryState queryState = new QueryState(state);
+
+            ParsedStatement.Prepared prepared = QueryProcessor.parseStatement(query, queryState);
+            prepared.statement.validate(state);
+
+            QueryOptions options = QueryOptions.forInternalCalls(Collections.<ByteBuffer>emptyList());
+
+            lastSchemaChangeResult = prepared.statement.executeInternal(queryState, options);
         }
         catch (Exception e)
         {
@@ -366,34 +550,117 @@
         return Schema.instance.getCFMetaData(KEYSPACE, currentTable());
     }
 
+    protected com.datastax.driver.core.ResultSet executeNet(int protocolVersion, String query, Object... values) throws Throwable
+    {
+        requireNetwork();
+
+        return session[protocolVersion-1].execute(formatQuery(query), values);
+    }
+
+    protected Session sessionNet(int protocolVersion)
+    {
+        requireNetwork();
+
+        return session[protocolVersion-1];
+    }
+
+    private String formatQuery(String query)
+    {
+        String currentTable = currentTable();
+        return currentTable == null ? query : String.format(query, KEYSPACE + "." + currentTable);
+    }
+
     protected UntypedResultSet execute(String query, Object... values) throws Throwable
     {
-        try
-        {
-            query = currentTable() == null ? query : String.format(query, KEYSPACE + "." + currentTable());
+        query = formatQuery(query);
 
-            UntypedResultSet rs;
-            if (USE_PREPARED_VALUES)
+        UntypedResultSet rs;
+        if (usePrepared)
+        {
+            logger.info("Executing: {} with values {}", query, formatAllValues(values));
+            if (reusePrepared)
             {
-                logger.info("Executing: {} with values {}", query, formatAllValues(values));
-                rs = QueryProcessor.executeOnceInternal(query, transformValues(values));
+                rs = QueryProcessor.executeInternal(query, transformValues(values));
+
+                // If a test uses a "USE ...", then presumably its statements use relative table. In that case, a USE
+                // change the meaning of the current keyspace, so we don't want a following statement to reuse a previously
+                // prepared statement at this wouldn't use the right keyspace. To avoid that, we drop the previously
+                // prepared statement.
+                if (query.startsWith("USE"))
+                    QueryProcessor.clearInternalStatementsCache();
             }
             else
             {
-                query = replaceValues(query, values);
-                logger.info("Executing: {}", query);
-                rs = QueryProcessor.executeOnceInternal(query);
+                rs = QueryProcessor.executeOnceInternal(query, transformValues(values));
             }
-            if (rs != null)
-                logger.info("Got {} rows", rs.size());
-            return rs;
         }
-        catch (RuntimeException e)
+        else
         {
-            Throwable cause = e.getCause() != null ? e.getCause() : e;
-            logger.info("Got error: {}", cause.getMessage() == null ? cause.toString() : cause.getMessage());
-            throw cause;
+            query = replaceValues(query, values);
+            logger.info("Executing: {}", query);
+            rs = QueryProcessor.executeOnceInternal(query);
         }
+        if (rs != null)
+            logger.info("Got {} rows", rs.size());
+        return rs;
+    }
+
+    protected void assertRowsNet(int protocolVersion, ResultSet result, Object[]... rows)
+    {
+        if (result == null)
+        {
+            if (rows.length > 0)
+                Assert.fail(String.format("No rows returned by query but %d expected", rows.length));
+            return;
+        }
+
+        ColumnDefinitions meta = result.getColumnDefinitions();
+        Iterator<Row> iter = result.iterator();
+        int i = 0;
+        while (iter.hasNext() && i < rows.length)
+        {
+            Object[] expected = rows[i];
+            Row actual = iter.next();
+
+            Assert.assertEquals(String.format("Invalid number of (expected) values provided for row %d (using protocol version %d)",
+                                              i, protocolVersion),
+                                meta.size(), expected.length);
+
+            for (int j = 0; j < meta.size(); j++)
+            {
+                DataType type = meta.getType(j);
+                ByteBuffer expectedByteValue = type.serialize(expected[j], ProtocolVersion.fromInt(protocolVersion));
+                int expectedBytes = expectedByteValue.remaining();
+                ByteBuffer actualValue = actual.getBytesUnsafe(meta.getName(j));
+                int actualBytes = actualValue.remaining();
+
+                if (!Objects.equal(expectedByteValue, actualValue))
+                    Assert.fail(String.format("Invalid value for row %d column %d (%s of type %s), " +
+                                              "expected <%s> (%d bytes) but got <%s> (%d bytes) " +
+                                              "(using protocol version %d)",
+                                              i, j, meta.getName(j), type,
+                                              type.format(expected[j]),
+                                              expectedBytes,
+                                              type.format(type.deserialize(actualValue, ProtocolVersion.fromInt(protocolVersion))),
+                                              actualBytes,
+                                              protocolVersion));
+            }
+            i++;
+        }
+
+        if (iter.hasNext())
+        {
+            while (iter.hasNext())
+            {
+                iter.next();
+                i++;
+            }
+            Assert.fail(String.format("Got less rows than expected. Expected %d but got %d (using protocol version %d).",
+                                      rows.length, i, protocolVersion));
+        }
+
+        Assert.assertTrue(String.format("Got %s rows than expected. Expected %d but got %d (using protocol version %d)",
+                                        rows.length>i ? "less" : "more", rows.length, i, protocolVersion), i == rows.length);
     }
 
     protected void assertRows(UntypedResultSet result, Object[]... rows)
@@ -447,7 +714,7 @@
             Assert.fail(String.format("Got less rows than expected. Expected %d but got %d.", rows.length, i));
         }
 
-        Assert.assertTrue(String.format("Got more rows than expected. Expected %d but got %d", rows.length, i), i == rows.length);
+        Assert.assertTrue(String.format("Got %s rows than expected. Expected %d but got %d", rows.length>i ? "less" : "more", rows.length, i), i == rows.length);
     }
 
     protected void assertRowCount(UntypedResultSet result, int numExpectedRows)
@@ -509,6 +776,24 @@
         return ret.toArray(a);
     }
 
+    protected void assertColumnNames(UntypedResultSet result, String... expectedColumnNames)
+    {
+        if (result == null)
+        {
+            Assert.fail("No rows returned by query.");
+            return;
+        }
+
+        List<ColumnSpecification> metadata = result.metadata();
+        Assert.assertEquals("Got less columns than expected.", expectedColumnNames.length, metadata.size());
+
+        for (int i = 0, m = metadata.size(); i < m; i++)
+        {
+            ColumnSpecification columnSpec = metadata.get(i);
+            Assert.assertEquals(expectedColumnNames[i], columnSpec.name.toString());
+        }
+    }
+
     protected void assertAllRows(Object[]... rows) throws Throwable
     {
         assertRows(execute("SELECT * FROM %s"), rows);
@@ -521,12 +806,10 @@
 
     protected void assertEmpty(UntypedResultSet result) throws Throwable
     {
-        if (result != null && result.size() != 0)
-            throw new InvalidRequestException(String.format("Expected empty result but got %d rows", result.size()));
+        if (result != null && !result.isEmpty())
+            throw new AssertionError(String.format("Expected empty result but got %d rows", result.size()));
     }
 
-
-
     protected void assertInvalid(String query, Object... values) throws Throwable
     {
         assertInvalidMessage(null, query, values);
@@ -574,6 +857,19 @@
                : replaceValues(query, values);
     }
 
+    protected void assertValidSyntax(String query) throws Throwable
+    {
+        try
+        {
+            QueryProcessor.parseStatement(query);
+        }
+        catch(SyntaxException e)
+        {
+            Assert.fail(String.format("Expected query syntax to be valid but was invalid. Query is: %s; Error is %s",
+                                      query, e.getMessage()));
+        }
+    }
+
     protected void assertInvalidSyntax(String query, Object... values) throws Throwable
     {
         assertInvalidSyntaxMessage(null, query, values);
@@ -584,10 +880,7 @@
         try
         {
             execute(query, values);
-            String q = USE_PREPARED_VALUES
-                     ? query + " (values: " + formatAllValues(values) + ")"
-                     : replaceValues(query, values);
-            Assert.fail("Query should have invalid syntax but no error was thrown. Query is: " + q);
+            Assert.fail("Query should have invalid syntax but no error was thrown. Query is: " + queryInfo(query, values));
         }
         catch (SyntaxException e)
         {
@@ -669,6 +962,11 @@
                 buffers[i] = null;
                 continue;
             }
+            else if (value == ByteBufferUtil.UNSET_BYTE_BUFFER)
+            {
+                buffers[i] = ByteBufferUtil.UNSET_BYTE_BUFFER;
+                continue;
+            }
 
             try
             {
@@ -794,10 +1092,11 @@
         AbstractType type = typeFor(value);
         String s = type.getString(type.decompose(value));
 
-        if (type instanceof UTF8Type)
+        if (type instanceof InetAddressType || type instanceof TimestampType)
+            return String.format("'%s'", s);
+        else if (type instanceof UTF8Type)
             return String.format("'%s'", s.replaceAll("'", "''"));
-
-        if (type instanceof BytesType)
+        else if (type instanceof BytesType)
             return "0x" + s;
 
         return s;
@@ -873,6 +1172,12 @@
         if (value instanceof ByteBuffer || value instanceof TupleValue || value == null)
             return BytesType.instance;
 
+        if (value instanceof Byte)
+            return ByteType.instance;
+
+        if (value instanceof Short)
+            return ShortType.instance;
+
         if (value instanceof Integer)
             return Int32Type.instance;
 
diff --git a/test/unit/org/apache/cassandra/cql3/ColumnConditionTest.java b/test/unit/org/apache/cassandra/cql3/ColumnConditionTest.java
index 3b1a826..c8b3a2f 100644
--- a/test/unit/org/apache/cassandra/cql3/ColumnConditionTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ColumnConditionTest.java
@@ -31,6 +31,7 @@
 import java.nio.ByteBuffer;
 import java.util.*;
 
+import static org.apache.cassandra.utils.ByteBufferUtil.UNSET_BYTE_BUFFER;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.fail;
@@ -82,6 +83,7 @@
         assertTrue(isSatisfiedBy(bound, null, null));
         assertFalse(isSatisfiedBy(bound, ONE, null));
         assertFalse(isSatisfiedBy(bound, null, ONE));
+        assertThrowsIRE(bound, UNSET_BYTE_BUFFER, ONE);
 
         // NEQ
         condition = ColumnCondition.condition(definition, new Constants.Value(ONE), Operator.NEQ);
@@ -95,6 +97,7 @@
         assertFalse(isSatisfiedBy(bound, null, null));
         assertTrue(isSatisfiedBy(bound, ONE, null));
         assertTrue(isSatisfiedBy(bound, null, ONE));
+        assertThrowsIRE(bound, UNSET_BYTE_BUFFER, ONE);
 
         // LT
         condition = ColumnCondition.condition(definition, new Constants.Value(ONE), Operator.LT);
@@ -107,6 +110,7 @@
         assertFalse(isSatisfiedBy(bound, ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER));
         assertThrowsIRE(bound, null, ONE);
         assertFalse(isSatisfiedBy(bound, ONE, null));
+        assertThrowsIRE(bound, UNSET_BYTE_BUFFER, ONE);
 
         // LTE
         condition = ColumnCondition.condition(definition, new Constants.Value(ONE), Operator.LTE);
@@ -119,6 +123,7 @@
         assertTrue(isSatisfiedBy(bound, ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER));
         assertThrowsIRE(bound, null, ONE);
         assertFalse(isSatisfiedBy(bound, ONE, null));
+        assertThrowsIRE(bound, UNSET_BYTE_BUFFER, ONE);
 
         // GT
         condition = ColumnCondition.condition(definition, new Constants.Value(ONE), Operator.GT);
@@ -131,6 +136,7 @@
         assertFalse(isSatisfiedBy(bound, ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER));
         assertThrowsIRE(bound, null, ONE);
         assertFalse(isSatisfiedBy(bound, ONE, null));
+        assertThrowsIRE(bound, UNSET_BYTE_BUFFER, ONE);
 
         // GT
         condition = ColumnCondition.condition(definition, new Constants.Value(ONE), Operator.GTE);
@@ -143,6 +149,7 @@
         assertTrue(isSatisfiedBy(bound, ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER));
         assertThrowsIRE(bound, null, ONE);
         assertFalse(isSatisfiedBy(bound, ONE, null));
+        assertThrowsIRE(bound, UNSET_BYTE_BUFFER, ONE);
     }
 
     private static List<ByteBuffer> list(ByteBuffer... values)
diff --git a/test/unit/org/apache/cassandra/cql3/CqlParserTest.java b/test/unit/org/apache/cassandra/cql3/CqlParserTest.java
index 84509e8..13c4685 100644
--- a/test/unit/org/apache/cassandra/cql3/CqlParserTest.java
+++ b/test/unit/org/apache/cassandra/cql3/CqlParserTest.java
@@ -36,7 +36,7 @@
         SyntaxErrorCounter firstCounter = new SyntaxErrorCounter();
         SyntaxErrorCounter secondCounter = new SyntaxErrorCounter();
 
-        CharStream stream = new ANTLRStringStream("SELECT * FORM test;");
+        CharStream stream = new ANTLRStringStream("SELECT * FORM users");
         CqlLexer lexer = new CqlLexer(stream);
 
         TokenStream tokenStream = new CommonTokenStream(lexer);
diff --git a/test/unit/org/apache/cassandra/cql3/DeleteTest.java b/test/unit/org/apache/cassandra/cql3/DeleteTest.java
index c8aa660..812d729 100644
--- a/test/unit/org/apache/cassandra/cql3/DeleteTest.java
+++ b/test/unit/org/apache/cassandra/cql3/DeleteTest.java
@@ -25,15 +25,12 @@
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.service.EmbeddedCassandraService;
 import org.junit.Assert;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import java.io.IOException;
-
 public class DeleteTest extends SchemaLoader
 {
     private static EmbeddedCassandraService cassandra;
diff --git a/test/unit/org/apache/cassandra/cql3/ErrorCollectorTest.java b/test/unit/org/apache/cassandra/cql3/ErrorCollectorTest.java
index fca93df..899aeec 100644
--- a/test/unit/org/apache/cassandra/cql3/ErrorCollectorTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ErrorCollectorTest.java
@@ -107,6 +107,30 @@
         assertEquals(expected, builder.toString());
     }
 
+    /**
+     * With ANTLR 3.5.2 it appears that some tokens can contains unexpected values: a line = 0 
+     * and a charPositionInLine = -1.
+     */
+    @Test
+    public void testAppendSnippetWithInvalidToken()
+    {
+        String query = "select * fom users";
+
+        ErrorCollector collector = new ErrorCollector(query);
+
+        StringBuilder builder = new StringBuilder();
+
+        Token from = new MockToken(1, 5, "select");
+        Token to = new MockToken(0, -1, "");
+        Token offending = new MockToken(0, -1, "");
+
+        collector.appendSnippet(builder, from, to, offending);
+
+        String expected = "";
+
+        assertEquals(expected, builder.toString());
+    }
+
     @Test
     public void testAppendSnippetWithInvalidToToken()
     {
diff --git a/test/unit/org/apache/cassandra/cql3/KeyCacheCqlTest.java b/test/unit/org/apache/cassandra/cql3/KeyCacheCqlTest.java
index f2bea8e..8471682 100644
--- a/test/unit/org/apache/cassandra/cql3/KeyCacheCqlTest.java
+++ b/test/unit/org/apache/cassandra/cql3/KeyCacheCqlTest.java
@@ -19,19 +19,19 @@
 package org.apache.cassandra.cql3;
 
 import java.util.ArrayList;
+import java.util.Iterator;
 import java.util.List;
 
 import org.junit.Assert;
 import org.junit.Test;
 
 import com.google.common.collect.ImmutableSet;
-import com.yammer.metrics.Metrics;
-import com.yammer.metrics.core.MetricName;
 
 import org.apache.cassandra.cache.KeyCacheKey;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.metrics.CacheMetrics;
+import org.apache.cassandra.metrics.CassandraMetricsRegistry;
 import org.apache.cassandra.service.CacheService;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.Pair;
@@ -79,8 +79,8 @@
             assertEquals(500, result.size());
         }
 
-        long hits = metrics.hits.count();
-        long requests = metrics.requests.count();
+        long hits = metrics.hits.getCount();
+        long requests = metrics.requests.getCount();
         assertEquals(4900, hits);
         assertEquals(5250, requests);
 
@@ -96,8 +96,8 @@
         }
 
         metrics = CacheService.instance.keyCache.getMetrics();
-        hits = metrics.hits.count();
-        requests = metrics.requests.count();
+        hits = metrics.hits.getCount();
+        requests = metrics.requests.getCount();
         assertEquals(10000, hits);
         assertEquals(10500, requests);
 
@@ -158,8 +158,8 @@
             assertEquals(500, result.size());
         }
 
-        long hits = metrics.hits.count();
-        long requests = metrics.requests.count();
+        long hits = metrics.hits.getCount();
+        long requests = metrics.requests.getCount();
         assertEquals(4900, hits);
         assertEquals(5250, requests);
 
@@ -175,8 +175,8 @@
         }
 
         metrics = CacheService.instance.keyCache.getMetrics();
-        hits = metrics.hits.count();
-        requests = metrics.requests.count();
+        hits = metrics.hits.getCount();
+        requests = metrics.requests.getCount();
         assertEquals(10000, hits);
         assertEquals(10500, requests);
 
@@ -191,8 +191,10 @@
         // then load saved
         CacheService.instance.keyCache.loadSaved();
 
-        for (KeyCacheKey key : CacheService.instance.keyCache.getKeySet())
+        Iterator<KeyCacheKey> iter = CacheService.instance.keyCache.keyIterator();
+        while(iter.hasNext())
         {
+            KeyCacheKey key = iter.next();
             Assert.assertFalse(key.ksAndCFName.left.equals("KEYSPACE"));
             Assert.assertFalse(key.ksAndCFName.right.startsWith(table));
         }
@@ -250,15 +252,16 @@
 
     private static void clearCache()
     {
-        for (MetricName name : ImmutableSet.copyOf(Metrics.defaultRegistry().allMetrics().keySet()))
+        for (String name : ImmutableSet.copyOf(CassandraMetricsRegistry.Metrics.getMetrics().keySet()))
         {
-            Metrics.defaultRegistry().removeMetric(name);
+            CassandraMetricsRegistry.Metrics.remove(name);
         }
+
         CacheService.instance.keyCache.clear();
         CacheMetrics metrics = CacheService.instance.keyCache.getMetrics();
-        Assert.assertEquals(0, metrics.entries.value().intValue());
-        Assert.assertEquals(0L, metrics.hits.count());
-        Assert.assertEquals(0L, metrics.requests.count());
-        Assert.assertEquals(0L, metrics.size.value().longValue());
+        Assert.assertEquals(0, metrics.entries.getValue().intValue());
+        Assert.assertEquals(0L, metrics.hits.getCount());
+        Assert.assertEquals(0L, metrics.requests.getCount());
+        Assert.assertEquals(0L, metrics.size.getValue().longValue());
     }
 }
diff --git a/test/unit/org/apache/cassandra/cql3/NonNativeTimestampTest.java b/test/unit/org/apache/cassandra/cql3/NonNativeTimestampTest.java
index 8e3fbed..80c5e3b3 100644
--- a/test/unit/org/apache/cassandra/cql3/NonNativeTimestampTest.java
+++ b/test/unit/org/apache/cassandra/cql3/NonNativeTimestampTest.java
@@ -50,7 +50,7 @@
     }
 
     @Test
-    public void setServerTimestampForNonCqlNativeStatements() throws RequestValidationException, RequestExecutionException, CharacterCodingException, UnsupportedEncodingException
+    public void setServerTimestampForNonCqlNativeStatements() throws RequestValidationException, RequestExecutionException
     {
         String createKsCQL = "CREATE KEYSPACE non_native_ts_test" +
                              " WITH REPLICATION = { 'class' : 'SimpleStrategy', 'replication_factor' : 1 };";
diff --git a/test/unit/org/apache/cassandra/cql3/OutOfSpaceBase.java b/test/unit/org/apache/cassandra/cql3/OutOfSpaceBase.java
index c0023dc..826d6e6 100644
--- a/test/unit/org/apache/cassandra/cql3/OutOfSpaceBase.java
+++ b/test/unit/org/apache/cassandra/cql3/OutOfSpaceBase.java
@@ -23,6 +23,7 @@
 import java.util.UUID;
 import java.util.concurrent.ExecutionException;
 
+import org.junit.After;
 import org.junit.Assert;
 
 import org.apache.cassandra.db.BlacklistedDirectories;
@@ -84,4 +85,11 @@
                 return;
         fail("Expected commit log to remain dirty for the affected table.");
     }
+
+
+    @After
+    public void afterTest() throws Throwable
+    {
+        // Override CQLTester's afterTest method; clean-up will fail due to flush failing.
+    }
 }
diff --git a/test/unit/org/apache/cassandra/cql3/RangeTombstoneMergeTest.java b/test/unit/org/apache/cassandra/cql3/RangeTombstoneMergeTest.java
index 0460a16..71634e9 100644
--- a/test/unit/org/apache/cassandra/cql3/RangeTombstoneMergeTest.java
+++ b/test/unit/org/apache/cassandra/cql3/RangeTombstoneMergeTest.java
@@ -30,8 +30,8 @@
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
 import org.apache.cassandra.db.composites.*;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.ISSTableScanner;
-import org.apache.cassandra.io.sstable.SSTableReader;
 
 public class RangeTombstoneMergeTest extends CQLTester
 {
diff --git a/test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java b/test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java
index 6f0e439..7b72ef8 100644
--- a/test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ThriftCompatibilityTest.java
@@ -17,7 +17,11 @@
  */
 package org.apache.cassandra.cql3;
 
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
@@ -27,18 +31,20 @@
 
 public class ThriftCompatibilityTest extends SchemaLoader
 {
-    private static UntypedResultSet execute(String query) throws Throwable
+    @BeforeClass
+    public static void defineSchema() throws Exception
     {
-        try
-        {
-            return QueryProcessor.executeInternal(String.format(query));
-        }
-        catch (RuntimeException exc)
-        {
-            if (exc.getCause() != null)
-                throw exc.getCause();
-            throw exc;
-        }
+        // The before class annotation of SchemaLoader will prepare the service so no need to do it here
+        SchemaLoader.createKeyspace("thriftcompat",
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    jdbcSparseCFMD("thriftcompat", "JdbcInteger", Int32Type.instance)
+                                            .addColumnDefinition(integerColumn("thriftcompat", "JdbcInteger")));
+    }
+
+    private static UntypedResultSet execute(String query)
+    {
+        return QueryProcessor.executeInternal(String.format(query));
     }
 
     /** Test For CASSANDRA-8178 */
@@ -46,10 +52,10 @@
     public void testNonTextComparator() throws Throwable
     {
         // the comparator is IntegerType, and there is a column named 42 with a UTF8Type validation type
-        execute("INSERT INTO \"Keyspace1\".\"JdbcInteger\" (key, \"42\") VALUES (0x00000001, 'abc')");
-        execute("UPDATE \"Keyspace1\".\"JdbcInteger\" SET \"42\" = 'abc' WHERE key = 0x00000001");
-        execute("DELETE \"42\" FROM \"Keyspace1\".\"JdbcInteger\" WHERE key = 0x00000000");
-        UntypedResultSet results = execute("SELECT key, \"42\" FROM \"Keyspace1\".\"JdbcInteger\"");
+        execute("INSERT INTO \"thriftcompat\".\"JdbcInteger\" (key, \"42\") VALUES (0x00000001, 'abc')");
+        execute("UPDATE \"thriftcompat\".\"JdbcInteger\" SET \"42\" = 'abc' WHERE key = 0x00000001");
+        execute("DELETE \"42\" FROM \"thriftcompat\".\"JdbcInteger\" WHERE key = 0x00000000");
+        UntypedResultSet results = execute("SELECT key, \"42\" FROM \"thriftcompat\".\"JdbcInteger\"");
         assertEquals(1, results.size());
         UntypedResultSet.Row row = results.iterator().next();
         assertEquals(ByteBufferUtil.bytes(1), row.getBytes("key"));
diff --git a/test/unit/org/apache/cassandra/cql3/UDHelperTest.java b/test/unit/org/apache/cassandra/cql3/UDHelperTest.java
new file mode 100644
index 0000000..4a5e78e
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/UDHelperTest.java
@@ -0,0 +1,146 @@
+/*
+ * 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.cql3;
+
+import java.nio.ByteBuffer;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.functions.UDHelper;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.AsciiType;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.db.marshal.ByteType;
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.CounterColumnType;
+import org.apache.cassandra.db.marshal.DateType;
+import org.apache.cassandra.db.marshal.DecimalType;
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.db.marshal.FloatType;
+import org.apache.cassandra.db.marshal.InetAddressType;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.db.marshal.IntegerType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.ReversedType;
+import org.apache.cassandra.db.marshal.ShortType;
+import org.apache.cassandra.db.marshal.SimpleDateType;
+import org.apache.cassandra.db.marshal.TimeType;
+import org.apache.cassandra.db.marshal.TimeUUIDType;
+import org.apache.cassandra.db.marshal.TimestampType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.db.marshal.UUIDType;
+import org.apache.cassandra.serializers.MarshalException;
+import org.apache.cassandra.serializers.TypeSerializer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class UDHelperTest
+{
+    static class UFTestCustomType extends AbstractType<String>
+    {
+
+        public ByteBuffer fromString(String source) throws MarshalException
+        {
+            return ByteBuffer.wrap(source.getBytes());
+        }
+
+        public Term fromJSONObject(Object parsed) throws MarshalException
+        {
+            throw new UnsupportedOperationException();
+        }
+
+        public TypeSerializer<String> getSerializer()
+        {
+            return UTF8Type.instance.getSerializer();
+        }
+
+        public int compare(ByteBuffer o1, ByteBuffer o2)
+        {
+            return o1.compareTo(o2);
+        }
+    }
+
+    @Test
+    public void testEmptyVariableLengthTypes()
+    {
+        AbstractType<?>[] types = new AbstractType<?>[]{
+                                                       AsciiType.instance,
+                                                       BytesType.instance,
+                                                       UTF8Type.instance,
+                                                       new UFTestCustomType()
+        };
+
+        for (AbstractType<?> type : types)
+        {
+            Assert.assertFalse("type " + type.getClass().getName(),
+                               UDHelper.isNullOrEmpty(type, ByteBufferUtil.EMPTY_BYTE_BUFFER));
+        }
+    }
+
+    @Test
+    public void testNonEmptyPrimitiveTypes()
+    {
+        AbstractType<?>[] types = new AbstractType<?>[]{
+                                                       TimeType.instance,
+                                                       SimpleDateType.instance,
+                                                       ByteType.instance,
+                                                       ShortType.instance
+        };
+
+        for (AbstractType<?> type : types)
+        {
+            try
+            {
+                type.getSerializer().validate(ByteBufferUtil.EMPTY_BYTE_BUFFER);
+                Assert.fail(type.getClass().getSimpleName());
+            }
+            catch (MarshalException e)
+            {
+                //
+            }
+        }
+    }
+
+    @Test
+    public void testEmptiableTypes()
+    {
+        AbstractType<?>[] types = new AbstractType<?>[]{
+                                                       BooleanType.instance,
+                                                       CounterColumnType.instance,
+                                                       DateType.instance,
+                                                       DecimalType.instance,
+                                                       DoubleType.instance,
+                                                       FloatType.instance,
+                                                       InetAddressType.instance,
+                                                       Int32Type.instance,
+                                                       IntegerType.instance,
+                                                       LongType.instance,
+                                                       TimestampType.instance,
+                                                       TimeUUIDType.instance,
+                                                       UUIDType.instance
+        };
+
+        for (AbstractType<?> type : types)
+        {
+            Assert.assertTrue(type.getClass().getSimpleName(), UDHelper.isNullOrEmpty(type, ByteBufferUtil.EMPTY_BYTE_BUFFER));
+            Assert.assertTrue("reversed " + type.getClass().getSimpleName(),
+                              UDHelper.isNullOrEmpty(ReversedType.getInstance(type), ByteBufferUtil.EMPTY_BYTE_BUFFER));
+        }
+    }
+}
diff --git a/test/unit/org/apache/cassandra/cql3/functions/TimeFctsTest.java b/test/unit/org/apache/cassandra/cql3/functions/TimeFctsTest.java
new file mode 100644
index 0000000..f6fca20
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/functions/TimeFctsTest.java
@@ -0,0 +1,206 @@
+/*
+ * 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.cql3.functions;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.List;
+
+import org.junit.Test;
+
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.SimpleDateType;
+import org.apache.cassandra.db.marshal.TimeUUIDType;
+import org.apache.cassandra.db.marshal.TimestampType;
+import org.apache.cassandra.transport.Server;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.UUIDGen;
+import org.joda.time.DateTime;
+import org.joda.time.DateTimeZone;
+import org.joda.time.format.DateTimeFormat;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+public class TimeFctsTest
+{
+    @Test
+    public void testMinTimeUuid()
+    {
+        DateTime dateTime = DateTimeFormat.forPattern("yyyy-MM-dd hh:mm:ss")
+                .withZone(DateTimeZone.UTC)
+                .parseDateTime("2015-05-21 11:03:02");
+
+        long timeInMillis = dateTime.getMillis();
+        ByteBuffer input = TimestampType.instance.fromString("2015-05-21 11:03:02+00");
+        ByteBuffer output = executeFunction(TimeFcts.minTimeuuidFct, input);
+        assertEquals(UUIDGen.minTimeUUID(timeInMillis), TimeUUIDType.instance.compose(output));
+    }
+
+    @Test
+    public void testMaxTimeUuid()
+    {
+        DateTime dateTime = DateTimeFormat.forPattern("yyyy-MM-dd hh:mm:ss")
+                .withZone(DateTimeZone.UTC)
+                .parseDateTime("2015-05-21 11:03:02");
+
+        long timeInMillis = dateTime.getMillis();
+        ByteBuffer input = TimestampType.instance.fromString("2015-05-21 11:03:02+00");
+        ByteBuffer output = executeFunction(TimeFcts.maxTimeuuidFct, input);
+        assertEquals(UUIDGen.maxTimeUUID(timeInMillis), TimeUUIDType.instance.compose(output));
+    }
+
+    @Test
+    public void testDateOf()
+    {
+        DateTime dateTime = DateTimeFormat.forPattern("yyyy-MM-dd hh:mm:ss")
+                .withZone(DateTimeZone.UTC)
+                .parseDateTime("2015-05-21 11:03:02");
+
+        long timeInMillis = dateTime.getMillis();
+        ByteBuffer input = ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes(timeInMillis, 0));
+        ByteBuffer output = executeFunction(TimeFcts.dateOfFct, input);
+        assertEquals(dateTime.toDate(), TimestampType.instance.compose(output));
+    }
+
+    @Test
+    public void testTimeUuidToTimestamp()
+    {
+        DateTime dateTime = DateTimeFormat.forPattern("yyyy-MM-dd hh:mm:ss")
+                .withZone(DateTimeZone.UTC)
+                .parseDateTime("2015-05-21 11:03:02");
+
+        long timeInMillis = dateTime.getMillis();
+        ByteBuffer input = ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes(timeInMillis, 0));
+        ByteBuffer output = executeFunction(TimeFcts.timeUuidToTimestamp, input);
+        assertEquals(dateTime.toDate(), TimestampType.instance.compose(output));
+    }
+
+    @Test
+    public void testUnixTimestampOfFct()
+    {
+        DateTime dateTime = DateTimeFormat.forPattern("yyyy-MM-dd hh:mm:ss")
+                .withZone(DateTimeZone.UTC)
+                .parseDateTime("2015-05-21 11:03:02");
+
+        long timeInMillis = dateTime.getMillis();
+        ByteBuffer input = ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes(timeInMillis, 0));
+        ByteBuffer output = executeFunction(TimeFcts.unixTimestampOfFct, input);
+        assertEquals(timeInMillis, LongType.instance.compose(output).longValue());
+    }
+
+    @Test
+    public void testTimeUuidToUnixTimestamp()
+    {
+        DateTime dateTime = DateTimeFormat.forPattern("yyyy-MM-dd hh:mm:ss")
+                .withZone(DateTimeZone.UTC)
+                .parseDateTime("2015-05-21 11:03:02");
+
+        long timeInMillis = dateTime.getMillis();
+        ByteBuffer input = ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes(timeInMillis, 0));
+        ByteBuffer output = executeFunction(TimeFcts.timeUuidToUnixTimestamp, input);
+        assertEquals(timeInMillis, LongType.instance.compose(output).longValue());
+    }
+
+    @Test
+    public void testTimeUuidToDate()
+    {
+        DateTime dateTime = DateTimeFormat.forPattern("yyyy-MM-dd hh:mm:ss")
+                .withZone(DateTimeZone.UTC)
+                .parseDateTime("2015-05-21 11:03:02");
+
+        long timeInMillis = dateTime.getMillis();
+        ByteBuffer input = ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes(timeInMillis, 0));
+        ByteBuffer output = executeFunction(TimeFcts.timeUuidtoDate, input);
+
+        long expectedTime = DateTimeFormat.forPattern("yyyy-MM-dd")
+                                          .withZone(DateTimeZone.UTC)
+                                          .parseDateTime("2015-05-21")
+                                          .getMillis();
+
+        assertEquals(expectedTime, SimpleDateType.instance.toTimeInMillis(output));
+    }
+
+    @Test
+    public void testDateToTimestamp()
+    {
+        DateTime dateTime = DateTimeFormat.forPattern("yyyy-MM-dd")
+                                          .withZone(DateTimeZone.UTC)
+                                          .parseDateTime("2015-05-21");
+
+        ByteBuffer input = SimpleDateType.instance.fromString("2015-05-21");
+        ByteBuffer output = executeFunction(TimeFcts.dateToTimestamp, input);
+        assertEquals(dateTime.toDate(), TimestampType.instance.compose(output));
+    }
+
+    @Test
+    public void testDateToUnixTimestamp()
+    {
+        DateTime dateTime = DateTimeFormat.forPattern("yyyy-MM-dd")
+                                          .withZone(DateTimeZone.UTC)
+                                          .parseDateTime("2015-05-21");
+
+        ByteBuffer input = SimpleDateType.instance.fromString("2015-05-21");
+        ByteBuffer output = executeFunction(TimeFcts.dateToUnixTimestamp, input);
+        assertEquals(dateTime.getMillis(), LongType.instance.compose(output).longValue());
+    }
+
+    @Test
+    public void testTimestampToDate()
+    {
+        DateTime dateTime = DateTimeFormat.forPattern("yyyy-MM-dd")
+                                          .withZone(DateTimeZone.UTC)
+                                          .parseDateTime("2015-05-21");
+
+        ByteBuffer input = TimestampType.instance.fromString("2015-05-21 11:03:02+00");
+        ByteBuffer output = executeFunction(TimeFcts.timestampToDate, input);
+        assertEquals(dateTime.getMillis(), SimpleDateType.instance.toTimeInMillis(output));
+    }
+
+    @Test
+    public void testTimestampToDateWithEmptyInput()
+    {
+        ByteBuffer output = executeFunction(TimeFcts.timestampToDate, ByteBufferUtil.EMPTY_BYTE_BUFFER);
+        assertNull(output);
+    }
+
+    @Test
+    public void testTimestampToUnixTimestamp()
+    {
+        DateTime dateTime = DateTimeFormat.forPattern("yyyy-MM-dd hh:mm:ss")
+                                          .withZone(DateTimeZone.UTC)
+                                          .parseDateTime("2015-05-21 11:03:02");
+
+        ByteBuffer input = TimestampType.instance.decompose(dateTime.toDate());
+        ByteBuffer output = executeFunction(TimeFcts.timestampToUnixTimestamp, input);
+        assertEquals(dateTime.getMillis(), LongType.instance.compose(output).longValue());
+    }
+
+    @Test
+    public void testTimestampToUnixTimestampWithEmptyInput()
+    {
+        ByteBuffer output = executeFunction(TimeFcts.timestampToUnixTimestamp, ByteBufferUtil.EMPTY_BYTE_BUFFER);
+        assertNull(output);
+    }
+
+    private static ByteBuffer executeFunction(Function function, ByteBuffer input)
+    {
+        List<ByteBuffer> params = Arrays.asList(input);
+        return ((ScalarFunction) function).execute(Server.CURRENT_VERSION, params);
+    }
+}
diff --git a/test/unit/org/apache/cassandra/cql3/restrictions/PrimaryKeyRestrictionSetTest.java b/test/unit/org/apache/cassandra/cql3/restrictions/PrimaryKeyRestrictionSetTest.java
new file mode 100644
index 0000000..bd5395a
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/restrictions/PrimaryKeyRestrictionSetTest.java
@@ -0,0 +1,1957 @@
+/*
+ * 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.cql3.restrictions;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.junit.Test;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.cql3.Term.MultiItemTerminal;
+import org.apache.cassandra.cql3.statements.Bound;
+import org.apache.cassandra.db.ColumnFamilyType;
+import org.apache.cassandra.db.composites.Composite;
+import org.apache.cassandra.db.composites.Composite.EOC;
+import org.apache.cassandra.db.composites.Composites;
+import org.apache.cassandra.db.composites.CompoundSparseCellNameType;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.db.marshal.ReversedType;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static java.util.Arrays.asList;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class PrimaryKeyRestrictionSetTest
+{
+    @Test
+    public void testBoundsAsCompositesWithNoRestrictions()
+    {
+        CFMetaData cfMetaData = newCFMetaData(Sort.ASC);
+
+        PrimaryKeyRestrictions restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+
+        List<Composite> bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertEmptyComposite(bounds.get(0));
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertEmptyComposite(bounds.get(0));
+    }
+
+    /**
+     * Test 'clustering_0 = 1' with only one clustering column
+     */
+    @Test
+    public void testBoundsAsCompositesWithOneEqRestrictionsAndOneClusteringColumn()
+    {
+        CFMetaData cfMetaData = newCFMetaData(Sort.ASC);
+
+        ByteBuffer clustering_0 = ByteBufferUtil.bytes(1);
+        Restriction eq = newSingleEq(cfMetaData, 0, clustering_0);
+
+        PrimaryKeyRestrictions restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(eq);
+
+        List<Composite> bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), clustering_0, EOC.START);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), clustering_0, EOC.END);
+    }
+
+    /**
+     * Test 'clustering_1 = 1' with 2 clustering columns
+     */
+    @Test
+    public void testBoundsAsCompositesWithOneEqRestrictionsAndTwoClusteringColumns()
+    {
+        CFMetaData cfMetaData = newCFMetaData(Sort.ASC, Sort.ASC);
+
+        ByteBuffer clustering_0 = ByteBufferUtil.bytes(1);
+        Restriction eq = newSingleEq(cfMetaData, 0, clustering_0);
+
+        PrimaryKeyRestrictions restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(eq);
+
+        List<Composite> bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), clustering_0, EOC.START);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), clustering_0, EOC.END);
+    }
+
+    /**
+     * Test 'clustering_0 IN (1, 2, 3)' with only one clustering column
+     */
+    @Test
+    public void testBoundsAsCompositesWithOneInRestrictionsAndOneClusteringColumn()
+    {
+        ByteBuffer value1 = ByteBufferUtil.bytes(1);
+        ByteBuffer value2 = ByteBufferUtil.bytes(2);
+        ByteBuffer value3 = ByteBufferUtil.bytes(3);
+
+        CFMetaData cfMetaData = newCFMetaData(Sort.ASC, Sort.ASC);
+
+        Restriction in = newSingleIN(cfMetaData, 0, value1, value2, value3);
+
+        PrimaryKeyRestrictions restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(in);
+
+        List<Composite> bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(3, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.START);
+        assertComposite(bounds.get(1), value2, EOC.START);
+        assertComposite(bounds.get(2), value3, EOC.START);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(3, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.END);
+        assertComposite(bounds.get(1), value2, EOC.END);
+        assertComposite(bounds.get(2), value3, EOC.END);
+    }
+
+    /**
+     * Test slice restriction (e.g 'clustering_0 > 1') with only one clustering column
+     */
+    @Test
+    public void testBoundsAsCompositesWithSliceRestrictionsAndOneClusteringColumn()
+    {
+        CFMetaData cfMetaData = newCFMetaData(Sort.ASC, Sort.ASC);
+
+        ByteBuffer value1 = ByteBufferUtil.bytes(1);
+        ByteBuffer value2 = ByteBufferUtil.bytes(2);
+
+        Restriction slice = newSingleSlice(cfMetaData, 0, Bound.START, false, value1);
+        PrimaryKeyRestrictions restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        List<Composite> bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.END);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertEmptyComposite(bounds.get(0));
+
+        slice = newSingleSlice(cfMetaData, 0, Bound.START, true, value1);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.NONE);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertEmptyComposite(bounds.get(0));
+
+        slice = newSingleSlice(cfMetaData, 0, Bound.END, true, value1);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertEmptyComposite(bounds.get(0));
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.END);
+
+        slice = newSingleSlice(cfMetaData, 0, Bound.END, false, value1);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertEmptyComposite(bounds.get(0));
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.START);
+
+        slice = newSingleSlice(cfMetaData, 0, Bound.START, false, value1);
+        Restriction slice2 = newSingleSlice(cfMetaData, 0, Bound.END, false, value2);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.END);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value2, EOC.START);
+
+        slice = newSingleSlice(cfMetaData, 0, Bound.START, true, value1);
+        slice2 = newSingleSlice(cfMetaData, 0, Bound.END, true, value2);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.NONE);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value2, EOC.END);
+    }
+
+    /**
+     * Test slice restriction (e.g 'clustering_0 > 1') with only one descending clustering column
+     */
+    @Test
+    public void testBoundsAsCompositesWithSliceRestrictionsAndOneDescendingClusteringColumn()
+    {
+        CFMetaData cfMetaData = newCFMetaData(Sort.DESC, Sort.DESC);
+
+        ByteBuffer value1 = ByteBufferUtil.bytes(1);
+        ByteBuffer value2 = ByteBufferUtil.bytes(2);
+
+        Restriction slice = newSingleSlice(cfMetaData, 0, Bound.START, false, value1);
+        PrimaryKeyRestrictions restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        List<Composite> bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertEmptyComposite(bounds.get(0));
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.START);
+
+        slice = newSingleSlice(cfMetaData, 0, Bound.START, true, value1);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertEmptyComposite(bounds.get(0));
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.END);
+
+        slice = newSingleSlice(cfMetaData, 0, Bound.END, true, value1);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.NONE);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertEmptyComposite(bounds.get(0));
+
+        slice = newSingleSlice(cfMetaData, 0, Bound.END, false, value1);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.END);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertEmptyComposite(bounds.get(0));
+
+        slice = newSingleSlice(cfMetaData, 0, Bound.START, false, value1);
+        Restriction slice2 = newSingleSlice(cfMetaData, 0, Bound.END, false, value2);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value2, EOC.END);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.START);
+
+        slice = newSingleSlice(cfMetaData, 0, Bound.START, true, value1);
+        slice2 = newSingleSlice(cfMetaData, 0, Bound.END, true, value2);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value2, EOC.NONE);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.END);
+    }
+
+    /**
+     * Test 'clustering_0 = 1 AND clustering_1 IN (1, 2, 3)'
+     */
+    @Test
+    public void testBoundsAsCompositesWithEqAndInRestrictions()
+    {
+        CFMetaData cfMetaData = newCFMetaData(Sort.ASC, Sort.ASC);
+
+        ByteBuffer value1 = ByteBufferUtil.bytes(1);
+        ByteBuffer value2 = ByteBufferUtil.bytes(2);
+        ByteBuffer value3 = ByteBufferUtil.bytes(3);
+        Restriction eq = newSingleEq(cfMetaData, 0, value1);
+        Restriction in = newSingleIN(cfMetaData, 1, value1, value2, value3);
+        PrimaryKeyRestrictions restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(eq).mergeWith(in);
+
+        List<Composite> bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(3, bounds.size());
+        assertComposite(bounds.get(0), value1, value1, EOC.START);
+        assertComposite(bounds.get(1), value1, value2, EOC.START);
+        assertComposite(bounds.get(2), value1, value3, EOC.START);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(3, bounds.size());
+        assertComposite(bounds.get(0), value1, value1, EOC.END);
+        assertComposite(bounds.get(1), value1, value2, EOC.END);
+        assertComposite(bounds.get(2), value1, value3, EOC.END);
+    }
+
+    /**
+     * Test equal and slice restrictions (e.g 'clustering_0 = 0 clustering_1 > 1')
+     */
+    @Test
+    public void testBoundsAsCompositesWithEqAndSliceRestrictions()
+    {
+        CFMetaData cfMetaData = newCFMetaData(Sort.ASC, Sort.ASC);
+
+        ByteBuffer value1 = ByteBufferUtil.bytes(1);
+        ByteBuffer value2 = ByteBufferUtil.bytes(2);
+        ByteBuffer value3 = ByteBufferUtil.bytes(3);
+
+        Restriction eq = newSingleEq(cfMetaData, 0, value3);
+
+        Restriction slice = newSingleSlice(cfMetaData, 1, Bound.START, false, value1);
+        PrimaryKeyRestrictions restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(eq).mergeWith(slice);
+
+        List<Composite> bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value3, value1, EOC.END);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value3, EOC.END);
+
+        slice = newSingleSlice(cfMetaData, 1, Bound.START, true, value1);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(eq).mergeWith(slice);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value3, value1, EOC.NONE);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value3, EOC.END);
+
+        slice = newSingleSlice(cfMetaData, 1, Bound.END, true, value1);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(eq).mergeWith(slice);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value3, EOC.START);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value3, value1, EOC.END);
+
+        slice = newSingleSlice(cfMetaData, 1, Bound.END, false, value1);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(eq).mergeWith(slice);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value3, EOC.START);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value3, value1, EOC.START);
+
+        slice = newSingleSlice(cfMetaData, 1, Bound.START, false, value1);
+        Restriction slice2 = newSingleSlice(cfMetaData, 1, Bound.END, false, value2);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(eq).mergeWith(slice).mergeWith(slice2);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value3, value1, EOC.END);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value3, value2, EOC.START);
+
+        slice = newSingleSlice(cfMetaData, 1, Bound.START, true, value1);
+        slice2 = newSingleSlice(cfMetaData, 1, Bound.END, true, value2);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(eq).mergeWith(slice).mergeWith(slice2);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value3, value1, EOC.NONE);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value3, value2, EOC.END);
+    }
+
+    /**
+     * Test '(clustering_0, clustering_1) = (1, 2)' with two clustering column
+     */
+    @Test
+    public void testBoundsAsCompositesWithMultiEqRestrictions()
+    {
+        CFMetaData cfMetaData = newCFMetaData(Sort.ASC, Sort.ASC);
+
+        ByteBuffer value1 = ByteBufferUtil.bytes(1);
+        ByteBuffer value2 = ByteBufferUtil.bytes(2);
+        Restriction eq = newMultiEq(cfMetaData, 0, value1, value2);
+        PrimaryKeyRestrictions restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(eq);
+
+        List<Composite> bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, EOC.START);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, EOC.END);
+    }
+
+    /**
+     * Test '(clustering_0, clustering_1) IN ((1, 2), (2, 3))' with two clustering column
+     */
+    @Test
+    public void testBoundsAsCompositesWithMultiInRestrictions()
+    {
+        CFMetaData cfMetaData = newCFMetaData(Sort.ASC, Sort.ASC);
+
+        ByteBuffer value1 = ByteBufferUtil.bytes(1);
+        ByteBuffer value2 = ByteBufferUtil.bytes(2);
+        ByteBuffer value3 = ByteBufferUtil.bytes(3);
+        Restriction in = newMultiIN(cfMetaData, 0, asList(value1, value2), asList(value2, value3));
+        PrimaryKeyRestrictions restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(in);
+
+        List<Composite> bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, EOC.START);
+        assertComposite(bounds.get(1), value2, value3, EOC.START);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, EOC.END);
+        assertComposite(bounds.get(1), value2, value3, EOC.END);
+    }
+
+    /**
+     * Test multi-column slice restrictions (e.g '(clustering_0) > (1)') with only one clustering column
+     */
+    @Test
+    public void testBoundsAsCompositesWithMultiSliceRestrictionsWithOneClusteringColumn()
+    {
+        CFMetaData cfMetaData = newCFMetaData(Sort.ASC);
+
+        ByteBuffer value1 = ByteBufferUtil.bytes(1);
+        ByteBuffer value2 = ByteBufferUtil.bytes(2);
+
+        Restriction slice = newMultiSlice(cfMetaData, 0, Bound.START, false, value1);
+        PrimaryKeyRestrictions restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        List<Composite> bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.END);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertEmptyComposite(bounds.get(0));
+
+        slice = newMultiSlice(cfMetaData, 0, Bound.START, true, value1);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.NONE);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertEmptyComposite(bounds.get(0));
+
+        slice = newMultiSlice(cfMetaData, 0, Bound.END, true, value1);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertEmptyComposite(bounds.get(0));
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.END);
+
+        slice = newMultiSlice(cfMetaData, 0, Bound.END, false, value1);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertEmptyComposite(bounds.get(0));
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.START);
+
+        slice = newMultiSlice(cfMetaData, 0, Bound.START, false, value1);
+        Restriction slice2 = newMultiSlice(cfMetaData, 0, Bound.END, false, value2);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.END);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value2, EOC.START);
+
+        slice = newMultiSlice(cfMetaData, 0, Bound.START, true, value1);
+        slice2 = newMultiSlice(cfMetaData, 0, Bound.END, true, value2);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.NONE);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value2, EOC.END);
+    }
+
+    /**
+     * Test multi-column slice restrictions (e.g '(clustering_0) > (1)') with only one clustering column in reverse
+     * order
+     */
+    @Test
+    public void testBoundsAsCompositesWithMultiSliceRestrictionsWithOneDescendingClusteringColumn()
+    {
+        CFMetaData cfMetaData = newCFMetaData(Sort.DESC);
+
+        ByteBuffer value1 = ByteBufferUtil.bytes(1);
+        ByteBuffer value2 = ByteBufferUtil.bytes(2);
+
+        Restriction slice = newMultiSlice(cfMetaData, 0, Bound.START, false, value1);
+        PrimaryKeyRestrictions restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        List<Composite> bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertEmptyComposite(bounds.get(0));
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.START);
+
+        slice = newMultiSlice(cfMetaData, 0, Bound.START, true, value1);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertEmptyComposite(bounds.get(0));
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.END);
+
+        slice = newMultiSlice(cfMetaData, 0, Bound.END, true, value1);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.NONE);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertEmptyComposite(bounds.get(0));
+
+        slice = newMultiSlice(cfMetaData, 0, Bound.END, false, value1);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.END);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertEmptyComposite(bounds.get(0));
+
+        slice = newMultiSlice(cfMetaData, 0, Bound.START, false, value1);
+        Restriction slice2 = newMultiSlice(cfMetaData, 0, Bound.END, false, value2);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value2, EOC.END);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.START);
+
+        slice = newMultiSlice(cfMetaData, 0, Bound.START, true, value1);
+        slice2 = newMultiSlice(cfMetaData, 0, Bound.END, true, value2);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value2, EOC.NONE);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.END);
+    }
+
+    /**
+     * Test multi-column slice restrictions (e.g '(clustering_0, clustering_1) > (1, 2)')
+     */
+    @Test
+    public void testBoundsAsCompositesWithMultiSliceRestrictionsWithTwoClusteringColumn()
+    {
+        CFMetaData cfMetaData = newCFMetaData(Sort.ASC, Sort.ASC);
+
+        ByteBuffer value1 = ByteBufferUtil.bytes(1);
+        ByteBuffer value2 = ByteBufferUtil.bytes(2);
+
+        // (clustering_0, clustering1) > (1, 2)
+        Restriction slice = newMultiSlice(cfMetaData, 0, Bound.START, false, value1, value2);
+        PrimaryKeyRestrictions restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        List<Composite> bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, EOC.END);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertEmptyComposite(bounds.get(0));
+
+        // (clustering_0, clustering1) >= (1, 2)
+        slice = newMultiSlice(cfMetaData, 0, Bound.START, true, value1, value2);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, EOC.NONE);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertEmptyComposite(bounds.get(0));
+
+        // (clustering_0, clustering1) <= (1, 2)
+        slice = newMultiSlice(cfMetaData, 0, Bound.END, true, value1, value2);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertEmptyComposite(bounds.get(0));
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, EOC.END);
+
+        // (clustering_0, clustering1) < (1, 2)
+        slice = newMultiSlice(cfMetaData, 0, Bound.END, false, value1, value2);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertEmptyComposite(bounds.get(0));
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, EOC.START);
+
+        // (clustering_0, clustering1) > (1, 2) AND (clustering_0) < (2)
+        slice = newMultiSlice(cfMetaData, 0, Bound.START, false, value1, value2);
+        Restriction slice2 = newMultiSlice(cfMetaData, 0, Bound.END, false, value2);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, EOC.END);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value2, EOC.START);
+
+        // (clustering_0, clustering1) >= (1, 2) AND (clustering_0, clustering1) <= (2, 1)
+        slice = newMultiSlice(cfMetaData, 0, Bound.START, true, value1, value2);
+        slice2 = newMultiSlice(cfMetaData, 0, Bound.END, true, value2, value1);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, EOC.NONE);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value2, value1, EOC.END);
+    }
+
+    /**
+     * Test multi-column slice restrictions with 2 descending clustering columns (e.g '(clustering_0, clustering_1) > (1, 2)')
+     */
+    @Test
+    public void testBoundsAsCompositesWithMultiSliceRestrictionsWithTwoDescendingClusteringColumns()
+    {
+        CFMetaData cfMetaData = newCFMetaData(Sort.DESC, Sort.DESC);
+
+        ByteBuffer value1 = ByteBufferUtil.bytes(1);
+        ByteBuffer value2 = ByteBufferUtil.bytes(2);
+
+        // (clustering_0, clustering1) > (1, 2)
+        Restriction slice = newMultiSlice(cfMetaData, 0, Bound.START, false, value1, value2);
+        PrimaryKeyRestrictions restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        List<Composite> bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertEmptyComposite(bounds.get(0));
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, EOC.START);
+
+        // (clustering_0, clustering1) >= (1, 2)
+        slice = newMultiSlice(cfMetaData, 0, Bound.START, true, value1, value2);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertEmptyComposite(bounds.get(0));
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, EOC.END);
+
+        // (clustering_0, clustering1) <= (1, 2)
+        slice = newMultiSlice(cfMetaData, 0, Bound.END, true, value1, value2);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, EOC.NONE);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertEmptyComposite(bounds.get(0));
+
+        // (clustering_0, clustering1) < (1, 2)
+        slice = newMultiSlice(cfMetaData, 0, Bound.END, false, value1, value2);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, EOC.END);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertEmptyComposite(bounds.get(0));
+
+
+        // (clustering_0, clustering1) > (1, 2) AND (clustering_0) < (2)
+        slice = newMultiSlice(cfMetaData, 0, Bound.START, false, value1, value2);
+        Restriction slice2 = newMultiSlice(cfMetaData, 0, Bound.END, false, value2);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value2, EOC.END);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, EOC.START);
+
+        // (clustering_0, clustering1) >= (1, 2) AND (clustering_0, clustering1) <= (2, 1)
+        slice = newMultiSlice(cfMetaData, 0, Bound.START, true, value1, value2);
+        slice2 = newMultiSlice(cfMetaData, 0, Bound.END, true, value2, value1);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value2, value1, EOC.NONE);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, EOC.END);
+    }
+
+    /**
+     * Test multi-column slice restrictions with 1 descending clustering column and 1 ascending
+     * (e.g '(clustering_0, clustering_1) > (1, 2)')
+     */
+    @Test
+    public void testBoundsAsCompositesWithMultiSliceRestrictionsWithOneDescendingAndOneAscendingClusteringColumns()
+    {
+        CFMetaData cfMetaData = newCFMetaData(Sort.DESC, Sort.ASC);
+
+        ByteBuffer value1 = ByteBufferUtil.bytes(1);
+        ByteBuffer value2 = ByteBufferUtil.bytes(2);
+
+        // (clustering_0, clustering1) > (1, 2)
+        Restriction slice = newMultiSlice(cfMetaData, 0, Bound.START, false, value1, value2);
+        PrimaryKeyRestrictions restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        List<Composite> bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertEmptyComposite(bounds.get(0));
+        assertComposite(bounds.get(1), value1, value2, EOC.END);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.START);
+        assertComposite(bounds.get(1), value1, EOC.END);
+
+        // (clustering_0, clustering1) >= (1, 2)
+        slice = newMultiSlice(cfMetaData, 0, Bound.START, true, value1, value2);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertEmptyComposite(bounds.get(0));
+        assertComposite(bounds.get(1), value1, value2, EOC.NONE);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.START);
+        assertComposite(bounds.get(1), value1, EOC.END);
+
+        // (clustering_0, clustering1) <= (1, 2)
+        slice = newMultiSlice(cfMetaData, 0, Bound.END, true, value1, value2);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.START);
+        assertComposite(bounds.get(1), value1, EOC.END);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, EOC.END);
+        assertEmptyComposite(bounds.get(1));
+
+        // (clustering_0, clustering1) < (1, 2)
+        slice = newMultiSlice(cfMetaData, 0, Bound.END, false, value1, value2);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.START);
+        assertComposite(bounds.get(1), value1, EOC.END);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, EOC.START);
+        assertEmptyComposite(bounds.get(1));
+
+        // (clustering_0, clustering1) > (1, 2) AND (clustering_0) < (2)
+        slice = newMultiSlice(cfMetaData, 0, Bound.START, false, value1, value2);
+        Restriction slice2 = newMultiSlice(cfMetaData, 0, Bound.END, false, value2);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertComposite(bounds.get(0), value2, EOC.END);
+        assertComposite(bounds.get(1), value1, value2, EOC.END);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.START);
+        assertComposite(bounds.get(1), value1, EOC.END);
+
+        // (clustering_0) > (1) AND (clustering_0, clustering1) < (2, 1)
+        slice = newMultiSlice(cfMetaData, 0, Bound.START, false, value1);
+        slice2 = newMultiSlice(cfMetaData, 0, Bound.END, false, value2, value1);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertComposite(bounds.get(0), value2, EOC.START);
+        assertComposite(bounds.get(1), value2, EOC.END);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertComposite(bounds.get(0), value2, value1, EOC.START);
+        assertComposite(bounds.get(1), value1, EOC.START);
+
+        // (clustering_0, clustering1) >= (1, 2) AND (clustering_0, clustering1) <= (2, 1)
+        slice = newMultiSlice(cfMetaData, 0, Bound.START, true, value1, value2);
+        slice2 = newMultiSlice(cfMetaData, 0, Bound.END, true, value2, value1);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(3, bounds.size());
+        assertComposite(bounds.get(0), value2, EOC.START);
+        assertComposite(bounds.get(1), value2, EOC.END);
+        assertComposite(bounds.get(2), value1, value2, EOC.NONE);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(3, bounds.size());
+        assertComposite(bounds.get(0), value2, value1, EOC.END);
+        assertComposite(bounds.get(1), value1, EOC.START);
+        assertComposite(bounds.get(2), value1, EOC.END);
+    }
+
+    /**
+     * Test multi-column slice restrictions with 1 descending clustering column and 1 ascending
+     * (e.g '(clustering_0, clustering_1) > (1, 2)')
+     */
+    @Test
+    public void testBoundsAsCompositesWithMultiSliceRestrictionsWithOneAscendingAndOneDescendingClusteringColumns()
+    {
+        CFMetaData cfMetaData = newCFMetaData(Sort.ASC, Sort.DESC);
+
+        ByteBuffer value1 = ByteBufferUtil.bytes(1);
+        ByteBuffer value2 = ByteBufferUtil.bytes(2);
+
+        // (clustering_0, clustering1) > (1, 2)
+        Restriction slice = newMultiSlice(cfMetaData, 0, Bound.START, false, value1, value2);
+        PrimaryKeyRestrictions restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        List<Composite> bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.START);
+        assertComposite(bounds.get(1), value1, EOC.END);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, EOC.START);
+        assertEmptyComposite(bounds.get(1));
+
+        // (clustering_0, clustering1) >= (1, 2)
+        slice = newMultiSlice(cfMetaData, 0, Bound.START, true, value1, value2);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.START);
+        assertComposite(bounds.get(1), value1, EOC.END);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, EOC.END);
+        assertEmptyComposite(bounds.get(1));
+
+        // (clustering_0, clustering1) <= (1, 2)
+        slice = newMultiSlice(cfMetaData, 0, Bound.END, true, value1, value2);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertEmptyComposite(bounds.get(0));
+        assertComposite(bounds.get(1), value1, value2, EOC.NONE);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.START);
+        assertComposite(bounds.get(1), value1, EOC.END);
+
+        // (clustering_0, clustering1) < (1, 2)
+        slice = newMultiSlice(cfMetaData, 0, Bound.END, false, value1, value2);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertEmptyComposite(bounds.get(0));
+        assertComposite(bounds.get(1), value1, value2, EOC.END);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.START);
+        assertComposite(bounds.get(1), value1, EOC.END);
+
+        // (clustering_0, clustering1) > (1, 2) AND (clustering_0) < (2)
+        slice = newMultiSlice(cfMetaData, 0, Bound.START, false, value1, value2);
+        Restriction slice2 = newMultiSlice(cfMetaData, 0, Bound.END, false, value2);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.START);
+        assertComposite(bounds.get(1), value1, EOC.END);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, EOC.START);
+        assertComposite(bounds.get(1), value2, EOC.START);
+
+        // (clustering_0, clustering1) >= (1, 2) AND (clustering_0, clustering1) <= (2, 1)
+        slice = newMultiSlice(cfMetaData, 0, Bound.START, true, value1, value2);
+        slice2 = newMultiSlice(cfMetaData, 0, Bound.END, true, value2, value1);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(3, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.START);
+        assertComposite(bounds.get(1), value1, EOC.END);
+        assertComposite(bounds.get(2), value2, value1, EOC.NONE);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(3, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, EOC.END);
+        assertComposite(bounds.get(1), value2, EOC.START);
+        assertComposite(bounds.get(2), value2, EOC.END);
+    }
+
+    /**
+     * Test multi-column slice restrictions with 2 ascending clustering column and 2 descending
+     * (e.g '(clustering_0, clustering1, clustering_3, clustering4) > (1, 2, 3, 4)')
+     */
+    @Test
+    public void testBoundsAsCompositesWithMultiSliceRestrictionsWithTwoAscendingAndTwoDescendingClusteringColumns()
+    {
+        CFMetaData cfMetaData = newCFMetaData(Sort.ASC, Sort.ASC, Sort.DESC, Sort.DESC);
+
+        ByteBuffer value1 = ByteBufferUtil.bytes(1);
+        ByteBuffer value2 = ByteBufferUtil.bytes(2);
+        ByteBuffer value3 = ByteBufferUtil.bytes(3);
+        ByteBuffer value4 = ByteBufferUtil.bytes(4);
+
+        // (clustering_0, clustering1, clustering_2, clustering_3) > (1, 2, 3, 4)
+        Restriction slice = newMultiSlice(cfMetaData, 0, Bound.START, false, value1, value2, value3, value4);
+        PrimaryKeyRestrictions restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        List<Composite> bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, EOC.START);
+        assertComposite(bounds.get(1), value1, value2, EOC.END);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, value3, value4, EOC.START);
+        assertEmptyComposite(bounds.get(1));
+
+        // clustering_0 = 1 AND (clustering_1, clustering_2, clustering_3) > (2, 3, 4)
+        Restriction eq = newSingleEq(cfMetaData, 0, value1);
+        slice = newMultiSlice(cfMetaData, 1, Bound.START, false, value2, value3, value4);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+        restrictions = restrictions.mergeWith(eq);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, EOC.START);
+        assertComposite(bounds.get(1), value1, value2, EOC.END);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, value3, value4, EOC.START);
+        assertComposite(bounds.get(1), value1, EOC.END);
+
+        // clustering_0 = 1 AND (clustering_1, clustering_2, clustering_3) > (2, 3, 4)
+        Restriction in = newSingleIN(cfMetaData, 0, value1, value2);
+        slice = newMultiSlice(cfMetaData, 1, Bound.START, false, value2, value3, value4);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+        restrictions = restrictions.mergeWith(in);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(4, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, EOC.START);
+        assertComposite(bounds.get(1), value1, value2, EOC.END);
+        assertComposite(bounds.get(2), value2, value2, EOC.START);
+        assertComposite(bounds.get(3), value2, value2, EOC.END);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(4, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, value3, value4, EOC.START);
+        assertComposite(bounds.get(1), value1, EOC.END);
+        assertComposite(bounds.get(2), value2, value2, value3, value4, EOC.START);
+        assertComposite(bounds.get(3), value2, EOC.END);
+
+        // (clustering_0, clustering1) >= (1, 2)
+        slice = newMultiSlice(cfMetaData, 0, Bound.START, true, value1, value2);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, EOC.NONE);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertEmptyComposite(bounds.get(0));
+
+        // (clustering_0, clustering1, clustering_2, clustering_3) >= (1, 2, 3, 4)
+        slice = newMultiSlice(cfMetaData, 0, Bound.START, true, value1, value2, value3, value4);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, EOC.START);
+        assertComposite(bounds.get(1), value1, value2, EOC.END);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, value3, value4, EOC.END);
+        assertEmptyComposite(bounds.get(1));
+
+        // (clustering_0, clustering1, clustering_2, clustering_3) <= (1, 2, 3, 4)
+        slice = newMultiSlice(cfMetaData, 0, Bound.END, true, value1, value2, value3, value4);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertEmptyComposite(bounds.get(0));
+        assertComposite(bounds.get(1), value1, value2, value3, value4, EOC.NONE);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, EOC.START);
+        assertComposite(bounds.get(1), value1, value2, EOC.END);
+
+        // (clustering_0, clustering1, clustering_2, clustering_3) < (1, 2, 3, 4)
+        slice = newMultiSlice(cfMetaData, 0, Bound.END, false, value1, value2, value3, value4);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertEmptyComposite(bounds.get(0));
+        assertComposite(bounds.get(1), value1, value2, value3, value4, EOC.END);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, EOC.START);
+        assertComposite(bounds.get(1), value1, value2, EOC.END);
+
+        // (clustering_0, clustering1, clustering_2, clustering_3) > (1, 2, 3, 4) AND (clustering_0, clustering_1) < (2, 3)
+        slice = newMultiSlice(cfMetaData, 0, Bound.START, false, value1, value2, value3, value4);
+        Restriction slice2 = newMultiSlice(cfMetaData, 0, Bound.END, false, value2, value3);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, EOC.START);
+        assertComposite(bounds.get(1), value1, value2, EOC.END);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, value3, value4, EOC.START);
+        assertComposite(bounds.get(1), value2, value3, EOC.START);
+
+        // (clustering_0, clustering1, clustering_2, clustering_3) >= (1, 2, 3, 4) AND (clustering_0, clustering1, clustering_2, clustering_3) <= (4, 3, 2, 1)
+        slice = newMultiSlice(cfMetaData, 0, Bound.START, true, value1, value2, value3, value4);
+        slice2 = newMultiSlice(cfMetaData, 0, Bound.END, true, value4, value3, value2, value1);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(3, bounds.size());
+        assertComposite(bounds.get(0), value1, value2,  EOC.START);
+        assertComposite(bounds.get(1), value1, value2, EOC.END);
+        assertComposite(bounds.get(2), value4, value3, value2, value1, EOC.NONE);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(3, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, value3, value4, EOC.END);
+        assertComposite(bounds.get(1), value4, value3, EOC.START);
+        assertComposite(bounds.get(2), value4, value3, EOC.END);
+    }
+
+    /**
+     * Test multi-column slice restrictions with ascending, descending, ascending and descending columns
+     * (e.g '(clustering_0, clustering1, clustering_3, clustering4) > (1, 2, 3, 4)')
+     */
+    @Test
+    public void testBoundsAsCompositesWithMultiSliceRestrictionsMixingAscendingDescendingClusteringColumns()
+    {
+        CFMetaData cfMetaData = newCFMetaData(Sort.ASC, Sort.DESC, Sort.ASC, Sort.DESC);
+
+        ByteBuffer value1 = ByteBufferUtil.bytes(1);
+        ByteBuffer value2 = ByteBufferUtil.bytes(2);
+        ByteBuffer value3 = ByteBufferUtil.bytes(3);
+        ByteBuffer value4 = ByteBufferUtil.bytes(4);
+
+        // (clustering_0, clustering1, clustering_2, clustering_3) > (1, 2, 3, 4)
+        Restriction slice = newMultiSlice(cfMetaData, 0, Bound.START, false, value1, value2, value3, value4);
+        PrimaryKeyRestrictions restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        List<Composite> bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(4, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.START);
+        assertComposite(bounds.get(1), value1, value2, value3, EOC.START);
+        assertComposite(bounds.get(2), value1, value2, value3, EOC.END);
+        assertComposite(bounds.get(3), value1, EOC.END);
+
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(4, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, EOC.START);
+        assertComposite(bounds.get(1), value1, value2, value3, value4, EOC.START);
+        assertComposite(bounds.get(2), value1, value2, EOC.END);
+        assertEmptyComposite(bounds.get(3));
+
+        // clustering_0 = 1 AND (clustering_1, clustering_2, clustering_3) > (2, 3, 4)
+        Restriction eq = newSingleEq(cfMetaData, 0, value1);
+        slice = newMultiSlice(cfMetaData, 1, Bound.START, false, value2, value3, value4);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+        restrictions = restrictions.mergeWith(eq);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(3, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.START);
+        assertComposite(bounds.get(1), value1, value2, value3, EOC.START);
+        assertComposite(bounds.get(2), value1, value2, value3, EOC.END);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(3, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, EOC.START);
+        assertComposite(bounds.get(1), value1, value2, value3, value4, EOC.START);
+        assertComposite(bounds.get(2), value1, value2, EOC.END);
+
+        // (clustering_0, clustering1) >= (1, 2)
+        slice = newMultiSlice(cfMetaData, 0, Bound.START, true, value1, value2);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.START);
+        assertComposite(bounds.get(1), value1, EOC.END);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, EOC.END);
+        assertEmptyComposite(bounds.get(1));
+
+        // (clustering_0, clustering1, clustering_2, clustering_3) >= (1, 2, 3, 4)
+        slice = newMultiSlice(cfMetaData, 0, Bound.START, true, value1, value2, value3, value4);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(4, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.START);
+        assertComposite(bounds.get(1), value1, value2, value3, EOC.START);
+        assertComposite(bounds.get(2), value1, value2, value3, EOC.END);
+        assertComposite(bounds.get(3), value1, EOC.END);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(4, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, EOC.START);
+        assertComposite(bounds.get(1), value1, value2, value3, value4, EOC.END);
+        assertComposite(bounds.get(2), value1, value2, EOC.END);
+        assertEmptyComposite(bounds.get(3));
+
+
+        // (clustering_0, clustering1, clustering_2, clustering_3) <= (1, 2, 3, 4)
+        slice = newMultiSlice(cfMetaData, 0, Bound.END, true, value1, value2, value3, value4);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(4, bounds.size());
+        assertEmptyComposite(bounds.get(0));
+        assertComposite(bounds.get(1), value1, value2, EOC.START);
+        assertComposite(bounds.get(2), value1, value2, value3, value4, EOC.NONE);
+        assertComposite(bounds.get(3), value1, value2, EOC.END);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(4, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.START);
+        assertComposite(bounds.get(1), value1, value2, value3, EOC.START);
+        assertComposite(bounds.get(2), value1, value2, value3, EOC.END);
+        assertComposite(bounds.get(3), value1, EOC.END);
+
+        // (clustering_0, clustering1, clustering_2, clustering_3) < (1, 2, 3, 4)
+        slice = newMultiSlice(cfMetaData, 0, Bound.END, false, value1, value2, value3, value4);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(4, bounds.size());
+        assertEmptyComposite(bounds.get(0));
+        assertComposite(bounds.get(1), value1, value2, EOC.START);
+        assertComposite(bounds.get(2), value1, value2, value3, value4, EOC.END);
+        assertComposite(bounds.get(3), value1, value2, EOC.END);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(4, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.START);
+        assertComposite(bounds.get(1), value1, value2, value3, EOC.START);
+        assertComposite(bounds.get(2), value1, value2, value3, EOC.END);
+        assertComposite(bounds.get(3), value1, EOC.END);
+
+        // (clustering_0, clustering1, clustering_2, clustering_3) > (1, 2, 3, 4) AND (clustering_0, clustering_1) < (2, 3)
+        slice = newMultiSlice(cfMetaData, 0, Bound.START, false, value1, value2, value3, value4);
+        Restriction slice2 = newMultiSlice(cfMetaData, 0, Bound.END, false, value2, value3);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(5, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.START);
+        assertComposite(bounds.get(1), value1, value2, value3, EOC.START);
+        assertComposite(bounds.get(2), value1, value2, value3, EOC.END);
+        assertComposite(bounds.get(3), value1, EOC.END);
+        assertComposite(bounds.get(4), value2, value3, EOC.END);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(5, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, EOC.START);
+        assertComposite(bounds.get(1), value1, value2, value3, value4, EOC.START);
+        assertComposite(bounds.get(2), value1, value2, EOC.END);
+        assertComposite(bounds.get(3), value2, EOC.START);
+        assertComposite(bounds.get(4), value2, EOC.END);
+
+        // (clustering_0, clustering1, clustering_2, clustering_3) >= (1, 2, 3, 4) AND (clustering_0, clustering1, clustering_2, clustering_3) <= (4, 3, 2, 1)
+        slice = newMultiSlice(cfMetaData, 0, Bound.START, true, value1, value2, value3, value4);
+        slice2 = newMultiSlice(cfMetaData, 0, Bound.END, true, value4, value3, value2, value1);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(slice).mergeWith(slice2);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(7, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.START);
+        assertComposite(bounds.get(1), value1, value2, value3, EOC.START);
+        assertComposite(bounds.get(2), value1, value2, value3, EOC.END);
+        assertComposite(bounds.get(3), value1, EOC.END);
+        assertComposite(bounds.get(4), value4, value3, EOC.START);
+        assertComposite(bounds.get(5), value4, value3, value2, value1, EOC.NONE);
+        assertComposite(bounds.get(6), value4, value3, EOC.END);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(7, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, EOC.START);
+        assertComposite(bounds.get(1), value1, value2, value3, value4, EOC.END);
+        assertComposite(bounds.get(2), value1, value2, EOC.END);
+        assertComposite(bounds.get(3), value4, EOC.START);
+        assertComposite(bounds.get(4), value4, value3, value2, EOC.START);
+        assertComposite(bounds.get(5), value4, value3, value2, EOC.END);
+        assertComposite(bounds.get(6), value4, EOC.END);
+    }
+
+    /**
+     * Test mixing single and multi equals restrictions (e.g. clustering_0 = 1 AND (clustering_1, clustering_2) = (2, 3))
+     */
+    @Test
+    public void testBoundsAsCompositesWithSingleEqAndMultiEqRestrictions()
+    {
+        CFMetaData cfMetaData = newCFMetaData(Sort.ASC, Sort.ASC, Sort.ASC, Sort.ASC);
+
+        ByteBuffer value1 = ByteBufferUtil.bytes(1);
+        ByteBuffer value2 = ByteBufferUtil.bytes(2);
+        ByteBuffer value3 = ByteBufferUtil.bytes(3);
+        ByteBuffer value4 = ByteBufferUtil.bytes(4);
+
+        // clustering_0 = 1 AND (clustering_1, clustering_2) = (2, 3)
+        Restriction singleEq = newSingleEq(cfMetaData, 0, value1);
+        Restriction multiEq = newMultiEq(cfMetaData, 1, value2, value3);
+        PrimaryKeyRestrictions restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(singleEq).mergeWith(multiEq);
+
+        List<Composite> bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, value3, EOC.START);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, value3, EOC.END);
+
+        // clustering_0 = 1 AND clustering_1 = 2 AND (clustering_2, clustering_3) = (3, 4)
+        singleEq = newSingleEq(cfMetaData, 0, value1);
+        Restriction singleEq2 = newSingleEq(cfMetaData, 1, value2);
+        multiEq = newMultiEq(cfMetaData, 2, value3, value4);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(singleEq).mergeWith(singleEq2).mergeWith(multiEq);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, value3, value4, EOC.START);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, value3, value4, EOC.END);
+
+        // (clustering_0, clustering_1) = (1, 2) AND clustering_2 = 3
+        singleEq = newSingleEq(cfMetaData, 2, value3);
+        multiEq = newMultiEq(cfMetaData, 0, value1, value2);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(singleEq).mergeWith(multiEq);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, value3, EOC.START);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, value3, EOC.END);
+
+        // clustering_0 = 1 AND (clustering_1, clustering_2) = (2, 3) AND clustering_3 = 4
+        singleEq = newSingleEq(cfMetaData, 0, value1);
+        singleEq2 = newSingleEq(cfMetaData, 3, value4);
+        multiEq = newMultiEq(cfMetaData, 1, value2, value3);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(singleEq).mergeWith(multiEq).mergeWith(singleEq2);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, value3, value4, EOC.START);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, value3, value4, EOC.END);
+    }
+
+    /**
+     * Test clustering_0 = 1 AND (clustering_1, clustering_2) IN ((2, 3), (4, 5))
+     */
+    @Test
+    public void testBoundsAsCompositesWithSingleEqAndMultiINRestrictions()
+    {
+        CFMetaData cfMetaData = newCFMetaData(Sort.ASC, Sort.ASC, Sort.ASC, Sort.ASC);
+
+        ByteBuffer value1 = ByteBufferUtil.bytes(1);
+        ByteBuffer value2 = ByteBufferUtil.bytes(2);
+        ByteBuffer value3 = ByteBufferUtil.bytes(3);
+        ByteBuffer value4 = ByteBufferUtil.bytes(4);
+        ByteBuffer value5 = ByteBufferUtil.bytes(5);
+
+        // clustering_0 = 1 AND (clustering_1, clustering_2) IN ((2, 3), (4, 5))
+        Restriction singleEq = newSingleEq(cfMetaData, 0, value1);
+        Restriction multiIN = newMultiIN(cfMetaData, 1, asList(value2, value3), asList(value4, value5));
+        PrimaryKeyRestrictions restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(singleEq).mergeWith(multiIN);
+
+        List<Composite> bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, value3, EOC.START);
+        assertComposite(bounds.get(1), value1, value4, value5, EOC.START);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, value3, EOC.END);
+        assertComposite(bounds.get(1), value1, value4, value5, EOC.END);
+
+        // clustering_0 = 1 AND (clustering_1, clustering_2) IN ((2, 3))
+        singleEq = newSingleEq(cfMetaData, 0, value1);
+        multiIN = newMultiIN(cfMetaData, 1, asList(value2, value3));
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(multiIN).mergeWith(singleEq);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, value3, EOC.START);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, value3, EOC.END);
+
+        // clustering_0 = 1 AND clustering_1 = 5 AND (clustering_2, clustering_3) IN ((2, 3), (4, 5))
+        singleEq = newSingleEq(cfMetaData, 0, value1);
+        Restriction singleEq2 = newSingleEq(cfMetaData, 1, value5);
+        multiIN = newMultiIN(cfMetaData, 2, asList(value2, value3), asList(value4, value5));
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(singleEq).mergeWith(multiIN).mergeWith(singleEq2);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertComposite(bounds.get(0), value1, value5, value2, value3, EOC.START);
+        assertComposite(bounds.get(1), value1, value5, value4, value5, EOC.START);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertComposite(bounds.get(0), value1, value5, value2, value3, EOC.END);
+        assertComposite(bounds.get(1), value1, value5, value4, value5, EOC.END);
+    }
+
+    /**
+     * Test mixing single equal restrictions with multi-column slice restrictions
+     * (e.g. clustering_0 = 1 AND (clustering_1, clustering_2) > (2, 3))
+     */
+    @Test
+    public void testBoundsAsCompositesWithSingleEqAndSliceRestrictions()
+    {
+        CFMetaData cfMetaData = newCFMetaData(Sort.ASC, Sort.ASC, Sort.ASC);
+
+        ByteBuffer value1 = ByteBufferUtil.bytes(1);
+        ByteBuffer value2 = ByteBufferUtil.bytes(2);
+        ByteBuffer value3 = ByteBufferUtil.bytes(3);
+        ByteBuffer value4 = ByteBufferUtil.bytes(4);
+        ByteBuffer value5 = ByteBufferUtil.bytes(5);
+
+        // clustering_0 = 1 AND (clustering_1, clustering_2) > (2, 3)
+        Restriction singleEq = newSingleEq(cfMetaData, 0, value1);
+        Restriction multiSlice = newMultiSlice(cfMetaData, 1, Bound.START, false, value2, value3);
+        PrimaryKeyRestrictions restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(singleEq).mergeWith(multiSlice);
+
+        List<Composite> bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, value3, EOC.END);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, EOC.END);
+
+        // clustering_0 = 1 AND (clustering_1, clustering_2) > (2, 3) AND (clustering_1) < (4)
+        singleEq = newSingleEq(cfMetaData, 0, value1);
+        multiSlice = newMultiSlice(cfMetaData, 1, Bound.START, false, value2, value3);
+        Restriction multiSlice2 = newMultiSlice(cfMetaData, 1, Bound.END, false, value4);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(multiSlice2).mergeWith(singleEq).mergeWith(multiSlice);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, value3, EOC.END);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, value4, EOC.START);
+
+        // clustering_0 = 1 AND (clustering_1, clustering_2) => (2, 3) AND (clustering_1, clustering_2) <= (4, 5)
+        singleEq = newSingleEq(cfMetaData, 0, value1);
+        multiSlice = newMultiSlice(cfMetaData, 1, Bound.START, true, value2, value3);
+        multiSlice2 = newMultiSlice(cfMetaData, 1, Bound.END, true, value4, value5);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(multiSlice2).mergeWith(singleEq).mergeWith(multiSlice);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, value3, EOC.NONE);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, value4, value5, EOC.END);
+    }
+
+    /**
+     * Test mixing multi equal restrictions with single-column slice restrictions
+     * (e.g. clustering_0 = 1 AND (clustering_1, clustering_2) > (2, 3))
+     */
+    @Test
+    public void testBoundsAsCompositesWithMultiEqAndSingleSliceRestrictions()
+    {
+        CFMetaData cfMetaData = newCFMetaData(Sort.ASC, Sort.ASC, Sort.ASC);
+
+        ByteBuffer value1 = ByteBufferUtil.bytes(1);
+        ByteBuffer value2 = ByteBufferUtil.bytes(2);
+        ByteBuffer value3 = ByteBufferUtil.bytes(3);
+
+        // (clustering_0, clustering_1) = (1, 2) AND clustering_2 > 3
+        Restriction multiEq = newMultiEq(cfMetaData, 0, value1, value2);
+        Restriction singleSlice = newSingleSlice(cfMetaData, 2, Bound.START, false, value3);
+        PrimaryKeyRestrictions restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(multiEq).mergeWith(singleSlice);
+
+        List<Composite> bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, value3, EOC.END);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0),  value1, value2, EOC.END);
+    }
+
+    @Test
+    public void testBoundsAsCompositesWithSeveralMultiColumnRestrictions()
+    {
+        CFMetaData cfMetaData = newCFMetaData(Sort.ASC, Sort.ASC, Sort.ASC, Sort.ASC);
+
+        ByteBuffer value1 = ByteBufferUtil.bytes(1);
+        ByteBuffer value2 = ByteBufferUtil.bytes(2);
+        ByteBuffer value3 = ByteBufferUtil.bytes(3);
+        ByteBuffer value4 = ByteBufferUtil.bytes(4);
+        ByteBuffer value5 = ByteBufferUtil.bytes(5);
+
+        // (clustering_0, clustering_1) = (1, 2) AND (clustering_2, clustering_3) > (3, 4)
+        Restriction multiEq = newMultiEq(cfMetaData, 0, value1, value2);
+        Restriction multiSlice = newMultiSlice(cfMetaData, 2, Bound.START, false, value3, value4);
+        PrimaryKeyRestrictions restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(multiEq).mergeWith(multiSlice);
+
+        List<Composite> bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, value3, value4, EOC.END);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0),  value1, value2, EOC.END);
+
+        // (clustering_0, clustering_1) = (1, 2) AND (clustering_2, clustering_3) IN ((3, 4), (4, 5))
+        multiEq = newMultiEq(cfMetaData, 0, value1, value2);
+        Restriction multiIN = newMultiIN(cfMetaData, 2, asList(value3, value4), asList(value4, value5));
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(multiEq).mergeWith(multiIN);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, value3, value4, EOC.START);
+        assertComposite(bounds.get(1), value1, value2, value4, value5, EOC.START);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(2, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, value3, value4, EOC.END);
+        assertComposite(bounds.get(1), value1, value2, value4, value5, EOC.END);
+
+        // (clustering_0, clustering_1) = (1, 2) AND (clustering_2, clustering_3) = (3, 4)
+        multiEq = newMultiEq(cfMetaData, 0, value1, value2);
+        Restriction multiEq2 = newMultiEq(cfMetaData, 2, value3, value4);
+        restrictions = new PrimaryKeyRestrictionSet(cfMetaData.comparator);
+        restrictions = restrictions.mergeWith(multiEq).mergeWith(multiEq2);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.START, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, value3, value4, EOC.START);
+
+        bounds = restrictions.boundsAsComposites(cfMetaData, Bound.END, QueryOptions.DEFAULT);
+        assertEquals(1, bounds.size());
+        assertComposite(bounds.get(0), value1, value2, value3, value4, EOC.END);
+    }
+
+    /**
+     * Asserts that the specified <code>Composite</code> is an empty one.
+     *
+     * @param composite the composite to check
+     */
+    private static void assertEmptyComposite(Composite composite)
+    {
+        assertEquals(Composites.EMPTY, composite);
+    }
+
+    /**
+     * Asserts that the specified <code>Composite</code> contains the specified element and the specified EOC.
+     *
+     * @param composite the composite to check
+     * @param element the expected element of the composite
+     * @param eoc the expected EOC of the composite
+     */
+    private static void assertComposite(Composite composite, ByteBuffer element, EOC eoc)
+    {
+        assertComposite(composite, eoc, element);
+    }
+
+    /**
+     * Asserts that the specified <code>Composite</code> contains the 2 specified element and the specified EOC.
+     *
+     * @param composite the composite to check
+     * @param eoc the expected EOC of the composite
+     * @param elements the expected element of the composite
+     */
+    private static void assertComposite(Composite composite, ByteBuffer firstElement, ByteBuffer secondElement, EOC eoc)
+    {
+        assertComposite(composite, eoc, firstElement, secondElement);
+    }
+
+    /**
+     * Asserts that the specified <code>Composite</code> contains the 3 specified element and the specified EOC.
+     *
+     * @param composite the composite to check
+     * @param firstElement the first expected element of the composite
+     * @param secondElement the second expected element of the composite
+     * @param thirdElement the third expected element of the composite
+     * @param eoc the expected EOC of the composite
+     * @param elements the expected element of the composite
+     */
+    private static void assertComposite(Composite composite,
+                                        ByteBuffer firstElement,
+                                        ByteBuffer secondElement,
+                                        ByteBuffer thirdElement,
+                                        EOC eoc)
+    {
+        assertComposite(composite, eoc, firstElement, secondElement, thirdElement);
+    }
+
+    /**
+     * Asserts that the specified <code>Composite</code> contains the 4 specified element and the specified EOC.
+     *
+     * @param composite the composite to check
+     * @param firstElement the first expected element of the composite
+     * @param secondElement the second expected element of the composite
+     * @param thirdElement the third expected element of the composite
+     * @param fourthElement the fourth expected element of the composite
+     * @param eoc the expected EOC of the composite
+     * @param elements the expected element of the composite
+     */
+    private static void assertComposite(Composite composite,
+                                        ByteBuffer firstElement,
+                                        ByteBuffer secondElement,
+                                        ByteBuffer thirdElement,
+                                        ByteBuffer fourthElement,
+                                        EOC eoc)
+    {
+        assertComposite(composite, eoc, firstElement, secondElement, thirdElement, fourthElement);
+    }
+
+    /**
+     * Asserts that the specified <code>Composite</code> contains the specified elements and EOC.
+     *
+     * @param composite the composite to check
+     * @param eoc the expected EOC of the composite
+     * @param elements the expected elements of the composite
+     */
+    private static void assertComposite(Composite composite, EOC eoc, ByteBuffer... elements)
+    {
+        assertEquals("the composite size is not the expected one:", elements.length, composite.size());
+        for (int i = 0, m = elements.length; i < m; i++)
+        {
+            ByteBuffer element = elements[i];
+            assertTrue(String.format("the element %s of the composite is not the expected one: expected %s but was %s",
+                                     i,
+                                     ByteBufferUtil.toInt(element),
+                                     ByteBufferUtil.toInt(composite.get(i))),
+                       element.equals(composite.get(i)));
+        }
+        assertEquals("the EOC of the composite is not the expected one:", eoc, composite.eoc());
+    }
+
+    /**
+     * Creates a new <code>CFMetaData</code> instance.
+     *
+     * @param numberOfClusteringColumns the number of clustering column
+     * @return a new <code>CFMetaData</code> instance
+     */
+    private static CFMetaData newCFMetaData(Sort... sorts)
+    {
+        List<AbstractType<?>> types = new ArrayList<>();
+
+        for (Sort sort : sorts)
+            types.add(sort == Sort.ASC ? Int32Type.instance : ReversedType.getInstance(Int32Type.instance));
+
+        CompoundSparseCellNameType cType = new CompoundSparseCellNameType(types);
+        CFMetaData cfMetaData = new CFMetaData("keyspace", "test", ColumnFamilyType.Standard, cType);
+
+        for (int i = 0; i < sorts.length; i++)
+        {
+            ByteBuffer name = ByteBufferUtil.bytes("clustering_" + i);
+            ColumnDefinition columnDef = ColumnDefinition.clusteringKeyDef(cfMetaData, name, types.get(i), i);
+            cfMetaData.addColumnDefinition(columnDef);
+        }
+        cfMetaData.rebuild();
+        return cfMetaData;
+    }
+
+    /**
+     * Creates a new <code>SingleColumnRestriction.EQ</code> instance for the specified clustering column.
+     *
+     * @param cfMetaData the column family meta data
+     * @param index the clustering column index
+     * @param value the equality value
+     * @return a new <code>SingleColumnRestriction.EQ</code> instance for the specified clustering column
+     */
+    private static Restriction newSingleEq(CFMetaData cfMetaData, int index, ByteBuffer value)
+    {
+        ColumnDefinition columnDef = getClusteringColumnDefinition(cfMetaData, index);
+        return new SingleColumnRestriction.EQ(columnDef, toTerm(value));
+    }
+
+    /**
+     * Creates a new <code>MultiColumnRestriction.EQ</code> instance for the specified clustering column.
+     *
+     * @param cfMetaData the column family meta data
+     * @param index the clustering column index
+     * @param value the equality value
+     * @return a new <code>MultiColumnRestriction.EQ</code> instance for the specified clustering column
+     */
+    private static Restriction newMultiEq(CFMetaData cfMetaData, int firstIndex, ByteBuffer... values)
+    {
+        List<ColumnDefinition> columnDefinitions = new ArrayList<>();
+        for (int i = 0; i < values.length; i++)
+        {
+            columnDefinitions.add(getClusteringColumnDefinition(cfMetaData, firstIndex + i));
+        }
+        return new MultiColumnRestriction.EQ(columnDefinitions, toMultiItemTerminal(values));
+    }
+
+    /**
+     * Creates a new <code>MultiColumnRestriction.IN</code> instance for the specified clustering column.
+     *
+     * @param cfMetaData the column family meta data
+     * @param firstIndex the index of the first clustering column
+     * @param values the in values
+     * @return a new <code>MultiColumnRestriction.IN</code> instance for the specified clustering column
+     */
+    @SafeVarargs
+    private static Restriction newMultiIN(CFMetaData cfMetaData, int firstIndex, List<ByteBuffer>... values)
+    {
+        List<ColumnDefinition> columnDefinitions = new ArrayList<>();
+        List<Term> terms = new ArrayList<>();
+        for (int i = 0; i < values.length; i++)
+        {
+            columnDefinitions.add(getClusteringColumnDefinition(cfMetaData, firstIndex + i));
+            terms.add(toMultiItemTerminal(values[i].toArray(new ByteBuffer[0])));
+        }
+        return new MultiColumnRestriction.InWithValues(columnDefinitions, terms);
+    }
+
+    /**
+     * Creates a new <code>SingleColumnRestriction.IN</code> instance for the specified clustering column.
+     *
+     * @param cfMetaData the column family meta data
+     * @param index the clustering column index
+     * @param values the in values
+     * @return a new <code>SingleColumnRestriction.IN</code> instance for the specified clustering column
+     */
+    private static Restriction newSingleIN(CFMetaData cfMetaData, int index, ByteBuffer... values)
+    {
+        ColumnDefinition columnDef = getClusteringColumnDefinition(cfMetaData, index);
+        return new SingleColumnRestriction.InWithValues(columnDef, toTerms(values));
+    }
+
+    /**
+     * Returns the clustering <code>ColumnDefinition</code> for the specified position.
+     *
+     * @param cfMetaData the column family meta data
+     * @param index the clustering column index
+     * @return the clustering <code>ColumnDefinition</code> for the specified position.
+     */
+    private static ColumnDefinition getClusteringColumnDefinition(CFMetaData cfMetaData, int index)
+    {
+        return cfMetaData.clusteringColumns().get(index);
+    }
+
+    /**
+     * Creates a new <code>SingleColumnRestriction.Slice</code> instance for the specified clustering column.
+     *
+     * @param cfMetaData the column family meta data
+     * @param index the clustering column index
+     * @param bound the slice bound
+     * @param inclusive <code>true</code> if the bound is inclusive
+     * @param value the bound value
+     * @return a new <code>SingleColumnRestriction.Slice</code> instance for the specified clustering column
+     */
+    private static Restriction newSingleSlice(CFMetaData cfMetaData, int index, Bound bound, boolean inclusive, ByteBuffer value)
+    {
+        ColumnDefinition columnDef = getClusteringColumnDefinition(cfMetaData, index);
+        return new SingleColumnRestriction.Slice(columnDef, bound, inclusive, toTerm(value));
+    }
+
+    /**
+     * Creates a new <code>SingleColumnRestriction.Slice</code> instance for the specified clustering column.
+     *
+     * @param cfMetaData the column family meta data
+     * @param index the clustering column index
+     * @param bound the slice bound
+     * @param inclusive <code>true</code> if the bound is inclusive
+     * @param value the bound value
+     * @return a new <code>SingleColumnRestriction.Slice</code> instance for the specified clustering column
+     */
+    private static Restriction newMultiSlice(CFMetaData cfMetaData, int firstIndex, Bound bound, boolean inclusive, ByteBuffer... values)
+    {
+        List<ColumnDefinition> columnDefinitions = new ArrayList<>();
+        for (int i = 0; i < values.length; i++)
+        {
+            columnDefinitions.add(getClusteringColumnDefinition(cfMetaData, i + firstIndex));
+        }
+        return new MultiColumnRestriction.Slice(columnDefinitions, bound, inclusive, toMultiItemTerminal(values));
+    }
+
+    /**
+     * Converts the specified values into a <code>MultiItemTerminal</code>.
+     *
+     * @param values the values to convert.
+     * @return the term corresponding to the specified values.
+     */
+    private static MultiItemTerminal toMultiItemTerminal(ByteBuffer... values)
+    {
+        return new Tuples.Value(values);
+    }
+
+    /**
+     * Converts the specified value into a term.
+     *
+     * @param value the value to convert.
+     * @return the term corresponding to the specified value.
+     */
+    private static Term toTerm(ByteBuffer value)
+    {
+        return new Constants.Value(value);
+    }
+
+    /**
+     * Converts the specified values into a <code>List</code> of terms.
+     *
+     * @param values the values to convert.
+     * @return a <code>List</code> of terms corresponding to the specified values.
+     */
+    private static List<Term> toTerms(ByteBuffer... values)
+    {
+        List<Term> terms = new ArrayList<>();
+        for (ByteBuffer value : values)
+            terms.add(toTerm(value));
+        return terms;
+    }
+
+    private static enum Sort
+    {
+        ASC,
+        DESC;
+    }
+}
diff --git a/test/unit/org/apache/cassandra/cql3/selection/SelectionColumnMappingTest.java b/test/unit/org/apache/cassandra/cql3/selection/SelectionColumnMappingTest.java
new file mode 100644
index 0000000..8757b19
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/selection/SelectionColumnMappingTest.java
@@ -0,0 +1,530 @@
+/*
+ *
+ * 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.cql3.selection;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.cql3.statements.SelectStatement;
+import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.exceptions.RequestExecutionException;
+import org.apache.cassandra.exceptions.RequestValidationException;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class SelectionColumnMappingTest extends CQLTester
+{
+    private static final ColumnDefinition NULL_DEF = null;
+    String tableName;
+    String typeName;
+    UserType userType;
+    String functionName;
+
+    @BeforeClass
+    public static void setUpClass()
+    {
+        DatabaseDescriptor.setPartitioner(ByteOrderedPartitioner.instance);
+    }
+
+    @Test
+    public void testSelectionColumnMapping() throws Throwable
+    {
+        // Organised as a single test to avoid the overhead of
+        // table creation for each variant
+
+        typeName = createType("CREATE TYPE %s (f1 int, f2 text)");
+        tableName = createTable("CREATE TABLE %s (" +
+                                " k int PRIMARY KEY," +
+                                " v1 int," +
+                                " v2 ascii," +
+                                " v3 frozen<" + typeName + ">)");
+        userType = Schema.instance.getKSMetaData(KEYSPACE).userTypes.getType(ByteBufferUtil.bytes(typeName));
+        functionName = createFunction(KEYSPACE, "int, ascii",
+                                      "CREATE FUNCTION %s (i int, a ascii) " +
+                                      "CALLED ON NULL INPUT " +
+                                      "RETURNS int " +
+                                      "LANGUAGE java " +
+                                      "AS 'return Integer.valueOf(i);'");
+        execute("INSERT INTO %s (k, v1 ,v2, v3) VALUES (1, 1, 'foo', {f1:1, f2:'bar'})");
+
+        testSimpleTypes();
+        testWildcard();
+        testSimpleTypesWithAliases();
+        testUserTypes();
+        testUserTypesWithAliases();
+        testWritetimeAndTTL();
+        testWritetimeAndTTLWithAliases();
+        testFunction();
+        testNoArgFunction();
+        testUserDefinedFunction();
+        testOverloadedFunction();
+        testFunctionWithAlias();
+        testNoArgumentFunction();
+        testNestedFunctions();
+        testNestedFunctionsWithArguments();
+        testCount();
+        testDuplicateFunctionsWithoutAliases();
+        testDuplicateFunctionsWithAliases();
+        testSelectDistinct();
+        testMultipleAliasesOnSameColumn();
+        testMixedColumnTypes();
+        testMultipleUnaliasedSelectionOfSameColumn();
+        testUserDefinedAggregate();
+    }
+
+    @Test
+    public void testMultipleArgumentFunction() throws Throwable
+    {
+        // demonstrate behaviour of token() with composite partition key
+        tableName = createTable("CREATE TABLE %s (a int, b text, PRIMARY KEY ((a, b)))");
+        ColumnSpecification tokenSpec = columnSpecification("system.token(a, b)", BytesType.instance);
+        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
+                                                                .addMapping(tokenSpec, columnDefinitions("a", "b"));
+        // we don't use verify like with the other tests because this query will produce no results
+        SelectStatement statement = getSelect("SELECT token(a,b) FROM %s");
+        verifyColumnMapping(expected, statement);
+        statement.executeInternal(QueryState.forInternalCalls(), QueryOptions.DEFAULT);
+    }
+
+    private void testSimpleTypes() throws Throwable
+    {
+        // simple column identifiers without aliases are represented in
+        // ResultSet.Metadata by the underlying ColumnDefinition
+        ColumnSpecification kSpec = columnSpecification("k", Int32Type.instance);
+        ColumnSpecification v1Spec = columnSpecification("v1", Int32Type.instance);
+        ColumnSpecification v2Spec = columnSpecification("v2", AsciiType.instance);
+        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
+                                                                .addMapping(kSpec, columnDefinition("k"))
+                                                                .addMapping(v1Spec, columnDefinition("v1"))
+                                                                .addMapping(v2Spec, columnDefinition("v2"));
+
+        verify(expected, "SELECT k, v1, v2 FROM %s");
+    }
+
+    private void testWildcard() throws Throwable
+    {
+        // Wildcard select represents each column in the table with a ColumnDefinition
+        // in the ResultSet metadata
+        ColumnDefinition kSpec = columnDefinition("k");
+        ColumnDefinition v1Spec = columnDefinition("v1");
+        ColumnDefinition v2Spec = columnDefinition("v2");
+        ColumnDefinition v3Spec = columnDefinition("v3");
+        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
+                                                                .addMapping(kSpec, columnDefinition("k"))
+                                                                .addMapping(v1Spec, columnDefinition("v1"))
+                                                                .addMapping(v2Spec, columnDefinition("v2"))
+                                                                .addMapping(v3Spec, columnDefinition("v3"));
+
+        verify(expected, "SELECT * FROM %s");
+    }
+
+    private void testSimpleTypesWithAliases() throws Throwable
+    {
+        // simple column identifiers with aliases are represented in ResultSet.Metadata
+        // by a ColumnSpecification based on the underlying ColumnDefinition
+        ColumnSpecification kSpec = columnSpecification("k_alias", Int32Type.instance);
+        ColumnSpecification v1Spec = columnSpecification("v1_alias", Int32Type.instance);
+        ColumnSpecification v2Spec = columnSpecification("v2_alias", AsciiType.instance);
+        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
+                                                                .addMapping(kSpec, columnDefinition("k"))
+                                                                .addMapping(v1Spec, columnDefinition("v1"))
+                                                                .addMapping(v2Spec, columnDefinition("v2"));
+
+        verify(expected, "SELECT k AS k_alias, v1 AS v1_alias, v2 AS v2_alias FROM %s");
+    }
+
+    private void testUserTypes() throws Throwable
+    {
+        // User type fields are represented in ResultSet.Metadata by a
+        // ColumnSpecification denoting the name and type of the particular field
+        ColumnSpecification f1Spec = columnSpecification("v3.f1", Int32Type.instance);
+        ColumnSpecification f2Spec = columnSpecification("v3.f2", UTF8Type.instance);
+        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
+                                                                .addMapping(f1Spec, columnDefinition("v3"))
+                                                                .addMapping(f2Spec, columnDefinition("v3"));
+
+        verify(expected, "SELECT v3.f1, v3.f2 FROM %s");
+    }
+
+    private void testUserTypesWithAliases() throws Throwable
+    {
+        // User type fields with aliases are represented in ResultSet.Metadata
+        // by a ColumnSpecification with the alias name and the type of the actual field
+        ColumnSpecification f1Spec = columnSpecification("f1_alias", Int32Type.instance);
+        ColumnSpecification f2Spec = columnSpecification("f2_alias", UTF8Type.instance);
+        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
+                                                                .addMapping(f1Spec, columnDefinition("v3"))
+                                                                .addMapping(f2Spec, columnDefinition("v3"));
+
+        verify(expected, "SELECT v3.f1 AS f1_alias, v3.f2 AS f2_alias FROM %s");
+    }
+
+    private void testWritetimeAndTTL() throws Throwable
+    {
+        // writetime and ttl are represented in ResultSet.Metadata by a ColumnSpecification
+        // with the function name plus argument and a long or int type respectively
+        ColumnSpecification wtSpec = columnSpecification("writetime(v1)", LongType.instance);
+        ColumnSpecification ttlSpec = columnSpecification("ttl(v2)", Int32Type.instance);
+        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
+                                                                .addMapping(wtSpec, columnDefinition("v1"))
+                                                                .addMapping(ttlSpec, columnDefinition("v2"));
+
+        verify(expected, "SELECT writetime(v1), ttl(v2) FROM %s");
+    }
+
+    private void testWritetimeAndTTLWithAliases() throws Throwable
+    {
+        // writetime and ttl with aliases are represented in ResultSet.Metadata
+        // by a ColumnSpecification with the alias name and the appropriate numeric type
+        ColumnSpecification wtSpec = columnSpecification("wt_alias", LongType.instance);
+        ColumnSpecification ttlSpec = columnSpecification("ttl_alias", Int32Type.instance);
+        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
+                                                                .addMapping(wtSpec, columnDefinition("v1"))
+                                                                .addMapping(ttlSpec, columnDefinition("v2"));
+
+        verify(expected, "SELECT writetime(v1) AS wt_alias, ttl(v2) AS ttl_alias FROM %s");
+    }
+
+    private void testFunction() throws Throwable
+    {
+        // a function such as intasblob(<col>) is represented in ResultSet.Metadata
+        // by a ColumnSpecification with the function name plus args and the type set
+        // to the function's return type
+        ColumnSpecification fnSpec = columnSpecification("system.intasblob(v1)", BytesType.instance);
+        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
+                                                                .addMapping(fnSpec, columnDefinition("v1"));
+
+        verify(expected, "SELECT intasblob(v1) FROM %s");
+    }
+
+    private void testNoArgFunction() throws Throwable
+    {
+        // a no-arg function such as now() is represented in ResultSet.Metadata
+        // but has no mapping to any underlying column
+        ColumnSpecification fnSpec = columnSpecification("system.now()", TimeUUIDType.instance);
+        SelectionColumnMapping expected = SelectionColumnMapping.newMapping().addMapping(fnSpec, NULL_DEF);
+
+        verify(expected, "SELECT now() FROM %s");
+    }
+
+    private void testOverloadedFunction() throws Throwable
+    {
+        String fnName = createFunction(KEYSPACE, "int",
+                                       "CREATE FUNCTION %s (input int) " +
+                                       "RETURNS NULL ON NULL INPUT " +
+                                       "RETURNS text " +
+                                       "LANGUAGE java " +
+                                       "AS 'return \"Hello World\";'");
+        createFunctionOverload(fnName, "text",
+                               "CREATE FUNCTION %s (input text) " +
+                               "RETURNS NULL ON NULL INPUT " +
+                               "RETURNS text " +
+                               "LANGUAGE java " +
+                               "AS 'return \"Hello World\";'");
+
+        createFunctionOverload(fnName, "int, text",
+                               "CREATE FUNCTION %s (input1 int, input2 text) " +
+                               "RETURNS NULL ON NULL INPUT " +
+                               "RETURNS text " +
+                               "LANGUAGE java " +
+                               "AS 'return \"Hello World\";'");
+        ColumnSpecification fnSpec1 = columnSpecification(fnName + "(v1)", UTF8Type.instance);
+        ColumnSpecification fnSpec2 = columnSpecification(fnName + "(v2)", UTF8Type.instance);
+        ColumnSpecification fnSpec3 = columnSpecification(fnName + "(v1, v2)", UTF8Type.instance);
+        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
+                                                                .addMapping(fnSpec1, columnDefinition("v1"))
+                                                                .addMapping(fnSpec2, columnDefinition("v2"))
+                                                                .addMapping(fnSpec3, columnDefinitions("v1", "v2"));
+
+        verify(expected, String.format("SELECT %1$s(v1), %1$s(v2), %1$s(v1, v2) FROM %%s", fnName));
+    }
+
+    private void testCount() throws Throwable
+    {
+        // SELECT COUNT does not necessarily include any mappings, but it must always return
+        // a singleton list from getColumnSpecifications() in order for the ResultSet.Metadata
+        // to be constructed correctly:
+        // * COUNT(*) / COUNT(1) do not generate any mappings, as no specific columns are referenced
+        // * COUNT(foo) does generate a mapping from the 'system.count' column spec to foo
+        ColumnSpecification count = columnSpecification("count", LongType.instance);
+        SelectionColumnMapping expected = SelectionColumnMapping.newMapping().addMapping(count, NULL_DEF);
+
+        verify(expected, "SELECT COUNT(*) FROM %s");
+        verify(expected, "SELECT COUNT(1) FROM %s");
+
+        ColumnSpecification aliased = columnSpecification("count_alias", LongType.instance);
+        expected = SelectionColumnMapping.newMapping().addMapping(aliased, NULL_DEF);
+
+        verify(expected, "SELECT COUNT(*) AS count_alias FROM %s");
+        verify(expected, "SELECT COUNT(1) AS count_alias FROM %s");
+
+        ColumnSpecification countV1 = columnSpecification("system.count(v1)", LongType.instance);
+        expected = SelectionColumnMapping.newMapping().addMapping(countV1, columnDefinition("v1"));
+        verify(expected, "SELECT COUNT(v1) FROM %s");
+
+        ColumnSpecification countV1Alias = columnSpecification("count_v1", LongType.instance);
+        expected = SelectionColumnMapping.newMapping().addMapping(countV1Alias, columnDefinition("v1"));
+        verify(expected, "SELECT COUNT(v1) AS count_v1 FROM %s");
+    }
+
+    private void testUserDefinedFunction() throws Throwable
+    {
+        // UDFs are basically represented in the same way as system functions
+        String functionCall = String.format("%s(v1, v2)", functionName);
+        ColumnSpecification fnSpec = columnSpecification(functionCall, Int32Type.instance);
+        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
+                                                                .addMapping(fnSpec, columnDefinitions("v1", "v2"));
+        verify(expected, "SELECT " + functionCall + " FROM %s");
+    }
+
+    private void testFunctionWithAlias() throws Throwable
+    {
+        // a function with an alias is represented in ResultSet.Metadata by a
+        // ColumnSpecification with the alias and the type set to the function's
+        // return type
+        ColumnSpecification fnSpec = columnSpecification("fn_alias", BytesType.instance);
+        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
+                                                                .addMapping(fnSpec, columnDefinition("v1"));
+
+        verify(expected, "SELECT intasblob(v1) AS fn_alias FROM %s");
+    }
+
+    public void testNoArgumentFunction() throws Throwable
+    {
+        SelectionColumns expected = SelectionColumnMapping.newMapping()
+                                                          .addMapping(columnSpecification("system.now()",
+                                                                                          TimeUUIDType.instance),
+                                                                      NULL_DEF);
+        verify(expected, "SELECT now() FROM %s");
+    }
+
+    public void testNestedFunctionsWithArguments() throws Throwable
+    {
+        SelectionColumns expected = SelectionColumnMapping.newMapping()
+                                                          .addMapping(columnSpecification("system.blobasint(system.intasblob(v1))",
+                                                                                          Int32Type.instance),
+                                                                      columnDefinition("v1"));
+        verify(expected, "SELECT blobasint(intasblob(v1)) FROM %s");
+    }
+
+    public void testNestedFunctions() throws Throwable
+    {
+        SelectionColumns expected = SelectionColumnMapping.newMapping()
+                                                          .addMapping(columnSpecification("system.tounixtimestamp(system.now())",
+                                                                                          LongType.instance),
+                                                                      NULL_DEF);
+        verify(expected, "SELECT tounixtimestamp(now()) FROM %s");
+    }
+
+    public void testDuplicateFunctionsWithoutAliases() throws Throwable
+    {
+        // where duplicate functions are present, the ColumnSpecification list will
+        // contain an entry per-duplicate but the mappings will be deduplicated (i.e.
+        // a single mapping k/v pair regardless of the number of duplicates)
+        ColumnSpecification spec = columnSpecification("system.intasblob(v1)", BytesType.instance);
+        SelectionColumns expected = SelectionColumnMapping.newMapping()
+                                                          .addMapping(spec, columnDefinition("v1"))
+                                                          .addMapping(spec, columnDefinition("v1"));
+        verify(expected, "SELECT intasblob(v1), intasblob(v1) FROM %s");
+    }
+
+    public void testDuplicateFunctionsWithAliases() throws Throwable
+    {
+        // where duplicate functions are present with distinct aliases, they are
+        // represented as any other set of distinct columns would be - an entry
+        // in theColumnSpecification list and a separate k/v mapping for each
+        SelectionColumns expected = SelectionColumnMapping.newMapping()
+                                                          .addMapping(columnSpecification("blob_1", BytesType.instance),
+                                                                      columnDefinition("v1"))
+                                                          .addMapping(columnSpecification("blob_2", BytesType.instance),
+                                                                      columnDefinition("v1"));
+        verify(expected, "SELECT intasblob(v1) AS blob_1, intasblob(v1) AS blob_2 FROM %s");
+    }
+
+    public void testSelectDistinct() throws Throwable
+    {
+        SelectionColumns expected = SelectionColumnMapping.newMapping().addMapping(columnSpecification("k",
+                                                                                                       Int32Type.instance),
+                                                                                   columnDefinition("k"));
+        verify(expected, "SELECT DISTINCT k FROM %s");
+
+    }
+
+    private void testMultipleAliasesOnSameColumn() throws Throwable
+    {
+        // Multiple result columns derived from the same underlying column are
+        // represented by ColumnSpecifications
+        ColumnSpecification alias1 = columnSpecification("alias_1", Int32Type.instance);
+        ColumnSpecification alias2 = columnSpecification("alias_2", Int32Type.instance);
+        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
+                                                                .addMapping(alias1, columnDefinition("v1"))
+                                                                .addMapping(alias2, columnDefinition("v1"));
+
+        verify(expected, "SELECT v1 AS alias_1, v1 AS alias_2 FROM %s");
+    }
+
+    private void testMultipleUnaliasedSelectionOfSameColumn() throws Throwable
+    {
+        // simple column identifiers without aliases are represented in
+        // ResultSet.Metadata by the underlying ColumnDefinition
+        SelectionColumns expected = SelectionColumnMapping.newMapping()
+                                                          .addMapping(columnSpecification("v1", Int32Type.instance),
+                                                                      columnDefinition("v1"))
+                                                          .addMapping(columnSpecification("v1", Int32Type.instance),
+                                                                      columnDefinition("v1"));
+
+        verify(expected, "SELECT v1, v1 FROM %s");
+    }
+
+    private void testMixedColumnTypes() throws Throwable
+    {
+        ColumnSpecification kSpec = columnSpecification("k_alias", Int32Type.instance);
+        ColumnSpecification v1Spec = columnSpecification("writetime(v1)", LongType.instance);
+        ColumnSpecification v2Spec = columnSpecification("ttl_alias", Int32Type.instance);
+        ColumnSpecification f1Spec = columnSpecification("v3.f1", Int32Type.instance);
+        ColumnSpecification f2Spec = columnSpecification("f2_alias", UTF8Type.instance);
+        ColumnSpecification f3Spec = columnSpecification("v3", userType);
+
+        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
+                                                                .addMapping(kSpec, columnDefinition("k"))
+                                                                .addMapping(v1Spec, columnDefinition("v1"))
+                                                                .addMapping(v2Spec, columnDefinition("v2"))
+                                                                .addMapping(f1Spec, columnDefinition("v3"))
+                                                                .addMapping(f2Spec, columnDefinition("v3"))
+                                                                .addMapping(f3Spec, columnDefinition("v3"));
+
+
+        verify(expected, "SELECT k AS k_alias," +
+                         "       writetime(v1)," +
+                         "       ttl(v2) as ttl_alias," +
+                         "       v3.f1," +
+                         "       v3.f2 AS f2_alias," +
+                         "       v3" +
+                         " FROM %s");
+    }
+
+    private void testUserDefinedAggregate() throws Throwable
+    {
+        String sFunc = parseFunctionName(createFunction(KEYSPACE, "int",
+                                                        " CREATE FUNCTION %s (a int, b int)" +
+                                                        " RETURNS NULL ON NULL INPUT" +
+                                                        " RETURNS int" +
+                                                        " LANGUAGE javascript" +
+                                                        " AS 'a + b'")).name;
+
+        String aFunc = createAggregate(KEYSPACE, "int, int",
+                                       " CREATE AGGREGATE %s (int)" +
+                                       " SFUNC " + sFunc +
+                                       " STYPE int" +
+                                       " INITCOND 0");
+
+        String plusOne = createFunction(KEYSPACE, "int",
+                                        " CREATE FUNCTION %s (a int)" +
+                                        " RETURNS NULL ON NULL INPUT" +
+                                        " RETURNS int" +
+                                        " LANGUAGE javascript" +
+                                        " AS 'a+1'");
+
+        String sqFunc = createFunction(KEYSPACE, "int",
+                                       " CREATE FUNCTION %s (a int)" +
+                                       " RETURNS NULL ON NULL INPUT" +
+                                       " RETURNS int" +
+                                       " LANGUAGE javascript" +
+                                       " AS 'a*a'");
+
+        ColumnDefinition v1 = columnDefinition("v1");
+        SelectionColumns expected = SelectionColumnMapping.newMapping()
+                                                          .addMapping(columnSpecification(aFunc + "(v1)",
+                                                                                          Int32Type.instance),
+                                                                      v1);
+        verify(expected, String.format("SELECT %s(v1) FROM %%s", aFunc));
+
+        // aggregate with nested udfs as input
+        String specName = String.format("%s(%s(%s(v1)))", aFunc, sqFunc, plusOne);
+        expected = SelectionColumnMapping.newMapping().addMapping(columnSpecification(specName, Int32Type.instance),
+                                                                  v1);
+        verify(expected, String.format("SELECT %s FROM %%s", specName));
+    }
+
+    private void verify(SelectionColumns expected, String query) throws Throwable
+    {
+        SelectStatement statement = getSelect(query);
+        verifyColumnMapping(expected, statement);
+        checkExecution(statement, expected.getColumnSpecifications());
+    }
+
+    private void checkExecution(SelectStatement statement, List<ColumnSpecification> expectedResultColumns)
+    throws RequestExecutionException, RequestValidationException
+    {
+        UntypedResultSet rs = UntypedResultSet.create(statement.executeInternal(QueryState.forInternalCalls(),
+                                                                                QueryOptions.DEFAULT).result);
+
+        assertEquals(expectedResultColumns, rs.one().getColumns());
+    }
+
+    private SelectStatement getSelect(String query) throws RequestValidationException
+    {
+        CQLStatement statement = QueryProcessor.getStatement(String.format(query, KEYSPACE + "." + tableName),
+                                                             ClientState.forInternalCalls()).statement;
+        assertTrue(statement instanceof SelectStatement);
+        return (SelectStatement)statement;
+    }
+
+    private void verifyColumnMapping(SelectionColumns expected, SelectStatement select)
+    {
+        assertEquals(expected, select.getSelection().getColumnMapping());
+    }
+
+    private Iterable<ColumnDefinition> columnDefinitions(String...names)
+    {
+        List<ColumnDefinition> defs = new ArrayList<>();
+        for (String n : names)
+            defs.add(columnDefinition(n));
+        return defs;
+    }
+
+    private ColumnDefinition columnDefinition(String name)
+    {
+        return Schema.instance.getCFMetaData(KEYSPACE, tableName)
+                              .getColumnDefinition(new ColumnIdentifier(name, true));
+
+    }
+
+    private ColumnSpecification columnSpecification(String name, AbstractType<?> type)
+    {
+        return new ColumnSpecification(KEYSPACE,
+                                       tableName,
+                                       new ColumnIdentifier(name, true),
+                                       type);
+    }
+}
diff --git a/test/unit/org/apache/cassandra/cql3/statements/SelectStatementTest.java b/test/unit/org/apache/cassandra/cql3/statements/SelectStatementTest.java
deleted file mode 100644
index 2a1330d..0000000
--- a/test/unit/org/apache/cassandra/cql3/statements/SelectStatementTest.java
+++ /dev/null
@@ -1,965 +0,0 @@
-/*
- * 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.cql3.statements;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.junit.Test;
-
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.cql3.*;
-import org.apache.cassandra.cql3.Term.MultiItemTerminal;
-import org.apache.cassandra.db.ColumnFamilyType;
-import org.apache.cassandra.db.composites.Composite;
-import org.apache.cassandra.db.composites.Composite.EOC;
-import org.apache.cassandra.db.composites.Composites;
-import org.apache.cassandra.db.composites.CompoundSparseCellNameType;
-import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.db.marshal.Int32Type;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-import static java.util.Arrays.asList;
-import static org.junit.Assert.assertEquals;
-
-public class SelectStatementTest
-{
-    @Test
-    public void testBuildBoundWithNoRestrictions() throws InvalidRequestException
-    {
-        Restriction[] restrictions = new Restriction[2];
-
-        List<Composite> bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(1, bounds.size());
-        assertEmptyComposite(bounds.get(0));
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(1, bounds.size());
-        assertEmptyComposite(bounds.get(0));
-    }
-
-    /**
-     * Test 'clustering_0 = 1' with only one clustering column
-     */
-    @Test
-    public void testBuildBoundWithOneEqRestrictionsAndOneClusteringColumn() throws InvalidRequestException
-    {
-        ByteBuffer clustering_0 = ByteBufferUtil.bytes(1);
-        SingleColumnRestriction.EQ eq = new SingleColumnRestriction.EQ(toTerm(clustering_0), false);
-        Restriction[] restrictions = new Restriction[] { eq };
-
-        List<Composite> bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), clustering_0, EOC.START);
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), clustering_0, EOC.END);
-    }
-
-    /**
-     * Test 'clustering_1 = 1' with 2 clustering columns
-     */
-    @Test
-    public void testBuildBoundWithOneEqRestrictionsAndTwoClusteringColumns() throws InvalidRequestException
-    {
-        ByteBuffer clustering_2 = ByteBufferUtil.bytes(1);
-        SingleColumnRestriction.EQ eq = new SingleColumnRestriction.EQ(toTerm(clustering_2), false);
-        Restriction[] restrictions = new Restriction[] { eq, null };
-
-        List<Composite> bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), clustering_2, EOC.START);
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), clustering_2, EOC.END);
-    }
-
-    /**
-     * Test 'clustering_0 IN (1, 2, 3)' with only one clustering column
-     */
-    @Test
-    public void testBuildBoundWithOneInRestrictionsAndOneClusteringColumn() throws InvalidRequestException
-    {
-        ByteBuffer value1 = ByteBufferUtil.bytes(1);
-        ByteBuffer value2 = ByteBufferUtil.bytes(2);
-        ByteBuffer value3 = ByteBufferUtil.bytes(3);
-        SingleColumnRestriction.IN in = new SingleColumnRestriction.InWithValues(toTerms(value1, value2, value3));
-        Restriction[] restrictions = new Restriction[] { in };
-
-        List<Composite> bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(3, bounds.size());
-        assertComposite(bounds.get(0), value1, EOC.START);
-        assertComposite(bounds.get(1), value2, EOC.START);
-        assertComposite(bounds.get(2), value3, EOC.START);
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(3, bounds.size());
-        assertComposite(bounds.get(0), value1, EOC.END);
-        assertComposite(bounds.get(1), value2, EOC.END);
-        assertComposite(bounds.get(2), value3, EOC.END);
-    }
-
-    /**
-     * Test slice restriction (e.g 'clustering_0 > 1') with only one clustering column
-     */
-    @Test
-    public void testBuildBoundWithSliceRestrictionsAndOneClusteringColumn() throws InvalidRequestException
-    {
-        ByteBuffer value1 = ByteBufferUtil.bytes(1);
-        ByteBuffer value2 = ByteBufferUtil.bytes(2);
-
-        SingleColumnRestriction.Slice slice = new SingleColumnRestriction.Slice(false);
-        slice.setBound(Operator.GT, toTerm(value1));
-        Restriction[] restrictions = new Restriction[] { slice };
-
-        List<Composite> bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value1, EOC.END);
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(1, bounds.size());
-        assertEmptyComposite(bounds.get(0));
-
-        slice = new SingleColumnRestriction.Slice(false);
-        slice.setBound(Operator.GTE, toTerm(value1));
-        restrictions = new Restriction[] { slice };
-
-        bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value1, EOC.NONE);
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(1, bounds.size());
-        assertEmptyComposite(bounds.get(0));
-
-        slice = new SingleColumnRestriction.Slice(false);
-        slice.setBound(Operator.LTE, toTerm(value1));
-        restrictions = new Restriction[] { slice };
-
-        bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(1, bounds.size());
-        assertEmptyComposite(bounds.get(0));
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value1, EOC.END);
-
-        slice = new SingleColumnRestriction.Slice(false);
-        slice.setBound(Operator.LT, toTerm(value1));
-        restrictions = new Restriction[] { slice };
-
-        bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(1, bounds.size());
-        assertEmptyComposite(bounds.get(0));
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value1, EOC.START);
-
-        slice = new SingleColumnRestriction.Slice(false);
-        slice.setBound(Operator.GT, toTerm(value1));
-        slice.setBound(Operator.LT, toTerm(value2));
-        restrictions = new Restriction[] { slice };
-
-        bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value1, EOC.END);
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value2, EOC.START);
-
-        slice = new SingleColumnRestriction.Slice(false);
-        slice.setBound(Operator.GTE, toTerm(value1));
-        slice.setBound(Operator.LTE, toTerm(value1));
-        restrictions = new Restriction[] { slice };
-
-        bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value1, EOC.NONE);
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value1, EOC.END);
-    }
-
-    /**
-     * Test 'clustering_0 = 1 AND clustering_1 IN (1, 2, 3)' with two clustering columns
-     */
-    @Test
-    public void testBuildBoundWithEqAndInRestrictions() throws InvalidRequestException
-    {
-        ByteBuffer value1 = ByteBufferUtil.bytes(1);
-        ByteBuffer value2 = ByteBufferUtil.bytes(2);
-        ByteBuffer value3 = ByteBufferUtil.bytes(3);
-        SingleColumnRestriction.EQ eq = new SingleColumnRestriction.EQ(toTerm(value1), false);
-        SingleColumnRestriction.IN in = new SingleColumnRestriction.InWithValues(toTerms(value1, value2, value3));
-        Restriction[] restrictions = new Restriction[] { eq, in };
-
-        List<Composite> bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(3, bounds.size());
-        assertComposite(bounds.get(0), value1, value1, EOC.START);
-        assertComposite(bounds.get(1), value1, value2, EOC.START);
-        assertComposite(bounds.get(2), value1, value3, EOC.START);
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(3, bounds.size());
-        assertComposite(bounds.get(0), value1, value1, EOC.END);
-        assertComposite(bounds.get(1), value1, value2, EOC.END);
-        assertComposite(bounds.get(2), value1, value3, EOC.END);
-    }
-
-    /**
-     * Test slice restriction (e.g 'clustering_0 > 1') with only one clustering column
-     */
-    @Test
-    public void testBuildBoundWithEqAndSliceRestrictions() throws InvalidRequestException
-    {
-        ByteBuffer value1 = ByteBufferUtil.bytes(1);
-        ByteBuffer value2 = ByteBufferUtil.bytes(2);
-        ByteBuffer value3 = ByteBufferUtil.bytes(3);
-
-        SingleColumnRestriction.EQ eq = new SingleColumnRestriction.EQ(toTerm(value3), false);
-
-        SingleColumnRestriction.Slice slice = new SingleColumnRestriction.Slice(false);
-        slice.setBound(Operator.GT, toTerm(value1));
-        Restriction[] restrictions = new Restriction[] { eq, slice };
-
-        List<Composite> bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value3, value1, EOC.END);
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value3, EOC.END);
-
-        slice = new SingleColumnRestriction.Slice(false);
-        slice.setBound(Operator.GTE, toTerm(value1));
-        restrictions = new Restriction[] { eq, slice };
-
-        bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value3, value1, EOC.NONE);
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value3, EOC.END);
-
-        slice = new SingleColumnRestriction.Slice(false);
-        slice.setBound(Operator.LTE, toTerm(value1));
-        restrictions = new Restriction[] { eq, slice };
-
-        bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value3, EOC.START);
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value3, value1, EOC.END);
-
-        slice = new SingleColumnRestriction.Slice(false);
-        slice.setBound(Operator.LT, toTerm(value1));
-        restrictions = new Restriction[] { eq, slice };
-
-        bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value3, EOC.START);
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value3, value1, EOC.START);
-
-        slice = new SingleColumnRestriction.Slice(false);
-        slice.setBound(Operator.GT, toTerm(value1));
-        slice.setBound(Operator.LT, toTerm(value2));
-        restrictions = new Restriction[] { eq, slice };
-
-        bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value3, value1, EOC.END);
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value3, value2, EOC.START);
-
-        slice = new SingleColumnRestriction.Slice(false);
-        slice.setBound(Operator.GTE, toTerm(value1));
-        slice.setBound(Operator.LTE, toTerm(value1));
-        restrictions = new Restriction[] { eq, slice };
-
-        bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value3, value1, EOC.NONE);
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value3, value1, EOC.END);
-    }
-
-    /**
-     * Test '(clustering_0, clustering_1) = (1, 2)' with two clustering column
-     */
-    @Test
-    public void testBuildBoundWithMultiEqRestrictions() throws InvalidRequestException
-    {
-        ByteBuffer value1 = ByteBufferUtil.bytes(1);
-        ByteBuffer value2 = ByteBufferUtil.bytes(2);
-        MultiColumnRestriction.EQ eq = new MultiColumnRestriction.EQ(toMultiItemTerminal(value1, value2), false);
-        Restriction[] restrictions = new Restriction[] { eq, eq };
-
-        List<Composite> bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value1, value2, EOC.START);
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value1, value2, EOC.END);
-    }
-
-    /**
-     * Test '(clustering_0, clustering_1) IN ((1, 2), (2, 3))' with two clustering column
-     */
-    @Test
-    public void testBuildBoundWithMultiInRestrictions() throws InvalidRequestException
-    {
-        ByteBuffer value1 = ByteBufferUtil.bytes(1);
-        ByteBuffer value2 = ByteBufferUtil.bytes(2);
-        ByteBuffer value3 = ByteBufferUtil.bytes(3);
-        List<MultiItemTerminal> terms = asList(toMultiItemTerminal(value1, value2), toMultiItemTerminal(value2, value3));
-        MultiColumnRestriction.IN in = new MultiColumnRestriction.InWithValues(terms);
-        Restriction[] restrictions = new Restriction[] { in, in };
-
-        List<Composite> bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(2, bounds.size());
-        assertComposite(bounds.get(0), value1, value2, EOC.START);
-        assertComposite(bounds.get(1), value2, value3, EOC.START);
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(2, bounds.size());
-        assertComposite(bounds.get(0), value1, value2, EOC.END);
-        assertComposite(bounds.get(1), value2, value3, EOC.END);
-    }
-
-    /**
-     * Test multi-column slice restrictions (e.g '(clustering_0) > (1)') with only one clustering column
-     */
-    @Test
-    public void testBuildBoundWithMultiSliceRestrictionsWithOneClusteringColumn() throws InvalidRequestException
-    {
-        ByteBuffer value1 = ByteBufferUtil.bytes(1);
-        ByteBuffer value2 = ByteBufferUtil.bytes(2);
-
-        MultiColumnRestriction.Slice slice = new MultiColumnRestriction.Slice(false);
-        slice.setBound(Operator.GT, toMultiItemTerminal(value1));
-        Restriction[] restrictions = new Restriction[] { slice };
-
-        List<Composite> bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value1, EOC.END);
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(1, bounds.size());
-        assertEmptyComposite(bounds.get(0));
-
-        slice = new MultiColumnRestriction.Slice(false);
-        slice.setBound(Operator.GTE, toMultiItemTerminal(value1));
-        restrictions = new Restriction[] { slice };
-
-        bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value1, EOC.NONE);
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(1, bounds.size());
-        assertEmptyComposite(bounds.get(0));
-
-        slice = new MultiColumnRestriction.Slice(false);
-        slice.setBound(Operator.LTE, toMultiItemTerminal(value1));
-        restrictions = new Restriction[] { slice };
-
-        bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(1, bounds.size());
-        assertEmptyComposite(bounds.get(0));
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value1, EOC.END);
-
-        slice = new MultiColumnRestriction.Slice(false);
-        slice.setBound(Operator.LT, toMultiItemTerminal(value1));
-        restrictions = new Restriction[] { slice };
-
-        bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(1, bounds.size());
-        assertEmptyComposite(bounds.get(0));
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value1, EOC.START);
-
-        slice = new MultiColumnRestriction.Slice(false);
-        slice.setBound(Operator.GT, toMultiItemTerminal(value1));
-        slice.setBound(Operator.LT, toMultiItemTerminal(value2));
-        restrictions = new Restriction[] { slice };
-
-        bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value1, EOC.END);
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value2, EOC.START);
-
-        slice = new MultiColumnRestriction.Slice(false);
-        slice.setBound(Operator.GTE, toMultiItemTerminal(value1));
-        slice.setBound(Operator.LTE, toMultiItemTerminal(value2));
-        restrictions = new Restriction[] { slice };
-
-        bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value1, EOC.NONE);
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value2, EOC.END);
-    }
-
-    /**
-     * Test multi-column slice restrictions (e.g '(clustering_0, clustering_1) > (1, 2)') with only one clustering
-     * column
-     */
-    @Test
-    public void testBuildBoundWithMultiSliceRestrictionsWithTwoClusteringColumn() throws InvalidRequestException
-    {
-        ByteBuffer value1 = ByteBufferUtil.bytes(1);
-        ByteBuffer value2 = ByteBufferUtil.bytes(2);
-
-        // (clustering_0, clustering1) > (1, 2)
-        MultiColumnRestriction.Slice slice = new MultiColumnRestriction.Slice(false);
-        slice.setBound(Operator.GT, toMultiItemTerminal(value1, value2));
-        Restriction[] restrictions = new Restriction[] { slice, slice };
-
-        List<Composite> bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value1, value2, EOC.END);
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(1, bounds.size());
-        assertEmptyComposite(bounds.get(0));
-
-        // (clustering_0, clustering1) >= (1, 2)
-        slice = new MultiColumnRestriction.Slice(false);
-        slice.setBound(Operator.GTE, toMultiItemTerminal(value1, value2));
-        restrictions = new Restriction[] { slice, slice };
-
-        bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value1, value2, EOC.NONE);
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(1, bounds.size());
-        assertEmptyComposite(bounds.get(0));
-
-        // (clustering_0, clustering1) <= (1, 2)
-        slice = new MultiColumnRestriction.Slice(false);
-        slice.setBound(Operator.LTE, toMultiItemTerminal(value1, value2));
-        restrictions = new Restriction[] { slice, slice };
-
-        bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(1, bounds.size());
-        assertEmptyComposite(bounds.get(0));
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value1, value2, EOC.END);
-
-        // (clustering_0, clustering1) < (1, 2)
-        slice = new MultiColumnRestriction.Slice(false);
-        slice.setBound(Operator.LT, toMultiItemTerminal(value1, value2));
-        restrictions = new Restriction[] { slice, slice };
-
-        bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(1, bounds.size());
-        assertEmptyComposite(bounds.get(0));
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value1, value2, EOC.START);
-
-        // (clustering_0, clustering1) > (1, 2) AND (clustering_0) < (2)
-        slice = new MultiColumnRestriction.Slice(false);
-        slice.setBound(Operator.GT, toMultiItemTerminal(value1, value2));
-        slice.setBound(Operator.LT, toMultiItemTerminal(value2));
-        restrictions = new Restriction[] { slice, slice };
-
-        bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value1, value2, EOC.END);
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value2, EOC.START);
-
-        // (clustering_0, clustering1) >= (1, 2) AND (clustering_0, clustering1) <= (2, 1)
-        slice = new MultiColumnRestriction.Slice(false);
-        slice.setBound(Operator.GTE, toMultiItemTerminal(value1, value2));
-        slice.setBound(Operator.LTE, toMultiItemTerminal(value2, value1));
-        restrictions = new Restriction[] { slice, slice };
-
-        bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value1, value2, EOC.NONE);
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value2, value1, EOC.END);
-    }
-
-    /**
-     * Test mixing single and multi equals restrictions (e.g. clustering_0 = 1 AND (clustering_1, clustering_2) = (2, 3))
-     */
-    @Test
-    public void testBuildBoundWithSingleEqAndMultiEqRestrictions() throws InvalidRequestException
-    {
-        ByteBuffer value1 = ByteBufferUtil.bytes(1);
-        ByteBuffer value2 = ByteBufferUtil.bytes(2);
-        ByteBuffer value3 = ByteBufferUtil.bytes(3);
-        ByteBuffer value4 = ByteBufferUtil.bytes(4);
-
-        // clustering_0 = 1 AND (clustering_1, clustering_2) = (2, 3)
-        SingleColumnRestriction.EQ singleEq = new SingleColumnRestriction.EQ(toTerm(value1), false);
-        MultiColumnRestriction.EQ multiEq = new MultiColumnRestriction.EQ(toMultiItemTerminal(value2, value3), false);
-        Restriction[] restrictions = new Restriction[] { singleEq, multiEq, multiEq };
-
-        List<Composite> bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value1, value2, value3, EOC.START);
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value1, value2, value3, EOC.END);
-
-        // clustering_0 = 1 AND clustering_1 = 2 AND (clustering_2, clustering_3) = (3, 4)
-        singleEq = new SingleColumnRestriction.EQ(toTerm(value1), false);
-        SingleColumnRestriction.EQ singleEq2 = new SingleColumnRestriction.EQ(toTerm(value2), false);
-        multiEq = new MultiColumnRestriction.EQ(toMultiItemTerminal(value3, value4), false);
-        restrictions = new Restriction[] { singleEq, singleEq2, multiEq, multiEq };
-
-        bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value1, value2, value3, value4, EOC.START);
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value1, value2, value3, value4, EOC.END);
-
-        // (clustering_0, clustering_1) = (1, 2) AND clustering_2 = 3
-        singleEq = new SingleColumnRestriction.EQ(toTerm(value3), false);
-        multiEq = new MultiColumnRestriction.EQ(toMultiItemTerminal(value1, value2), false);
-        restrictions = new Restriction[] { multiEq, multiEq, singleEq };
-
-        bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value1, value2, value3, EOC.START);
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value1, value2, value3, EOC.END);
-
-        // clustering_0 = 1 AND (clustering_1, clustering_2) = (2, 3) AND clustering_3 = 4
-        singleEq = new SingleColumnRestriction.EQ(toTerm(value1), false);
-        singleEq2 = new SingleColumnRestriction.EQ(toTerm(value4), false);
-        multiEq = new MultiColumnRestriction.EQ(toMultiItemTerminal(value2, value3), false);
-        restrictions = new Restriction[] { singleEq, multiEq, multiEq, singleEq2 };
-
-        bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value1, value2, value3, value4, EOC.START);
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value1, value2, value3, value4, EOC.END);
-    }
-
-    /**
-     * Test clustering_0 = 1 AND (clustering_1, clustering_2) IN ((2, 3), (4, 5))
-     */
-    @Test
-    public void testBuildBoundWithSingleEqAndMultiINRestrictions() throws InvalidRequestException
-    {
-        ByteBuffer value1 = ByteBufferUtil.bytes(1);
-        ByteBuffer value2 = ByteBufferUtil.bytes(2);
-        ByteBuffer value3 = ByteBufferUtil.bytes(3);
-        ByteBuffer value4 = ByteBufferUtil.bytes(4);
-        ByteBuffer value5 = ByteBufferUtil.bytes(5);
-
-        // clustering_0 = 1 AND (clustering_1, clustering_2) IN ((2, 3), (4, 5))
-        SingleColumnRestriction.EQ singleEq = new SingleColumnRestriction.EQ(toTerm(value1), false);
-        MultiColumnRestriction.IN multiIn =
-                new MultiColumnRestriction.InWithValues(asList(toMultiItemTerminal(value2, value3),
-                                                               toMultiItemTerminal(value4, value5)));
-
-        Restriction[] restrictions = new Restriction[] { singleEq, multiIn, multiIn };
-
-        List<Composite> bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(2, bounds.size());
-        assertComposite(bounds.get(0), value1, value2, value3, EOC.START);
-        assertComposite(bounds.get(1), value1, value4, value5, EOC.START);
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(2, bounds.size());
-        assertComposite(bounds.get(0), value1, value2, value3, EOC.END);
-        assertComposite(bounds.get(1), value1, value4, value5, EOC.END);
-
-        // clustering_0 = 1 AND (clustering_1, clustering_2) IN ((2, 3))
-        singleEq = new SingleColumnRestriction.EQ(toTerm(value1), false);
-        multiIn = new MultiColumnRestriction.InWithValues(asList(toMultiItemTerminal(value2, value3),
-                                                                 toMultiItemTerminal(value4, value5)));
-
-        restrictions = new Restriction[] { singleEq, multiIn, multiIn };
-
-        bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(2, bounds.size());
-        assertComposite(bounds.get(0), value1, value2, value3, EOC.START);
-        assertComposite(bounds.get(1), value1, value4, value5, EOC.START);
-
-        // clustering_0 = 1 AND clustering_1 = 5 AND (clustering_2, clustering_3) IN ((2, 3), (4, 5))
-        singleEq = new SingleColumnRestriction.EQ(toTerm(value1), false);
-        SingleColumnRestriction.EQ singleEq2 = new SingleColumnRestriction.EQ(toTerm(value5), false);
-        multiIn = new MultiColumnRestriction.InWithValues(asList(toMultiItemTerminal(value2, value3),
-                                                                 toMultiItemTerminal(value4, value5)));
-
-        restrictions = new Restriction[] { singleEq, singleEq2, multiIn, multiIn };
-
-        bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(2, bounds.size());
-        assertComposite(bounds.get(0), value1, value5, value2, value3, EOC.START);
-        assertComposite(bounds.get(1), value1, value5, value4, value5, EOC.START);
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(2, bounds.size());
-        assertComposite(bounds.get(0), value1, value5, value2, value3, EOC.END);
-        assertComposite(bounds.get(1), value1, value5, value4, value5, EOC.END);
-    }
-
-    /**
-     * Test mixing single equal restrictions with multi-column slice restrictions
-     * (e.g. clustering_0 = 1 AND (clustering_1, clustering_2) > (2, 3))
-     */
-    @Test
-    public void testBuildBoundWithSingleEqAndSliceRestrictions() throws InvalidRequestException
-    {
-        ByteBuffer value1 = ByteBufferUtil.bytes(1);
-        ByteBuffer value2 = ByteBufferUtil.bytes(2);
-        ByteBuffer value3 = ByteBufferUtil.bytes(3);
-        ByteBuffer value4 = ByteBufferUtil.bytes(4);
-        ByteBuffer value5 = ByteBufferUtil.bytes(5);
-
-        // clustering_0 = 1 AND (clustering_1, clustering_2) > (2, 3)
-        SingleColumnRestriction.EQ singleEq = new SingleColumnRestriction.EQ(toTerm(value1), false);
-        MultiColumnRestriction.Slice multiSlice = new MultiColumnRestriction.Slice(false);
-        multiSlice.setBound(Operator.GT, toMultiItemTerminal(value2, value3));
-
-        Restriction[] restrictions = new Restriction[] { singleEq, multiSlice, multiSlice };
-
-        List<Composite> bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value1, value2, value3, EOC.END);
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value1, EOC.END);
-
-        // clustering_0 = 1 AND (clustering_1, clustering_2) > (2, 3) AND (clustering_1) < (4)
-        singleEq = new SingleColumnRestriction.EQ(toTerm(value1), false);
-        multiSlice = new MultiColumnRestriction.Slice(false);
-        multiSlice.setBound(Operator.GT, toMultiItemTerminal(value2, value3));
-        multiSlice.setBound(Operator.LT, toMultiItemTerminal(value4));
-
-        restrictions = new Restriction[] { singleEq, multiSlice, multiSlice };
-
-        bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value1, value2, value3, EOC.END);
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value1, value4, EOC.START);
-
-        // clustering_0 = 1 AND (clustering_1, clustering_2) => (2, 3) AND (clustering_1, clustering_2) <= (4, 5)
-        singleEq = new SingleColumnRestriction.EQ(toTerm(value1), false);
-        multiSlice = new MultiColumnRestriction.Slice(false);
-        multiSlice.setBound(Operator.GTE, toMultiItemTerminal(value2, value3));
-        multiSlice.setBound(Operator.LTE, toMultiItemTerminal(value4, value5));
-
-        restrictions = new Restriction[] { singleEq, multiSlice, multiSlice };
-
-        bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value1, value2, value3, EOC.NONE);
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value1, value4, value5, EOC.END);
-    }
-
-    /**
-     * Test mixing multi equal restrictions with single-column slice restrictions
-     * (e.g. clustering_0 = 1 AND (clustering_1, clustering_2) > (2, 3))
-     */
-    @Test
-    public void testBuildBoundWithMultiEqAndSingleSliceRestrictions() throws InvalidRequestException
-    {
-        ByteBuffer value1 = ByteBufferUtil.bytes(1);
-        ByteBuffer value2 = ByteBufferUtil.bytes(2);
-        ByteBuffer value3 = ByteBufferUtil.bytes(3);
-
-        // (clustering_0, clustering_1) = (1, 2) AND clustering_2 > 3
-        MultiColumnRestriction.EQ multiEq = new MultiColumnRestriction.EQ(toMultiItemTerminal(value1, value2), false);
-        SingleColumnRestriction.Slice singleSlice = new SingleColumnRestriction.Slice(false);
-        singleSlice.setBound(Operator.GT, toTerm(value3));
-
-        Restriction[] restrictions = new Restriction[] { multiEq, multiEq, singleSlice };
-
-        List<Composite> bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value1, value2, value3, EOC.END);
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0),  value1, value2, EOC.END);
-    }
-
-    @Test
-    public void testBuildBoundWithSeveralMultiColumnRestrictions() throws InvalidRequestException
-    {
-        ByteBuffer value1 = ByteBufferUtil.bytes(1);
-        ByteBuffer value2 = ByteBufferUtil.bytes(2);
-        ByteBuffer value3 = ByteBufferUtil.bytes(3);
-        ByteBuffer value4 = ByteBufferUtil.bytes(4);
-        ByteBuffer value5 = ByteBufferUtil.bytes(5);
-
-        // (clustering_0, clustering_1) = (1, 2) AND (clustering_2, clustering_3) > (3, 4)
-        MultiColumnRestriction.EQ multiEq = new MultiColumnRestriction.EQ(toMultiItemTerminal(value1, value2), false);
-        MultiColumnRestriction.Slice multiSlice = new MultiColumnRestriction.Slice(false);
-        multiSlice.setBound(Operator.GT, toMultiItemTerminal(value3, value4));
-
-        Restriction[] restrictions = new Restriction[] { multiEq, multiEq, multiSlice, multiSlice};
-
-        List<Composite> bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value1, value2, value3, value4, EOC.END);
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0),  value1, value2, EOC.END);
-
-        // (clustering_0, clustering_1) = (1, 2) AND (clustering_2, clustering_3) IN ((3, 4), (4, 5))
-        multiEq = new MultiColumnRestriction.EQ(toMultiItemTerminal(value1, value2), false);
-        MultiColumnRestriction.IN multiIn =
-                new MultiColumnRestriction.InWithValues(asList(toMultiItemTerminal(value3, value4),
-                                                               toMultiItemTerminal(value4, value5)));
-
-        restrictions = new Restriction[] { multiEq, multiEq, multiIn, multiIn};
-
-        bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(2, bounds.size());
-        assertComposite(bounds.get(0), value1, value2, value3, value4, EOC.START);
-        assertComposite(bounds.get(1), value1, value2, value4, value5, EOC.START);
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(2, bounds.size());
-        assertComposite(bounds.get(0), value1, value2, value3, value4, EOC.END);
-        assertComposite(bounds.get(1), value1, value2, value4, value5, EOC.END);
-
-        // (clustering_0, clustering_1) = (1, 2) AND (clustering_2, clustering_3) IN ((3, 4), (4, 5))
-        multiEq = new MultiColumnRestriction.EQ(toMultiItemTerminal(value1, value2), false);
-        MultiColumnRestriction.EQ multiEq2 = new MultiColumnRestriction.EQ(toMultiItemTerminal(value3, value4), false);
-
-        restrictions = new Restriction[] { multiEq, multiEq, multiEq2, multiEq2};
-
-        bounds = executeBuildBound(restrictions, Bound.START);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value1, value2, value3, value4, EOC.START);
-
-        bounds = executeBuildBound(restrictions, Bound.END);
-        assertEquals(1, bounds.size());
-        assertComposite(bounds.get(0), value1, value2, value3, value4, EOC.END);
-    }
-
-    /**
-     * Asserts that the specified <code>Composite</code> is an empty one.
-     *
-     * @param composite the composite to check
-     */
-    private static void assertEmptyComposite(Composite composite)
-    {
-        assertEquals(Composites.EMPTY, composite);
-    }
-
-    /**
-     * Asserts that the specified <code>Composite</code> contains the specified element and the specified EOC.
-     *
-     * @param composite the composite to check
-     * @param element the expected element of the composite
-     * @param eoc the expected EOC of the composite
-     */
-    private static void assertComposite(Composite composite, ByteBuffer element, EOC eoc)
-    {
-        assertComposite(composite, eoc, element);
-    }
-
-    /**
-     * Asserts that the specified <code>Composite</code> contains the 2 specified element and the specified EOC.
-     *
-     * @param composite the composite to check
-     * @param eoc the expected EOC of the composite
-     * @param elements the expected element of the composite
-     */
-    private static void assertComposite(Composite composite, ByteBuffer firstElement, ByteBuffer secondElement, EOC eoc)
-    {
-        assertComposite(composite, eoc, firstElement, secondElement);
-    }
-
-    /**
-     * Asserts that the specified <code>Composite</code> contains the 3 specified element and the specified EOC.
-     *
-     * @param composite the composite to check
-     * @param firstElement the first expected element of the composite
-     * @param secondElement the second expected element of the composite
-     * @param thirdElement the third expected element of the composite
-     * @param eoc the expected EOC of the composite
-     * @param elements the expected element of the composite
-     */
-    private static void assertComposite(Composite composite,
-                                        ByteBuffer firstElement,
-                                        ByteBuffer secondElement,
-                                        ByteBuffer thirdElement,
-                                        EOC eoc)
-    {
-        assertComposite(composite, eoc, firstElement, secondElement, thirdElement);
-    }
-
-    /**
-     * Asserts that the specified <code>Composite</code> contains the 4 specified element and the specified EOC.
-     *
-     * @param composite the composite to check
-     * @param firstElement the first expected element of the composite
-     * @param secondElement the second expected element of the composite
-     * @param thirdElement the third expected element of the composite
-     * @param fourthElement the fourth expected element of the composite
-     * @param eoc the expected EOC of the composite
-     * @param elements the expected element of the composite
-     */
-    private static void assertComposite(Composite composite,
-                                        ByteBuffer firstElement,
-                                        ByteBuffer secondElement,
-                                        ByteBuffer thirdElement,
-                                        ByteBuffer fourthElement,
-                                        EOC eoc)
-    {
-        assertComposite(composite, eoc, firstElement, secondElement, thirdElement, fourthElement);
-    }
-
-    /**
-     * Asserts that the specified <code>Composite</code> contains the specified elements and EOC.
-     *
-     * @param composite the composite to check
-     * @param eoc the expected EOC of the composite
-     * @param elements the expected elements of the composite
-     */
-    private static void assertComposite(Composite composite, EOC eoc, ByteBuffer... elements)
-    {
-        assertEquals("the composite size is not the expected one:", elements.length, composite.size());
-        for (int i = 0, m = elements.length; i < m; i++)
-        {
-            ByteBuffer element = elements[i];
-            assertEquals("the element " + i + " of the composite is not the expected one:", element, composite.get(i));
-        }
-        assertEquals("the EOC of the composite is not the expected one:", eoc, composite.eoc());
-    }
-
-    /**
-     * Calls the <code>SelectStatement.buildBound</code> with the specified restrictions.
-     *
-     * @param restrictions the restrictions
-     * @return the result from the method call to <code>SelectStatement.buildBound</code>
-     * @throws InvalidRequestException if the method call throw an exception
-     */
-    private static List<Composite> executeBuildBound(Restriction[] restrictions,
-                                                     Bound bound) throws InvalidRequestException
-    {
-        List<AbstractType<?>> types = new ArrayList<>();
-
-        for (int i = 0, m = restrictions.length; i < m; i++)
-            types.add(Int32Type.instance);
-
-        CompoundSparseCellNameType cType = new CompoundSparseCellNameType(types);
-        CFMetaData cfMetaData = new CFMetaData("keyspace", "test", ColumnFamilyType.Standard, cType);
-
-        List<ColumnDefinition> columnDefs = new ArrayList<>();
-        for (int i = 0, m = restrictions.length; i < m; i++)
-        {
-            ByteBuffer name = ByteBufferUtil.bytes("clustering_" + i);
-            columnDefs.add(ColumnDefinition.clusteringKeyDef(cfMetaData, name, types.get(i), i));
-        }
-
-        return SelectStatement.buildBound(bound, columnDefs, restrictions, false, cType, QueryOptions.DEFAULT);
-    }
-
-    /**
-     * Converts the specified values into a <code>MultiItemTerminal</code>.
-     *
-     * @param values the values to convert.
-     * @return the term corresponding to the specified values.
-     */
-    private static MultiItemTerminal toMultiItemTerminal(ByteBuffer... values)
-    {
-        return new Tuples.Value(values);
-    }
-
-    /**
-     * Converts the specified value into a term.
-     *
-     * @param value the value to convert.
-     * @return the term corresponding to the specified value.
-     */
-    private static Term toTerm(ByteBuffer value)
-    {
-        return new Constants.Value(value);
-    }
-
-    /**
-     * Converts the specified values into a <code>List</code> of terms.
-     *
-     * @param values the values to convert.
-     * @return a <code>List</code> of terms corresponding to the specified values.
-     */
-    private static List<Term> toTerms(ByteBuffer... values)
-    {
-        List<Term> terms = new ArrayList<>();
-        for (ByteBuffer value : values)
-            terms.add(toTerm(value));
-        return terms;
-    }
-}
diff --git a/test/unit/org/apache/cassandra/cql3/statements/SelectionColumnMappingTest.java b/test/unit/org/apache/cassandra/cql3/statements/SelectionColumnMappingTest.java
deleted file mode 100644
index 141291f..0000000
--- a/test/unit/org/apache/cassandra/cql3/statements/SelectionColumnMappingTest.java
+++ /dev/null
@@ -1,403 +0,0 @@
-/*
- *
- * 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.cql3.statements;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.cql3.*;
-import org.apache.cassandra.db.marshal.*;
-import org.apache.cassandra.dht.ByteOrderedPartitioner;
-import org.apache.cassandra.exceptions.RequestExecutionException;
-import org.apache.cassandra.exceptions.RequestValidationException;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.QueryState;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-public class SelectionColumnMappingTest extends CQLTester
-{
-    private static final ColumnDefinition NULL_DEF = null;
-
-    String tableName;
-    String typeName;
-
-    @BeforeClass
-    public static void setUpClass()
-    {
-        DatabaseDescriptor.setPartitioner(new ByteOrderedPartitioner());
-    }
-
-    @Test
-    public void testSelectionColumnMapping() throws Throwable
-    {
-        // Organised as a single test to avoid the overhead of
-        // table creation for each variant
-        typeName = createType("CREATE TYPE %s (f1 int, f2 text)");
-        tableName = createTable("CREATE TABLE %s (" +
-                                    " k int PRIMARY KEY," +
-                                    " v1 int," +
-                                    " v2 ascii," +
-                                    " v3 frozen<" + typeName + ">)");
-        execute("INSERT INTO %s (k, v1 ,v2, v3) VALUES (1, 1, 'foo', {f1:1, f2:'bar'})");
-
-        testSimpleTypes();
-        testWildcard();
-        testSimpleTypesWithAliases();
-        testUserTypes();
-        testUserTypesWithAliases();
-        testWritetimeAndTTL();
-        testWritetimeAndTTLWithAliases();
-        testFunction();
-        testFunctionWithAlias();
-        testNoArgumentFunction();
-        testNestedFunctions();
-        testNestedFunctionsWithArguments();
-        testCount();
-        testDuplicateFunctionsWithoutAliases();
-        testDuplicateFunctionsWithAliases();
-        testSelectDistinct();
-        testMultipleAliasesOnSameColumn();
-        testMixedColumnTypes();
-        testMultipleUnaliasedSelectionOfSameColumn();
-    }
-
-    @Test
-    public void testMultipleArgumentFunction() throws Throwable
-    {
-        // token() is currently the only function which accepts multiple arguments
-        tableName = createTable("CREATE TABLE %s (a int, b text, PRIMARY KEY ((a, b)))");
-        ColumnSpecification tokenSpec = columnSpecification("token(a, b)", BytesType.instance);
-        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
-                                                                .addMapping(tokenSpec, columnDefinitions("a", "b"));
-
-        // we don't use verify like with the other tests because this query will produce no results
-        SelectStatement statement = getSelect("SELECT token(a,b) FROM %s");
-        verifyColumnMapping(expected, statement);
-        statement.executeInternal(QueryState.forInternalCalls(), QueryOptions.DEFAULT);
-    }
-
-    private void testSimpleTypes() throws Throwable
-    {
-        // simple column identifiers without aliases are represented in
-        // ResultSet.Metadata by the underlying ColumnDefinition
-        ColumnDefinition kDef = columnDefinition("k");
-        ColumnDefinition v1Def = columnDefinition("v1");
-        ColumnDefinition v2Def = columnDefinition("v2");
-        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
-                                                                .addMapping(kDef, columnDefinition("k"))
-                                                                .addMapping(v1Def, columnDefinition("v1"))
-                                                                .addMapping(v2Def, columnDefinition("v2"));
-
-        verify(expected, "SELECT k, v1, v2 FROM %s");
-    }
-
-    private void testWildcard() throws Throwable
-    {
-        // Wildcard select should behave just as though we had
-        // explicitly selected each column
-        ColumnDefinition kDef = columnDefinition("k");
-        ColumnDefinition v1Def = columnDefinition("v1");
-        ColumnDefinition v2Def = columnDefinition("v2");
-        ColumnDefinition v3Def = columnDefinition("v3");
-        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
-                                                                .addMapping(kDef, columnDefinition("k"))
-                                                                .addMapping(v1Def, columnDefinition("v1"))
-                                                                .addMapping(v2Def, columnDefinition("v2"))
-                                                                .addMapping(v3Def, columnDefinition("v3"));
-
-        verify(expected, "SELECT * FROM %s");
-    }
-
-    private void testSimpleTypesWithAliases() throws Throwable
-    {
-        // simple column identifiers with aliases are represented in ResultSet.Metadata
-        // by a ColumnSpecification based on the underlying ColumnDefinition
-        ColumnSpecification kSpec = columnSpecification("k_alias", Int32Type.instance);
-        ColumnSpecification v1Spec = columnSpecification("v1_alias", Int32Type.instance);
-        ColumnSpecification v2Spec = columnSpecification("v2_alias", AsciiType.instance);
-        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
-                                                                .addMapping(kSpec, columnDefinition("k"))
-                                                                .addMapping(v1Spec, columnDefinition("v1"))
-                                                                .addMapping(v2Spec, columnDefinition("v2"));
-
-        verify(expected, "SELECT k AS k_alias, v1 AS v1_alias, v2 AS v2_alias FROM %s");
-    }
-
-    private void testUserTypes() throws Throwable
-    {
-        // User type fields are represented in ResultSet.Metadata by a
-        // ColumnSpecification denoting the name and type of the particular field
-        ColumnSpecification f1Spec = columnSpecification("v3.f1", Int32Type.instance);
-        ColumnSpecification f2Spec = columnSpecification("v3.f2", UTF8Type.instance);
-        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
-                                                                .addMapping(f1Spec, columnDefinition("v3"))
-                                                                .addMapping(f2Spec, columnDefinition("v3"));
-
-        verify(expected, "SELECT v3.f1, v3.f2 FROM %s");
-    }
-
-    private void testUserTypesWithAliases() throws Throwable
-    {
-        // User type fields with aliases are represented in ResultSet.Metadata
-        // by a ColumnSpecification with the alias name and the type of the actual field
-        ColumnSpecification f1Spec = columnSpecification("f1_alias", Int32Type.instance);
-        ColumnSpecification f2Spec = columnSpecification("f2_alias", UTF8Type.instance);
-        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
-                                                                .addMapping(f1Spec, columnDefinition("v3"))
-                                                                .addMapping(f2Spec, columnDefinition("v3"));
-
-        verify(expected, "SELECT v3.f1 AS f1_alias, v3.f2 AS f2_alias FROM %s");
-    }
-
-    private void testWritetimeAndTTL() throws Throwable
-    {
-        // writetime and ttl are represented in ResultSet.Metadata by a ColumnSpecification
-        // with the function name plus argument and a long or int type respectively
-        ColumnSpecification wtSpec = columnSpecification("writetime(v1)", LongType.instance);
-        ColumnSpecification ttlSpec = columnSpecification("ttl(v2)", Int32Type.instance);
-        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
-                                                                .addMapping(wtSpec, columnDefinition("v1"))
-                                                                .addMapping(ttlSpec, columnDefinition("v2"));
-
-        verify(expected, "SELECT writetime(v1), ttl(v2) FROM %s");
-    }
-
-    private void testWritetimeAndTTLWithAliases() throws Throwable
-    {
-        // writetime and ttl with aliases are represented in ResultSet.Metadata
-        // by a ColumnSpecification with the alias name and the appropriate numeric type
-        ColumnSpecification wtSpec = columnSpecification("wt_alias", LongType.instance);
-        ColumnSpecification ttlSpec = columnSpecification("ttl_alias", Int32Type.instance);
-        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
-                                                                .addMapping(wtSpec, columnDefinition("v1"))
-                                                                .addMapping(ttlSpec, columnDefinition("v2"));
-
-        verify(expected, "SELECT writetime(v1) AS wt_alias, ttl(v2) AS ttl_alias FROM %s");
-    }
-
-    private void testFunction() throws Throwable
-    {
-        // a function such as intasblob(<col>) is represented in ResultSet.Metadata
-        // by a ColumnSpecification with the function name plus args and the type set
-        // to the function's return type
-        ColumnSpecification fnSpec = columnSpecification("intasblob(v1)", BytesType.instance);
-        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
-                                                                .addMapping(fnSpec, columnDefinition("v1"));
-
-        verify(expected, "SELECT intasblob(v1) FROM %s");
-    }
-
-    private void testFunctionWithAlias() throws Throwable
-    {
-        // a function with an alias is represented in ResultSet.Metadata by a
-        // ColumnSpecification with the alias and the type set to the function's
-        // return type
-        ColumnSpecification fnSpec = columnSpecification("fn_alias", BytesType.instance);
-        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
-                                                                .addMapping(fnSpec, columnDefinition("v1"));
-
-        verify(expected, "SELECT intasblob(v1) AS fn_alias FROM %s");
-    }
-
-    public void testNoArgumentFunction() throws Throwable
-    {
-        SelectionColumns expected = SelectionColumnMapping.newMapping()
-                                                          .addMapping(columnSpecification("now()",
-                                                                                          TimeUUIDType.instance),
-                                                                      NULL_DEF);
-        verify(expected, "SELECT now() FROM %s");
-    }
-
-    public void testNestedFunctionsWithArguments() throws Throwable
-    {
-        SelectionColumns expected = SelectionColumnMapping.newMapping()
-                                                          .addMapping(columnSpecification("blobasint(intasblob(v1))",
-                                                                                          Int32Type.instance),
-                                                                      columnDefinition("v1"));
-        verify(expected, "SELECT blobasint(intasblob(v1)) FROM %s");
-    }
-
-    public void testNestedFunctions() throws Throwable
-    {
-        SelectionColumns expected = SelectionColumnMapping.newMapping()
-                                                          .addMapping(columnSpecification("unixtimestampof(now())",
-                                                                                          LongType.instance),
-                                                                      NULL_DEF);
-        verify(expected, "SELECT unixtimestampof(now()) FROM %s");
-    }
-
-    public void testCount() throws Throwable
-    {
-        SelectionColumns expected = SelectionColumnMapping.newMapping()
-                                                          .addMapping(columnSpecification("count", LongType.instance),
-                                                                      NULL_DEF);
-        verify(expected, "SELECT count(*) FROM %s");
-        verify(expected, "SELECT count(1) FROM %s");
-
-        expected = SelectionColumnMapping.newMapping()
-                                         .addMapping(columnSpecification("other_count", LongType.instance), NULL_DEF);
-        verify(expected, "SELECT count(*) AS other_count FROM %s");
-        verify(expected, "SELECT count(1) AS other_count FROM %s");
-    }
-
-    public void testDuplicateFunctionsWithoutAliases() throws Throwable
-    {
-        // where duplicate functions are present, the ColumnSpecification list will
-        // contain an entry per-duplicate but the mappings will be deduplicated (i.e.
-        // a single mapping k/v pair regardless of the number of duplicates)
-        ColumnSpecification spec = columnSpecification("intasblob(v1)", BytesType.instance);
-        SelectionColumns expected = SelectionColumnMapping.newMapping()
-                                                          .addMapping(spec, columnDefinition("v1"))
-                                                          .addMapping(spec, columnDefinition("v1"));
-        verify(expected, "SELECT intasblob(v1), intasblob(v1) FROM %s");
-    }
-
-    public void testDuplicateFunctionsWithAliases() throws Throwable
-    {
-        // where duplicate functions are present with distinct aliases, they are
-        // represented as any other set of distinct columns would be - an entry
-        // in theColumnSpecification list and a separate k/v mapping for each
-        SelectionColumns expected = SelectionColumnMapping.newMapping()
-                                                          .addMapping(columnSpecification("blob_1", BytesType.instance),
-                                                                      columnDefinition("v1"))
-                                                          .addMapping(columnSpecification("blob_2", BytesType.instance),
-                                                                      columnDefinition("v1"));
-        verify(expected, "SELECT intasblob(v1) AS blob_1, intasblob(v1) AS blob_2 FROM %s");
-    }
-
-    public void testSelectDistinct() throws Throwable
-    {
-        SelectionColumns expected = SelectionColumnMapping.newMapping().addMapping(columnDefinition("k"),
-                                                                                   columnDefinition("k"));
-        verify(expected, "SELECT DISTINCT k FROM %s");
-
-    }
-
-    private void testMultipleAliasesOnSameColumn() throws Throwable
-    {
-        // Multiple result columns derived from the same underlying column are
-        // represented by ColumnSpecifications
-        ColumnSpecification alias1 = columnSpecification("alias_1", Int32Type.instance);
-        ColumnSpecification alias2 = columnSpecification("alias_2", Int32Type.instance);
-        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
-                                                                .addMapping(alias1, columnDefinition("v1"))
-                                                                .addMapping(alias2, columnDefinition("v1"));
-
-        verify(expected, "SELECT v1 AS alias_1, v1 AS alias_2 FROM %s");
-    }
-
-    private void testMultipleUnaliasedSelectionOfSameColumn() throws Throwable
-    {
-        // simple column identifiers without aliases are represented in
-        // ResultSet.Metadata by the underlying ColumnDefinition
-        ColumnDefinition v1 = columnDefinition("v1");
-        SelectionColumns expected = SelectionColumnMapping.newMapping()
-                                                          .addMapping(v1, v1)
-                                                          .addMapping(v1, v1);
-
-        verify(expected, "SELECT v1, v1 FROM %s");
-    }
-
-    private void testMixedColumnTypes() throws Throwable
-    {
-        ColumnSpecification kSpec = columnSpecification("k_alias", Int32Type.instance);
-        ColumnSpecification v1Spec = columnSpecification("writetime(v1)", LongType.instance);
-        ColumnSpecification v2Spec = columnSpecification("ttl_alias", Int32Type.instance);
-        ColumnSpecification f1Spec = columnSpecification("v3.f1", Int32Type.instance);
-        ColumnSpecification f2Spec = columnSpecification("f2_alias", UTF8Type.instance);
-
-        SelectionColumnMapping expected = SelectionColumnMapping.newMapping()
-                                                                .addMapping(kSpec, columnDefinition("k"))
-                                                                .addMapping(v1Spec, columnDefinition("v1"))
-                                                                .addMapping(v2Spec, columnDefinition("v2"))
-                                                                .addMapping(f1Spec, columnDefinition("v3"))
-                                                                .addMapping(f2Spec, columnDefinition("v3"))
-                                                                .addMapping(columnDefinition("v3"), columnDefinition("v3"));
-
-        verify(expected, "SELECT k AS k_alias," +
-                         "       writetime(v1)," +
-                         "       ttl(v2) as ttl_alias," +
-                         "       v3.f1," +
-                         "       v3.f2 AS f2_alias," +
-                         "       v3" +
-                         " FROM %s");
-    }
-
-    private void verify(SelectionColumns expected, String query) throws Throwable
-    {
-        SelectStatement statement = getSelect(query);
-        verifyColumnMapping(expected, statement);
-        checkExecution(statement, expected.getColumnSpecifications());
-    }
-
-    private void checkExecution(SelectStatement statement, List<ColumnSpecification> expectedResultColumns)
-    throws RequestExecutionException, RequestValidationException
-    {
-        UntypedResultSet rs = UntypedResultSet.create(statement.executeInternal(QueryState.forInternalCalls(),
-                                                                                QueryOptions.DEFAULT).result);
-
-        assertEquals(expectedResultColumns, rs.one().getColumns());
-    }
-
-    private SelectStatement getSelect(String query) throws RequestValidationException
-    {
-        CQLStatement statement = QueryProcessor.getStatement(String.format(query, KEYSPACE + "." + tableName),
-                                                             ClientState.forInternalCalls()).statement;
-        assertTrue(statement instanceof SelectStatement);
-        return (SelectStatement)statement;
-    }
-
-    private void verifyColumnMapping(SelectionColumns expected, SelectStatement select)
-    {
-        assertEquals(expected, select.getSelection().getColumnMapping());
-    }
-
-    private ColumnDefinition columnDefinition(String name)
-    {
-        return Schema.instance.getCFMetaData(KEYSPACE, tableName)
-                              .getColumnDefinition(new ColumnIdentifier(name, true));
-
-    }
-
-    private Iterable<ColumnDefinition> columnDefinitions(String...name)
-    {
-        List<ColumnDefinition> list = new ArrayList<>();
-        for (String n : name)
-            list.add(columnDefinition(n));
-        return list;
-    }
-
-    private ColumnSpecification columnSpecification(String name, AbstractType<?> type)
-    {
-        return new ColumnSpecification(KEYSPACE,
-                                       tableName,
-                                       new ColumnIdentifier(name, true),
-                                       type);
-    }
-}
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/CollectionsTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/CollectionsTest.java
index 31dd5a6..99d9695 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/CollectionsTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/CollectionsTest.java
@@ -242,6 +242,106 @@
         assertRows(execute("SELECT l FROM %s WHERE k = 0"), row((Object) null));
     }
 
+    @Test
+    public void testMapWithUnsetValues() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, m map<text,text>)");
+        // set up
+        Object m = map("k", "v");
+        execute("INSERT INTO %s (k, m) VALUES (10, ?)", m);
+        assertRows(execute("SELECT m FROM %s WHERE k = 10"),
+                   row(m)
+        );
+
+        // test putting an unset map, should not delete the contents
+        execute("INSERT INTO %s (k, m) VALUES (10, ?)", unset());
+        assertRows(execute("SELECT m FROM %s WHERE k = 10"),
+                   row(m)
+        );
+        // test unset variables in a map update operaiotn, should not delete the contents
+        execute("UPDATE %s SET m['k'] = ? WHERE k = 10", unset());
+        assertRows(execute("SELECT m FROM %s WHERE k = 10"),
+                   row(m)
+        );
+        assertInvalidMessage("Invalid unset map key", "UPDATE %s SET m[?] = 'foo' WHERE k = 10", unset());
+
+        // test unset value for map key
+        assertInvalidMessage("Invalid unset map key", "DELETE m[?] FROM %s WHERE k = 10", unset());
+    }
+
+    @Test
+    public void testListWithUnsetValues() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, l list<text>)");
+        // set up
+        Object l = list("foo", "foo");
+        execute("INSERT INTO %s (k, l) VALUES (10, ?)", l);
+        assertRows(execute("SELECT l FROM %s WHERE k = 10"),
+                   row(l)
+        );
+
+        // replace list with unset value
+        execute("INSERT INTO %s (k, l) VALUES (10, ?)", unset());
+        assertRows(execute("SELECT l FROM %s WHERE k = 10"),
+                   row(l)
+        );
+
+        // add to position
+        execute("UPDATE %s SET l[1] = ? WHERE k = 10", unset());
+        assertRows(execute("SELECT l FROM %s WHERE k = 10"),
+                   row(l)
+        );
+
+        // set in index
+        assertInvalidMessage("Invalid unset value for list index", "UPDATE %s SET l[?] = 'foo' WHERE k = 10", unset());
+
+        // remove element by index
+        execute("DELETE l[?] FROM %s WHERE k = 10", unset());
+        assertRows(execute("SELECT l FROM %s WHERE k = 10"),
+                   row(l)
+        );
+
+        // remove all occurrences of element
+        execute("UPDATE %s SET l = l - ? WHERE k = 10", unset());
+        assertRows(execute("SELECT l FROM %s WHERE k = 10"),
+                   row(l)
+        );
+
+        // select with in clause
+        assertInvalidMessage("Invalid unset value for column k", "SELECT * FROM %s WHERE k IN ?", unset());
+        assertInvalidMessage("Invalid unset value for column k", "SELECT * FROM %s WHERE k IN (?)", unset());
+    }
+
+    @Test
+    public void testSetWithUnsetValues() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, s set<text>)");
+
+        Object s = set("bar", "baz", "foo");
+        execute("INSERT INTO %s (k, s) VALUES (10, ?)", s);
+        assertRows(execute("SELECT s FROM %s WHERE k = 10"),
+                   row(s)
+        );
+
+        // replace set with unset value
+        execute("INSERT INTO %s (k, s) VALUES (10, ?)", unset());
+        assertRows(execute("SELECT s FROM %s WHERE k = 10"),
+                   row(s)
+        );
+
+        // add to set
+        execute("UPDATE %s SET s = s + ? WHERE k = 10", unset());
+        assertRows(execute("SELECT s FROM %s WHERE k = 10"),
+                   row(s)
+        );
+
+        // remove all occurrences of element
+        execute("UPDATE %s SET s = s - ? WHERE k = 10", unset());
+        assertRows(execute("SELECT s FROM %s WHERE k = 10"),
+                   row(s)
+        );
+    }
+
     /**
      * Migrated from cql_tests.py:TestCQL.set_test()
      */
@@ -485,7 +585,6 @@
         assertInvalid("alter table %s add v set<int>");
     }
 
-
     /**
      * Test for 9838.
      */
@@ -508,4 +607,145 @@
         assertRows(execute("select s_list from %s where k1='a'"), row(list(0)));
     }
 
+    @Test
+    public void testInvalidInputForList() throws Throwable
+    {
+        createTable("CREATE TABLE %s(pk int PRIMARY KEY, l list<text>)");
+        assertInvalidMessage("Not enough bytes to read a list",
+                             "INSERT INTO %s (pk, l) VALUES (?, ?)", 1, "test");
+        assertInvalidMessage("Not enough bytes to read a list",
+                             "INSERT INTO %s (pk, l) VALUES (?, ?)", 1, Long.MAX_VALUE);
+        assertInvalidMessage("Not enough bytes to read a list",
+                             "INSERT INTO %s (pk, l) VALUES (?, ?)", 1, "");
+        assertInvalidMessage("The data cannot be deserialized as a list",
+                             "INSERT INTO %s (pk, l) VALUES (?, ?)", 1, -1);
+    }
+
+    @Test
+    public void testInvalidInputForSet() throws Throwable
+    {
+        createTable("CREATE TABLE %s(pk int PRIMARY KEY, s set<text>)");
+        assertInvalidMessage("Not enough bytes to read a set",
+                             "INSERT INTO %s (pk, s) VALUES (?, ?)", 1, "test");
+        assertInvalidMessage("String didn't validate.",
+                             "INSERT INTO %s (pk, s) VALUES (?, ?)", 1, Long.MAX_VALUE);
+        assertInvalidMessage("Not enough bytes to read a set",
+                             "INSERT INTO %s (pk, s) VALUES (?, ?)", 1, "");
+        assertInvalidMessage("The data cannot be deserialized as a set",
+                             "INSERT INTO %s (pk, s) VALUES (?, ?)", 1, -1);
+    }
+
+    @Test
+    public void testInvalidInputForMap() throws Throwable
+    {
+        createTable("CREATE TABLE %s(pk int PRIMARY KEY, m map<text, text>)");
+        assertInvalidMessage("Not enough bytes to read a map",
+                             "INSERT INTO %s (pk, m) VALUES (?, ?)", 1, "test");
+        assertInvalidMessage("String didn't validate.",
+                             "INSERT INTO %s (pk, m) VALUES (?, ?)", 1, Long.MAX_VALUE);
+        assertInvalidMessage("Not enough bytes to read a map",
+                             "INSERT INTO %s (pk, m) VALUES (?, ?)", 1, "");
+        assertInvalidMessage("The data cannot be deserialized as a map",
+                             "INSERT INTO %s (pk, m) VALUES (?, ?)", 1, -1);
+    }
+
+    @Test
+    public void testMultipleOperationOnListWithinTheSameQuery() throws Throwable
+    {
+        createTable("CREATE TABLE %s (pk int PRIMARY KEY, l list<int>)");
+        execute("INSERT INTO %s (pk, l) VALUES (1, [1, 2, 3, 4])");
+
+        // Checks that when the same element is updated twice the update with the greatest value is the one taken into account
+        execute("UPDATE %s SET l[?] = ?, l[?] = ?  WHERE pk = ?", 2, 7, 2, 8, 1);
+        assertRows(execute("SELECT * FROM %s WHERE pk = ?", 1) , row(1, list(1, 2, 8, 4)));
+
+        execute("UPDATE %s SET l[?] = ?, l[?] = ?  WHERE pk = ?", 2, 9, 2, 6, 1);
+        assertRows(execute("SELECT * FROM %s WHERE pk = ?", 1) , row(1, list(1, 2, 9, 4)));
+
+        // Checks that deleting twice the same element will result in the deletion of the element with the index
+        // and of the following element.
+        execute("DELETE l[?], l[?] FROM %s WHERE pk = ?", 2, 2, 1);
+        assertRows(execute("SELECT * FROM %s WHERE pk = ?", 1) , row(1, list(1, 2)));
+
+        // Checks that the set operation is performed on the added elements and that the greatest value win
+        execute("UPDATE %s SET l = l + ?, l[?] = ?  WHERE pk = ?", list(3, 4), 3, 7, 1);
+        assertRows(execute("SELECT * FROM %s WHERE pk = ?", 1) , row(1, list(1, 2, 3, 7)));
+
+        execute("UPDATE %s SET l = l + ?, l[?] = ?  WHERE pk = ?", list(6, 8), 4, 5, 1);
+        assertRows(execute("SELECT * FROM %s WHERE pk = ?", 1) , row(1, list(1, 2, 3, 7, 6, 8)));
+
+        // Checks that the order of the operations matters
+        assertInvalidMessage("List index 6 out of bound, list has size 6",
+                             "UPDATE %s SET l[?] = ?, l = l + ? WHERE pk = ?", 6, 5, list(9), 1);
+
+        // Checks that the updated element is deleted.
+        execute("UPDATE %s SET l[?] = ? , l = l - ? WHERE pk = ?", 2, 8, list(8), 1);
+        assertRows(execute("SELECT * FROM %s WHERE pk = ?", 1) , row(1, list(1, 2, 7, 6)));
+
+        // Checks that we cannot update an element that has been removed.
+        assertInvalidMessage("List index 3 out of bound, list has size 3",
+                             "UPDATE %s SET l = l - ?, l[?] = ?  WHERE pk = ?", list(6), 3, 4, 1);
+
+        // Checks that the element is updated before the other ones are shifted.
+        execute("UPDATE %s SET l[?] = ? , l = l - ? WHERE pk = ?", 2, 8, list(1), 1);
+        assertRows(execute("SELECT * FROM %s WHERE pk = ?", 1) , row(1, list(2, 8, 6)));
+
+        // Checks that the element are shifted before the element is updated.
+        execute("UPDATE %s SET l = l - ?, l[?] = ?  WHERE pk = ?", list(2, 6), 0, 9, 1);
+        assertRows(execute("SELECT * FROM %s WHERE pk = ?", 1) , row(1, list(9)));
+    }
+
+    @Test
+    public void testMultipleOperationOnMapWithinTheSameQuery() throws Throwable
+    {
+        createTable("CREATE TABLE %s (pk int PRIMARY KEY, m map<int, int>)");
+        execute("INSERT INTO %s (pk, m) VALUES (1, {0 : 1, 1 : 2, 2 : 3, 3 : 4})");
+
+        // Checks that when the same element is updated twice the update with the greatest value is the one taken into account
+        execute("UPDATE %s SET m[?] = ?, m[?] = ?  WHERE pk = ?", 2, 7, 2, 8, 1);
+        assertRows(execute("SELECT * FROM %s WHERE pk = 1") , row(1, map(0, 1, 1, 2, 2, 8, 3, 4)));
+
+        execute("UPDATE %s SET m[?] = ?, m[?] = ?  WHERE pk = ?", 2, 9, 2, 6, 1);
+        assertRows(execute("SELECT * FROM %s WHERE pk = 1") , row(1, map(0, 1, 1, 2, 2, 9, 3, 4)));
+
+        // Checks that deleting twice the same element has no side effect
+        execute("DELETE m[?], m[?] FROM %s WHERE pk = ?", 2, 2, 1);
+        assertRows(execute("SELECT * FROM %s WHERE pk = ?", 1) , row(1, map(0, 1, 1, 2, 3, 4)));
+
+        // Checks that the set operation is performed on the added elements and that the greatest value win
+        execute("UPDATE %s SET m = m + ?, m[?] = ?  WHERE pk = ?", map(4, 5), 4, 7, 1);
+        assertRows(execute("SELECT * FROM %s WHERE pk = ?", 1) , row(1, map(0, 1, 1, 2, 3, 4, 4, 7)));
+
+        execute("UPDATE %s SET m = m + ?, m[?] = ?  WHERE pk = ?", map(4, 8), 4, 6, 1);
+        assertRows(execute("SELECT * FROM %s WHERE pk = ?", 1) , row(1, map(0, 1, 1, 2, 3, 4, 4, 8)));
+
+        // Checks that, as tombstones win over updates for the same timestamp, the removed element is not readded
+        execute("UPDATE %s SET m = m - ?, m[?] = ?  WHERE pk = ?", set(4), 4, 9, 1);
+        assertRows(execute("SELECT * FROM %s WHERE pk = ?", 1) , row(1, map(0, 1, 1, 2, 3, 4)));
+
+        // Checks that the update is taken into account before the removal
+        execute("UPDATE %s SET m[?] = ?,  m = m - ?  WHERE pk = ?", 5, 9, set(5), 1);
+        assertRows(execute("SELECT * FROM %s WHERE pk = ?", 1) , row(1, map(0, 1, 1, 2, 3, 4)));
+
+        // Checks that the set operation is merged with the change of the append and that the greatest value win
+        execute("UPDATE %s SET m[?] = ?, m = m + ?  WHERE pk = ?", 5, 9, map(5, 8, 6, 9), 1);
+        assertRows(execute("SELECT * FROM %s WHERE pk = ?", 1) , row(1, map(0, 1, 1, 2, 3, 4, 5, 9, 6, 9)));
+
+        execute("UPDATE %s SET m[?] = ?, m = m + ?  WHERE pk = ?", 7, 1, map(7, 2), 1);
+        assertRows(execute("SELECT * FROM %s WHERE pk = ?", 1) , row(1, map(0, 1, 1, 2, 3, 4, 5, 9, 6, 9, 7, 2)));
+    }
+
+    @Test
+    public void testMultipleOperationOnSetWithinTheSameQuery() throws Throwable
+    {
+        createTable("CREATE TABLE %s (pk int PRIMARY KEY, s set<int>)");
+        execute("INSERT INTO %s (pk, s) VALUES (1, {0, 1, 2})");
+
+        // Checks that the two operation are merged and that the tombstone always win
+        execute("UPDATE %s SET s = s + ? , s = s - ?  WHERE pk = ?", set(3, 4), set(3), 1);
+        assertRows(execute("SELECT * FROM %s WHERE pk = 1") , row(1, set(0, 1, 2, 4)));
+
+        execute("UPDATE %s SET s = s - ? , s = s + ?  WHERE pk = ?", set(3), set(3, 4), 1);
+        assertRows(execute("SELECT * FROM %s WHERE pk = 1") , row(1, set(0, 1, 2, 4)));
+    }
 }
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/CountersTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/CountersTest.java
index c66a04d..41b73bc 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/CountersTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/CountersTest.java
@@ -82,4 +82,45 @@
         assertInvalidThrow(InvalidRequestException.class,
                            String.format("CREATE TABLE %s (k int PRIMARY KEY, m map<text, counter>)", tableName));
     }
+
+    @Test
+    public void testCounterUpdatesWithUnset() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, c counter)");
+
+        // set up
+        execute("UPDATE %s SET c = c + 1 WHERE k = 10");
+        assertRows(execute("SELECT c FROM %s WHERE k = 10"),
+                   row(1L)
+        );
+        // increment
+        execute("UPDATE %s SET c = c + ? WHERE k = 10", 1L);
+        assertRows(execute("SELECT c FROM %s WHERE k = 10"),
+                   row(2L)
+        );
+        execute("UPDATE %s SET c = c + ? WHERE k = 10", unset());
+        assertRows(execute("SELECT c FROM %s WHERE k = 10"),
+                   row(2L) // no change to the counter value
+        );
+        // decrement
+        execute("UPDATE %s SET c = c - ? WHERE k = 10", 1L);
+        assertRows(execute("SELECT c FROM %s WHERE k = 10"),
+                   row(1L)
+        );
+        execute("UPDATE %s SET c = c - ? WHERE k = 10", unset());
+        assertRows(execute("SELECT c FROM %s WHERE k = 10"),
+                   row(1L) // no change to the counter value
+        );
+    }
+
+    /**
+     * Test for the validation bug of #9395.
+     */
+    @Test
+    public void testProhibitReversedCounterAsPartOfPrimaryKey() throws Throwable
+    {
+        assertInvalidThrowMessage("counter type is not supported for PRIMARY KEY part a",
+                                  InvalidRequestException.class, String.format("CREATE TABLE %s.%s (a counter, b int, PRIMARY KEY (b, a)) WITH CLUSTERING ORDER BY (a desc);", KEYSPACE, createTableName()));
+    }
+
 }
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/FrozenCollectionsTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/FrozenCollectionsTest.java
index fb50e83..b590843 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/FrozenCollectionsTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/FrozenCollectionsTest.java
@@ -85,8 +85,8 @@
         );
 
         assertRows(execute("SELECT * FROM %s WHERE k IN ?", list(set(4, 5, 6), set())),
-                row(set(), 0),
-                row(set(4, 5, 6), 0)
+                   row(set(), 0),
+                   row(set(4, 5, 6), 0)
         );
 
         assertRows(execute("SELECT * FROM %s WHERE token(k) >= token(?)", set(4, 5, 6)),
@@ -154,9 +154,9 @@
         );
 
         assertRows(execute("SELECT * FROM %s WHERE k IN ?", list(map(set(4, 5, 6), list(1, 2, 3)), map(), map(set(), list(1, 2, 3)))),
-            row(map(), 0),
-            row(map(set(), list(1, 2, 3)), 0),
-            row(map(set(4, 5, 6), list(1, 2, 3)), 0)
+                   row(map(), 0),
+                   row(map(set(), list(1, 2, 3)), 0),
+                   row(map(set(4, 5, 6), list(1, 2, 3)), 0)
         );
 
         assertRows(execute("SELECT * FROM %s WHERE token(k) >= token(?)", map(set(4, 5, 6), list(1, 2, 3))),
@@ -625,19 +625,19 @@
                              "SELECT * FROM %s WHERE c CONTAINS KEY ?", 1);
 
         // normal indexes on frozen collections don't support CONTAINS or CONTAINS KEY
-        assertInvalidMessage("Cannot restrict column \"b\" by a CONTAINS relation without a secondary index",
+        assertInvalidMessage("Cannot restrict clustering columns by a CONTAINS relation without a secondary index",
                              "SELECT * FROM %s WHERE b CONTAINS ?", 1);
 
-        assertInvalidMessage("Cannot restrict column \"b\" by a CONTAINS relation without a secondary index",
+        assertInvalidMessage("Cannot restrict clustering columns by a CONTAINS relation without a secondary index",
                              "SELECT * FROM %s WHERE b CONTAINS ? ALLOW FILTERING", 1);
 
-        assertInvalidMessage("No secondary indexes on the restricted columns support the provided operator",
+        assertInvalidMessage("Predicates on non-primary-key columns (d) are not yet supported for non secondary index queries",
                              "SELECT * FROM %s WHERE d CONTAINS KEY ?", 1);
 
-        assertInvalidMessage("No secondary indexes on the restricted columns support the provided operator",
+        assertInvalidMessage("Predicates on non-primary-key columns (d) are not yet supported for non secondary index queries",
                              "SELECT * FROM %s WHERE d CONTAINS KEY ? ALLOW FILTERING", 1);
 
-        assertInvalidMessage("Cannot restrict column \"b\" by a CONTAINS relation without a secondary index",
+        assertInvalidMessage("Cannot restrict clustering columns by a CONTAINS relation without a secondary index",
                              "SELECT * FROM %s WHERE b CONTAINS ? AND d CONTAINS KEY ? ALLOW FILTERING", 1, 1);
 
         // index lookup on b
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/JsonTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/JsonTest.java
new file mode 100644
index 0000000..6892e79
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/JsonTest.java
@@ -0,0 +1,1109 @@
+/*
+ * 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.cql3.validation.entities;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.Json;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.serializers.SimpleDateSerializer;
+import org.apache.cassandra.serializers.TimeSerializer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.net.InetAddress;
+import java.text.SimpleDateFormat;
+import java.util.*;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.fail;
+
+public class JsonTest extends CQLTester
+{
+    @BeforeClass
+    public static void setUp()
+    {
+        DatabaseDescriptor.setPartitioner(ByteOrderedPartitioner.instance);
+    }
+
+    @Test
+    public void testFromJsonFct() throws Throwable
+    {
+        String typeName = createType("CREATE TYPE %s (a int, b uuid, c set<text>)");
+        createTable("CREATE TABLE %s (" +
+                "k int PRIMARY KEY, " +
+                "asciival ascii, " +
+                "bigintval bigint, " +
+                "blobval blob, " +
+                "booleanval boolean, " +
+                "dateval date, " +
+                "decimalval decimal, " +
+                "doubleval double, " +
+                "floatval float, " +
+                "inetval inet, " +
+                "intval int, " +
+                "smallintval smallint, " +
+                "textval text, " +
+                "timeval time, " +
+                "timestampval timestamp, " +
+                "timeuuidval timeuuid, " +
+                "tinyintval tinyint, " +
+                "uuidval uuid," +
+                "varcharval varchar, " +
+                "varintval varint, " +
+                "listval list<int>, " +
+                "frozenlistval frozen<list<int>>, " +
+                "setval set<uuid>, " +
+                "frozensetval frozen<set<uuid>>, " +
+                "mapval map<ascii, int>," +
+                "frozenmapval frozen<map<ascii, int>>," +
+                "tupleval frozen<tuple<int, ascii, uuid>>," +
+                "udtval frozen<" + typeName + ">)");
+
+
+        // fromJson() can only be used when the receiver type is known
+        assertInvalidMessage("fromJson() cannot be used in the selection clause", "SELECT fromJson(asciival) FROM %s", 0, 0);
+
+        String func1 = createFunction(KEYSPACE, "int", "CREATE FUNCTION %s (a int) CALLED ON NULL INPUT RETURNS text LANGUAGE java AS $$ return a.toString(); $$");
+        createFunctionOverload(func1, "int", "CREATE FUNCTION %s (a text) CALLED ON NULL INPUT RETURNS text LANGUAGE java AS $$ return new String(a); $$");
+
+        assertInvalidMessage("Ambiguous call to function",
+                "INSERT INTO %s (k, textval) VALUES (?, " + func1 + "(fromJson(?)))", 0, "123");
+
+        // fails JSON parsing
+        assertInvalidMessage("Could not decode JSON string '\u038E\u0394\u03B4\u03E0'",
+                "INSERT INTO %s (k, asciival) VALUES (?, fromJson(?))", 0, "\u038E\u0394\u03B4\u03E0");
+
+        // handle nulls
+        execute("INSERT INTO %s (k, asciival) VALUES (?, fromJson(?))", 0, null);
+
+        // ================ ascii ================
+        execute("INSERT INTO %s (k, asciival) VALUES (?, fromJson(?))", 0, "\"ascii text\"");
+        assertRows(execute("SELECT k, asciival FROM %s WHERE k = ?", 0), row(0, "ascii text"));
+
+        execute("INSERT INTO %s (k, asciival) VALUES (?, fromJson(?))", 0, "\"ascii \\\" text\"");
+        assertRows(execute("SELECT k, asciival FROM %s WHERE k = ?", 0), row(0, "ascii \" text"));
+
+        assertInvalidMessage("Invalid ASCII character in string literal",
+                "INSERT INTO %s (k, asciival) VALUES (?, fromJson(?))", 0, "\"\\u1fff\\u2013\\u33B4\\u2014\"");
+
+        assertInvalidMessage("Expected an ascii string, but got a Integer",
+                "INSERT INTO %s (k, asciival) VALUES (?, fromJson(?))", 0, "123");
+
+        // test that we can use fromJson() in other valid places in queries
+        assertRows(execute("SELECT asciival FROM %s WHERE k = fromJson(?)", "0"), row("ascii \" text"));
+        execute("UPDATE %s SET asciival = fromJson(?) WHERE k = fromJson(?)", "\"ascii \\\" text\"", "0");
+        execute("DELETE FROM %s WHERE k = fromJson(?)", "0");
+
+        // ================ bigint ================
+        execute("INSERT INTO %s (k, bigintval) VALUES (?, fromJson(?))", 0, "123123123123");
+        assertRows(execute("SELECT k, bigintval FROM %s WHERE k = ?", 0), row(0, 123123123123L));
+
+        // strings are also accepted
+        execute("INSERT INTO %s (k, bigintval) VALUES (?, fromJson(?))", 0, "\"123123123123\"");
+        assertRows(execute("SELECT k, bigintval FROM %s WHERE k = ?", 0), row(0, 123123123123L));
+
+        // overflow (Long.MAX_VALUE + 1)
+        assertInvalidMessage("Expected a bigint value, but got a",
+                "INSERT INTO %s (k, bigintval) VALUES (?, fromJson(?))", 0, "9223372036854775808");
+
+        assertInvalidMessage("Expected a bigint value, but got a",
+                "INSERT INTO %s (k, bigintval) VALUES (?, fromJson(?))", 0, "123.456");
+
+        assertInvalidMessage("Unable to make long from",
+                "INSERT INTO %s (k, bigintval) VALUES (?, fromJson(?))", 0, "\"abc\"");
+
+        assertInvalidMessage("Expected a bigint value, but got a",
+                "INSERT INTO %s (k, bigintval) VALUES (?, fromJson(?))", 0, "[\"abc\"]");
+
+        // ================ blob ================
+        execute("INSERT INTO %s (k, blobval) VALUES (?, fromJson(?))", 0, "\"0x00000001\"");
+        assertRows(execute("SELECT k, blobval FROM %s WHERE k = ?", 0), row(0, ByteBufferUtil.bytes(1)));
+
+        assertInvalidMessage("Value 'xyzz' is not a valid blob representation",
+            "INSERT INTO %s (k, blobval) VALUES (?, fromJson(?))", 0, "\"xyzz\"");
+
+        assertInvalidMessage("String representation of blob is missing 0x prefix: 123",
+                "INSERT INTO %s (k, blobval) VALUES (?, fromJson(?))", 0, "\"123\"");
+
+        assertInvalidMessage("Value '0x123' is not a valid blob representation",
+                "INSERT INTO %s (k, blobval) VALUES (?, fromJson(?))", 0, "\"0x123\"");
+
+        assertInvalidMessage("Value '123' is not a valid blob representation",
+                "INSERT INTO %s (k, blobval) VALUES (?, fromJson(?))", 0, "123");
+
+        // ================ boolean ================
+        execute("INSERT INTO %s (k, booleanval) VALUES (?, fromJson(?))", 0, "true");
+        assertRows(execute("SELECT k, booleanval FROM %s WHERE k = ?", 0), row(0, true));
+
+        execute("INSERT INTO %s (k, booleanval) VALUES (?, fromJson(?))", 0, "false");
+        assertRows(execute("SELECT k, booleanval FROM %s WHERE k = ?", 0), row(0, false));
+
+        // strings are also accepted
+        execute("INSERT INTO %s (k, booleanval) VALUES (?, fromJson(?))", 0, "\"false\"");
+        assertRows(execute("SELECT k, booleanval FROM %s WHERE k = ?", 0), row(0, false));
+
+        assertInvalidMessage("Unable to make boolean from",
+                "INSERT INTO %s (k, booleanval) VALUES (?, fromJson(?))", 0, "\"abc\"");
+
+        assertInvalidMessage("Expected a boolean value, but got a Integer",
+                "INSERT INTO %s (k, booleanval) VALUES (?, fromJson(?))", 0, "123");
+
+        // ================ date ================
+        execute("INSERT INTO %s (k, dateval) VALUES (?, fromJson(?))", 0, "\"1987-03-23\"");
+        assertRows(execute("SELECT k, dateval FROM %s WHERE k = ?", 0), row(0, SimpleDateSerializer.dateStringToDays("1987-03-23")));
+
+        assertInvalidMessage("Expected a string representation of a date",
+                "INSERT INTO %s (k, dateval) VALUES (?, fromJson(?))", 0, "123");
+
+        assertInvalidMessage("Unable to coerce 'xyz' to a formatted date",
+                "INSERT INTO %s (k, dateval) VALUES (?, fromJson(?))", 0, "\"xyz\"");
+
+        // ================ decimal ================
+        execute("INSERT INTO %s (k, decimalval) VALUES (?, fromJson(?))", 0, "123123.123123");
+        assertRows(execute("SELECT k, decimalval FROM %s WHERE k = ?", 0), row(0, new BigDecimal("123123.123123")));
+
+        execute("INSERT INTO %s (k, decimalval) VALUES (?, fromJson(?))", 0, "123123");
+        assertRows(execute("SELECT k, decimalval FROM %s WHERE k = ?", 0), row(0, new BigDecimal("123123")));
+
+        // accept strings for numbers that cannot be represented as doubles
+        execute("INSERT INTO %s (k, decimalval) VALUES (?, fromJson(?))", 0, "\"123123.123123\"");
+        assertRows(execute("SELECT k, decimalval FROM %s WHERE k = ?", 0), row(0, new BigDecimal("123123.123123")));
+
+        execute("INSERT INTO %s (k, decimalval) VALUES (?, fromJson(?))", 0, "\"-1.23E-12\"");
+        assertRows(execute("SELECT k, decimalval FROM %s WHERE k = ?", 0), row(0, new BigDecimal("-1.23E-12")));
+
+        assertInvalidMessage("Value 'xyzz' is not a valid representation of a decimal value",
+                "INSERT INTO %s (k, decimalval) VALUES (?, fromJson(?))", 0, "\"xyzz\"");
+
+        assertInvalidMessage("Value 'true' is not a valid representation of a decimal value",
+                "INSERT INTO %s (k, decimalval) VALUES (?, fromJson(?))", 0, "true");
+
+        // ================ double ================
+        execute("INSERT INTO %s (k, doubleval) VALUES (?, fromJson(?))", 0, "123123.123123");
+        assertRows(execute("SELECT k, doubleval FROM %s WHERE k = ?", 0), row(0, 123123.123123d));
+
+        execute("INSERT INTO %s (k, doubleval) VALUES (?, fromJson(?))", 0, "123123");
+        assertRows(execute("SELECT k, doubleval FROM %s WHERE k = ?", 0), row(0, 123123.0d));
+
+        // strings are also accepted
+        execute("INSERT INTO %s (k, doubleval) VALUES (?, fromJson(?))", 0, "\"123123\"");
+        assertRows(execute("SELECT k, doubleval FROM %s WHERE k = ?", 0), row(0, 123123.0d));
+
+        assertInvalidMessage("Unable to make double from",
+                "INSERT INTO %s (k, doubleval) VALUES (?, fromJson(?))", 0, "\"xyzz\"");
+
+        assertInvalidMessage("Expected a double value, but got",
+                "INSERT INTO %s (k, doubleval) VALUES (?, fromJson(?))", 0, "true");
+
+        // ================ float ================
+        execute("INSERT INTO %s (k, floatval) VALUES (?, fromJson(?))", 0, "123123.123123");
+        assertRows(execute("SELECT k, floatval FROM %s WHERE k = ?", 0), row(0, 123123.123123f));
+
+        execute("INSERT INTO %s (k, floatval) VALUES (?, fromJson(?))", 0, "123123");
+        assertRows(execute("SELECT k, floatval FROM %s WHERE k = ?", 0), row(0, 123123.0f));
+
+        // strings are also accepted
+        execute("INSERT INTO %s (k, floatval) VALUES (?, fromJson(?))", 0, "\"123123.0\"");
+        assertRows(execute("SELECT k, floatval FROM %s WHERE k = ?", 0), row(0, 123123.0f));
+
+        assertInvalidMessage("Unable to make float from",
+                "INSERT INTO %s (k, floatval) VALUES (?, fromJson(?))", 0, "\"xyzz\"");
+
+        assertInvalidMessage("Expected a float value, but got a",
+                "INSERT INTO %s (k, floatval) VALUES (?, fromJson(?))", 0, "true");
+
+        // ================ inet ================
+        execute("INSERT INTO %s (k, inetval) VALUES (?, fromJson(?))", 0, "\"127.0.0.1\"");
+        assertRows(execute("SELECT k, inetval FROM %s WHERE k = ?", 0), row(0, InetAddress.getByName("127.0.0.1")));
+
+        execute("INSERT INTO %s (k, inetval) VALUES (?, fromJson(?))", 0, "\"::1\"");
+        assertRows(execute("SELECT k, inetval FROM %s WHERE k = ?", 0), row(0, InetAddress.getByName("::1")));
+
+        assertInvalidMessage("Unable to make inet address from 'xyzz'",
+                "INSERT INTO %s (k, inetval) VALUES (?, fromJson(?))", 0, "\"xyzz\"");
+
+        assertInvalidMessage("Expected a string representation of an inet value, but got a Integer",
+                "INSERT INTO %s (k, inetval) VALUES (?, fromJson(?))", 0, "123");
+
+        // ================ int ================
+        execute("INSERT INTO %s (k, intval) VALUES (?, fromJson(?))", 0, "123123");
+        assertRows(execute("SELECT k, intval FROM %s WHERE k = ?", 0), row(0, 123123));
+
+        // strings are also accepted
+        execute("INSERT INTO %s (k, intval) VALUES (?, fromJson(?))", 0, "\"123123\"");
+        assertRows(execute("SELECT k, intval FROM %s WHERE k = ?", 0), row(0, 123123));
+
+        // int overflow (2 ^ 32, or Integer.MAX_INT + 1)
+        assertInvalidMessage("Expected an int value, but got a",
+                "INSERT INTO %s (k, intval) VALUES (?, fromJson(?))", 0, "2147483648");
+
+        assertInvalidMessage("Expected an int value, but got a",
+                "INSERT INTO %s (k, intval) VALUES (?, fromJson(?))", 0, "123.456");
+
+        assertInvalidMessage("Unable to make int from",
+                "INSERT INTO %s (k, intval) VALUES (?, fromJson(?))", 0, "\"xyzz\"");
+
+        assertInvalidMessage("Expected an int value, but got a",
+                "INSERT INTO %s (k, intval) VALUES (?, fromJson(?))", 0, "true");
+
+        // ================ smallint ================
+        execute("INSERT INTO %s (k, smallintval) VALUES (?, fromJson(?))", 0, "32767");
+        assertRows(execute("SELECT k, smallintval FROM %s WHERE k = ?", 0), row(0, (short) 32767));
+
+        // strings are also accepted
+        execute("INSERT INTO %s (k, smallintval) VALUES (?, fromJson(?))", 0, "\"32767\"");
+        assertRows(execute("SELECT k, smallintval FROM %s WHERE k = ?", 0), row(0, (short) 32767));
+
+        // smallint overflow (Short.MAX_VALUE + 1)
+        assertInvalidMessage("Unable to make short from",
+                "INSERT INTO %s (k, smallintval) VALUES (?, fromJson(?))", 0, "32768");
+
+        assertInvalidMessage("Unable to make short from",
+                "INSERT INTO %s (k, smallintval) VALUES (?, fromJson(?))", 0, "123.456");
+
+        assertInvalidMessage("Unable to make short from",
+                "INSERT INTO %s (k, smallintval) VALUES (?, fromJson(?))", 0, "\"xyzz\"");
+
+        assertInvalidMessage("Expected a short value, but got a Boolean",
+                "INSERT INTO %s (k, smallintval) VALUES (?, fromJson(?))", 0, "true");
+
+        // ================ tinyint ================
+        execute("INSERT INTO %s (k, tinyintval) VALUES (?, fromJson(?))", 0, "127");
+        assertRows(execute("SELECT k, tinyintval FROM %s WHERE k = ?", 0), row(0, (byte) 127));
+
+        // strings are also accepted
+        execute("INSERT INTO %s (k, tinyintval) VALUES (?, fromJson(?))", 0, "\"127\"");
+        assertRows(execute("SELECT k, tinyintval FROM %s WHERE k = ?", 0), row(0, (byte) 127));
+
+        // tinyint overflow (Byte.MAX_VALUE + 1)
+        assertInvalidMessage("Unable to make byte from",
+                "INSERT INTO %s (k, tinyintval) VALUES (?, fromJson(?))", 0, "128");
+
+        assertInvalidMessage("Unable to make byte from",
+                "INSERT INTO %s (k, tinyintval) VALUES (?, fromJson(?))", 0, "123.456");
+
+        assertInvalidMessage("Unable to make byte from",
+                "INSERT INTO %s (k, tinyintval) VALUES (?, fromJson(?))", 0, "\"xyzz\"");
+
+        assertInvalidMessage("Expected a byte value, but got a Boolean",
+                "INSERT INTO %s (k, tinyintval) VALUES (?, fromJson(?))", 0, "true");
+
+        // ================ text (varchar) ================
+        execute("INSERT INTO %s (k, textval) VALUES (?, fromJson(?))", 0, "\"\"");
+        assertRows(execute("SELECT k, textval FROM %s WHERE k = ?", 0), row(0, ""));
+
+        execute("INSERT INTO %s (k, textval) VALUES (?, fromJson(?))", 0, "\"abcd\"");
+        assertRows(execute("SELECT k, textval FROM %s WHERE k = ?", 0), row(0, "abcd"));
+
+        execute("INSERT INTO %s (k, textval) VALUES (?, fromJson(?))", 0, "\"some \\\" text\"");
+        assertRows(execute("SELECT k, textval FROM %s WHERE k = ?", 0), row(0, "some \" text"));
+
+        execute("INSERT INTO %s (k, textval) VALUES (?, fromJson(?))", 0, "\"\\u2013\"");
+        assertRows(execute("SELECT k, textval FROM %s WHERE k = ?", 0), row(0, "\u2013"));
+
+        assertInvalidMessage("Expected a UTF-8 string, but got a Integer",
+                "INSERT INTO %s (k, textval) VALUES (?, fromJson(?))", 0, "123");
+
+        // ================ time ================
+        execute("INSERT INTO %s (k, timeval) VALUES (?, fromJson(?))", 0, "\"07:35:07.000111222\"");
+        assertRows(execute("SELECT k, timeval FROM %s WHERE k = ?", 0), row(0, TimeSerializer.timeStringToLong("07:35:07.000111222")));
+
+        assertInvalidMessage("Expected a string representation of a time value",
+                "INSERT INTO %s (k, timeval) VALUES (?, fromJson(?))", 0, "123456");
+
+        assertInvalidMessage("Unable to coerce 'xyz' to a formatted time",
+                "INSERT INTO %s (k, timeval) VALUES (?, fromJson(?))", 0, "\"xyz\"");
+
+        // ================ timestamp ================
+        execute("INSERT INTO %s (k, timestampval) VALUES (?, fromJson(?))", 0, "123123123123");
+        assertRows(execute("SELECT k, timestampval FROM %s WHERE k = ?", 0), row(0, new Date(123123123123L)));
+
+        execute("INSERT INTO %s (k, timestampval) VALUES (?, fromJson(?))", 0, "\"2014-01-01\"");
+        assertRows(execute("SELECT k, timestampval FROM %s WHERE k = ?", 0), row(0, new SimpleDateFormat("y-M-d").parse("2014-01-01")));
+
+        assertInvalidMessage("Expected a long or a datestring representation of a timestamp value, but got a Double",
+                "INSERT INTO %s (k, timestampval) VALUES (?, fromJson(?))", 0, "123.456");
+
+        assertInvalidMessage("Unable to coerce 'abcd' to a formatted date",
+                "INSERT INTO %s (k, timestampval) VALUES (?, fromJson(?))", 0, "\"abcd\"");
+
+        // ================ timeuuid ================
+        execute("INSERT INTO %s (k, timeuuidval) VALUES (?, fromJson(?))", 0, "\"6bddc89a-5644-11e4-97fc-56847afe9799\"");
+        assertRows(execute("SELECT k, timeuuidval FROM %s WHERE k = ?", 0), row(0, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")));
+
+        execute("INSERT INTO %s (k, timeuuidval) VALUES (?, fromJson(?))", 0, "\"6BDDC89A-5644-11E4-97FC-56847AFE9799\"");
+        assertRows(execute("SELECT k, timeuuidval FROM %s WHERE k = ?", 0), row(0, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")));
+
+        assertInvalidMessage("TimeUUID supports only version 1 UUIDs",
+                "INSERT INTO %s (k, timeuuidval) VALUES (?, fromJson(?))", 0, "\"00000000-0000-0000-0000-000000000000\"");
+
+        assertInvalidMessage("Expected a string representation of a timeuuid, but got a Integer",
+                "INSERT INTO %s (k, timeuuidval) VALUES (?, fromJson(?))", 0, "123");
+
+         // ================ uuidval ================
+        execute("INSERT INTO %s (k, uuidval) VALUES (?, fromJson(?))", 0, "\"6bddc89a-5644-11e4-97fc-56847afe9799\"");
+        assertRows(execute("SELECT k, uuidval FROM %s WHERE k = ?", 0), row(0, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")));
+
+        execute("INSERT INTO %s (k, uuidval) VALUES (?, fromJson(?))", 0, "\"6BDDC89A-5644-11E4-97FC-56847AFE9799\"");
+        assertRows(execute("SELECT k, uuidval FROM %s WHERE k = ?", 0), row(0, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")));
+
+        assertInvalidMessage("Unable to make UUID from",
+                "INSERT INTO %s (k, uuidval) VALUES (?, fromJson(?))", 0, "\"00000000-0000-0000-zzzz-000000000000\"");
+
+        assertInvalidMessage("Expected a string representation of a uuid, but got a Integer",
+                "INSERT INTO %s (k, uuidval) VALUES (?, fromJson(?))", 0, "123");
+
+        // ================ varint ================
+        execute("INSERT INTO %s (k, varintval) VALUES (?, fromJson(?))", 0, "123123123123");
+        assertRows(execute("SELECT k, varintval FROM %s WHERE k = ?", 0), row(0, new BigInteger("123123123123")));
+
+        // accept strings for numbers that cannot be represented as longs
+        execute("INSERT INTO %s (k, varintval) VALUES (?, fromJson(?))", 0, "\"1234567890123456789012345678901234567890\"");
+        assertRows(execute("SELECT k, varintval FROM %s WHERE k = ?", 0), row(0, new BigInteger("1234567890123456789012345678901234567890")));
+
+        assertInvalidMessage("Value '123123.123' is not a valid representation of a varint value",
+                "INSERT INTO %s (k, varintval) VALUES (?, fromJson(?))", 0, "123123.123");
+
+        assertInvalidMessage("Value 'xyzz' is not a valid representation of a varint value",
+                "INSERT INTO %s (k, varintval) VALUES (?, fromJson(?))", 0, "\"xyzz\"");
+
+        assertInvalidMessage("Value '' is not a valid representation of a varint value",
+                "INSERT INTO %s (k, varintval) VALUES (?, fromJson(?))", 0, "\"\"");
+
+        assertInvalidMessage("Value 'true' is not a valid representation of a varint value",
+                "INSERT INTO %s (k, varintval) VALUES (?, fromJson(?))", 0, "true");
+
+        // ================ lists ================
+        execute("INSERT INTO %s (k, listval) VALUES (?, fromJson(?))", 0, "[1, 2, 3]");
+        assertRows(execute("SELECT k, listval FROM %s WHERE k = ?", 0), row(0, list(1, 2, 3)));
+
+        execute("INSERT INTO %s (k, listval) VALUES (?, fromJson(?))", 0, "[]");
+        assertRows(execute("SELECT k, listval FROM %s WHERE k = ?", 0), row(0, null));
+
+        assertInvalidMessage("Expected a list, but got a Integer",
+                "INSERT INTO %s (k, listval) VALUES (?, fromJson(?))", 0, "123");
+
+        assertInvalidMessage("Unable to make int from",
+                "INSERT INTO %s (k, listval) VALUES (?, fromJson(?))", 0, "[\"abc\"]");
+
+        assertInvalidMessage("Invalid null element in list",
+                "INSERT INTO %s (k, listval) VALUES (?, fromJson(?))", 0, "[null]");
+
+        // frozen
+        execute("INSERT INTO %s (k, frozenlistval) VALUES (?, fromJson(?))", 0, "[1, 2, 3]");
+        assertRows(execute("SELECT k, frozenlistval FROM %s WHERE k = ?", 0), row(0, list(1, 2, 3)));
+
+        // ================ sets ================
+        execute("INSERT INTO %s (k, setval) VALUES (?, fromJson(?))",
+                0, "[\"6bddc89a-5644-11e4-97fc-56847afe9798\", \"6bddc89a-5644-11e4-97fc-56847afe9799\"]");
+        assertRows(execute("SELECT k, setval FROM %s WHERE k = ?", 0),
+                row(0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))))
+        );
+
+        // duplicates are okay, just like in CQL
+        execute("INSERT INTO %s (k, setval) VALUES (?, fromJson(?))",
+                0, "[\"6bddc89a-5644-11e4-97fc-56847afe9798\", \"6bddc89a-5644-11e4-97fc-56847afe9798\", \"6bddc89a-5644-11e4-97fc-56847afe9799\"]");
+        assertRows(execute("SELECT k, setval FROM %s WHERE k = ?", 0),
+                row(0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))))
+        );
+
+        execute("INSERT INTO %s (k, setval) VALUES (?, fromJson(?))", 0, "[]");
+        assertRows(execute("SELECT k, setval FROM %s WHERE k = ?", 0), row(0, null));
+
+        assertInvalidMessage("Expected a list (representing a set), but got a Integer",
+                "INSERT INTO %s (k, setval) VALUES (?, fromJson(?))", 0, "123");
+
+        assertInvalidMessage("Unable to make UUID from",
+                "INSERT INTO %s (k, setval) VALUES (?, fromJson(?))", 0, "[\"abc\"]");
+
+        assertInvalidMessage("Invalid null element in set",
+                "INSERT INTO %s (k, setval) VALUES (?, fromJson(?))", 0, "[null]");
+
+        // frozen
+        execute("INSERT INTO %s (k, frozensetval) VALUES (?, fromJson(?))",
+                0, "[\"6bddc89a-5644-11e4-97fc-56847afe9798\", \"6bddc89a-5644-11e4-97fc-56847afe9799\"]");
+        assertRows(execute("SELECT k, frozensetval FROM %s WHERE k = ?", 0),
+                row(0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))))
+        );
+
+        execute("INSERT INTO %s (k, frozensetval) VALUES (?, fromJson(?))",
+                0, "[\"6bddc89a-5644-11e4-97fc-56847afe9799\", \"6bddc89a-5644-11e4-97fc-56847afe9798\"]");
+        assertRows(execute("SELECT k, frozensetval FROM %s WHERE k = ?", 0),
+                row(0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))))
+        );
+
+        // ================ maps ================
+        execute("INSERT INTO %s (k, mapval) VALUES (?, fromJson(?))", 0, "{\"a\": 1, \"b\": 2}");
+        assertRows(execute("SELECT k, mapval FROM %s WHERE k = ?", 0), row(0, map("a", 1, "b", 2)));
+
+        execute("INSERT INTO %s (k, mapval) VALUES (?, fromJson(?))", 0, "{}");
+        assertRows(execute("SELECT k, mapval FROM %s WHERE k = ?", 0), row(0, null));
+
+        assertInvalidMessage("Expected a map, but got a Integer",
+                "INSERT INTO %s (k, mapval) VALUES (?, fromJson(?))", 0, "123");
+
+        assertInvalidMessage("Invalid ASCII character in string literal",
+                "INSERT INTO %s (k, mapval) VALUES (?, fromJson(?))", 0, "{\"\\u1fff\\u2013\\u33B4\\u2014\": 1}");
+
+        assertInvalidMessage("Invalid null value in map",
+                "INSERT INTO %s (k, mapval) VALUES (?, fromJson(?))", 0, "{\"a\": null}");
+
+        // frozen
+        execute("INSERT INTO %s (k, frozenmapval) VALUES (?, fromJson(?))", 0, "{\"a\": 1, \"b\": 2}");
+        assertRows(execute("SELECT k, frozenmapval FROM %s WHERE k = ?", 0), row(0, map("a", 1, "b", 2)));
+
+        execute("INSERT INTO %s (k, frozenmapval) VALUES (?, fromJson(?))", 0, "{\"b\": 2, \"a\": 1}");
+        assertRows(execute("SELECT k, frozenmapval FROM %s WHERE k = ?", 0), row(0, map("a", 1, "b", 2)));
+
+        // ================ tuples ================
+        execute("INSERT INTO %s (k, tupleval) VALUES (?, fromJson(?))", 0, "[1, \"foobar\", \"6bddc89a-5644-11e4-97fc-56847afe9799\"]");
+        assertRows(execute("SELECT k, tupleval FROM %s WHERE k = ?", 0),
+            row(0, tuple(1, "foobar", UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")))
+        );
+
+        execute("INSERT INTO %s (k, tupleval) VALUES (?, fromJson(?))", 0, "[1, null, \"6bddc89a-5644-11e4-97fc-56847afe9799\"]");
+        assertRows(execute("SELECT k, tupleval FROM %s WHERE k = ?", 0),
+                row(0, tuple(1, null, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")))
+        );
+
+        assertInvalidMessage("Tuple contains extra items",
+                "INSERT INTO %s (k, tupleval) VALUES (?, fromJson(?))",
+                0, "[1, \"foobar\", \"6bddc89a-5644-11e4-97fc-56847afe9799\", 1, 2, 3]");
+
+        assertInvalidMessage("Tuple is missing items",
+                "INSERT INTO %s (k, tupleval) VALUES (?, fromJson(?))",
+                0, "[1, \"foobar\"]");
+
+        assertInvalidMessage("Unable to make int from",
+                "INSERT INTO %s (k, tupleval) VALUES (?, fromJson(?))",
+                0, "[\"not an int\", \"foobar\", \"6bddc89a-5644-11e4-97fc-56847afe9799\"]");
+
+        // ================ UDTs ================
+        execute("INSERT INTO %s (k, udtval) VALUES (?, fromJson(?))", 0, "{\"a\": 1, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\", \"c\": [\"foo\", \"bar\"]}");
+        assertRows(execute("SELECT k, udtval.a, udtval.b, udtval.c FROM %s WHERE k = ?", 0),
+                row(0, 1, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"), set("bar", "foo"))
+        );
+
+        // order of fields shouldn't matter
+        execute("INSERT INTO %s (k, udtval) VALUES (?, fromJson(?))", 0, "{\"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\", \"a\": 1, \"c\": [\"foo\", \"bar\"]}");
+        assertRows(execute("SELECT k, udtval.a, udtval.b, udtval.c FROM %s WHERE k = ?", 0),
+                row(0, 1, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"), set("bar", "foo"))
+        );
+
+        // test nulls
+        execute("INSERT INTO %s (k, udtval) VALUES (?, fromJson(?))", 0, "{\"a\": null, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\", \"c\": [\"foo\", \"bar\"]}");
+        assertRows(execute("SELECT k, udtval.a, udtval.b, udtval.c FROM %s WHERE k = ?", 0),
+                row(0, null, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"), set("bar", "foo"))
+        );
+
+        // test missing fields
+        execute("INSERT INTO %s (k, udtval) VALUES (?, fromJson(?))", 0, "{\"a\": 1, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\"}");
+        assertRows(execute("SELECT k, udtval.a, udtval.b, udtval.c FROM %s WHERE k = ?", 0),
+                row(0, 1, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"), null)
+        );
+
+        assertInvalidMessage("Unknown field", "INSERT INTO %s (k, udtval) VALUES (?, fromJson(?))", 0, "{\"xxx\": 1}");
+        assertInvalidMessage("Unable to make int from",
+                "INSERT INTO %s (k, udtval) VALUES (?, fromJson(?))", 0, "{\"a\": \"foobar\"}");
+    }
+
+    @Test
+    public void testToJsonFct() throws Throwable
+    {
+        String typeName = createType("CREATE TYPE %s (a int, b uuid, c set<text>)");
+        createTable("CREATE TABLE %s (" +
+                "k int PRIMARY KEY, " +
+                "asciival ascii, " +
+                "bigintval bigint, " +
+                "blobval blob, " +
+                "booleanval boolean, " +
+                "dateval date, " +
+                "decimalval decimal, " +
+                "doubleval double, " +
+                "floatval float, " +
+                "inetval inet, " +
+                "intval int, " +
+                "smallintval smallint, " +
+                "textval text, " +
+                "timeval time, " +
+                "timestampval timestamp, " +
+                "timeuuidval timeuuid, " +
+                "tinyintval tinyint, " +
+                "uuidval uuid," +
+                "varcharval varchar, " +
+                "varintval varint, " +
+                "listval list<int>, " +
+                "frozenlistval frozen<list<int>>, " +
+                "setval set<uuid>, " +
+                "frozensetval frozen<set<uuid>>, " +
+                "mapval map<ascii, int>, " +
+                "frozenmapval frozen<map<ascii, int>>, " +
+                "tupleval frozen<tuple<int, ascii, uuid>>," +
+                "udtval frozen<" + typeName + ">)");
+
+        // toJson() can only be used in selections
+        assertInvalidMessage("toJson() may only be used within the selection clause",
+                "INSERT INTO %s (k, asciival) VALUES (?, toJson(?))", 0, 0);
+        assertInvalidMessage("toJson() may only be used within the selection clause",
+                "UPDATE %s SET asciival = toJson(?) WHERE k = ?", 0, 0);
+        assertInvalidMessage("toJson() may only be used within the selection clause",
+                "DELETE FROM %s WHERE k = fromJson(toJson(?))", 0);
+
+        // ================ ascii ================
+        execute("INSERT INTO %s (k, asciival) VALUES (?, ?)", 0, "ascii text");
+        assertRows(execute("SELECT k, toJson(asciival) FROM %s WHERE k = ?", 0), row(0, "\"ascii text\""));
+
+        execute("INSERT INTO %s (k, asciival) VALUES (?, ?)", 0, "");
+        assertRows(execute("SELECT k, toJson(asciival) FROM %s WHERE k = ?", 0), row(0, "\"\""));
+
+        // ================ bigint ================
+        execute("INSERT INTO %s (k, bigintval) VALUES (?, ?)", 0, 123123123123L);
+        assertRows(execute("SELECT k, toJson(bigintval) FROM %s WHERE k = ?", 0), row(0, "123123123123"));
+
+        execute("INSERT INTO %s (k, bigintval) VALUES (?, ?)", 0, 0L);
+        assertRows(execute("SELECT k, toJson(bigintval) FROM %s WHERE k = ?", 0), row(0, "0"));
+
+        execute("INSERT INTO %s (k, bigintval) VALUES (?, ?)", 0, -123123123123L);
+        assertRows(execute("SELECT k, toJson(bigintval) FROM %s WHERE k = ?", 0), row(0, "-123123123123"));
+
+        // ================ blob ================
+        execute("INSERT INTO %s (k, blobval) VALUES (?, ?)", 0, ByteBufferUtil.bytes(1));
+        assertRows(execute("SELECT k, toJson(blobval) FROM %s WHERE k = ?", 0), row(0, "\"0x00000001\""));
+
+        execute("INSERT INTO %s (k, blobval) VALUES (?, ?)", 0, ByteBufferUtil.EMPTY_BYTE_BUFFER);
+        assertRows(execute("SELECT k, toJson(blobval) FROM %s WHERE k = ?", 0), row(0, "\"0x\""));
+
+        // ================ boolean ================
+        execute("INSERT INTO %s (k, booleanval) VALUES (?, ?)", 0, true);
+        assertRows(execute("SELECT k, toJson(booleanval) FROM %s WHERE k = ?", 0), row(0, "true"));
+
+        execute("INSERT INTO %s (k, booleanval) VALUES (?, ?)", 0, false);
+        assertRows(execute("SELECT k, toJson(booleanval) FROM %s WHERE k = ?", 0), row(0, "false"));
+
+        // ================ date ================
+        execute("INSERT INTO %s (k, dateval) VALUES (?, ?)", 0, SimpleDateSerializer.dateStringToDays("1987-03-23"));
+        assertRows(execute("SELECT k, toJson(dateval) FROM %s WHERE k = ?", 0), row(0, "\"1987-03-23\""));
+
+        // ================ decimal ================
+        execute("INSERT INTO %s (k, decimalval) VALUES (?, ?)", 0, new BigDecimal("123123.123123"));
+        assertRows(execute("SELECT k, toJson(decimalval) FROM %s WHERE k = ?", 0), row(0, "123123.123123"));
+
+        execute("INSERT INTO %s (k, decimalval) VALUES (?, ?)", 0, new BigDecimal("-1.23E-12"));
+        assertRows(execute("SELECT k, toJson(decimalval) FROM %s WHERE k = ?", 0), row(0, "-1.23E-12"));
+
+        // ================ double ================
+        execute("INSERT INTO %s (k, doubleval) VALUES (?, ?)", 0, 123123.123123d);
+        assertRows(execute("SELECT k, toJson(doubleval) FROM %s WHERE k = ?", 0), row(0, "123123.123123"));
+
+        execute("INSERT INTO %s (k, doubleval) VALUES (?, ?)", 0, 123123d);
+        assertRows(execute("SELECT k, toJson(doubleval) FROM %s WHERE k = ?", 0), row(0, "123123.0"));
+
+        // ================ float ================
+        execute("INSERT INTO %s (k, floatval) VALUES (?, ?)", 0, 123.123f);
+        assertRows(execute("SELECT k, toJson(floatval) FROM %s WHERE k = ?", 0), row(0, "123.123"));
+
+        execute("INSERT INTO %s (k, floatval) VALUES (?, ?)", 0, 123123f);
+        assertRows(execute("SELECT k, toJson(floatval) FROM %s WHERE k = ?", 0), row(0, "123123.0"));
+
+        // ================ inet ================
+        execute("INSERT INTO %s (k, inetval) VALUES (?, ?)", 0, InetAddress.getByName("127.0.0.1"));
+        assertRows(execute("SELECT k, toJson(inetval) FROM %s WHERE k = ?", 0), row(0, "\"127.0.0.1\""));
+
+        execute("INSERT INTO %s (k, inetval) VALUES (?, ?)", 0, InetAddress.getByName("::1"));
+        assertRows(execute("SELECT k, toJson(inetval) FROM %s WHERE k = ?", 0), row(0, "\"0:0:0:0:0:0:0:1\""));
+
+        // ================ int ================
+        execute("INSERT INTO %s (k, intval) VALUES (?, ?)", 0, 123123);
+        assertRows(execute("SELECT k, toJson(intval) FROM %s WHERE k = ?", 0), row(0, "123123"));
+
+        execute("INSERT INTO %s (k, intval) VALUES (?, ?)", 0, 0);
+        assertRows(execute("SELECT k, toJson(intval) FROM %s WHERE k = ?", 0), row(0, "0"));
+
+        execute("INSERT INTO %s (k, intval) VALUES (?, ?)", 0, -123123);
+        assertRows(execute("SELECT k, toJson(intval) FROM %s WHERE k = ?", 0), row(0, "-123123"));
+
+        // ================ smallint ================
+        execute("INSERT INTO %s (k, smallintval) VALUES (?, ?)", 0, (short) 32767);
+        assertRows(execute("SELECT k, toJson(smallintval) FROM %s WHERE k = ?", 0), row(0, "32767"));
+
+        execute("INSERT INTO %s (k, smallintval) VALUES (?, ?)", 0, (short) 0);
+        assertRows(execute("SELECT k, toJson(smallintval) FROM %s WHERE k = ?", 0), row(0, "0"));
+
+        execute("INSERT INTO %s (k, smallintval) VALUES (?, ?)", 0, (short) -32768);
+        assertRows(execute("SELECT k, toJson(smallintval) FROM %s WHERE k = ?", 0), row(0, "-32768"));
+
+        // ================ tinyint ================
+        execute("INSERT INTO %s (k, tinyintval) VALUES (?, ?)", 0, (byte) 127);
+        assertRows(execute("SELECT k, toJson(tinyintval) FROM %s WHERE k = ?", 0), row(0, "127"));
+
+        execute("INSERT INTO %s (k, tinyintval) VALUES (?, ?)", 0, (byte) 0);
+        assertRows(execute("SELECT k, toJson(tinyintval) FROM %s WHERE k = ?", 0), row(0, "0"));
+
+        execute("INSERT INTO %s (k, tinyintval) VALUES (?, ?)", 0, (byte) -128);
+        assertRows(execute("SELECT k, toJson(tinyintval) FROM %s WHERE k = ?", 0), row(0, "-128"));
+
+        // ================ text (varchar) ================
+        execute("INSERT INTO %s (k, textval) VALUES (?, ?)", 0, "");
+        assertRows(execute("SELECT k, toJson(textval) FROM %s WHERE k = ?", 0), row(0, "\"\""));
+
+        execute("INSERT INTO %s (k, textval) VALUES (?, ?)", 0, "abcd");
+        assertRows(execute("SELECT k, toJson(textval) FROM %s WHERE k = ?", 0), row(0, "\"abcd\""));
+
+        execute("INSERT INTO %s (k, textval) VALUES (?, ?)", 0, "\u8422");
+        assertRows(execute("SELECT k, toJson(textval) FROM %s WHERE k = ?", 0), row(0, "\"\u8422\""));
+
+        execute("INSERT INTO %s (k, textval) VALUES (?, ?)", 0, "\u0000");
+        assertRows(execute("SELECT k, toJson(textval) FROM %s WHERE k = ?", 0), row(0, "\"\\u0000\""));
+
+        // ================ timestamp ================
+        execute("INSERT INTO %s (k, timeval) VALUES (?, ?)", 0, 123L);
+        assertRows(execute("SELECT k, toJson(timeval) FROM %s WHERE k = ?", 0), row(0, "\"00:00:00.000000123\""));
+
+        // ================ timestamp ================
+        SimpleDateFormat sdf = new SimpleDateFormat("y-M-d");
+        sdf.setTimeZone(TimeZone.getTimeZone("UDT"));
+        execute("INSERT INTO %s (k, timestampval) VALUES (?, ?)", 0, sdf.parse("2014-01-01"));
+        assertRows(execute("SELECT k, toJson(timestampval) FROM %s WHERE k = ?", 0), row(0, "\"2014-01-01 00:00:00.000Z\""));
+
+        // ================ timeuuid ================
+        execute("INSERT INTO %s (k, timeuuidval) VALUES (?, ?)", 0, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"));
+        assertRows(execute("SELECT k, toJson(timeuuidval) FROM %s WHERE k = ?", 0), row(0, "\"6bddc89a-5644-11e4-97fc-56847afe9799\""));
+
+         // ================ uuidval ================
+        execute("INSERT INTO %s (k, uuidval) VALUES (?, ?)", 0, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"));
+        assertRows(execute("SELECT k, toJson(uuidval) FROM %s WHERE k = ?", 0), row(0, "\"6bddc89a-5644-11e4-97fc-56847afe9799\""));
+
+        // ================ varint ================
+        execute("INSERT INTO %s (k, varintval) VALUES (?, ?)", 0, new BigInteger("123123123123123123123"));
+        assertRows(execute("SELECT k, toJson(varintval) FROM %s WHERE k = ?", 0), row(0, "123123123123123123123"));
+
+        // ================ lists ================
+        execute("INSERT INTO %s (k, listval) VALUES (?, ?)", 0, list(1, 2, 3));
+        assertRows(execute("SELECT k, toJson(listval) FROM %s WHERE k = ?", 0), row(0, "[1, 2, 3]"));
+
+        execute("INSERT INTO %s (k, listval) VALUES (?, ?)", 0, list());
+        assertRows(execute("SELECT k, toJson(listval) FROM %s WHERE k = ?", 0), row(0, "null"));
+
+        // frozen
+        execute("INSERT INTO %s (k, frozenlistval) VALUES (?, ?)", 0, list(1, 2, 3));
+        assertRows(execute("SELECT k, toJson(frozenlistval) FROM %s WHERE k = ?", 0), row(0, "[1, 2, 3]"));
+
+        // ================ sets ================
+        execute("INSERT INTO %s (k, setval) VALUES (?, ?)",
+                0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))));
+        assertRows(execute("SELECT k, toJson(setval) FROM %s WHERE k = ?", 0),
+                row(0, "[\"6bddc89a-5644-11e4-97fc-56847afe9798\", \"6bddc89a-5644-11e4-97fc-56847afe9799\"]")
+        );
+
+        execute("INSERT INTO %s (k, setval) VALUES (?, ?)", 0, set());
+        assertRows(execute("SELECT k, toJson(setval) FROM %s WHERE k = ?", 0), row(0, "null"));
+
+        // frozen
+        execute("INSERT INTO %s (k, frozensetval) VALUES (?, ?)",
+                0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))));
+        assertRows(execute("SELECT k, toJson(frozensetval) FROM %s WHERE k = ?", 0),
+                row(0, "[\"6bddc89a-5644-11e4-97fc-56847afe9798\", \"6bddc89a-5644-11e4-97fc-56847afe9799\"]")
+        );
+
+        // ================ maps ================
+        execute("INSERT INTO %s (k, mapval) VALUES (?, ?)", 0, map("a", 1, "b", 2));
+        assertRows(execute("SELECT k, toJson(mapval) FROM %s WHERE k = ?", 0), row(0, "{\"a\": 1, \"b\": 2}"));
+
+        execute("INSERT INTO %s (k, mapval) VALUES (?, ?)", 0, map());
+        assertRows(execute("SELECT k, toJson(mapval) FROM %s WHERE k = ?", 0), row(0, "null"));
+
+        // frozen
+        execute("INSERT INTO %s (k, frozenmapval) VALUES (?, ?)", 0, map("a", 1, "b", 2));
+        assertRows(execute("SELECT k, toJson(frozenmapval) FROM %s WHERE k = ?", 0), row(0, "{\"a\": 1, \"b\": 2}"));
+
+        // ================ tuples ================
+        execute("INSERT INTO %s (k, tupleval) VALUES (?, ?)", 0, tuple(1, "foobar", UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")));
+        assertRows(execute("SELECT k, toJson(tupleval) FROM %s WHERE k = ?", 0),
+            row(0, "[1, \"foobar\", \"6bddc89a-5644-11e4-97fc-56847afe9799\"]")
+        );
+
+        execute("INSERT INTO %s (k, tupleval) VALUES (?, ?)", 0, tuple(1, "foobar", null));
+        assertRows(execute("SELECT k, toJson(tupleval) FROM %s WHERE k = ?", 0),
+                row(0, "[1, \"foobar\", null]")
+        );
+
+        // ================ UDTs ================
+        execute("INSERT INTO %s (k, udtval) VALUES (?, {a: ?, b: ?, c: ?})", 0, 1, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"), set("foo", "bar"));
+        assertRows(execute("SELECT k, toJson(udtval) FROM %s WHERE k = ?", 0),
+                row(0, "{\"a\": 1, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\", \"c\": [\"bar\", \"foo\"]}")
+        );
+
+        execute("INSERT INTO %s (k, udtval) VALUES (?, {a: ?, b: ?})", 0, 1, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"));
+        assertRows(execute("SELECT k, toJson(udtval) FROM %s WHERE k = ?", 0),
+                row(0, "{\"a\": 1, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\", \"c\": null}")
+        );
+    }
+
+    @Test
+    public void testSelectJsonSyntax() throws Throwable
+    {
+        // tests SELECT JSON statements
+        createTable("CREATE TABLE %s (k int primary key, v int)");
+        execute("INSERT INTO %s (k, v) VALUES (0, 0)");
+        execute("INSERT INTO %s (k, v) VALUES (1, 1)");
+
+        assertRows(execute("SELECT JSON * FROM %s"),
+                row("{\"k\": 0, \"v\": 0}"),
+                row("{\"k\": 1, \"v\": 1}")
+        );
+
+        assertRows(execute("SELECT JSON k, v FROM %s"),
+                row("{\"k\": 0, \"v\": 0}"),
+                row("{\"k\": 1, \"v\": 1}")
+        );
+
+        assertRows(execute("SELECT JSON v, k FROM %s"),
+                row("{\"v\": 0, \"k\": 0}"),
+                row("{\"v\": 1, \"k\": 1}")
+        );
+
+        assertRows(execute("SELECT JSON v as foo, k as bar FROM %s"),
+                row("{\"foo\": 0, \"bar\": 0}"),
+                row("{\"foo\": 1, \"bar\": 1}")
+        );
+
+        assertRows(execute("SELECT JSON ttl(v), k FROM %s"),
+                row("{\"ttl(v)\": null, \"k\": 0}"),
+                row("{\"ttl(v)\": null, \"k\": 1}")
+        );
+
+        assertRows(execute("SELECT JSON ttl(v) as foo, k FROM %s"),
+                row("{\"foo\": null, \"k\": 0}"),
+                row("{\"foo\": null, \"k\": 1}")
+        );
+
+        assertRows(execute("SELECT JSON count(*) FROM %s"),
+                row("{\"count\": 2}")
+        );
+
+        assertRows(execute("SELECT JSON count(*) as foo FROM %s"),
+                row("{\"foo\": 2}")
+        );
+
+        assertRows(execute("SELECT JSON toJson(blobAsInt(intAsBlob(v))) FROM %s LIMIT 1"),
+                row("{\"system.tojson(system.blobasint(system.intasblob(v)))\": \"0\"}")
+        );
+    }
+
+    @Test
+    public void testInsertJsonSyntax() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int primary key, v int)");
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"v\": 0}");
+        assertRows(execute("SELECT * FROM %s"),
+                row(0, 0)
+        );
+
+        // without specifying column names
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"v\": 0}");
+        assertRows(execute("SELECT * FROM %s"),
+                row(0, 0)
+        );
+
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"v\": null}");
+        assertRows(execute("SELECT * FROM %s"),
+                row(0, null)
+        );
+
+        execute("INSERT INTO %s JSON ?", "{\"v\": 1, \"k\": 0}");
+        assertRows(execute("SELECT * FROM %s"),
+                row(0, 1)
+        );
+
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0}");
+        assertRows(execute("SELECT * FROM %s"),
+                row(0, null)
+        );
+
+        if (USE_PREPARED_VALUES)
+            assertInvalidMessage("Got null for INSERT JSON values", "INSERT INTO %s JSON ?", new Object[]{null});
+
+        assertInvalidMessage("Got null for INSERT JSON values", "INSERT INTO %s JSON ?", "null");
+        assertInvalidMessage("Could not decode JSON string as a map", "INSERT INTO %s JSON ?", "\"notamap\"");
+        assertInvalidMessage("Could not decode JSON string as a map", "INSERT INTO %s JSON ?", "12.34");
+        assertInvalidMessage("JSON values map contains unrecognized column",
+                "INSERT INTO %s JSON ?",
+                "{\"k\": 0, \"v\": 0, \"zzz\": 0}");
+
+        assertInvalidMessage("Unable to make int from",
+                "INSERT INTO %s JSON ?",
+                "{\"k\": 0, \"v\": \"notanint\"}");
+    }
+
+    @Test
+    public void testCaseSensitivity() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int primary key, \"Foo\" int)");
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"\\\"Foo\\\"\": 0}");
+        execute("INSERT INTO %s JSON ?", "{\"K\": 0, \"\\\"Foo\\\"\": 0}");
+        execute("INSERT INTO %s JSON ?", "{\"\\\"k\\\"\": 0, \"\\\"Foo\\\"\": 0}");
+
+        // results should preserve and quote case-sensitive identifiers
+        assertRows(execute("SELECT JSON * FROM %s"), row("{\"k\": 0, \"\\\"Foo\\\"\": 0}"));
+        assertRows(execute("SELECT JSON k, \"Foo\" as foo FROM %s"), row("{\"k\": 0, \"foo\": 0}"));
+        assertRows(execute("SELECT JSON k, \"Foo\" as \"Bar\" FROM %s"), row("{\"k\": 0, \"\\\"Bar\\\"\": 0}"));
+
+        assertInvalid("INSERT INTO %s JSON ?", "{\"k\": 0, \"foo\": 0}");
+        assertInvalid("INSERT INTO %s JSON ?", "{\"k\": 0, \"\\\"foo\\\"\": 0}");
+
+        // user-defined types also need to handle case-sensitivity
+        String typeName = createType("CREATE TYPE %s (a int, \"Foo\" int)");
+        createTable("CREATE TABLE %s (k int primary key, v frozen<" + typeName + ">)");
+
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"v\": {\"a\": 0, \"\\\"Foo\\\"\": 0}}");
+        assertRows(execute("SELECT JSON k, v FROM %s"), row("{\"k\": 0, \"v\": {\"a\": 0, \"\\\"Foo\\\"\": 0}}"));
+
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"v\": {\"A\": 0, \"\\\"Foo\\\"\": 0}}");
+        assertRows(execute("SELECT JSON k, v FROM %s"), row("{\"k\": 0, \"v\": {\"a\": 0, \"\\\"Foo\\\"\": 0}}"));
+    }
+
+    @Test
+    public void testInsertJsonSyntaxWithCollections() throws Throwable
+    {
+        createTable("CREATE TABLE %s (" +
+                "k int PRIMARY KEY, " +
+                "m map<text, boolean>, " +
+                "mf frozen<map<text, boolean>>, " +
+                "s set<int>, " +
+                "sf frozen<set<int>>, " +
+                "l list<int>, " +
+                "lf frozen<list<int>>)");
+
+        // map
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"m\": {\"a\": true, \"b\": false}}");
+        assertRows(execute("SELECT k, m FROM %s"), row(0, map("a", true, "b", false)));
+
+        // frozen map
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"mf\": {\"a\": true, \"b\": false}}");
+        assertRows(execute("SELECT k, mf FROM %s"), row(0, map("a", true, "b", false)));
+
+        // set
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"s\": [3, 1, 2]}");
+        assertRows(execute("SELECT k, s FROM %s"), row(0, set(1, 2, 3)));
+
+        // frozen set
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"sf\": [3, 1, 2]}");
+        assertRows(execute("SELECT k, sf FROM %s"), row(0, set(1, 2, 3)));
+
+        // list
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"l\": [1, 2, 3]}");
+        assertRows(execute("SELECT k, l FROM %s"), row(0, list(1, 2, 3)));
+
+        // frozen list
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"lf\": [1, 2, 3]}");
+        assertRows(execute("SELECT k, lf FROM %s"), row(0, list(1, 2, 3)));
+    }
+
+    @Test
+    public void testInsertJsonSyntaxWithNonNativeMapKeys() throws Throwable
+    {
+        // JSON doesn't allow non-string keys, so we accept string representations of any type as map keys and
+        // return maps with string keys when necessary.
+
+        String typeName = createType("CREATE TYPE %s (a int)");
+        createTable("CREATE TABLE %s (" +
+                "k int PRIMARY KEY, " +
+                "intmap map<int, boolean>, " +
+                "bigintmap map<bigint, boolean>, " +
+                "varintmap map<varint, boolean>, " +
+                "smallintmap map<smallint, boolean>, " +
+                "tinyintmap map<tinyint, boolean>, " +
+                "booleanmap map<boolean, boolean>, " +
+                "floatmap map<float, boolean>, " +
+                "doublemap map<double, boolean>, " +
+                "decimalmap map<decimal, boolean>, " +
+                "tuplemap map<frozen<tuple<int, text>>, boolean>, " +
+                "udtmap map<frozen<" + typeName + ">, boolean>, " +
+                "setmap map<frozen<set<int>>, boolean>, " +
+                "listmap map<frozen<list<int>>, boolean>, " +
+                "textsetmap map<frozen<set<text>>, boolean>, " +
+                "nestedsetmap map<frozen<map<set<text>, text>>, boolean>, " +
+                "frozensetmap frozen<map<set<int>, boolean>>)");
+
+        // int keys
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"intmap\": {\"0\": true, \"1\": false}}");
+        assertRows(execute("SELECT JSON k, intmap FROM %s"), row("{\"k\": 0, \"intmap\": {\"0\": true, \"1\": false}}"));
+
+        // bigint keys
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"bigintmap\": {\"0\": true, \"1\": false}}");
+        assertRows(execute("SELECT JSON k, bigintmap FROM %s"), row("{\"k\": 0, \"bigintmap\": {\"0\": true, \"1\": false}}"));
+
+        // varint keys
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"varintmap\": {\"0\": true, \"1\": false}}");
+        assertRows(execute("SELECT JSON k, varintmap FROM %s"), row("{\"k\": 0, \"varintmap\": {\"0\": true, \"1\": false}}"));
+
+        // smallint keys
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"smallintmap\": {\"0\": true, \"1\": false}}");
+        assertRows(execute("SELECT JSON k, smallintmap FROM %s"), row("{\"k\": 0, \"smallintmap\": {\"0\": true, \"1\": false}}"));
+
+        // tinyint keys
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"tinyintmap\": {\"0\": true, \"1\": false}}");
+        assertRows(execute("SELECT JSON k, tinyintmap FROM %s"), row("{\"k\": 0, \"tinyintmap\": {\"0\": true, \"1\": false}}"));
+
+        // boolean keys
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"booleanmap\": {\"true\": true, \"false\": false}}");
+        assertRows(execute("SELECT JSON k, booleanmap FROM %s"), row("{\"k\": 0, \"booleanmap\": {\"false\": false, \"true\": true}}"));
+
+        // float keys
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"floatmap\": {\"1.23\": true, \"4.56\": false}}");
+        assertRows(execute("SELECT JSON k, floatmap FROM %s"), row("{\"k\": 0, \"floatmap\": {\"1.23\": true, \"4.56\": false}}"));
+
+        // double keys
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"doublemap\": {\"1.23\": true, \"4.56\": false}}");
+        assertRows(execute("SELECT JSON k, doublemap FROM %s"), row("{\"k\": 0, \"doublemap\": {\"1.23\": true, \"4.56\": false}}"));
+
+        // decimal keys
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"decimalmap\": {\"1.23\": true, \"4.56\": false}}");
+        assertRows(execute("SELECT JSON k, decimalmap FROM %s"), row("{\"k\": 0, \"decimalmap\": {\"1.23\": true, \"4.56\": false}}"));
+
+        // tuple<int, text> keys
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"tuplemap\": {\"[0, \\\"a\\\"]\": true, \"[1, \\\"b\\\"]\": false}}");
+        assertRows(execute("SELECT JSON k, tuplemap FROM %s"), row("{\"k\": 0, \"tuplemap\": {\"[0, \\\"a\\\"]\": true, \"[1, \\\"b\\\"]\": false}}"));
+
+        // UDT keys
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"udtmap\": {\"{\\\"a\\\": 0}\": true, \"{\\\"a\\\": 1}\": false}}");
+        assertRows(execute("SELECT JSON k, udtmap FROM %s"), row("{\"k\": 0, \"udtmap\": {\"{\\\"a\\\": 0}\": true, \"{\\\"a\\\": 1}\": false}}"));
+
+        // set<int> keys
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"setmap\": {\"[0, 1, 2]\": true, \"[3, 4, 5]\": false}}");
+        assertRows(execute("SELECT JSON k, setmap FROM %s"), row("{\"k\": 0, \"setmap\": {\"[0, 1, 2]\": true, \"[3, 4, 5]\": false}}"));
+
+        // list<int> keys
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"listmap\": {\"[0, 1, 2]\": true, \"[3, 4, 5]\": false}}");
+        assertRows(execute("SELECT JSON k, listmap FROM %s"), row("{\"k\": 0, \"listmap\": {\"[0, 1, 2]\": true, \"[3, 4, 5]\": false}}"));
+
+        // set<text> keys
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"textsetmap\": {\"[\\\"0\\\", \\\"1\\\"]\": true, \"[\\\"3\\\", \\\"4\\\"]\": false}}");
+        assertRows(execute("SELECT JSON k, textsetmap FROM %s"), row("{\"k\": 0, \"textsetmap\": {\"[\\\"0\\\", \\\"1\\\"]\": true, \"[\\\"3\\\", \\\"4\\\"]\": false}}"));
+
+        // map<set<text>, text> keys
+        String innerKey1 = "[\"0\", \"1\"]";
+        String fullKey1 = String.format("{\"%s\": \"%s\"}", Json.quoteAsJsonString(innerKey1), "a");
+        String stringKey1 = Json.quoteAsJsonString(fullKey1);
+        String innerKey2 = "[\"3\", \"4\"]";
+        String fullKey2 = String.format("{\"%s\": \"%s\"}", Json.quoteAsJsonString(innerKey2), "b");
+        String stringKey2 = Json.quoteAsJsonString(fullKey2);
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"nestedsetmap\": {\"" + stringKey1 + "\": true, \"" + stringKey2 + "\": false}}");
+        assertRows(execute("SELECT JSON k, nestedsetmap FROM %s"), row("{\"k\": 0, \"nestedsetmap\": {\"" + stringKey1 + "\": true, \"" + stringKey2 + "\": false}}"));
+
+        // set<int> keys in a frozen map
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"frozensetmap\": {\"[0, 1, 2]\": true, \"[3, 4, 5]\": false}}");
+        assertRows(execute("SELECT JSON k, frozensetmap FROM %s"), row("{\"k\": 0, \"frozensetmap\": {\"[0, 1, 2]\": true, \"[3, 4, 5]\": false}}"));
+    }
+
+    @Test
+    public void testInsertJsonSyntaxWithTuplesAndUDTs() throws Throwable
+    {
+        String typeName = createType("CREATE TYPE %s (a int, b frozen<set<int>>, c tuple<int, int>)");
+        createTable("CREATE TABLE %s (" +
+                "k int PRIMARY KEY, " +
+                "a frozen<" + typeName + ">, " +
+                "b tuple<int, boolean>)");
+
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"a\": {\"a\": 0, \"b\": [1, 2, 3], \"c\": [0, 1]}, \"b\": [0, true]}");
+        assertRows(execute("SELECT k, a.a, a.b, a.c, b FROM %s"), row(0, 0, set(1, 2, 3), tuple(0, 1), tuple(0, true)));
+
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"a\": {\"a\": 0, \"b\": [1, 2, 3], \"c\": null}, \"b\": null}");
+        assertRows(execute("SELECT k, a.a, a.b, a.c, b FROM %s"), row(0, 0, set(1, 2, 3), null, null));
+    }
+
+    // done for CASSANDRA-11146
+    @Test
+    public void testAlterUDT() throws Throwable
+    {
+        String typeName = createType("CREATE TYPE %s (a int)");
+        createTable("CREATE TABLE %s (" +
+                "k int PRIMARY KEY, " +
+                "a frozen<" + typeName + ">)");
+
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"a\": {\"a\": 0}}");
+        assertRows(execute("SELECT JSON * FROM %s"), row("{\"k\": 0, \"a\": {\"a\": 0}}"));
+
+        schemaChange("ALTER TYPE " + KEYSPACE + "." + typeName + " ADD b boolean");
+        assertRows(execute("SELECT JSON * FROM %s"), row("{\"k\": 0, \"a\": {\"a\": 0, \"b\": null}}"));
+
+        execute("INSERT INTO %s JSON ?", "{\"k\": 0, \"a\": {\"a\": 0, \"b\": true}}");
+        assertRows(execute("SELECT JSON * FROM %s"), row("{\"k\": 0, \"a\": {\"a\": 0, \"b\": true}}"));
+    }
+
+    // done for CASSANDRA-11048
+    @Test
+    public void testJsonThreadSafety() throws Throwable
+    {
+        int numThreads = 10;
+        final int numRows = 5000;
+
+        createTable("CREATE TABLE %s (" +
+                "k text PRIMARY KEY, " +
+                "v text)");
+
+        for (int i = 0; i < numRows; i++)
+            execute("INSERT INTO %s (k, v) VALUES (?, ?)", "" + i, "" + i);
+
+        long seed = System.nanoTime();
+        System.out.println("Seed " + seed);
+        final Random rand = new Random(seed);
+
+        final Runnable worker = new Runnable()
+        {
+            @Override
+            public void run()
+            {
+                try
+                {
+                    for (int i = 0; i < numRows; i++)
+                    {
+                        String key = "" + rand.nextInt(numRows);
+                        assertRows(execute("SELECT JSON * FROM %s WHERE k = ?", key),
+                                row(String.format("{\"k\": \"%s\", \"v\": \"%s\"}", key, key)));
+                    }
+                }
+                catch (Throwable exc)
+                {
+                    exc.printStackTrace();
+                    fail(exc.getMessage());
+                }
+            }
+        };
+
+        ExecutorService executor = Executors.newFixedThreadPool(numThreads);
+        List<Future> futures = new ArrayList<>();
+        for (int i = 0; i < numThreads; i++)
+            futures.add(executor.submit(worker));
+
+        for (Future future : futures)
+            future.get(30, TimeUnit.SECONDS);
+
+        executor.shutdown();
+        Assert.assertTrue(executor.awaitTermination(30, TimeUnit.SECONDS));
+    }
+}
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexOnMapEntriesTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexOnMapEntriesTest.java
new file mode 100644
index 0000000..fb0d027
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexOnMapEntriesTest.java
@@ -0,0 +1,348 @@
+/*
+ * 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.cql3.validation.entities;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.commons.lang3.StringUtils;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class SecondaryIndexOnMapEntriesTest extends CQLTester
+{
+    @BeforeClass
+    public static void setUp()
+    {
+        DatabaseDescriptor.setPartitioner(ByteOrderedPartitioner.instance);
+    }
+
+    @Test
+    public void testShouldNotCreateIndexOnFrozenMaps() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k TEXT PRIMARY KEY, v FROZEN<MAP<TEXT, TEXT>>)");
+        assertIndexInvalidForColumn("v");
+    }
+
+    @Test
+    public void testShouldNotCreateIndexOnNonMapTypes() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k TEXT PRIMARY KEY, i INT, t TEXT, b BLOB, s SET<TEXT>, l LIST<TEXT>, tu TUPLE<TEXT>)");
+        assertIndexInvalidForColumn("i");
+        assertIndexInvalidForColumn("t");
+        assertIndexInvalidForColumn("b");
+        assertIndexInvalidForColumn("s");
+        assertIndexInvalidForColumn("l");
+        assertIndexInvalidForColumn("tu");
+    }
+
+    @Test
+    public void testShouldValidateMapKeyAndValueTypes() throws Throwable
+    {
+        createSimpleTableAndIndex();
+
+        String query = "SELECT * FROM %s WHERE v[?] = ?";
+        Object validKey = "valid key";
+        Object invalidKey = 31415;
+        Object validValue = 31415;
+        Object invalidValue = "invalid value";
+        assertInvalid(query, invalidKey, invalidValue);
+        assertInvalid(query, invalidKey, validValue);
+        assertInvalid(query, validKey, invalidValue);
+        assertReturnsNoRows(query, validKey, validValue);
+    }
+
+    @Test
+    public void testShouldFindRowsMatchingSingleEqualityRestriction() throws Throwable
+    {
+        createSimpleTableAndIndex();
+        Object[] foo = insertIntoSimpleTable("foo", map("a", 1,
+                                                        "c", 3));
+        Object[] bar = insertIntoSimpleTable("bar", map("a", 1,
+                                                        "b", 2));
+        Object[] baz = insertIntoSimpleTable("baz", map("b", 2,
+                                                        "c", 5,
+                                                        "d", 4));
+        Object[] qux = insertIntoSimpleTable("qux", map("b", 2,
+                                                        "d", 4));
+
+        assertRowsForConditions(entry("a", 1), bar, foo);
+        assertRowsForConditions(entry("b", 2), bar, baz, qux);
+        assertRowsForConditions(entry("c", 3), foo);
+        assertRowsForConditions(entry("c", 5), baz);
+        assertRowsForConditions(entry("d", 4), baz, qux);
+    }
+
+    @Test
+    public void testRequireFilteringDirectiveIfMultipleRestrictionsSpecified() throws Throwable
+    {
+        createSimpleTableAndIndex();
+        String baseQuery = "SELECT * FROM %s WHERE v['foo'] = 31415 AND v['baz'] = 31416";
+        assertInvalid(baseQuery);
+        assertReturnsNoRows(baseQuery + " ALLOW FILTERING");
+    }
+
+    @Test
+    public void testShouldFindRowsMatchingMultipleEqualityRestrictions() throws Throwable
+    {
+        createSimpleTableAndIndex();
+
+        Object[] foo = insertIntoSimpleTable("foo", map("k1", 1));
+        Object[] bar = insertIntoSimpleTable("bar", map("k1", 1,
+                                                        "k2", 2));
+        Object[] baz = insertIntoSimpleTable("baz", map("k2", 2,
+                                                        "k3", 3));
+        Object[] qux = insertIntoSimpleTable("qux", map("k2", 2,
+                                                        "k3", 3,
+                                                        "k4", 4));
+
+        assertRowsForConditions(entry("k1", 1),
+                                bar, foo);
+        assertRowsForConditions(entry("k1", 1).entry("k2", 2),
+                                bar);
+        assertNoRowsForConditions(entry("k1", 1).entry("k2", 2).entry("k3", 3));
+        assertRowsForConditions(entry("k2", 2).entry("k3", 3),
+                                baz, qux);
+        assertRowsForConditions(entry("k2", 2).entry("k3", 3).entry("k4", 4),
+                                qux);
+        assertRowsForConditions(entry("k3", 3).entry("k4", 4),
+                                qux);
+        assertNoRowsForConditions(entry("k3", 3).entry("k4", 4).entry("k5", 5));
+    }
+
+    @Test
+    public void testShouldFindRowsMatchingEqualityAndContainsRestrictions() throws Throwable
+    {
+        createSimpleTableAndIndex();
+
+        Object[] foo = insertIntoSimpleTable("foo", map("common", 31415,
+                                                        "k1", 1,
+                                                        "k2", 2,
+                                                        "k3", 3));
+        Object[] bar = insertIntoSimpleTable("bar", map("common", 31415,
+                                                        "k3", 3,
+                                                        "k4", 4,
+                                                        "k5", 5));
+        Object[] baz = insertIntoSimpleTable("baz", map("common", 31415,
+                                                        "k5", 5,
+                                                        "k6", 6,
+                                                        "k7", 7));
+
+        assertRowsForConditions(entry("common", 31415),
+                                bar, baz, foo);
+        assertRowsForConditions(entry("common", 31415).key("k1"),
+                                foo);
+        assertRowsForConditions(entry("common", 31415).key("k2"),
+                                foo);
+        assertRowsForConditions(entry("common", 31415).key("k3"),
+                                bar, foo);
+        assertRowsForConditions(entry("common", 31415).key("k3").value(2),
+                                foo);
+        assertRowsForConditions(entry("common", 31415).key("k3").value(3),
+                                bar, foo);
+        assertRowsForConditions(entry("common", 31415).key("k3").value(4),
+                                bar);
+        assertRowsForConditions(entry("common", 31415).key("k3").key("k5"),
+                                bar);
+        assertRowsForConditions(entry("common", 31415).key("k5"),
+                                bar, baz);
+        assertRowsForConditions(entry("common", 31415).key("k5").value(4),
+                                bar);
+        assertRowsForConditions(entry("common", 31415).key("k5").value(5),
+                                bar, baz);
+        assertRowsForConditions(entry("common", 31415).key("k5").value(6),
+                                baz);
+        assertNoRowsForConditions(entry("common", 31415).key("k5").value(8));
+    }
+
+    @Test
+    public void testShouldNotAcceptUnsupportedRelationsOnEntries() throws Throwable
+    {
+        createSimpleTableAndIndex();
+        assertInvalidRelation("< 31415");
+        assertInvalidRelation("<= 31415");
+        assertInvalidRelation("> 31415");
+        assertInvalidRelation(">= 31415");
+        assertInvalidRelation("IN (31415, 31416, 31417)");
+        assertInvalidRelation("CONTAINS 31415");
+        assertInvalidRelation("CONTAINS KEY 'foo'");
+    }
+
+    @Test
+    public void testShouldRecognizeAlteredOrDeletedMapEntries() throws Throwable
+    {
+        createSimpleTableAndIndex();
+        Object[] foo = insertIntoSimpleTable("foo", map("common", 31415,
+                                                        "target", 8192));
+        Object[] bar = insertIntoSimpleTable("bar", map("common", 31415,
+                                                        "target", 8192));
+        Object[] baz = insertIntoSimpleTable("baz", map("common", 31415,
+                                                        "target", 8192));
+
+        assertRowsForConditions(entry("target", 8192),
+                                bar, baz, foo);
+        baz = updateMapInSimpleTable(baz, "target", 4096);
+        assertRowsForConditions(entry("target", 8192),
+                                bar, foo);
+        bar = updateMapInSimpleTable(bar, "target", null);
+        assertRowsForConditions(entry("target", 8192),
+                                foo);
+        execute("DELETE FROM %s WHERE k = 'foo'");
+        assertNoRowsForConditions(entry("target", 8192));
+        assertRowsForConditions(entry("common", 31415),
+                                bar, baz);
+        assertRowsForConditions(entry("target", 4096),
+                                baz);
+    }
+
+    @Test
+    public void testShouldRejectQueriesForNullEntries() throws Throwable
+    {
+        createSimpleTableAndIndex();
+        assertInvalid("SELECT * FROM %s WHERE v['somekey'] = null");
+    }
+
+    @Test
+    public void testShouldTreatQueriesAgainstFrozenMapIndexesAsInvalid() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k TEXT PRIMARY KEY, v FROZEN<MAP<TEXT, TEXT>>)");
+        createIndex("CREATE INDEX ON %s(FULL(V))");
+
+        try
+        {
+            execute("SELECT * FROM %s WHERE v['somekey'] = 'somevalue'");
+            fail("Expected index query to fail");
+        }
+        catch (InvalidRequestException e)
+        {
+            String expectedMessage = "Map-entry equality predicates on frozen map column v are not supported";
+            assertTrue("Expected error message to contain '" + expectedMessage + "' but got '" +
+                       e.getMessage() + "'", e.getMessage().contains(expectedMessage));
+        }
+    }
+
+    private void assertIndexInvalidForColumn(String colname) throws Throwable
+    {
+        String query = String.format("CREATE INDEX ON %%s(ENTRIES(%s))", colname);
+        assertInvalid(query);
+    }
+
+    private void assertReturnsNoRows(String query, Object... params) throws Throwable
+    {
+        assertRows(execute(query, params));
+    }
+
+    private void createSimpleTableAndIndex() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k TEXT PRIMARY KEY, v MAP<TEXT, INT>)");
+        createIndex("CREATE INDEX ON %s(ENTRIES(v))");
+    }
+
+    private Object[] insertIntoSimpleTable(String key, Object value) throws Throwable
+    {
+        String query = "INSERT INTO %s (k, v) VALUES (?, ?)";
+        execute(query, key, value);
+        return row(key, value);
+    }
+
+    private void assertRowsForConditions(IndexWhereClause whereClause, Object[]... rows) throws Throwable
+    {
+        assertRows(execute("SELECT * FROM %s WHERE " + whereClause.text(), whereClause.params()), rows);
+    }
+
+    private void assertNoRowsForConditions(IndexWhereClause whereClause) throws Throwable
+    {
+        assertRowsForConditions(whereClause);
+    }
+
+    private void assertInvalidRelation(String rel) throws Throwable
+    {
+        String query = "SELECT * FROM %s WHERE v " + rel;
+        assertInvalid(query);
+    }
+
+    private Object[] updateMapInSimpleTable(Object[] row, String mapKey, Integer mapValue) throws Throwable
+    {
+        execute("UPDATE %s SET v[?] = ? WHERE k = ?", mapKey, mapValue, row[0]);
+        UntypedResultSet rawResults = execute("SELECT * FROM %s WHERE k = ?", row[0]);
+        Map<Object, Object> value = (Map<Object, Object>)row[1];
+        if (mapValue == null)
+        {
+            value.remove(mapKey);
+        }
+        else
+        {
+            value.put(mapKey, mapValue);
+        }
+        return row;
+    }
+
+    private IndexWhereClause entry(Object key, Object value)
+    {
+        return (new IndexWhereClause()).entry(key, value);
+    }
+
+    private static final class IndexWhereClause
+    {
+        private final List<String> preds = new ArrayList<>();
+        private final List<Object> params = new ArrayList<>();
+
+        public IndexWhereClause entry(Object key, Object value)
+        {
+            preds.add("v[?] = ?");
+            params.add(key);
+            params.add(value);
+            return this;
+        }
+
+        public IndexWhereClause key(Object key)
+        {
+            preds.add("v CONTAINS KEY ?");
+            params.add(key);
+            return this;
+        }
+
+        public IndexWhereClause value(Object value)
+        {
+            preds.add("v CONTAINS ?");
+            params.add(value);
+            return this;
+        }
+
+        public String text()
+        {
+            if (preds.size() == 1)
+                return preds.get(0);
+            return StringUtils.join(preds, " AND ") + " ALLOW FILTERING";
+        }
+
+        public Object[] params()
+        {
+            return params.toArray();
+        }
+    }
+}
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
index 1e63e3a..e3616f6 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
@@ -17,23 +17,36 @@
  */
 package org.apache.cassandra.cql3.validation.entities;
 
-import java.nio.ByteBuffer;
-import java.util.HashMap;
-import java.util.Locale;
-import java.util.Map;
-import java.util.UUID;
-
-import org.apache.commons.lang.StringUtils;
-
-import org.junit.Test;
-
-import org.apache.cassandra.cql3.CQLTester;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.exceptions.SyntaxException;
-import org.apache.cassandra.utils.FBUtilities;
-
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.db.ColumnFamily;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.IndexExpression;
+import org.apache.cassandra.db.composites.CellName;
+import org.apache.cassandra.db.index.IndexNotAvailableException;
+import org.apache.cassandra.db.index.PerRowSecondaryIndex;
+import org.apache.cassandra.db.index.SecondaryIndexSearcher;
+import org.apache.cassandra.db.index.composites.CompositesSearcher;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.exceptions.SyntaxException;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.concurrent.OpOrder.Group;
+import org.apache.commons.lang3.StringUtils;
+import org.junit.Test;
+
 import static org.apache.cassandra.utils.ByteBufferUtil.EMPTY_BYTE_BUFFER;
 import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
 
@@ -65,7 +78,7 @@
     /**
      * Test creating and dropping an index with the specified name.
      *
-     * @param indexName the index name
+     * @param indexName         the index name
      * @param addKeyspaceOnDrop add the keyspace name in the drop statement
      * @throws Throwable if an error occurs
      */
@@ -86,7 +99,8 @@
         execute("INSERT INTO %s (a, b) values (?, ?);", 3, 1);
 
         assertRows(execute("SELECT * FROM %s where b = ?", 1), row(1, 1), row(3, 1));
-        assertInvalidMessage("Index '" + removeQuotes(indexName.toLowerCase(Locale.US)) + "' could not be found in any of the tables of keyspace 'system'", "DROP INDEX " + indexName);
+        assertInvalidMessage("Index '" + removeQuotes(indexName.toLowerCase(Locale.US)) + "' could not be found in any of the tables of keyspace 'system'",
+                             "DROP INDEX " + indexName);
 
         if (addKeyspaceOnDrop)
         {
@@ -95,10 +109,10 @@
         else
         {
             execute("USE " + KEYSPACE);
-            dropIndex("DROP INDEX " + indexName);
+            execute("DROP INDEX " + indexName);
         }
 
-        assertInvalidMessage("No secondary indexes on the restricted columns support the provided operators",
+        assertInvalidMessage("Predicates on non-primary-key columns (b) are not yet supported for non secondary index queries",
                              "SELECT * FROM %s where b = ?", 1);
         dropIndex("DROP INDEX IF EXISTS " + indexName);
         assertInvalidMessage("Index '" + removeQuotes(indexName.toLowerCase(Locale.US)) + "' could not be found", "DROP INDEX " + indexName);
@@ -667,6 +681,32 @@
     }
 
     @Test
+    public void testIndexQueriesWithIndexNotReady() throws Throwable
+    {
+        createTable("CREATE TABLE %s (pk int, ck int, value int, PRIMARY KEY (pk, ck))");
+
+        for (int i = 0; i < 10; i++)
+            for (int j = 0; j < 10; j++)
+                execute("INSERT INTO %s (pk, ck, value) VALUES (?, ?, ?)", i, j, i + j);
+
+        createIndex("CREATE CUSTOM INDEX testIndex ON %s (value) USING '" + IndexBlockingOnInitialization.class.getName()
+                + "'");
+        try
+        {
+            execute("SELECT value FROM %s WHERE value = 2");
+            fail();
+        }
+        catch (IndexNotAvailableException e)
+        {
+            assertTrue(true);
+        }
+        finally
+        {
+            execute("DROP index " + KEYSPACE + ".testIndex");
+        }
+    }
+
+    @Test
     public void testWithEmptyRestrictionValueAndSecondaryIndex() throws Throwable
     {
         createTable("CREATE TABLE %s (pk blob, c blob, v blob, PRIMARY KEY ((pk), c))");
@@ -861,4 +901,110 @@
         assertEmpty(execute("SELECT * FROM %s WHERE a = 3"));
         assertEmpty(execute("SELECT * FROM %s WHERE a = 5"));
     }
+
+    /**
+     * Custom index used to test the behavior of the system when the index is not ready.
+     * As Custom indices cannot by <code>PerColumnSecondaryIndex</code> we use a <code>PerRowSecondaryIndex</code>
+     * to avoid the check but return a <code>CompositesSearcher</code>.
+     */
+    public static class IndexBlockingOnInitialization extends PerRowSecondaryIndex
+    {
+        private volatile CountDownLatch latch = new CountDownLatch(1);
+
+        @Override
+        public void index(ByteBuffer rowKey, ColumnFamily cf)
+        {
+            try
+            {
+                latch.await();
+            }
+            catch (InterruptedException e)
+            {
+                Thread.interrupted();
+            }
+        }
+
+        @Override
+        public void delete(DecoratedKey key, Group opGroup)
+        {
+        }
+
+        @Override
+        public void init()
+        {
+        }
+
+        @Override
+        public void reload()
+        {
+        }
+
+        @Override
+        public void validateOptions() throws ConfigurationException
+        {
+        }
+
+        @Override
+        public String getIndexName()
+        {
+            return "testIndex";
+        }
+
+        @Override
+        protected SecondaryIndexSearcher createSecondaryIndexSearcher(Set<ByteBuffer> columns)
+        {
+            return new CompositesSearcher(baseCfs.indexManager, columns)
+            {
+                @Override
+                public boolean canHandleIndexClause(List<IndexExpression> clause)
+                {
+                    return true;
+                }
+
+                @Override
+                public void validate(IndexExpression indexExpression) throws InvalidRequestException
+                {
+                }
+            };
+        }
+
+        @Override
+        public void forceBlockingFlush()
+        {
+        }
+
+        @Override
+        public ColumnFamilyStore getIndexCfs()
+        {
+            return baseCfs;
+        }
+
+        @Override
+        public void removeIndex(ByteBuffer columnName)
+        {
+            latch.countDown();
+        }
+
+        @Override
+        public void invalidate()
+        {
+        }
+
+        @Override
+        public void truncateBlocking(long truncatedAt)
+        {
+        }
+
+        @Override
+        public boolean indexes(CellName name)
+        {
+            return false;
+        }
+
+        @Override
+        public long estimateResultRows()
+        {
+            return 0;
+        }
+    }
 }
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/TimestampTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/TimestampTest.java
index 95d194d..3e70cd0 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/TimestampTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/TimestampTest.java
@@ -144,4 +144,12 @@
                       "UPDATE %1$s SET c = c + 1 WHERE k = 0; " +
                       "APPLY BATCH");
     }
+
+    @Test
+    public void testInsertTimestampWithUnset() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, i int)");
+        execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TIMESTAMP ?", unset()); // treat as 'now'
+    }
+
 }
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java
index 177def7..0783dd1 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java
@@ -125,6 +125,19 @@
                              "INSERT INTO %s (k, t) VALUES (0, (2, 'foo', 3.1, 'bar'))");
     }
 
+    @Test
+    public void testTupleWithUnsetValues() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, t tuple<int, text, double>)");
+        // invalid positional field substitution
+        assertInvalidMessage("Invalid unset value for tuple field number 1",
+                             "INSERT INTO %s (k, t) VALUES(0, (3, ?, 2.1))", unset());
+
+        createIndex("CREATE INDEX tuple_index ON %s (t)");
+        // select using unset
+        assertInvalidMessage("Invalid unset value for tuple field number 0", "SELECT * FROM %s WHERE k = ? and t = (?,?,?)", unset(), unset(), unset(), unset());
+    }
+	
     /**
      * Test the syntax introduced by #4851,
      * migrated from cql_tests.py:TestCQL.tuple_notation_test()
@@ -180,4 +193,14 @@
                    row(0, 0, "b"),
                    row(0, 0, "c"));
     }
+
+    @Test
+    public void testInvalidInputForTuple() throws Throwable
+    {
+        createTable("CREATE TABLE %s(pk int PRIMARY KEY, t tuple<text, text>)");
+        assertInvalidMessage("Not enough bytes to read 0th component",
+                             "INSERT INTO %s (pk, t) VALUES (?, ?)", 1, "test");
+        assertInvalidMessage("Not enough bytes to read 0th component",
+                             "INSERT INTO %s (pk, t) VALUES (?, ?)", 1, Long.MAX_VALUE);
+    }
 }
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/TypeTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/TypeTest.java
index f23ce35..1354fb0 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/TypeTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/TypeTest.java
@@ -22,7 +22,6 @@
 
 import org.junit.Test;
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 public class TypeTest extends CQLTester
@@ -50,8 +49,13 @@
     public void testDateCompatibility() throws Throwable
     {
         createTable("CREATE TABLE %s (a int, b timestamp, c bigint, d varint, PRIMARY KEY (a, b, c, d))");
-        execute("INSERT INTO %s (a, b, c, d) VALUES (0, unixTimestampOf(now()), dateOf(now()), dateOf(now()))");
-        UntypedResultSet results = execute("SELECT * FROM %s WHERE a=0 AND b < unixTimestampOf(now())");
+
+        execute("INSERT INTO %s (a, b, c, d) VALUES (0, toUnixTimestamp(now()), toTimestamp(now()), toTimestamp(now()))");
+        UntypedResultSet results = execute("SELECT * FROM %s WHERE a=0 AND b < toUnixTimestamp(now())");
+        assertEquals(1, results.size());
+
+        execute("INSERT INTO %s (a, b, c, d) VALUES (1, unixTimestampOf(now()), dateOf(now()), dateOf(now()))");
+        results = execute("SELECT * FROM %s WHERE a=1 AND b < toUnixTimestamp(now())");
         assertEquals(1, results.size());
     }
 
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/UFAuthTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/UFAuthTest.java
new file mode 100644
index 0000000..b2b9946
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UFAuthTest.java
@@ -0,0 +1,717 @@
+/*
+ * 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.cql3.validation.entities;
+
+import java.lang.reflect.Field;
+import java.util.*;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableSet;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.auth.*;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.Attributes;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.functions.FunctionName;
+import org.apache.cassandra.cql3.functions.Functions;
+import org.apache.cassandra.cql3.statements.BatchStatement;
+import org.apache.cassandra.cql3.statements.ModificationStatement;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.exceptions.*;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.utils.Pair;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class UFAuthTest extends CQLTester
+{
+    private static final Logger logger = LoggerFactory.getLogger(UFAuthTest.class);
+
+    String roleName = "test_role";
+    AuthenticatedUser user;
+    RoleResource role;
+    ClientState clientState;
+
+    @BeforeClass
+    public static void setupAuthorizer()
+    {
+        try
+        {
+            IAuthorizer authorizer = new StubAuthorizer();
+            Field authorizerField = DatabaseDescriptor.class.getDeclaredField("authorizer");
+            authorizerField.setAccessible(true);
+            authorizerField.set(null, authorizer);
+            DatabaseDescriptor.setPermissionsValidity(0);
+        }
+        catch (IllegalAccessException | NoSuchFieldException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Before
+    public void setup() throws Throwable
+    {
+        ((StubAuthorizer) DatabaseDescriptor.getAuthorizer()).clear();
+        setupClientState();
+        setupTable("CREATE TABLE %s (k int, v1 int, v2 int, PRIMARY KEY (k, v1))");
+    }
+
+    @Test
+    public void functionInSelection() throws Throwable
+    {
+        String functionName = createSimpleFunction();
+        String cql = String.format("SELECT k, %s FROM %s WHERE k = 1;",
+                                   functionCall(functionName),
+                                   KEYSPACE + "." + currentTable());
+        assertPermissionsOnFunction(cql, functionName);
+    }
+
+    @Test
+    public void functionInSelectPKRestriction() throws Throwable
+    {
+        String functionName = createSimpleFunction();
+        String cql = String.format("SELECT * FROM %s WHERE k = %s",
+                                   KEYSPACE + "." + currentTable(),
+                                   functionCall(functionName));
+        assertPermissionsOnFunction(cql, functionName);
+    }
+
+    @Test
+    public void functionInSelectClusteringRestriction() throws Throwable
+    {
+        String functionName = createSimpleFunction();
+        String cql = String.format("SELECT * FROM %s WHERE k = 0 AND v1 = %s",
+                                   KEYSPACE + "." + currentTable(),
+                                   functionCall(functionName));
+        assertPermissionsOnFunction(cql, functionName);
+    }
+
+    @Test
+    public void functionInSelectInRestriction() throws Throwable
+    {
+        String functionName = createSimpleFunction();
+        String cql = String.format("SELECT * FROM %s WHERE k IN (%s, %s)",
+                                   KEYSPACE + "." + currentTable(),
+                                   functionCall(functionName),
+                                   functionCall(functionName));
+        assertPermissionsOnFunction(cql, functionName);
+    }
+
+    @Test
+    public void functionInSelectMultiColumnInRestriction() throws Throwable
+    {
+        setupTable("CREATE TABLE %s (k int, v1 int, v2 int, v3 int, PRIMARY KEY (k, v1, v2))");
+        String functionName = createSimpleFunction();
+        String cql = String.format("SELECT * FROM %s WHERE k=0 AND (v1, v2) IN ((%s, %s))",
+                                   KEYSPACE + "." + currentTable(),
+                                   functionCall(functionName),
+                                   functionCall(functionName));
+        assertPermissionsOnFunction(cql, functionName);
+    }
+
+    @Test
+    public void functionInSelectMultiColumnEQRestriction() throws Throwable
+    {
+        setupTable("CREATE TABLE %s (k int, v1 int, v2 int, v3 int, PRIMARY KEY (k, v1, v2))");
+        String functionName = createSimpleFunction();
+        String cql = String.format("SELECT * FROM %s WHERE k=0 AND (v1, v2) = (%s, %s)",
+                                   KEYSPACE + "." + currentTable(),
+                                   functionCall(functionName),
+                                   functionCall(functionName));
+        assertPermissionsOnFunction(cql, functionName);
+    }
+
+    @Test
+    public void functionInSelectMultiColumnSliceRestriction() throws Throwable
+    {
+        setupTable("CREATE TABLE %s (k int, v1 int, v2 int, v3 int, PRIMARY KEY (k, v1, v2))");
+        String functionName = createSimpleFunction();
+        String cql = String.format("SELECT * FROM %s WHERE k=0 AND (v1, v2) < (%s, %s)",
+                                   KEYSPACE + "." + currentTable(),
+                                   functionCall(functionName),
+                                   functionCall(functionName));
+        assertPermissionsOnFunction(cql, functionName);
+    }
+
+    @Test
+    public void functionInSelectTokenEQRestriction() throws Throwable
+    {
+        String functionName = createSimpleFunction();
+        String cql = String.format("SELECT * FROM %s WHERE token(k) = token(%s)",
+                                   KEYSPACE + "." + currentTable(),
+                                   functionCall(functionName));
+        assertPermissionsOnFunction(cql, functionName);
+    }
+
+    @Test
+    public void functionInSelectTokenSliceRestriction() throws Throwable
+    {
+        String functionName = createSimpleFunction();
+        String cql = String.format("SELECT * FROM %s WHERE token(k) < token(%s)",
+                                   KEYSPACE + "." + currentTable(),
+                                   functionCall(functionName));
+        assertPermissionsOnFunction(cql, functionName);
+    }
+
+    @Test
+    public void functionInPKForInsert() throws Throwable
+    {
+        String functionName = createSimpleFunction();
+        String cql = String.format("INSERT INTO %s (k, v1, v2) VALUES (%s, 0, 0)",
+                                   KEYSPACE + "." + currentTable(),
+                                   functionCall(functionName));
+        assertPermissionsOnFunction(cql, functionName);
+    }
+
+    @Test
+    public void functionInClusteringValuesForInsert() throws Throwable
+    {
+        String functionName = createSimpleFunction();
+        String cql = String.format("INSERT INTO %s (k, v1, v2) VALUES (0, %s, 0)",
+                                   KEYSPACE + "." + currentTable(),
+                                   functionCall(functionName));
+        assertPermissionsOnFunction(cql, functionName);
+    }
+
+    @Test
+    public void functionInPKForDelete() throws Throwable
+    {
+        String functionName = createSimpleFunction();
+        String cql = String.format("DELETE FROM %s WHERE k = %s",
+                                   KEYSPACE + "." + currentTable(),
+                                   functionCall(functionName));
+        assertPermissionsOnFunction(cql, functionName);
+    }
+
+    @Test
+    public void functionInClusteringValuesForDelete() throws Throwable
+    {
+        String functionName = createSimpleFunction();
+        String cql = String.format("DELETE FROM %s WHERE k = 0 AND v1 = %s",
+                                   KEYSPACE + "." + currentTable(),
+                                   functionCall(functionName));
+        assertPermissionsOnFunction(cql, functionName);
+    }
+
+    @Test
+    public void testBatchStatement() throws Throwable
+    {
+        List<ModificationStatement> statements = new ArrayList<>();
+        List<String> functions = new ArrayList<>();
+        for (int i = 0; i < 3; i++)
+        {
+            String functionName = createSimpleFunction();
+            ModificationStatement stmt =
+            (ModificationStatement) getStatement(String.format("INSERT INTO %s (k, v1, v2) " +
+                                                               "VALUES (%s, %s, %s)",
+                                                               KEYSPACE + "." + currentTable(),
+                                                               i, i, functionCall(functionName)));
+            functions.add(functionName);
+            statements.add(stmt);
+        }
+        BatchStatement batch = new BatchStatement(-1, BatchStatement.Type.LOGGED, statements, Attributes.none());
+        assertUnauthorized(batch, functions);
+
+        grantExecuteOnFunction(functions.get(0));
+        assertUnauthorized(batch, functions.subList(1, functions.size()));
+
+        grantExecuteOnFunction(functions.get(1));
+        assertUnauthorized(batch, functions.subList(2, functions.size()));
+
+        grantExecuteOnFunction(functions.get(2));
+        batch.checkAccess(clientState);
+    }
+
+    @Test
+    public void testNestedFunctions() throws Throwable
+    {
+        String innerFunctionName = createSimpleFunction();
+        String outerFunctionName = createFunction("int",
+                                                  "CREATE FUNCTION %s(input int) " +
+                                                  " CALLED ON NULL INPUT" +
+                                                  " RETURNS int" +
+                                                  " LANGUAGE java" +
+                                                  " AS 'return Integer.valueOf(0);'");
+        assertPermissionsOnNestedFunctions(innerFunctionName, outerFunctionName);
+    }
+
+    @Test
+    public void functionInRegularCondition() throws Throwable
+    {
+        String functionName = createSimpleFunction();
+        String cql = String.format("UPDATE %s SET v2 = 0 WHERE k = 0 AND v1 = 0 IF v2 = %s",
+                                   KEYSPACE + "." + currentTable(),
+                                   functionCall(functionName));
+        assertPermissionsOnFunction(cql, functionName);
+    }
+    @Test
+    public void functionInStaticColumnCondition() throws Throwable
+    {
+        setupTable("CREATE TABLE %s (k int, s int STATIC, v1 int, v2 int, PRIMARY KEY(k, v1))");
+        String functionName = createSimpleFunction();
+        String cql = String.format("UPDATE %s SET v2 = 0 WHERE k = 0 AND v1 = 0 IF s = %s",
+                                   KEYSPACE + "." + currentTable(),
+                                   functionCall(functionName));
+        assertPermissionsOnFunction(cql, functionName);
+    }
+
+    @Test
+    public void functionInCollectionLiteralCondition() throws Throwable
+    {
+        setupTable("CREATE TABLE %s (k int, v1 int, m_val map<int, int>, PRIMARY KEY(k))");
+        String functionName = createSimpleFunction();
+        String cql = String.format("UPDATE %s SET v1 = 0 WHERE k = 0 IF m_val = {%s : %s}",
+                                   KEYSPACE + "." + currentTable(),
+                                   functionCall(functionName),
+                                   functionCall(functionName));
+        assertPermissionsOnFunction(cql, functionName);
+    }
+
+    @Test
+    public void functionInCollectionElementCondition() throws Throwable
+    {
+        setupTable("CREATE TABLE %s (k int, v1 int, m_val map<int, int>, PRIMARY KEY(k))");
+        String functionName = createSimpleFunction();
+        String cql = String.format("UPDATE %s SET v1 = 0 WHERE k = 0 IF m_val[%s] = %s",
+                                   KEYSPACE + "." + currentTable(),
+                                   functionCall(functionName),
+                                   functionCall(functionName));
+        assertPermissionsOnFunction(cql, functionName);
+    }
+
+    @Test
+    public void systemFunctionsRequireNoExplicitPrivileges() throws Throwable
+    {
+        // with terminal arguments, so evaluated at prepare time
+        String cql = String.format("UPDATE %s SET v2 = 0 WHERE k = blobasint(intasblob(0))",
+                                   KEYSPACE + "." + currentTable());
+        getStatement(cql).checkAccess(clientState);
+
+        // with non-terminal arguments, so evaluated at execution
+        String functionName = createSimpleFunction();
+        grantExecuteOnFunction(functionName);
+        cql = String.format("UPDATE %s SET v2 = 0 WHERE k = blobasint(intasblob(%s))",
+                            KEYSPACE + "." + currentTable(),
+                            functionCall(functionName));
+        getStatement(cql).checkAccess(clientState);
+    }
+
+    @Test
+    public void requireExecutePermissionOnComponentFunctionsWhenDefiningAggregate() throws Throwable
+    {
+        String sFunc = createSimpleStateFunction();
+        String fFunc = createSimpleFinalFunction();
+        // aside from the component functions, we need CREATE on the keyspace's functions
+        DatabaseDescriptor.getAuthorizer().grant(AuthenticatedUser.SYSTEM_USER,
+                                                 ImmutableSet.of(Permission.CREATE),
+                                                 FunctionResource.keyspace(KEYSPACE),
+                                                 role);
+        String aggDef = String.format(aggregateCql(sFunc, fFunc),
+                                      KEYSPACE + ".aggregate_for_permissions_test");
+
+        assertUnauthorized(aggDef, sFunc, "int, int");
+        grantExecuteOnFunction(sFunc);
+
+        assertUnauthorized(aggDef, fFunc, "int");
+        grantExecuteOnFunction(fFunc);
+
+        getStatement(aggDef).checkAccess(clientState);
+    }
+
+    @Test
+    public void revokeExecutePermissionsOnAggregateComponents() throws Throwable
+    {
+        String sFunc = createSimpleStateFunction();
+        String fFunc = createSimpleFinalFunction();
+        String aggDef = aggregateCql(sFunc, fFunc);
+        grantExecuteOnFunction(sFunc);
+        grantExecuteOnFunction(fFunc);
+
+        String aggregate = createAggregate(KEYSPACE, "int", aggDef);
+        grantExecuteOnFunction(aggregate);
+
+        String cql = String.format("SELECT %s(v1) FROM %s",
+                                   aggregate,
+                                   KEYSPACE + "." + currentTable());
+        getStatement(cql).checkAccess(clientState);
+
+        // check that revoking EXECUTE permission on any one of the
+        // component functions means we lose the ability to execute it
+        revokeExecuteOnFunction(aggregate);
+        assertUnauthorized(cql, aggregate, "int");
+        grantExecuteOnFunction(aggregate);
+        getStatement(cql).checkAccess(clientState);
+
+        revokeExecuteOnFunction(sFunc);
+        assertUnauthorized(cql, sFunc, "int, int");
+        grantExecuteOnFunction(sFunc);
+        getStatement(cql).checkAccess(clientState);
+
+        revokeExecuteOnFunction(fFunc);
+        assertUnauthorized(cql, fFunc, "int");
+        grantExecuteOnFunction(fFunc);
+        getStatement(cql).checkAccess(clientState);
+    }
+
+    @Test
+    public void functionWrappingAggregate() throws Throwable
+    {
+        String outerFunc = createFunction("int",
+                                          "CREATE FUNCTION %s(input int) " +
+                                          "CALLED ON NULL INPUT " +
+                                          "RETURNS int " +
+                                          "LANGUAGE java " +
+                                          "AS 'return input;'");
+
+        String sFunc = createSimpleStateFunction();
+        String fFunc = createSimpleFinalFunction();
+        String aggDef = aggregateCql(sFunc, fFunc);
+        grantExecuteOnFunction(sFunc);
+        grantExecuteOnFunction(fFunc);
+
+        String aggregate = createAggregate(KEYSPACE, "int", aggDef);
+
+        String cql = String.format("SELECT %s(%s(v1)) FROM %s",
+                                   outerFunc,
+                                   aggregate,
+                                   KEYSPACE + "." + currentTable());
+
+        assertUnauthorized(cql, outerFunc, "int");
+        grantExecuteOnFunction(outerFunc);
+
+        assertUnauthorized(cql, aggregate, "int");
+        grantExecuteOnFunction(aggregate);
+
+        getStatement(cql).checkAccess(clientState);
+    }
+
+    @Test
+    public void aggregateWrappingFunction() throws Throwable
+    {
+        String innerFunc = createFunction("int",
+                                          "CREATE FUNCTION %s(input int) " +
+                                          "CALLED ON NULL INPUT " +
+                                          "RETURNS int " +
+                                          "LANGUAGE java " +
+                                          "AS 'return input;'");
+
+        String sFunc = createSimpleStateFunction();
+        String fFunc = createSimpleFinalFunction();
+        String aggDef = aggregateCql(sFunc, fFunc);
+        grantExecuteOnFunction(sFunc);
+        grantExecuteOnFunction(fFunc);
+
+        String aggregate = createAggregate(KEYSPACE, "int", aggDef);
+
+        String cql = String.format("SELECT %s(%s(v1)) FROM %s",
+                                   aggregate,
+                                   innerFunc,
+                                   KEYSPACE + "." + currentTable());
+
+        assertUnauthorized(cql, aggregate, "int");
+        grantExecuteOnFunction(aggregate);
+
+        assertUnauthorized(cql, innerFunc, "int");
+        grantExecuteOnFunction(innerFunc);
+
+        getStatement(cql).checkAccess(clientState);
+    }
+
+    private void assertPermissionsOnNestedFunctions(String innerFunction, String outerFunction) throws Throwable
+    {
+        String cql = String.format("SELECT k, %s FROM %s WHERE k=0",
+                                   functionCall(outerFunction, functionCall(innerFunction)),
+                                   KEYSPACE + "." + currentTable());
+        // fail fast with an UAE on the first function
+        assertUnauthorized(cql, outerFunction, "int");
+        grantExecuteOnFunction(outerFunction);
+
+        // after granting execute on the first function, still fail due to the inner function
+        assertUnauthorized(cql, innerFunction, "");
+        grantExecuteOnFunction(innerFunction);
+
+        // now execution of both is permitted
+        getStatement(cql).checkAccess(clientState);
+    }
+
+    private void assertPermissionsOnFunction(String cql, String functionName) throws Throwable
+    {
+        assertPermissionsOnFunction(cql, functionName, "");
+    }
+
+    private void assertPermissionsOnFunction(String cql, String functionName, String argTypes) throws Throwable
+    {
+        assertUnauthorized(cql, functionName, argTypes);
+        grantExecuteOnFunction(functionName);
+        getStatement(cql).checkAccess(clientState);
+    }
+
+    private void assertUnauthorized(BatchStatement batch, Iterable<String> functionNames) throws Throwable
+    {
+        try
+        {
+            batch.checkAccess(clientState);
+            fail("Expected an UnauthorizedException, but none was thrown");
+        }
+        catch (UnauthorizedException e)
+        {
+            String functions = String.format("(%s)", Joiner.on("|").join(functionNames));
+            assertTrue(e.getLocalizedMessage()
+                        .matches(String.format("User %s has no EXECUTE permission on <function %s\\(\\)> or any of its parents",
+                                               roleName,
+                                               functions)));
+        }
+    }
+
+    private void assertUnauthorized(String cql, String functionName, String argTypes) throws Throwable
+    {
+        try
+        {
+            getStatement(cql).checkAccess(clientState);
+            fail("Expected an UnauthorizedException, but none was thrown");
+        }
+        catch (UnauthorizedException e)
+        {
+            assertEquals(String.format("User %s has no EXECUTE permission on <function %s(%s)> or any of its parents",
+                                       roleName,
+                                       functionName,
+                                       argTypes),
+                         e.getLocalizedMessage());
+        }
+    }
+
+    private void grantExecuteOnFunction(String functionName)
+    {
+            DatabaseDescriptor.getAuthorizer().grant(AuthenticatedUser.SYSTEM_USER,
+                                                     ImmutableSet.of(Permission.EXECUTE),
+                                                     functionResource(functionName),
+                                                     role);
+    }
+
+    private void revokeExecuteOnFunction(String functionName)
+    {
+        DatabaseDescriptor.getAuthorizer().revoke(AuthenticatedUser.SYSTEM_USER,
+                                                  ImmutableSet.of(Permission.EXECUTE),
+                                                  functionResource(functionName),
+                                                  role);
+    }
+
+    void setupClientState()
+    {
+
+        try
+        {
+            role = RoleResource.role(roleName);
+            // use reflection to set the logged in user so that we don't need to
+            // bother setting up an IRoleManager
+            user = new AuthenticatedUser(roleName);
+            clientState = ClientState.forInternalCalls();
+            Field userField = ClientState.class.getDeclaredField("user");
+            userField.setAccessible(true);
+            userField.set(clientState, user);
+        }
+        catch (IllegalAccessException | NoSuchFieldException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    private void setupTable(String tableDef) throws Throwable
+    {
+        createTable(tableDef);
+        // test user needs SELECT & MODIFY on the table regardless of permissions on any function
+        DatabaseDescriptor.getAuthorizer().grant(AuthenticatedUser.SYSTEM_USER,
+                                                 ImmutableSet.of(Permission.SELECT, Permission.MODIFY),
+                                                 DataResource.table(KEYSPACE, currentTable()),
+                                                 RoleResource.role(user.getName()));
+    }
+
+    private String aggregateCql(String sFunc, String fFunc)
+    {
+        return "CREATE AGGREGATE %s(int) " +
+               "SFUNC " + shortFunctionName(sFunc) + " " +
+               "STYPE int " +
+               "FINALFUNC " + shortFunctionName(fFunc) + " " +
+               "INITCOND 0";
+    }
+
+    private String createSimpleStateFunction() throws Throwable
+    {
+        return createFunction("int, int",
+                              "CREATE FUNCTION %s(a int, b int) " +
+                              "CALLED ON NULL INPUT " +
+                              "RETURNS int " +
+                              "LANGUAGE java " +
+                              "AS 'return Integer.valueOf( (a != null ? a.intValue() : 0 ) + b.intValue());'");
+    }
+
+    private String createSimpleFinalFunction() throws Throwable
+    {
+        return createFunction("int",
+                              "CREATE FUNCTION %s(a int) " +
+                              "CALLED ON NULL INPUT " +
+                              "RETURNS int " +
+                              "LANGUAGE java " +
+                              "AS 'return a;'");
+    }
+
+    private String createSimpleFunction() throws Throwable
+    {
+        return createFunction("",
+                              "CREATE FUNCTION %s() " +
+                              "  CALLED ON NULL INPUT " +
+                              "  RETURNS int " +
+                              "  LANGUAGE java " +
+                              "  AS 'return Integer.valueOf(0);'");
+    }
+
+    private String createFunction(String argTypes, String functionDef) throws Throwable
+    {
+        return createFunction(KEYSPACE, argTypes, functionDef);
+    }
+
+    private CQLStatement getStatement(String cql)
+    {
+        return QueryProcessor.getStatement(cql, clientState).statement;
+    }
+
+    private FunctionResource functionResource(String functionName)
+    {
+        // Note that this is somewhat brittle as it assumes that function names are
+        // truly unique. As such, it will break in the face of overloading.
+        // It is here to avoid having to duplicate the functionality of CqlParser
+        // for transforming cql types into AbstractTypes
+        FunctionName fn = parseFunctionName(functionName);
+        List<Function> functions = Functions.find(fn);
+        assertEquals(String.format("Expected a single function definition for %s, but found %s",
+                                   functionName,
+                                   functions.size()),
+                     1, functions.size());
+        return FunctionResource.function(fn.keyspace, fn.name, functions.get(0).argTypes());
+    }
+
+    private String functionCall(String functionName, String...args)
+    {
+        return String.format("%s(%s)", functionName, Joiner.on(",").join(args));
+    }
+
+    static class StubAuthorizer implements IAuthorizer
+    {
+        Map<Pair<String, IResource>, Set<Permission>> userPermissions = new HashMap<>();
+
+        private void clear()
+        {
+            userPermissions.clear();
+        }
+
+        public Set<Permission> authorize(AuthenticatedUser user, IResource resource)
+        {
+            Pair<String, IResource> key = Pair.create(user.getName(), resource);
+            Set<Permission> perms = userPermissions.get(key);
+            return perms != null ? perms : Collections.<Permission>emptySet();
+        }
+
+        public void grant(AuthenticatedUser performer,
+                          Set<Permission> permissions,
+                          IResource resource,
+                          RoleResource grantee) throws RequestValidationException, RequestExecutionException
+        {
+            Pair<String, IResource> key = Pair.create(grantee.getRoleName(), resource);
+            Set<Permission> perms = userPermissions.get(key);
+            if (null == perms)
+            {
+                perms = new HashSet<>();
+                userPermissions.put(key, perms);
+            }
+            perms.addAll(permissions);
+        }
+
+        public void revoke(AuthenticatedUser performer,
+                           Set<Permission> permissions,
+                           IResource resource,
+                           RoleResource revokee) throws RequestValidationException, RequestExecutionException
+        {
+            Pair<String, IResource> key = Pair.create(revokee.getRoleName(), resource);
+            Set<Permission> perms = userPermissions.get(key);
+            if (null != perms)
+                perms.removeAll(permissions);
+            if (perms.isEmpty())
+                userPermissions.remove(key);
+        }
+
+        public Set<PermissionDetails> list(AuthenticatedUser performer,
+                                           Set<Permission> permissions,
+                                           IResource resource,
+                                           RoleResource grantee) throws RequestValidationException, RequestExecutionException
+        {
+            Pair<String, IResource> key = Pair.create(grantee.getRoleName(), resource);
+            Set<Permission> perms = userPermissions.get(key);
+            if (perms == null)
+                return Collections.emptySet();
+
+
+            Set<PermissionDetails> details = new HashSet<>();
+            for (Permission permission : perms)
+            {
+                if (permissions.contains(permission))
+                    details.add(new PermissionDetails(grantee.getRoleName(), resource, permission));
+            }
+            return details;
+        }
+
+        public void revokeAllFrom(RoleResource revokee)
+        {
+            for (Pair<String, IResource> key : userPermissions.keySet())
+                if (key.left.equals(revokee.getRoleName()))
+                    userPermissions.remove(key);
+        }
+
+        public void revokeAllOn(IResource droppedResource)
+        {
+            for (Pair<String, IResource> key : userPermissions.keySet())
+                if (key.right.equals(droppedResource))
+                    userPermissions.remove(key);
+
+        }
+
+        public Set<? extends IResource> protectedResources()
+        {
+            return Collections.emptySet();
+        }
+
+        public void validateConfiguration() throws ConfigurationException
+        {
+
+        }
+
+        public void setup()
+        {
+
+        }
+    }
+}
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/UFIdentificationTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/UFIdentificationTest.java
new file mode 100644
index 0000000..ecc9d47
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UFIdentificationTest.java
@@ -0,0 +1,378 @@
+/*
+ * 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.cql3.validation.entities;
+
+import java.util.*;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.Iterables;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.Attributes;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.statements.BatchStatement;
+import org.apache.cassandra.cql3.statements.ModificationStatement;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.service.ClientState;
+
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Checks the collection of Function objects returned by CQLStatement.getFunction
+ * matches expectations. This is intended to verify the various subcomponents of
+ * the statement (Operations, Terms, Restrictions, RestrictionSet, Selection,
+ * Selector, SelectorFactories etc) properly report any constituent functions.
+ * Some purely terminal functions are resolved at preparation, so those are not
+ * included in the reported list. They still need to be surveyed, to verify the
+ * calling client has the necessary permissions. UFAuthTest includes tests which
+ * verify this more thoroughly than we can here.
+ */
+public class UFIdentificationTest extends CQLTester
+{
+    private com.google.common.base.Function<Function, String> toFunctionNames = new com.google.common.base.Function<Function, String>()
+    {
+        public String apply(Function f)
+        {
+            return f.name().keyspace + "." + f.name().name;
+        }
+    };
+
+    String tFunc;
+    String iFunc;
+    String lFunc;
+    String sFunc;
+    String mFunc;
+    String uFunc;
+    String udtFunc;
+
+    String userType;
+
+    @Before
+    public void setup() throws Throwable
+    {
+        userType = KEYSPACE + "." + createType("CREATE TYPE %s (t text, i int)");
+
+        createTable("CREATE TABLE %s (" +
+                    "   key int, " +
+                    "   t_sc text STATIC," +
+                    "   i_cc int, " +
+                    "   t_cc text, " +
+                    "   i_val int," +
+                    "   l_val list<int>," +
+                    "   s_val set<int>," +
+                    "   m_val map<int, int>," +
+                    "   u_val timeuuid," +
+                    "   udt_val frozen<" + userType + ">," +
+                    "   PRIMARY KEY (key, i_cc, t_cc)" +
+                    ")");
+
+        tFunc = createEchoFunction("text");
+        iFunc = createEchoFunction("int");
+        lFunc = createEchoFunction("list<int>");
+        sFunc = createEchoFunction("set<int>");
+        mFunc = createEchoFunction("map<int, int>");
+        uFunc = createEchoFunction("timeuuid");
+        udtFunc = createEchoFunction(userType);
+    }
+
+    @Test
+    public void testSimpleModificationStatement() throws Throwable
+    {
+        assertFunctions(cql("INSERT INTO %s (key, t_sc) VALUES (0, %s)", functionCall(tFunc, "'foo'")), tFunc);
+        assertFunctions(cql("INSERT INTO %s (key, i_cc) VALUES (0, %s)", functionCall(iFunc, "1")), iFunc);
+        assertFunctions(cql("INSERT INTO %s (key, t_cc) VALUES (0, %s)", functionCall(tFunc, "'foo'")), tFunc);
+        assertFunctions(cql("INSERT INTO %s (key, i_val) VALUES (0, %s)", functionCall(iFunc, "1")), iFunc);
+        assertFunctions(cql("INSERT INTO %s (key, l_val) VALUES (0, %s)", functionCall(lFunc, "[1]")), lFunc);
+        assertFunctions(cql("INSERT INTO %s (key, s_val) VALUES (0, %s)", functionCall(sFunc, "{1}")), sFunc);
+        assertFunctions(cql("INSERT INTO %s (key, m_val) VALUES (0, %s)", functionCall(mFunc, "{1:1}")), mFunc);
+        assertFunctions(cql("INSERT INTO %s (key, udt_val) VALUES (0,%s)", functionCall(udtFunc, "{i : 1, t : 'foo'}")), udtFunc);
+        assertFunctions(cql("INSERT INTO %s (key, u_val) VALUES (0, %s)", functionCall(uFunc, "now()")), uFunc, "system.now");
+    }
+
+    @Test
+    public void testNonTerminalCollectionLiterals() throws Throwable
+    {
+        String iFunc2 = createEchoFunction("int");
+        String mapValue = String.format("{%s:%s}", functionCall(iFunc, "1"), functionCall(iFunc2, "1"));
+        assertFunctions(cql("INSERT INTO %s (key, m_val) VALUES (0, %s)", mapValue), iFunc, iFunc2);
+
+        String listValue = String.format("[%s]", functionCall(iFunc, "1"));
+        assertFunctions(cql("INSERT INTO %s (key, l_val) VALUES (0, %s)", listValue), iFunc);
+
+        String setValue = String.format("{%s}", functionCall(iFunc, "1"));
+        assertFunctions(cql("INSERT INTO %s (key, s_val) VALUES (0, %s)", setValue), iFunc);
+    }
+
+    @Test
+    public void testNonTerminalUDTLiterals() throws Throwable
+    {
+        String udtValue = String.format("{ i: %s, t : %s } ", functionCall(iFunc, "1"), functionCall(tFunc, "'foo'"));
+        assertFunctions(cql("INSERT INTO %s (key, udt_val) VALUES (0, %s)", udtValue), iFunc, tFunc);
+    }
+
+    @Test
+    public void testModificationStatementWithConditions() throws Throwable
+    {
+        assertFunctions(cql("UPDATE %s SET i_val=0 WHERE key=0 IF t_sc=%s", functionCall(tFunc, "'foo'")), tFunc);
+        assertFunctions(cql("UPDATE %s SET i_val=0 WHERE key=0 IF i_val=%s", functionCall(iFunc, "1")), iFunc);
+        assertFunctions(cql("UPDATE %s SET i_val=0 WHERE key=0 IF l_val=%s", functionCall(lFunc, "[1]")), lFunc);
+        assertFunctions(cql("UPDATE %s SET i_val=0 WHERE key=0 IF s_val=%s", functionCall(sFunc, "{1}")), sFunc);
+        assertFunctions(cql("UPDATE %s SET i_val=0 WHERE key=0 IF m_val=%s", functionCall(mFunc, "{1:1}")), mFunc);
+
+
+        String iFunc2 = createEchoFunction("int");
+        assertFunctions(cql("UPDATE %s SET i_val=0 WHERE key=0 IF i_val IN (%s, %S)",
+                            functionCall(iFunc, "1"),
+                            functionCall(iFunc2, "2")),
+                        iFunc, iFunc2);
+
+        assertFunctions(cql("UPDATE %s SET i_val=0 WHERE key=0 IF u_val=%s",
+                            functionCall(uFunc, "now()")),
+                        uFunc, "system.now");
+
+        // conditions on collection elements
+        assertFunctions(cql("UPDATE %s SET i_val=0 WHERE key=0 IF l_val[%s] = %s",
+                            functionCall(iFunc, "1"),
+                            functionCall(iFunc2, "1")),
+                        iFunc, iFunc2);
+        assertFunctions(cql("UPDATE %s SET i_val=0 WHERE key=0 IF m_val[%s] = %s",
+                            functionCall(iFunc, "1"),
+                            functionCall(iFunc2, "1")),
+                        iFunc, iFunc2);
+    }
+
+    @Test @Ignore
+    // Technically, attributes like timestamp and ttl are Terms so could potentially
+    // resolve to function calls (& so you can call getFunctions on them)
+    // However, this is currently disallowed by CQL syntax
+    public void testModificationStatementWithAttributesFromFunction() throws Throwable
+    {
+        String longFunc = createEchoFunction("bigint");
+        assertFunctions(cql("INSERT INTO %s (key, i_cc, t_cc, i_val) VALUES (0, 0, 'foo', 0) USING TIMESTAMP %s",
+                            functionCall(longFunc, "9999")),
+                        longFunc);
+
+        assertFunctions(cql("INSERT INTO %s (key, i_cc, t_cc, i_val) VALUES (0, 0, 'foo', 0) USING TTL %s",
+                            functionCall(iFunc, "8888")),
+                        iFunc);
+
+        assertFunctions(cql("INSERT INTO %s (key, i_cc, t_cc, i_val) VALUES (0, 0, 'foo', 0) USING TIMESTAMP %s AND TTL %s",
+                            functionCall(longFunc, "9999"), functionCall(iFunc, "8888")),
+                        longFunc, iFunc);
+    }
+
+    @Test
+    public void testModificationStatementWithNestedFunctions() throws Throwable
+    {
+        String iFunc2 = createEchoFunction("int");
+        String iFunc3 = createEchoFunction("int");
+        String iFunc4 = createEchoFunction("int");
+        String iFunc5 = createEchoFunction("int");
+        String iFunc6 = createEchoFunction("int");
+        String nestedFunctionCall = nestedFunctionCall(iFunc6, iFunc5,
+                                                       nestedFunctionCall(iFunc4, iFunc3,
+                                                                          nestedFunctionCall(iFunc2, iFunc, "1")));
+
+        assertFunctions(cql("DELETE FROM %s WHERE key=%s", nestedFunctionCall),
+                        iFunc, iFunc2, iFunc3, iFunc4, iFunc5, iFunc6);
+    }
+
+    @Test
+    public void testSelectStatementSimpleRestrictions() throws Throwable
+    {
+        assertFunctions(cql("SELECT i_val FROM %s WHERE key=%s", functionCall(iFunc, "1")), iFunc);
+        assertFunctions(cql("SELECT i_val FROM %s WHERE key=0 AND i_cc=%s AND t_cc='foo'", functionCall(iFunc, "1")), iFunc);
+        assertFunctions(cql("SELECT i_val FROM %s WHERE key=0 AND i_cc=0 AND t_cc=%s", functionCall(tFunc, "'foo'")), tFunc);
+
+        String iFunc2 = createEchoFunction("int");
+        String tFunc2 = createEchoFunction("text");
+        assertFunctions(cql("SELECT i_val FROM %s WHERE key=%s AND i_cc=%s AND t_cc=%s",
+                            functionCall(iFunc, "1"),
+                            functionCall(iFunc2, "1"),
+                            functionCall(tFunc2, "'foo'")),
+                        iFunc, iFunc2, tFunc2);
+    }
+
+    @Test
+    public void testSelectStatementRestrictionsWithNestedFunctions() throws Throwable
+    {
+        String iFunc2 = createEchoFunction("int");
+        String iFunc3 = createEchoFunction("int");
+        String iFunc4 = createEchoFunction("int");
+        String iFunc5 = createEchoFunction("int");
+        String iFunc6 = createEchoFunction("int");
+        String nestedFunctionCall = nestedFunctionCall(iFunc6, iFunc5,
+                                                       nestedFunctionCall(iFunc3, iFunc4,
+                                                                          nestedFunctionCall(iFunc, iFunc2, "1")));
+
+        assertFunctions(cql("SELECT i_val FROM %s WHERE key=%s", nestedFunctionCall),
+                        iFunc, iFunc2, iFunc3, iFunc4, iFunc5, iFunc6);
+    }
+
+    @Test
+    public void testNonTerminalTupleInSelectRestrictions() throws Throwable
+    {
+        assertFunctions(cql("SELECT i_val FROM %s WHERE key=0 AND (i_cc, t_cc) IN ((%s, %s))",
+                            functionCall(iFunc, "1"),
+                            functionCall(tFunc, "'foo'")),
+                        iFunc, tFunc);
+
+        assertFunctions(cql("SELECT i_val FROM %s WHERE key=0 AND (i_cc, t_cc) = (%s, %s)",
+                            functionCall(iFunc, "1"),
+                            functionCall(tFunc, "'foo'")),
+                        iFunc, tFunc);
+
+        assertFunctions(cql("SELECT i_val FROM %s WHERE key=0 AND (i_cc, t_cc) > (%s, %s)",
+                            functionCall(iFunc, "1"),
+                            functionCall(tFunc, "'foo'")),
+                        iFunc, tFunc);
+
+        assertFunctions(cql("SELECT i_val FROM %s WHERE key=0 AND (i_cc, t_cc) < (%s, %s)",
+                            functionCall(iFunc, "1"),
+                            functionCall(tFunc, "'foo'")),
+                        iFunc, tFunc);
+
+         assertFunctions(cql("SELECT i_val FROM %s WHERE key=0 AND (i_cc, t_cc) > (%s, %s) AND (i_cc, t_cc) < (%s, %s)",
+                            functionCall(iFunc, "1"),
+                            functionCall(tFunc, "'foo'"),
+                            functionCall(iFunc, "1"),
+                            functionCall(tFunc, "'foo'")),
+                         iFunc, tFunc);
+    }
+
+    @Test
+    public void testNestedFunctionInTokenRestriction() throws Throwable
+    {
+        String iFunc2 = createEchoFunction("int");
+        assertFunctions(cql("SELECT i_val FROM %s WHERE token(key) = token(%s)", functionCall(iFunc, "1")),
+                        "system.token", iFunc);
+        assertFunctions(cql("SELECT i_val FROM %s WHERE token(key) > token(%s)", functionCall(iFunc, "1")),
+                        "system.token", iFunc);
+        assertFunctions(cql("SELECT i_val FROM %s WHERE token(key) < token(%s)", functionCall(iFunc, "1")),
+                        "system.token", iFunc);
+        assertFunctions(cql("SELECT i_val FROM %s WHERE token(key) > token(%s) AND token(key) < token(%s)",
+                            functionCall(iFunc, "1"),
+                            functionCall(iFunc2, "1")),
+                        "system.token", iFunc, iFunc2);
+    }
+
+    @Test
+    public void testSelectStatementSimpleSelections() throws Throwable
+    {
+        String iFunc2 = createEchoFunction("int");
+        execute("INSERT INTO %s (key, i_cc, t_cc, i_val) VALUES (0, 0, 'foo', 0)");
+        assertFunctions(cql2("SELECT i_val, %s FROM %s WHERE key=0", functionCall(iFunc, "i_val")), iFunc);
+        assertFunctions(cql2("SELECT i_val, %s FROM %s WHERE key=0", nestedFunctionCall(iFunc, iFunc2, "i_val")), iFunc, iFunc2);
+    }
+
+    @Test
+    public void testSelectStatementNestedSelections() throws Throwable
+    {
+        String iFunc2 = createEchoFunction("int");
+        execute("INSERT INTO %s (key, i_cc, t_cc, i_val) VALUES (0, 0, 'foo', 0)");
+        assertFunctions(cql2("SELECT i_val, %s FROM %s WHERE key=0", functionCall(iFunc, "i_val")), iFunc);
+        assertFunctions(cql2("SELECT i_val, %s FROM %s WHERE key=0", nestedFunctionCall(iFunc, iFunc2, "i_val")), iFunc, iFunc2);
+    }
+
+    @Test
+    public void testBatchStatement() throws Throwable
+    {
+        String iFunc2 = createEchoFunction("int");
+        List<ModificationStatement> statements = new ArrayList<>();
+        statements.add(modificationStatement(cql("INSERT INTO %s (key, i_cc, t_cc) VALUES (%s, 0, 'foo')",
+                                                 functionCall(iFunc, "0"))));
+        statements.add(modificationStatement(cql("INSERT INTO %s (key, i_cc, t_cc) VALUES (1, %s, 'foo')",
+                                                 functionCall(iFunc2, "1"))));
+        statements.add(modificationStatement(cql("INSERT INTO %s (key, i_cc, t_cc) VALUES (2, 2, %s)",
+                                                 functionCall(tFunc, "'foo'"))));
+
+        BatchStatement batch = new BatchStatement(-1, BatchStatement.Type.LOGGED, statements, Attributes.none());
+        assertFunctions(batch, iFunc, iFunc2, tFunc);
+    }
+
+    @Test
+    public void testBatchStatementWithConditions() throws Throwable
+    {
+        List<ModificationStatement> statements = new ArrayList<>();
+        statements.add(modificationStatement(cql("UPDATE %s SET i_val = %s WHERE key=0 AND i_cc=0 and t_cc='foo' IF l_val = %s",
+                                                 functionCall(iFunc, "0"), functionCall(lFunc, "[1]"))));
+        statements.add(modificationStatement(cql("UPDATE %s SET i_val = %s WHERE key=0 AND i_cc=1 and t_cc='foo' IF s_val = %s",
+                                                 functionCall(iFunc, "0"), functionCall(sFunc, "{1}"))));
+
+        BatchStatement batch = new BatchStatement(-1, BatchStatement.Type.LOGGED, statements, Attributes.none());
+        assertFunctions(batch, iFunc, lFunc, sFunc);
+    }
+
+    private ModificationStatement modificationStatement(String cql)
+    {
+        return (ModificationStatement) QueryProcessor.getStatement(cql, ClientState.forInternalCalls()).statement;
+    }
+
+    private void assertFunctions(String cql, String... function)
+    {
+        CQLStatement stmt = QueryProcessor.getStatement(cql, ClientState.forInternalCalls()).statement;
+        assertFunctions(stmt, function);
+    }
+
+    private void assertFunctions(CQLStatement stmt, String... function)
+    {
+        Set<String> expected = com.google.common.collect.Sets.newHashSet(function);
+        Set<String> actual = com.google.common.collect.Sets.newHashSet(Iterables.transform(stmt.getFunctions(),
+                                                                                           toFunctionNames));
+        assertTrue(com.google.common.collect.Sets.symmetricDifference(expected, actual).isEmpty());
+    }
+
+    private String cql(String template, String... params)
+    {
+        String tableName = KEYSPACE + "." + currentTable();
+        return String.format(template, com.google.common.collect.Lists.asList(tableName, params).toArray());
+    }
+
+    // Alternative query builder - appends the table name to the supplied params,
+    // for stmts of the form "SELECT x, %s FROM %s WHERE y=0"
+    private String cql2(String template, String... params)
+    {
+        Object[] args = Arrays.copyOf(params, params.length + 1);
+        args[params.length] = KEYSPACE + "." + currentTable();
+        return String.format(template, args);
+    }
+
+    private String functionCall(String fName, String... args)
+    {
+        return String.format("%s(%s)", fName, Joiner.on(",").join(args));
+    }
+
+    private String nestedFunctionCall(String outer, String inner, String innerArgs)
+    {
+        return functionCall(outer, functionCall(inner, innerArgs));
+    }
+
+    private String createEchoFunction(String type) throws Throwable
+    {
+        return createFunction(KEYSPACE, type,
+           "CREATE FUNCTION %s(input " + type + ")" +
+           " CALLED ON NULL INPUT" +
+           " RETURNS " + type +
+           " LANGUAGE java" +
+           " AS ' return input;'");
+    }
+}
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/UFJavaTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/UFJavaTest.java
new file mode 100644
index 0000000..caef808
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UFJavaTest.java
@@ -0,0 +1,1177 @@
+/*
+ * 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.cql3.validation.entities;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.datastax.driver.core.DataType;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.TupleType;
+import com.datastax.driver.core.TupleValue;
+import com.datastax.driver.core.UDTValue;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.cql3.functions.FunctionName;
+import org.apache.cassandra.cql3.functions.Functions;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.exceptions.FunctionExecutionException;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.transport.Server;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class UFJavaTest extends CQLTester
+{
+    @BeforeClass
+    public static void setUp()
+    {
+        DatabaseDescriptor.setPartitioner(ByteOrderedPartitioner.instance);
+    }
+
+
+    @Test
+    public void testJavaFunctionNoParameters() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int primary key, val double)");
+
+        String functionBody = "\n  return 1L;\n";
+
+        String fName = createFunction(KEYSPACE, "",
+                                      "CREATE OR REPLACE FUNCTION %s() " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS bigint " +
+                                      "LANGUAGE JAVA\n" +
+                                      "AS '" +functionBody + "';");
+
+        assertRows(execute("SELECT language, body FROM system.schema_functions WHERE keyspace_name=? AND function_name=?",
+                           KEYSPACE, parseFunctionName(fName).name),
+                   row("java", functionBody));
+
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 1, 1d);
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 2, 2d);
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 3, 3d);
+        assertRows(execute("SELECT key, val, " + fName + "() FROM %s"),
+                   row(1, 1d, 1L),
+                   row(2, 2d, 1L),
+                   row(3, 3d, 1L)
+        );
+    }
+
+    @Test
+    public void testJavaFunctionInvalidBodies() throws Throwable
+    {
+        try
+        {
+            execute("CREATE OR REPLACE FUNCTION " + KEYSPACE + ".jfinv() " +
+                    "RETURNS NULL ON NULL INPUT " +
+                    "RETURNS bigint " +
+                    "LANGUAGE JAVA\n" +
+                    "AS '\n" +
+                    "foobarbaz" +
+                    "\n';");
+            Assert.fail();
+        }
+        catch (InvalidRequestException e)
+        {
+            Assert.assertTrue(e.getMessage(), e.getMessage().contains("Java source compilation failed"));
+            Assert.assertTrue(e.getMessage(), e.getMessage().contains("insert \";\" to complete BlockStatements"));
+        }
+
+        try
+        {
+            execute("CREATE OR REPLACE FUNCTION " + KEYSPACE + ".jfinv() " +
+                    "RETURNS NULL ON NULL INPUT " +
+                    "RETURNS bigint " +
+                    "LANGUAGE JAVA\n" +
+                    "AS '\n" +
+                    "foobarbaz;" +
+                    "\n';");
+            Assert.fail();
+        }
+        catch (InvalidRequestException e)
+        {
+            Assert.assertTrue(e.getMessage(), e.getMessage().contains("Java source compilation failed"));
+            Assert.assertTrue(e.getMessage(), e.getMessage().contains("foobarbaz cannot be resolved to a type"));
+        }
+    }
+
+    @Test
+    public void testJavaFunctionInvalidReturn() throws Throwable
+    {
+        assertInvalidMessage("system keyspace is not user-modifiable",
+                             "CREATE OR REPLACE FUNCTION jfir(val double) " +
+                             "RETURNS NULL ON NULL INPUT " +
+                             "RETURNS double " +
+                             "LANGUAGE JAVA\n" +
+                             "AS 'return 1L;';");
+    }
+
+    @Test
+    public void testJavaFunctionArgumentTypeMismatch() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int primary key, val bigint)");
+
+        String fName = createFunction(KEYSPACE, "double",
+                                      "CREATE OR REPLACE FUNCTION %s(val double)" +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS double " +
+                                      "LANGUAGE JAVA " +
+                                      "AS 'return Double.valueOf(val);';");
+
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 1, 1L);
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 2, 2L);
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 3, 3L);
+        assertInvalidMessage("val cannot be passed as argument 0 of function",
+                             "SELECT key, val, " + fName + "(val) FROM %s");
+    }
+
+    @Test
+    public void testJavaFunction() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int primary key, val double)");
+
+        String functionBody = '\n' +
+                              "  // parameter val is of type java.lang.Double\n" +
+                              "  /* return type is of type java.lang.Double */\n" +
+                              "  if (val == null) {\n" +
+                              "    return null;\n" +
+                              "  }\n" +
+                              "  return Math.sin(val);\n";
+
+        String fName = createFunction(KEYSPACE, "double",
+                                      "CREATE OR REPLACE FUNCTION %s(val double) " +
+                                      "CALLED ON NULL INPUT " +
+                                      "RETURNS double " +
+                                      "LANGUAGE JAVA " +
+                                      "AS '" + functionBody + "';");
+
+        FunctionName fNameName = parseFunctionName(fName);
+
+        assertRows(execute("SELECT language, body FROM system.schema_functions WHERE keyspace_name=? AND function_name=?",
+                           fNameName.keyspace, fNameName.name),
+                   row("java", functionBody));
+
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 1, 1d);
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 2, 2d);
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 3, 3d);
+        assertRows(execute("SELECT key, val, " + fName + "(val) FROM %s"),
+                   row(1, 1d, Math.sin(1d)),
+                   row(2, 2d, Math.sin(2d)),
+                   row(3, 3d, Math.sin(3d))
+        );
+    }
+
+    @Test
+    public void testJavaFunctionCounter() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int primary key, val counter)");
+
+        String fName = createFunction(KEYSPACE, "counter",
+                                      "CREATE OR REPLACE FUNCTION %s(val counter) " +
+                                      "CALLED ON NULL INPUT " +
+                                      "RETURNS bigint " +
+                                      "LANGUAGE JAVA " +
+                                      "AS 'return val + 1;';");
+
+        execute("UPDATE %s SET val = val + 1 WHERE key = 1");
+        assertRows(execute("SELECT key, val, " + fName + "(val) FROM %s"),
+                   row(1, 1L, 2L));
+        execute("UPDATE %s SET val = val + 1 WHERE key = 1");
+        assertRows(execute("SELECT key, val, " + fName + "(val) FROM %s"),
+                   row(1, 2L, 3L));
+        execute("UPDATE %s SET val = val + 2 WHERE key = 1");
+        assertRows(execute("SELECT key, val, " + fName + "(val) FROM %s"),
+                   row(1, 4L, 5L));
+        execute("UPDATE %s SET val = val - 2 WHERE key = 1");
+        assertRows(execute("SELECT key, val, " + fName + "(val) FROM %s"),
+                   row(1, 2L, 3L));
+    }
+
+    @Test
+    public void testJavaKeyspaceFunction() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int primary key, val double)");
+
+        String functionBody = '\n' +
+                              "  // parameter val is of type java.lang.Double\n" +
+                              "  /* return type is of type java.lang.Double */\n" +
+                              "  if (val == null) {\n" +
+                              "    return null;\n" +
+                              "  }\n" +
+                              "  return Math.sin( val );\n";
+
+        String fName = createFunction(KEYSPACE_PER_TEST, "double",
+                                      "CREATE OR REPLACE FUNCTION %s(val double) " +
+                                      "CALLED ON NULL INPUT " +
+                                      "RETURNS double " +
+                                      "LANGUAGE JAVA " +
+                                      "AS '" + functionBody + "';");
+
+        FunctionName fNameName = parseFunctionName(fName);
+
+        assertRows(execute("SELECT language, body FROM system.schema_functions WHERE keyspace_name=? AND function_name=?",
+                           fNameName.keyspace, fNameName.name),
+                   row("java", functionBody));
+
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 1, 1d);
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 2, 2d);
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 3, 3d);
+        assertRows(execute("SELECT key, val, " + fName + "(val) FROM %s"),
+                   row(1, 1d, Math.sin(1d)),
+                   row(2, 2d, Math.sin(2d)),
+                   row(3, 3d, Math.sin(3d))
+        );
+    }
+
+    @Test
+    public void testJavaRuntimeException() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int primary key, val double)");
+
+        String functionBody = '\n' +
+                              "  throw new RuntimeException(\"oh no!\");\n";
+
+        String fName = createFunction(KEYSPACE_PER_TEST, "double",
+                                      "CREATE OR REPLACE FUNCTION %s(val double) " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS double " +
+                                      "LANGUAGE JAVA\n" +
+                                      "AS '" + functionBody + "';");
+
+        FunctionName fNameName = parseFunctionName(fName);
+
+        assertRows(execute("SELECT language, body FROM system.schema_functions WHERE keyspace_name=? AND function_name=?",
+                           fNameName.keyspace, fNameName.name),
+                   row("java", functionBody));
+
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 1, 1d);
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 2, 2d);
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 3, 3d);
+
+        // function throws a RuntimeException which is wrapped by FunctionExecutionException
+        assertInvalidThrowMessage("java.lang.RuntimeException: oh no", FunctionExecutionException.class,
+                                  "SELECT key, val, " + fName + "(val) FROM %s");
+    }
+
+    @Test
+    public void testJavaDollarQuotedFunction() throws Throwable
+    {
+        String functionBody = '\n' +
+                              "  // parameter val is of type java.lang.Double\n" +
+                              "  /* return type is of type java.lang.Double */\n" +
+                              "  if (input == null) {\n" +
+                              "    return null;\n" +
+                              "  }\n" +
+                              "  return \"'\"+Math.sin(input)+'\\\'';\n";
+
+        String fName = createFunction(KEYSPACE_PER_TEST, "double",
+                                      "CREATE FUNCTION %s( input double ) " +
+                                      "CALLED ON NULL INPUT " +
+                                      "RETURNS text " +
+                                      "LANGUAGE java\n" +
+                                      "AS $$" + functionBody + "$$;");
+
+        FunctionName fNameName = parseFunctionName(fName);
+
+        assertRows(execute("SELECT language, body FROM system.schema_functions WHERE keyspace_name=? AND function_name=?",
+                           fNameName.keyspace, fNameName.name),
+                   row("java", functionBody));
+    }
+
+    @Test
+    public void testJavaSimpleCollections() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int primary key, lst list<double>, st set<text>, mp map<int, boolean>)");
+
+        String fList = createFunction(KEYSPACE_PER_TEST, "list<double>",
+                                      "CREATE FUNCTION %s( lst list<double> ) " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS list<double> " +
+                                      "LANGUAGE java\n" +
+                                      "AS $$return lst;$$;");
+        String fSet = createFunction(KEYSPACE_PER_TEST, "set<text>",
+                                     "CREATE FUNCTION %s( st set<text> ) " +
+                                     "RETURNS NULL ON NULL INPUT " +
+                                     "RETURNS set<text> " +
+                                     "LANGUAGE java\n" +
+                                     "AS $$return st;$$;");
+        String fMap = createFunction(KEYSPACE_PER_TEST, "map<int, boolean>",
+                                     "CREATE FUNCTION %s( mp map<int, boolean> ) " +
+                                     "RETURNS NULL ON NULL INPUT " +
+                                     "RETURNS map<int, boolean> " +
+                                     "LANGUAGE java\n" +
+                                     "AS $$return mp;$$;");
+
+        List<Double> list = Arrays.asList(1d, 2d, 3d);
+        Set<String> set = new TreeSet<>(Arrays.asList("one", "three", "two"));
+        Map<Integer, Boolean> map = new TreeMap<>();
+        map.put(1, true);
+        map.put(2, false);
+        map.put(3, true);
+
+        execute("INSERT INTO %s (key, lst, st, mp) VALUES (1, ?, ?, ?)", list, set, map);
+
+        assertRows(execute("SELECT " + fList + "(lst), " + fSet + "(st), " + fMap + "(mp) FROM %s WHERE key = 1"),
+                   row(list, set, map));
+
+        // same test - but via native protocol
+        for (int version = Server.VERSION_2; version <= maxProtocolVersion; version++)
+            assertRowsNet(version,
+                          executeNet(version, "SELECT " + fList + "(lst), " + fSet + "(st), " + fMap + "(mp) FROM %s WHERE key = 1"),
+                          row(list, set, map));
+    }
+
+    @Test
+    public void testJavaTupleType() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int primary key, tup frozen<tuple<double, text, int, boolean>>)");
+
+        String fName = createFunction(KEYSPACE, "tuple<double, text, int, boolean>",
+                                      "CREATE FUNCTION %s( tup tuple<double, text, int, boolean> ) " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS tuple<double, text, int, boolean> " +
+                                      "LANGUAGE java\n" +
+                                      "AS $$return tup;$$;");
+
+        Object t = tuple(1d, "foo", 2, true);
+
+        execute("INSERT INTO %s (key, tup) VALUES (1, ?)", t);
+
+        assertRows(execute("SELECT tup FROM %s WHERE key = 1"),
+                   row(t));
+
+        assertRows(execute("SELECT " + fName + "(tup) FROM %s WHERE key = 1"),
+                   row(t));
+    }
+
+    @Test
+    public void testJavaTupleTypeCollection() throws Throwable
+    {
+        String tupleTypeDef = "tuple<double, list<double>, set<text>, map<int, boolean>>";
+
+        createTable("CREATE TABLE %s (key int primary key, tup frozen<" + tupleTypeDef + ">)");
+
+        String fTup0 = createFunction(KEYSPACE_PER_TEST, tupleTypeDef,
+                                      "CREATE FUNCTION %s( tup " + tupleTypeDef + " ) " +
+                                      "CALLED ON NULL INPUT " +
+                                      "RETURNS " + tupleTypeDef + ' ' +
+                                      "LANGUAGE java\n" +
+                                      "AS $$return " +
+                                      "       tup;$$;");
+        String fTup1 = createFunction(KEYSPACE_PER_TEST, tupleTypeDef,
+                                      "CREATE FUNCTION %s( tup " + tupleTypeDef + " ) " +
+                                      "CALLED ON NULL INPUT " +
+                                      "RETURNS double " +
+                                      "LANGUAGE java\n" +
+                                      "AS $$return " +
+                                      "       Double.valueOf(tup.getDouble(0));$$;");
+        String fTup2 = createFunction(KEYSPACE_PER_TEST, tupleTypeDef,
+                                      "CREATE FUNCTION %s( tup " + tupleTypeDef + " ) " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS list<double> " +
+                                      "LANGUAGE java\n" +
+                                      "AS $$return " +
+                                      "       tup.getList(1, Double.class);$$;");
+        String fTup3 = createFunction(KEYSPACE_PER_TEST, tupleTypeDef,
+                                      "CREATE FUNCTION %s( tup " + tupleTypeDef + " ) " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS set<text> " +
+                                      "LANGUAGE java\n" +
+                                      "AS $$return " +
+                                      "       tup.getSet(2, String.class);$$;");
+        String fTup4 = createFunction(KEYSPACE_PER_TEST, tupleTypeDef,
+                                      "CREATE FUNCTION %s( tup " + tupleTypeDef + " ) " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS map<int, boolean> " +
+                                      "LANGUAGE java\n" +
+                                      "AS $$return " +
+                                      "       tup.getMap(3, Integer.class, Boolean.class);$$;");
+
+        List<Double> list = Arrays.asList(1d, 2d, 3d);
+        Set<String> set = new TreeSet<>(Arrays.asList("one", "three", "two"));
+        Map<Integer, Boolean> map = new TreeMap<>();
+        map.put(1, true);
+        map.put(2, false);
+        map.put(3, true);
+
+        Object t = tuple(1d, list, set, map);
+
+        execute("INSERT INTO %s (key, tup) VALUES (1, ?)", t);
+
+        assertRows(execute("SELECT " + fTup0 + "(tup) FROM %s WHERE key = 1"),
+                   row(t));
+        assertRows(execute("SELECT " + fTup1 + "(tup) FROM %s WHERE key = 1"),
+                   row(1d));
+        assertRows(execute("SELECT " + fTup2 + "(tup) FROM %s WHERE key = 1"),
+                   row(list));
+        assertRows(execute("SELECT " + fTup3 + "(tup) FROM %s WHERE key = 1"),
+                   row(set));
+        assertRows(execute("SELECT " + fTup4 + "(tup) FROM %s WHERE key = 1"),
+                   row(map));
+
+        TupleType tType = TupleType.of(DataType.cdouble(),
+                                       DataType.list(DataType.cdouble()),
+                                       DataType.set(DataType.text()),
+                                       DataType.map(DataType.cint(), DataType.cboolean()));
+        TupleValue tup = tType.newValue(1d, list, set, map);
+        for (int version = Server.VERSION_2; version <= maxProtocolVersion; version++)
+        {
+            assertRowsNet(version,
+                          executeNet(version, "SELECT " + fTup0 + "(tup) FROM %s WHERE key = 1"),
+                          row(tup));
+            assertRowsNet(version,
+                          executeNet(version, "SELECT " + fTup1 + "(tup) FROM %s WHERE key = 1"),
+                          row(1d));
+            assertRowsNet(version,
+                          executeNet(version, "SELECT " + fTup2 + "(tup) FROM %s WHERE key = 1"),
+                          row(list));
+            assertRowsNet(version,
+                          executeNet(version, "SELECT " + fTup3 + "(tup) FROM %s WHERE key = 1"),
+                          row(set));
+            assertRowsNet(version,
+                          executeNet(version, "SELECT " + fTup4 + "(tup) FROM %s WHERE key = 1"),
+                          row(map));
+        }
+    }
+
+    @Test
+    public void testJavaUserTypeWithUse() throws Throwable
+    {
+        String type = createType("CREATE TYPE %s (txt text, i int)");
+        createTable("CREATE TABLE %s (key int primary key, udt frozen<" + KEYSPACE + '.' + type + ">)");
+        execute("INSERT INTO %s (key, udt) VALUES (1, {txt: 'one', i:1})");
+
+        for (int version = Server.VERSION_2; version <= maxProtocolVersion; version++)
+        {
+            executeNet(version, "USE " + KEYSPACE);
+
+            executeNet(version,
+                       "CREATE FUNCTION f_use1( udt " + type + " ) " +
+                       "RETURNS NULL ON NULL INPUT " +
+                       "RETURNS " + type + " " +
+                       "LANGUAGE java " +
+                       "AS $$return " +
+                       "     udt;$$;");
+            try
+            {
+                List<Row> rowsNet = executeNet(version, "SELECT f_use1(udt) FROM %s WHERE key = 1").all();
+                Assert.assertEquals(1, rowsNet.size());
+                UDTValue udtVal = rowsNet.get(0).getUDTValue(0);
+                Assert.assertEquals("one", udtVal.getString("txt"));
+                Assert.assertEquals(1, udtVal.getInt("i"));
+            }
+            finally
+            {
+                executeNet(version, "DROP FUNCTION f_use1");
+            }
+        }
+    }
+
+    @Test
+    public void testJavaUserType() throws Throwable
+    {
+        String type = KEYSPACE + '.' + createType("CREATE TYPE %s (txt text, i int)");
+
+        createTable("CREATE TABLE %s (key int primary key, udt frozen<" + type + ">)");
+
+        String fUdt0 = createFunction(KEYSPACE, type,
+                                      "CREATE FUNCTION %s( udt " + type + " ) " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS " + type + " " +
+                                      "LANGUAGE java " +
+                                      "AS $$return " +
+                                      "     udt;$$;");
+        String fUdt1 = createFunction(KEYSPACE, type,
+                                      "CREATE FUNCTION %s( udt " + type + ") " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS text " +
+                                      "LANGUAGE java " +
+                                      "AS $$return " +
+                                      "     udt.getString(\"txt\");$$;");
+        String fUdt2 = createFunction(KEYSPACE, type,
+                                      "CREATE FUNCTION %s( udt " + type + ") " +
+                                      "CALLED ON NULL INPUT " +
+                                      "RETURNS int " +
+                                      "LANGUAGE java " +
+                                      "AS $$return " +
+                                      "     Integer.valueOf(udt.getInt(\"i\"));$$;");
+
+        execute("INSERT INTO %s (key, udt) VALUES (1, {txt: 'one', i:1})");
+
+        UntypedResultSet rows = execute("SELECT " + fUdt0 + "(udt) FROM %s WHERE key = 1");
+        Assert.assertEquals(1, rows.size());
+        assertRows(execute("SELECT " + fUdt1 + "(udt) FROM %s WHERE key = 1"),
+                   row("one"));
+        assertRows(execute("SELECT " + fUdt2 + "(udt) FROM %s WHERE key = 1"),
+                   row(1));
+
+        for (int version = Server.VERSION_2; version <= maxProtocolVersion; version++)
+        {
+            List<Row> rowsNet = executeNet(version, "SELECT " + fUdt0 + "(udt) FROM %s WHERE key = 1").all();
+            Assert.assertEquals(1, rowsNet.size());
+            UDTValue udtVal = rowsNet.get(0).getUDTValue(0);
+            Assert.assertEquals("one", udtVal.getString("txt"));
+            Assert.assertEquals(1, udtVal.getInt("i"));
+            assertRowsNet(version,
+                          executeNet(version, "SELECT " + fUdt1 + "(udt) FROM %s WHERE key = 1"),
+                          row("one"));
+            assertRowsNet(version,
+                          executeNet(version, "SELECT " + fUdt2 + "(udt) FROM %s WHERE key = 1"),
+                          row(1));
+        }
+    }
+
+    @Test
+    public void testJavaUserTypeRenameField() throws Throwable
+    {
+        String type = KEYSPACE + '.' + createType("CREATE TYPE %s (txt text, i int)");
+
+        createTable("CREATE TABLE %s (key int primary key, udt frozen<" + type + ">)");
+
+        String fName = createFunction(KEYSPACE, type,
+                                      "CREATE FUNCTION %s( udt " + type + " ) " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS text " +
+                                      "LANGUAGE java\n" +
+                                      "AS $$return udt.getString(\"txt\");$$;");
+
+        execute("INSERT INTO %s (key, udt) VALUES (1, {txt: 'one', i:1})");
+
+        assertRows(execute("SELECT " + fName + "(udt) FROM %s WHERE key = 1"),
+                   row("one"));
+
+        execute("ALTER TYPE " + type + " RENAME txt TO str");
+
+        assertInvalidMessage("txt is not a field defined in this UDT",
+                             "SELECT " + fName + "(udt) FROM %s WHERE key = 1");
+
+        execute("ALTER TYPE " + type + " RENAME str TO txt");
+
+        assertRows(execute("SELECT " + fName + "(udt) FROM %s WHERE key = 1"),
+                   row("one"));
+    }
+
+    @Test
+    public void testJavaUserTypeAddFieldWithReplace() throws Throwable
+    {
+        String type = KEYSPACE + '.' + createType("CREATE TYPE %s (txt text, i int)");
+
+        createTable("CREATE TABLE %s (key int primary key, udt frozen<" + type + ">)");
+
+        String fName1replace = createFunction(KEYSPACE, type,
+                                              "CREATE FUNCTION %s( udt " + type + ") " +
+                                              "RETURNS NULL ON NULL INPUT " +
+                                              "RETURNS text " +
+                                              "LANGUAGE java\n" +
+                                              "AS $$return udt.getString(\"txt\");$$;");
+        String fName2replace = createFunction(KEYSPACE, type,
+                                              "CREATE FUNCTION %s( udt " + type + " ) " +
+                                              "CALLED ON NULL INPUT " +
+                                              "RETURNS int " +
+                                              "LANGUAGE java\n" +
+                                              "AS $$return Integer.valueOf(udt.getInt(\"i\"));$$;");
+        String fName3replace = createFunction(KEYSPACE, type,
+                                              "CREATE FUNCTION %s( udt " + type + " ) " +
+                                              "CALLED ON NULL INPUT " +
+                                              "RETURNS double " +
+                                              "LANGUAGE java\n" +
+                                              "AS $$return Double.valueOf(udt.getDouble(\"added\"));$$;");
+        String fName4replace = createFunction(KEYSPACE, type,
+                                              "CREATE FUNCTION %s( udt " + type + " ) " +
+                                              "RETURNS NULL ON NULL INPUT " +
+                                              "RETURNS " + type + " " +
+                                              "LANGUAGE java\n" +
+                                              "AS $$return udt;$$;");
+
+        String fName1noReplace = createFunction(KEYSPACE, type,
+                                                "CREATE FUNCTION %s( udt " + type + " ) " +
+                                                "RETURNS NULL ON NULL INPUT " +
+                                                "RETURNS text " +
+                                                "LANGUAGE java\n" +
+                                                "AS $$return udt.getString(\"txt\");$$;");
+        String fName2noReplace = createFunction(KEYSPACE, type,
+                                                "CREATE FUNCTION %s( udt " + type + " ) " +
+                                                "CALLED ON NULL INPUT " +
+                                                "RETURNS int " +
+                                                "LANGUAGE java\n" +
+                                                "AS $$return Integer.valueOf(udt.getInt(\"i\"));$$;");
+        String fName3noReplace = createFunction(KEYSPACE, type,
+                                                "CREATE FUNCTION %s( udt " + type + " ) " +
+                                                "CALLED ON NULL INPUT " +
+                                                "RETURNS double " +
+                                                "LANGUAGE java\n" +
+                                                "AS $$return Double.valueOf(udt.getDouble(\"added\"));$$;");
+        String fName4noReplace = createFunction(KEYSPACE, type,
+                                                "CREATE FUNCTION %s( udt " + type + " ) " +
+                                                "RETURNS NULL ON NULL INPUT " +
+                                                "RETURNS " + type + " " +
+                                                "LANGUAGE java\n" +
+                                                "AS $$return udt;$$;");
+
+        execute("INSERT INTO %s (key, udt) VALUES (1, {txt: 'one', i:1})");
+
+        assertRows(execute("SELECT " + fName1replace + "(udt) FROM %s WHERE key = 1"),
+                   row("one"));
+        assertRows(execute("SELECT " + fName2replace + "(udt) FROM %s WHERE key = 1"),
+                   row(1));
+
+        // add field
+
+        execute("ALTER TYPE " + type + " ADD added double");
+
+        execute("INSERT INTO %s (key, udt) VALUES (2, {txt: 'two', i:2, added: 2})");
+
+        // note: type references of functions remain at the state _before_ the type mutation
+        // means we need to recreate the functions
+
+        execute(String.format("CREATE OR REPLACE FUNCTION %s( udt %s ) " +
+                              "RETURNS NULL ON NULL INPUT " +
+                              "RETURNS text " +
+                              "LANGUAGE java\n" +
+                              "AS $$return " +
+                              "     udt.getString(\"txt\");$$;",
+                              fName1replace, type));
+        Assert.assertEquals(1, Functions.find(parseFunctionName(fName1replace)).size());
+        execute(String.format("CREATE OR REPLACE FUNCTION %s( udt %s ) " +
+                              "CALLED ON NULL INPUT " +
+                              "RETURNS int " +
+                              "LANGUAGE java\n" +
+                              "AS $$return " +
+                              "     Integer.valueOf(udt.getInt(\"i\"));$$;",
+                              fName2replace, type));
+        Assert.assertEquals(1, Functions.find(parseFunctionName(fName2replace)).size());
+        execute(String.format("CREATE OR REPLACE FUNCTION %s( udt %s ) " +
+                              "CALLED ON NULL INPUT " +
+                              "RETURNS double " +
+                              "LANGUAGE java\n" +
+                              "AS $$return " +
+                              "     Double.valueOf(udt.getDouble(\"added\"));$$;",
+                              fName3replace, type));
+        Assert.assertEquals(1, Functions.find(parseFunctionName(fName3replace)).size());
+        execute(String.format("CREATE OR REPLACE FUNCTION %s( udt %s ) " +
+                              "RETURNS NULL ON NULL INPUT " +
+                              "RETURNS %s " +
+                              "LANGUAGE java\n" +
+                              "AS $$return " +
+                              "     udt;$$;",
+                              fName4replace, type, type));
+        Assert.assertEquals(1, Functions.find(parseFunctionName(fName4replace)).size());
+
+        assertRows(execute("SELECT " + fName1replace + "(udt) FROM %s WHERE key = 2"),
+                   row("two"));
+        assertRows(execute("SELECT " + fName2replace + "(udt) FROM %s WHERE key = 2"),
+                   row(2));
+        assertRows(execute("SELECT " + fName3replace + "(udt) FROM %s WHERE key = 2"),
+                   row(2d));
+        assertRows(execute("SELECT " + fName3replace + "(udt) FROM %s WHERE key = 1"),
+                   row(0d));
+
+        // un-replaced functions will work since the user type has changed
+        // and the UDF has exchanged the user type reference
+
+        assertRows(execute("SELECT " + fName1noReplace + "(udt) FROM %s WHERE key = 2"),
+                   row("two"));
+        assertRows(execute("SELECT " + fName2noReplace + "(udt) FROM %s WHERE key = 2"),
+                   row(2));
+        assertRows(execute("SELECT " + fName3noReplace + "(udt) FROM %s WHERE key = 2"),
+                   row(2d));
+        assertRows(execute("SELECT " + fName3noReplace + "(udt) FROM %s WHERE key = 1"),
+                   row(0d));
+
+        execute("DROP FUNCTION " + fName1replace);
+        execute("DROP FUNCTION " + fName2replace);
+        execute("DROP FUNCTION " + fName3replace);
+        execute("DROP FUNCTION " + fName4replace);
+        execute("DROP FUNCTION " + fName1noReplace);
+        execute("DROP FUNCTION " + fName2noReplace);
+        execute("DROP FUNCTION " + fName3noReplace);
+        execute("DROP FUNCTION " + fName4noReplace);
+    }
+
+    @Test
+    public void testJavaUTCollections() throws Throwable
+    {
+        String type = KEYSPACE + '.' + createType("CREATE TYPE %s (txt text, i int)");
+
+        createTable(String.format("CREATE TABLE %%s " +
+                                  "(key int primary key, lst list<frozen<%s>>, st set<frozen<%s>>, mp map<int, frozen<%s>>)",
+                                  type, type, type));
+
+        String fName1 = createFunction(KEYSPACE, "list<frozen<" + type + ">>",
+                                       "CREATE FUNCTION %s( lst list<frozen<" + type + ">> ) " +
+                                       "RETURNS NULL ON NULL INPUT " +
+                                       "RETURNS text " +
+                                       "LANGUAGE java\n" +
+                                       "AS $$" +
+                                       "     com.datastax.driver.core.UDTValue udtVal = (com.datastax.driver.core.UDTValue)lst.get(1);" +
+                                       "     return udtVal.getString(\"txt\");$$;");
+        String fName2 = createFunction(KEYSPACE, "set<frozen<" + type + ">>",
+                                       "CREATE FUNCTION %s( st set<frozen<" + type + ">> ) " +
+                                       "RETURNS NULL ON NULL INPUT " +
+                                       "RETURNS text " +
+                                       "LANGUAGE java\n" +
+                                       "AS $$" +
+                                       "     com.datastax.driver.core.UDTValue udtVal = (com.datastax.driver.core.UDTValue)st.iterator().next();" +
+                                       "     return udtVal.getString(\"txt\");$$;");
+        String fName3 = createFunction(KEYSPACE, "map<int, frozen<" + type + ">>",
+                                       "CREATE FUNCTION %s( mp map<int, frozen<" + type + ">> ) " +
+                                       "RETURNS NULL ON NULL INPUT " +
+                                       "RETURNS text " +
+                                       "LANGUAGE java\n" +
+                                       "AS $$" +
+                                       "     com.datastax.driver.core.UDTValue udtVal = (com.datastax.driver.core.UDTValue)mp.get(Integer.valueOf(3));" +
+                                       "     return udtVal.getString(\"txt\");$$;");
+
+        execute("INSERT INTO %s (key, lst, st, mp) values (1, " +
+                "[ {txt: 'one', i:1}, {txt: 'three', i:1}, {txt: 'one', i:1} ] , " +
+                "{ {txt: 'one', i:1}, {txt: 'three', i:3}, {txt: 'two', i:2} }, " +
+                "{ 1: {txt: 'one', i:1}, 2: {txt: 'one', i:3}, 3: {txt: 'two', i:2} })");
+
+        assertRows(execute("SELECT " + fName1 + "(lst), " + fName2 + "(st), " + fName3 + "(mp) FROM %s WHERE key = 1"),
+                   row("three", "one", "two"));
+
+        for (int version = Server.VERSION_2; version <= maxProtocolVersion; version++)
+            assertRowsNet(version,
+                          executeNet(version, "SELECT " + fName1 + "(lst), " + fName2 + "(st), " + fName3 + "(mp) FROM %s WHERE key = 1"),
+                          row("three", "one", "two"));
+    }
+
+    @Test
+    public void testFunctionWithFrozenSetType() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<set<int>>)");
+        createIndex("CREATE INDEX ON %s (FULL(b))");
+
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 0, set());
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 1, set(1, 2, 3));
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 2, set(4, 5, 6));
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 3, set(7, 8, 9));
+
+        assertInvalidMessage("The function arguments should not be frozen",
+                             "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".frozenSetArg(values frozen<set<int>>) " +
+                             "CALLED ON NULL INPUT " +
+                             "RETURNS int " +
+                             "LANGUAGE java\n" +
+                             "AS 'int sum = 0; for (Object value : values) {sum += value;} return sum;';");
+
+        assertInvalidMessage("The function return type should not be frozen",
+                             "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".frozenReturnType(values set<int>) " +
+                             "CALLED ON NULL INPUT " +
+                             "RETURNS frozen<set<int>> " +
+                             "LANGUAGE java\n" +
+                             "AS 'return values;';");
+
+        String functionName = createFunction(KEYSPACE,
+                                             "set<int>",
+                                             "CREATE FUNCTION %s (values set<int>) " +
+                                             "CALLED ON NULL INPUT " +
+                                             "RETURNS int " +
+                                             "LANGUAGE java\n" +
+                                             "AS 'int sum = 0; for (Object value : values) {sum += ((Integer) value);} return sum;';");
+
+        assertRows(execute("SELECT a, " + functionName + "(b) FROM %s"),
+                   row(0, 0),
+                   row(1, 6),
+                   row(2, 15),
+                   row(3, 24));
+
+        functionName = createFunction(KEYSPACE,
+                                      "set<int>",
+                                      "CREATE FUNCTION %s (values set<int>) " +
+                                      "CALLED ON NULL INPUT " +
+                                      "RETURNS set<int> " +
+                                      "LANGUAGE java\n" +
+                                      "AS 'return values;';");
+
+        assertRows(execute("SELECT a FROM %s WHERE b = " + functionName + "(?)", set(1, 2, 3)),
+                   row(1));
+
+        assertInvalidMessage("The function arguments should not be frozen",
+                             "DROP FUNCTION " + functionName + "(frozen<set<int>>);");
+    }
+
+    @Test
+    public void testFunctionWithFrozenListType() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<list<int>>)");
+        createIndex("CREATE INDEX ON %s (FULL(b))");
+
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 0, list());
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 1, list(1, 2, 3));
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 2, list(4, 5, 6));
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 3, list(7, 8, 9));
+
+        assertInvalidMessage("The function arguments should not be frozen",
+                             "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".withFrozenArg(values frozen<list<int>>) " +
+                             "CALLED ON NULL INPUT " +
+                             "RETURNS int " +
+                             "LANGUAGE java\n" +
+                             "AS 'int sum = 0; for (Object value : values) {sum += value;} return sum;';");
+
+        assertInvalidMessage("The function return type should not be frozen",
+                             "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".frozenReturnType(values list<int>) " +
+                             "CALLED ON NULL INPUT " +
+                             "RETURNS frozen<list<int>> " +
+                             "LANGUAGE java\n" +
+                             "AS 'return values;';");
+
+        String functionName = createFunction(KEYSPACE,
+                                             "list<int>",
+                                             "CREATE FUNCTION %s (values list<int>) " +
+                                             "CALLED ON NULL INPUT " +
+                                             "RETURNS int " +
+                                             "LANGUAGE java\n" +
+                                             "AS 'int sum = 0; for (Object value : values) {sum += ((Integer) value);} return sum;';");
+
+        assertRows(execute("SELECT a, " + functionName + "(b) FROM %s"),
+                   row(0, 0),
+                   row(1, 6),
+                   row(2, 15),
+                   row(3, 24));
+
+        functionName = createFunction(KEYSPACE,
+                                      "list<int>",
+                                      "CREATE FUNCTION %s (values list<int>) " +
+                                      "CALLED ON NULL INPUT " +
+                                      "RETURNS list<int> " +
+                                      "LANGUAGE java\n" +
+                                      "AS 'return values;';");
+
+        assertRows(execute("SELECT a FROM %s WHERE b = " + functionName + "(?)", set(1, 2, 3)),
+                   row(1));
+
+        assertInvalidMessage("The function arguments should not be frozen",
+                             "DROP FUNCTION " + functionName + "(frozen<list<int>>);");
+    }
+
+    @Test
+    public void testFunctionWithFrozenMapType() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<map<int, int>>)");
+        createIndex("CREATE INDEX ON %s (FULL(b))");
+
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 0, map());
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 1, map(1, 1, 2, 2, 3, 3));
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 2, map(4, 4, 5, 5, 6, 6));
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 3, map(7, 7, 8, 8, 9, 9));
+
+        assertInvalidMessage("The function arguments should not be frozen",
+                             "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".withFrozenArg(values frozen<map<int, int>>) " +
+                             "CALLED ON NULL INPUT " +
+                             "RETURNS int " +
+                             "LANGUAGE java\n" +
+                             "AS 'int sum = 0; for (Object value : values.values()) {sum += value;} return sum;';");
+
+        assertInvalidMessage("The function return type should not be frozen",
+                             "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".frozenReturnType(values map<int, int>) " +
+                             "CALLED ON NULL INPUT " +
+                             "RETURNS frozen<map<int, int>> " +
+                             "LANGUAGE java\n" +
+                             "AS 'return values;';");
+
+        String functionName = createFunction(KEYSPACE,
+                                             "map<int, int>",
+                                             "CREATE FUNCTION %s (values map<int, int>) " +
+                                             "CALLED ON NULL INPUT " +
+                                             "RETURNS int " +
+                                             "LANGUAGE java\n" +
+                                             "AS 'int sum = 0; for (Object value : values.values()) {sum += ((Integer) value);} return sum;';");
+
+        assertRows(execute("SELECT a, " + functionName + "(b) FROM %s"),
+                   row(0, 0),
+                   row(1, 6),
+                   row(2, 15),
+                   row(3, 24));
+
+        functionName = createFunction(KEYSPACE,
+                                      "map<int, int>",
+                                      "CREATE FUNCTION %s (values map<int, int>) " +
+                                      "CALLED ON NULL INPUT " +
+                                      "RETURNS map<int, int> " +
+                                      "LANGUAGE java\n" +
+                                      "AS 'return values;';");
+
+        assertRows(execute("SELECT a FROM %s WHERE b = " + functionName + "(?)", map(1, 1, 2, 2, 3, 3)),
+                   row(1));
+
+        assertInvalidMessage("The function arguments should not be frozen",
+                             "DROP FUNCTION " + functionName + "(frozen<map<int, int>>);");
+    }
+
+    @Test
+    public void testFunctionWithFrozenTupleType() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<tuple<int, int>>)");
+        createIndex("CREATE INDEX ON %s (b)");
+
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 0, tuple());
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 1, tuple(1, 2));
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 2, tuple(4, 5));
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 3, tuple(7, 8));
+
+        assertInvalidMessage("The function arguments should not be frozen",
+                             "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".withFrozenArg(values frozen<tuple<int, int>>) " +
+                             "CALLED ON NULL INPUT " +
+                             "RETURNS text " +
+                             "LANGUAGE java\n" +
+                             "AS 'return values.toString();';");
+
+        assertInvalidMessage("The function return type should not be frozen",
+                             "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".frozenReturnType(values tuple<int, int>) " +
+                             "CALLED ON NULL INPUT " +
+                             "RETURNS frozen<tuple<int, int>> " +
+                             "LANGUAGE java\n" +
+                             "AS 'return values;';");
+
+        String functionName = createFunction(KEYSPACE,
+                                             "tuple<int, int>",
+                                             "CREATE FUNCTION %s (values tuple<int, int>) " +
+                                             "CALLED ON NULL INPUT " +
+                                             "RETURNS text " +
+                                             "LANGUAGE java\n" +
+                                             "AS 'return values.toString();';");
+
+        assertRows(execute("SELECT a, " + functionName + "(b) FROM %s"),
+                   row(0, "(null, null)"),
+                   row(1, "(1, 2)"),
+                   row(2, "(4, 5)"),
+                   row(3, "(7, 8)"));
+
+        functionName = createFunction(KEYSPACE,
+                                      "tuple<int, int>",
+                                      "CREATE FUNCTION %s (values tuple<int, int>) " +
+                                      "CALLED ON NULL INPUT " +
+                                      "RETURNS tuple<int, int> " +
+                                      "LANGUAGE java\n" +
+                                      "AS 'return values;';");
+
+        assertRows(execute("SELECT a FROM %s WHERE b = " + functionName + "(?)", tuple(1, 2)),
+                   row(1));
+
+        assertInvalidMessage("The function arguments should not be frozen",
+                             "DROP FUNCTION " + functionName + "(frozen<tuple<int, int>>);");
+    }
+
+    @Test
+    public void testFunctionWithFrozenUDType() throws Throwable
+    {
+        String myType = createType("CREATE TYPE %s (f int)");
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<" + myType + ">)");
+        createIndex("CREATE INDEX ON %s (b)");
+
+        execute("INSERT INTO %s (a, b) VALUES (?, {f : ?})", 0, 0);
+        execute("INSERT INTO %s (a, b) VALUES (?, {f : ?})", 1, 1);
+        execute("INSERT INTO %s (a, b) VALUES (?, {f : ?})", 2, 4);
+        execute("INSERT INTO %s (a, b) VALUES (?, {f : ?})", 3, 7);
+
+        assertInvalidMessage("The function arguments should not be frozen",
+                             "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".withFrozenArg(values frozen<" + myType + ">) " +
+                             "CALLED ON NULL INPUT " +
+                             "RETURNS text " +
+                             "LANGUAGE java\n" +
+                             "AS 'return values.toString();';");
+
+        assertInvalidMessage("The function return type should not be frozen",
+                             "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".frozenReturnType(values " + myType + ") " +
+                             "CALLED ON NULL INPUT " +
+                             "RETURNS frozen<" + myType + "> " +
+                             "LANGUAGE java\n" +
+                             "AS 'return values;';");
+
+        String functionName = createFunction(KEYSPACE,
+                                             myType,
+                                             "CREATE FUNCTION %s (values " + myType + ") " +
+                                             "CALLED ON NULL INPUT " +
+                                             "RETURNS text " +
+                                             "LANGUAGE java\n" +
+                                             "AS 'return values.toString();';");
+
+        assertRows(execute("SELECT a, " + functionName + "(b) FROM %s"),
+                   row(0, "{f:0}"),
+                   row(1, "{f:1}"),
+                   row(2, "{f:4}"),
+                   row(3, "{f:7}"));
+
+        functionName = createFunction(KEYSPACE,
+                                      myType,
+                                      "CREATE FUNCTION %s (values " + myType + ") " +
+                                      "CALLED ON NULL INPUT " +
+                                      "RETURNS " + myType + " " +
+                                      "LANGUAGE java\n" +
+                                      "AS 'return values;';");
+
+        assertRows(execute("SELECT a FROM %s WHERE b = " + functionName + "({f: ?})", 1),
+                   row(1));
+
+        assertInvalidMessage("The function arguments should not be frozen",
+                             "DROP FUNCTION " + functionName + "(frozen<" + myType + ">);");
+    }
+
+    @Test
+    public void testEmptyString() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int primary key, sval text, aval ascii, bval blob, empty_int int)");
+        execute("INSERT INTO %s (key, sval, aval, bval, empty_int) VALUES (?, ?, ?, ?, blobAsInt(0x))", 1, "", "", ByteBuffer.allocate(0));
+
+        String fNameSRC = createFunction(KEYSPACE_PER_TEST, "text",
+                                         "CREATE OR REPLACE FUNCTION %s(val text) " +
+                                         "CALLED ON NULL INPUT " +
+                                         "RETURNS text " +
+                                         "LANGUAGE JAVA\n" +
+                                         "AS 'return val;'");
+
+        String fNameSCC = createFunction(KEYSPACE_PER_TEST, "text",
+                                         "CREATE OR REPLACE FUNCTION %s(val text) " +
+                                         "CALLED ON NULL INPUT " +
+                                         "RETURNS text " +
+                                         "LANGUAGE JAVA\n" +
+                                         "AS 'return \"\";'");
+
+        String fNameSRN = createFunction(KEYSPACE_PER_TEST, "text",
+                                         "CREATE OR REPLACE FUNCTION %s(val text) " +
+                                         "RETURNS NULL ON NULL INPUT " +
+                                         "RETURNS text " +
+                                         "LANGUAGE JAVA\n" +
+                                         "AS 'return val;'");
+
+        String fNameSCN = createFunction(KEYSPACE_PER_TEST, "text",
+                                         "CREATE OR REPLACE FUNCTION %s(val text) " +
+                                         "RETURNS NULL ON NULL INPUT " +
+                                         "RETURNS text " +
+                                         "LANGUAGE JAVA\n" +
+                                         "AS 'return \"\";'");
+
+        String fNameBRC = createFunction(KEYSPACE_PER_TEST, "blob",
+                                         "CREATE OR REPLACE FUNCTION %s(val blob) " +
+                                         "CALLED ON NULL INPUT " +
+                                         "RETURNS blob " +
+                                         "LANGUAGE JAVA\n" +
+                                         "AS 'return val;'");
+
+        String fNameBCC = createFunction(KEYSPACE_PER_TEST, "blob",
+                                         "CREATE OR REPLACE FUNCTION %s(val blob) " +
+                                         "CALLED ON NULL INPUT " +
+                                         "RETURNS blob " +
+                                         "LANGUAGE JAVA\n" +
+                                         "AS 'return ByteBuffer.allocate(0);'");
+
+        String fNameBRN = createFunction(KEYSPACE_PER_TEST, "blob",
+                                         "CREATE OR REPLACE FUNCTION %s(val blob) " +
+                                         "RETURNS NULL ON NULL INPUT " +
+                                         "RETURNS blob " +
+                                         "LANGUAGE JAVA\n" +
+                                         "AS 'return val;'");
+
+        String fNameBCN = createFunction(KEYSPACE_PER_TEST, "blob",
+                                         "CREATE OR REPLACE FUNCTION %s(val blob) " +
+                                         "RETURNS NULL ON NULL INPUT " +
+                                         "RETURNS blob " +
+                                         "LANGUAGE JAVA\n" +
+                                         "AS 'return ByteBuffer.allocate(0);'");
+
+        String fNameIRC = createFunction(KEYSPACE_PER_TEST, "int",
+                                         "CREATE OR REPLACE FUNCTION %s(val int) " +
+                                         "CALLED ON NULL INPUT " +
+                                         "RETURNS int " +
+                                         "LANGUAGE JAVA\n" +
+                                         "AS 'return val;'");
+
+        String fNameICC = createFunction(KEYSPACE_PER_TEST, "int",
+                                         "CREATE OR REPLACE FUNCTION %s(val int) " +
+                                         "CALLED ON NULL INPUT " +
+                                         "RETURNS int " +
+                                         "LANGUAGE JAVA\n" +
+                                         "AS 'return 0;'");
+
+        String fNameIRN = createFunction(KEYSPACE_PER_TEST, "int",
+                                         "CREATE OR REPLACE FUNCTION %s(val int) " +
+                                         "RETURNS NULL ON NULL INPUT " +
+                                         "RETURNS int " +
+                                         "LANGUAGE JAVA\n" +
+                                         "AS 'return val;'");
+
+        String fNameICN = createFunction(KEYSPACE_PER_TEST, "int",
+                                         "CREATE OR REPLACE FUNCTION %s(val int) " +
+                                         "RETURNS NULL ON NULL INPUT " +
+                                         "RETURNS int " +
+                                         "LANGUAGE JAVA\n" +
+                                         "AS 'return 0;'");
+
+        assertRows(execute("SELECT " + fNameSRC + "(sval) FROM %s"), row(""));
+        assertRows(execute("SELECT " + fNameSRN + "(sval) FROM %s"), row(""));
+        assertRows(execute("SELECT " + fNameSCC + "(sval) FROM %s"), row(""));
+        assertRows(execute("SELECT " + fNameSCN + "(sval) FROM %s"), row(""));
+        assertRows(execute("SELECT " + fNameSRC + "(aval) FROM %s"), row(""));
+        assertRows(execute("SELECT " + fNameSRN + "(aval) FROM %s"), row(""));
+        assertRows(execute("SELECT " + fNameSCC + "(aval) FROM %s"), row(""));
+        assertRows(execute("SELECT " + fNameSCN + "(aval) FROM %s"), row(""));
+        assertRows(execute("SELECT " + fNameBRC + "(bval) FROM %s"), row(ByteBufferUtil.EMPTY_BYTE_BUFFER));
+        assertRows(execute("SELECT " + fNameBRN + "(bval) FROM %s"), row(ByteBufferUtil.EMPTY_BYTE_BUFFER));
+        assertRows(execute("SELECT " + fNameBCC + "(bval) FROM %s"), row(ByteBufferUtil.EMPTY_BYTE_BUFFER));
+        assertRows(execute("SELECT " + fNameBCN + "(bval) FROM %s"), row(ByteBufferUtil.EMPTY_BYTE_BUFFER));
+        assertRows(execute("SELECT " + fNameIRC + "(empty_int) FROM %s"), row(new Object[]{null}));
+        assertRows(execute("SELECT " + fNameIRN + "(empty_int) FROM %s"), row(new Object[]{null}));
+        assertRows(execute("SELECT " + fNameICC + "(empty_int) FROM %s"), row(0));
+        assertRows(execute("SELECT " + fNameICN + "(empty_int) FROM %s"), row(new Object[]{null}));
+    }
+
+    @Test
+    public void testAllNativeTypes() throws Throwable
+    {
+        StringBuilder sig = new StringBuilder();
+        StringBuilder args = new StringBuilder();
+        for (CQL3Type.Native type : CQL3Type.Native.values())
+        {
+            if (sig.length() > 0)
+                sig.append(',');
+            sig.append(type.toString());
+
+            if (args.length() > 0)
+                args.append(',');
+            args.append("arg").append(type.toString()).append(' ').append(type.toString());
+        }
+        createFunction(KEYSPACE, sig.toString(),
+                       "CREATE OR REPLACE FUNCTION %s(" + args + ") " +
+                       "RETURNS NULL ON NULL INPUT " +
+                       "RETURNS int " +
+                       "LANGUAGE JAVA\n" +
+                       "AS 'return 0;'");
+
+        for (CQL3Type.Native type : CQL3Type.Native.values())
+        {
+            createFunction(KEYSPACE_PER_TEST, type.toString(),
+                           "CREATE OR REPLACE FUNCTION %s(val " + type.toString() + ") " +
+                           "RETURNS NULL ON NULL INPUT " +
+                           "RETURNS int " +
+                           "LANGUAGE JAVA\n" +
+                           "AS 'return 0;'");
+        }
+    }
+}
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/UFScriptTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/UFScriptTest.java
new file mode 100644
index 0000000..d3050a5
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UFScriptTest.java
@@ -0,0 +1,504 @@
+/*
+ * 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.cql3.validation.entities;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.util.Arrays;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.UUID;
+
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.datastax.driver.core.DataType;
+import com.datastax.driver.core.TupleType;
+import com.datastax.driver.core.TupleValue;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.cql3.functions.FunctionName;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.exceptions.FunctionExecutionException;
+import org.apache.cassandra.transport.Server;
+import org.apache.cassandra.utils.UUIDGen;
+
+public class UFScriptTest extends CQLTester
+{
+    @BeforeClass
+    public static void setUp()
+    {
+        DatabaseDescriptor.setPartitioner(ByteOrderedPartitioner.instance);
+    }
+
+    @Test
+    public void testJavascriptSimpleCollections() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int primary key, lst list<double>, st set<text>, mp map<int, boolean>)");
+
+        String fName1 = createFunction(KEYSPACE_PER_TEST, "list<double>",
+                                       "CREATE FUNCTION %s( lst list<double> ) " +
+                                       "RETURNS NULL ON NULL INPUT " +
+                                       "RETURNS list<double> " +
+                                       "LANGUAGE javascript\n" +
+                                       "AS 'lst;';");
+        String fName2 = createFunction(KEYSPACE_PER_TEST, "set<text>",
+                                       "CREATE FUNCTION %s( st set<text> ) " +
+                                       "RETURNS NULL ON NULL INPUT " +
+                                       "RETURNS set<text> " +
+                                       "LANGUAGE javascript\n" +
+                                       "AS 'st;';");
+        String fName3 = createFunction(KEYSPACE_PER_TEST, "map<int, boolean>",
+                                       "CREATE FUNCTION %s( mp map<int, boolean> ) " +
+                                       "RETURNS NULL ON NULL INPUT " +
+                                       "RETURNS map<int, boolean> " +
+                                       "LANGUAGE javascript\n" +
+                                       "AS 'mp;';");
+
+        List<Double> list = Arrays.asList(1d, 2d, 3d);
+        Set<String> set = new TreeSet<>(Arrays.asList("one", "three", "two"));
+        Map<Integer, Boolean> map = new TreeMap<>();
+        map.put(1, true);
+        map.put(2, false);
+        map.put(3, true);
+
+        execute("INSERT INTO %s (key, lst, st, mp) VALUES (1, ?, ?, ?)", list, set, map);
+
+        assertRows(execute("SELECT lst, st, mp FROM %s WHERE key = 1"),
+                   row(list, set, map));
+
+        assertRows(execute("SELECT " + fName1 + "(lst), " + fName2 + "(st), " + fName3 + "(mp) FROM %s WHERE key = 1"),
+                   row(list, set, map));
+
+        for (int version = Server.VERSION_2; version <= maxProtocolVersion; version++)
+            assertRowsNet(version,
+                          executeNet(version, "SELECT " + fName1 + "(lst), " + fName2 + "(st), " + fName3 + "(mp) FROM %s WHERE key = 1"),
+                          row(list, set, map));
+    }
+
+    @Test
+    public void testJavascriptTupleType() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int primary key, tup frozen<tuple<double, text, int, boolean>>)");
+
+        String fName = createFunction(KEYSPACE_PER_TEST, "tuple<double, text, int, boolean>",
+                                      "CREATE FUNCTION %s( tup tuple<double, text, int, boolean> ) " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS tuple<double, text, int, boolean> " +
+                                      "LANGUAGE javascript\n" +
+                                      "AS $$tup;$$;");
+
+        Object t = tuple(1d, "foo", 2, true);
+
+        execute("INSERT INTO %s (key, tup) VALUES (1, ?)", t);
+
+        assertRows(execute("SELECT tup FROM %s WHERE key = 1"),
+                   row(t));
+
+        assertRows(execute("SELECT " + fName + "(tup) FROM %s WHERE key = 1"),
+                   row(t));
+    }
+
+    @Test
+    public void testJavascriptTupleTypeCollection() throws Throwable
+    {
+        String tupleTypeDef = "tuple<double, list<double>, set<text>, map<int, boolean>>";
+        createTable("CREATE TABLE %s (key int primary key, tup frozen<" + tupleTypeDef + ">)");
+
+        String fTup1 = createFunction(KEYSPACE_PER_TEST, tupleTypeDef,
+                                      "CREATE FUNCTION %s( tup " + tupleTypeDef + " ) " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS tuple<double, list<double>, set<text>, map<int, boolean>> " +
+                                      "LANGUAGE javascript\n" +
+                                      "AS $$" +
+                                      "       tup;$$;");
+        String fTup2 = createFunction(KEYSPACE_PER_TEST, tupleTypeDef,
+                                      "CREATE FUNCTION %s( tup " + tupleTypeDef + " ) " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS double " +
+                                      "LANGUAGE javascript\n" +
+                                      "AS $$" +
+                                      "       tup.getDouble(0);$$;");
+        String fTup3 = createFunction(KEYSPACE_PER_TEST, tupleTypeDef,
+                                      "CREATE FUNCTION %s( tup " + tupleTypeDef + " ) " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS list<double> " +
+                                      "LANGUAGE javascript\n" +
+                                      "AS $$" +
+                                      "       tup.getList(1, java.lang.Class.forName(\"java.lang.Double\"));$$;");
+        String fTup4 = createFunction(KEYSPACE_PER_TEST, tupleTypeDef,
+                                      "CREATE FUNCTION %s( tup " + tupleTypeDef + " ) " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS set<text> " +
+                                      "LANGUAGE javascript\n" +
+                                      "AS $$" +
+                                      "       tup.getSet(2, java.lang.Class.forName(\"java.lang.String\"));$$;");
+        String fTup5 = createFunction(KEYSPACE_PER_TEST, tupleTypeDef,
+                                      "CREATE FUNCTION %s( tup " + tupleTypeDef + " ) " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS map<int, boolean> " +
+                                      "LANGUAGE javascript\n" +
+                                      "AS $$" +
+                                      "       tup.getMap(3, java.lang.Class.forName(\"java.lang.Integer\"), java.lang.Class.forName(\"java.lang.Boolean\"));$$;");
+
+        List<Double> list = Arrays.asList(1d, 2d, 3d);
+        Set<String> set = new TreeSet<>(Arrays.asList("one", "three", "two"));
+        Map<Integer, Boolean> map = new TreeMap<>();
+        map.put(1, true);
+        map.put(2, false);
+        map.put(3, true);
+
+        Object t = tuple(1d, list, set, map);
+
+        execute("INSERT INTO %s (key, tup) VALUES (1, ?)", t);
+
+        assertRows(execute("SELECT " + fTup1 + "(tup) FROM %s WHERE key = 1"),
+                   row(t));
+        assertRows(execute("SELECT " + fTup2 + "(tup) FROM %s WHERE key = 1"),
+                   row(1d));
+        assertRows(execute("SELECT " + fTup3 + "(tup) FROM %s WHERE key = 1"),
+                   row(list));
+        assertRows(execute("SELECT " + fTup4 + "(tup) FROM %s WHERE key = 1"),
+                   row(set));
+        assertRows(execute("SELECT " + fTup5 + "(tup) FROM %s WHERE key = 1"),
+                   row(map));
+
+        // same test - but via native protocol
+        TupleType tType = TupleType.of(DataType.cdouble(),
+                                       DataType.list(DataType.cdouble()),
+                                       DataType.set(DataType.text()),
+                                       DataType.map(DataType.cint(), DataType.cboolean()));
+        TupleValue tup = tType.newValue(1d, list, set, map);
+        for (int version = Server.VERSION_2; version <= maxProtocolVersion; version++)
+        {
+            assertRowsNet(version,
+                          executeNet(version, "SELECT " + fTup1 + "(tup) FROM %s WHERE key = 1"),
+                          row(tup));
+            assertRowsNet(version,
+                          executeNet(version, "SELECT " + fTup2 + "(tup) FROM %s WHERE key = 1"),
+                          row(1d));
+            assertRowsNet(version,
+                          executeNet(version, "SELECT " + fTup3 + "(tup) FROM %s WHERE key = 1"),
+                          row(list));
+            assertRowsNet(version,
+                          executeNet(version, "SELECT " + fTup4 + "(tup) FROM %s WHERE key = 1"),
+                          row(set));
+            assertRowsNet(version,
+                          executeNet(version, "SELECT " + fTup5 + "(tup) FROM %s WHERE key = 1"),
+                          row(map));
+        }
+    }
+
+    @Test
+    public void testJavascriptUserType() throws Throwable
+    {
+        String type = createType("CREATE TYPE %s (txt text, i int)");
+
+        createTable("CREATE TABLE %s (key int primary key, udt frozen<" + type + ">)");
+
+        String fUdt1 = createFunction(KEYSPACE, type,
+                                      "CREATE FUNCTION %s( udt " + type + " ) " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS " + type + " " +
+                                      "LANGUAGE javascript\n" +
+                                      "AS $$" +
+                                      "     udt;$$;");
+        String fUdt2 = createFunction(KEYSPACE, type,
+                                      "CREATE FUNCTION %s( udt " + type + " ) " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS text " +
+                                      "LANGUAGE javascript\n" +
+                                      "AS $$" +
+                                      "     udt.getString(\"txt\");$$;");
+        String fUdt3 = createFunction(KEYSPACE, type,
+                                      "CREATE FUNCTION %s( udt " + type + " ) " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS int " +
+                                      "LANGUAGE javascript\n" +
+                                      "AS $$" +
+                                      "     udt.getInt(\"i\");$$;");
+
+        execute("INSERT INTO %s (key, udt) VALUES (1, {txt: 'one', i:1})");
+
+        UntypedResultSet rows = execute("SELECT " + fUdt1 + "(udt) FROM %s WHERE key = 1");
+        Assert.assertEquals(1, rows.size());
+        assertRows(execute("SELECT " + fUdt2 + "(udt) FROM %s WHERE key = 1"),
+                   row("one"));
+        assertRows(execute("SELECT " + fUdt3 + "(udt) FROM %s WHERE key = 1"),
+                   row(1));
+    }
+
+    @Test
+    public void testJavascriptUTCollections() throws Throwable
+    {
+        String type = createType("CREATE TYPE %s (txt text, i int)");
+
+        createTable(String.format("CREATE TABLE %%s " +
+                                  "(key int primary key, lst list<frozen<%s>>, st set<frozen<%s>>, mp map<int, frozen<%s>>)",
+                                  type, type, type));
+
+        String fName = createFunction(KEYSPACE, "list<frozen<" + type + ">>",
+                                      "CREATE FUNCTION %s( lst list<frozen<" + type + ">> ) " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS text " +
+                                      "LANGUAGE javascript\n" +
+                                      "AS $$" +
+                                      "        lst.get(1).getString(\"txt\");$$;");
+        createFunctionOverload(fName, "set<frozen<" + type + ">>",
+                               "CREATE FUNCTION %s( st set<frozen<" + type + ">> ) " +
+                               "RETURNS NULL ON NULL INPUT " +
+                               "RETURNS text " +
+                               "LANGUAGE javascript\n" +
+                               "AS $$" +
+                               "        st.iterator().next().getString(\"txt\");$$;");
+        createFunctionOverload(fName, "map<int, frozen<" + type + ">>",
+                               "CREATE FUNCTION %s( mp map<int, frozen<" + type + ">> ) " +
+                               "RETURNS NULL ON NULL INPUT " +
+                               "RETURNS text " +
+                               "LANGUAGE javascript\n" +
+                               "AS $$" +
+                               "        mp.get(java.lang.Integer.valueOf(3)).getString(\"txt\");$$;");
+
+        execute("INSERT INTO %s (key, lst, st, mp) values (1, " +
+                // list<frozen<UDT>>
+                "[ {txt: 'one', i:1}, {txt: 'three', i:1}, {txt: 'one', i:1} ] , " +
+                // set<frozen<UDT>>
+                "{ {txt: 'one', i:1}, {txt: 'three', i:3}, {txt: 'two', i:2} }, " +
+                // map<int, frozen<UDT>>
+                "{ 1: {txt: 'one', i:1}, 2: {txt: 'one', i:3}, 3: {txt: 'two', i:2} })");
+
+        assertRows(execute("SELECT " + fName + "(lst) FROM %s WHERE key = 1"),
+                   row("three"));
+        assertRows(execute("SELECT " + fName + "(st) FROM %s WHERE key = 1"),
+                   row("one"));
+        assertRows(execute("SELECT " + fName + "(mp) FROM %s WHERE key = 1"),
+                   row("two"));
+
+        String cqlSelect = "SELECT " + fName + "(lst), " + fName + "(st), " + fName + "(mp) FROM %s WHERE key = 1";
+        assertRows(execute(cqlSelect),
+                   row("three", "one", "two"));
+
+        // same test - but via native protocol
+        for (int version = Server.VERSION_2; version <= maxProtocolVersion; version++)
+            assertRowsNet(version,
+                          executeNet(version, cqlSelect),
+                          row("three", "one", "two"));
+    }
+
+    @Test
+    public void testJavascriptFunction() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int primary key, val double)");
+
+        String functionBody = '\n' +
+                              "  Math.sin(val);\n";
+
+        String fName = createFunction(KEYSPACE, "double",
+                                      "CREATE OR REPLACE FUNCTION %s(val double) " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS double " +
+                                      "LANGUAGE javascript\n" +
+                                      "AS '" + functionBody + "';");
+
+        FunctionName fNameName = parseFunctionName(fName);
+
+        assertRows(execute("SELECT language, body FROM system.schema_functions WHERE keyspace_name=? AND function_name=?",
+                           fNameName.keyspace, fNameName.name),
+                   row("javascript", functionBody));
+
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 1, 1d);
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 2, 2d);
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 3, 3d);
+        assertRows(execute("SELECT key, val, " + fName + "(val) FROM %s"),
+                   row(1, 1d, Math.sin(1d)),
+                   row(2, 2d, Math.sin(2d)),
+                   row(3, 3d, Math.sin(3d))
+        );
+    }
+
+    @Test
+    public void testJavascriptBadReturnType() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int primary key, val double)");
+
+        String fName = createFunction(KEYSPACE, "double",
+                                      "CREATE OR REPLACE FUNCTION %s(val double) " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS double " +
+                                      "LANGUAGE javascript\n" +
+                                      "AS '\"string\";';");
+
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 1, 1d);
+        // throws IRE with ClassCastException
+        assertInvalidMessage("Invalid value for CQL type double", "SELECT key, val, " + fName + "(val) FROM %s");
+    }
+
+    @Test
+    public void testJavascriptThrow() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int primary key, val double)");
+
+        String fName = createFunction(KEYSPACE, "double",
+                                      "CREATE OR REPLACE FUNCTION %s(val double) " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS double " +
+                                      "LANGUAGE javascript\n" +
+                                      "AS 'throw \"fool\";';");
+
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 1, 1d);
+        // throws IRE with ScriptException
+        assertInvalidThrowMessage("fool", FunctionExecutionException.class,
+                                  "SELECT key, val, " + fName + "(val) FROM %s");
+    }
+
+    @Test
+    public void testJavascriptCompileFailure() throws Throwable
+    {
+        assertInvalidMessage("Failed to compile function 'cql_test_keyspace.scrinv'",
+                             "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".scrinv(val double) " +
+                             "RETURNS NULL ON NULL INPUT " +
+                             "RETURNS double " +
+                             "LANGUAGE javascript\n" +
+                             "AS 'foo bar';");
+    }
+
+    @Test
+    public void testScriptInvalidLanguage() throws Throwable
+    {
+        assertInvalidMessage("Invalid language 'artificial_intelligence' for function 'cql_test_keyspace.scrinv'",
+                             "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".scrinv(val double) " +
+                             "RETURNS NULL ON NULL INPUT " +
+                             "RETURNS double " +
+                             "LANGUAGE artificial_intelligence\n" +
+                             "AS 'question for 42?';");
+    }
+
+    @Test
+    public void testScriptReturnTypeCasting() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int primary key, val double)");
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 1, 1d);
+
+        Object[][] variations = {
+        new Object[]    {   "true",     "boolean",  true    },
+        new Object[]    {   "false",    "boolean",  false   },
+        new Object[]    {   "100",      "tinyint",  (byte)100 },
+        new Object[]    {   "100.",     "tinyint",  (byte)100 },
+        new Object[]    {   "100",      "smallint", (short)100 },
+        new Object[]    {   "100.",     "smallint", (short)100 },
+        new Object[]    {   "100",      "int",      100     },
+        new Object[]    {   "100.",     "int",      100     },
+        new Object[]    {   "100",      "double",   100d    },
+        new Object[]    {   "100.",     "double",   100d    },
+        new Object[]    {   "100",      "bigint",   100L    },
+        new Object[]    {   "100.",     "bigint",   100L    },
+        new Object[]    { "100", "varint", BigInteger.valueOf(100L)    },
+        new Object[]    {   "100.",     "varint",   BigInteger.valueOf(100L)    },
+        new Object[]    { "parseInt(\"100\");", "decimal", BigDecimal.valueOf(100d)    },
+        new Object[]    {   "100.",     "decimal",  BigDecimal.valueOf(100d)    },
+        };
+
+        for (Object[] variation : variations)
+        {
+            Object functionBody = variation[0];
+            Object returnType = variation[1];
+            Object expectedResult = variation[2];
+
+            String fName = createFunction(KEYSPACE, "double",
+                                          "CREATE OR REPLACE FUNCTION %s(val double) " +
+                                          "RETURNS NULL ON NULL INPUT " +
+                                          "RETURNS " +returnType + ' ' +
+                                          "LANGUAGE javascript " +
+                                          "AS '" + functionBody + ";';");
+            assertRows(execute("SELECT key, val, " + fName + "(val) FROM %s"),
+                       row(1, 1d, expectedResult));
+        }
+    }
+
+    @Test
+    public void testScriptParamReturnTypes() throws Throwable
+    {
+        UUID ruuid = UUID.randomUUID();
+        UUID tuuid = UUIDGen.getTimeUUID();
+
+        createTable("CREATE TABLE %s (key int primary key, " +
+                    "tival tinyint, sival smallint, ival int, lval bigint, fval float, dval double, vval varint, ddval decimal, " +
+                    "timval time, dtval date, tsval timestamp, uval uuid, tuval timeuuid)");
+        execute("INSERT INTO %s (key, tival, sival, ival, lval, fval, dval, vval, ddval, timval, dtval, tsval, uval, tuval) VALUES " +
+                "(?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)", 1,
+                (byte)1, (short)1, 1, 1L, 1f, 1d, BigInteger.valueOf(1L), BigDecimal.valueOf(1d), 1L, Integer.MAX_VALUE, new Date(1), ruuid, tuuid);
+
+        Object[][] variations = {
+        new Object[] {  "tinyint",  "tival",    (byte)1,                (byte)2  },
+        new Object[] {  "smallint", "sival",    (short)1,               (short)2  },
+        new Object[] {  "int",      "ival",     1,                      2  },
+        new Object[] {  "bigint",   "lval",     1L,                     2L  },
+        new Object[] {  "float",    "fval",     1f,                     2f  },
+        new Object[] {  "double",   "dval",     1d,                     2d  },
+        new Object[] {  "varint",   "vval",     BigInteger.valueOf(1L), BigInteger.valueOf(2L)  },
+        new Object[] {  "decimal",  "ddval",    BigDecimal.valueOf(1d), BigDecimal.valueOf(2d)  },
+        new Object[] {  "time",     "timval",   1L,                     2L  },
+        };
+
+        for (Object[] variation : variations)
+        {
+            Object type = variation[0];
+            Object col = variation[1];
+            Object expected1 = variation[2];
+            Object expected2 = variation[3];
+            String fName = createFunction(KEYSPACE, type.toString(),
+                                          "CREATE OR REPLACE FUNCTION %s(val " + type + ") " +
+                                          "RETURNS NULL ON NULL INPUT " +
+                                          "RETURNS " + type + ' ' +
+                                          "LANGUAGE javascript " +
+                                          "AS 'val+1;';");
+            assertRows(execute("SELECT key, " + col + ", " + fName + '(' + col + ") FROM %s"),
+                       row(1, expected1, expected2));
+        }
+
+        variations = new Object[][] {
+        new Object[] {  "timestamp","tsval",    new Date(1),            new Date(1)  },
+        new Object[] {  "uuid",     "uval",     ruuid,                  ruuid  },
+        new Object[] {  "timeuuid", "tuval",    tuuid,                  tuuid  },
+        new Object[] {  "date",     "dtval",    Integer.MAX_VALUE,      Integer.MAX_VALUE },
+        };
+
+        for (Object[] variation : variations)
+        {
+            Object type = variation[0];
+            Object col = variation[1];
+            Object expected1 = variation[2];
+            Object expected2 = variation[3];
+            String fName = createFunction(KEYSPACE, type.toString(),
+                                          "CREATE OR REPLACE FUNCTION %s(val " + type + ") " +
+                                          "RETURNS NULL ON NULL INPUT " +
+                                          "RETURNS " + type + ' ' +
+                                          "LANGUAGE javascript " +
+                                          "AS 'val;';");
+            assertRows(execute("SELECT key, " + col + ", " + fName + '(' + col + ") FROM %s"),
+                       row(1, expected1, expected2));
+        }
+    }
+}
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
new file mode 100644
index 0000000..d4d2a10
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UFTest.java
@@ -0,0 +1,1111 @@
+/*
+ * 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.cql3.validation.entities;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.UUID;
+
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.exceptions.InvalidQueryException;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.cql3.functions.FunctionName;
+import org.apache.cassandra.cql3.functions.Functions;
+import org.apache.cassandra.cql3.functions.UDFunction;
+import org.apache.cassandra.db.marshal.CollectionType;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.transport.Event;
+import org.apache.cassandra.transport.Server;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.UUIDGen;
+
+public class UFTest extends CQLTester
+{
+    @BeforeClass
+    public static void setUp()
+    {
+        DatabaseDescriptor.setPartitioner(ByteOrderedPartitioner.instance);
+    }
+
+    @Test
+    public void testNonExistingOnes() throws Throwable
+    {
+        assertInvalidMessage("Cannot drop non existing function", "DROP FUNCTION " + KEYSPACE + ".func_does_not_exist");
+        assertInvalidMessage("Cannot drop non existing function", "DROP FUNCTION " + KEYSPACE + ".func_does_not_exist(int,text)");
+        assertInvalidMessage("Cannot drop non existing function", "DROP FUNCTION keyspace_does_not_exist.func_does_not_exist");
+        assertInvalidMessage("Cannot drop non existing function", "DROP FUNCTION keyspace_does_not_exist.func_does_not_exist(int,text)");
+
+        execute("DROP FUNCTION IF EXISTS " + KEYSPACE + ".func_does_not_exist");
+        execute("DROP FUNCTION IF EXISTS " + KEYSPACE + ".func_does_not_exist(int,text)");
+        execute("DROP FUNCTION IF EXISTS keyspace_does_not_exist.func_does_not_exist");
+        execute("DROP FUNCTION IF EXISTS keyspace_does_not_exist.func_does_not_exist(int,text)");
+    }
+
+    @Test
+    public void testSchemaChange() throws Throwable
+    {
+        String f = createFunction(KEYSPACE,
+                                  "double, double",
+                                  "CREATE OR REPLACE FUNCTION %s(state double, val double) " +
+                                  "RETURNS NULL ON NULL INPUT " +
+                                  "RETURNS double " +
+                                  "LANGUAGE javascript " +
+                                  "AS '\"string\";';");
+
+        assertLastSchemaChange(Event.SchemaChange.Change.CREATED, Event.SchemaChange.Target.FUNCTION,
+                               KEYSPACE, parseFunctionName(f).name,
+                               "double", "double");
+
+        createFunctionOverload(f,
+                               "double, double",
+                               "CREATE OR REPLACE FUNCTION %s(state int, val int) " +
+                               "RETURNS NULL ON NULL INPUT " +
+                               "RETURNS int " +
+                               "LANGUAGE javascript " +
+                               "AS '\"string\";';");
+
+        assertLastSchemaChange(Event.SchemaChange.Change.CREATED, Event.SchemaChange.Target.FUNCTION,
+                               KEYSPACE, parseFunctionName(f).name,
+                               "int", "int");
+
+        schemaChange("CREATE OR REPLACE FUNCTION " + f + "(state int, val int) " +
+                     "RETURNS NULL ON NULL INPUT " +
+                     "RETURNS int " +
+                     "LANGUAGE javascript " +
+                     "AS '\"string\";';");
+
+        assertLastSchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.FUNCTION,
+                               KEYSPACE, parseFunctionName(f).name,
+                               "int", "int");
+
+        schemaChange("DROP FUNCTION " + f + "(double, double)");
+
+        assertLastSchemaChange(Event.SchemaChange.Change.DROPPED, Event.SchemaChange.Target.FUNCTION,
+                               KEYSPACE, parseFunctionName(f).name,
+                               "double", "double");
+    }
+
+    @Test
+    public void testFunctionDropOnKeyspaceDrop() throws Throwable
+    {
+        String fSin = createFunction(KEYSPACE_PER_TEST, "double",
+                                     "CREATE FUNCTION %s ( input double ) " +
+                                     "CALLED ON NULL INPUT " +
+                                     "RETURNS double " +
+                                     "LANGUAGE java " +
+                                     "AS 'return Double.valueOf(Math.sin(input.doubleValue()));'");
+
+        FunctionName fSinName = parseFunctionName(fSin);
+
+        Assert.assertEquals(1, Functions.find(parseFunctionName(fSin)).size());
+
+        assertRows(execute("SELECT function_name, language FROM system.schema_functions WHERE keyspace_name=?", KEYSPACE_PER_TEST),
+                   row(fSinName.name, "java"));
+
+        dropPerTestKeyspace();
+
+        assertRows(execute("SELECT function_name, language FROM system.schema_functions WHERE keyspace_name=?", KEYSPACE_PER_TEST));
+
+        Assert.assertEquals(0, Functions.find(fSinName).size());
+    }
+
+    @Test
+    public void testFunctionDropPreparedStatement() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int PRIMARY KEY, d double)");
+
+        String fSin = createFunction(KEYSPACE_PER_TEST, "double",
+                                     "CREATE FUNCTION %s ( input double ) " +
+                                     "CALLED ON NULL INPUT " +
+                                     "RETURNS double " +
+                                     "LANGUAGE java " +
+                                     "AS 'return Double.valueOf(Math.sin(input.doubleValue()));'");
+
+        FunctionName fSinName = parseFunctionName(fSin);
+
+        Assert.assertEquals(1, Functions.find(parseFunctionName(fSin)).size());
+
+        // create a pairs of Select and Inserts. One statement in each pair uses the function so when we
+        // drop it those statements should be removed from the cache in QueryProcessor. The other statements
+        // should be unaffected.
+
+        ResultMessage.Prepared preparedSelect1 = QueryProcessor.prepare(
+                                                                       String.format("SELECT key, %s(d) FROM %s.%s", fSin, KEYSPACE, currentTable()),
+                                                                       ClientState.forInternalCalls(), false);
+        ResultMessage.Prepared preparedSelect2 = QueryProcessor.prepare(
+                                                    String.format("SELECT key FROM %s.%s", KEYSPACE, currentTable()),
+                                                    ClientState.forInternalCalls(), false);
+        ResultMessage.Prepared preparedInsert1 = QueryProcessor.prepare(
+                                                      String.format("INSERT INTO %s.%s (key, d) VALUES (?, %s(?))", KEYSPACE, currentTable(), fSin),
+                                                      ClientState.forInternalCalls(), false);
+        ResultMessage.Prepared preparedInsert2 = QueryProcessor.prepare(
+                                                      String.format("INSERT INTO %s.%s (key, d) VALUES (?, ?)", KEYSPACE, currentTable()),
+                                                      ClientState.forInternalCalls(), false);
+
+        Assert.assertNotNull(QueryProcessor.instance.getPrepared(preparedSelect1.statementId));
+        Assert.assertNotNull(QueryProcessor.instance.getPrepared(preparedSelect2.statementId));
+        Assert.assertNotNull(QueryProcessor.instance.getPrepared(preparedInsert1.statementId));
+        Assert.assertNotNull(QueryProcessor.instance.getPrepared(preparedInsert2.statementId));
+
+        execute("DROP FUNCTION " + fSin + "(double);");
+
+        // the statements which use the dropped function should be removed from cache, with the others remaining
+        Assert.assertNull(QueryProcessor.instance.getPrepared(preparedSelect1.statementId));
+        Assert.assertNotNull(QueryProcessor.instance.getPrepared(preparedSelect2.statementId));
+        Assert.assertNull(QueryProcessor.instance.getPrepared(preparedInsert1.statementId));
+        Assert.assertNotNull(QueryProcessor.instance.getPrepared(preparedInsert2.statementId));
+
+        execute("CREATE FUNCTION " + fSin + " ( input double ) " +
+                "RETURNS NULL ON NULL INPUT " +
+                "RETURNS double " +
+                "LANGUAGE java " +
+                "AS 'return Double.valueOf(Math.sin(input));'");
+
+        Assert.assertEquals(1, Functions.find(fSinName).size());
+
+        preparedSelect1= QueryProcessor.prepare(
+                                         String.format("SELECT key, %s(d) FROM %s.%s", fSin, KEYSPACE, currentTable()),
+                                         ClientState.forInternalCalls(), false);
+        preparedInsert1 = QueryProcessor.prepare(
+                                         String.format("INSERT INTO %s.%s (key, d) VALUES (?, %s(?))", KEYSPACE, currentTable(), fSin),
+                                         ClientState.forInternalCalls(), false);
+        Assert.assertNotNull(QueryProcessor.instance.getPrepared(preparedSelect1.statementId));
+        Assert.assertNotNull(QueryProcessor.instance.getPrepared(preparedInsert1.statementId));
+
+        dropPerTestKeyspace();
+
+        // again, only the 2 statements referencing the function should be removed from cache
+        // this time because the statements select from tables in KEYSPACE, only the function
+        // is scoped to KEYSPACE_PER_TEST
+        Assert.assertNull(QueryProcessor.instance.getPrepared(preparedSelect1.statementId));
+        Assert.assertNotNull(QueryProcessor.instance.getPrepared(preparedSelect2.statementId));
+        Assert.assertNull(QueryProcessor.instance.getPrepared(preparedInsert1.statementId));
+        Assert.assertNotNull(QueryProcessor.instance.getPrepared(preparedInsert2.statementId));
+    }
+
+    @Test
+    public void testDropFunctionDropsPreparedStatementsWithDelayedValues() throws Throwable
+    {
+        // test that dropping a function removes stmts which use
+        // it to provide a DelayedValue collection from the
+        // cache in QueryProcessor
+        checkDelayedValuesCorrectlyIdentifyFunctionsInUse(false);
+    }
+
+    @Test
+    public void testDropKeyspaceContainingFunctionDropsPreparedStatementsWithDelayedValues() throws Throwable
+    {
+        // test that dropping a function removes stmts which use
+        // it to provide a DelayedValue collection from the
+        // cache in QueryProcessor
+        checkDelayedValuesCorrectlyIdentifyFunctionsInUse(true);
+    }
+
+    private ResultMessage.Prepared prepareStatementWithDelayedValue(CollectionType.Kind kind, String function)
+    {
+        String collectionType;
+        String literalArgs;
+        switch (kind)
+        {
+            case LIST:
+                collectionType = "list<double>";
+                literalArgs = String.format("[%s(0.0)]", function);
+                break;
+            case SET:
+                collectionType = "set<double>";
+                literalArgs = String.format("{%s(0.0)}", function);
+                break;
+            case MAP:
+                collectionType = "map<double, double>";
+                literalArgs = String.format("{%s(0.0):0.0}", function);
+                break;
+            default:
+                Assert.fail("Unsupported collection type " + kind);
+                collectionType = null;
+                literalArgs = null;
+        }
+
+        createTable("CREATE TABLE %s (" +
+                    " key int PRIMARY KEY," +
+                    " val " + collectionType + ')');
+
+        ResultMessage.Prepared prepared = QueryProcessor.prepare(
+                                                                String.format("INSERT INTO %s.%s (key, val) VALUES (?, %s)",
+                                                                             KEYSPACE,
+                                                                             currentTable(),
+                                                                             literalArgs),
+                                                                ClientState.forInternalCalls(), false);
+        Assert.assertNotNull(QueryProcessor.instance.getPrepared(prepared.statementId));
+        return prepared;
+    }
+
+    private ResultMessage.Prepared prepareStatementWithDelayedValueTuple(String function)
+    {
+        createTable("CREATE TABLE %s (" +
+                    " key int PRIMARY KEY," +
+                    " val tuple<double> )");
+
+        ResultMessage.Prepared prepared = QueryProcessor.prepare(
+                                                                String.format("INSERT INTO %s.%s (key, val) VALUES (?, (%s(0.0)))",
+                                                                             KEYSPACE,
+                                                                             currentTable(),
+                                                                             function),
+                                                                ClientState.forInternalCalls(), false);
+        Assert.assertNotNull(QueryProcessor.instance.getPrepared(prepared.statementId));
+        return prepared;
+    }
+
+    private void checkDelayedValuesCorrectlyIdentifyFunctionsInUse(boolean dropKeyspace) throws Throwable
+    {
+        // prepare a statement which doesn't use any function for a control
+        createTable("CREATE TABLE %s (" +
+                    " key int PRIMARY KEY," +
+                    " val double)");
+        ResultMessage.Prepared control = QueryProcessor.prepare(
+                                                               String.format("INSERT INTO %s.%s (key, val) VALUES (?, ?)",
+                                                                            KEYSPACE,
+                                                                            currentTable()),
+                                                               ClientState.forInternalCalls(), false);
+        Assert.assertNotNull(QueryProcessor.instance.getPrepared(control.statementId));
+
+        // a function that we'll drop and verify that statements which use it to
+        // provide a DelayedValue are removed from the cache in QueryProcessor
+        String function = createFunction(KEYSPACE_PER_TEST, "double",
+                                        "CREATE FUNCTION %s ( input double ) " +
+                                        "CALLED ON NULL INPUT " +
+                                        "RETURNS double " +
+                                        "LANGUAGE javascript " +
+                                        "AS 'input'");
+        Assert.assertEquals(1, Functions.find(parseFunctionName(function)).size());
+
+        List<ResultMessage.Prepared> prepared = new ArrayList<>();
+        // prepare statements which use the function to provide a DelayedValue
+        prepared.add(prepareStatementWithDelayedValue(CollectionType.Kind.LIST, function));
+        prepared.add(prepareStatementWithDelayedValue(CollectionType.Kind.SET, function));
+        prepared.add(prepareStatementWithDelayedValue(CollectionType.Kind.MAP, function));
+        prepared.add(prepareStatementWithDelayedValueTuple(function));
+
+        // what to drop - the function is scoped to the per-test keyspace, but the prepared statements
+        // select from the per-fixture keyspace. So if we drop the per-test keyspace, the function
+        // should be removed along with the statements that reference it. The control statement should
+        // remain present in the cache. Likewise, if we actually drop the function itself the control
+        // statement should not be removed, but the others should be
+        if (dropKeyspace)
+            dropPerTestKeyspace();
+        else
+            execute("DROP FUNCTION " + function);
+
+        Assert.assertNotNull(QueryProcessor.instance.getPrepared(control.statementId));
+        for (ResultMessage.Prepared removed : prepared)
+            Assert.assertNull(QueryProcessor.instance.getPrepared(removed.statementId));
+    }
+
+    @Test
+    public void testFunctionCreationAndDrop() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int PRIMARY KEY, d double)");
+
+        execute("INSERT INTO %s(key, d) VALUES (?, ?)", 1, 1d);
+        execute("INSERT INTO %s(key, d) VALUES (?, ?)", 2, 2d);
+        execute("INSERT INTO %s(key, d) VALUES (?, ?)", 3, 3d);
+
+        // simple creation
+        String fSin = createFunction(KEYSPACE_PER_TEST, "double",
+                                     "CREATE FUNCTION %s ( input double ) " +
+                                     "CALLED ON NULL INPUT " +
+                                     "RETURNS double " +
+                                     "LANGUAGE java " +
+                                     "AS 'return Math.sin(input);'");
+        // check we can't recreate the same function
+        assertInvalidMessage("already exists",
+                             "CREATE FUNCTION " + fSin + " ( input double ) " +
+                             "CALLED ON NULL INPUT " +
+                             "RETURNS double " +
+                             "LANGUAGE java AS 'return Double.valueOf(Math.sin(input.doubleValue()));'");
+
+        // but that it doesn't comply with "IF NOT EXISTS"
+        execute("CREATE FUNCTION IF NOT EXISTS " + fSin + " ( input double ) " +
+                "CALLED ON NULL INPUT " +
+                "RETURNS double " +
+                "LANGUAGE java AS 'return Double.valueOf(Math.sin(input.doubleValue()));'");
+
+        // Validate that it works as expected
+        assertRows(execute("SELECT key, " + fSin + "(d) FROM %s"),
+            row(1, Math.sin(1d)),
+            row(2, Math.sin(2d)),
+            row(3, Math.sin(3d))
+        );
+
+        // Replace the method with incompatible return type
+        assertInvalidMessage("the new return type text is not compatible with the return type double of existing function",
+                             "CREATE OR REPLACE FUNCTION " + fSin + " ( input double ) " +
+                             "CALLED ON NULL INPUT " +
+                             "RETURNS text " +
+                             "LANGUAGE java AS 'return Double.valueOf(42d);'");
+
+        // proper replacement
+        execute("CREATE OR REPLACE FUNCTION " + fSin + " ( input double ) " +
+                "CALLED ON NULL INPUT " +
+                "RETURNS double " +
+                "LANGUAGE java AS 'return Double.valueOf(42d);'");
+
+        // Validate the method as been replaced
+        assertRows(execute("SELECT key, " + fSin + "(d) FROM %s"),
+            row(1, 42.0),
+            row(2, 42.0),
+            row(3, 42.0)
+        );
+
+        // same function but other keyspace
+        String fSin2 = createFunction(KEYSPACE, "double",
+                                      "CREATE FUNCTION %s ( input double ) " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS double " +
+                                      "LANGUAGE java " +
+                                      "AS 'return Math.sin(input);'");
+        assertRows(execute("SELECT key, " + fSin2 + "(d) FROM %s"),
+            row(1, Math.sin(1d)),
+            row(2, Math.sin(2d)),
+            row(3, Math.sin(3d))
+        );
+
+        // Drop
+        execute("DROP FUNCTION " + fSin);
+        execute("DROP FUNCTION " + fSin2);
+
+        // Drop unexisting function
+        assertInvalidMessage("Cannot drop non existing function", "DROP FUNCTION " + fSin);
+        // but don't complain with "IF EXISTS"
+        execute("DROP FUNCTION IF EXISTS " + fSin);
+
+        // can't drop native functions
+        assertInvalidMessage("system keyspace is not user-modifiable", "DROP FUNCTION totimestamp");
+        assertInvalidMessage("system keyspace is not user-modifiable", "DROP FUNCTION uuid");
+
+        // sin() no longer exists
+        assertInvalidMessage("Unknown function", "SELECT key, sin(d) FROM %s");
+    }
+
+    @Test
+    public void testFunctionExecution() throws Throwable
+    {
+        createTable("CREATE TABLE %s (v text PRIMARY KEY)");
+
+        execute("INSERT INTO %s(v) VALUES (?)", "aaa");
+
+        String fRepeat = createFunction(KEYSPACE_PER_TEST, "text,int",
+                                        "CREATE FUNCTION %s(v text, n int) " +
+                                        "RETURNS NULL ON NULL INPUT " +
+                                        "RETURNS text " +
+                                        "LANGUAGE java " +
+                                        "AS 'StringBuilder sb = new StringBuilder();\n" +
+                                        "    for (int i = 0; i < n; i++)\n" +
+                                        "        sb.append(v);\n" +
+                                        "    return sb.toString();'");
+
+        assertRows(execute("SELECT v FROM %s WHERE v=" + fRepeat + "(?, ?)", "a", 3), row("aaa"));
+        assertEmpty(execute("SELECT v FROM %s WHERE v=" + fRepeat + "(?, ?)", "a", 2));
+    }
+
+    @Test
+    public void testFunctionExecutionWithReversedTypeAsOutput() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, v text, PRIMARY KEY(k, v)) WITH CLUSTERING ORDER BY (v DESC)");
+
+        String fRepeat = createFunction(KEYSPACE_PER_TEST, "text",
+                                        "CREATE FUNCTION %s(v text) " +
+                                        "RETURNS NULL ON NULL INPUT " +
+                                        "RETURNS text " +
+                                        "LANGUAGE java " +
+                                        "AS 'return v + v;'");
+
+        execute("INSERT INTO %s(k, v) VALUES (?, " + fRepeat + "(?))", 1, "a");
+    }
+
+    @Test
+    public void testFunctionOverloading() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k text PRIMARY KEY, v int)");
+
+        execute("INSERT INTO %s(k, v) VALUES (?, ?)", "f2", 1);
+
+        String fOverload = createFunction(KEYSPACE_PER_TEST, "varchar",
+                                          "CREATE FUNCTION %s ( input varchar ) " +
+                                          "RETURNS NULL ON NULL INPUT " +
+                                          "RETURNS text " +
+                                          "LANGUAGE java " +
+                                          "AS 'return \"f1\";'");
+        createFunctionOverload(fOverload,
+                               "int",
+                               "CREATE OR REPLACE FUNCTION %s(i int) " +
+                               "RETURNS NULL ON NULL INPUT " +
+                               "RETURNS text " +
+                               "LANGUAGE java " +
+                               "AS 'return \"f2\";'");
+        createFunctionOverload(fOverload,
+                               "text,text",
+                               "CREATE OR REPLACE FUNCTION %s(v1 text, v2 text) " +
+                               "RETURNS NULL ON NULL INPUT " +
+                               "RETURNS text " +
+                               "LANGUAGE java " +
+                               "AS 'return \"f3\";'");
+        createFunctionOverload(fOverload,
+                               "ascii",
+                               "CREATE OR REPLACE FUNCTION %s(v ascii) " +
+                               "RETURNS NULL ON NULL INPUT " +
+                               "RETURNS text " +
+                               "LANGUAGE java " +
+                               "AS 'return \"f1\";'");
+
+        // text == varchar, so this should be considered as a duplicate
+        assertInvalidMessage("already exists",
+                             "CREATE FUNCTION " + fOverload + "(v varchar) " +
+                             "RETURNS NULL ON NULL INPUT " +
+                             "RETURNS text " +
+                             "LANGUAGE java AS 'return \"f1\";'");
+
+        assertRows(execute("SELECT " + fOverload + "(k), " + fOverload + "(v), " + fOverload + "(k, k) FROM %s"),
+            row("f1", "f2", "f3")
+        );
+
+        forcePreparedValues();
+        // This shouldn't work if we use preparation since there no way to know which overload to use
+        assertInvalidMessage("Ambiguous call to function", "SELECT v FROM %s WHERE k = " + fOverload + "(?)", "foo");
+        stopForcingPreparedValues();
+
+        // but those should since we specifically cast
+        assertEmpty(execute("SELECT v FROM %s WHERE k = " + fOverload + "((text)?)", "foo"));
+        assertRows(execute("SELECT v FROM %s WHERE k = " + fOverload + "((int)?)", 3), row(1));
+        assertEmpty(execute("SELECT v FROM %s WHERE k = " + fOverload + "((ascii)?)", "foo"));
+        // And since varchar == text, this should work too
+        assertEmpty(execute("SELECT v FROM %s WHERE k = " + fOverload + "((varchar)?)", "foo"));
+
+        // no such functions exist...
+        assertInvalidMessage("non existing function", "DROP FUNCTION " + fOverload + "(boolean)");
+        assertInvalidMessage("non existing function", "DROP FUNCTION " + fOverload + "(bigint)");
+
+        // 'overloaded' has multiple overloads - so it has to fail (CASSANDRA-7812)
+        assertInvalidMessage("matches multiple function definitions", "DROP FUNCTION " + fOverload);
+        execute("DROP FUNCTION " + fOverload + "(varchar)");
+        assertInvalidMessage("none of its type signatures match", "SELECT v FROM %s WHERE k = " + fOverload + "((text)?)", "foo");
+        execute("DROP FUNCTION " + fOverload + "(text, text)");
+        assertInvalidMessage("none of its type signatures match", "SELECT v FROM %s WHERE k = " + fOverload + "((text)?,(text)?)", "foo", "bar");
+        execute("DROP FUNCTION " + fOverload + "(ascii)");
+        assertInvalidMessage("cannot be passed as argument 0 of function", "SELECT v FROM %s WHERE k = " + fOverload + "((ascii)?)", "foo");
+        // single-int-overload must still work
+        assertRows(execute("SELECT v FROM %s WHERE k = " + fOverload + "((int)?)", 3), row(1));
+        // overloaded has just one overload now - so the following DROP FUNCTION is not ambigious (CASSANDRA-7812)
+        execute("DROP FUNCTION " + fOverload);
+    }
+
+    @Test
+    public void testCreateOrReplaceJavaFunction() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int primary key, val double)");
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 1, 1d);
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 2, 2d);
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 3, 3d);
+
+        String fName = createFunction(KEYSPACE_PER_TEST, "double",
+                "CREATE FUNCTION %s( input double ) " +
+                "CALLED ON NULL INPUT " +
+                "RETURNS double " +
+                "LANGUAGE java " +
+                "AS '\n" +
+                "  // parameter val is of type java.lang.Double\n" +
+                "  /* return type is of type java.lang.Double */\n" +
+                "  if (input == null) {\n" +
+                "    return null;\n" +
+                "  }\n" +
+                "  return Math.sin( input );\n" +
+                "';");
+
+        // just check created function
+        assertRows(execute("SELECT key, val, " + fName + "(val) FROM %s"),
+                   row(1, 1d, Math.sin(1d)),
+                   row(2, 2d, Math.sin(2d)),
+                   row(3, 3d, Math.sin(3d))
+        );
+
+        execute("CREATE OR REPLACE FUNCTION " + fName + "( input double ) " +
+                "CALLED ON NULL INPUT " +
+                "RETURNS double " +
+                "LANGUAGE java\n" +
+                "AS '\n" +
+                "  return input;\n" +
+                "';");
+
+        // check if replaced function returns correct result
+        assertRows(execute("SELECT key, val, " + fName + "(val) FROM %s"),
+                   row(1, 1d, 1d),
+                   row(2, 2d, 2d),
+                   row(3, 3d, 3d)
+        );
+    }
+
+    @Test
+    public void testFunctionInTargetKeyspace() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int primary key, val double)");
+
+        execute("CREATE TABLE " + KEYSPACE_PER_TEST + ".second_tab (key int primary key, val double)");
+
+        String fName = createFunction(KEYSPACE_PER_TEST, "double",
+                                      "CREATE OR REPLACE FUNCTION %s(val double) " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS double " +
+                                      "LANGUAGE JAVA " +
+                                      "AS 'return Double.valueOf(val);';");
+
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 1, 1d);
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 2, 2d);
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 3, 3d);
+        assertInvalidMessage("Unknown function",
+                             "SELECT key, val, " + parseFunctionName(fName).name + "(val) FROM %s");
+
+        execute("INSERT INTO " + KEYSPACE_PER_TEST + ".second_tab (key, val) VALUES (?, ?)", 1, 1d);
+        execute("INSERT INTO " + KEYSPACE_PER_TEST + ".second_tab (key, val) VALUES (?, ?)", 2, 2d);
+        execute("INSERT INTO " + KEYSPACE_PER_TEST + ".second_tab (key, val) VALUES (?, ?)", 3, 3d);
+        assertRows(execute("SELECT key, val, " + fName + "(val) FROM " + KEYSPACE_PER_TEST + ".second_tab"),
+                   row(1, 1d, 1d),
+                   row(2, 2d, 2d),
+                   row(3, 3d, 3d)
+        );
+    }
+
+    @Test
+    public void testFunctionWithReservedName() throws Throwable
+    {
+        execute("CREATE TABLE " + KEYSPACE_PER_TEST + ".second_tab (key int primary key, val double)");
+
+        String fName = createFunction(KEYSPACE_PER_TEST, "",
+                                      "CREATE OR REPLACE FUNCTION %s() " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS timestamp " +
+                                      "LANGUAGE JAVA " +
+                                      "AS 'return null;';");
+
+        execute("INSERT INTO " + KEYSPACE_PER_TEST + ".second_tab (key, val) VALUES (?, ?)", 1, 1d);
+        execute("INSERT INTO " + KEYSPACE_PER_TEST + ".second_tab (key, val) VALUES (?, ?)", 2, 2d);
+        execute("INSERT INTO " + KEYSPACE_PER_TEST + ".second_tab (key, val) VALUES (?, ?)", 3, 3d);
+
+        // ensure that system now() is executed
+        UntypedResultSet rows = execute("SELECT key, val, now() FROM " + KEYSPACE_PER_TEST + ".second_tab");
+        Assert.assertEquals(3, rows.size());
+        UntypedResultSet.Row row = rows.iterator().next();
+        Date ts = row.getTimestamp(row.getColumns().get(2).name.toString());
+        Assert.assertNotNull(ts);
+
+        // ensure that KEYSPACE_PER_TEST's now() is executed
+        rows = execute("SELECT key, val, " + fName + "() FROM " + KEYSPACE_PER_TEST + ".second_tab");
+        Assert.assertEquals(3, rows.size());
+        row = rows.iterator().next();
+        Assert.assertFalse(row.has(row.getColumns().get(2).name.toString()));
+    }
+
+    @Test
+    public void testFunctionInSystemKS() throws Throwable
+    {
+        execute("CREATE OR REPLACE FUNCTION " + KEYSPACE + ".totimestamp(val timeuuid) " +
+                "RETURNS NULL ON NULL INPUT " +
+                "RETURNS timestamp " +
+                "LANGUAGE JAVA\n" +
+
+                "AS 'return null;';");
+
+        assertInvalidMessage("system keyspace is not user-modifiable",
+                             "CREATE OR REPLACE FUNCTION system.jnft(val double) " +
+                             "RETURNS NULL ON NULL INPUT " +
+                             "RETURNS double " +
+                             "LANGUAGE JAVA\n" +
+                             "AS 'return null;';");
+        assertInvalidMessage("system keyspace is not user-modifiable",
+                             "CREATE OR REPLACE FUNCTION system.totimestamp(val timeuuid) " +
+                             "RETURNS NULL ON NULL INPUT " +
+                             "RETURNS timestamp " +
+                             "LANGUAGE JAVA\n" +
+
+                             "AS 'return null;';");
+        assertInvalidMessage("system keyspace is not user-modifiable",
+                             "DROP FUNCTION system.now");
+
+        // KS for executeInternal() is system
+        assertInvalidMessage("system keyspace is not user-modifiable",
+                             "CREATE OR REPLACE FUNCTION jnft(val double) " +
+                             "RETURNS NULL ON NULL INPUT " +
+                             "RETURNS double " +
+                             "LANGUAGE JAVA\n" +
+                             "AS 'return null;';");
+        assertInvalidMessage("system keyspace is not user-modifiable",
+                             "CREATE OR REPLACE FUNCTION totimestamp(val timeuuid) " +
+                             "RETURNS NULL ON NULL INPUT " +
+                             "RETURNS timestamp " +
+                             "LANGUAGE JAVA\n" +
+                             "AS 'return null;';");
+        assertInvalidMessage("system keyspace is not user-modifiable",
+                             "DROP FUNCTION now");
+    }
+
+    @Test
+    public void testFunctionNonExistingKeyspace() throws Throwable
+    {
+        assertInvalidMessage("to non existing keyspace",
+                             "CREATE OR REPLACE FUNCTION this_ks_does_not_exist.jnft(val double) " +
+                             "RETURNS NULL ON NULL INPUT " +
+                             "RETURNS double " +
+                             "LANGUAGE JAVA\n" +
+                             "AS 'return null;';");
+    }
+
+    @Test
+    public void testFunctionAfterOnDropKeyspace() throws Throwable
+    {
+        dropPerTestKeyspace();
+
+        assertInvalidMessage("to non existing keyspace",
+                             "CREATE OR REPLACE FUNCTION " + KEYSPACE_PER_TEST + ".jnft(val double) " +
+                             "RETURNS NULL ON NULL INPUT " +
+                             "RETURNS double " +
+                             "LANGUAGE JAVA\n" +
+                             "AS 'return null;';");
+    }
+
+    @Test
+    public void testWrongKeyspace() throws Throwable
+    {
+        String typeName = createType("CREATE TYPE %s (txt text, i int)");
+        String type = KEYSPACE + '.' + typeName;
+
+        assertInvalidMessage(String.format("Statement on keyspace %s cannot refer to a user type in keyspace %s; user types can only be used in the keyspace they are defined in",
+                                           KEYSPACE_PER_TEST, KEYSPACE),
+                             "CREATE FUNCTION " + KEYSPACE_PER_TEST + ".test_wrong_ks( val int ) " +
+                             "CALLED ON NULL INPUT " +
+                             "RETURNS " + type + " " +
+                             "LANGUAGE java\n" +
+                             "AS $$return val;$$;");
+
+        assertInvalidMessage(String.format("Statement on keyspace %s cannot refer to a user type in keyspace %s; user types can only be used in the keyspace they are defined in",
+                                           KEYSPACE_PER_TEST, KEYSPACE),
+                             "CREATE FUNCTION " + KEYSPACE_PER_TEST + ".test_wrong_ks( val " + type + " ) " +
+                             "CALLED ON NULL INPUT " +
+                             "RETURNS int " +
+                             "LANGUAGE java\n" +
+                             "AS $$return val;$$;");
+    }
+
+    @Test
+    public void testComplexNullValues() throws Throwable
+    {
+        String type = KEYSPACE + '.' + createType("CREATE TYPE %s (txt text, i int)");
+
+        createTable("CREATE TABLE %s (key int primary key, lst list<double>, st set<text>, mp map<int, boolean>," +
+                    "tup frozen<tuple<double, text, int, boolean>>, udt frozen<" + type + ">)");
+
+        String fList = createFunction(KEYSPACE, "list<double>",
+                                      "CREATE FUNCTION %s( coll list<double> ) " +
+                                      "CALLED ON NULL INPUT " +
+                                      "RETURNS list<double> " +
+                                      "LANGUAGE java\n" +
+                                      "AS $$return coll;$$;");
+        String fSet = createFunction(KEYSPACE, "set<text>",
+                                     "CREATE FUNCTION %s( coll set<text> ) " +
+                                     "CALLED ON NULL INPUT " +
+                                     "RETURNS set<text> " +
+                                     "LANGUAGE java\n" +
+                                     "AS $$return coll;$$;");
+        String fMap = createFunction(KEYSPACE, "map<int, boolean>",
+                                     "CREATE FUNCTION %s( coll map<int, boolean> ) " +
+                                     "CALLED ON NULL INPUT " +
+                                     "RETURNS map<int, boolean> " +
+                                     "LANGUAGE java\n" +
+                                     "AS $$return coll;$$;");
+        String fTup = createFunction(KEYSPACE, "tuple<double, text, int, boolean>",
+                                     "CREATE FUNCTION %s( val tuple<double, text, int, boolean> ) " +
+                                     "CALLED ON NULL INPUT " +
+                                     "RETURNS tuple<double, text, int, boolean> " +
+                                     "LANGUAGE java\n" +
+                                     "AS $$return val;$$;");
+        String fUdt = createFunction(KEYSPACE, type,
+                                     "CREATE FUNCTION %s( val " + type + " ) " +
+                                     "CALLED ON NULL INPUT " +
+                                     "RETURNS " + type + " " +
+                                     "LANGUAGE java\n" +
+                                     "AS $$return val;$$;");
+        List<Double> list = Arrays.asList(1d, 2d, 3d);
+        Set<String> set = new TreeSet<>(Arrays.asList("one", "three", "two"));
+        Map<Integer, Boolean> map = new TreeMap<>();
+        map.put(1, true);
+        map.put(2, false);
+        map.put(3, true);
+        Object t = tuple(1d, "one", 42, false);
+
+        execute("INSERT INTO %s (key, lst, st, mp, tup, udt) VALUES (1, ?, ?, ?, ?, {txt: 'one', i:1})", list, set, map, t);
+        execute("INSERT INTO %s (key, lst, st, mp, tup, udt) VALUES (2, ?, ?, ?, ?, null)", null, null, null, null);
+
+        execute("SELECT " +
+                fList + "(lst), " +
+                fSet + "(st), " +
+                fMap + "(mp), " +
+                fTup + "(tup), " +
+                fUdt + "(udt) FROM %s WHERE key = 1");
+        UntypedResultSet.Row row = execute("SELECT " +
+                                           fList + "(lst) as l, " +
+                                           fSet + "(st) as s, " +
+                                           fMap + "(mp) as m, " +
+                                           fTup + "(tup) as t, " +
+                                           fUdt + "(udt) as u " +
+                                           "FROM %s WHERE key = 1").one();
+        Assert.assertNotNull(row.getBytes("l"));
+        Assert.assertNotNull(row.getBytes("s"));
+        Assert.assertNotNull(row.getBytes("m"));
+        Assert.assertNotNull(row.getBytes("t"));
+        Assert.assertNotNull(row.getBytes("u"));
+        row = execute("SELECT " +
+                      fList + "(lst) as l, " +
+                      fSet + "(st) as s, " +
+                      fMap + "(mp) as m, " +
+                      fTup + "(tup) as t, " +
+                      fUdt + "(udt) as u " +
+                      "FROM %s WHERE key = 2").one();
+        Assert.assertNull(row.getBytes("l"));
+        Assert.assertNull(row.getBytes("s"));
+        Assert.assertNull(row.getBytes("m"));
+        Assert.assertNull(row.getBytes("t"));
+        Assert.assertNull(row.getBytes("u"));
+
+        for (int version = Server.VERSION_2; version <= maxProtocolVersion; version++)
+        {
+            Row r = executeNet(version, "SELECT " +
+                                        fList + "(lst) as l, " +
+                                        fSet + "(st) as s, " +
+                                        fMap + "(mp) as m, " +
+                                        fTup + "(tup) as t, " +
+                                        fUdt + "(udt) as u " +
+                                        "FROM %s WHERE key = 1").one();
+            Assert.assertNotNull(r.getBytesUnsafe("l"));
+            Assert.assertNotNull(r.getBytesUnsafe("s"));
+            Assert.assertNotNull(r.getBytesUnsafe("m"));
+            Assert.assertNotNull(r.getBytesUnsafe("t"));
+            Assert.assertNotNull(r.getBytesUnsafe("u"));
+            r = executeNet(version, "SELECT " +
+                                    fList + "(lst) as l, " +
+                                    fSet + "(st) as s, " +
+                                    fMap + "(mp) as m, " +
+                                    fTup + "(tup) as t, " +
+                                    fUdt + "(udt) as u " +
+                                    "FROM %s WHERE key = 2").one();
+            Assert.assertNull(r.getBytesUnsafe("l"));
+            Assert.assertNull(r.getBytesUnsafe("s"));
+            Assert.assertNull(r.getBytesUnsafe("m"));
+            Assert.assertNull(r.getBytesUnsafe("t"));
+            Assert.assertNull(r.getBytesUnsafe("u"));
+        }
+    }
+
+    @Test
+    public void testUserTypeDrop() throws Throwable
+    {
+        String type = KEYSPACE + '.' + createType("CREATE TYPE %s (txt text, i int)");
+
+        createTable("CREATE TABLE %s (key int primary key, udt frozen<" + type + ">)");
+
+        String fName = createFunction(KEYSPACE, type,
+                                      "CREATE FUNCTION %s( udt " + type + " ) " +
+                                      "CALLED ON NULL INPUT " +
+                                      "RETURNS int " +
+                                      "LANGUAGE java " +
+                                      "AS $$return " +
+                                      "     Integer.valueOf(udt.getInt(\"i\"));$$;");
+
+        FunctionName fNameName = parseFunctionName(fName);
+
+        Assert.assertEquals(1, Functions.find(fNameName).size());
+
+        ResultMessage.Prepared prepared = QueryProcessor.prepare(String.format("SELECT key, %s(udt) FROM %s.%s", fName, KEYSPACE, currentTable()),
+                                                                 ClientState.forInternalCalls(), false);
+        Assert.assertNotNull(QueryProcessor.instance.getPrepared(prepared.statementId));
+
+        // UT still referenced by table
+        assertInvalidMessage("Cannot drop user type", "DROP TYPE " + type);
+
+        execute("DROP TABLE %s");
+
+        // UT still referenced by UDF
+        assertInvalidMessage("as it is still used by function", "DROP TYPE " + type);
+
+        Assert.assertNull(QueryProcessor.instance.getPrepared(prepared.statementId));
+
+        // function stays
+        Assert.assertEquals(1, Functions.find(fNameName).size());
+    }
+
+    @Test
+    public void testDuplicateArgNames() throws Throwable
+    {
+        assertInvalidMessage("duplicate argument names for given function",
+                             "CREATE OR REPLACE FUNCTION " + KEYSPACE + ".scrinv(val double, val text) " +
+                             "RETURNS NULL ON NULL INPUT " +
+                             "RETURNS text " +
+                             "LANGUAGE javascript\n" +
+                             "AS '\"foo bar\";';");
+    }
+
+    private static class TypesTestDef
+    {
+        final String udfType;
+        final String tableType;
+        final String columnName;
+        final Object referenceValue;
+
+        String fCheckArgAndReturn;
+
+        String fCalledOnNull;
+        String fReturnsNullOnNull;
+
+        TypesTestDef(String udfType, String tableType, String columnName, Object referenceValue)
+        {
+            this.udfType = udfType;
+            this.tableType = tableType;
+            this.columnName = columnName;
+            this.referenceValue = referenceValue;
+        }
+    }
+
+    @Test
+    public void testTypesWithAndWithoutNulls() throws Throwable
+    {
+        // test various combinations of types against UDFs with CALLED ON NULL or RETURNS NULL ON NULL
+
+        String type = createType("CREATE TYPE %s (txt text, i int)");
+
+        TypesTestDef[] typeDefs =
+        {
+        //                udf type,            table type,                 column, reference value
+        new TypesTestDef("timestamp", "timestamp", "ts", new Date()),
+        new TypesTestDef("date", "date", "dt", 12345),
+        new TypesTestDef("time", "time", "tim", 12345L),
+        new TypesTestDef("uuid", "uuid", "uu", UUID.randomUUID()),
+        new TypesTestDef("timeuuid", "timeuuid", "tu", UUIDGen.getTimeUUID()),
+        new TypesTestDef("tinyint", "tinyint", "ti", (byte) 42),
+        new TypesTestDef("smallint", "smallint", "si", (short) 43),
+        new TypesTestDef("int", "int", "i", 44),
+        new TypesTestDef("bigint", "bigint", "b", 45L),
+        new TypesTestDef("float", "float", "f", 46f),
+        new TypesTestDef("double", "double", "d", 47d),
+        new TypesTestDef("boolean", "boolean", "x", true),
+        new TypesTestDef("ascii", "ascii", "a", "tqbfjutld"),
+        new TypesTestDef("text", "text", "t", "k\u00f6lsche jung"),
+        //new TypesTestDef(type,                 "frozen<" + type + '>',     "u",    null),
+        new TypesTestDef("tuple<int, text>", "frozen<tuple<int, text>>", "tup", tuple(1, "foo"))
+        };
+
+        String createTableDDL = "CREATE TABLE %s (key int PRIMARY KEY";
+        String insertDML = "INSERT INTO %s (key";
+        List<Object> values = new ArrayList<>();
+        for (TypesTestDef typeDef : typeDefs)
+        {
+            createTableDDL += ", " + typeDef.columnName + ' ' + typeDef.tableType;
+            insertDML += ", " + typeDef.columnName;
+            String typeName = typeDef.udfType;
+            typeDef.fCheckArgAndReturn = createFunction(KEYSPACE,
+                                                        typeName,
+                                                        "CREATE OR REPLACE FUNCTION %s(val " + typeName + ") " +
+                                                        "CALLED ON NULL INPUT " +
+                                                        "RETURNS " + typeName + ' ' +
+                                                        "LANGUAGE java\n" +
+                                                        "AS 'return val;';");
+            typeDef.fCalledOnNull = createFunction(KEYSPACE,
+                                                   typeName,
+                                                   "CREATE OR REPLACE FUNCTION %s(val " + typeName + ") " +
+                                                   "CALLED ON NULL INPUT " +
+                                                   "RETURNS text " +
+                                                   "LANGUAGE java\n" +
+                                                   "AS 'return \"called\";';");
+            typeDef.fReturnsNullOnNull = createFunction(KEYSPACE,
+                                                        typeName,
+                                                        "CREATE OR REPLACE FUNCTION %s(val " + typeName + ") " +
+                                                        "RETURNS NULL ON NULL INPUT " +
+                                                        "RETURNS text " +
+                                                        "LANGUAGE java\n" +
+                                                        "AS 'return \"called\";';");
+            values.add(typeDef.referenceValue);
+        }
+
+        createTableDDL += ')';
+        createTable(createTableDDL);
+
+        insertDML += ") VALUES (1";
+        for (TypesTestDef ignored : typeDefs)
+            insertDML += ", ?";
+        insertDML += ')';
+
+        execute(insertDML, values.toArray());
+
+        // second row with null values
+        for (int i = 0; i < values.size(); i++)
+            values.set(i, null);
+        execute(insertDML.replace('1', '2'), values.toArray());
+
+        // check argument input + return
+        for (TypesTestDef typeDef : typeDefs)
+        {
+            assertRows(execute("SELECT " + typeDef.fCheckArgAndReturn + '(' + typeDef.columnName + ") FROM %s WHERE key = 1"),
+                       row(new Object[]{ typeDef.referenceValue }));
+        }
+
+        // check for CALLED ON NULL INPUT with non-null arguments
+        for (TypesTestDef typeDef : typeDefs)
+        {
+            assertRows(execute("SELECT " + typeDef.fCalledOnNull + '(' + typeDef.columnName + ") FROM %s WHERE key = 1"),
+                       row(new Object[]{ "called" }));
+        }
+
+        // check for CALLED ON NULL INPUT with null arguments
+        for (TypesTestDef typeDef : typeDefs)
+        {
+            assertRows(execute("SELECT " + typeDef.fCalledOnNull + '(' + typeDef.columnName + ") FROM %s WHERE key = 2"),
+                       row(new Object[]{ "called" }));
+        }
+
+        // check for RETURNS NULL ON NULL INPUT with non-null arguments
+        for (TypesTestDef typeDef : typeDefs)
+        {
+            assertRows(execute("SELECT " + typeDef.fReturnsNullOnNull + '(' + typeDef.columnName + ") FROM %s WHERE key = 1"),
+                       row(new Object[]{ "called" }));
+        }
+
+        // check for RETURNS NULL ON NULL INPUT with null arguments
+        for (TypesTestDef typeDef : typeDefs)
+        {
+            assertRows(execute("SELECT " + typeDef.fReturnsNullOnNull + '(' + typeDef.columnName + ") FROM %s WHERE key = 2"),
+                       row(new Object[]{ null }));
+        }
+
+    }
+
+    @Test
+    public void testReplaceAllowNulls() throws Throwable
+    {
+        String fNulls = createFunction(KEYSPACE,
+                                       "int",
+                                       "CREATE OR REPLACE FUNCTION %s(val int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS text " +
+                                       "LANGUAGE java\n" +
+                                       "AS 'return \"foo bar\";';");
+        String fNoNulls = createFunction(KEYSPACE,
+                                         "int",
+                                         "CREATE OR REPLACE FUNCTION %s(val int) " +
+                                         "RETURNS NULL ON NULL INPUT " +
+                                         "RETURNS text " +
+                                         "LANGUAGE java\n" +
+                                         "AS 'return \"foo bar\";';");
+
+        assertInvalid("CREATE OR REPLACE FUNCTION " + fNulls + "(val int) " +
+                      "RETURNS NULL ON NULL INPUT " +
+                      "RETURNS text " +
+                      "LANGUAGE java\n" +
+                      "AS 'return \"foo bar\";';");
+        assertInvalid("CREATE OR REPLACE FUNCTION " + fNoNulls + "(val int) " +
+                      "CALLED ON NULL INPUT " +
+                      "RETURNS text " +
+                      "LANGUAGE java\n" +
+                      "AS 'return \"foo bar\";';");
+
+        execute("CREATE OR REPLACE FUNCTION " + fNulls + "(val int) " +
+                "CALLED ON NULL INPUT " +
+                "RETURNS text " +
+                "LANGUAGE java\n" +
+                "AS 'return \"foo bar\";';");
+        execute("CREATE OR REPLACE FUNCTION " + fNoNulls + "(val int) " +
+                "RETURNS NULL ON NULL INPUT " +
+                "RETURNS text " +
+                "LANGUAGE java\n" +
+                "AS 'return \"foo bar\";';");
+    }
+
+    @Test
+    public void testBrokenFunction() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int primary key, dval double)");
+        execute("INSERT INTO %s (key, dval) VALUES (?, ?)", 1, 1d);
+
+        String fName = createFunction(KEYSPACE_PER_TEST, "double",
+                                      "CREATE OR REPLACE FUNCTION %s(val double) " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS double " +
+                                      "LANGUAGE JAVA\n" +
+                                      "AS 'throw new RuntimeException();';");
+
+        UDFunction f = (UDFunction) Functions.find(parseFunctionName(fName)).get(0);
+
+        Functions.addOrReplaceFunction(UDFunction.createBrokenFunction(f.name(), f.argNames(), f.argTypes(), f.returnType(), true,
+                                                                       "java", f.body(), new InvalidRequestException("foo bar is broken")));
+
+        assertInvalidThrowMessage("foo bar is broken", InvalidRequestException.class,
+                                  "SELECT key, " + fName + "(dval) FROM %s");
+    }
+
+    @Test
+    public void testFunctionExecutionExceptionNet() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int primary key, dval double)");
+        execute("INSERT INTO %s (key, dval) VALUES (?, ?)", 1, 1d);
+
+        String fName = createFunction(KEYSPACE_PER_TEST, "double",
+                                      "CREATE OR REPLACE FUNCTION %s(val double) " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS double " +
+                                      "LANGUAGE JAVA\n" +
+                                      "AS 'throw new RuntimeException();'");
+
+        for (int version = Server.VERSION_2; version <= maxProtocolVersion; version++)
+        {
+            try
+            {
+                assertRowsNet(version,
+                              executeNet(version, "SELECT " + fName + "(dval) FROM %s WHERE key = 1"));
+                Assert.fail();
+            }
+            catch (com.datastax.driver.core.exceptions.FunctionExecutionException fee)
+            {
+                // Java driver neither throws FunctionExecutionException nor does it set the exception code correctly
+                Assert.assertTrue(version >= Server.VERSION_4);
+            }
+            catch (InvalidQueryException e)
+            {
+                Assert.assertTrue(version < Server.VERSION_4);
+            }
+        }
+    }
+}
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
index d96abdb..3803e5c 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
@@ -45,6 +45,17 @@
     }
 
     @Test
+    public void testInvalidInputForUserType() throws Throwable
+    {
+        String myType = createType("CREATE TYPE %s (f int)");
+        createTable("CREATE TABLE %s(pk int PRIMARY KEY, t frozen<" + myType + ">)");
+        assertInvalidMessage("Not enough bytes to read 0th field f",
+                             "INSERT INTO %s (pk, t) VALUES (?, ?)", 1, "test");
+        assertInvalidMessage("Not enough bytes to read 0th field f",
+                             "INSERT INTO %s (pk, t) VALUES (?, ?)", 1, Long.MAX_VALUE);
+    }
+
+    @Test
     public void testCassandra8105() throws Throwable
     {
         String ut1 = createType("CREATE TYPE %s (a int, b int)");
@@ -126,6 +137,21 @@
     }
 
     @Test
+    public void testUDTWithUnsetValues() throws Throwable
+    {
+        // set up
+        String myType = createType("CREATE TYPE %s (x int, y int)");
+        String myOtherType = createType("CREATE TYPE %s (a frozen<" + myType + ">)");
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, v frozen<" + myType + ">, z frozen<" + myOtherType + ">)");
+
+        assertInvalidMessage("Invalid unset value for field 'y' of user defined type " + myType,
+                             "INSERT INTO %s (k, v) VALUES (10, {x:?, y:?})", 1, unset());
+
+        assertInvalidMessage("Invalid unset value for field 'y' of user defined type " + myType,
+                             "INSERT INTO %s (k, v, z) VALUES (10, {x:?, y:?}, {a:{x: ?, y: ?}})", 1, 1, 1, unset());
+    }
+
+    @Test
     public void testAlteringUserTypeNestedWithinMap() throws Throwable
     {
         // test frozen and non-frozen collections
diff --git a/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/CrcCheckChanceTest.java b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/CrcCheckChanceTest.java
index dcea930..98d7d70 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/CrcCheckChanceTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/CrcCheckChanceTest.java
@@ -143,7 +143,7 @@
         }
 
         DatabaseDescriptor.setCompactionThroughputMbPerSec(1);
-        List<Future<?>> futures = CompactionManager.instance.submitMaximal(cfs, CompactionManager.GC_ALL); 
+        List<Future<?>> futures = CompactionManager.instance.submitMaximal(cfs, CompactionManager.getDefaultGcBefore(cfs), false); 
         execute("DROP TABLE %s");
 
         try
diff --git a/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/PgStringTest.java b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/PgStringTest.java
new file mode 100644
index 0000000..0d03b94
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/PgStringTest.java
@@ -0,0 +1,77 @@
+/*
+ * 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.cql3.validation.miscellaneous;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.exceptions.SyntaxException;
+
+public class PgStringTest extends CQLTester
+{
+    @Test
+    public void testPgSyleFunction() throws Throwable
+    {
+        execute("create or replace function "+KEYSPACE+".pgfun1 ( input double ) called on null input returns text language java\n" +
+                "AS $$return \"foobar\";$$");
+    }
+
+    @Test
+    public void testPgSyleInsert() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key ascii primary key, val text)");
+
+        // some non-terminated pg-strings
+        assertInvalidSyntax("INSERT INTO %s (key, val) VALUES ($ $key_empty$$, $$'' value for empty$$)");
+        assertInvalidSyntax("INSERT INTO %s (key, val) VALUES ($$key_empty$$, $$'' value for empty$ $)");
+        assertInvalidSyntax("INSERT INTO %s (key, val) VALUES ($$key_empty$ $, $$'' value for empty$$)");
+
+        // different pg-style markers for multiple strings
+        execute("INSERT INTO %s (key, val) VALUES ($$prim$ $ $key$$, $$some '' arbitrary value$$)");
+        // same empty pg-style marker for multiple strings
+        execute("INSERT INTO %s (key, val) VALUES ($$key_empty$$, $$'' value for empty$$)");
+        // stange but valid pg-style
+        execute("INSERT INTO %s (key, val) VALUES ($$$foo$_$foo$$, $$$'' value for empty$$)");
+        // these are conventional quoted strings
+        execute("INSERT INTO %s (key, val) VALUES ('$txt$key$$$$txt$', '$txt$'' other value$txt$')");
+
+        assertRows(execute("SELECT key, val FROM %s WHERE key='prim$ $ $key'"),
+                   row("prim$ $ $key", "some '' arbitrary value")
+        );
+        assertRows(execute("SELECT key, val FROM %s WHERE key='key_empty'"),
+                   row("key_empty", "'' value for empty")
+        );
+        assertRows(execute("SELECT key, val FROM %s WHERE key='$foo$_$foo'"),
+                   row("$foo$_$foo", "$'' value for empty")
+        );
+        assertRows(execute("SELECT key, val FROM %s WHERE key='$txt$key$$$$txt$'"),
+                   row("$txt$key$$$$txt$", "$txt$' other value$txt$")
+        );
+
+        // invalid syntax
+        assertInvalidSyntax("INSERT INTO %s (key, val) VALUES ($ascii$prim$$$key$invterm$, $txt$some '' arbitrary value$txt$)");
+    }
+
+    @Test(expected = SyntaxException.class)
+    public void testMarkerPgFail() throws Throwable
+    {
+        // must throw SyntaxException - not StringIndexOutOfBoundsException or similar
+        execute("create function "+KEYSPACE+".pgfun1 ( input double ) called on null input returns bigint language java\n" +
+                "AS $javasrc$return 0L;$javasrc$;");
+    }
+}
diff --git a/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/RoleSyntaxTest.java b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/RoleSyntaxTest.java
new file mode 100644
index 0000000..0cb1de2
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/RoleSyntaxTest.java
@@ -0,0 +1,161 @@
+/*
+ * 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.cql3.validation.miscellaneous;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.CQLTester;
+
+public class RoleSyntaxTest extends CQLTester
+{
+    private final String NO_QUOTED_USERNAME = "Quoted strings are are not supported for user names " +
+                                              "and USER is deprecated, please use ROLE";
+    @Test
+    public void standardOptionsSyntaxTest() throws Throwable
+    {
+        assertValidSyntax("CREATE ROLE r WITH LOGIN = true AND SUPERUSER = false AND PASSWORD = 'foo'");
+        assertValidSyntax("CREATE ROLE r WITH PASSWORD = 'foo' AND LOGIN = true AND SUPERUSER = false");
+        assertValidSyntax("CREATE ROLE r WITH SUPERUSER = true AND PASSWORD = 'foo' AND LOGIN = false");
+        assertValidSyntax("CREATE ROLE r WITH LOGIN = true AND PASSWORD = 'foo' AND SUPERUSER = false");
+        assertValidSyntax("CREATE ROLE r WITH SUPERUSER = true AND PASSWORD = 'foo' AND LOGIN = false");
+
+        assertValidSyntax("ALTER ROLE r WITH LOGIN = true AND SUPERUSER = false AND PASSWORD = 'foo'");
+        assertValidSyntax("ALTER ROLE r WITH PASSWORD = 'foo' AND LOGIN = true AND SUPERUSER = false");
+        assertValidSyntax("ALTER ROLE r WITH SUPERUSER = true AND PASSWORD = 'foo' AND LOGIN = false");
+        assertValidSyntax("ALTER ROLE r WITH LOGIN = true AND PASSWORD = 'foo' AND SUPERUSER = false");
+        assertValidSyntax("ALTER ROLE r WITH SUPERUSER = true AND PASSWORD = 'foo' AND LOGIN = false");
+    }
+
+    @Test
+    public void customOptionsSyntaxTest() throws Throwable
+    {
+        assertValidSyntax("CREATE ROLE r WITH OPTIONS = {'a':'b', 'b':1}");
+        assertInvalidSyntax("CREATE ROLE r WITH OPTIONS = 'term'");
+        assertInvalidSyntax("CREATE ROLE r WITH OPTIONS = 99");
+
+        assertValidSyntax("ALTER ROLE r WITH OPTIONS = {'a':'b', 'b':1}");
+        assertInvalidSyntax("ALTER ROLE r WITH OPTIONS = 'term'");
+        assertInvalidSyntax("ALTER ROLE r WITH OPTIONS = 99");
+    }
+
+    @Test
+    public void createSyntaxTest() throws Throwable
+    {
+        assertValidSyntax("CREATE ROLE r1");
+        assertValidSyntax("CREATE ROLE 'r1'");
+        assertValidSyntax("CREATE ROLE \"r1\"");
+        assertValidSyntax("CREATE ROLE $$r1$$");
+        assertValidSyntax("CREATE ROLE $$ r1 ' x $ x ' $$");
+        assertValidSyntax("CREATE USER u1");
+        assertValidSyntax("CREATE USER 'u1'");
+        assertValidSyntax("CREATE USER $$u1$$");
+        assertValidSyntax("CREATE USER $$ u1 ' x $ x ' $$");
+        // user names may not be quoted names
+        assertInvalidSyntax("CREATE USER \"u1\"", NO_QUOTED_USERNAME);
+    }
+
+    @Test
+    public void dropSyntaxTest() throws Throwable
+    {
+        assertValidSyntax("DROP ROLE r1");
+        assertValidSyntax("DROP ROLE 'r1'");
+        assertValidSyntax("DROP ROLE \"r1\"");
+        assertValidSyntax("DROP ROLE $$r1$$");
+        assertValidSyntax("DROP ROLE $$ r1 ' x $ x ' $$");
+        assertValidSyntax("DROP USER u1");
+        assertValidSyntax("DROP USER 'u1'");
+        assertValidSyntax("DROP USER $$u1$$");
+        assertValidSyntax("DROP USER $$ u1 ' x $ x ' $$");
+        // user names may not be quoted names
+        assertInvalidSyntax("DROP USER \"u1\"", NO_QUOTED_USERNAME);
+    }
+
+    @Test
+    public void alterSyntaxTest() throws Throwable
+    {
+        assertValidSyntax("ALTER ROLE r1 WITH PASSWORD = 'password'");
+        assertValidSyntax("ALTER ROLE 'r1' WITH PASSWORD = 'password'");
+        assertValidSyntax("ALTER ROLE \"r1\" WITH PASSWORD = 'password'");
+        assertValidSyntax("ALTER ROLE $$r1$$ WITH PASSWORD = 'password'");
+        assertValidSyntax("ALTER ROLE $$ r1 ' x $ x ' $$ WITH PASSWORD = 'password'");
+        // ALTER has slightly different form for USER (no =)
+        assertValidSyntax("ALTER USER u1 WITH PASSWORD 'password'");
+        assertValidSyntax("ALTER USER 'u1' WITH PASSWORD 'password'");
+        assertValidSyntax("ALTER USER $$u1$$ WITH PASSWORD 'password'");
+        assertValidSyntax("ALTER USER $$ u1 ' x $ x ' $$ WITH PASSWORD 'password'");
+        // user names may not be quoted names
+        assertInvalidSyntax("ALTER USER \"u1\" WITH PASSWORD 'password'", NO_QUOTED_USERNAME);
+    }
+
+    @Test
+    public void grantRevokePermissionsSyntaxTest() throws Throwable
+    {
+        // grant/revoke on RoleResource
+        assertValidSyntax("GRANT ALTER ON ROLE r1 TO r2");
+        assertValidSyntax("GRANT ALTER ON ROLE 'r1' TO \"r2\"");
+        assertValidSyntax("GRANT ALTER ON ROLE \"r1\" TO 'r2'");
+        assertValidSyntax("GRANT ALTER ON ROLE $$r1$$ TO $$ r '2' $$");
+        assertValidSyntax("REVOKE ALTER ON ROLE r1 FROM r2");
+        assertValidSyntax("REVOKE ALTER ON ROLE 'r1' FROM \"r2\"");
+        assertValidSyntax("REVOKE ALTER ON ROLE \"r1\" FROM 'r2'");
+        assertValidSyntax("REVOKE ALTER ON ROLE $$r1$$ FROM $$ r '2' $$");
+
+        // grant/revoke on DataResource
+        assertValidSyntax("GRANT SELECT ON KEYSPACE ks TO r1");
+        assertValidSyntax("GRANT SELECT ON KEYSPACE ks TO 'r1'");
+        assertValidSyntax("GRANT SELECT ON KEYSPACE ks TO \"r1\"");
+        assertValidSyntax("GRANT SELECT ON KEYSPACE ks TO $$ r '1' $$");
+        assertValidSyntax("REVOKE SELECT ON KEYSPACE ks FROM r1");
+        assertValidSyntax("REVOKE SELECT ON KEYSPACE ks FROM 'r1'");
+        assertValidSyntax("REVOKE SELECT ON KEYSPACE ks FROM \"r1\"");
+        assertValidSyntax("REVOKE SELECT ON KEYSPACE ks FROM $$ r '1' $$");
+    }
+
+    @Test
+    public void listPermissionsSyntaxTest() throws Throwable
+    {
+        assertValidSyntax("LIST ALL PERMISSIONS ON ALL ROLES OF r1");
+        assertValidSyntax("LIST ALL PERMISSIONS ON ALL ROLES OF 'r1'");
+        assertValidSyntax("LIST ALL PERMISSIONS ON ALL ROLES OF \"r1\"");
+        assertValidSyntax("LIST ALL PERMISSIONS ON ALL ROLES OF $$ r '1' $$");
+        assertValidSyntax("LIST ALL PERMISSIONS ON ROLE 'r1' OF r2");
+        assertValidSyntax("LIST ALL PERMISSIONS ON ROLE \"r1\" OF r2");
+        assertValidSyntax("LIST ALL PERMISSIONS ON ROLE $$ r '1' $$ OF r2");
+        assertValidSyntax("LIST ALL PERMISSIONS ON ROLE 'r1' OF 'r2'");
+        assertValidSyntax("LIST ALL PERMISSIONS ON ROLE \"r1\" OF \"r2\"");
+        assertValidSyntax("LIST ALL PERMISSIONS ON ROLE $$r1$$ OF $$ r '2' $$");
+
+        assertValidSyntax("LIST ALL PERMISSIONS ON ALL KEYSPACES OF r1");
+        assertValidSyntax("LIST ALL PERMISSIONS ON ALL KEYSPACES OF 'r1'");
+        assertValidSyntax("LIST ALL PERMISSIONS ON ALL KEYSPACES OF \"r1\"");
+        assertValidSyntax("LIST ALL PERMISSIONS ON ALL KEYSPACES OF $$ r '1' $$");
+        assertValidSyntax("LIST ALL PERMISSIONS OF r1");
+        assertValidSyntax("LIST ALL PERMISSIONS OF 'r1'");
+        assertValidSyntax("LIST ALL PERMISSIONS OF \"r1\"");
+        assertValidSyntax("LIST ALL PERMISSIONS OF $$ r '1' $$");
+    }
+
+    @Test
+    public void listRolesSyntaxTest() throws Throwable
+    {
+        assertValidSyntax("LIST ROLES OF r1");
+        assertValidSyntax("LIST ROLES OF 'r1'");
+        assertValidSyntax("LIST ROLES OF \"r1\"");
+        assertValidSyntax("LIST ROLES OF $$ r '1' $$");
+    }
+}
diff --git a/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/TombstonesTest.java b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/TombstonesTest.java
index 857eb40..5980372 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/TombstonesTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/miscellaneous/TombstonesTest.java
@@ -19,10 +19,11 @@
 
 import java.util.concurrent.TimeUnit;
 
+import com.google.common.base.Throwables;
+
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
-
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.db.filter.TombstoneOverwhelmingException;
@@ -85,7 +86,10 @@
         }
         catch (Throwable e)
         {
-            assertTrue(e instanceof TombstoneOverwhelmingException);
+            String error = "Expected exception instanceof TombstoneOverwhelmingException instead got "
+                          + System.lineSeparator()
+                          + Throwables.getStackTraceAsString(e);
+            assertTrue(error, e instanceof TombstoneOverwhelmingException);
         }
     }
 
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java
new file mode 100644
index 0000000..e7f47a2
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/AggregationTest.java
@@ -0,0 +1,1742 @@
+/*
+ * 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.cql3.validation.operations;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.text.SimpleDateFormat;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.TimeZone;
+
+import org.apache.commons.lang3.time.DateUtils;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.cql3.UntypedResultSet.Row;
+import org.apache.cassandra.cql3.functions.Functions;
+import org.apache.cassandra.cql3.functions.UDAggregate;
+import org.apache.cassandra.db.SystemKeyspace;
+import org.apache.cassandra.exceptions.FunctionExecutionException;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.transport.Event;
+import org.apache.cassandra.transport.messages.ResultMessage;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+public class AggregationTest extends CQLTester
+{
+    @Test
+    public void testNonExistingOnes() throws Throwable
+    {
+        assertInvalidMessage("Cannot drop non existing aggregate", "DROP AGGREGATE " + KEYSPACE + ".aggr_does_not_exist");
+        assertInvalidMessage("Cannot drop non existing aggregate", "DROP AGGREGATE " + KEYSPACE + ".aggr_does_not_exist(int,text)");
+        assertInvalidMessage("Cannot drop non existing aggregate", "DROP AGGREGATE keyspace_does_not_exist.aggr_does_not_exist");
+        assertInvalidMessage("Cannot drop non existing aggregate", "DROP AGGREGATE keyspace_does_not_exist.aggr_does_not_exist(int,text)");
+
+        execute("DROP AGGREGATE IF EXISTS " + KEYSPACE + ".aggr_does_not_exist");
+        execute("DROP AGGREGATE IF EXISTS " + KEYSPACE + ".aggr_does_not_exist(int,text)");
+        execute("DROP AGGREGATE IF EXISTS keyspace_does_not_exist.aggr_does_not_exist");
+        execute("DROP AGGREGATE IF EXISTS keyspace_does_not_exist.aggr_does_not_exist(int,text)");
+    }
+
+    @Test
+    public void testFunctions() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c double, d decimal, e smallint, f tinyint, primary key (a, b))");
+
+        // Test with empty table
+        assertColumnNames(execute("SELECT COUNT(*) FROM %s"), "count");
+        assertRows(execute("SELECT COUNT(*) FROM %s"), row(0L));
+        assertColumnNames(execute("SELECT max(b), min(b), sum(b), avg(b)," +
+                                  "max(c), sum(c), avg(c)," +
+                                  "sum(d), avg(d)," +
+                                  "max(e), min(e), sum(e), avg(e)," +
+                                  "max(f), min(f), sum(f), avg(f) FROM %s"),
+                          "system.max(b)", "system.min(b)", "system.sum(b)", "system.avg(b)",
+                          "system.max(c)", "system.sum(c)", "system.avg(c)",
+                          "system.sum(d)", "system.avg(d)",
+                          "system.max(e)", "system.min(e)", "system.sum(e)", "system.avg(e)",
+                          "system.max(f)", "system.min(f)", "system.sum(f)", "system.avg(f)");
+        assertRows(execute("SELECT max(b), min(b), sum(b), avg(b)," +
+                           "max(c), sum(c), avg(c)," +
+                           "sum(d), avg(d)," +
+                           "max(e), min(e), sum(e), avg(e)," +
+                           "max(f), min(f), sum(f), avg(f) FROM %s"),
+                   row(null, null, 0, 0, null, 0.0, 0.0, new BigDecimal("0"), new BigDecimal("0"),
+                       null, null, (short)0, (short)0,
+                       null, null, (byte)0, (byte)0));
+
+        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (1, 1, 11.5, 11.5, 1, 1)");
+        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (1, 2, 9.5, 1.5, 2, 2)");
+        execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (1, 3, 9.0, 2.0, 3, 3)");
+
+        assertRows(execute("SELECT max(b), min(b), sum(b), avg(b) , max(c), sum(c), avg(c), sum(d), avg(d)," +
+                           "max(e), min(e), sum(e), avg(e)," +
+                           "max(f), min(f), sum(f), avg(f)" +
+                           " FROM %s"),
+                   row(3, 1, 6, 2, 11.5, 30.0, 10.0, new BigDecimal("15.0"), new BigDecimal("5.0"),
+                       (short)3, (short)1, (short)6, (short)2,
+                       (byte)3, (byte)1, (byte)6, (byte)2));
+
+        execute("INSERT INTO %s (a, b, d) VALUES (1, 5, 1.0)");
+        assertRows(execute("SELECT COUNT(*) FROM %s"), row(4L));
+        assertRows(execute("SELECT COUNT(1) FROM %s"), row(4L));
+        assertRows(execute("SELECT COUNT(b), count(c), count(e), count(f) FROM %s"), row(4L, 3L, 3L, 3L));
+        // Makes sure that LIMIT does not affect the result of aggregates
+        assertRows(execute("SELECT COUNT(b), count(c), count(e), count(f) FROM %s LIMIT 2"), row(4L, 3L, 3L, 3L));
+        assertRows(execute("SELECT COUNT(b), count(c), count(e), count(f) FROM %s WHERE a = 1 LIMIT 2"),
+                   row(4L, 3L, 3L, 3L));
+    }
+
+    @Test
+    public void testCountStarFunction() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c double, primary key (a, b))");
+
+        // Test with empty table
+        assertColumnNames(execute("SELECT COUNT(*) FROM %s"), "count");
+        assertRows(execute("SELECT COUNT(*) FROM %s"), row(0L));
+        assertColumnNames(execute("SELECT COUNT(1) FROM %s"), "count");
+        assertRows(execute("SELECT COUNT(1) FROM %s"), row(0L));
+        assertColumnNames(execute("SELECT COUNT(*), COUNT(*) FROM %s"), "count", "count");
+        assertRows(execute("SELECT COUNT(*), COUNT(*) FROM %s"), row(0L, 0L));
+
+        // Test with alias
+        assertColumnNames(execute("SELECT COUNT(*) as myCount FROM %s"), "mycount");
+        assertRows(execute("SELECT COUNT(*) as myCount FROM %s"), row(0L));
+        assertColumnNames(execute("SELECT COUNT(1) as myCount FROM %s"), "mycount");
+        assertRows(execute("SELECT COUNT(1) as myCount FROM %s"), row(0L));
+
+        // Test invalid call
+        assertInvalidSyntaxMessage("Only COUNT(1) is supported, got COUNT(2)", "SELECT COUNT(2) FROM %s");
+
+        // Test with other aggregates
+        assertColumnNames(execute("SELECT COUNT(*), max(b), b FROM %s"), "count", "system.max(b)", "b");
+        assertRows(execute("SELECT COUNT(*), max(b), b  FROM %s"), row(0L, null, null));
+        assertColumnNames(execute("SELECT COUNT(1), max(b), b FROM %s"), "count", "system.max(b)", "b");
+        assertRows(execute("SELECT COUNT(1), max(b), b  FROM %s"), row(0L, null, null));
+
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 1, 11.5)");
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 2, 9.5)");
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 3, 9.0)");
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 5, 1.0)");
+
+        assertRows(execute("SELECT COUNT(*) FROM %s"), row(4L));
+        assertRows(execute("SELECT COUNT(1) FROM %s"), row(4L));
+        assertRows(execute("SELECT max(b), b, COUNT(*) FROM %s"), row(5, 1, 4L));
+        assertRows(execute("SELECT max(b), COUNT(1), b FROM %s"), row(5, 4L, 1));
+        // Makes sure that LIMIT does not affect the result of aggregates
+        assertRows(execute("SELECT max(b), COUNT(1), b FROM %s LIMIT 2"), row(5, 4L, 1));
+        assertRows(execute("SELECT max(b), COUNT(1), b FROM %s WHERE a = 1 LIMIT 2"), row(5, 4L, 1));
+    }
+
+    @Test
+    public void testAggregateWithColumns() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, primary key (a, b))");
+
+        // Test with empty table
+        assertColumnNames(execute("SELECT count(b), max(b) as max, b, c as first FROM %s"),
+                          "system.count(b)", "max", "b", "first");
+        assertRows(execute("SELECT count(b), max(b) as max, b, c as first FROM %s"),
+                           row(0L, null, null, null));
+
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 2, null)");
+        execute("INSERT INTO %s (a, b, c) VALUES (2, 4, 6)");
+        execute("INSERT INTO %s (a, b, c) VALUES (4, 8, 12)");
+
+        assertRows(execute("SELECT count(b), max(b) as max, b, c as first FROM %s"),
+                   row(3L, 8, 2, null));
+    }
+
+    @Test
+    public void testAggregateOnCounters() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b counter, primary key (a))");
+
+        // Test with empty table
+        assertColumnNames(execute("SELECT count(b), max(b) as max, b FROM %s"),
+                          "system.count(b)", "max", "b");
+        assertRows(execute("SELECT count(b), max(b) as max, b FROM %s"),
+                   row(0L, null, null));
+
+        execute("UPDATE %s SET b = b + 1 WHERE a = 1");
+        execute("UPDATE %s SET b = b + 1 WHERE a = 1");
+
+        assertRows(execute("SELECT count(b), max(b) as max, min(b) as min, avg(b) as avg, sum(b) as sum FROM %s"),
+                   row(1L, 2L, 2L, 2L, 2L));
+        flush();
+        assertRows(execute("SELECT count(b), max(b) as max, min(b) as min, avg(b) as avg, sum(b) as sum FROM %s"),
+                   row(1L, 2L, 2L, 2L, 2L));
+
+        execute("UPDATE %s SET b = b + 2 WHERE a = 1");
+
+        assertRows(execute("SELECT count(b), max(b) as max, min(b) as min, avg(b) as avg, sum(b) as sum FROM %s"),
+                   row(1L, 4L, 4L, 4L, 4L));
+
+        execute("UPDATE %s SET b = b - 2 WHERE a = 1");
+
+        assertRows(execute("SELECT count(b), max(b) as max, min(b) as min, avg(b) as avg, sum(b) as sum FROM %s"),
+                   row(1L, 2L, 2L, 2L, 2L));
+        flush();
+        assertRows(execute("SELECT count(b), max(b) as max, min(b) as min, avg(b) as avg, sum(b) as sum FROM %s"),
+                   row(1L, 2L, 2L, 2L, 2L));
+
+        execute("UPDATE %s SET b = b + 1 WHERE a = 2");
+        execute("UPDATE %s SET b = b + 1 WHERE a = 2");
+        execute("UPDATE %s SET b = b + 2 WHERE a = 2");
+
+        assertRows(execute("SELECT count(b), max(b) as max, min(b) as min, avg(b) as avg, sum(b) as sum FROM %s"),
+                   row(2L, 4L, 2L, 3L, 6L));
+    }
+
+    @Test
+    public void testAggregateWithUdtFields() throws Throwable
+    {
+        String myType = createType("CREATE TYPE %s (x int)");
+        createTable("CREATE TABLE %s (a int primary key, b frozen<" + myType + ">, c frozen<" + myType + ">)");
+
+        // Test with empty table
+        assertColumnNames(execute("SELECT count(b.x), max(b.x) as max, b.x, c.x as first FROM %s"),
+                          "system.count(b.x)", "max", "b.x", "first");
+        assertRows(execute("SELECT count(b.x), max(b.x) as max, b.x, c.x as first FROM %s"),
+                           row(0L, null, null, null));
+
+        execute("INSERT INTO %s (a, b, c) VALUES (1, {x:2}, null)");
+        execute("INSERT INTO %s (a, b, c) VALUES (2, {x:4}, {x:6})");
+        execute("INSERT INTO %s (a, b, c) VALUES (4, {x:8}, {x:12})");
+
+        assertRows(execute("SELECT count(b.x), max(b.x) as max, b.x, c.x as first FROM %s"),
+                   row(3L, 8, 2, null));
+
+        assertInvalidMessage("Invalid field selection: max(b) of type blob is not a user type",
+                             "SELECT max(b).x as max FROM %s");
+    }
+
+    @Test
+    public void testAggregateWithFunctions() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b double, c double, primary key(a, b))");
+
+        String copySign = createFunction(KEYSPACE,
+                                         "double, double",
+                                         "CREATE OR REPLACE FUNCTION %s(magnitude double, sign double) " +
+                                         "RETURNS NULL ON NULL INPUT " +
+                                         "RETURNS double " +
+                                         "LANGUAGE JAVA " +
+                                         "AS 'return Double.valueOf(Math.copySign(magnitude, sign));';");
+
+        // Test with empty table
+        assertColumnNames(execute("SELECT count(b), max(b) as max, " + copySign + "(b, c), " + copySign + "(c, b) as first FROM %s"),
+                          "system.count(b)", "max", copySign + "(b, c)", "first");
+        assertRows(execute("SELECT count(b), max(b) as max, " + copySign + "(b, c), " + copySign + "(c, b) as first FROM %s"),
+                           row(0L, null, null, null));
+
+        execute("INSERT INTO %s (a, b, c) VALUES (0, -1.2, 2.1)");
+        execute("INSERT INTO %s (a, b, c) VALUES (0, 1.3, -3.4)");
+        execute("INSERT INTO %s (a, b, c) VALUES (0, 1.4, 1.2)");
+
+        assertRows(execute("SELECT count(b), max(b) as max, " + copySign + "(b, c), " + copySign + "(c, b) as first FROM %s"),
+                   row(3L, 1.4, 1.2, -2.1));
+
+        execute("INSERT INTO %s (a, b, c) VALUES (1, -1.2, null)");
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 1.3, -3.4)");
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 1.4, 1.2)");
+        assertRows(execute("SELECT count(b), max(b) as max, " + copySign + "(b, c), " + copySign + "(c, b) as first FROM %s WHERE a = 1"),
+                   row(3L, 1.4, null, null));
+    }
+
+    @Test
+    public void testAggregateWithWriteTimeOrTTL() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int primary key, b int, c int)");
+
+        // Test with empty table
+        assertColumnNames(execute("SELECT count(writetime(b)), min(ttl(b)) as min, writetime(b), ttl(c) as first FROM %s"),
+                          "system.count(writetime(b))", "min", "writetime(b)", "first");
+        assertRows(execute("SELECT count(writetime(b)), min(ttl(b)) as min, writetime(b), ttl(c) as first FROM %s"),
+                           row(0L, null, null, null));
+
+        long today = System.currentTimeMillis() * 1000;
+        long yesterday = today - (DateUtils.MILLIS_PER_DAY * 1000);
+
+        final int secondsPerMinute = 60;
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 2, null) USING TTL " + (20 * secondsPerMinute));
+        execute("INSERT INTO %s (a, b, c) VALUES (2, 4, 6) USING TTL " + (10 * secondsPerMinute));
+        execute("INSERT INTO %s (a, b, c) VALUES (4, 8, 12) USING TIMESTAMP " + yesterday );
+
+        assertRows(execute("SELECT count(writetime(b)), count(ttl(b)) FROM %s"),
+                   row(3L, 2L));
+
+        UntypedResultSet resultSet = execute("SELECT min(ttl(b)), ttl(b) FROM %s");
+        assertEquals(1, resultSet.size());
+        Row row = resultSet.one();
+        assertTrue(row.getInt("ttl(b)") > (10 * secondsPerMinute));
+        assertTrue(row.getInt("system.min(ttl(b))") <= (10 * secondsPerMinute));
+
+        resultSet = execute("SELECT min(writetime(b)), writetime(b) FROM %s");
+        assertEquals(1, resultSet.size());
+        row = resultSet.one();
+
+        assertTrue(row.getLong("writetime(b)") >= today);
+        assertTrue(row.getLong("system.min(writetime(b))") == yesterday);
+    }
+
+    @Test
+    public void testFunctionsWithCompactStorage() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int , b int, c double, primary key(a, b) ) WITH COMPACT STORAGE");
+
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 1, 11.5)");
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 2, 9.5)");
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 3, 9.0)");
+
+        assertRows(execute("SELECT max(b), min(b), sum(b), avg(b) , max(c), sum(c), avg(c) FROM %s"),
+                   row(3, 1, 6, 2, 11.5, 30.0, 10.0));
+
+        assertRows(execute("SELECT COUNT(*) FROM %s"), row(3L));
+        assertRows(execute("SELECT COUNT(1) FROM %s"), row(3L));
+        assertRows(execute("SELECT COUNT(*) FROM %s WHERE a = 1 AND b > 1"), row(2L));
+        assertRows(execute("SELECT COUNT(1) FROM %s WHERE a = 1 AND b > 1"), row(2L));
+        assertRows(execute("SELECT max(b), min(b), sum(b), avg(b) , max(c), sum(c), avg(c) FROM %s WHERE a = 1 AND b > 1"),
+                   row(3, 2, 5, 2, 9.5, 18.5, 9.25));
+    }
+
+    @Test
+    public void testInvalidCalls() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, primary key (a, b))");
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 1, 10)");
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 2, 9)");
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 3, 8)");
+
+        assertInvalidSyntax("SELECT max(b), max(c) FROM %s WHERE max(a) = 1");
+        assertInvalidMessage("aggregate functions cannot be used as arguments of aggregate functions", "SELECT max(sum(c)) FROM %s");
+        assertInvalidSyntax("SELECT COUNT(2) FROM %s");
+    }
+
+    @Test
+    public void testReversedType() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, primary key (a, b)) WITH CLUSTERING ORDER BY (b DESC)");
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 1, 10)");
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 2, 9)");
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 3, 8)");
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 4, 7)");
+
+        assertRows(execute("SELECT max(c), min(c), avg(c) FROM %s WHERE a = 1 AND b > 1"), row(9, 7, 8));
+    }
+
+    @Test
+    public void testNestedFunctions() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int primary key, b timeuuid, c double, d double)");
+
+        String copySign = createFunction(KEYSPACE,
+                                         "double, double",
+                                         "CREATE OR REPLACE FUNCTION %s(magnitude double, sign double) " +
+                                         "RETURNS NULL ON NULL INPUT " +
+                                         "RETURNS double " +
+                                         "LANGUAGE JAVA " +
+                                         "AS 'return Double.valueOf(Math.copySign(magnitude, sign));';");
+
+        assertColumnNames(execute("SELECT max(a), max(toUnixTimestamp(b)) FROM %s"), "system.max(a)", "system.max(system.tounixtimestamp(b))");
+        assertRows(execute("SELECT max(a), max(toUnixTimestamp(b)) FROM %s"), row(null, null));
+        assertColumnNames(execute("SELECT max(a), toUnixTimestamp(max(b)) FROM %s"), "system.max(a)", "system.tounixtimestamp(system.max(b))");
+        assertRows(execute("SELECT max(a), toUnixTimestamp(max(b)) FROM %s"), row(null, null));
+
+        assertColumnNames(execute("SELECT max(" + copySign + "(c, d)) FROM %s"), "system.max(" + copySign + "(c, d))");
+        assertRows(execute("SELECT max(" + copySign + "(c, d)) FROM %s"), row((Object) null));
+
+        execute("INSERT INTO %s (a, b, c, d) VALUES (1, maxTimeuuid('2011-02-03 04:05:00+0000'), -1.2, 2.1)");
+        execute("INSERT INTO %s (a, b, c, d) VALUES (2, maxTimeuuid('2011-02-03 04:06:00+0000'), 1.3, -3.4)");
+        execute("INSERT INTO %s (a, b, c, d) VALUES (3, maxTimeuuid('2011-02-03 04:10:00+0000'), 1.4, 1.2)");
+
+        SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd hh:mm:ss");
+        format.setTimeZone(TimeZone.getTimeZone("GMT"));
+        Date date = format.parse("2011-02-03 04:10:00");
+        date = DateUtils.truncate(date, Calendar.MILLISECOND);
+
+        assertRows(execute("SELECT max(a), max(toUnixTimestamp(b)) FROM %s"), row(3, date.getTime()));
+        assertRows(execute("SELECT max(a), toUnixTimestamp(max(b)) FROM %s"), row(3, date.getTime()));
+
+        assertRows(execute("SELECT " + copySign + "(max(c), min(c)) FROM %s"), row(-1.4));
+        assertRows(execute("SELECT " + copySign + "(c, d) FROM %s"), row(1.2), row(-1.3), row(1.4));
+        assertRows(execute("SELECT max(" + copySign + "(c, d)) FROM %s"), row(1.4));
+        assertRows(execute("SELECT " + copySign + "(c, max(c)) FROM %s"), row(1.2));
+        assertRows(execute("SELECT " + copySign + "(max(c), c) FROM %s"), row(-1.4));
+    }
+
+    @Test
+    public void testSchemaChange() throws Throwable
+    {
+        String f = createFunction(KEYSPACE,
+                                  "double, double",
+                                  "CREATE OR REPLACE FUNCTION %s(state double, val double) " +
+                                  "RETURNS NULL ON NULL INPUT " +
+                                  "RETURNS double " +
+                                  "LANGUAGE javascript " +
+                                  "AS '\"string\";';");
+
+        createFunctionOverload(f,
+                               "double, double",
+                               "CREATE OR REPLACE FUNCTION %s(state int, val int) " +
+                               "RETURNS NULL ON NULL INPUT " +
+                               "RETURNS int " +
+                               "LANGUAGE javascript " +
+                               "AS '\"string\";';");
+
+        String a = createAggregate(KEYSPACE,
+                                   "double",
+                                   "CREATE OR REPLACE AGGREGATE %s(double) " +
+                                   "SFUNC " + shortFunctionName(f) + " " +
+                                   "STYPE double " +
+                                   "INITCOND 0");
+
+        assertLastSchemaChange(Event.SchemaChange.Change.CREATED, Event.SchemaChange.Target.AGGREGATE,
+                               KEYSPACE, parseFunctionName(a).name,
+                               "double");
+
+        schemaChange("CREATE OR REPLACE AGGREGATE " + a + "(double) " +
+                     "SFUNC " + shortFunctionName(f) + " " +
+                     "STYPE double " +
+                     "INITCOND 0");
+
+        assertLastSchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.AGGREGATE,
+                               KEYSPACE, parseFunctionName(a).name,
+                               "double");
+
+        createAggregateOverload(a,
+                                "int",
+                                "CREATE OR REPLACE AGGREGATE %s(int) " +
+                                "SFUNC " + shortFunctionName(f) + " " +
+                                "STYPE int " +
+                                "INITCOND 0");
+
+        assertLastSchemaChange(Event.SchemaChange.Change.CREATED, Event.SchemaChange.Target.AGGREGATE,
+                               KEYSPACE, parseFunctionName(a).name,
+                               "int");
+
+        schemaChange("DROP AGGREGATE " + a + "(double)");
+
+        assertLastSchemaChange(Event.SchemaChange.Change.DROPPED, Event.SchemaChange.Target.AGGREGATE,
+                               KEYSPACE, parseFunctionName(a).name,
+                               "double");
+    }
+
+    @Test
+    public void testDropStatements() throws Throwable
+    {
+        String f = createFunction(KEYSPACE,
+                                  "double, double",
+                                  "CREATE OR REPLACE FUNCTION %s(state double, val double) " +
+                                  "RETURNS NULL ON NULL INPUT " +
+                                  "RETURNS double " +
+                                  "LANGUAGE javascript " +
+                                  "AS '\"string\";';");
+
+        createFunctionOverload(f,
+                               "double, double",
+                               "CREATE OR REPLACE FUNCTION %s(state int, val int) " +
+                               "RETURNS NULL ON NULL INPUT " +
+                               "RETURNS int " +
+                               "LANGUAGE javascript " +
+                               "AS '\"string\";';");
+
+        // DROP AGGREGATE must not succeed against a scalar
+        assertInvalidMessage("matches multiple function definitions", "DROP AGGREGATE " + f);
+        assertInvalidMessage("non existing", "DROP AGGREGATE " + f + "(double, double)");
+
+        String a = createAggregate(KEYSPACE,
+                                   "double",
+                                   "CREATE OR REPLACE AGGREGATE %s(double) " +
+                                   "SFUNC " + shortFunctionName(f) + " " +
+                                   "STYPE double " +
+                                   "INITCOND 0");
+        createAggregateOverload(a,
+                                "int",
+                                "CREATE OR REPLACE AGGREGATE %s(int) " +
+                                "SFUNC " + shortFunctionName(f) + " " +
+                                "STYPE int " +
+                                "INITCOND 0");
+
+        // DROP FUNCTION must not succeed against an aggregate
+        assertInvalidMessage("matches multiple function definitions", "DROP FUNCTION " + a);
+        assertInvalidMessage("non existing function", "DROP FUNCTION " + a + "(double)");
+
+        // ambigious
+        assertInvalidMessage("matches multiple function definitions", "DROP AGGREGATE " + a);
+        assertInvalidMessage("matches multiple function definitions", "DROP AGGREGATE IF EXISTS " + a);
+
+        execute("DROP AGGREGATE IF EXISTS " + KEYSPACE + ".non_existing");
+        execute("DROP AGGREGATE IF EXISTS " + a + "(int, text)");
+
+        execute("DROP AGGREGATE " + a + "(double)");
+
+        execute("DROP AGGREGATE IF EXISTS " + a + "(double)");
+    }
+
+    @Test
+    public void testDropReferenced() throws Throwable
+    {
+        String f = createFunction(KEYSPACE,
+                                  "double, double",
+                                  "CREATE OR REPLACE FUNCTION %s(state double, val double) " +
+                                  "RETURNS NULL ON NULL INPUT " +
+                                  "RETURNS double " +
+                                  "LANGUAGE javascript " +
+                                  "AS '\"string\";';");
+
+        String a = createAggregate(KEYSPACE,
+                                   "double",
+                                   "CREATE OR REPLACE AGGREGATE %s(double) " +
+                                   "SFUNC " + shortFunctionName(f) + " " +
+                                   "STYPE double " +
+                                   "INITCOND 0");
+
+        // DROP FUNCTION must not succeed because the function is still referenced by the aggregate
+        assertInvalidMessage("still referenced by", "DROP FUNCTION " + f);
+
+        execute("DROP AGGREGATE " + a + "(double)");
+    }
+
+    @Test
+    public void testJavaAggregateNoInit() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int primary key, b int)");
+        execute("INSERT INTO %s (a, b) VALUES (1, 1)");
+        execute("INSERT INTO %s (a, b) VALUES (2, 2)");
+        execute("INSERT INTO %s (a, b) VALUES (3, 3)");
+
+        String fState = createFunction(KEYSPACE,
+                                       "int, int",
+                                       "CREATE FUNCTION %s(a int, b int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS int " +
+                                       "LANGUAGE java " +
+                                       "AS 'return Integer.valueOf((a!=null?a.intValue():0) + b.intValue());'");
+
+        String fFinal = createFunction(KEYSPACE,
+                                       "int",
+                                       "CREATE FUNCTION %s(a int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS text " +
+                                       "LANGUAGE java " +
+                                       "AS 'return a.toString();'");
+
+        String a = createAggregate(KEYSPACE,
+                                   "int",
+                                   "CREATE AGGREGATE %s(int) " +
+                                   "SFUNC " + shortFunctionName(fState) + " " +
+                                   "STYPE int " +
+                                   "FINALFUNC " + shortFunctionName(fFinal));
+
+        // 1 + 2 + 3 = 6
+        assertRows(execute("SELECT " + a + "(b) FROM %s"), row("6"));
+
+        execute("DROP AGGREGATE " + a + "(int)");
+
+        assertInvalidMessage("Unknown function", "SELECT " + a + "(b) FROM %s");
+    }
+
+    @Test
+    public void testJavaAggregateNullInitcond() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int primary key, b int)");
+        execute("INSERT INTO %s (a, b) VALUES (1, 1)");
+        execute("INSERT INTO %s (a, b) VALUES (2, 2)");
+        execute("INSERT INTO %s (a, b) VALUES (3, 3)");
+
+        String fState = createFunction(KEYSPACE,
+                                       "int, int",
+                                       "CREATE FUNCTION %s(a int, b int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS int " +
+                                       "LANGUAGE java " +
+                                       "AS 'return Integer.valueOf((a!=null?a.intValue():0) + b.intValue());'");
+
+        String fFinal = createFunction(KEYSPACE,
+                                       "int",
+                                       "CREATE FUNCTION %s(a int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS text " +
+                                       "LANGUAGE java " +
+                                       "AS 'return a.toString();'");
+
+        String a = createAggregate(KEYSPACE,
+                                   "int",
+                                   "CREATE AGGREGATE %s(int) " +
+                                   "SFUNC " + shortFunctionName(fState) + " " +
+                                   "STYPE int " +
+                                   "FINALFUNC " + shortFunctionName(fFinal) + " " +
+                                   "INITCOND null");
+
+        // 1 + 2 + 3 = 6
+        assertRows(execute("SELECT " + a + "(b) FROM %s"), row("6"));
+
+        execute("DROP AGGREGATE " + a + "(int)");
+
+        assertInvalidMessage("Unknown function", "SELECT " + a + "(b) FROM %s");
+    }
+
+    @Test
+    public void testJavaAggregateInvalidInitcond() throws Throwable
+    {
+        String fState = createFunction(KEYSPACE,
+                                       "int, int",
+                                       "CREATE FUNCTION %s(a int, b int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS int " +
+                                       "LANGUAGE java " +
+                                       "AS 'return Integer.valueOf((a!=null?a.intValue():0) + b.intValue());'");
+
+        String fFinal = createFunction(KEYSPACE,
+                                       "int",
+                                       "CREATE FUNCTION %s(a int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS text " +
+                                       "LANGUAGE java " +
+                                       "AS 'return a.toString();'");
+
+        assertInvalidMessage("Invalid STRING constant (foobar)",
+                             "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(int)" +
+                             "SFUNC " + shortFunctionName(fState) + " " +
+                             "STYPE int " +
+                             "FINALFUNC " + shortFunctionName(fFinal) + " " +
+                             "INITCOND 'foobar'");
+    }
+
+    @Test
+    public void testJavaAggregateIncompatibleTypes() throws Throwable
+    {
+        String fState = createFunction(KEYSPACE,
+                                       "int, int",
+                                       "CREATE FUNCTION %s(a int, b int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS int " +
+                                       "LANGUAGE java " +
+                                       "AS 'return Integer.valueOf((a!=null?a.intValue():0) + b.intValue());'");
+
+        String fFinal = createFunction(KEYSPACE,
+                                       "int",
+                                       "CREATE FUNCTION %s(a int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS text " +
+                                       "LANGUAGE java " +
+                                       "AS 'return a.toString();'");
+
+        String fState2 = createFunction(KEYSPACE,
+                                        "int, int",
+                                        "CREATE FUNCTION %s(a double, b double) " +
+                                        "CALLED ON NULL INPUT " +
+                                        "RETURNS double " +
+                                        "LANGUAGE java " +
+                                        "AS 'return Double.valueOf((a!=null?a.doubleValue():0d) + b.doubleValue());'");
+
+        String fFinal2 = createFunction(KEYSPACE,
+                                        "int",
+                                        "CREATE FUNCTION %s(a double) " +
+                                        "CALLED ON NULL INPUT " +
+                                        "RETURNS text " +
+                                        "LANGUAGE java " +
+                                        "AS 'return a.toString();'");
+
+        assertInvalidMessage("does not exist or is not a scalar function",
+                             "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(double)" +
+                             "SFUNC " + shortFunctionName(fState) + " " +
+                             "STYPE double " +
+                             "FINALFUNC " + shortFunctionName(fFinal));
+        assertInvalidMessage("does not exist or is not a scalar function",
+                             "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(int)" +
+                             "SFUNC " + shortFunctionName(fState) + " " +
+                             "STYPE double " +
+                             "FINALFUNC " + shortFunctionName(fFinal));
+        assertInvalidMessage("does not exist or is not a scalar function",
+                             "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(double)" +
+                             "SFUNC " + shortFunctionName(fState) + " " +
+                             "STYPE int " +
+                             "FINALFUNC " + shortFunctionName(fFinal));
+        assertInvalidMessage("does not exist or is not a scalar function",
+                             "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(double)" +
+                             "SFUNC " + shortFunctionName(fState) + " " +
+                             "STYPE int");
+        assertInvalidMessage("does not exist or is not a scalar function",
+                             "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(int)" +
+                             "SFUNC " + shortFunctionName(fState) + " " +
+                             "STYPE double");
+
+        assertInvalidMessage("does not exist or is not a scalar function",
+                             "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(double)" +
+                             "SFUNC " + shortFunctionName(fState2) + " " +
+                             "STYPE double " +
+                             "FINALFUNC " + shortFunctionName(fFinal));
+
+        assertInvalidMessage("does not exist or is not a scalar function",
+                             "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(double)" +
+                             "SFUNC " + shortFunctionName(fState) + " " +
+                             "STYPE double " +
+                             "FINALFUNC " + shortFunctionName(fFinal2));
+    }
+
+    @Test
+    public void testJavaAggregateNonExistingFuncs() throws Throwable
+    {
+        String fState = createFunction(KEYSPACE,
+                                       "int, int",
+                                       "CREATE FUNCTION %s(a int, b int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS int " +
+                                       "LANGUAGE java " +
+                                       "AS 'return Integer.valueOf((a!=null?a.intValue():0) + b.intValue());'");
+
+        String fFinal = createFunction(KEYSPACE,
+                                       "int",
+                                       "CREATE FUNCTION %s(a int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS text " +
+                                       "LANGUAGE java " +
+                                       "AS 'return a.toString();'");
+
+        assertInvalidMessage("does not exist or is not a scalar function",
+                             "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(int)" +
+                             "SFUNC " + shortFunctionName(fState) + "_not_there " +
+                             "STYPE int " +
+                             "FINALFUNC " + shortFunctionName(fFinal));
+
+        assertInvalidMessage("does not exist or is not a scalar function",
+                             "CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(int)" +
+                             "SFUNC " + shortFunctionName(fState) + " " +
+                             "STYPE int " +
+                             "FINALFUNC " + shortFunctionName(fFinal) + "_not_there");
+
+        execute("CREATE AGGREGATE " + KEYSPACE + ".aggrInvalid(int)" +
+                "SFUNC " + shortFunctionName(fState) + " " +
+                "STYPE int " +
+                "FINALFUNC " + shortFunctionName(fFinal));
+        execute("DROP AGGREGATE " + KEYSPACE + ".aggrInvalid(int)");
+    }
+
+    @Test
+    public void testJavaAggregateFailingFuncs() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int primary key, b int)");
+        execute("INSERT INTO %s (a, b) VALUES (1, 1)");
+        execute("INSERT INTO %s (a, b) VALUES (2, 2)");
+        execute("INSERT INTO %s (a, b) VALUES (3, 3)");
+
+        String fState = createFunction(KEYSPACE,
+                                       "int, int",
+                                       "CREATE FUNCTION %s(a int, b int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS int " +
+                                       "LANGUAGE java " +
+                                       "AS 'throw new RuntimeException(\"thrown to unit test - not a bug\");'");
+
+        String fStateOK = createFunction(KEYSPACE,
+                                       "int, int",
+                                       "CREATE FUNCTION %s(a int, b int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS int " +
+                                       "LANGUAGE java " +
+                                       "AS 'return Integer.valueOf(42);'");
+
+        String fFinal = createFunction(KEYSPACE,
+                                       "int",
+                                       "CREATE FUNCTION %s(a int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS text " +
+                                       "LANGUAGE java " +
+                                       "AS 'throw new RuntimeException(\"thrown to unit test - not a bug\");'");
+
+        String fFinalOK = createFunction(KEYSPACE,
+                                       "int",
+                                       "CREATE FUNCTION %s(a int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS text " +
+                                       "LANGUAGE java " +
+                                       "AS 'return \"foobar\";'");
+
+        String a0 = createAggregate(KEYSPACE,
+                                    "int",
+                                    "CREATE AGGREGATE %s(int) " +
+                                    "SFUNC " + shortFunctionName(fState) + " " +
+                                    "STYPE int " +
+                                    "FINALFUNC " + shortFunctionName(fFinal) + " " +
+                                    "INITCOND null");
+        String a1 = createAggregate(KEYSPACE,
+                                    "int",
+                                    "CREATE AGGREGATE %s(int) " +
+                                    "SFUNC " + shortFunctionName(fStateOK) + " " +
+                                    "STYPE int " +
+                                    "FINALFUNC " + shortFunctionName(fFinal) + " " +
+                                    "INITCOND null");
+        String a2 = createAggregate(KEYSPACE,
+                                    "int",
+                                    "CREATE AGGREGATE %s(int) " +
+                                    "SFUNC " + shortFunctionName(fStateOK) + " " +
+                                    "STYPE int " +
+                                    "FINALFUNC " + shortFunctionName(fFinalOK) + " " +
+                                    "INITCOND null");
+
+        assertInvalidThrowMessage("java.lang.RuntimeException", FunctionExecutionException.class, "SELECT " + a0 + "(b) FROM %s");
+        assertInvalidThrowMessage("java.lang.RuntimeException", FunctionExecutionException.class, "SELECT " + a1 + "(b) FROM %s");
+        assertRows(execute("SELECT " + a2 + "(b) FROM %s"), row("foobar"));
+    }
+
+    @Test
+    public void testJavaAggregateWithoutStateOrFinal() throws Throwable
+    {
+        assertInvalidMessage("does not exist or is not a scalar function",
+                             "CREATE AGGREGATE " + KEYSPACE + ".jSumFooNE1(int) " +
+                             "SFUNC jSumFooNEstate " +
+                             "STYPE int");
+
+        String f = createFunction(KEYSPACE,
+                                  "int, int",
+                                  "CREATE FUNCTION %s(a int, b int) " +
+                                  "RETURNS NULL ON NULL INPUT " +
+                                  "RETURNS int " +
+                                  "LANGUAGE java " +
+                                  "AS 'return Integer.valueOf(a + b);'");
+
+        assertInvalidMessage("does not exist or is not a scalar function",
+                             "CREATE AGGREGATE " + KEYSPACE + ".jSumFooNE2(int) " +
+                             "SFUNC " + shortFunctionName(f) + " " +
+                             "STYPE int " +
+                             "FINALFUNC jSumFooNEfinal");
+
+        execute("DROP FUNCTION " + f + "(int, int)");
+    }
+
+    @Test
+    public void testJavaAggregate() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int primary key, b int)");
+        execute("INSERT INTO %s (a, b) VALUES (1, 1)");
+        execute("INSERT INTO %s (a, b) VALUES (2, 2)");
+        execute("INSERT INTO %s (a, b) VALUES (3, 3)");
+
+        String fState = createFunction(KEYSPACE,
+                                       "int, int",
+                                       "CREATE FUNCTION %s(a int, b int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS int " +
+                                       "LANGUAGE java " +
+                                       "AS 'return Integer.valueOf((a!=null?a.intValue():0) + b.intValue());'");
+
+        String fFinal = createFunction(KEYSPACE,
+                                       "int",
+                                       "CREATE FUNCTION %s(a int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS text " +
+                                       "LANGUAGE java " +
+                                       "AS 'return a.toString();'");
+
+        String a = createAggregate(KEYSPACE,
+                                   "int",
+                                   "CREATE AGGREGATE %s(int) " +
+                                   "SFUNC " + shortFunctionName(fState) + " " +
+                                   "STYPE int " +
+                                   "FINALFUNC " + shortFunctionName(fFinal) + " " +
+                                   "INITCOND 42");
+
+        // 42 + 1 + 2 + 3 = 48
+        assertRows(execute("SELECT " + a + "(b) FROM %s"), row("48"));
+
+        execute("DROP AGGREGATE " + a + "(int)");
+
+        execute("DROP FUNCTION " + fFinal + "(int)");
+        execute("DROP FUNCTION " + fState + "(int, int)");
+
+        assertInvalidMessage("Unknown function", "SELECT " + a + "(b) FROM %s");
+    }
+
+    @Test
+    public void testJavaAggregateSimple() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int primary key, b int)");
+        execute("INSERT INTO %s (a, b) VALUES (1, 1)");
+        execute("INSERT INTO %s (a, b) VALUES (2, 2)");
+        execute("INSERT INTO %s (a, b) VALUES (3, 3)");
+
+        String fState = createFunction(KEYSPACE,
+                                       "int, int",
+                                       "CREATE FUNCTION %s(a int, b int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS int " +
+                                       "LANGUAGE java " +
+                                       "AS 'return Integer.valueOf((a!=null?a.intValue():0) + b.intValue());'");
+
+        String a = createAggregate(KEYSPACE,
+                                   "int, int",
+                                   "CREATE AGGREGATE %s(int) " +
+                                   "SFUNC " + shortFunctionName(fState) + " " +
+                                   "STYPE int");
+
+        // 1 + 2 + 3 = 6
+        assertRows(execute("SELECT " + a + "(b) FROM %s"), row(6));
+
+        execute("DROP AGGREGATE " + a + "(int)");
+
+        execute("DROP FUNCTION " + fState + "(int, int)");
+
+        assertInvalidMessage("Unknown function", "SELECT " + a + "(b) FROM %s");
+    }
+
+    @Test
+    public void testJavaAggregateComplex() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int primary key, b int)");
+        execute("INSERT INTO %s (a, b) VALUES (1, 1)");
+        execute("INSERT INTO %s (a, b) VALUES (2, 2)");
+        execute("INSERT INTO %s (a, b) VALUES (3, 3)");
+
+        // build an average aggregation function using
+        // tuple<bigint,int> as state
+        // double as finaltype
+
+        String fState = createFunction(KEYSPACE,
+                                       "tuple<bigint, int>, int",
+                                       "CREATE FUNCTION %s(a tuple<bigint, int>, b int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS tuple<bigint, int> " +
+                                       "LANGUAGE java " +
+                                       "AS '" +
+                                       "a.setLong(0, a.getLong(0) + b.intValue());" +
+                                       "a.setInt(1, a.getInt(1) + 1);" +
+                                       "return a;" +
+                                       "'");
+
+        String fFinal = createFunction(KEYSPACE,
+                                       "tuple<bigint, int>",
+                                       "CREATE FUNCTION %s(a tuple<bigint, int>) " +
+                                       "RETURNS NULL ON NULL INPUT " +
+                                       "RETURNS double " +
+                                       "LANGUAGE java " +
+                                       "AS '" +
+                                       "double r = a.getLong(0);" +
+                                       "r /= a.getInt(1);" +
+                                       "return Double.valueOf(r);" +
+                                       "'");
+
+        String a = createAggregate(KEYSPACE,
+                                   "int",
+                                   "CREATE AGGREGATE %s(int) " +
+                                   "SFUNC " + shortFunctionName(fState) + " " +
+                                   "STYPE tuple<bigint, int> "+
+                                   "FINALFUNC " + shortFunctionName(fFinal) + " " +
+                                   "INITCOND (0, 0)");
+
+        // 1 + 2 + 3 = 6 / 3 = 2
+        assertRows(execute("SELECT " + a + "(b) FROM %s"), row(2d));
+
+    }
+
+    @Test
+    public void testJavascriptAggregate() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int primary key, b int)");
+        execute("INSERT INTO %s (a, b) VALUES (1, 1)");
+        execute("INSERT INTO %s (a, b) VALUES (2, 2)");
+        execute("INSERT INTO %s (a, b) VALUES (3, 3)");
+
+        String fState = createFunction(KEYSPACE,
+                                       "int, int",
+                                       "CREATE FUNCTION %s(a int, b int) " +
+                                       "RETURNS NULL ON NULL INPUT " +
+                                       "RETURNS int " +
+                                       "LANGUAGE javascript " +
+                                       "AS 'a + b;'");
+
+        String fFinal = createFunction(KEYSPACE,
+                                       "int",
+                                       "CREATE FUNCTION %s(a int) " +
+                                       "RETURNS NULL ON NULL INPUT " +
+                                       "RETURNS text " +
+                                       "LANGUAGE javascript " +
+                                       "AS '\"\"+a'");
+
+        String a = createFunction(KEYSPACE,
+                                  "int",
+                                  "CREATE AGGREGATE %s(int) " +
+                                  "SFUNC " + shortFunctionName(fState) + " " +
+                                  "STYPE int " +
+                                  "FINALFUNC " + shortFunctionName(fFinal) + " " +
+                                  "INITCOND 42");
+
+        // 42 + 1 + 2 + 3 = 48
+        assertRows(execute("SELECT " + a + "(b) FROM %s"), row("48"));
+
+        execute("DROP AGGREGATE " + a + "(int)");
+
+        execute("DROP FUNCTION " + fFinal + "(int)");
+        execute("DROP FUNCTION " + fState + "(int, int)");
+
+        assertInvalidMessage("Unknown function", "SELECT " + a + "(b) FROM %s");
+    }
+
+    @Test
+    public void testJavascriptAggregateSimple() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int primary key, b int)");
+        execute("INSERT INTO %s (a, b) VALUES (1, 1)");
+        execute("INSERT INTO %s (a, b) VALUES (2, 2)");
+        execute("INSERT INTO %s (a, b) VALUES (3, 3)");
+
+        String fState = createFunction(KEYSPACE,
+                                       "int, int",
+                                       "CREATE FUNCTION %s(a int, b int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS int " +
+                                       "LANGUAGE javascript " +
+                                       "AS 'a + b;'");
+
+        String a = createAggregate(KEYSPACE,
+                                   "int, int",
+                                   "CREATE AGGREGATE %s(int) " +
+                                   "SFUNC " + shortFunctionName(fState) + " " +
+                                   "STYPE int ");
+
+        // 1 + 2 + 3 = 6
+        assertRows(execute("SELECT " + a + "(b) FROM %s"), row(6));
+
+        execute("DROP AGGREGATE " + a + "(int)");
+
+        execute("DROP FUNCTION " + fState + "(int, int)");
+
+        assertInvalidMessage("Unknown function", "SELECT " + a + "(b) FROM %s");
+    }
+
+    @Test
+    public void testFunctionDropPreparedStatement() throws Throwable
+    {
+        String otherKS = "cqltest_foo";
+
+        execute("CREATE KEYSPACE IF NOT EXISTS " + otherKS + " WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3};");
+        try
+        {
+            execute("CREATE TABLE " + otherKS + ".jsdp (a int primary key, b int)");
+
+            String fState = createFunction(otherKS,
+                                           "int, int",
+                                           "CREATE FUNCTION %s(a int, b int) " +
+                                           "CALLED ON NULL INPUT " +
+                                           "RETURNS int " +
+                                           "LANGUAGE javascript " +
+                                           "AS 'a + b;'");
+
+            String a = createAggregate(otherKS,
+                                       "int",
+                                       "CREATE AGGREGATE %s(int) " +
+                                       "SFUNC " + shortFunctionName(fState) + " " +
+                                       "STYPE int");
+
+            ResultMessage.Prepared prepared = QueryProcessor.prepare("SELECT " + a + "(b) FROM " + otherKS + ".jsdp", ClientState.forInternalCalls(), false);
+            assertNotNull(QueryProcessor.instance.getPrepared(prepared.statementId));
+
+            execute("DROP AGGREGATE " + a + "(int)");
+            assertNull(QueryProcessor.instance.getPrepared(prepared.statementId));
+
+            //
+
+            execute("CREATE AGGREGATE " + a + "(int) " +
+                    "SFUNC " + shortFunctionName(fState) + " " +
+                    "STYPE int");
+
+            prepared = QueryProcessor.prepare("SELECT " + a + "(b) FROM " + otherKS + ".jsdp", ClientState.forInternalCalls(), false);
+            assertNotNull(QueryProcessor.instance.getPrepared(prepared.statementId));
+
+            execute("DROP KEYSPACE " + otherKS + ";");
+
+            assertNull(QueryProcessor.instance.getPrepared(prepared.statementId));
+        }
+        finally
+        {
+            execute("DROP KEYSPACE IF EXISTS " + otherKS + ";");
+        }
+    }
+
+    @Test
+    public void testAggregatesReferencedInAggregates() throws Throwable
+    {
+
+        String fState = createFunction(KEYSPACE,
+                                       "int, int",
+                                       "CREATE FUNCTION %s(a int, b int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS int " +
+                                       "LANGUAGE javascript " +
+                                       "AS 'a + b;'");
+
+        String a = createAggregate(KEYSPACE,
+                                   "int",
+                                   "CREATE AGGREGATE %s(int) " +
+                                   "SFUNC " + shortFunctionName(fState) + " " +
+                                   "STYPE int ");
+
+        assertInvalidMessage("does not exist or is not a scalar function",
+                             "CREATE AGGREGATE " + KEYSPACE + ".aggInv(int) " +
+                             "SFUNC " + shortFunctionName(a) + " " +
+                             "STYPE int ");
+
+        assertInvalidMessage("does not exist or is not a scalar function",
+                             "CREATE AGGREGATE " + KEYSPACE + ".aggInv(int) " +
+                             "SFUNC " + shortFunctionName(fState) + " " +
+                             "STYPE int " +
+                             "FINALFUNC " + shortFunctionName(a));
+    }
+
+    @Test
+    public void testCalledOnNullInput() throws Throwable
+    {
+        String fStateNonNull = createFunction(KEYSPACE,
+                                              "int, int",
+                                              "CREATE OR REPLACE FUNCTION %s(state int, val int) " +
+                                              "RETURNS NULL ON NULL INPUT " +
+                                              "RETURNS int " +
+                                              "LANGUAGE java\n" +
+                                              "AS 'return Integer.valueOf(state + val);';");
+        String fStateNull = createFunction(KEYSPACE,
+                                           "int, int",
+                                           "CREATE OR REPLACE FUNCTION %s(state int, val int) " +
+                                           "CALLED ON NULL INPUT " +
+                                           "RETURNS int " +
+                                           "LANGUAGE java\n" +
+                                           "AS 'return Integer.valueOf(" +
+                                           "   (state != null ? state.intValue() : 0) " +
+                                           "   + (val != null ? val.intValue() : 0));';");
+        String fStateAlwaysNull = createFunction(KEYSPACE,
+                                           "int, int",
+                                           "CREATE OR REPLACE FUNCTION %s(state int, val int) " +
+                                           "CALLED ON NULL INPUT " +
+                                           "RETURNS int " +
+                                           "LANGUAGE java\n" +
+                                           "AS 'return null;';");
+        String fFinalNonNull = createFunction(KEYSPACE,
+                                              "int",
+                                              "CREATE OR REPLACE FUNCTION %s(state int) " +
+                                              "RETURNS NULL ON NULL INPUT " +
+                                              "RETURNS int " +
+                                              "LANGUAGE java\n" +
+                                              "AS 'return Integer.valueOf(state);';");
+        String fFinalNull = createFunction(KEYSPACE,
+                                           "int",
+                                           "CREATE OR REPLACE FUNCTION %s(state int) " +
+                                           "CALLED ON NULL INPUT " +
+                                           "RETURNS int " +
+                                           "LANGUAGE java\n" +
+                                           "AS 'return state;';");
+
+        assertInvalid("CREATE AGGREGATE " + KEYSPACE + ".invAggr(int) " +
+                      "SFUNC " + shortFunctionName(fStateNonNull) + " " +
+                      "STYPE int");
+        assertInvalid("CREATE AGGREGATE " + KEYSPACE + ".invAggr(int) " +
+                      "SFUNC " + shortFunctionName(fStateNonNull) + " " +
+                      "STYPE int " +
+                      "FINALFUNC " + shortFunctionName(fFinalNonNull));
+
+        String aStateNull = createAggregate(KEYSPACE,
+                                               "int",
+                                               "CREATE AGGREGATE %s(int) " +
+                                               "SFUNC " + shortFunctionName(fStateNull) + " " +
+                                               "STYPE int");
+        String aStateNullFinalNull = createAggregate(KEYSPACE,
+                                                        "int",
+                                                        "CREATE AGGREGATE %s(int) " +
+                                                        "SFUNC " + shortFunctionName(fStateNull) + " " +
+                                                        "STYPE int " +
+                                                        "FINALFUNC " + shortFunctionName(fFinalNull));
+        String aStateNullFinalNonNull = createAggregate(KEYSPACE,
+                                                        "int",
+                                                        "CREATE AGGREGATE %s(int) " +
+                                                        "SFUNC " + shortFunctionName(fStateNull) + " " +
+                                                        "STYPE int " +
+                                                        "FINALFUNC " + shortFunctionName(fFinalNonNull));
+        String aStateNonNull = createAggregate(KEYSPACE,
+                                               "int",
+                                               "CREATE AGGREGATE %s(int) " +
+                                               "SFUNC " + shortFunctionName(fStateNonNull) + " " +
+                                               "STYPE int " +
+                                               "INITCOND 0");
+        String aStateNonNullFinalNull = createAggregate(KEYSPACE,
+                                                        "int",
+                                                        "CREATE AGGREGATE %s(int) " +
+                                                        "SFUNC " + shortFunctionName(fStateNonNull) + " " +
+                                                        "STYPE int " +
+                                                        "FINALFUNC " + shortFunctionName(fFinalNull) + " " +
+                                                        "INITCOND 0");
+        String aStateNonNullFinalNonNull = createAggregate(KEYSPACE,
+                                                           "int",
+                                                           "CREATE AGGREGATE %s(int) " +
+                                                           "SFUNC " + shortFunctionName(fStateNonNull) + " " +
+                                                           "STYPE int " +
+                                                           "FINALFUNC " + shortFunctionName(fFinalNonNull) + " " +
+                                                           "INITCOND 0");
+        String aStateAlwaysNullFinalNull = createAggregate(KEYSPACE,
+                                                           "int",
+                                                           "CREATE AGGREGATE %s(int) " +
+                                                           "SFUNC " + shortFunctionName(fStateAlwaysNull) + " " +
+                                                           "STYPE int " +
+                                                           "FINALFUNC " + shortFunctionName(fFinalNull));
+        String aStateAlwaysNullFinalNonNull = createAggregate(KEYSPACE,
+                                                           "int",
+                                                           "CREATE AGGREGATE %s(int) " +
+                                                           "SFUNC " + shortFunctionName(fStateAlwaysNull) + " " +
+                                                           "STYPE int " +
+                                                           "FINALFUNC " + shortFunctionName(fFinalNonNull));
+
+        createTable("CREATE TABLE %s (key int PRIMARY KEY, i int)");
+
+        execute("INSERT INTO %s (key, i) VALUES (0, null)");
+        execute("INSERT INTO %s (key, i) VALUES (1, 1)");
+        execute("INSERT INTO %s (key, i) VALUES (2, 2)");
+        execute("INSERT INTO %s (key, i) VALUES (3, 3)");
+
+        assertRows(execute("SELECT " + aStateNull + "(i) FROM %s WHERE key = 0"), row(0));
+        assertRows(execute("SELECT " + aStateNullFinalNull + "(i) FROM %s WHERE key = 0"), row(0));
+        assertRows(execute("SELECT " + aStateNullFinalNonNull + "(i) FROM %s WHERE key = 0"), row(0));
+        assertRows(execute("SELECT " + aStateNonNull + "(i) FROM %s WHERE key = 0"), row(0));
+        assertRows(execute("SELECT " + aStateNonNullFinalNull + "(i) FROM %s WHERE key = 0"), row(0));
+        assertRows(execute("SELECT " + aStateNonNullFinalNonNull + "(i) FROM %s WHERE key = 0"), row(0));
+        assertRows(execute("SELECT " + aStateAlwaysNullFinalNull + "(i) FROM %s WHERE key = 0"), row(new Object[]{null}));
+        assertRows(execute("SELECT " + aStateAlwaysNullFinalNonNull + "(i) FROM %s WHERE key = 0"), row(new Object[]{null}));
+
+        assertRows(execute("SELECT " + aStateNull + "(i) FROM %s WHERE key = 1"), row(1));
+        assertRows(execute("SELECT " + aStateNullFinalNull + "(i) FROM %s WHERE key = 1"), row(1));
+        assertRows(execute("SELECT " + aStateNullFinalNonNull + "(i) FROM %s WHERE key = 1"), row(1));
+        assertRows(execute("SELECT " + aStateNonNull + "(i) FROM %s WHERE key = 1"), row(1));
+        assertRows(execute("SELECT " + aStateNonNullFinalNull + "(i) FROM %s WHERE key = 1"), row(1));
+        assertRows(execute("SELECT " + aStateNonNullFinalNonNull + "(i) FROM %s WHERE key = 1"), row(1));
+        assertRows(execute("SELECT " + aStateAlwaysNullFinalNull + "(i) FROM %s WHERE key = 1"), row(new Object[]{null}));
+        assertRows(execute("SELECT " + aStateAlwaysNullFinalNonNull + "(i) FROM %s WHERE key = 1"), row(new Object[]{null}));
+
+        assertRows(execute("SELECT " + aStateNull + "(i) FROM %s WHERE key IN (1, 2, 3)"), row(6));
+        assertRows(execute("SELECT " + aStateNullFinalNull + "(i) FROM %s WHERE key IN (1, 2, 3)"), row(6));
+        assertRows(execute("SELECT " + aStateNullFinalNonNull + "(i) FROM %s WHERE key IN (1, 2, 3)"), row(6));
+        assertRows(execute("SELECT " + aStateNonNull + "(i) FROM %s WHERE key IN (1, 2, 3)"), row(6));
+        assertRows(execute("SELECT " + aStateNonNullFinalNull + "(i) FROM %s WHERE key IN (1, 2, 3)"), row(6));
+        assertRows(execute("SELECT " + aStateNonNullFinalNonNull + "(i) FROM %s WHERE key IN (1, 2, 3)"), row(6));
+        assertRows(execute("SELECT " + aStateAlwaysNullFinalNull + "(i) FROM %s WHERE key IN (1, 2, 3)"), row(new Object[]{null}));
+        assertRows(execute("SELECT " + aStateAlwaysNullFinalNonNull + "(i) FROM %s WHERE key IN (1, 2, 3)"), row(new Object[]{null}));
+    }
+
+    @Test
+    public void testBrokenAggregate() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int primary key, val int)");
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 1, 1);
+
+        String fState = createFunction(KEYSPACE,
+                                       "int, int",
+                                       "CREATE FUNCTION %s(a int, b int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS int " +
+                                       "LANGUAGE javascript " +
+                                       "AS 'a + b;'");
+
+        String a = createAggregate(KEYSPACE,
+                                   "int",
+                                   "CREATE AGGREGATE %s(int) " +
+                                   "SFUNC " + shortFunctionName(fState) + " " +
+                                   "STYPE int ");
+
+        UDAggregate f = (UDAggregate) Functions.find(parseFunctionName(a)).get(0);
+
+        Functions.addOrReplaceFunction(UDAggregate.createBroken(f.name(), f.argTypes(), f.returnType(),
+                                                                null, new InvalidRequestException("foo bar is broken")));
+
+        assertInvalidThrowMessage("foo bar is broken", InvalidRequestException.class,
+                                  "SELECT " + a + "(val) FROM %s");
+    }
+
+    @Test
+    public void testWrongStateType() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int primary key, val int)");
+        execute("INSERT INTO %s (key, val) VALUES (?, ?)", 1, 1);
+
+        String fState = createFunction(KEYSPACE,
+                                       "int, int",
+                                       "CREATE FUNCTION %s(a int, b int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS double " +
+                                       "LANGUAGE java " +
+                                       "AS 'return Double.valueOf(1.0);'");
+
+        String fFinal = createFunction(KEYSPACE,
+                                       "int",
+                                       "CREATE FUNCTION %s(a int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS int " +
+                                       "LANGUAGE java " +
+                                       "AS 'return Integer.valueOf(1);';");
+
+        assertInvalidMessage("return type must be the same as the first argument type - check STYPE, argument and return types",
+                             "CREATE AGGREGATE %s(int) " +
+                             "SFUNC " + shortFunctionName(fState) + ' ' +
+                             "STYPE int " +
+                             "FINALFUNC " + shortFunctionName(fFinal) + ' ' +
+                             "INITCOND 1");
+    }
+
+    @Test
+    public void testWrongKeyspace() throws Throwable
+    {
+        String typeName = createType("CREATE TYPE %s (txt text, i int)");
+        String type = KEYSPACE + '.' + typeName;
+
+        String fState = createFunction(KEYSPACE_PER_TEST,
+                                       "int, int",
+                                       "CREATE FUNCTION %s(a int, b int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS double " +
+                                       "LANGUAGE java " +
+                                       "AS 'return Double.valueOf(1.0);'");
+
+        String fFinal = createFunction(KEYSPACE_PER_TEST,
+                                       "int",
+                                       "CREATE FUNCTION %s(a int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS int " +
+                                       "LANGUAGE java " +
+                                       "AS 'return Integer.valueOf(1);';");
+
+        String fStateWrong = createFunction(KEYSPACE,
+                                       "int, int",
+                                       "CREATE FUNCTION %s(a int, b int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS double " +
+                                       "LANGUAGE java " +
+                                       "AS 'return Double.valueOf(1.0);'");
+
+        String fFinalWrong = createFunction(KEYSPACE,
+                                       "int",
+                                       "CREATE FUNCTION %s(a int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS int " +
+                                       "LANGUAGE java " +
+                                       "AS 'return Integer.valueOf(1);';");
+
+        assertInvalidMessage(String.format("Statement on keyspace %s cannot refer to a user type in keyspace %s; user types can only be used in the keyspace they are defined in",
+                                           KEYSPACE_PER_TEST, KEYSPACE),
+                             "CREATE AGGREGATE " + KEYSPACE_PER_TEST + ".test_wrong_ks(int) " +
+                             "SFUNC " + shortFunctionName(fState) + ' ' +
+                             "STYPE " + type + " " +
+                             "FINALFUNC " + shortFunctionName(fFinal) + ' ' +
+                             "INITCOND 1");
+
+        assertInvalidMessage("mismatched input", // specifying a function using "keyspace.functionname" is a syntax error
+                             "CREATE AGGREGATE " + KEYSPACE_PER_TEST + ".test_wrong_ks(int) " +
+                             "SFUNC " + fStateWrong + ' ' +
+                             "STYPE " + type + " " +
+                             "FINALFUNC " + shortFunctionName(fFinal) + ' ' +
+                             "INITCOND 1");
+
+        assertInvalidMessage("missing EOF", // specifying a function using "keyspace.functionname" is a syntax error
+                             "CREATE AGGREGATE " + KEYSPACE_PER_TEST + ".test_wrong_ks(int) " +
+                             "SFUNC " + shortFunctionName(fState) + ' ' +
+                             "STYPE " + type + " " +
+                             "FINALFUNC " + fFinalWrong + ' ' +
+                             "INITCOND 1");
+
+        assertInvalidMessage("missing EOF", // specifying a function using "keyspace.functionname" is a syntax error
+                             "CREATE AGGREGATE " + KEYSPACE_PER_TEST + ".test_wrong_ks(int) " +
+                             "SFUNC " + shortFunctionName(fState) + ' ' +
+                             "STYPE " + type + ' ' +
+                             "FINALFUNC " + SystemKeyspace.NAME + ".min " +
+                             "INITCOND 1");
+    }
+
+    @Test
+    public void testFunctionWithFrozenSetType() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<set<int>>)");
+        createIndex("CREATE INDEX ON %s (FULL(b))");
+
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 0, set());
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 1, set(1, 2, 3));
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 2, set(4, 5, 6));
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 3, set(7, 8, 9));
+
+        String fState = createFunction(KEYSPACE,
+                                       "set<int>",
+                                       "CREATE FUNCTION %s (state set<int>, values set<int>) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS set<int> " +
+                                       "LANGUAGE java\n" +
+                                       "AS 'return values;';");
+
+        String fFinal = createFunction(KEYSPACE,
+                                       "set<int>",
+                                       "CREATE FUNCTION %s(state set<int>) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS set<int> " +
+                                       "LANGUAGE java " +
+                                       "AS 'return state;'");
+
+        assertInvalidMessage("The function state type should not be frozen",
+                             "CREATE AGGREGATE %s(set<int>) " +
+                             "SFUNC " + parseFunctionName(fState).name + ' ' +
+                             "STYPE frozen<set<int>> " +
+                             "FINALFUNC " + parseFunctionName(fFinal).name + ' ' +
+                             "INITCOND null");
+
+        String aggregation = createAggregate(KEYSPACE,
+                                             "set<int>",
+                                             "CREATE AGGREGATE %s(set<int>) " +
+                                             "SFUNC " + parseFunctionName(fState).name + ' ' +
+                                             "STYPE set<int> " +
+                                             "FINALFUNC " + parseFunctionName(fFinal).name + ' ' +
+                                             "INITCOND null");
+
+        assertRows(execute("SELECT " + aggregation + "(b) FROM %s"),
+                   row(set(7, 8, 9)));
+
+        assertInvalidMessage("The function arguments should not be frozen",
+                             "DROP AGGREGATE %s (frozen<set<int>>);");
+    }
+
+    @Test
+    public void testFunctionWithFrozenListType() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<list<int>>)");
+        createIndex("CREATE INDEX ON %s (FULL(b))");
+
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 0, list());
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 1, list(1, 2, 3));
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 2, list(4, 5, 6));
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 3, list(7, 8, 9));
+
+        String fState = createFunction(KEYSPACE,
+                                       "list<int>",
+                                       "CREATE FUNCTION %s (state list<int>, values list<int>) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS list<int> " +
+                                       "LANGUAGE java\n" +
+                                       "AS 'return values;';");
+
+        String fFinal = createFunction(KEYSPACE,
+                                       "list<int>",
+                                       "CREATE FUNCTION %s(state list<int>) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS list<int> " +
+                                       "LANGUAGE java " +
+                                       "AS 'return state;'");
+
+        assertInvalidMessage("The function state type should not be frozen",
+                             "CREATE AGGREGATE %s(list<int>) " +
+                             "SFUNC " + parseFunctionName(fState).name + ' ' +
+                             "STYPE frozen<list<int>> " +
+                             "FINALFUNC " + parseFunctionName(fFinal).name + " " +
+                             "INITCOND null");
+
+        String aggregation = createAggregate(KEYSPACE,
+                                             "list<int>",
+                                             "CREATE AGGREGATE %s(list<int>) " +
+                                             "SFUNC " + parseFunctionName(fState).name + ' ' +
+                                             "STYPE list<int> " +
+                                             "FINALFUNC " + parseFunctionName(fFinal).name + ' ' +
+                                             "INITCOND null");
+
+        assertRows(execute("SELECT " + aggregation + "(b) FROM %s"),
+                   row(list(7, 8, 9)));
+
+        assertInvalidMessage("The function arguments should not be frozen",
+                             "DROP AGGREGATE %s (frozen<list<int>>);");
+    }
+
+    @Test
+    public void testFunctionWithFrozenMapType() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<map<int, int>>)");
+        createIndex("CREATE INDEX ON %s (FULL(b))");
+
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 0, map());
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 1, map(1, 2, 3, 4));
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 2, map(4, 5, 6, 7));
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 3, map(7, 8, 9, 10));
+
+        String fState = createFunction(KEYSPACE,
+                                       "map<int, int>",
+                                       "CREATE FUNCTION %s (state map<int, int>, values map<int, int>) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS map<int, int> " +
+                                       "LANGUAGE java\n" +
+                                       "AS 'return values;';");
+
+        String fFinal = createFunction(KEYSPACE,
+                                       "map<int, int>",
+                                       "CREATE FUNCTION %s(state map<int, int>) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS map<int, int> " +
+                                       "LANGUAGE java " +
+                                       "AS 'return state;'");
+
+        assertInvalidMessage("The function state type should not be frozen",
+                             "CREATE AGGREGATE %s(map<int, int>) " +
+                             "SFUNC " + parseFunctionName(fState).name + ' ' +
+                             "STYPE frozen<map<int, int>> " +
+                             "FINALFUNC " + parseFunctionName(fFinal).name + ' ' +
+                             "INITCOND null");
+
+        String aggregation = createAggregate(KEYSPACE,
+                                             "map<int, int>",
+                                             "CREATE AGGREGATE %s(map<int, int>) " +
+                                             "SFUNC " + parseFunctionName(fState).name + ' ' +
+                                             "STYPE map<int, int> " +
+                                             "FINALFUNC " + parseFunctionName(fFinal).name + ' ' +
+                                             "INITCOND null");
+
+        assertRows(execute("SELECT " + aggregation + "(b) FROM %s"),
+                   row(map(7, 8, 9, 10)));
+
+        assertInvalidMessage("The function arguments should not be frozen",
+                             "DROP AGGREGATE %s (frozen<map<int, int>>);");
+    }
+
+    @Test
+    public void testFunctionWithFrozenTupleType() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<tuple<int, int>>)");
+        createIndex("CREATE INDEX ON %s (b)");
+
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 0, tuple());
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 1, tuple(1, 2));
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 2, tuple(4, 5));
+        execute("INSERT INTO %s (a, b) VALUES (?, ?)", 3, tuple(7, 8));
+
+        String fState = createFunction(KEYSPACE,
+                                       "tuple<int, int>",
+                                       "CREATE FUNCTION %s (state tuple<int, int>, values tuple<int, int>) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS tuple<int, int> " +
+                                       "LANGUAGE java\n" +
+                                       "AS 'return values;';");
+
+        String fFinal = createFunction(KEYSPACE,
+                                       "tuple<int, int>",
+                                       "CREATE FUNCTION %s(state tuple<int, int>) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS tuple<int, int> " +
+                                       "LANGUAGE java " +
+                                       "AS 'return state;'");
+
+        assertInvalidMessage("The function state type should not be frozen",
+                             "CREATE AGGREGATE %s(tuple<int, int>) " +
+                             "SFUNC " + parseFunctionName(fState).name + ' ' +
+                             "STYPE frozen<tuple<int, int>> " +
+                             "FINALFUNC " + parseFunctionName(fFinal).name + ' ' +
+                             "INITCOND null");
+
+        String aggregation = createAggregate(KEYSPACE,
+                                             "tuple<int, int>",
+                                             "CREATE AGGREGATE %s(tuple<int, int>) " +
+                                             "SFUNC " + parseFunctionName(fState).name + ' ' +
+                                             "STYPE tuple<int, int> " +
+                                             "FINALFUNC " + parseFunctionName(fFinal).name + ' ' +
+                                             "INITCOND null");
+
+        assertRows(execute("SELECT " + aggregation + "(b) FROM %s"),
+                   row(tuple(7, 8)));
+
+        assertInvalidMessage("The function arguments should not be frozen",
+                             "DROP AGGREGATE %s (frozen<tuple<int, int>>);");
+    }
+
+    @Test
+    public void testFunctionWithFrozenUDFType() throws Throwable
+    {
+        String myType = createType("CREATE TYPE %s (f int)");
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b frozen<" + myType + ">)");
+        createIndex("CREATE INDEX ON %s (b)");
+
+        execute("INSERT INTO %s (a, b) VALUES (?, {f : ?})", 0, 1);
+        execute("INSERT INTO %s (a, b) VALUES (?, {f : ?})", 1, 2);
+        execute("INSERT INTO %s (a, b) VALUES (?, {f : ?})", 2, 4);
+        execute("INSERT INTO %s (a, b) VALUES (?, {f : ?})", 3, 7);
+
+        String fState = createFunction(KEYSPACE,
+                                       myType + ',' + myType,
+                                       "CREATE FUNCTION %s (state " + myType + ", values " + myType + ") " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS " + myType + " " +
+                                       "LANGUAGE java\n" +
+                                       "AS 'return values;';");
+
+        String fFinal = createFunction(KEYSPACE,
+                                       myType,
+                                       "CREATE FUNCTION %s(state " + myType + ") " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS " + myType + " " +
+                                       "LANGUAGE java " +
+                                       "AS 'return state;'");
+
+        assertInvalidMessage("The function state type should not be frozen",
+                             "CREATE AGGREGATE %s(" + myType + ") " +
+                             "SFUNC " + parseFunctionName(fState).name + ' ' +
+                             "STYPE frozen<" + myType + "> " +
+                             "FINALFUNC " + parseFunctionName(fFinal).name + ' ' +
+                             "INITCOND null");
+
+        String aggregation = createAggregate(KEYSPACE,
+                                             myType,
+                                             "CREATE AGGREGATE %s(" + myType + ") " +
+                                             "SFUNC " + parseFunctionName(fState).name + ' ' +
+                                             "STYPE " + myType + ' ' +
+                                             "FINALFUNC " + parseFunctionName(fFinal).name + ' ' +
+                                             "INITCOND null");
+
+        assertRows(execute("SELECT " + aggregation + "(b).f FROM %s"),
+                   row(7));
+
+        assertInvalidMessage("The function arguments should not be frozen",
+                             "DROP AGGREGATE %s (frozen<" + myType + ">);");
+    }
+
+    @Test
+    public void testEmptyValues() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int primary key, b text)");
+        execute("INSERT INTO %s (a, b) VALUES (1, '')");
+        execute("INSERT INTO %s (a, b) VALUES (2, '')");
+        execute("INSERT INTO %s (a, b) VALUES (3, '')");
+
+        String fCON = createFunction(KEYSPACE,
+                                     "text, text",
+                                     "CREATE FUNCTION %s(a text, b text) " +
+                                     "CALLED ON NULL INPUT " +
+                                     "RETURNS text " +
+                                     "LANGUAGE java " +
+                                     "AS 'return a + \"x\" + b + \"y\";'");
+
+        String fCONf = createFunction(KEYSPACE,
+                                     "text",
+                                     "CREATE FUNCTION %s(a text) " +
+                                     "CALLED ON NULL INPUT " +
+                                     "RETURNS text " +
+                                     "LANGUAGE java " +
+                                     "AS 'return \"fin\" + a;'");
+
+        String aCON = createAggregate(KEYSPACE,
+                                      "text",
+                                      "CREATE AGGREGATE %s(text) " +
+                                      "SFUNC " + shortFunctionName(fCON) + ' ' +
+                                      "STYPE text " +
+                                      "FINALFUNC " + shortFunctionName(fCONf) + ' ' +
+                                      "INITCOND ''");
+
+        String fRNON = createFunction(KEYSPACE,
+                                      "text, text",
+                                      "CREATE FUNCTION %s(a text, b text) " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS text " +
+                                      "LANGUAGE java " +
+                                      "AS 'return a + \"x\" + b + \"y\";'");
+
+        String fRNONf = createFunction(KEYSPACE,
+                                      "text",
+                                      "CREATE FUNCTION %s(a text) " +
+                                      "RETURNS NULL ON NULL INPUT " +
+                                      "RETURNS text " +
+                                      "LANGUAGE java " +
+                                      "AS 'return \"fin\" + a;'");
+
+        String aRNON = createAggregate(KEYSPACE,
+                                      "text",
+                                      "CREATE AGGREGATE %s(text) " +
+                                      "SFUNC " + shortFunctionName(fRNON) + ' ' +
+                                      "STYPE text " +
+                                      "FINALFUNC " + shortFunctionName(fRNONf) + ' ' +
+                                      "INITCOND ''");
+
+        assertRows(execute("SELECT " + aCON + "(b) FROM %s"), row("finxyxyxy"));
+        assertRows(execute("SELECT " + aRNON + "(b) FROM %s"), row("finxyxyxy"));
+
+        createTable("CREATE TABLE %s (a int primary key, b text)");
+        execute("INSERT INTO %s (a, b) VALUES (1, null)");
+        execute("INSERT INTO %s (a, b) VALUES (2, null)");
+        execute("INSERT INTO %s (a, b) VALUES (3, null)");
+
+        assertRows(execute("SELECT " + aCON + "(b) FROM %s"), row("finxnullyxnullyxnully"));
+        assertRows(execute("SELECT " + aRNON + "(b) FROM %s"), row("fin"));
+
+    }
+
+    @Test
+    public void testOrReplaceOptionals() throws Throwable
+    {
+        String fState = createFunction(KEYSPACE,
+                                       "list<text>, int",
+                                       "CREATE FUNCTION %s(s list<text>, i int) " +
+                                       "CALLED ON NULL INPUT " +
+                                       "RETURNS list<text> " +
+                                       "LANGUAGE java " +
+                                       "AS 'if (i != null) s.add(String.valueOf(i)); return s;'");
+
+        String fFinal = shortFunctionName(createFunction(KEYSPACE,
+                                                         "list<text>",
+                                                         "CREATE FUNCTION %s(s list<text>) " +
+                                                         "CALLED ON NULL INPUT " +
+                                                         "RETURNS list<text> " +
+                                                         "LANGUAGE java " +
+                                                         "AS 'return s;'"));
+
+        String a = createAggregate(KEYSPACE,
+                                   "int",
+                                   "CREATE AGGREGATE %s(int) " +
+                                   "SFUNC " + shortFunctionName(fState) + ' ' +
+                                   "STYPE list<text> ");
+
+        checkOptionals(a, null, null);
+
+        String ddlPrefix = "CREATE OR REPLACE AGGREGATE " + a + "(int) " +
+                           "SFUNC " + shortFunctionName(fState) + ' ' +
+                           "STYPE list<text> ";
+
+        // Test replacing INITCOND
+        for (String condition : new String[]{"", "INITCOND null"})
+        {
+            execute(ddlPrefix + "INITCOND [  ] ");
+            checkOptionals(a, null, ByteBuffer.allocate(4));
+
+            execute(ddlPrefix + condition);
+            checkOptionals(a, null, null);
+        }
+
+        // Test replacing FINALFUNC
+        execute(ddlPrefix + "FINALFUNC " + shortFunctionName(fFinal) + " ");
+        checkOptionals(a, shortFunctionName(fFinal), null);
+
+        execute(ddlPrefix);
+        checkOptionals(a, null, null);
+    }
+
+    private void checkOptionals(String aggregateName, String finalFunc, ByteBuffer initCond) throws Throwable
+    {
+        assertRows(execute("SELECT final_func, initcond FROM system.schema_aggregates WHERE keyspace_name=? AND aggregate_name=?", KEYSPACE, shortFunctionName(aggregateName)),
+                   row(finalFunc, initCond));
+    }
+}
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
index a6aad87..2cc8a18 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
@@ -131,6 +131,7 @@
         assertRows(execute("SELECT keyspace_name, durable_writes FROM system.schema_keyspaces"),
                    row("ks1", true),
                    row(KEYSPACE, true),
+                   row(KEYSPACE_PER_TEST, true),
                    row("ks2", false));
 
         execute("ALTER KEYSPACE ks1 WITH replication = { 'class' : 'NetworkTopologyStrategy', 'dc1' : 1 } AND durable_writes=False");
@@ -139,6 +140,7 @@
         assertRows(execute("SELECT keyspace_name, durable_writes, strategy_class FROM system.schema_keyspaces"),
                    row("ks1", false, "org.apache.cassandra.locator.NetworkTopologyStrategy"),
                    row(KEYSPACE, true, "org.apache.cassandra.locator.SimpleStrategy"),
+                   row(KEYSPACE_PER_TEST, true, "org.apache.cassandra.locator.SimpleStrategy"),
                    row("ks2", true, "org.apache.cassandra.locator.SimpleStrategy"));
 
         execute("USE ks1");
@@ -200,6 +202,23 @@
         }
     }
 
+    /**
+     * tests CASSANDRA-10027
+     */
+    @Test
+    public void testAlterColumnTypeToDate() throws Throwable
+    {
+        createTable("CREATE TABLE %s (key int PRIMARY KEY, c1 int);");
+        execute("INSERT INTO %s (key, c1) VALUES (1,1);");
+        execute("ALTER TABLE %s ALTER c1 TYPE date;");
+        assertRows(execute("SELECT * FROM %s"), row(1, 1));
+
+        createTable("CREATE TABLE %s (key int PRIMARY KEY, c1 varint);");
+        execute("INSERT INTO %s (key, c1) VALUES (1,1);");
+        assertInvalidMessage("Cannot change c1 from type varint to type date: types are incompatible.",
+                             "ALTER TABLE %s ALTER c1 TYPE date;");
+    }
+
     @Test // tests CASSANDRA-8879
     public void testAlterClusteringColumnTypeInCompactTable() throws Throwable
     {
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/BatchTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/BatchTest.java
index cb49e57..07117fd 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/BatchTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/BatchTest.java
@@ -85,4 +85,28 @@
                    row(0, 2));
     }
 
+    @Test
+    public void testBatchWithUnset() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, s text, i int)");
+
+        // test batch and update
+        String qualifiedTable = keyspace() + "." + currentTable();
+        execute("BEGIN BATCH " +
+                "INSERT INTO %s (k, s, i) VALUES (100, 'batchtext', 7); " +
+                "INSERT INTO " + qualifiedTable + " (k, s, i) VALUES (111, 'batchtext', 7); " +
+                "UPDATE " + qualifiedTable + " SET s=?, i=? WHERE k = 100; " +
+                "UPDATE " + qualifiedTable + " SET s=?, i=? WHERE k=111; " +
+                "APPLY BATCH;", null, unset(), unset(), null);
+        assertRows(execute("SELECT k, s, i FROM %s where k in (100,111)"),
+                   row(100, null, 7),
+                   row(111, "batchtext", null)
+        );
+    }
+
+    @Test
+    public void testBatchEmpty() throws Throwable
+    {
+        execute("BEGIN BATCH APPLY BATCH;");
+    }
 }
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
index fdb2ebb..398b851 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
@@ -35,6 +35,7 @@
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.triggers.ITrigger;
+import org.apache.cassandra.utils.ByteBufferUtil;
 
 import static junit.framework.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -48,6 +49,41 @@
         assertFalse(currentTableMetadata().isThriftCompatible());
     }
 
+    @Test
+    public void testCreateTableWithSmallintColumns() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a text, b smallint, c smallint, primary key (a, b));");
+        execute("INSERT INTO %s (a, b, c) VALUES ('1', 1, 2)");
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", "2", Short.MAX_VALUE, Short.MIN_VALUE);
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row("2", Short.MAX_VALUE, Short.MIN_VALUE),
+                   row("1", (short) 1, (short) 2));
+
+        assertInvalidMessage("Expected 2 bytes for a smallint (4)",
+                             "INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", "3", 1, 2);
+        assertInvalidMessage("Expected 2 bytes for a smallint (0)",
+                             "INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", "3", (short) 1, ByteBufferUtil.EMPTY_BYTE_BUFFER);
+    }
+
+    @Test
+    public void testCreateTinyintColumns() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a text, b tinyint, c tinyint, primary key (a, b));");
+        execute("INSERT INTO %s (a, b, c) VALUES ('1', 1, 2)");
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", "2", Byte.MAX_VALUE, Byte.MIN_VALUE);
+
+        assertRows(execute("SELECT * FROM %s"),
+                   row("2", Byte.MAX_VALUE, Byte.MIN_VALUE),
+                   row("1", (byte) 1, (byte) 2));
+
+        assertInvalidMessage("Expected 1 byte for a tinyint (4)",
+                             "INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", "3", 1, 2);
+
+        assertInvalidMessage("Expected 1 byte for a tinyint (0)",
+                             "INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", "3", (byte) 1, ByteBufferUtil.EMPTY_BYTE_BUFFER);
+    }
+
     /**
      * Creation and basic operations on a static table,
      * migrated from cql_tests.py:TestCQL.static_cf_test()
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/DropRecreateAndRestoreTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/DropRecreateAndRestoreTest.java
index 385ed3d..4a3a51d 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/DropRecreateAndRestoreTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/DropRecreateAndRestoreTest.java
@@ -70,7 +70,7 @@
         {
             // Restore to point in time.
             CommitLog.instance.archiver.restorePointInTime = time;
-            CommitLog.instance.resetUnsafe();
+            CommitLog.instance.resetUnsafe(true);
             CommitLog.instance.recover();
         }
         finally
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/DropTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/DropTest.java
index b0c0809..2c68390 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/DropTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/DropTest.java
@@ -27,8 +27,8 @@
     @Test
     public void testNonExistingOnes() throws Throwable
     {
-        assertInvalidMessage("Cannot drop non existing column family", "DROP TABLE " + KEYSPACE + ".table_does_not_exist");
-        assertInvalidMessage("Cannot drop non existing column family", "DROP TABLE keyspace_does_not_exist.table_does_not_exist");
+        assertInvalidMessage("Cannot drop non existing table", "DROP TABLE " + KEYSPACE + ".table_does_not_exist");
+        assertInvalidMessage("Cannot drop non existing table", "DROP TABLE keyspace_does_not_exist.table_does_not_exist");
 
         execute("DROP TABLE IF EXISTS " + KEYSPACE + ".table_does_not_exist");
         execute("DROP TABLE IF EXISTS keyspace_does_not_exist.table_does_not_exist");
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/InsertTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/InsertTest.java
index 99ec908..1d532cb 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/InsertTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/InsertTest.java
@@ -26,6 +26,39 @@
 public class InsertTest extends CQLTester
 {
     @Test
+    public void testInsertWithUnset() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, s text, i int)");
+
+        // insert using nulls
+        execute("INSERT INTO %s (k, s, i) VALUES (10, ?, ?)", "text", 10);
+        execute("INSERT INTO %s (k, s, i) VALUES (10, ?, ?)", null, null);
+        assertRows(execute("SELECT s, i FROM %s WHERE k = 10"),
+                   row(null, null) // sending null deletes the data
+        );
+        // insert using UNSET
+        execute("INSERT INTO %s (k, s, i) VALUES (11, ?, ?)", "text", 10);
+        execute("INSERT INTO %s (k, s, i) VALUES (11, ?, ?)", unset(), unset());
+        assertRows(execute("SELECT s, i FROM %s WHERE k=11"),
+                   row("text", 10) // unset columns does not delete the existing data
+        );
+
+        assertInvalidMessage("Invalid unset value for column k", "UPDATE %s SET i = 0 WHERE k = ?", unset());
+        assertInvalidMessage("Invalid unset value for column k", "DELETE FROM %s WHERE k = ?", unset());
+        assertInvalidMessage("Invalid unset value for argument in call to function blobasint", "SELECT * FROM %s WHERE k = blobAsInt(?)", unset());
+    }
+
+    @Test
+    public void testInsertTtlWithUnset() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, i int)");
+        execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL ?", unset()); // treat as 'unlimited'
+        assertRows(execute("SELECT ttl(i) FROM %s"),
+                   row(new Object[]{ null })
+        );
+    }
+
+    @Test
     public void testOverlyLargeInsertPK() throws Throwable
     {
         createTable("CREATE TABLE %s (a text, b text, PRIMARY KEY ((a), b))");
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfConditionTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfConditionTest.java
index 900cc7d..6396727 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfConditionTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/InsertUpdateIfConditionTest.java
@@ -18,6 +18,9 @@
 
 package org.apache.cassandra.cql3.validation.operations;
 
+import java.nio.ByteBuffer;
+import java.util.List;
+
 import org.junit.Test;
 
 import org.apache.cassandra.cql3.CQLTester;
@@ -41,8 +44,8 @@
         {
             execute("INSERT INTO %s (tkn, consumed) VALUES (?, FALSE)", i);
 
-            assertRows(execute("UPDATE %s SET consumed = TRUE WHERE tkn = ? IF consumed = FALSE", i), row(true));
-            assertRows(execute("UPDATE %s SET consumed = TRUE WHERE tkn = ? IF consumed = FALSE", i), row(false, true));
+            assertRows(execute("UPDATE %s SET consumed = TRUE WHERE tkn = ? IF consumed = ?", i, false), row(true));
+            assertRows(execute("UPDATE %s SET consumed = TRUE WHERE tkn = ? IF consumed = ?", i, false), row(false, true));
         }
     }
 
@@ -54,8 +57,11 @@
     {
         createTable(" CREATE TABLE %s (k int PRIMARY KEY, v1 int, v2 text, v3 int)");
 
+        assertInvalidMessage("Invalid 'unset' value in condition",
+                             "UPDATE %s SET v1 = 3, v2 = 'bar' WHERE k = 0 IF v1 = ?", unset());
+
         // Shouldn't apply
-        assertRows(execute("UPDATE %s SET v1 = 3, v2 = 'bar' WHERE k = 0 IF v1 = 4"), row(false));
+        assertRows(execute("UPDATE %s SET v1 = 3, v2 = 'bar' WHERE k = 0 IF v1 = ?", 4), row(false));
         assertRows(execute("UPDATE %s SET v1 = 3, v2 = 'bar' WHERE k = 0 IF EXISTS"), row(false));
 
         // Should apply
@@ -66,50 +72,50 @@
         assertRows(execute("SELECT * FROM %s"), row(0, 2, "foo", null));
 
         // Shouldn't apply
-        assertRows(execute("UPDATE %s SET v1 = 3, v2 = 'bar' WHERE k = 0 IF v1 = 4"), row(false, 2));
+        assertRows(execute("UPDATE %s SET v1 = 3, v2 = 'bar' WHERE k = 0 IF v1 = ?", 4), row(false, 2));
         assertRows(execute("SELECT * FROM %s"), row(0, 2, "foo", null));
 
         // Should apply (note: we want v2 before v1 in the statement order to exercise #5786)
-        assertRows(execute("UPDATE %s SET v2 = 'bar', v1 = 3 WHERE k = 0 IF v1 = 2"), row(true));
+        assertRows(execute("UPDATE %s SET v2 = 'bar', v1 = 3 WHERE k = 0 IF v1 = ?", 2), row(true));
         assertRows(execute("UPDATE %s SET v2 = 'bar', v1 = 3 WHERE k = 0 IF EXISTS"), row(true));
         assertRows(execute("SELECT * FROM %s"), row(0, 3, "bar", null));
 
         // Shouldn't apply, only one condition is ok
-        assertRows(execute("UPDATE %s SET v1 = 5, v2 = 'foobar' WHERE k = 0 IF v1 = 3 AND v2 = 'foo'"), row(false, 3, "bar"));
+        assertRows(execute("UPDATE %s SET v1 = 5, v2 = 'foobar' WHERE k = 0 IF v1 = ? AND v2 = ?", 3, "foo"), row(false, 3, "bar"));
         assertRows(execute("SELECT * FROM %s"), row(0, 3, "bar", null));
 
         // Should apply
-        assertRows(execute("UPDATE %s SET v1 = 5, v2 = 'foobar' WHERE k = 0 IF v1 = 3 AND v2 = 'bar'"), row(true));
+        assertRows(execute("UPDATE %s SET v1 = 5, v2 = 'foobar' WHERE k = 0 IF v1 = ? AND v2 = ?", 3, "bar"), row(true));
         assertRows(execute("SELECT * FROM %s"), row(0, 5, "foobar", null));
 
         // Shouldn't apply
-        assertRows(execute("DELETE v2 FROM %s WHERE k = 0 IF v1 = 3"), row(false, 5));
+        assertRows(execute("DELETE v2 FROM %s WHERE k = 0 IF v1 = ?", 3), row(false, 5));
         assertRows(execute("SELECT * FROM %s"), row(0, 5, "foobar", null));
 
         // Shouldn't apply
-        assertRows(execute("DELETE v2 FROM %s WHERE k = 0 IF v1 = null"), row(false, 5));
+        assertRows(execute("DELETE v2 FROM %s WHERE k = 0 IF v1 = ?", (Integer) null), row(false, 5));
         assertRows(execute("SELECT * FROM %s"), row(0, 5, "foobar", null));
 
         // Should apply
-        assertRows(execute("DELETE v2 FROM %s WHERE k = 0 IF v1 = 5"), row(true));
+        assertRows(execute("DELETE v2 FROM %s WHERE k = 0 IF v1 = ?", 5), row(true));
         assertRows(execute("SELECT * FROM %s"), row(0, 5, null, null));
 
         // Shouln't apply
-        assertRows(execute("DELETE v1 FROM %s WHERE k = 0 IF v3 = 4"), row(false, null));
+        assertRows(execute("DELETE v1 FROM %s WHERE k = 0 IF v3 = ?", 4), row(false, null));
 
         // Should apply
-        assertRows(execute("DELETE v1 FROM %s WHERE k = 0 IF v3 = null"), row(true));
+        assertRows(execute("DELETE v1 FROM %s WHERE k = 0 IF v3 = ?", (Integer) null), row(true));
         assertRows(execute("SELECT * FROM %s"), row(0, null, null, null));
 
         // Should apply
-        assertRows(execute("DELETE FROM %s WHERE k = 0 IF v1 = null"), row(true));
+        assertRows(execute("DELETE FROM %s WHERE k = 0 IF v1 = ?", (Integer) null), row(true));
         assertEmpty(execute("SELECT * FROM %s"));
 
         // Shouldn't apply
         assertRows(execute("UPDATE %s SET v1 = 3, v2 = 'bar' WHERE k = 0 IF EXISTS"), row(false));
 
         // Should apply
-        assertRows(execute("DELETE FROM %s WHERE k = 0 IF v1 IN (null)"), row(true));
+        assertRows(execute("DELETE FROM %s WHERE k = 0 IF v1 IN (?)", (Integer) null), row(true));
     }
 
     /**
@@ -123,15 +129,27 @@
         // non-EQ conditions
         execute("INSERT INTO %s (k, v1, v2) VALUES (0, 2, 'foo')");
 
-        assertRows(execute("UPDATE %s SET v2 = 'bar' WHERE k = 0 IF v1 < 3"), row(true));
-        assertRows(execute("UPDATE %s SET v2 = 'bar' WHERE k = 0 IF v1 <= 3"), row(true));
-        assertRows(execute("UPDATE %s SET v2 = 'bar' WHERE k = 0 IF v1 > 1"), row(true));
-        assertRows(execute("UPDATE %s SET v2 = 'bar' WHERE k = 0 IF v1 >= 1"), row(true));
-        assertRows(execute("UPDATE %s SET v2 = 'bar' WHERE k = 0 IF v1 != 1"), row(true));
-        assertRows(execute("UPDATE %s SET v2 = 'bar' WHERE k = 0 IF v1 != 2"), row(false, 2));
-        assertRows(execute("UPDATE %s SET v2 = 'bar' WHERE k = 0 IF v1 IN (0, 1, 2)"), row(true));
-        assertRows(execute("UPDATE %s SET v2 = 'bar' WHERE k = 0 IF v1 IN (142, 276)"), row(false, 2));
+        assertRows(execute("UPDATE %s SET v2 = 'bar' WHERE k = 0 IF v1 < ?", 3), row(true));
+        assertRows(execute("UPDATE %s SET v2 = 'bar' WHERE k = 0 IF v1 <= ?", 3), row(true));
+        assertRows(execute("UPDATE %s SET v2 = 'bar' WHERE k = 0 IF v1 > ?", 1), row(true));
+        assertRows(execute("UPDATE %s SET v2 = 'bar' WHERE k = 0 IF v1 >= ?", 1), row(true));
+        assertRows(execute("UPDATE %s SET v2 = 'bar' WHERE k = 0 IF v1 != ?", 1), row(true));
+        assertRows(execute("UPDATE %s SET v2 = 'bar' WHERE k = 0 IF v1 != ?", 2), row(false, 2));
+        assertRows(execute("UPDATE %s SET v2 = 'bar' WHERE k = 0 IF v1 IN (?, ?, ?)", 0, 1, 2), row(true));
+        assertRows(execute("UPDATE %s SET v2 = 'bar' WHERE k = 0 IF v1 IN ?", list(142, 276)), row(false, 2));
         assertRows(execute("UPDATE %s SET v2 = 'bar' WHERE k = 0 IF v1 IN ()"), row(false, 2));
+        assertRows(execute("UPDATE %s SET v2 = 'bar' WHERE k = 0 IF v1 IN (?, ?)", unset(), 1), row(false, 2));
+
+        assertInvalidMessage("Invalid 'unset' value in condition",
+                             "UPDATE %s SET v1 = 3, v2 = 'bar' WHERE k = 0 IF v1 < ?", unset());
+        assertInvalidMessage("Invalid 'unset' value in condition",
+                             "UPDATE %s SET v1 = 3, v2 = 'bar' WHERE k = 0 IF v1 <= ?", unset());
+        assertInvalidMessage("Invalid 'unset' value in condition",
+                             "UPDATE %s SET v1 = 3, v2 = 'bar' WHERE k = 0 IF v1 > ?", unset());
+        assertInvalidMessage("Invalid 'unset' value in condition",
+                             "UPDATE %s SET v1 = 3, v2 = 'bar' WHERE k = 0 IF v1 >= ?", unset());
+        assertInvalidMessage("Invalid 'unset' value in condition",
+                             "UPDATE %s SET v1 = 3, v2 = 'bar' WHERE k = 0 IF v1 != ?", unset());
     }
 
     /**
@@ -175,20 +193,31 @@
         assertRows(execute("DELETE FROM %s WHERE k='k' AND i=0 IF EXISTS"), row(false));
 
         // CASSANDRA-6430
-        assertInvalid("DELETE FROM %s WHERE k = 'k' IF EXISTS");
-        assertInvalid("DELETE FROM %s WHERE k = 'k' IF v = 'foo'");
-        assertInvalid("DELETE FROM %s WHERE i = 0 IF EXISTS");
-        assertInvalid("DELETE FROM %s WHERE k = 0 AND i > 0 IF EXISTS");
-        assertInvalid("DELETE FROM %s WHERE k = 0 AND i > 0 IF v = 'foo'");
+        assertInvalidMessage("DELETE statements must restrict all PRIMARY KEY columns with equality relations in order to use IF conditions, but column 'i' is not restricted",
+                             "DELETE FROM %s WHERE k = 'k' IF EXISTS");
+        assertInvalidMessage("DELETE statements must restrict all PRIMARY KEY columns with equality relations in order to use IF conditions, but column 'i' is not restricted",
+                             "DELETE FROM %s WHERE k = 'k' IF v = ?", "foo");
+        assertInvalidMessage("DELETE statements must restrict all PRIMARY KEY columns with equality relations in order to use IF conditions, but column 'k' is not restricted",
+                             "DELETE FROM %s WHERE i = 0 IF EXISTS");
+        assertInvalidMessage("Invalid INTEGER constant (0) for \"k\" of type text",
+                             "DELETE FROM %s WHERE k = 0 AND i > 0 IF EXISTS");
+
+        assertInvalidMessage("Invalid operator > for PRIMARY KEY part i",
+                             "DELETE FROM %s WHERE k = 'k' AND i > 0 IF EXISTS");
+        assertInvalidMessage("Invalid operator > for PRIMARY KEY part i",
+                             "DELETE FROM %s WHERE k = 'k' AND i > 0 IF v = ?", "foo");
+
+        assertInvalidMessage("Invalid 'unset' value in condition",
+                             "DELETE FROM %s WHERE k = 'k' AND i = 0 IF v = ?", unset());
 
         createTable("CREATE TABLE %s(k int, s int static, i int, v text, PRIMARY KEY(k, i))");
         execute("INSERT INTO %s (k, s, i, v) VALUES ( 1, 1, 2, '1')");
-        assertRows(execute("DELETE v FROM %s WHERE k = 1 AND i = 2 IF s != 1"), row(false, 1));
-        assertRows(execute("DELETE v FROM %s WHERE k = 1 AND i = 2 IF s = 1"), row(true));
+        assertRows(execute("DELETE v FROM %s WHERE k = 1 AND i = 2 IF s != ?", 1), row(false, 1));
+        assertRows(execute("DELETE v FROM %s WHERE k = 1 AND i = 2 IF s = ?", 1), row(true));
         assertRows(execute("SELECT * FROM %s WHERE k = 1 AND i = 2"), row(1, 2, 1, null));
 
-        assertRows(execute("DELETE FROM %s WHERE  k = 1 AND i = 2 IF s != 1"), row(false, 1));
-        assertRows(execute("DELETE FROM %s WHERE k = 1 AND i = 2 IF s = 1"), row(true));
+        assertRows(execute("DELETE FROM %s WHERE  k = 1 AND i = 2 IF s != ?", 1), row(false, 1));
+        assertRows(execute("DELETE FROM %s WHERE k = 1 AND i = 2 IF s = ?", 1), row(true));
         assertEmpty(execute("SELECT * FROM %s WHERE k = 1 AND i = 2"));
         assertRows(execute("SELECT * FROM %s WHERE k = 1"), row(1, null, 1, null));
     }
@@ -211,13 +240,13 @@
 
         execute("INSERT INTO %s(id, version) VALUES (0, 0)");
 
-        assertRows(execute("UPDATE %s SET v='foo', version=1 WHERE id=0 AND k='k1' IF version = 0"), row(true));
+        assertRows(execute("UPDATE %s SET v='foo', version=1 WHERE id=0 AND k='k1' IF version = ?", 0), row(true));
         assertRows(execute("SELECT * FROM %s"), row(0, "k1", 1, "foo"));
 
-        assertRows(execute("UPDATE %s SET v='bar', version=1 WHERE id=0 AND k='k2' IF version = 0"), row(false, 1));
+        assertRows(execute("UPDATE %s SET v='bar', version=1 WHERE id=0 AND k='k2' IF version = ?", 0), row(false, 1));
         assertRows(execute("SELECT * FROM %s"), row(0, "k1", 1, "foo"));
 
-        assertRows(execute("UPDATE %s SET v='bar', version=2 WHERE id=0 AND k='k2' IF version = 1"), row(true));
+        assertRows(execute("UPDATE %s SET v='bar', version=2 WHERE id=0 AND k='k2' IF version = ?", 1), row(true));
         assertRows(execute("SELECT * FROM %s"), row(0, "k1", 2, "foo"), row(0, "k2", 2, "bar"));
 
         // Testing batches
@@ -559,20 +588,22 @@
 
             execute("INSERT INTO %s(k, l) VALUES (0, ['foo', 'bar', 'foobar'])");
 
-            assertInvalid("DELETE FROM %s WHERE k=0 IF l[null] = 'foobar'");
-            assertInvalid("DELETE FROM %s WHERE k=0 IF l[-2] = 'foobar'");
+            assertInvalidMessage("Invalid null value for list element access",
+                                 "DELETE FROM %s WHERE k=0 IF l[?] = ?", null, "foobar");
+            assertInvalidMessage("Invalid negative list index -2",
+                                 "DELETE FROM %s WHERE k=0 IF l[?] = ?", -2, "foobar");
 
-            assertRows(execute("DELETE FROM %s WHERE k=0 IF l[1] = null"), row(false, list("foo", "bar", "foobar")));
-            assertRows(execute("DELETE FROM %s WHERE k=0 IF l[1] = 'foobar'"), row(false, list("foo", "bar", "foobar")));
+            assertRows(execute("DELETE FROM %s WHERE k=0 IF l[?] = ?", 1, null), row(false, list("foo", "bar", "foobar")));
+            assertRows(execute("DELETE FROM %s WHERE k=0 IF l[?] = ?", 1, "foobar"), row(false, list("foo", "bar", "foobar")));
             assertRows(execute("SELECT * FROM %s"), row(0, list("foo", "bar", "foobar")));
 
-            assertRows(execute("DELETE FROM %s WHERE k=0 IF l[1] = 'bar'"), row(true));
+            assertRows(execute("DELETE FROM %s WHERE k=0 IF l[?] = ?", 1, "bar"), row(true));
             assertEmpty(execute("SELECT * FROM %s"));
         }
     }
 
     /**
-     * Test expanded functionality from CASSANDRA-6839,
+     * Test expanded functionality from CASSANDRA-6839, 
      * migrated from cql_tests.py:TestCQL.expanded_list_item_conditional_test()
      */
     @Test
@@ -766,19 +797,21 @@
                                       : "map<text, text>"));
 
             execute("INSERT INTO %s (k, m) VALUES (0, {'foo' : 'bar'})");
-            assertInvalid("DELETE FROM %s WHERE k=0 IF m[null] = 'foo'");
-            assertRows(execute("DELETE FROM %s WHERE k=0 IF m['foo'] = 'foo'"), row(false, map("foo", "bar")));
-            assertRows(execute("DELETE FROM %s WHERE k=0 IF m['foo'] = null"), row(false, map("foo", "bar")));
+            assertInvalidMessage("Invalid null value for map element access",
+                                 "DELETE FROM %s WHERE k=0 IF m[?] = ?", null, "foo");
+            assertRows(execute("DELETE FROM %s WHERE k=0 IF m[?] = ?", "foo", "foo"), row(false, map("foo", "bar")));
+            assertRows(execute("DELETE FROM %s WHERE k=0 IF m[?] = ?", "foo", null), row(false, map("foo", "bar")));
             assertRows(execute("SELECT * FROM %s"), row(0, map("foo", "bar")));
 
-            assertRows(execute("DELETE FROM %s WHERE k=0 IF m['foo'] = 'bar'"), row(true));
+            assertRows(execute("DELETE FROM %s WHERE k=0 IF m[?] = ?", "foo", "bar"), row(true));
             assertEmpty(execute("SELECT * FROM %s"));
 
             execute("INSERT INTO %s(k, m) VALUES (1, null)");
             if (frozen)
-                assertInvalid("UPDATE %s set m['foo'] = 'bar', m['bar'] = 'foo' WHERE k = 1 IF m['foo'] IN ('blah', null)");
+                assertInvalidMessage("Invalid operation (m['foo'] = 'bar') for frozen collection column m",
+                                     "UPDATE %s set m['foo'] = 'bar', m['bar'] = 'foo' WHERE k = 1 IF m[?] IN (?, ?)", "foo", "blah", null);
             else
-                assertRows(execute("UPDATE %s set m['foo'] = 'bar', m['bar'] = 'foo' WHERE k = 1 IF m['foo'] IN ('blah', null)"), row(true));
+                assertRows(execute("UPDATE %s set m['foo'] = 'bar', m['bar'] = 'foo' WHERE k = 1 IF m[?] IN (?, ?)", "foo", "blah", null), row(true));
         }
     }
 
@@ -851,7 +884,7 @@
         assertInvalidThrow(expected, "UPDATE %s SET m = {'foo': 'bar'} WHERE k=0 IF " + condition);
         assertRows(execute("SELECT * FROM %s"), row(0, map("foo", "bar")));
     }
-    
+
     /**
      * Test for 7499,
      * migrated from cql_tests.py:TestCQL.cas_and_list_index_test()
@@ -876,12 +909,12 @@
     @Test
     public void testDropCreateKeyspaceIfNotExists() throws Throwable
     {
-        String keyspace =  KEYSPACE;
+        String keyspace =  KEYSPACE_PER_TEST;
 
-        schemaChange("DROP KEYSPACE IF EXISTS " + keyspace);
+        dropPerTestKeyspace();
 
         // try dropping when doesn't exist
-        schemaChange("DROP KEYSPACE IF EXISTS " + keyspace);
+        dropPerTestKeyspace();
 
         // create and confirm
         schemaChange("CREATE KEYSPACE IF NOT EXISTS " + keyspace + " WITH replication = { 'class':'SimpleStrategy', 'replication_factor':1} and durable_writes = true ");
@@ -987,6 +1020,8 @@
     {
         createTable("CREATE TABLE %s (a int, b int, s int static, d text, PRIMARY KEY (a, b))");
 
+        assertInvalidMessage("Invalid 'unset' value in condition", "UPDATE %s SET s = 6 WHERE a = 6 IF s = ?", unset());
+
         // pre-existing row
         execute("INSERT INTO %s (a, b, s, d) values (6, 6, 100, 'a')");
         assertRows(execute("UPDATE %s SET s = 6 WHERE a = 6 IF s = 100"),
@@ -1021,7 +1056,7 @@
         conditionalUpdatesWithNonExistingOrNullValues();
 
         // rejected: IN doesn't contain null
-        assertRows(execute("UPDATE %s SET s = 30 WHERE a = 3 IF s IN (10,20,30)"),
+        assertRows(execute("UPDATE %s SET s = 30 WHERE a = 3 IF s IN ?", list(10,20,30)),
                    row(false));
         assertRows(execute("SELECT * FROM %s WHERE a = 3"),
                    row(3, 3, null, null));
@@ -1029,7 +1064,7 @@
         // rejected: comparing number with NULL always returns false
         for (String operator: new String[] { ">", "<", ">=", "<=", "="})
         {
-            assertRows(execute("UPDATE %s SET s = 50 WHERE a = 5 IF s " + operator + " 3"),
+            assertRows(execute("UPDATE %s SET s = 50 WHERE a = 5 IF s " + operator + " ?", 3),
                        row(false));
             assertRows(execute("SELECT * FROM %s WHERE a = 5"),
                        row(5, 5, null, null));
@@ -1045,14 +1080,14 @@
         conditionalUpdatesWithNonExistingOrNullValues();
 
         // rejected: IN doesn't contain null
-        assertRows(execute("UPDATE %s SET s = 3 WHERE a = 3 IF s IN (10,20,30)"),
+        assertRows(execute("UPDATE %s SET s = 3 WHERE a = 3 IF s IN ?", list(10,20,30)),
                    row(false));
         assertEmpty(execute("SELECT a, s, d FROM %s WHERE a = 3"));
 
         // rejected: comparing number with NULL always returns false
         for (String operator : new String[]{ ">", "<", ">=", "<=", "=" })
         {
-            assertRows(execute("UPDATE %s SET s = 50 WHERE a = 5 IF s " + operator + " 3"),
+            assertRows(execute("UPDATE %s SET s = 50 WHERE a = 5 IF s " + operator + " ?", 3),
                        row(false));
             assertEmpty(execute("SELECT * FROM %s WHERE a = 5"));
         }
@@ -1060,17 +1095,17 @@
 
     private void conditionalUpdatesWithNonExistingOrNullValues() throws Throwable
     {
-        assertRows(execute("UPDATE %s SET s = 1 WHERE a = 1 IF s = NULL"),
+        assertRows(execute("UPDATE %s SET s = 1 WHERE a = 1 IF s = ?", (Integer) null),
                    row(true));
         assertRows(execute("SELECT a, s, d FROM %s WHERE a = 1"),
                    row(1, 1, null));
 
-        assertRows(execute("UPDATE %s SET s = 2 WHERE a = 2 IF s IN (10,20,NULL)"),
+        assertRows(execute("UPDATE %s SET s = 2 WHERE a = 2 IF s IN (?, ?, ?)", 10,20,null),
                    row(true));
         assertRows(execute("SELECT a, s, d FROM %s WHERE a = 2"),
                    row(2, 2, null));
 
-        assertRows(execute("UPDATE %s SET s = 4 WHERE a = 4 IF s != 4"),
+        assertRows(execute("UPDATE %s SET s = 4 WHERE a = 4 IF s != ?", 4 ),
                    row(true));
         assertRows(execute("SELECT a, s, d FROM %s WHERE a = 4"),
                    row(4, 4, null));
@@ -1182,23 +1217,23 @@
         for (int i = 1; i <= 5; i++)
             execute("INSERT INTO %s (a, b, s1, s2, v) VALUES (?, ?, ?, ?, ?)", i, i, i, null, i);
 
-        assertRows(execute("DELETE s1 FROM %s WHERE a = 1 IF s2 = NULL"),
+        assertRows(execute("DELETE s1 FROM %s WHERE a = 1 IF s2 = ?", (Integer) null),
                    row(true));
         assertRows(execute("SELECT * FROM %s WHERE a = 1"),
                    row(1, 1, null, null, 1));
 
         // rejected: IN doesn't contain null
-        assertRows(execute("DELETE s1 FROM %s WHERE a = 2 IF s2 IN (10,20,30)"),
+        assertRows(execute("DELETE s1 FROM %s WHERE a = 2 IF s2 IN ?", list(10,20,30)),
                    row(false, null));
         assertRows(execute("SELECT * FROM %s WHERE a = 2"),
                    row(2, 2, 2, null, 2));
 
-        assertRows(execute("DELETE s1 FROM %s WHERE a = 3 IF s2 IN (NULL,20,30)"),
+        assertRows(execute("DELETE s1 FROM %s WHERE a = 3 IF s2 IN (?, ?, ?)", null, 20, 30),
                    row(true));
         assertRows(execute("SELECT * FROM %s WHERE a = 3"),
                    row(3, 3, null, null, 3));
 
-        assertRows(execute("DELETE s1 FROM %s WHERE a = 4 IF s2 != 4"),
+        assertRows(execute("DELETE s1 FROM %s WHERE a = 4 IF s2 != ?", 4),
                    row(true));
         assertRows(execute("SELECT * FROM %s WHERE a = 4"),
                    row(4, 4, null, null, 4));
@@ -1206,7 +1241,7 @@
         // rejected: comparing number with NULL always returns false
         for (String operator : new String[]{ ">", "<", ">=", "<=", "=" })
         {
-            assertRows(execute("DELETE s1 FROM %s WHERE a = 5 IF s2 " + operator + " 3"),
+            assertRows(execute("DELETE s1 FROM %s WHERE a = 5 IF s2 " + operator + " ?", 3),
                        row(false, null));
             assertRows(execute("SELECT * FROM %s WHERE a = 5"),
                        row(5, 5, 5, null, 5));
@@ -1270,4 +1305,159 @@
         assertRows(execute("SELECT * FROM %s WHERE a = 7"),
                    row(7, 7, null, null, 7));
     }
+
+    @Test
+    public void testInMarkerWithUDTs() throws Throwable
+    {
+        String typename = createType("CREATE TYPE %s (a int, b text)");
+        String myType = KEYSPACE + '.' + typename;
+
+            createTable("CREATE TABLE %s (k int PRIMARY KEY, v frozen<" + myType + "> )");
+
+            Object v = userType(0, "abc");
+            execute("INSERT INTO %s (k, v) VALUES (?, ?)", 0, v);
+
+            // Does not apply
+            assertRows(execute("UPDATE %s SET v = {a: 0, b: 'bc'} WHERE k = 0 IF v IN (?, ?)", userType(1, "abc"), userType(0, "ac")),
+                       row(false, v));
+            assertRows(execute("UPDATE %s SET v = {a: 0, b: 'bc'} WHERE k = 0 IF v IN (?, ?)", userType(1, "abc"), null),
+                       row(false, v));
+            assertRows(execute("UPDATE %s SET v = {a: 0, b: 'bc'} WHERE k = 0 IF v IN (?, ?)", userType(1, "abc"), unset()),
+                       row(false, v));
+            assertRows(execute("UPDATE %s SET v = {a: 0, b: 'bc'} WHERE k = 0 IF v IN (?, ?)", null, null),
+                       row(false, v));
+            assertRows(execute("UPDATE %s SET v = {a: 0, b: 'bc'} WHERE k = 0 IF v IN (?, ?)", unset(), unset()),
+                       row(false, v));
+            assertRows(execute("UPDATE %s SET v = {a: 0, b: 'bc'} WHERE k = 0 IF v IN ?", list(userType(1, "abc"), userType(0, "ac"))),
+                       row(false, v));
+
+            // Does apply
+            assertRows(execute("UPDATE %s SET v = {a: 0, b: 'bc'} WHERE k = 0 IF v IN (?, ?)", userType(0, "abc"), userType(0, "ac")),
+                       row(true));
+            assertRows(execute("UPDATE %s SET v = {a: 1, b: 'bc'} WHERE k = 0 IF v IN (?, ?)", userType(0, "bc"), null),
+                       row(true));
+            assertRows(execute("UPDATE %s SET v = {a: 1, b: 'ac'} WHERE k = 0 IF v IN (?, ?, ?)", userType(0, "bc"), unset(), userType(1, "bc")),
+                       row(true));
+            assertRows(execute("UPDATE %s SET v = {a: 0, b: 'abc'} WHERE k = 0 IF v IN ?", list(userType(1, "ac"), userType(0, "ac"))),
+                       row(true));
+
+            assertInvalidMessage("Invalid null list in IN condition",
+                                 "UPDATE %s SET v = {a: 0, b: 'bc'} WHERE k = 0 IF v IN ?", (List<ByteBuffer>) null);
+            assertInvalidMessage("Invalid 'unset' value in condition",
+                                 "UPDATE %s SET v = {a: 0, b: 'bc'} WHERE k = 0 IF v IN ?", unset());
+    }
+
+    @Test
+    public void testInMarkerWithLists() throws Throwable
+    {
+        for (boolean frozen : new boolean[]{false, true})
+        {
+            createTable(String.format("CREATE TABLE %%s (k int PRIMARY KEY, l %s)",
+                                      frozen
+                                      ? "frozen<list<text>>"
+                                      : "list<text>"));
+
+            execute("INSERT INTO %s(k, l) VALUES (0, ['foo', 'bar', 'foobar'])");
+
+            // Does not apply
+            assertRows(execute("UPDATE %s SET l = ['foo', 'bar'] WHERE k = 0 IF l IN (?, ?)", list("foo", "foobar"), list("bar", "foobar")),
+                       row(false, list("foo", "bar", "foobar")));
+            assertRows(execute("UPDATE %s SET l = ['foo', 'bar'] WHERE k = 0 IF l IN (?, ?)", list("foo", "foobar"), null),
+                       row(false, list("foo", "bar", "foobar")));
+            assertRows(execute("UPDATE %s SET l = ['foo', 'bar'] WHERE k = 0 IF l IN (?, ?)", list("foo", "foobar"), unset()),
+                       row(false, list("foo", "bar", "foobar")));
+            assertRows(execute("UPDATE %s SET l = ['foo', 'bar'] WHERE k = 0 IF l IN (?, ?)", null, null),
+                       row(false, list("foo", "bar", "foobar")));
+            assertRows(execute("UPDATE %s SET l = ['foo', 'bar'] WHERE k = 0 IF l IN ?", list(list("foo", "foobar"), list("bar", "foobar"))),
+                       row(false, list("foo", "bar", "foobar")));
+            assertRows(execute("UPDATE %s SET l = ['foo', 'bar'] WHERE k = 0 IF l[?] IN ?", 1, list("foo", "foobar")),
+                       row(false, list("foo", "bar", "foobar")));
+            assertRows(execute("UPDATE %s SET l = ['foo', 'bar'] WHERE k = 0 IF l[?] IN (?, ?)", 1, "foo", "foobar"),
+                       row(false, list("foo", "bar", "foobar")));
+            assertRows(execute("UPDATE %s SET l = ['foo', 'bar'] WHERE k = 0 IF l[?] IN (?, ?)", 1, "foo", null),
+                       row(false, list("foo", "bar", "foobar")));
+            assertRows(execute("UPDATE %s SET l = ['foo', 'bar'] WHERE k = 0 IF l[?] IN (?, ?)", 1, "foo", unset()),
+                       row(false, list("foo", "bar", "foobar")));
+
+            // Does apply
+            assertRows(execute("UPDATE %s SET l = ['foo', 'bar'] WHERE k = 0 IF l IN (?, ?)", list("foo", "bar", "foobar"), list("bar", "foobar")),
+                       row(true));
+            assertRows(execute("UPDATE %s SET l = ['foo', 'foobar'] WHERE k = 0 IF l IN (?, ?, ?)", list("foo", "bar", "foobar"), null, list("foo", "bar")),
+                       row(true));
+            assertRows(execute("UPDATE %s SET l = ['foo', 'bar'] WHERE k = 0 IF l IN (?, ?, ?)", list("foo", "bar", "foobar"), unset(), list("foo", "foobar")),
+                       row(true));
+            assertRows(execute("UPDATE %s SET l = ['foo', 'foobar'] WHERE k = 0 IF l IN (?, ?)", list("bar", "foobar"), list("foo", "bar")),
+                       row(true));
+            assertRows(execute("UPDATE %s SET l = ['foo', 'bar'] WHERE k = 0 IF l[?] IN ?", 1, list("bar", "foobar")),
+                       row(true));
+            assertRows(execute("UPDATE %s SET l = ['foo', 'foobar'] WHERE k = 0 IF l[?] IN (?, ?, ?)", 1, "bar", null, "foobar"),
+                       row(true));
+            assertRows(execute("UPDATE %s SET l = ['foo', 'foobar'] WHERE k = 0 IF l[?] IN (?, ?, ?)", 1, "bar", unset(), "foobar"),
+                       row(true));
+
+            assertInvalidMessage("Invalid null list in IN condition",
+                                 "UPDATE %s SET l = ['foo', 'bar'] WHERE k = 0 IF l IN ?", (List<ByteBuffer>) null);
+            assertInvalidMessage("Invalid 'unset' value in condition",
+                                 "UPDATE %s SET l = ['foo', 'bar'] WHERE k = 0 IF l IN ?", unset());
+            assertInvalidMessage("Invalid 'unset' value in condition",
+                                 "UPDATE %s SET l = ['foo', 'bar'] WHERE k = 0 IF l[?] IN ?", 1, unset());
+        }
+    }
+
+    @Test
+    public void testInMarkerWithMaps() throws Throwable
+    {
+        for (boolean frozen : new boolean[] {false, true})
+        {
+            createTable(String.format("CREATE TABLE %%s (k int PRIMARY KEY, m %s)",
+                                      frozen
+                                      ? "frozen<map<text, text>>"
+                                      : "map<text, text>"));
+
+            execute("INSERT INTO %s (k, m) VALUES (0, {'foo' : 'bar'})");
+
+            // Does not apply
+            assertRows(execute("UPDATE %s SET m = {'foo' : 'foobar'} WHERE k = 0 IF m IN (?, ?)", map("foo", "foobar"), map("bar", "foobar")),
+                       row(false, map("foo", "bar")));
+            assertRows(execute("UPDATE %s SET  m = {'foo' : 'foobar'} WHERE k = 0 IF m IN (?, ?)", map("foo", "foobar"), null),
+                       row(false, map("foo", "bar")));
+            assertRows(execute("UPDATE %s SET  m = {'foo' : 'foobar'} WHERE k = 0 IF m IN (?, ?)", map("foo", "foobar"), unset()),
+                       row(false, map("foo", "bar")));
+            assertRows(execute("UPDATE %s SET  m = {'foo' : 'foobar'} WHERE k = 0 IF m IN (?, ?)", null, null),
+                       row(false, map("foo", "bar")));
+            assertRows(execute("UPDATE %s SET  m = {'foo' : 'foobar'} WHERE k = 0 IF m IN ?", list(map("foo", "foobar"), map("bar", "foobar"))),
+                       row(false, map("foo", "bar")));
+            assertRows(execute("UPDATE %s SET  m = {'foo' : 'foobar'} WHERE k = 0 IF m[?] IN ?", "foo", list("foo", "foobar")),
+                       row(false, map("foo", "bar")));
+            assertRows(execute("UPDATE %s SET  m = {'foo' : 'foobar'} WHERE k = 0 IF m[?] IN (?, ?)", "foo", "foo", "foobar"),
+                       row(false, map("foo", "bar")));
+            assertRows(execute("UPDATE %s SET  m = {'foo' : 'foobar'} WHERE k = 0 IF m[?] IN (?, ?)", "foo", "foo", null),
+                       row(false, map("foo", "bar")));
+            assertRows(execute("UPDATE %s SET  m = {'foo' : 'foobar'} WHERE k = 0 IF m[?] IN (?, ?)", "foo", "foo", unset()),
+                       row(false, map("foo", "bar")));
+
+            // Does apply
+            assertRows(execute("UPDATE %s SET m = {'foo' : 'foobar'} WHERE k = 0 IF m IN (?, ?)", map("foo", "foobar"), map("foo", "bar")),
+                       row(true));
+            assertRows(execute("UPDATE %s SET m = {'foo' : 'bar'} WHERE k = 0 IF m IN (?, ?, ?)", map("bar", "foobar"), null, map("foo", "foobar")),
+                       row(true));
+            assertRows(execute("UPDATE %s SET m = {'foo' : 'bar'} WHERE k = 0 IF m IN (?, ?, ?)", map("bar", "foobar"), unset(), map("foo", "bar")),
+                       row(true));
+            assertRows(execute("UPDATE %s SET m = {'foo' : 'foobar'} WHERE k = 0 IF m IN ?", list(map("foo", "bar"), map("bar", "foobar"))),
+                       row(true));
+            assertRows(execute("UPDATE %s SET m = {'foo' : 'bar'} WHERE k = 0 IF m[?] IN ?", "foo", list("bar", "foobar")),
+                       row(true));
+            assertRows(execute("UPDATE %s SET m = {'foo' : 'foobar'} WHERE k = 0 IF m[?] IN (?, ?, ?)", "foo", "bar", null, "foobar"),
+                       row(true));
+            assertRows(execute("UPDATE %s SET m = {'foo' : 'foobar'} WHERE k = 0 IF m[?] IN (?, ?, ?)", "foo", "bar", unset(), "foobar"),
+                       row(true));
+
+            assertInvalidMessage("Invalid null list in IN condition",
+                                 "UPDATE %s SET  m = {'foo' : 'foobar'} WHERE k = 0 IF m IN ?", (List<ByteBuffer>) null);
+            assertInvalidMessage("Invalid 'unset' value in condition",
+                                 "UPDATE %s SET  m = {'foo' : 'foobar'} WHERE k = 0 IF m IN ?", unset());
+            assertInvalidMessage("Invalid 'unset' value in condition",
+                                 "UPDATE %s SET  m = {'foo' : 'foobar'} WHERE k = 0 IF m[?] IN ?", "foo", unset());
+        }
+    }
 }
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectLimitTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectLimitTest.java
index 5cd24dd..5fbf36d 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectLimitTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectLimitTest.java
@@ -33,7 +33,7 @@
     @BeforeClass
     public static void setUp()
     {
-        DatabaseDescriptor.setPartitioner(new ByteOrderedPartitioner());
+        DatabaseDescriptor.setPartitioner(ByteOrderedPartitioner.instance);
     }
 
     /**
@@ -73,6 +73,7 @@
         // order of keys (even though 48 is after 2)
         assertRows(execute("SELECT * FROM %s WHERE userid IN (48, 2) LIMIT 1"),
                    row(2, "http://foo.com", 42L));
+
     }
 
     /**
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java
index 5f82328..0975662 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectMultiColumnRelationTest.java
@@ -17,30 +17,28 @@
  */
 package org.apache.cassandra.cql3.validation.operations;
 
-import org.junit.Ignore;
 import org.junit.Test;
 
 import org.apache.cassandra.cql3.CQLTester;
-
-import static org.junit.Assert.assertEquals;
+import org.apache.cassandra.cql3.restrictions.StatementRestrictions;
 
 public class SelectMultiColumnRelationTest extends CQLTester
 {
     @Test
     public void testSingleClusteringInvalidQueries() throws Throwable
     {
-        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
+        for (String compactOption : new String[] { "", " WITH COMPACT STORAGE" })
         {
             createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))" + compactOption);
 
             assertInvalidSyntax("SELECT * FROM %s WHERE () = (?, ?)", 1, 2);
-            assertInvalidMessage("Column \"b\" cannot be restricted by an equality relation and an inequality relation",
+            assertInvalidMessage("b cannot be restricted by more than one relation if it includes an Equal",
                                  "SELECT * FROM %s WHERE a = 0 AND (b) = (?) AND (b) > (?)", 0, 0);
             assertInvalidMessage("More than one restriction was found for the start bound on b",
                                  "SELECT * FROM %s WHERE a = 0 AND (b) > (?) AND (b) > (?)", 0, 1);
             assertInvalidMessage("More than one restriction was found for the start bound on b",
                                  "SELECT * FROM %s WHERE a = 0 AND (b) > (?) AND b > ?", 0, 1);
-            assertInvalidMessage("Multi-column relations can only be applied to clustering columns: a",
+            assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: a",
                                  "SELECT * FROM %s WHERE (a, b) = (?, ?)", 0, 0);
         }
     }
@@ -48,7 +46,7 @@
     @Test
     public void testMultiClusteringInvalidQueries() throws Throwable
     {
-        for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
+        for (String compactOption : new String[] { "", " WITH COMPACT STORAGE" })
         {
             createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c, d))" + compactOption);
 
@@ -71,53 +69,49 @@
                                  "SELECT * FROM %s WHERE a=0 AND (b, c, d) IN ((?, ?, ?, ?, ?))", 0, 1, 2, 3, 4);
 
             // Missing first clustering column
-            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted (preceding column \"b\" is not restricted)",
+            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted as preceding column \"b\" is not restricted",
                                  "SELECT * FROM %s WHERE a = 0 AND (c, d) = (?, ?)", 0, 0);
-            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted (preceding column \"b\" is not restricted)",
+            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted as preceding column \"b\" is not restricted",
                                  "SELECT * FROM %s WHERE a = 0 AND (c, d) > (?, ?)", 0, 0);
 
             // Nulls
-            assertInvalidMessage("Invalid null value in condition for column d",
+            assertInvalidMessage("Invalid null value in condition for columns: [b, c, d]",
                                  "SELECT * FROM %s WHERE a = 0 AND (b, c, d) IN ((?, ?, ?))", 1, 2, null);
 
             // Wrong type for 'd'
-            assertInvalidMessage("Expected 4 or 0 byte int (6)",
-                                 "SELECT * FROM %s WHERE a = 0 AND (b, c, d) = (?, ?, ?)", 1, 2, "foobar");
-
-            assertInvalidMessage("Invalid tuple type literal for b of type int",
-                                 "SELECT * FROM %s WHERE a = 0 AND b = (?, ?, ?)", 1, 2, 3);
+            assertInvalid("SELECT * FROM %s WHERE a = 0 AND (b, c, d) = (?, ?, ?)", 1, 2, "foobar");
+            assertInvalid("SELECT * FROM %s WHERE a = 0 AND b = (?, ?, ?)", 1, 2, 3);
 
             // Mix single and tuple inequalities
-            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column: c < ?",
+             assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column",
                                  "SELECT * FROM %s WHERE a = 0 AND (b, c, d) > (?, ?, ?) AND c < ?", 0, 1, 0, 1);
-            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column: (b, c, d) < (?, ?, ?)",
+            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column",
                                  "SELECT * FROM %s WHERE a = 0 AND c > ? AND (b, c, d) < (?, ?, ?)", 1, 1, 1, 0);
 
-            assertInvalidMessage("Multi-column relations can only be applied to clustering columns: a",
+            assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: a",
                                  "SELECT * FROM %s WHERE (a, b, c, d) IN ((?, ?, ?, ?))", 0, 1, 2, 3);
-            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted (preceding column \"b\" is not restricted)",
+            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted as preceding column \"b\" is not restricted",
                                  "SELECT * FROM %s WHERE (c, d) IN ((?, ?))", 0, 1);
-            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
+
+            assertInvalidMessage("Clustering column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
                                  "SELECT * FROM %s WHERE a = ? AND b > ?  AND (c, d) IN ((?, ?))", 0, 0, 0, 0);
 
-            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
+            assertInvalidMessage("Clustering column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
                                  "SELECT * FROM %s WHERE a = ? AND b > ?  AND (c, d) > (?, ?)", 0, 0, 0, 0);
-            assertInvalidMessage("PRIMARY KEY column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
+            assertInvalidMessage("Clustering column \"c\" cannot be restricted (preceding column \"b\" is restricted by a non-EQ relation)",
                                  "SELECT * FROM %s WHERE a = ? AND (c, d) > (?, ?) AND b > ?  ", 0, 0, 0, 0);
-            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column: (c) < (?)",
+
+            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column",
                                  "SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?) AND (b) < (?) AND (c) < (?)", 0, 0, 0, 0, 0);
-            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column: (b, c) > (?, ?)",
+            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column",
                                  "SELECT * FROM %s WHERE a = ? AND (c) < (?) AND (b, c) > (?, ?) AND (b) < (?)", 0, 0, 0, 0, 0);
-            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column: (b, c) > (?, ?)",
+            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column",
                                  "SELECT * FROM %s WHERE a = ? AND (b) < (?) AND (c) < (?) AND (b, c) > (?, ?)", 0, 0, 0, 0, 0);
-            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column: (b, c) > (?, ?)",
+            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column",
                                  "SELECT * FROM %s WHERE a = ? AND (b) < (?) AND c < ? AND (b, c) > (?, ?)", 0, 0, 0, 0, 0);
 
-            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column: (c) < (?)",
+            assertInvalidMessage("Column \"c\" cannot be restricted by two inequalities not starting with the same column",
                                  "SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?) AND (c) < (?)", 0, 0, 0, 0);
-
-            assertInvalidMessage("PRIMARY KEY column \"d\" cannot be restricted (preceding column \"c\" is restricted by an IN tuple notation)",
-                                 "SELECT * FROM %s WHERE a = ? AND (b, c) in ((?, ?), (?, ?)) AND d > ?", 0, 0, 0, 0, 0, 0);
         }
     }
 
@@ -139,9 +133,17 @@
             assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) = (?, ?)", 0, 1, 0, 0),
                        row(0, 1, 0, 0));
 
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and b IN (?, ?) and (c, d) = (?, ?)", 0, 0, 1, 0, 0),
+                       row(0, 0, 0, 0),
+                       row(0, 1, 0, 0));
+
             assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c) IN ((?))", 0, 1, 0),
                        row(0, 1, 0, 0));
 
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and b IN (?, ?) and (c) IN ((?))", 0, 0, 1, 0),
+                       row(0, 0, 0, 0),
+                       row(0, 1, 0, 0));
+
             assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c) IN ((?), (?))", 0, 1, 0, 1),
                        row(0, 1, 0, 0),
                        row(0, 1, 1, 0),
@@ -154,10 +156,22 @@
                        row(0, 1, 0, 0),
                        row(0, 1, 1, 1));
 
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and b IN (?, ?) and (c, d) IN ((?, ?), (?, ?))", 0, 0, 1, 0, 0, 1, 1),
+                       row(0, 0, 0, 0),
+                       row(0, 0, 1, 1),
+                       row(0, 1, 0, 0),
+                       row(0, 1, 1, 1));
+
             assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) > (?, ?)", 0, 1, 0, 0),
                        row(0, 1, 1, 0),
                        row(0, 1, 1, 1));
 
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and b IN (?, ?) and (c, d) > (?, ?)", 0, 0, 1, 0, 0),
+                       row(0, 0, 1, 0),
+                       row(0, 0, 1, 1),
+                       row(0, 1, 1, 0),
+                       row(0, 1, 1, 1));
+
             assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c, d) > (?, ?) and (c) <= (?) ", 0, 1, 0, 0, 1),
                        row(0, 1, 1, 0),
                        row(0, 1, 1, 1));
@@ -173,6 +187,10 @@
             assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) = (?, ?) and d = ?", 0, 0, 1, 0),
                        row(0, 0, 1, 0));
 
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) IN ((?, ?), (?, ?)) and d = ?", 0, 0, 1, 0, 0, 0),
+                       row(0, 0, 0, 0),
+                       row(0, 0, 1, 0));
+
             assertRows(execute("SELECT * FROM %s WHERE a = ? and b = ? and (c) = (?) and d = ?", 0, 0, 1, 0),
                        row(0, 0, 1, 0));
 
@@ -188,11 +206,15 @@
 
             assertRows(execute("SELECT * FROM %s WHERE a = ? and d < 1 and (b, c) = (?, ?) and d >= ?", 0, 0, 1, 0),
                        row(0, 0, 1, 0));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and d < 1 and (b, c) IN ((?, ?), (?, ?)) and d >= ?", 0, 0, 1, 0, 0, 0),
+                       row(0, 0, 0, 0),
+                       row(0, 0, 1, 0));
         }
     }
 
     @Test
-    public void testMultipleMultiColumnRelation() throws Throwable
+    public void testSeveralMultiColumnRelation() throws Throwable
     {
         for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
         {
@@ -209,12 +231,17 @@
             assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) = (?, ?)", 0, 1, 0, 0),
                        row(0, 1, 0, 0));
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c) = (?) and (d) = (?)", 0, 1, 0, 0),
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) IN ((?), (?)) and (c, d) = (?, ?)", 0, 0, 1, 0, 0),
+                       row(0, 0, 0, 0),
                        row(0, 1, 0, 0));
 
             assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c) IN ((?))", 0, 1, 0),
                        row(0, 1, 0, 0));
 
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) IN ((?),(?)) and (c) IN ((?))", 0, 0, 1, 0),
+                       row(0, 0, 0, 0),
+                       row(0, 1, 0, 0));
+
             assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c) IN ((?), (?))", 0, 1, 0, 1),
                        row(0, 1, 0, 0),
                        row(0, 1, 1, 0),
@@ -227,10 +254,22 @@
                        row(0, 1, 0, 0),
                        row(0, 1, 1, 1));
 
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) IN ((?), (?)) and (c, d) IN ((?, ?), (?, ?))", 0, 0, 1, 0, 0, 1, 1),
+                       row(0, 0, 0, 0),
+                       row(0, 0, 1, 1),
+                       row(0, 1, 0, 0),
+                       row(0, 1, 1, 1));
+
             assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) > (?, ?)", 0, 1, 0, 0),
                        row(0, 1, 1, 0),
                        row(0, 1, 1, 1));
 
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) IN ((?),(?)) and (c, d) > (?, ?)", 0, 0, 1, 0, 0),
+                       row(0, 0, 1, 0),
+                       row(0, 0, 1, 1),
+                       row(0, 1, 1, 0),
+                       row(0, 1, 1, 1));
+
             assertRows(execute("SELECT * FROM %s WHERE a = ? and (b) = (?) and (c, d) > (?, ?) and (c) <= (?) ", 0, 1, 0, 0, 1),
                        row(0, 1, 1, 0),
                        row(0, 1, 1, 1));
@@ -243,7 +282,18 @@
                        row(0, 1, 0, 0),
                        row(0, 1, 1, 0));
 
-            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) = (?, ?) and (d) = (?)", 0, 0, 1, 0),
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) = (?, ?) and d = ?", 0, 0, 1, 0),
+                       row(0, 0, 1, 0));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (b, c) IN ((?, ?), (?, ?)) and d = ?", 0, 0, 1, 0, 0, 0),
+                       row(0, 0, 0, 0),
+                       row(0, 0, 1, 0));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (d) < (1) and (b, c) = (?, ?) and (d) >= (?)", 0, 0, 1, 0),
+                       row(0, 0, 1, 0));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? and (d) < (1) and (b, c) IN ((?, ?), (?, ?)) and (d) >= (?)", 0, 0, 1, 0, 0, 0),
+                       row(0, 0, 0, 0),
                        row(0, 0, 1, 0));
         }
     }
@@ -254,10 +304,12 @@
         for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
         {
             createTable("CREATE TABLE %s (a int PRIMARY KEY, b int)" + compactOption);
-
-            assertInvalid("SELECT * FROM %s WHERE (a) > (?)", 0);
-            assertInvalid("SELECT * FROM %s WHERE (a) = (?)", 0);
-            assertInvalid("SELECT * FROM %s WHERE (b) = (?)", 0);
+            assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: a",
+                                 "SELECT * FROM %s WHERE (a) > (?)", 0);
+            assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: a",
+                                 "SELECT * FROM %s WHERE (a) = (?)", 0);
+            assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: b",
+                                 "SELECT * FROM %s WHERE (b) = (?)", 0);
         }
     }
 
@@ -327,7 +379,8 @@
         for (String compactOption : new String[]{"", " WITH COMPACT STORAGE"})
         {
             createTable("CREATE TABLE %s (a int PRIMARY KEY, b int)" + compactOption);
-            assertInvalid("SELECT * FROM %s WHERE a = 0 AND (b) != (0)");
+            assertInvalidMessage("Unsupported \"!=\" relation: (b) != (0)",
+                    "SELECT * FROM %s WHERE a = 0 AND (b) != (0)");
         }
     }
 
@@ -602,11 +655,15 @@
                     row(0, 0, 1, 1)
             );
 
+            assertEmpty(execute("SELECT * FROM %s WHERE a = ? and (b) IN ()", 0));
+
             assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) IN ((?, ?)) ORDER BY b DESC, c DESC, d DESC", 0, 0, 1),
                     row(0, 0, 1, 1),
                     row(0, 0, 1, 0)
             );
 
+            assertEmpty(execute("SELECT * FROM %s WHERE a = ? AND (b, c) IN () ORDER BY b DESC, c DESC, d DESC", 0));
+
             // IN on both partition key and clustering key
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 0, 0, 0);
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 0, 1, 0);
@@ -736,8 +793,10 @@
                     row(0, 0, 1, 0)
             );
 
-            // preserve pre-6875 behavior (even though the query result is technically incorrect)
-            assertEmpty(execute("SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?)", 0, 1, 0));
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) > (?, ?)", 0, 1, 0),
+                    row(0,1, 1, 1),
+                    row(0, 1, 1, 0)
+                    );
         }
     }
 
@@ -762,36 +821,83 @@
                    row(0, 1, 0, 0, 0),
                    row(0, 1, 1, 0, 1),
                    row(0, 1, 1, 1, 2));
+
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE (b, c) = (?, ?)", 1, 1);
         assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) = (?, ?)", 0, 1, 1),
                    row(0, 1, 1, 0, 1),
                    row(0, 1, 1, 1, 2));
         assertRows(execute("SELECT * FROM %s WHERE (b, c) = (?, ?) ALLOW FILTERING", 1, 1),
                    row(0, 1, 1, 0, 1),
                    row(0, 1, 1, 1, 2));
+
         assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b, c) = (?, ?) AND e = ?", 0, 1, 1, 2),
                    row(0, 1, 1, 1, 2));
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE (b, c) = (?, ?) AND e = ?", 1, 1, 2);
         assertRows(execute("SELECT * FROM %s WHERE (b, c) = (?, ?) AND e = ? ALLOW FILTERING", 1, 1, 2),
                    row(0, 1, 1, 1, 2));
-        assertRows(execute("SELECT * FROM %s WHERE (b) IN ((?)) AND e = ? ", 1, 2),
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b) IN ((?)) AND e = ? ALLOW FILTERING", 0, 1, 2),
+                   row(0, 1, 1, 1, 2));
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE (b) IN ((?)) AND e = ?", 1, 2);
+        assertRows(execute("SELECT * FROM %s WHERE (b) IN ((?)) AND e = ? ALLOW FILTERING", 1, 2),
                    row(0, 1, 1, 1, 2));
 
-        assertRows(execute("SELECT * FROM %s WHERE (b) IN ((?), (?)) AND e = ?", 0, 1, 2),
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE (b) IN ((?), (?)) AND e = ?", 0, 1, 2);
+        assertRows(execute("SELECT * FROM %s WHERE (b) IN ((?), (?)) AND e = ? ALLOW FILTERING", 0, 1, 2),
                    row(0, 0, 1, 1, 2),
                    row(0, 1, 1, 1, 2));
 
-        assertRows(execute("SELECT * FROM %s WHERE (b, c) IN ((?, ?)) AND e = ?", 0, 1, 2),
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE (b, c) IN ((?, ?)) AND e = ?", 0, 1, 2);
+        assertRows(execute("SELECT * FROM %s WHERE (b, c) IN ((?, ?)) AND e = ? ALLOW FILTERING", 0, 1, 2),
                    row(0, 0, 1, 1, 2));
 
-        assertRows(execute("SELECT * FROM %s WHERE (b, c) IN ((?, ?), (?, ?)) AND e = ?", 0, 1, 1, 1, 2),
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE (b, c) IN ((?, ?), (?, ?)) AND e = ?", 0, 1, 1, 1, 2);
+        assertRows(execute("SELECT * FROM %s WHERE (b, c) IN ((?, ?), (?, ?)) AND e = ? ALLOW FILTERING", 0, 1, 1, 1, 2),
                    row(0, 0, 1, 1, 2),
                    row(0, 1, 1, 1, 2));
 
-        assertRows(execute("SELECT * FROM %s WHERE (b) >= (?) AND e = ?", 1, 2),
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE (b) >= (?) AND e = ?", 1, 2);
+        assertRows(execute("SELECT * FROM %s WHERE (b) >= (?) AND e = ? ALLOW FILTERING", 1, 2),
                    row(0, 1, 1, 1, 2));
 
-        assertRows(execute("SELECT * FROM %s WHERE (b, c) >= (?, ?) AND e = ?", 1, 1, 2),
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE (b, c) >= (?, ?) AND e = ?", 1, 1, 2);
+        assertRows(execute("SELECT * FROM %s WHERE (b, c) >= (?, ?) AND e = ? ALLOW FILTERING", 1, 1, 2),
                    row(0, 1, 1, 1, 2));
+    }
 
+    @Test
+    public void testMultiColumnRestrictionsWithIndex() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, v int, PRIMARY KEY (a, b, c, d, e))");
+        createIndex("CREATE INDEX ON %s (v)");
+        for (int i = 1; i <= 5; i++)
+        {
+            execute("INSERT INTO %s (a,b,c,d,e,v) VALUES (?,?,?,?,?,?)", 0, i, 0, 0, 0, 0);
+            execute("INSERT INTO %s (a,b,c,d,e,v) VALUES (?,?,?,?,?,?)", 0, i, i, 0, 0, 0);
+            execute("INSERT INTO %s (a,b,c,d,e,v) VALUES (?,?,?,?,?,?)", 0, i, i, i, 0, 0);
+            execute("INSERT INTO %s (a,b,c,d,e,v) VALUES (?,?,?,?,?,?)", 0, i, i, i, i, 0);
+            execute("INSERT INTO %s (a,b,c,d,e,v) VALUES (?,?,?,?,?,?)", 0, i, i, i, i, i);
+        }
+
+        String errorMsg = "Multi-column slice restrictions cannot be used for filtering.";
+        assertInvalidMessage(errorMsg,
+                             "SELECT * FROM %s WHERE a = 0 AND (c,d) < (2,2) AND v = 0 ALLOW FILTERING");
+        assertInvalidMessage(errorMsg,
+                             "SELECT * FROM %s WHERE a = 0 AND (d,e) < (2,2) AND b = 1 AND v = 0 ALLOW FILTERING");
+        assertInvalidMessage(errorMsg,
+                             "SELECT * FROM %s WHERE a = 0 AND b = 1 AND (d,e) < (2,2) AND v = 0 ALLOW FILTERING");
+        assertInvalidMessage(errorMsg,
+                             "SELECT * FROM %s WHERE a = 0 AND b > 1 AND (d,e) < (2,2) AND v = 0 ALLOW FILTERING");
+        assertInvalidMessage(errorMsg,
+                             "SELECT * FROM %s WHERE a = 0 AND (b,c) > (1,0) AND (d,e) < (2,2) AND v = 0 ALLOW FILTERING");
     }
 
     @Test
@@ -811,27 +917,35 @@
 
         execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 2, 0, 0, 5);
 
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE a = ? AND (c) = (?)");
         assertRows(execute("SELECT * FROM %s WHERE a = ? AND (c) = (?) ALLOW FILTERING", 0, 1),
                    row(0, 0, 1, 0, 0, 3),
                    row(0, 0, 1, 1, 0, 4),
                    row(0, 0, 1, 1, 1, 5));
 
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE a = ? AND (c, d) = (?, ?)", 0, 1, 1);
         assertRows(execute("SELECT * FROM %s WHERE a = ? AND (c, d) = (?, ?) ALLOW FILTERING", 0, 1, 1),
                    row(0, 0, 1, 1, 0, 4),
                    row(0, 0, 1, 1, 1, 5));
 
-        assertInvalidMessage("Partition key part b must be restricted since preceding part is",
+        assertInvalidMessage("Partition key parts: b must be restricted as other parts are",
                              "SELECT * FROM %s WHERE a = ? AND (c, d) IN ((?, ?)) ALLOW FILTERING", 0, 1, 1);
 
-        assertInvalidMessage("Partition key part b must be restricted since preceding part is",
+        assertInvalidMessage("Partition key parts: b must be restricted as other parts are",
                              "SELECT * FROM %s WHERE a = ? AND (c, d) >= (?, ?) ALLOW FILTERING", 0, 1, 1);
 
         assertRows(execute("SELECT * FROM %s WHERE a = ? AND b = ? AND (c) IN ((?)) AND f = ?", 0, 0, 1, 5),
                    row(0, 0, 1, 1, 1, 5));
-
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE a = ? AND (c) IN ((?)) AND f = ?", 0, 1, 5);
         assertRows(execute("SELECT * FROM %s WHERE a = ? AND (c) IN ((?)) AND f = ? ALLOW FILTERING", 0, 1, 5),
                    row(0, 0, 1, 1, 1, 5));
 
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE a = ? AND (c) IN ((?), (?)) AND f = ?", 0, 1, 2, 5);
+
         assertRows(execute("SELECT * FROM %s WHERE a = ? AND b = ? AND (c) IN ((?), (?)) AND f = ?", 0, 0, 1, 2, 5),
                    row(0, 0, 1, 1, 1, 5),
                    row(0, 0, 2, 0, 0, 5));
@@ -842,10 +956,14 @@
 
         assertRows(execute("SELECT * FROM %s WHERE a = ? AND b = ? AND (c, d) IN ((?, ?)) AND f = ?", 0, 0, 1, 0, 3),
                    row(0, 0, 1, 0, 0, 3));
-
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE a = ? AND (c, d) IN ((?, ?)) AND f = ?", 0, 1, 0, 3);
         assertRows(execute("SELECT * FROM %s WHERE a = ? AND (c, d) IN ((?, ?)) AND f = ? ALLOW FILTERING", 0, 1, 0, 3),
                    row(0, 0, 1, 0, 0, 3));
 
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE a = ? AND (c) >= (?) AND f = ?", 0, 1, 5);
+
         assertRows(execute("SELECT * FROM %s WHERE a = ? AND b = ? AND (c) >= (?) AND f = ?", 0, 0, 1, 5),
                    row(0, 0, 1, 1, 1, 5),
                    row(0, 0, 2, 0, 0, 5));
@@ -857,157 +975,915 @@
         assertRows(execute("SELECT * FROM %s WHERE a = ? AND b = ? AND (c, d) >= (?, ?) AND f = ?", 0, 0, 1, 1, 5),
                    row(0, 0, 1, 1, 1, 5),
                    row(0, 0, 2, 0, 0, 5));
-
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE a = ? AND (c, d) >= (?, ?) AND f = ?", 0, 1, 1, 5);
         assertRows(execute("SELECT * FROM %s WHERE a = ? AND (c, d) >= (?, ?) AND f = ? ALLOW FILTERING", 0, 1, 1, 5),
                    row(0, 0, 1, 1, 1, 5),
                    row(0, 0, 2, 0, 0, 5));
     }
 
-    /**
-     * Check select on tuple relations with mixed ASC | DESC clustering, see CASSANDRA-7281
-     * migrated from cql_tests.py:TestCQL.tuple_query_mixed_order_columns_test to tuple_query_mixed_order_columns_test9
-     */
-    @Ignore // CASSANDRA-7281 not yet delivered
-    public void testMixedOrderClustering1() throws Throwable
+    @Test
+    public void testINWithDuplicateValue() throws Throwable
     {
-        createTableForMixedOrderClusteringTest("DESC", "ASC", "DESC", "ASC");
+        for (String compactOption : new String[] { "", " WITH COMPACT STORAGE" })
+        {
+            createTable("CREATE TABLE %s (k1 int, k2 int, v int, PRIMARY KEY (k1, k2))" + compactOption);
+            execute("INSERT INTO %s (k1,  k2, v) VALUES (?, ?, ?)", 1, 1, 1);
 
-        assertRows(execute("SELECT * FROM %s WHERE a=0 AND (b, c, d, e) > (0, 1, 1, 0)"),
-                   row(0, 2, 0, 0, 0),
-                   row(0, 1, 0, 0, 0),
-                   row(0, 0, 1, 2, -1),
-                   row(0, 0, 1, 1, 1),
-                   row(0, 0, 2, 1, -3),
-                   row(0, 0, 2, 0, 3));
+            assertRows(execute("SELECT * FROM %s WHERE k1 IN (?, ?) AND (k2) IN ((?), (?))", 1, 1, 1, 2),
+                       row(1, 1, 1));
+            assertRows(execute("SELECT * FROM %s WHERE k1 = ? AND (k2) IN ((?), (?))", 1, 1, 1),
+                       row(1, 1, 1));
+        }
     }
 
-    @Ignore // CASSANDRA-7281 not yet delivered
-    public void testMixedOrderClustering2() throws Throwable
+    @Test
+    public void testWithUnsetValues() throws Throwable
     {
-        createTableForMixedOrderClusteringTest("DESC", "DESC", "DESC", "ASC");
+        createTable("CREATE TABLE %s (k int, i int, j int, s text, PRIMARY KEY(k,i,j))");
+        createIndex("CREATE INDEX s_index ON %s (s)");
 
-        assertRows(execute("SELECT * FROM %s WHERE a=0 AND (b, c, d, e) > (0, 1, 1, 0)"),
-                   row(0, 2, 0, 0, 0),
-                   row(0, 1, 0, 0, 0),
-                   row(0, 0, 2, 1, -3),
-                   row(0, 0, 2, 0, 3),
-                   row(0, 0, 1, 2, -1),
-                   row(0, 0, 1, 1, 1));
+        assertInvalidMessage("Invalid unset value for tuple field number 0",
+                             "SELECT * from %s WHERE (i, j) = (?,?) ALLOW FILTERING", unset(), 1);
+        assertInvalidMessage("Invalid unset value for tuple field number 0",
+                             "SELECT * from %s WHERE (i, j) IN ((?,?)) ALLOW FILTERING", unset(), 1);
+        assertInvalidMessage("Invalid unset value for tuple field number 1",
+                             "SELECT * from %s WHERE (i, j) > (1,?) ALLOW FILTERING", unset());
+        assertInvalidMessage("Invalid unset value for tuple (i,j)",
+                             "SELECT * from %s WHERE (i, j) = ? ALLOW FILTERING", unset());
+        assertInvalidMessage("Invalid unset value for tuple (j)",
+                             "SELECT * from %s WHERE i = ? AND (j) > ? ALLOW FILTERING", 1, unset());
+        assertInvalidMessage("Invalid unset value for tuple (i,j)",
+                             "SELECT * from %s WHERE (i, j) IN (?, ?) ALLOW FILTERING", unset(), tuple(1, 1));
+        assertInvalidMessage("Invalid unset value for in(i,j)",
+                             "SELECT * from %s WHERE (i, j) IN ? ALLOW FILTERING", unset());
     }
 
-    @Ignore // CASSANDRA-7281 not yet delivered
-    public void testMixedOrderClustering3() throws Throwable
+    @Test
+    public void testMixedOrderColumns1() throws Throwable
     {
-        createTableForMixedOrderClusteringTest("ASC", "DESC", "DESC", "ASC");
+        for (String compactOption : new String[]{"", " COMPACT STORAGE AND "})
+        {
+            createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, PRIMARY KEY (a, b, c, d, e)) WITH " +
+                        compactOption +
+                        " CLUSTERING ORDER BY (b DESC, c ASC, d DESC, e ASC)");
 
-        assertRows(execute("SELECT * FROM %s WHERE a=0 AND (b, c, d, e) > (0, 1, 1, 0)"),
-                   row(0, 0, 2, 1, -3),
-                   row(0, 0, 2, 0, 3),
-                   row(0, 0, 1, 2, -1),
-                   row(0, 0, 1, 1, 1),
-                   row(0, 1, 0, 0, 0),
-                   row(0, 2, 0, 0, 0));
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 2, 0, -1, 0);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 2, 0, -1, 1);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 2, 0, 1, 1);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, -1, 0, 0);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, -1, 1, 1);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, -1, 1, 0);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 0, 1, -1);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 0, 1, 1);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 0, 0, -1);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 0, 0, 0);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 0, 0, 1);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 0, -1, -1);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 1, 0, -1);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 1, 0, 0);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 1, 0, -1);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 1, 0, 0);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 1, 0, 1);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 1, -1, 0);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 0, 0, 0, 0);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, -1, 0, -1, 0);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, -1, 0, 0, 0);
+            assertRows(execute(
+            "SELECT * FROM %s" +
+            " WHERE a = ? " +
+            "AND (b,c,d,e)<=(?,?,?,?) " +
+            "AND (b)>(?)", 0, 2, 0, 1, 1, -1),
+
+                       row(0, 2, 0, 1, 1),
+                       row(0, 2, 0, -1, 0),
+                       row(0, 2, 0, -1, 1),
+                       row(0, 1, -1, 1, 0),
+                       row(0, 1, -1, 1, 1),
+                       row(0, 1, -1, 0, 0),
+                       row(0, 1, 0, 1, -1),
+                       row(0, 1, 0, 1, 1),
+                       row(0, 1, 0, 0, -1),
+                       row(0, 1, 0, 0, 0),
+                       row(0, 1, 0, 0, 1),
+                       row(0, 1, 0, -1, -1),
+                       row(0, 1, 1, 0, -1),
+                       row(0, 1, 1, 0, 0),
+                       row(0, 1, 1, 0, 1),
+                       row(0, 1, 1, -1, 0),
+                       row(0, 0, 0, 0, 0)
+            );
+
+
+            assertRows(execute(
+            "SELECT * FROM %s" +
+            " WHERE a = ? " +
+            "AND (b,c,d,e)<=(?,?,?,?) " +
+            "AND (b)>=(?)", 0, 2, 0, 1, 1, -1),
+
+                       row(0, 2, 0, 1, 1),
+                       row(0, 2, 0, -1, 0),
+                       row(0, 2, 0, -1, 1),
+                       row(0, 1, -1, 1, 0),
+                       row(0, 1, -1, 1, 1),
+                       row(0, 1, -1, 0, 0),
+                       row(0, 1, 0, 1, -1),
+                       row(0, 1, 0, 1, 1),
+                       row(0, 1, 0, 0, -1),
+                       row(0, 1, 0, 0, 0),
+                       row(0, 1, 0, 0, 1),
+                       row(0, 1, 0, -1, -1),
+                       row(0, 1, 1, 0, -1),
+                       row(0, 1, 1, 0, 0),
+                       row(0, 1, 1, 0, 1),
+                       row(0, 1, 1, -1, 0),
+                       row(0, 0, 0, 0, 0),
+                       row(0, -1, 0, 0, 0),
+                       row(0, -1, 0, -1, 0)
+            );
+
+            assertRows(execute(
+            "SELECT * FROM %s" +
+            " WHERE a = ? " +
+            "AND (b,c,d)>=(?,?,?)" +
+            "AND (b,c,d,e)<(?,?,?,?) ", 0, 1, 1, 0, 1, 1, 0, 1),
+                       row(0, 1, 1, 0, -1),
+                       row(0, 1, 1, 0, 0)
+
+            );
+
+            assertRows(execute(
+            "SELECT * FROM %s" +
+            " WHERE a = ? " +
+            "AND (b,c,d,e)>(?,?,?,?)" +
+            "AND (b,c,d)<=(?,?,?) ", 0, -1, 0, -1, -1, 2, 0, -1),
+
+                       row(0, 2, 0, -1, 0),
+                       row(0, 2, 0, -1, 1),
+                       row(0, 1, -1, 1, 0),
+                       row(0, 1, -1, 1, 1),
+                       row(0, 1, -1, 0, 0),
+                       row(0, 1, 0, 1, -1),
+                       row(0, 1, 0, 1, 1),
+                       row(0, 1, 0, 0, -1),
+                       row(0, 1, 0, 0, 0),
+                       row(0, 1, 0, 0, 1),
+                       row(0, 1, 0, -1, -1),
+                       row(0, 1, 1, 0, -1),
+                       row(0, 1, 1, 0, 0),
+                       row(0, 1, 1, 0, 1),
+                       row(0, 1, 1, -1, 0),
+                       row(0, 0, 0, 0, 0),
+                       row(0, -1, 0, 0, 0),
+                       row(0, -1, 0, -1, 0)
+            );
+
+            assertRows(execute(
+            "SELECT * FROM %s" +
+            " WHERE a = ? " +
+            "AND (b,c,d,e) < (?,?,?,?) " +
+            "AND (b,c,d,e)>(?,?,?,?)", 0, 1, 0, 0, 0, 1, 0, -1, -1),
+                       row(0, 1, 0, 0, -1)
+            );
+
+            assertRows(execute(
+            "SELECT * FROM %s" +
+            " WHERE a = ? " +
+            "AND (b,c,d,e) <= (?,?,?,?) " +
+            "AND (b,c,d,e)>(?,?,?,?)", 0, 1, 0, 0, 0, 1, 0, -1, -1),
+                       row(0, 1, 0, 0, -1),
+                       row(0, 1, 0, 0, 0)
+            );
+
+            assertRows(execute(
+            "SELECT * FROM %s" +
+            " WHERE a = ? " +
+            "AND (b)<(?) " +
+            "AND (b,c,d,e)>(?,?,?,?)", 0, 2, -1, 0, -1, -1),
+
+                       row(0, 1, -1, 1, 0),
+                       row(0, 1, -1, 1, 1),
+                       row(0, 1, -1, 0, 0),
+                       row(0, 1, 0, 1, -1),
+                       row(0, 1, 0, 1, 1),
+                       row(0, 1, 0, 0, -1),
+                       row(0, 1, 0, 0, 0),
+                       row(0, 1, 0, 0, 1),
+                       row(0, 1, 0, -1, -1),
+                       row(0, 1, 1, 0, -1),
+                       row(0, 1, 1, 0, 0),
+                       row(0, 1, 1, 0, 1),
+                       row(0, 1, 1, -1, 0),
+                       row(0, 0, 0, 0, 0),
+                       row(0, -1, 0, 0, 0),
+                       row(0, -1, 0, -1, 0)
+
+            );
+
+
+            assertRows(execute(
+            "SELECT * FROM %s" +
+            " WHERE a = ? " +
+            "AND (b)<(?) " +
+            "AND (b)>(?)", 0, 2, -1),
+
+                       row(0, 1, -1, 1, 0),
+                       row(0, 1, -1, 1, 1),
+                       row(0, 1, -1, 0, 0),
+                       row(0, 1, 0, 1, -1),
+                       row(0, 1, 0, 1, 1),
+                       row(0, 1, 0, 0, -1),
+                       row(0, 1, 0, 0, 0),
+                       row(0, 1, 0, 0, 1),
+                       row(0, 1, 0, -1, -1),
+                       row(0, 1, 1, 0, -1),
+                       row(0, 1, 1, 0, 0),
+                       row(0, 1, 1, 0, 1),
+                       row(0, 1, 1, -1, 0),
+                       row(0, 0, 0, 0, 0)
+
+            );
+
+            assertRows(execute(
+            "SELECT * FROM %s" +
+            " WHERE a = ? " +
+            "AND (b)<(?) " +
+            "AND (b)>=(?)", 0, 2, -1),
+
+                       row(0, 1, -1, 1, 0),
+                       row(0, 1, -1, 1, 1),
+                       row(0, 1, -1, 0, 0),
+                       row(0, 1, 0, 1, -1),
+                       row(0, 1, 0, 1, 1),
+                       row(0, 1, 0, 0, -1),
+                       row(0, 1, 0, 0, 0),
+                       row(0, 1, 0, 0, 1),
+                       row(0, 1, 0, -1, -1),
+                       row(0, 1, 1, 0, -1),
+                       row(0, 1, 1, 0, 0),
+                       row(0, 1, 1, 0, 1),
+                       row(0, 1, 1, -1, 0),
+                       row(0, 0, 0, 0, 0),
+                       row(0, -1, 0, 0, 0),
+                       row(0, -1, 0, -1, 0)
+            );
+
+            assertRows(execute(
+            "SELECT * FROM %s" +
+            " WHERE a = ? " +
+            "AND (b,c,d,e)<=(?,?,?,?) " +
+            "AND (b,c,d,e)>(?,?,?,?)", 0, 2, 0, 1, 1, -1, 0, -1, -1),
+
+                       row(0, 2, 0, 1, 1),
+                       row(0, 2, 0, -1, 0),
+                       row(0, 2, 0, -1, 1),
+                       row(0, 1, -1, 1, 0),
+                       row(0, 1, -1, 1, 1),
+                       row(0, 1, -1, 0, 0),
+                       row(0, 1, 0, 1, -1),
+                       row(0, 1, 0, 1, 1),
+                       row(0, 1, 0, 0, -1),
+                       row(0, 1, 0, 0, 0),
+                       row(0, 1, 0, 0, 1),
+                       row(0, 1, 0, -1, -1),
+                       row(0, 1, 1, 0, -1),
+                       row(0, 1, 1, 0, 0),
+                       row(0, 1, 1, 0, 1),
+                       row(0, 1, 1, -1, 0),
+                       row(0, 0, 0, 0, 0),
+                       row(0, -1, 0, 0, 0),
+                       row(0, -1, 0, -1, 0)
+            );
+
+            assertRows(execute(
+            "SELECT * FROM %s" +
+            " WHERE a = ? " +
+            "AND (b,c)<=(?,?) " +
+            "AND (b,c,d,e)>(?,?,?,?)", 0, 2, 0, -1, 0, -1, -1),
+
+                       row(0, 2, 0, 1, 1),
+                       row(0, 2, 0, -1, 0),
+                       row(0, 2, 0, -1, 1),
+                       row(0, 1, -1, 1, 0),
+                       row(0, 1, -1, 1, 1),
+                       row(0, 1, -1, 0, 0),
+                       row(0, 1, 0, 1, -1),
+                       row(0, 1, 0, 1, 1),
+                       row(0, 1, 0, 0, -1),
+                       row(0, 1, 0, 0, 0),
+                       row(0, 1, 0, 0, 1),
+                       row(0, 1, 0, -1, -1),
+                       row(0, 1, 1, 0, -1),
+                       row(0, 1, 1, 0, 0),
+                       row(0, 1, 1, 0, 1),
+                       row(0, 1, 1, -1, 0),
+                       row(0, 0, 0, 0, 0),
+                       row(0, -1, 0, 0, 0),
+                       row(0, -1, 0, -1, 0)
+            );
+
+            assertRows(execute(
+            "SELECT * FROM %s" +
+            " WHERE a = ? " +
+            "AND (b,c,d)<=(?,?,?) " +
+            "AND (b,c,d,e)>(?,?,?,?)", 0, 2, 0, -1, -1, 0, -1, -1),
+
+                       row(0, 2, 0, -1, 0),
+                       row(0, 2, 0, -1, 1),
+                       row(0, 1, -1, 1, 0),
+                       row(0, 1, -1, 1, 1),
+                       row(0, 1, -1, 0, 0),
+                       row(0, 1, 0, 1, -1),
+                       row(0, 1, 0, 1, 1),
+                       row(0, 1, 0, 0, -1),
+                       row(0, 1, 0, 0, 0),
+                       row(0, 1, 0, 0, 1),
+                       row(0, 1, 0, -1, -1),
+                       row(0, 1, 1, 0, -1),
+                       row(0, 1, 1, 0, 0),
+                       row(0, 1, 1, 0, 1),
+                       row(0, 1, 1, -1, 0),
+                       row(0, 0, 0, 0, 0),
+                       row(0, -1, 0, 0, 0),
+                       row(0, -1, 0, -1, 0)
+            );
+
+            assertRows(execute(
+            "SELECT * FROM %s" +
+            " WHERE a = ? " +
+            "AND (b,c,d,e)>(?,?,?,?)" +
+            "AND (b,c,d)<=(?,?,?) ", 0, -1, 0, -1, -1, 2, 0, -1),
+
+                       row(0, 2, 0, -1, 0),
+                       row(0, 2, 0, -1, 1),
+                       row(0, 1, -1, 1, 0),
+                       row(0, 1, -1, 1, 1),
+                       row(0, 1, -1, 0, 0),
+                       row(0, 1, 0, 1, -1),
+                       row(0, 1, 0, 1, 1),
+                       row(0, 1, 0, 0, -1),
+                       row(0, 1, 0, 0, 0),
+                       row(0, 1, 0, 0, 1),
+                       row(0, 1, 0, -1, -1),
+                       row(0, 1, 1, 0, -1),
+                       row(0, 1, 1, 0, 0),
+                       row(0, 1, 1, 0, 1),
+                       row(0, 1, 1, -1, 0),
+                       row(0, 0, 0, 0, 0),
+                       row(0, -1, 0, 0, 0),
+                       row(0, -1, 0, -1, 0)
+            );
+
+            assertRows(execute(
+            "SELECT * FROM %s" +
+            " WHERE a = ? " +
+            "AND (b,c,d)>=(?,?,?)" +
+            "AND (b,c,d,e)<(?,?,?,?) ", 0, 1, 1, 0, 1, 1, 0, 1),
+                       row(0, 1, 1, 0, -1),
+                       row(0, 1, 1, 0, 0)
+            );
+            assertRows(execute(
+            "SELECT * FROM %s" +
+            " WHERE a = ? " +
+            "AND (b,c,d,e)<(?,?,?,?) " +
+            "AND (b,c,d)>=(?,?,?)", 0, 1, 1, 0, 1, 1, 1, 0),
+                       row(0, 1, 1, 0, -1),
+                       row(0, 1, 1, 0, 0)
+
+            );
+
+            assertRows(execute(
+            "SELECT * FROM %s" +
+            " WHERE a = ? " +
+            "AND (b,c)<(?,?) " +
+            "AND (b,c,d,e)>(?,?,?,?)", 0, 2, 0, -1, 0, -1, -1),
+                       row(0, 1, -1, 1, 0),
+                       row(0, 1, -1, 1, 1),
+                       row(0, 1, -1, 0, 0),
+                       row(0, 1, 0, 1, -1),
+                       row(0, 1, 0, 1, 1),
+                       row(0, 1, 0, 0, -1),
+                       row(0, 1, 0, 0, 0),
+                       row(0, 1, 0, 0, 1),
+                       row(0, 1, 0, -1, -1),
+                       row(0, 1, 1, 0, -1),
+                       row(0, 1, 1, 0, 0),
+                       row(0, 1, 1, 0, 1),
+                       row(0, 1, 1, -1, 0),
+                       row(0, 0, 0, 0, 0),
+                       row(0, -1, 0, 0, 0),
+                       row(0, -1, 0, -1, 0)
+            );
+
+            assertRows(execute(
+            "SELECT * FROM %s" +
+            " WHERE a = ? " +
+            "AND (b,c)<(?,?) " +
+            "AND (b,c,d,e)>(?,?,?,?)", 0, 2, 0, -1, 0, -1, -1),
+                       row(0, 1, -1, 1, 0),
+                       row(0, 1, -1, 1, 1),
+                       row(0, 1, -1, 0, 0),
+                       row(0, 1, 0, 1, -1),
+                       row(0, 1, 0, 1, 1),
+                       row(0, 1, 0, 0, -1),
+                       row(0, 1, 0, 0, 0),
+                       row(0, 1, 0, 0, 1),
+                       row(0, 1, 0, -1, -1),
+                       row(0, 1, 1, 0, -1),
+                       row(0, 1, 1, 0, 0),
+                       row(0, 1, 1, 0, 1),
+                       row(0, 1, 1, -1, 0),
+                       row(0, 0, 0, 0, 0),
+                       row(0, -1, 0, 0, 0),
+                       row(0, -1, 0, -1, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b,c,d,e) <= (?,?,?,?)", 0, 1, 0, 0, 0),
+                       row(0, 1, -1, 1, 0),
+                       row(0, 1, -1, 1, 1),
+                       row(0, 1, -1, 0, 0),
+                       row(0, 1, 0, 0, -1),
+                       row(0, 1, 0, 0, 0),
+                       row(0, 1, 0, -1, -1),
+                       row(0, 0, 0, 0, 0),
+                       row(0, -1, 0, 0, 0),
+                       row(0, -1, 0, -1, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b,c,d,e) > (?,?,?,?)", 0, 1, 0, 0, 0),
+                       row(0, 2, 0, 1, 1),
+                       row(0, 2, 0, -1, 0),
+                       row(0, 2, 0, -1, 1),
+                       row(0, 1, 0, 1, -1),
+                       row(0, 1, 0, 1, 1),
+                       row(0, 1, 0, 0, 1),
+                       row(0, 1, 1, 0, -1),
+                       row(0, 1, 1, 0, 0),
+                       row(0, 1, 1, 0, 1),
+                       row(0, 1, 1, -1, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b,c,d,e) >= (?,?,?,?)", 0, 1, 0, 0, 0),
+                       row(0, 2, 0, 1, 1),
+                       row(0, 2, 0, -1, 0),
+                       row(0, 2, 0, -1, 1),
+                       row(0, 1, 0, 1, -1),
+                       row(0, 1, 0, 1, 1),
+                       row(0, 1, 0, 0, 0),
+                       row(0, 1, 0, 0, 1),
+                       row(0, 1, 1, 0, -1),
+                       row(0, 1, 1, 0, 0),
+                       row(0, 1, 1, 0, 1),
+                       row(0, 1, 1, -1, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b,c,d) >= (?,?,?)", 0, 1, 0, 0),
+                       row(0, 2, 0, 1, 1),
+                       row(0, 2, 0, -1, 0),
+                       row(0, 2, 0, -1, 1),
+                       row(0, 1, 0, 1, -1),
+                       row(0, 1, 0, 1, 1),
+                       row(0, 1, 0, 0, -1),
+                       row(0, 1, 0, 0, 0),
+                       row(0, 1, 0, 0, 1),
+                       row(0, 1, 1, 0, -1),
+                       row(0, 1, 1, 0, 0),
+                       row(0, 1, 1, 0, 1),
+                       row(0, 1, 1, -1, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b,c,d) > (?,?,?)", 0, 1, 0, 0),
+                       row(0, 2, 0, 1, 1),
+                       row(0, 2, 0, -1, 0),
+                       row(0, 2, 0, -1, 1),
+                       row(0, 1, 0, 1, -1),
+                       row(0, 1, 0, 1, 1),
+                       row(0, 1, 1, 0, -1),
+                       row(0, 1, 1, 0, 0),
+                       row(0, 1, 1, 0, 1),
+                       row(0, 1, 1, -1, 0)
+            );
+        }
     }
 
-    @Ignore // CASSANDRA-7281 not yet delivered
-    public void testMixedOrderClustering4() throws Throwable
+    @Test
+    public void testMixedOrderColumns2() throws Throwable
     {
-        createTableForMixedOrderClusteringTest("DESC", "ASC", "ASC", "DESC");
+        for (String compactOption : new String[]{"", " COMPACT STORAGE AND "})
+        {
+            createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, PRIMARY KEY (a, b, c, d, e)) WITH " +
+                        compactOption +
+                        "CLUSTERING ORDER BY (b DESC, c ASC, d ASC, e ASC)");
 
-        assertRows(execute("SELECT * FROM %s WHERE a=0 AND (b, c, d, e) > (0, 1, 1, 0)"),
-                   row(0, 2, 0, 0, 0),
-                   row(0, 1, 0, 0, 0),
-                   row(0, 0, 1, 1, 1),
-                   row(0, 0, 1, 2, -1),
-                   row(0, 0, 2, 0, 3),
-                   row(0, 0, 2, 1, -3));
+            // b and d are reversed in the clustering order
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 2, 0, -1, 0);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 2, 0, -1, 1);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, -1, 0, 0);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, -1, 1, 0);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, -1, 1, 1);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 0, 1, -1);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 0, 1, 1);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 0, 0, -1);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 0, 0, 0);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 0, 0, 1);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 0, -1, -1);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 1, 0, -1);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 1, 0, 0);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 1, 0, -1);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 1, 0, 0);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 1, 0, 1);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 1, -1, 0);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 0, 0, 0, 0);
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b,c,d,e) <= (?,?,?,?)", 0, 1, 0, 0, 0),
+                       row(0, 1, -1, 0, 0),
+                       row(0, 1, -1, 1, 0),
+                       row(0, 1, -1, 1, 1),
+                       row(0, 1, 0, -1, -1),
+                       row(0, 1, 0, 0, -1),
+                       row(0, 1, 0, 0, 0),
+                       row(0, 0, 0, 0, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b,c,d,e) > (?,?,?,?)", 0, 1, 0, 0, 0),
+                       row(0, 2, 0, -1, 0),
+                       row(0, 2, 0, -1, 1),
+                       row(0, 1, 0, 0, 1),
+                       row(0, 1, 0, 1, -1),
+                       row(0, 1, 0, 1, 1),
+                       row(0, 1, 1, -1, 0),
+                       row(0, 1, 1, 0, -1),
+                       row(0, 1, 1, 0, 0),
+                       row(0, 1, 1, 0, 1)
+            );
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b,c,d,e) >= (?,?,?,?)", 0, 1, 0, 0, 0),
+                       row(0, 2, 0, -1, 0),
+                       row(0, 2, 0, -1, 1),
+                       row(0, 1, 0, 0, 0),
+                       row(0, 1, 0, 0, 1),
+                       row(0, 1, 0, 1, -1),
+                       row(0, 1, 0, 1, 1),
+                       row(0, 1, 1, -1, 0),
+                       row(0, 1, 1, 0, -1),
+                       row(0, 1, 1, 0, 0),
+                       row(0, 1, 1, 0, 1)
+            );
+        }
     }
 
-    @Ignore // CASSANDRA-7281 not yet delivered
-    public void testMixedOrderClustering5() throws Throwable
+    @Test
+    public void testMixedOrderColumns3() throws Throwable
     {
-        createTableForMixedOrderClusteringTest("DESC", "DESC", "DESC", "DESC");
+        for (String compactOption : new String[]{"", " COMPACT STORAGE AND "})
+        {
+            createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b, c)) WITH " +
+                        compactOption +
+                        "CLUSTERING ORDER BY (b DESC, c ASC)");
 
-        assertRows(execute("SELECT * FROM %s WHERE a=0 AND (b, c, d, e) > (0, 1, 1, 0)"),
-                   row(0, 2, 0, 0, 0),
-                   row(0, 1, 0, 0, 0),
-                   row(0, 0, 2, 1, -3),
-                   row(0, 0, 2, 0, 3),
-                   row(0, 0, 1, 2, -1),
-                   row(0, 0, 1, 1, 1));
+            execute("INSERT INTO %s (a, b, c) VALUES (?,?,?);", 0, 2, 3);
+            execute("INSERT INTO %s (a, b, c) VALUES (?,?,?);", 0, 2, 4);
+            execute("INSERT INTO %s (a, b, c) VALUES (?,?,?);", 0, 4, 4);
+            execute("INSERT INTO %s (a, b, c) VALUES (?,?,?);", 0, 3, 4);
+            execute("INSERT INTO %s (a, b, c) VALUES (?,?,?);", 0, 4, 5);
+            execute("INSERT INTO %s (a, b, c) VALUES (?,?,?);", 0, 4, 6);
+
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b,c)>=(?,?) AND (b,c)<(?,?) ALLOW FILTERING", 0, 2, 3, 4, 5),
+                       row(0, 4, 4), row(0, 3, 4), row(0, 2, 3), row(0, 2, 4)
+            );
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b,c)>=(?,?) AND (b,c)<=(?,?) ALLOW FILTERING", 0, 2, 3, 4, 5),
+                       row(0, 4, 4), row(0, 4, 5), row(0, 3, 4), row(0, 2, 3), row(0, 2, 4)
+            );
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b,c)<(?,?) ALLOW FILTERING", 0, 4, 5),
+                       row(0, 4, 4), row(0, 3, 4), row(0, 2, 3), row(0, 2, 4)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b,c)>(?,?) ALLOW FILTERING", 0, 4, 5),
+                       row(0, 4, 6)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b)<(?) and (b)>(?) ALLOW FILTERING", 0, 4, 2),
+                       row(0, 3, 4)
+            );
+        }
     }
 
-    @Ignore // CASSANDRA-7281 not yet delivered
-    public void testMixedOrderClustering6() throws Throwable
+    @Test
+    public void testMixedOrderColumns4() throws Throwable
     {
-        createTableForMixedOrderClusteringTest("ASC", "ASC", "ASC", "ASC");
+        for (String compactOption : new String[]{"", " COMPACT STORAGE AND "})
+        {
+            createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, PRIMARY KEY (a, b, c, d, e)) WITH " +
+                        compactOption +
+                        "CLUSTERING ORDER BY (b ASC, c DESC, d DESC, e ASC)");
 
-        assertRows(execute("SELECT * FROM %s WHERE a=0 AND (b, c, d, e) > (0, 1, 1, 0)"),
-                   row(0, 0, 1, 1, 1),
-                   row(0, 0, 1, 2, -1),
-                   row(0, 0, 2, 0, 3),
-                   row(0, 0, 2, 1, -3),
-                   row(0, 1, 0, 0, 0),
-                   row(0, 2, 0, 0, 0));
-    }
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 2, 0, -1, 0);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 2, 0, -1, 1);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 2, 0, 1, 1);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 2, -1, 1, 1);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 2, -3, 1, 1);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, -1, 0, 0);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, -1, 1, 1);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, -1, 1, 0);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 0, 1, -1);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 0, 1, 1);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 0, 0, -1);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 0, 0, 0);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 0, 0, 1);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 0, -1, -1);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 1, 0, -1);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 1, 0, 0);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 1, 0, -1);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 1, 0, 0);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 1, 0, 1);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 1, 1, -1, 0);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, 0, 0, 0, 0);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, -1, 0, -1, 0);
+            execute("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?)", 0, -1, 0, 0, 0);
 
-    @Ignore // CASSANDRA-7281 not yet delivered
-    public void testMixedOrderClustering7() throws Throwable
-    {
-        createTableForMixedOrderClusteringTest("DESC", "ASC", "DESC", "ASC");
+            assertRows(execute(
+            "SELECT * FROM %s" +
+            " WHERE a = ? " +
+            "AND (b,c,d,e)<(?,?,?,?) " +
+            "AND (b,c,d,e)>(?,?,?,?)", 0, 2, 0, 1, 1, -1, 0, -1, -1),
 
-        assertRows(execute("SELECT * FROM %s WHERE a=0 AND (b, c, d, e) <= (0, 1, 1, 0)"),
-                   row(0, 0, 0, 0, 0),
-                   row(0, 0, 1, 1, -1),
-                   row(0, 0, 1, 1, 0),
-                   row(0, 0, 1, 0, 2),
-                   row(0, -1, 2, 2, 2));
-    }
+                       row(0, -1, 0, 0, 0),
+                       row(0, -1, 0, -1, 0),
+                       row(0, 0, 0, 0, 0),
+                       row(0, 1, 1, 0, -1),
+                       row(0, 1, 1, 0, 0),
+                       row(0, 1, 1, 0, 1),
+                       row(0, 1, 1, -1, 0),
+                       row(0, 1, 0, 1, -1),
+                       row(0, 1, 0, 1, 1),
+                       row(0, 1, 0, 0, -1),
+                       row(0, 1, 0, 0, 0),
+                       row(0, 1, 0, 0, 1),
+                       row(0, 1, 0, -1, -1),
+                       row(0, 1, -1, 1, 0),
+                       row(0, 1, -1, 1, 1),
+                       row(0, 1, -1, 0, 0),
+                       row(0, 2, 0, -1, 0),
+                       row(0, 2, 0, -1, 1),
+                       row(0, 2, -1, 1, 1),
+                       row(0, 2, -3, 1, 1)
 
-    @Ignore // CASSANDRA-7281 not yet delivered
-    public void testMixedOrderClustering8() throws Throwable
-    {
-        createTableForMixedOrderClusteringTest("ASC", "DESC", "DESC", "ASC");
+            );
 
-        assertRows(execute("SELECT * FROM %s WHERE a=0 AND (b, c, d, e) <= (0, 1, 1, 0)"),
-                   row(0, -1, 2, 2, 2),
-                   row(0, 0, 1, 1, -1),
-                   row(0, 0, 1, 1, 0),
-                   row(0, 0, 1, 0, 2),
-                   row(0, 0, 0, 0, 0));
-    }
 
-    @Ignore // CASSANDRA-7281 not yet delivered
-    public void testMixedOrderClustering9() throws Throwable
-    {
-        createTableForMixedOrderClusteringTest("DESC", "ASC", "DESC", "DESC");
+            assertRows(execute(
+            "SELECT * FROM %s" +
+            " WHERE a = ? " +
+            "AND (b,c,d,e) < (?,?,?,?) " +
+            "AND (b,c,d,e)>(?,?,?,?)", 0, 1, 0, 0, 0, 1, 0, -1, -1),
+                       row(0, 1, 0, 0, -1)
+            );
 
-        assertRows(execute("SELECT * FROM %s WHERE a=0 AND (b, c, d, e) <= (0, 1, 1, 0)"),
-                   row(0, 0, 0, 0, 0),
-                   row(0, 0, 1, 1, 0),
-                   row(0, 0, 1, 1, -1),
-                   row(0, 0, 1, 0, 2),
-                   row(0, -1, 2, 2, 2));
-    }
+            assertRows(execute(
+            "SELECT * FROM %s" +
+            " WHERE a = ? " +
+            "AND (b,c,d,e) <= (?,?,?,?) " +
+            "AND (b,c,d,e)>(?,?,?,?)", 0, 1, 0, 0, 0, 1, 0, -1, -1),
+                       row(0, 1, 0, 0, -1),
+                       row(0, 1, 0, 0, 0)
+            );
 
-    private void createTableForMixedOrderClusteringTest(String ... formats) throws Throwable
-    {
-        assertEquals(4, formats.length);
 
-        String clustering = String.format("WITH CLUSTERING ORDER BY (b %s, c %s, d %s, e %s)", (Object[])formats);
-        createTable("CREATE TABLE %s (a int, b int, c int, d int , e int, PRIMARY KEY (a, b, c, d, e) ) " + clustering);
+            assertRows(execute(
+            "SELECT * FROM %s" +
+            " WHERE a = ? " +
+            "AND (b,c,d,e)<=(?,?,?,?) " +
+            "AND (b,c,d,e)>(?,?,?,?)", 0, 2, 0, 1, 1, -1, 0, -1, -1),
 
-        execute("INSERT INTO %s (a, b, c, d, e) VALUES (0, 2, 0, 0, 0)");
-        execute("INSERT INTO %s (a, b, c, d, e) VALUES (0, 1, 0, 0, 0)");
-        execute("INSERT INTO %s (a, b, c, d, e) VALUES (0, 0, 0, 0, 0)");
-        execute("INSERT INTO %s (a, b, c, d, e) VALUES (0, 0, 1, 2, -1)");
-        execute("INSERT INTO %s (a, b, c, d, e) VALUES (0, 0, 1, 1, -1)");
-        execute("INSERT INTO %s (a, b, c, d, e) VALUES (0, 0, 1, 1, 0)");
-        execute("INSERT INTO %s (a, b, c, d, e) VALUES (0, 0, 1, 1, 1)");
-        execute("INSERT INTO %s (a, b, c, d, e) VALUES (0, 0, 1, 0, 2)");
-        execute("INSERT INTO %s (a, b, c, d, e) VALUES (0, 0, 2, 1, -3)");
-        execute("INSERT INTO %s (a, b, c, d, e) VALUES (0, 0, 2, 0, 3)");
-        execute("INSERT INTO %s (a, b, c, d, e) VALUES (0, -1, 2, 2, 2)");
+                       row(0, -1, 0, 0, 0),
+                       row(0, -1, 0, -1, 0),
+                       row(0, 0, 0, 0, 0),
+                       row(0, 1, 1, 0, -1),
+                       row(0, 1, 1, 0, 0),
+                       row(0, 1, 1, 0, 1),
+                       row(0, 1, 1, -1, 0),
+                       row(0, 1, 0, 1, -1),
+                       row(0, 1, 0, 1, 1),
+                       row(0, 1, 0, 0, -1),
+                       row(0, 1, 0, 0, 0),
+                       row(0, 1, 0, 0, 1),
+                       row(0, 1, 0, -1, -1),
+                       row(0, 1, -1, 1, 0),
+                       row(0, 1, -1, 1, 1),
+                       row(0, 1, -1, 0, 0),
+                       row(0, 2, 0, 1, 1),
+                       row(0, 2, 0, -1, 0),
+                       row(0, 2, 0, -1, 1),
+                       row(0, 2, -1, 1, 1),
+                       row(0, 2, -3, 1, 1)
+            );
+
+            assertRows(execute(
+            "SELECT * FROM %s" +
+            " WHERE a = ? " +
+            "AND (b,c)<=(?,?) " +
+            "AND (b,c,d,e)>(?,?,?,?)", 0, 2, 0, -1, 0, -1, -1),
+
+                       row(0, -1, 0, 0, 0),
+                       row(0, -1, 0, -1, 0),
+                       row(0, 0, 0, 0, 0),
+                       row(0, 1, 1, 0, -1),
+                       row(0, 1, 1, 0, 0),
+                       row(0, 1, 1, 0, 1),
+                       row(0, 1, 1, -1, 0),
+                       row(0, 1, 0, 1, -1),
+                       row(0, 1, 0, 1, 1),
+                       row(0, 1, 0, 0, -1),
+                       row(0, 1, 0, 0, 0),
+                       row(0, 1, 0, 0, 1),
+                       row(0, 1, 0, -1, -1),
+                       row(0, 1, -1, 1, 0),
+                       row(0, 1, -1, 1, 1),
+                       row(0, 1, -1, 0, 0),
+                       row(0, 2, 0, 1, 1),
+                       row(0, 2, 0, -1, 0),
+                       row(0, 2, 0, -1, 1),
+                       row(0, 2, -1, 1, 1),
+                       row(0, 2, -3, 1, 1)
+            );
+
+            assertRows(execute(
+            "SELECT * FROM %s" +
+            " WHERE a = ? " +
+            "AND (b,c)<(?,?) " +
+            "AND (b,c,d,e)>(?,?,?,?)", 0, 2, 0, -1, 0, -1, -1),
+                       row(0, -1, 0, 0, 0),
+                       row(0, -1, 0, -1, 0),
+                       row(0, 0, 0, 0, 0),
+                       row(0, 1, 1, 0, -1),
+                       row(0, 1, 1, 0, 0),
+                       row(0, 1, 1, 0, 1),
+                       row(0, 1, 1, -1, 0),
+                       row(0, 1, 0, 1, -1),
+                       row(0, 1, 0, 1, 1),
+                       row(0, 1, 0, 0, -1),
+                       row(0, 1, 0, 0, 0),
+                       row(0, 1, 0, 0, 1),
+                       row(0, 1, 0, -1, -1),
+                       row(0, 1, -1, 1, 0),
+                       row(0, 1, -1, 1, 1),
+                       row(0, 1, -1, 0, 0),
+                       row(0, 2, -1, 1, 1),
+                       row(0, 2, -3, 1, 1)
+            );
+
+            assertRows(execute(
+            "SELECT * FROM %s" +
+            " WHERE a = ? " +
+            "AND (b,c,d,e)<=(?,?,?,?) " +
+            "AND (b)>=(?)", 0, 2, 0, 1, 1, -1),
+
+                       row(0, -1, 0, 0, 0),
+                       row(0, -1, 0, -1, 0),
+                       row(0, 0, 0, 0, 0),
+                       row(0, 1, 1, 0, -1),
+                       row(0, 1, 1, 0, 0),
+                       row(0, 1, 1, 0, 1),
+                       row(0, 1, 1, -1, 0),
+                       row(0, 1, 0, 1, -1),
+                       row(0, 1, 0, 1, 1),
+                       row(0, 1, 0, 0, -1),
+                       row(0, 1, 0, 0, 0),
+                       row(0, 1, 0, 0, 1),
+                       row(0, 1, 0, -1, -1),
+                       row(0, 1, -1, 1, 0),
+                       row(0, 1, -1, 1, 1),
+                       row(0, 1, -1, 0, 0),
+                       row(0, 2, 0, 1, 1),
+                       row(0, 2, 0, -1, 0),
+                       row(0, 2, 0, -1, 1),
+                       row(0, 2, -1, 1, 1),
+                       row(0, 2, -3, 1, 1)
+            );
+
+            assertRows(execute(
+            "SELECT * FROM %s" +
+            " WHERE a = ? " +
+            "AND (b,c,d,e)<=(?,?,?,?) " +
+            "AND (b)>(?)", 0, 2, 0, 1, 1, -1),
+
+                       row(0, 0, 0, 0, 0),
+                       row(0, 1, 1, 0, -1),
+                       row(0, 1, 1, 0, 0),
+                       row(0, 1, 1, 0, 1),
+                       row(0, 1, 1, -1, 0),
+                       row(0, 1, 0, 1, -1),
+                       row(0, 1, 0, 1, 1),
+                       row(0, 1, 0, 0, -1),
+                       row(0, 1, 0, 0, 0),
+                       row(0, 1, 0, 0, 1),
+                       row(0, 1, 0, -1, -1),
+                       row(0, 1, -1, 1, 0),
+                       row(0, 1, -1, 1, 1),
+                       row(0, 1, -1, 0, 0),
+                       row(0, 2, 0, 1, 1),
+                       row(0, 2, 0, -1, 0),
+                       row(0, 2, 0, -1, 1),
+                       row(0, 2, -1, 1, 1),
+                       row(0, 2, -3, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b,c,d,e) <= (?,?,?,?)", 0, 1, 0, 0, 0),
+                       row(0, -1, 0, 0, 0),
+                       row(0, -1, 0, -1, 0),
+                       row(0, 0, 0, 0, 0),
+                       row(0, 1, 0, 0, -1),
+                       row(0, 1, 0, 0, 0),
+                       row(0, 1, 0, -1, -1),
+                       row(0, 1, -1, 1, 0),
+                       row(0, 1, -1, 1, 1),
+                       row(0, 1, -1, 0, 0)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b,c,d,e) > (?,?,?,?)", 0, 1, 0, 0, 0),
+                       row(0, 1, 1, 0, -1),
+                       row(0, 1, 1, 0, 0),
+                       row(0, 1, 1, 0, 1),
+                       row(0, 1, 1, -1, 0),
+                       row(0, 1, 0, 1, -1),
+                       row(0, 1, 0, 1, 1),
+                       row(0, 1, 0, 0, 1),
+                       row(0, 2, 0, 1, 1),
+                       row(0, 2, 0, -1, 0),
+                       row(0, 2, 0, -1, 1),
+                       row(0, 2, -1, 1, 1),
+                       row(0, 2, -3, 1, 1)
+
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b,c,d,e) >= (?,?,?,?)", 0, 1, 0, 0, 0),
+                       row(0, 1, 1, 0, -1),
+                       row(0, 1, 1, 0, 0),
+                       row(0, 1, 1, 0, 1),
+                       row(0, 1, 1, -1, 0),
+                       row(0, 1, 0, 1, -1),
+                       row(0, 1, 0, 1, 1),
+                       row(0, 1, 0, 0, 0),
+                       row(0, 1, 0, 0, 1),
+                       row(0, 2, 0, 1, 1),
+                       row(0, 2, 0, -1, 0),
+                       row(0, 2, 0, -1, 1),
+                       row(0, 2, -1, 1, 1),
+                       row(0, 2, -3, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b,c,d) >= (?,?,?)", 0, 1, 0, 0),
+                       row(0, 1, 1, 0, -1),
+                       row(0, 1, 1, 0, 0),
+                       row(0, 1, 1, 0, 1),
+                       row(0, 1, 1, -1, 0),
+                       row(0, 1, 0, 1, -1),
+                       row(0, 1, 0, 1, 1),
+                       row(0, 1, 0, 0, -1),
+                       row(0, 1, 0, 0, 0),
+                       row(0, 1, 0, 0, 1),
+                       row(0, 2, 0, 1, 1),
+                       row(0, 2, 0, -1, 0),
+                       row(0, 2, 0, -1, 1),
+                       row(0, 2, -1, 1, 1),
+                       row(0, 2, -3, 1, 1)
+            );
+
+            assertRows(execute("SELECT * FROM %s WHERE a = ? AND (b,c,d) > (?,?,?)", 0, 1, 0, 0),
+                       row(0, 1, 1, 0, -1),
+                       row(0, 1, 1, 0, 0),
+                       row(0, 1, 1, 0, 1),
+                       row(0, 1, 1, -1, 0),
+                       row(0, 1, 0, 1, -1),
+                       row(0, 1, 0, 1, 1),
+                       row(0, 2, 0, 1, 1),
+                       row(0, 2, 0, -1, 0),
+                       row(0, 2, 0, -1, 1),
+                       row(0, 2, -1, 1, 1),
+                       row(0, 2, -3, 1, 1)
+            );
+
+            assertRows(execute(
+            "SELECT * FROM %s" +
+            " WHERE a = ? " +
+            "AND (b) < (?) ", 0, 0),
+                       row(0, -1, 0, 0, 0), row(0, -1, 0, -1, 0)
+            );
+            assertRows(execute(
+            "SELECT * FROM %s" +
+            " WHERE a = ? " +
+            "AND (b) <= (?) ", 0, -1),
+                       row(0, -1, 0, 0, 0), row(0, -1, 0, -1, 0)
+            );
+            assertRows(execute(
+            "SELECT * FROM %s" +
+            " WHERE a = ? " +
+            "AND (b,c,d,e) < (?,?,?,?) and (b,c,d,e) > (?,?,?,?) ", 0, 2, 0, 0, 0, 2, -2, 0, 0),
+                       row(0, 2, 0, -1, 0),
+                       row(0, 2, 0, -1, 1),
+                       row(0, 2, -1, 1, 1)
+            );
+        }
     }
 
     /**
@@ -1031,4 +1907,16 @@
                    row(0, 0, 2, 2, 2),
                    row(0, 0, 3, 3, 3));
     }
+
+    @Test
+    public void testInvalidColumnNames() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c))");
+        assertInvalidMessage("Undefined name e in where clause ('(b, e) = (0, 0)')", "SELECT * FROM %s WHERE (b, e) = (0, 0)");
+        assertInvalidMessage("Undefined name e in where clause ('(b, e) IN ((0, 1), (2, 4))')", "SELECT * FROM %s WHERE (b, e) IN ((0, 1), (2, 4))");
+        assertInvalidMessage("Undefined name e in where clause ('(b, e) > (0, 1)')", "SELECT * FROM %s WHERE (b, e) > (0, 1) and b <= 2");
+        assertInvalidMessage("Aliases aren't allowed in the where clause ('(b, e) = (0, 0)')", "SELECT c AS e FROM %s WHERE (b, e) = (0, 0)");
+        assertInvalidMessage("Aliases aren't allowed in the where clause ('(b, e) IN ((0, 1), (2, 4))')", "SELECT c AS e FROM %s WHERE (b, e) IN ((0, 1), (2, 4))");
+        assertInvalidMessage("Aliases aren't allowed in the where clause ('(b, e) > (0, 1)')", "SELECT c AS e FROM %s WHERE (b, e) > (0, 1) and b <= 2");
+    }
  }
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderByTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderByTest.java
index 2cc0d6c..32d800a 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderByTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderByTest.java
@@ -22,6 +22,7 @@
 import org.apache.cassandra.cql3.CQLTester;
 
 import static org.junit.Assert.assertTrue;
+import static java.util.Arrays.asList;
 
 public class SelectOrderByTest extends CQLTester
 {
@@ -35,31 +36,37 @@
             execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 1, 1);
             execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 2, 2);
 
-            assertRows(execute("SELECT * FROM %s WHERE a=? ORDER BY b ASC", 0),
-                    row(0, 0, 0),
-                    row(0, 1, 1),
-                    row(0, 2, 2)
-            );
+            for (boolean flush : new boolean[]{true, false})
+            {
+                if (flush)
+                    flush();
 
-            assertRows(execute("SELECT * FROM %s WHERE a=? ORDER BY b DESC", 0),
-                    row(0, 2, 2),
-                    row(0, 1, 1),
-                    row(0, 0, 0)
-            );
+                assertRows(execute("SELECT * FROM %s WHERE a=? ORDER BY b ASC", 0),
+                           row(0, 0, 0),
+                           row(0, 1, 1),
+                           row(0, 2, 2)
+                        );
 
-            // order by the only column in the selection
-            assertRows(execute("SELECT b FROM %s WHERE a=? ORDER BY b ASC", 0),
-                    row(0), row(1), row(2));
+                assertRows(execute("SELECT * FROM %s WHERE a=? ORDER BY b DESC", 0),
+                           row(0, 2, 2),
+                           row(0, 1, 1),
+                           row(0, 0, 0)
+                        );
 
-            assertRows(execute("SELECT b FROM %s WHERE a=? ORDER BY b DESC", 0),
-                    row(2), row(1), row(0));
+                // order by the only column in the selection
+                assertRows(execute("SELECT b FROM %s WHERE a=? ORDER BY b ASC", 0),
+                           row(0), row(1), row(2));
 
-            // order by a column not in the selection
-            assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b ASC", 0),
-                    row(0), row(1), row(2));
+                assertRows(execute("SELECT b FROM %s WHERE a=? ORDER BY b DESC", 0),
+                           row(2), row(1), row(0));
 
-            assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b DESC", 0),
-                    row(2), row(1), row(0));
+                // order by a column not in the selection
+                assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b ASC", 0),
+                           row(0), row(1), row(2));
+
+                assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b DESC", 0),
+                           row(2), row(1), row(0));
+            }
         }
     }
 
@@ -73,22 +80,28 @@
             execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 1, 1);
             execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 2, 2);
 
-            // order by the only column in the selection
-            assertRows(execute("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b ASC", 0),
-                    row(0), row(1), row(2));
+            for (boolean flush : new boolean[]{true, false})
+            {
+                if (flush)
+                    flush();
 
-            assertRows(execute("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b DESC", 0),
-                    row(2), row(1), row(0));
+                // order by the only column in the selection
+                assertRows(execute("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b ASC", 0),
+                           row(0), row(1), row(2));
 
-            // order by a column not in the selection
-            assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b ASC", 0),
-                    row(0), row(1), row(2));
+                assertRows(execute("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b DESC", 0),
+                           row(2), row(1), row(0));
 
-            assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b DESC", 0),
-                    row(2), row(1), row(0));
+                // order by a column not in the selection
+                assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b ASC", 0),
+                           row(0), row(1), row(2));
 
-            assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY c ASC", 0);
-            assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY c DESC", 0);
+                assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b DESC", 0),
+                           row(2), row(1), row(0));
+
+                assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY c ASC", 0);
+                assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY c DESC", 0);
+            }
         }
     }
 
@@ -104,15 +117,20 @@
             execute("INSERT INTO %s (a, b, c) VALUES (?, ?, {a: ?})", 0, 1, 1);
             execute("INSERT INTO %s (a, b, c) VALUES (?, ?, {a: ?})", 0, 2, 2);
 
-            // order by a column not in the selection
-            assertRows(execute("SELECT c.a FROM %s WHERE a=? ORDER BY b ASC", 0),
-                    row(0), row(1), row(2));
+            for (boolean flush : new boolean[]{true, false})
+            {
+                if (flush)
+                    flush();
+                // order by a column not in the selection
+                assertRows(execute("SELECT c.a FROM %s WHERE a=? ORDER BY b ASC", 0),
+                           row(0), row(1), row(2));
 
-            assertRows(execute("SELECT c.a FROM %s WHERE a=? ORDER BY b DESC", 0),
-                    row(2), row(1), row(0));
+                assertRows(execute("SELECT c.a FROM %s WHERE a=? ORDER BY b DESC", 0),
+                           row(2), row(1), row(0));
 
-            assertRows(execute("SELECT blobAsInt(intAsBlob(c.a)) FROM %s WHERE a=? ORDER BY b DESC", 0),
-                       row(2), row(1), row(0));
+                assertRows(execute("SELECT blobAsInt(intAsBlob(c.a)) FROM %s WHERE a=? ORDER BY b DESC", 0),
+                           row(2), row(1), row(0));
+            }
             dropTable("DROP TABLE %s");
         }
     }
@@ -128,62 +146,68 @@
         execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 4);
         execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 2, 5);
 
-        assertRows(execute("SELECT * FROM %s WHERE a=? ORDER BY b ASC", 0),
-                row(0, 0, 0, 0),
-                row(0, 0, 1, 1),
-                row(0, 0, 2, 2),
-                row(0, 1, 0, 3),
-                row(0, 1, 1, 4),
-                row(0, 1, 2, 5)
-        );
+        for (boolean flush : new boolean[]{true, false})
+        {
+            if (flush)
+                flush();
 
-        assertRows(execute("SELECT * FROM %s WHERE a=? ORDER BY b DESC", 0),
-                row(0, 1, 2, 5),
-                row(0, 1, 1, 4),
-                row(0, 1, 0, 3),
-                row(0, 0, 2, 2),
-                row(0, 0, 1, 1),
-                row(0, 0, 0, 0)
-        );
+            assertRows(execute("SELECT * FROM %s WHERE a=? ORDER BY b ASC", 0),
+                       row(0, 0, 0, 0),
+                       row(0, 0, 1, 1),
+                       row(0, 0, 2, 2),
+                       row(0, 1, 0, 3),
+                       row(0, 1, 1, 4),
+                       row(0, 1, 2, 5)
+                    );
 
-        assertRows(execute("SELECT * FROM %s WHERE a=? ORDER BY b DESC, c DESC", 0),
-                row(0, 1, 2, 5),
-                row(0, 1, 1, 4),
-                row(0, 1, 0, 3),
-                row(0, 0, 2, 2),
-                row(0, 0, 1, 1),
-                row(0, 0, 0, 0)
-        );
+            assertRows(execute("SELECT * FROM %s WHERE a=? ORDER BY b DESC", 0),
+                       row(0, 1, 2, 5),
+                       row(0, 1, 1, 4),
+                       row(0, 1, 0, 3),
+                       row(0, 0, 2, 2),
+                       row(0, 0, 1, 1),
+                       row(0, 0, 0, 0)
+                    );
 
-        assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY c ASC", 0);
-        assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY c DESC", 0);
-        assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY b ASC, c DESC", 0);
-        assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY b DESC, c ASC", 0);
-        assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY d ASC", 0);
+            assertRows(execute("SELECT * FROM %s WHERE a=? ORDER BY b DESC, c DESC", 0),
+                       row(0, 1, 2, 5),
+                       row(0, 1, 1, 4),
+                       row(0, 1, 0, 3),
+                       row(0, 0, 2, 2),
+                       row(0, 0, 1, 1),
+                       row(0, 0, 0, 0)
+                    );
 
-        // select and order by b
-        assertRows(execute("SELECT b FROM %s WHERE a=? ORDER BY b ASC", 0),
-                row(0), row(0), row(0), row(1), row(1), row(1));
-        assertRows(execute("SELECT b FROM %s WHERE a=? ORDER BY b DESC", 0),
-                row(1), row(1), row(1), row(0), row(0), row(0));
+            assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY c ASC", 0);
+            assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY c DESC", 0);
+            assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY b ASC, c DESC", 0);
+            assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY b DESC, c ASC", 0);
+            assertInvalid("SELECT * FROM %s WHERE a=? ORDER BY d ASC", 0);
 
-        // select c, order by b
-        assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b ASC", 0),
-                row(0), row(1), row(2), row(0), row(1), row(2));
-        assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b DESC", 0),
-                row(2), row(1), row(0), row(2), row(1), row(0));
+            // select and order by b
+            assertRows(execute("SELECT b FROM %s WHERE a=? ORDER BY b ASC", 0),
+                       row(0), row(0), row(0), row(1), row(1), row(1));
+            assertRows(execute("SELECT b FROM %s WHERE a=? ORDER BY b DESC", 0),
+                       row(1), row(1), row(1), row(0), row(0), row(0));
 
-        // select c, order by b, c
-        assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b ASC, c ASC", 0),
-                row(0), row(1), row(2), row(0), row(1), row(2));
-        assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b DESC, c DESC", 0),
-                row(2), row(1), row(0), row(2), row(1), row(0));
+            // select c, order by b
+            assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b ASC", 0),
+                       row(0), row(1), row(2), row(0), row(1), row(2));
+            assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b DESC", 0),
+                       row(2), row(1), row(0), row(2), row(1), row(0));
 
-        // select d, order by b, c
-        assertRows(execute("SELECT d FROM %s WHERE a=? ORDER BY b ASC, c ASC", 0),
-                row(0), row(1), row(2), row(3), row(4), row(5));
-        assertRows(execute("SELECT d FROM %s WHERE a=? ORDER BY b DESC, c DESC", 0),
-                row(5), row(4), row(3), row(2), row(1), row(0));
+            // select c, order by b, c
+            assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b ASC, c ASC", 0),
+                       row(0), row(1), row(2), row(0), row(1), row(2));
+            assertRows(execute("SELECT c FROM %s WHERE a=? ORDER BY b DESC, c DESC", 0),
+                       row(2), row(1), row(0), row(2), row(1), row(0));
+
+            // select d, order by b, c
+            assertRows(execute("SELECT d FROM %s WHERE a=? ORDER BY b ASC, c ASC", 0),
+                       row(0), row(1), row(2), row(3), row(4), row(5));
+            assertRows(execute("SELECT d FROM %s WHERE a=? ORDER BY b DESC, c DESC", 0),
+                       row(5), row(4), row(3), row(2), row(1), row(0));
+        }
     }
 
     @Test
@@ -197,41 +221,46 @@
         execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 1, 4);
         execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 2, 5);
 
-        assertInvalid("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY c ASC", 0);
-        assertInvalid("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY c DESC", 0);
-        assertInvalid("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b ASC, c DESC", 0);
-        assertInvalid("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b DESC, c ASC", 0);
-        assertInvalid("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY d ASC", 0);
+        for (boolean flush : new boolean[]{true, false})
+        {
+            if (flush)
+                flush();
 
-        // select and order by b
-        assertRows(execute("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b ASC", 0),
-                   row(0), row(0), row(0), row(1), row(1), row(1));
-        assertRows(execute("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b DESC", 0),
-                   row(1), row(1), row(1), row(0), row(0), row(0));
+            assertInvalid("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY c ASC", 0);
+            assertInvalid("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY c DESC", 0);
+            assertInvalid("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b ASC, c DESC", 0);
+            assertInvalid("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b DESC, c ASC", 0);
+            assertInvalid("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY d ASC", 0);
 
-        assertRows(execute("SELECT b, blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b ASC", 0),
-                row(0, 0), row(0, 0), row(0, 0), row(1, 1), row(1, 1), row(1, 1));
-        assertRows(execute("SELECT b, blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b DESC", 0),
-                row(1, 1), row(1, 1), row(1, 1), row(0, 0), row(0, 0), row(0, 0));
+            // select and order by b
+            assertRows(execute("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b ASC", 0),
+                       row(0), row(0), row(0), row(1), row(1), row(1));
+            assertRows(execute("SELECT blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b DESC", 0),
+                       row(1), row(1), row(1), row(0), row(0), row(0));
 
-        // select c, order by b
-        assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b ASC", 0),
-                row(0), row(1), row(2), row(0), row(1), row(2));
-        assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b DESC", 0),
-                row(2), row(1), row(0), row(2), row(1), row(0));
+            assertRows(execute("SELECT b, blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b ASC", 0),
+                       row(0, 0), row(0, 0), row(0, 0), row(1, 1), row(1, 1), row(1, 1));
+            assertRows(execute("SELECT b, blobAsInt(intAsBlob(b)) FROM %s WHERE a=? ORDER BY b DESC", 0),
+                       row(1, 1), row(1, 1), row(1, 1), row(0, 0), row(0, 0), row(0, 0));
 
-        // select c, order by b, c
-        assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b ASC, c ASC", 0),
-                row(0), row(1), row(2), row(0), row(1), row(2));
-        assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b DESC, c DESC", 0),
-                row(2), row(1), row(0), row(2), row(1), row(0));
+            // select c, order by b
+            assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b ASC", 0),
+                       row(0), row(1), row(2), row(0), row(1), row(2));
+            assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b DESC", 0),
+                       row(2), row(1), row(0), row(2), row(1), row(0));
 
-        // select d, order by b, c
-        assertRows(execute("SELECT blobAsInt(intAsBlob(d)) FROM %s WHERE a=? ORDER BY b ASC, c ASC", 0),
-                row(0), row(1), row(2), row(3), row(4), row(5));
-        assertRows(execute("SELECT blobAsInt(intAsBlob(d)) FROM %s WHERE a=? ORDER BY b DESC, c DESC", 0),
-                row(5), row(4), row(3), row(2), row(1), row(0));
+            // select c, order by b, c
+            assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b ASC, c ASC", 0),
+                       row(0), row(1), row(2), row(0), row(1), row(2));
+            assertRows(execute("SELECT blobAsInt(intAsBlob(c)) FROM %s WHERE a=? ORDER BY b DESC, c DESC", 0),
+                       row(2), row(1), row(0), row(2), row(1), row(0));
 
+            // select d, order by b, c
+            assertRows(execute("SELECT blobAsInt(intAsBlob(d)) FROM %s WHERE a=? ORDER BY b ASC, c ASC", 0),
+                       row(0), row(1), row(2), row(3), row(4), row(5));
+            assertRows(execute("SELECT blobAsInt(intAsBlob(d)) FROM %s WHERE a=? ORDER BY b DESC, c DESC", 0),
+                       row(5), row(4), row(3), row(2), row(1), row(0));
+        }
     }
 
     /**
@@ -246,8 +275,14 @@
         for (int i = 0; i < 10; i++)
             execute("INSERT INTO %s (k, c, v) VALUES (0, ?, ?)", i, i);
 
-        assertRows(execute("SELECT v FROM %s WHERE k = 0 ORDER BY c DESC"),
-                   row(9), row(8), row(7), row(6), row(5), row(4), row(3), row(2), row(1), row(0));
+        for (boolean flush : new boolean[]{true, false})
+        {
+            if (flush)
+                flush();
+
+            assertRows(execute("SELECT v FROM %s WHERE k = 0 ORDER BY c DESC"),
+                       row(9), row(8), row(7), row(6), row(5), row(4), row(3), row(2), row(1), row(0));
+        }
 
         createTable("CREATE TABLE %s (k int, c1 int, c2 int, v int, PRIMARY KEY (k, c1, c2)) WITH COMPACT STORAGE");
 
@@ -255,15 +290,21 @@
             for (int j = 0; j < 2; j++)
                 execute("INSERT INTO %s (k, c1, c2, v) VALUES (0, ?, ?, ?)", i, j, i * 2 + j);
 
-        assertInvalid("SELECT v FROM %s WHERE k = 0 ORDER BY c DESC");
-        assertInvalid("SELECT v FROM %s WHERE k = 0 ORDER BY c2 DESC");
-        assertInvalid("SELECT v FROM %s WHERE k = 0 ORDER BY k DESC");
+        for (boolean flush : new boolean[]{true, false})
+        {
+            if (flush)
+                flush();
 
-        assertRows(execute("SELECT v FROM %s WHERE k = 0 ORDER BY c1 DESC"),
-                   row(7), row(6), row(5), row(4), row(3), row(2), row(1), row(0));
+            assertInvalid("SELECT v FROM %s WHERE k = 0 ORDER BY c DESC");
+            assertInvalid("SELECT v FROM %s WHERE k = 0 ORDER BY c2 DESC");
+            assertInvalid("SELECT v FROM %s WHERE k = 0 ORDER BY k DESC");
 
-        assertRows(execute("SELECT v FROM %s WHERE k = 0 ORDER BY c1"),
-                   row(0), row(1), row(2), row(3), row(4), row(5), row(6), row(7));
+            assertRows(execute("SELECT v FROM %s WHERE k = 0 ORDER BY c1 DESC"),
+                       row(7), row(6), row(5), row(4), row(3), row(2), row(1), row(0));
+
+            assertRows(execute("SELECT v FROM %s WHERE k = 0 ORDER BY c1"),
+                       row(0), row(1), row(2), row(3), row(4), row(5), row(6), row(7));
+        }
     }
 
     /**
@@ -280,23 +321,29 @@
         execute("INSERT INTO %s (row, number, string) VALUES ('row', 3, 'three')");
         execute("INSERT INTO %s (row, number, string) VALUES ('row', 4, 'four')");
 
-        assertRows(execute("SELECT number FROM %s WHERE row='row' AND number < 3 ORDER BY number ASC"),
-                   row(1), row(2));
+        for (boolean flush : new boolean[]{true, false})
+        {
+            if (flush)
+                flush();
 
-        assertRows(execute("SELECT number FROM %s WHERE row='row' AND number >= 3 ORDER BY number ASC"),
-                   row(3), row(4));
+            assertRows(execute("SELECT number FROM %s WHERE row='row' AND number < 3 ORDER BY number ASC"),
+                       row(1), row(2));
 
-        assertRows(execute("SELECT number FROM %s WHERE row='row' AND number < 3 ORDER BY number DESC"),
-                   row(2), row(1));
+            assertRows(execute("SELECT number FROM %s WHERE row='row' AND number >= 3 ORDER BY number ASC"),
+                       row(3), row(4));
 
-        assertRows(execute("SELECT number FROM %s WHERE row='row' AND number >= 3 ORDER BY number DESC"),
-                   row(4), row(3));
+            assertRows(execute("SELECT number FROM %s WHERE row='row' AND number < 3 ORDER BY number DESC"),
+                       row(2), row(1));
 
-        assertRows(execute("SELECT number FROM %s WHERE row='row' AND number > 3 ORDER BY number DESC"),
-                   row(4));
+            assertRows(execute("SELECT number FROM %s WHERE row='row' AND number >= 3 ORDER BY number DESC"),
+                       row(4), row(3));
 
-        assertRows(execute("SELECT number FROM %s WHERE row='row' AND number <= 3 ORDER BY number DESC"),
-                   row(3), row(2), row(1));
+            assertRows(execute("SELECT number FROM %s WHERE row='row' AND number > 3 ORDER BY number DESC"),
+                       row(4));
+
+            assertRows(execute("SELECT number FROM %s WHERE row='row' AND number <= 3 ORDER BY number DESC"),
+                       row(3), row(2), row(1));
+        }
     }
 
     /**
@@ -316,7 +363,7 @@
     }
 
     /**
-     * Check that order-by works with IN (#4327, #10363)
+     * Check that order-by works with IN (#4327)
      * migrated from cql_tests.py:TestCQL.order_by_with_in_test()
      */
     @Test
@@ -329,14 +376,20 @@
         execute("INSERT INTO %s (my_id, col1, value) VALUES ( 'key3', 2, 'b')");
         execute("INSERT INTO %s (my_id, col1, value) VALUES ( 'key4', 4, 'd')");
 
-        assertRows(execute("SELECT col1 FROM %s WHERE my_id in('key1', 'key2', 'key3') ORDER BY col1"),
-                   row(1), row(2), row(3));
+        for (boolean flush : new boolean[]{true, false})
+        {
+            if (flush)
+                flush();
 
-        assertRows(execute("SELECT col1, my_id FROM %s WHERE my_id in('key1', 'key2', 'key3') ORDER BY col1"),
-                   row(1, "key1"), row(2, "key3"), row(3, "key2"));
+            assertRows(execute("SELECT col1 FROM %s WHERE my_id in('key1', 'key2', 'key3') ORDER BY col1"),
+                       row(1), row(2), row(3));
 
-        assertRows(execute("SELECT my_id, col1 FROM %s WHERE my_id in('key1', 'key2', 'key3') ORDER BY col1"),
-                   row("key1", 1), row("key3", 2), row("key2", 3));
+            assertRows(execute("SELECT col1, my_id FROM %s WHERE my_id in('key1', 'key2', 'key3') ORDER BY col1"),
+                       row(1, "key1"), row(2, "key3"), row(3, "key2"));
+
+            assertRows(execute("SELECT my_id, col1 FROM %s WHERE my_id in('key1', 'key2', 'key3') ORDER BY col1"),
+                       row("key1", 1), row("key3", 2), row("key2", 3));
+        }
 
         createTable("CREATE TABLE %s (pk1 int, pk2 int, c int, v text, PRIMARY KEY ((pk1, pk2), c) )");
         execute("INSERT INTO %s (pk1, pk2, c, v) VALUES (?, ?, ?, ?)", 1, 1, 2, "A");
@@ -344,25 +397,31 @@
         execute("INSERT INTO %s (pk1, pk2, c, v) VALUES (?, ?, ?, ?)", 1, 3, 3, "C");
         execute("INSERT INTO %s (pk1, pk2, c, v) VALUES (?, ?, ?, ?)", 1, 1, 4, "D");
 
-        assertRows(execute("SELECT v, ttl(v), c FROM %s where pk1 = ? AND pk2 IN (?, ?) ORDER BY c; ", 1, 1, 2),
-                   row("B", null, 1),
-                   row("A", null, 2),
-                   row("D", null, 4));
+        for (boolean flush : new boolean[]{true, false})
+        {
+            if (flush)
+                flush();
 
-        assertRows(execute("SELECT v, ttl(v), c as name_1 FROM %s where pk1 = ? AND pk2 IN (?, ?) ORDER BY c; ", 1, 1, 2),
-                   row("B", null, 1),
-                   row("A", null, 2),
-                   row("D", null, 4));
+            assertRows(execute("SELECT v, ttl(v), c FROM %s where pk1 = ? AND pk2 IN (?, ?) ORDER BY c; ", 1, 1, 2),
+                       row("B", null, 1),
+                       row("A", null, 2),
+                       row("D", null, 4));
 
-        assertRows(execute("SELECT v FROM %s where pk1 = ? AND pk2 IN (?, ?) ORDER BY c; ", 1, 1, 2),
-                   row("B"),
-                   row("A"),
-                   row("D"));
+            assertRows(execute("SELECT v, ttl(v), c as name_1 FROM %s where pk1 = ? AND pk2 IN (?, ?) ORDER BY c; ", 1, 1, 2),
+                       row("B", null, 1),
+                       row("A", null, 2),
+                       row("D", null, 4));
 
-        assertRows(execute("SELECT v as c FROM %s where pk1 = ? AND pk2 IN (?, ?) ORDER BY c; ", 1, 1, 2),
-                   row("B"),
-                   row("A"),
-                   row("D"));
+            assertRows(execute("SELECT v FROM %s where pk1 = ? AND pk2 IN (?, ?) ORDER BY c; ", 1, 1, 2),
+                       row("B"),
+                       row("A"),
+                       row("D"));
+
+            assertRows(execute("SELECT v as c FROM %s where pk1 = ? AND pk2 IN (?, ?) ORDER BY c; ", 1, 1, 2),
+                       row("B"),
+                       row("A"),
+                       row("D"));
+        }
 
         createTable("CREATE TABLE %s (pk1 int, pk2 int, c1 int, c2 int, v text, PRIMARY KEY ((pk1, pk2), c1, c2) )");
         execute("INSERT INTO %s (pk1, pk2, c1, c2, v) VALUES (?, ?, ?, ?, ?)", 1, 1, 4, 4, "A");
@@ -370,25 +429,80 @@
         execute("INSERT INTO %s (pk1, pk2, c1, c2, v) VALUES (?, ?, ?, ?, ?)", 1, 3, 3, 3, "C");
         execute("INSERT INTO %s (pk1, pk2, c1, c2, v) VALUES (?, ?, ?, ?, ?)", 1, 1, 4, 1, "D");
 
-        assertRows(execute("SELECT v, ttl(v), c1, c2 FROM %s where pk1 = ? AND pk2 IN (?, ?) ORDER BY c1, c2; ", 1, 1, 2),
-                   row("B", null, 1, 2),
-                   row("D", null, 4, 1),
-                   row("A", null, 4, 4));
+        for (boolean flush : new boolean[]{true, false})
+        {
+            if (flush)
+                flush();
 
-        assertRows(execute("SELECT v, ttl(v), c1 as name_1, c2 as name_2 FROM %s where pk1 = ? AND pk2 IN (?, ?) ORDER BY c1, c2; ", 1, 1, 2),
-                   row("B", null, 1, 2),
-                   row("D", null, 4, 1),
-                   row("A", null, 4, 4));
+            assertRows(execute("SELECT v, ttl(v), c1, c2 FROM %s where pk1 = ? AND pk2 IN (?, ?) ORDER BY c1, c2; ", 1, 1, 2),
+                       row("B", null, 1, 2),
+                       row("D", null, 4, 1),
+                       row("A", null, 4, 4));
 
-        assertRows(execute("SELECT v FROM %s where pk1 = ? AND pk2 IN (?, ?) ORDER BY c1, c2; ", 1, 1, 2),
-                   row("B"),
-                   row("D"),
-                   row("A"));
+            assertRows(execute("SELECT v, ttl(v), c1 as name_1, c2 as name_2 FROM %s where pk1 = ? AND pk2 IN (?, ?) ORDER BY c1, c2; ", 1, 1, 2),
+                       row("B", null, 1, 2),
+                       row("D", null, 4, 1),
+                       row("A", null, 4, 4));
 
-        assertRows(execute("SELECT v as c2 FROM %s where pk1 = ? AND pk2 IN (?, ?) ORDER BY c1, c2; ", 1, 1, 2),
-                   row("B"),
-                   row("D"),
-                   row("A"));
+            assertRows(execute("SELECT v FROM %s where pk1 = ? AND pk2 IN (?, ?) ORDER BY c1, c2; ", 1, 1, 2),
+                       row("B"),
+                       row("D"),
+                       row("A"));
+
+            assertRows(execute("SELECT v as c2 FROM %s where pk1 = ? AND pk2 IN (?, ?) ORDER BY c1, c2; ", 1, 1, 2),
+                       row("B"),
+                       row("D"),
+                       row("A"));
+        }
+    }
+
+    @Test
+    public void testOrderByForInClauseWithNullValue() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, s int static, d int, PRIMARY KEY (a, b, c))");
+
+        execute("INSERT INTO %s (a, b, c, d) VALUES (1, 1, 1, 1)");
+        execute("INSERT INTO %s (a, b, c, d) VALUES (1, 1, 2, 1)");
+        execute("INSERT INTO %s (a, b, c, d) VALUES (2, 2, 1, 1)");
+        execute("INSERT INTO %s (a, b, c, d) VALUES (2, 2, 2, 1)");
+
+        execute("UPDATE %s SET s = 1 WHERE a = 1");
+        execute("UPDATE %s SET s = 2 WHERE a = 2");
+        execute("UPDATE %s SET s = 3 WHERE a = 3");
+
+        for (boolean flush : new boolean[]{true, false})
+        {
+            if (flush)
+                flush();
+
+            assertRows(execute("SELECT a, b, c, d, s FROM %s WHERE a IN (1, 2, 3) ORDER BY b DESC"),
+                       row(2, 2, 2, 1, 2),
+                       row(2, 2, 1, 1, 2),
+                       row(1, 1, 2, 1, 1),
+                       row(1, 1, 1, 1, 1),
+                       row(3, null, null, null, 3));
+
+            assertRows(execute("SELECT a, b, c, d, s FROM %s WHERE a IN (1, 2, 3) ORDER BY b ASC"),
+                       row(3, null, null, null, 3),
+                       row(1, 1, 1, 1, 1),
+                       row(1, 1, 2, 1, 1),
+                       row(2, 2, 1, 1, 2),
+                       row(2, 2, 2, 1, 2));
+
+            assertRows(execute("SELECT a, b, c, d, s FROM %s WHERE a IN (1, 2, 3) ORDER BY b DESC , c DESC"),
+                       row(2, 2, 2, 1, 2),
+                       row(2, 2, 1, 1, 2),
+                       row(1, 1, 2, 1, 1),
+                       row(1, 1, 1, 1, 1),
+                       row(3, null, null, null, 3));
+
+            assertRows(execute("SELECT a, b, c, d, s FROM %s WHERE a IN (1, 2, 3) ORDER BY b ASC, c ASC"),
+                       row(3, null, null, null, 3),
+                       row(1, 1, 1, 1, 1),
+                       row(1, 1, 2, 1, 1),
+                       row(2, 2, 1, 1, 2),
+                       row(2, 2, 2, 1, 2));
+        }
     }
 
     /**
@@ -403,13 +517,19 @@
         for(int i =0; i < 10; i++)
             execute("INSERT INTO %s (k, c, v) VALUES (0, ?, ?)", i, i);
 
-        assertRows(execute("SELECT c, v FROM %s WHERE k = 0 ORDER BY c ASC"),
-                   row(0, 0), row(1, 1), row(2, 2), row(3, 3), row(4, 4),
-                   row(5, 5), row(6, 6), row(7, 7), row(8, 8), row(9, 9));
+        for (boolean flush : new boolean[]{true, false})
+        {
+            if (flush)
+                flush();
 
-        assertRows(execute("SELECT c, v FROM %s WHERE k = 0 ORDER BY c DESC"),
-                   row(9, 9), row(8, 8), row(7, 7), row(6, 6), row(5, 5),
-                   row(4, 4), row(3, 3), row(2, 2), row(1, 1), row(0, 0));
+            assertRows(execute("SELECT c, v FROM %s WHERE k = 0 ORDER BY c ASC"),
+                       row(0, 0), row(1, 1), row(2, 2), row(3, 3), row(4, 4),
+                       row(5, 5), row(6, 6), row(7, 7), row(8, 8), row(9, 9));
+
+            assertRows(execute("SELECT c, v FROM %s WHERE k = 0 ORDER BY c DESC"),
+                       row(9, 9), row(8, 8), row(7, 7), row(6, 6), row(5, 5),
+                       row(4, 4), row(3, 3), row(2, 2), row(1, 1), row(0, 0));
+        }
 
         createTable("CREATE TABLE %s (k int, c1 int, c2 int, v text, PRIMARY KEY (k, c1, c2)) WITH CLUSTERING ORDER BY (c1 ASC, c2 DESC)");
 
@@ -417,28 +537,34 @@
             for(int j = 0; j < 10; j++)
                 execute("INSERT INTO %s (k, c1, c2, v) VALUES (0, ?, ?, ?)", i, j, String.format("%d%d", i, j));
 
-        assertInvalid("SELECT c1, c2, v FROM %s WHERE k = 0 ORDER BY c1 ASC, c2 ASC");
-        assertInvalid("SELECT c1, c2, v FROM %s WHERE k = 0 ORDER BY c1 DESC, c2 DESC");
+        for (boolean flush : new boolean[]{true, false})
+        {
+            if (flush)
+                flush();
 
-        Object[][] expectedRows = new Object[100][];
-        for(int i = 0; i < 10; i++)
-            for(int j = 9; j >= 0; j--)
-                expectedRows[i * 10 + (9 - j)] = row(i, j, String.format("%d%d", i, j));
+            assertInvalid("SELECT c1, c2, v FROM %s WHERE k = 0 ORDER BY c1 ASC, c2 ASC");
+            assertInvalid("SELECT c1, c2, v FROM %s WHERE k = 0 ORDER BY c1 DESC, c2 DESC");
 
-        assertRows(execute("SELECT c1, c2, v FROM %s WHERE k = 0 ORDER BY c1 ASC"),
-                   expectedRows);
+            Object[][] expectedRows = new Object[100][];
+            for(int i = 0; i < 10; i++)
+                for(int j = 9; j >= 0; j--)
+                    expectedRows[i * 10 + (9 - j)] = row(i, j, String.format("%d%d", i, j));
 
-        assertRows(execute("SELECT c1, c2, v FROM %s WHERE k = 0 ORDER BY c1 ASC, c2 DESC"),
-                   expectedRows);
+            assertRows(execute("SELECT c1, c2, v FROM %s WHERE k = 0 ORDER BY c1 ASC"),
+                       expectedRows);
 
-        for(int i = 9; i >= 0; i--)
-            for(int j = 0; j < 10; j++)
-                expectedRows[(9 - i) * 10 + j] = row(i, j, String.format("%d%d", i, j));
+            assertRows(execute("SELECT c1, c2, v FROM %s WHERE k = 0 ORDER BY c1 ASC, c2 DESC"),
+                       expectedRows);
 
-        assertRows(execute("SELECT c1, c2, v FROM %s WHERE k = 0 ORDER BY c1 DESC, c2 ASC"),
-                   expectedRows);
+            for(int i = 9; i >= 0; i--)
+                for(int j = 0; j < 10; j++)
+                    expectedRows[(9 - i) * 10 + j] = row(i, j, String.format("%d%d", i, j));
 
-        assertInvalid("SELECT c1, c2, v FROM %s WHERE k = 0 ORDER BY c2 DESC, c1 ASC");
+            assertRows(execute("SELECT c1, c2, v FROM %s WHERE k = 0 ORDER BY c1 DESC, c2 ASC"),
+                       expectedRows);
+
+            assertInvalid("SELECT c1, c2, v FROM %s WHERE k = 0 ORDER BY c2 DESC, c1 ASC");
+        }
     }
 
     /**
@@ -453,18 +579,24 @@
             for (int j = 0; j < 2; j++)
                 execute("INSERT INTO %s (k, c1, c2) VALUES ('foo', ?, ?)", i, j);
 
-        assertRows(execute("SELECT c1, c2 FROM %s WHERE k = 'foo'"),
-                   row(0, 1), row(0, 0), row(1, 1), row(1, 0));
+        for (boolean flush : new boolean[]{true, false})
+        {
+            if (flush)
+                flush();
 
-        assertRows(execute("SELECT c1, c2 FROM %s WHERE k = 'foo' ORDER BY c1 ASC, c2 DESC"),
-                   row(0, 1), row(0, 0), row(1, 1), row(1, 0));
+            assertRows(execute("SELECT c1, c2 FROM %s WHERE k = 'foo'"),
+                       row(0, 1), row(0, 0), row(1, 1), row(1, 0));
 
-        assertRows(execute("SELECT c1, c2 FROM %s WHERE k = 'foo' ORDER BY c1 DESC, c2 ASC"),
-                   row(1, 0), row(1, 1), row(0, 0), row(0, 1));
+            assertRows(execute("SELECT c1, c2 FROM %s WHERE k = 'foo' ORDER BY c1 ASC, c2 DESC"),
+                       row(0, 1), row(0, 0), row(1, 1), row(1, 0));
 
-        assertInvalid("SELECT c1, c2 FROM %s WHERE k = 'foo' ORDER BY c2 DESC");
-        assertInvalid("SELECT c1, c2 FROM %s WHERE k = 'foo' ORDER BY c2 ASC");
-        assertInvalid("SELECT c1, c2 FROM %s WHERE k = 'foo' ORDER BY c1 ASC, c2 ASC");
+            assertRows(execute("SELECT c1, c2 FROM %s WHERE k = 'foo' ORDER BY c1 DESC, c2 ASC"),
+                       row(1, 0), row(1, 1), row(0, 0), row(0, 1));
+
+            assertInvalid("SELECT c1, c2 FROM %s WHERE k = 'foo' ORDER BY c2 DESC");
+            assertInvalid("SELECT c1, c2 FROM %s WHERE k = 'foo' ORDER BY c2 ASC");
+            assertInvalid("SELECT c1, c2 FROM %s WHERE k = 'foo' ORDER BY c1 ASC, c2 ASC");
+        }
     }
 
     /**
@@ -478,9 +610,15 @@
         execute("INSERT INTO %s(k, c1, c2) VALUES (0, 0, 1)");
         execute("INSERT INTO %s(k, c1, c2) VALUES (0, 0, 2)");
 
-        assertRows(execute("SELECT * FROM %s WHERE k=0 AND c1 = 0 AND c2 IN (2, 0) ORDER BY c1 DESC"),
-                   row(0, 0, 2),
-                   row(0, 0, 0));
+        for (boolean flush : new boolean[]{true, false})
+        {
+            if (flush)
+                flush();
+
+            assertRows(execute("SELECT * FROM %s WHERE k=0 AND c1 = 0 AND c2 IN (2, 0) ORDER BY c1 DESC"),
+                       row(0, 0, 2),
+                       row(0, 0, 0));
+        }
     }
 
     /**
@@ -499,44 +637,99 @@
         execute("INSERT INTO %s (k, c1, c2, v) VALUES (1, 1, 1, 4)");
         execute("INSERT INTO %s (k, c1, c2, v) VALUES (1, 1, 2, 5)");
 
-        assertRows(execute("SELECT * FROM %s WHERE k=0 AND c1 = 0 AND c2 IN (2, 0)"),
-                   row(0, 0, 0, 0),
-                   row(0, 0, 2, 2));
-        assertRows(execute("SELECT * FROM %s WHERE k=0 AND c1 = 0 AND c2 IN (2, 0) ORDER BY c1 ASC, c2 ASC"),
-                   row(0, 0, 0, 0),
-                   row(0, 0, 2, 2));
+        for (boolean flush : new boolean[]{true, false})
+        {
+            if (flush)
+                flush();
 
-        // check that we don 't need to select the column on which we order
-        assertRows(execute("SELECT v FROM %s WHERE k=0 AND c1 = 0 AND c2 IN (2, 0)"),
-                   row(0),
-                   row(2));
-        assertRows(execute("SELECT v FROM %s WHERE k=0 AND c1 = 0 AND c2 IN (2, 0) ORDER BY c1 ASC"),
-                   row(0),
-                   row(2));
-        assertRows(execute("SELECT v FROM %s WHERE k=0 AND c1 = 0 AND c2 IN (2, 0) ORDER BY c1 DESC"),
-                   row(2),
-                   row(0));
-        assertRows(execute("SELECT v FROM %s WHERE k IN (1, 0)"),
-                   row(0),
-                   row(1),
-                   row(2),
-                   row(3),
-                   row(4),
-                   row(5));
+            assertRows(execute("SELECT * FROM %s WHERE k=0 AND c1 = 0 AND c2 IN (2, 0)"),
+                       row(0, 0, 0, 0),
+                       row(0, 0, 2, 2));
+            assertRows(execute("SELECT * FROM %s WHERE k=0 AND c1 = 0 AND c2 IN (2, 0) ORDER BY c1 ASC, c2 ASC"),
+                       row(0, 0, 0, 0),
+                       row(0, 0, 2, 2));
 
-        assertRows(execute("SELECT v FROM %s WHERE k IN (1, 0) ORDER BY c1 ASC"),
-                   row(0),
-                   row(1),
-                   row(2),
-                   row(3),
-                   row(4),
-                   row(5));
+            // check that we don 't need to select the column on which we order
+            assertRows(execute("SELECT v FROM %s WHERE k=0 AND c1 = 0 AND c2 IN (2, 0)"),
+                       row(0),
+                       row(2));
+            assertRows(execute("SELECT v FROM %s WHERE k=0 AND c1 = 0 AND c2 IN (2, 0) ORDER BY c1 ASC"),
+                       row(0),
+                       row(2));
+            assertRows(execute("SELECT v FROM %s WHERE k=0 AND c1 = 0 AND c2 IN (2, 0) ORDER BY c1 DESC"),
+                       row(2),
+                       row(0));
 
-        // we should also be able to use functions in the select clause (additional test for CASSANDRA - 8286)
-        Object[][] results = getRows(execute("SELECT writetime(v) FROM %s WHERE k IN (1, 0) ORDER BY c1 ASC"));
+            assertRows(execute("SELECT v FROM %s WHERE k IN (1, 0)"),
+                       row(0),
+                       row(1),
+                       row(2),
+                       row(3),
+                       row(4),
+                       row(5));
 
-        // since we don 't know the write times, just assert that the order matches the order we expect
-        assertTrue(isFirstIntSorted(results));
+            assertRows(execute("SELECT v FROM %s WHERE k IN (1, 0) ORDER BY c1 ASC"),
+                       row(0),
+                       row(1),
+                       row(2),
+                       row(3),
+                       row(4),
+                       row(5));
+
+            // we should also be able to use functions in the select clause (additional test for CASSANDRA - 8286)
+            Object[][] results = getRows(execute("SELECT writetime(v) FROM %s WHERE k IN (1, 0) ORDER BY c1 ASC"));
+
+            // since we don 't know the write times, just assert that the order matches the order we expect
+            assertTrue(isFirstIntSorted(results));
+        }
+    }
+
+    @Test
+    public void testInOrderByWithTwoPartitionKeyColumns() throws Throwable
+    {
+        for (String option : asList("", "WITH CLUSTERING ORDER BY (col_3 DESC)"))
+        {
+            createTable("CREATE TABLE %s (col_1 int, col_2 int, col_3 int, PRIMARY KEY ((col_1, col_2), col_3)) " + option);
+            execute("INSERT INTO %s (col_1, col_2, col_3) VALUES(?, ?, ?)", 1, 1, 1);
+            execute("INSERT INTO %s (col_1, col_2, col_3) VALUES(?, ?, ?)", 1, 1, 2);
+            execute("INSERT INTO %s (col_1, col_2, col_3) VALUES(?, ?, ?)", 1, 1, 13);
+            execute("INSERT INTO %s (col_1, col_2, col_3) VALUES(?, ?, ?)", 1, 2, 10);
+            execute("INSERT INTO %s (col_1, col_2, col_3) VALUES(?, ?, ?)", 1, 2, 11);
+
+            for (boolean flush : new boolean[]{true, false})
+            {
+                if (flush)
+                    flush();
+
+                assertRows(execute("select * from %s where col_1=? and col_2 IN (?, ?) order by col_3;", 1, 1, 2),
+                           row(1, 1, 1),
+                           row(1, 1, 2),
+                           row(1, 2, 10),
+                           row(1, 2, 11),
+                           row(1, 1, 13));
+
+                assertRows(execute("select * from %s where col_1=? and col_2 IN (?, ?) order by col_3 desc;", 1, 1, 2),
+                           row(1, 1, 13),
+                           row(1, 2, 11),
+                           row(1, 2, 10),
+                           row(1, 1, 2),
+                           row(1, 1, 1));
+
+                assertRows(execute("select * from %s where col_2 IN (?, ?) and col_1=? order by col_3;", 1, 2, 1),
+                           row(1, 1, 1),
+                           row(1, 1, 2),
+                           row(1, 2, 10),
+                           row(1, 2, 11),
+                           row(1, 1, 13));
+
+                assertRows(execute("select * from %s where col_2 IN (?, ?) and col_1=? order by col_3 desc;", 1, 2, 1),
+                           row(1, 1, 13),
+                           row(1, 2, 11),
+                           row(1, 2, 10),
+                           row(1, 1, 2),
+                           row(1, 1, 1));
+            }
+        }
     }
 
     private boolean isFirstIntSorted(Object[][] rows)
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderedPartitionerTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderedPartitionerTest.java
index e0d1ca2..9609906 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderedPartitionerTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderedPartitionerTest.java
@@ -20,6 +20,8 @@
  */
 package org.apache.cassandra.cql3.validation.operations;
 
+import java.util.Arrays;
+
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -37,7 +39,7 @@
     @BeforeClass
     public static void setUp()
     {
-        DatabaseDescriptor.setPartitioner(new ByteOrderedPartitioner());
+        DatabaseDescriptor.setPartitioner(ByteOrderedPartitioner.instance);
     }
 
     @Test
@@ -49,17 +51,30 @@
         assertRows(execute("SELECT * FROM %s WHERE token(a) >= token(?)", 0), row(0, "a"));
         assertRows(execute("SELECT * FROM %s WHERE token(a) >= token(?) and token(a) < token(?)", 0, 1), row(0, "a"));
         assertInvalid("SELECT * FROM %s WHERE token(a) > token(?)", "a");
-        assertInvalid("SELECT * FROM %s WHERE token(a, b) >= token(?, ?)", "b", 0);
-        assertInvalid("SELECT * FROM %s WHERE token(a) >= token(?) and token(a) >= token(?)", 0, 1);
-        assertInvalid("SELECT * FROM %s WHERE token(a) >= token(?) and token(a) = token(?)", 0, 1);
+        assertInvalidMessage("The token() function must contains only partition key components",
+                             "SELECT * FROM %s WHERE token(a, b) >= token(?, ?)", "b", 0);
+        assertInvalidMessage("More than one restriction was found for the start bound on a",
+                             "SELECT * FROM %s WHERE token(a) >= token(?) and token(a) >= token(?)", 0, 1);
+        assertInvalidMessage("Columns \"a\" cannot be restricted by both an equality and an inequality relation",
+                             "SELECT * FROM %s WHERE token(a) >= token(?) and token(a) = token(?)", 0, 1);
         assertInvalidSyntax("SELECT * FROM %s WHERE token(a) = token(?) and token(a) IN (token(?))", 0, 1);
+
+        assertInvalidMessage("More than one restriction was found for the start bound on a",
+                             "SELECT * FROM %s WHERE token(a) > token(?) AND token(a) > token(?)", 1, 2);
+        assertInvalidMessage("More than one restriction was found for the end bound on a",
+                             "SELECT * FROM %s WHERE token(a) <= token(?) AND token(a) < token(?)", 1, 2);
+        assertInvalidMessage("Columns \"a\" cannot be restricted by both an equality and an inequality relation",
+                             "SELECT * FROM %s WHERE token(a) > token(?) AND token(a) = token(?)", 1, 2);
+        assertInvalidMessage("a cannot be restricted by more than one relation if it includes an Equal",
+                             "SELECT * FROM %s WHERE  token(a) = token(?) AND token(a) > token(?)", 1, 2);
     }
 
     @Test
     public void testTokenFunctionWithPartitionKeyAndClusteringKeyArguments() throws Throwable
     {
         createTable("CREATE TABLE IF NOT EXISTS %s (a int, b text, PRIMARY KEY (a, b))");
-        assertInvalid("SELECT * FROM %s WHERE token(a, b) > token(0, 'c')");
+        assertInvalidMessage("The token() function must contains only partition key components",
+                             "SELECT * FROM %s WHERE token(a, b) > token(0, 'c')");
     }
 
     @Test
@@ -78,9 +93,150 @@
                            0, "d"),
                    row(0, "b"),
                    row(0, "c"));
-        assertInvalid("SELECT * FROM %s WHERE token(a) > token(?) and token(b) > token(?)", 0, "a");
-        assertInvalid("SELECT * FROM %s WHERE token(a) > token(?, ?) and token(a) < token(?, ?) and token(b) > token(?, ?) ", 0, "a", 0, "d", 0, "a");
-        assertInvalid("SELECT * FROM %s WHERE token(b, a) > token(0, 'c')");
+        assertInvalidMessage("The token() function must be applied to all partition key components or none of them",
+                             "SELECT * FROM %s WHERE token(a) > token(?) and token(b) > token(?)", 0, "a");
+        assertInvalidMessage("The token() function must be applied to all partition key components or none of them",
+                             "SELECT * FROM %s WHERE token(a) > token(?, ?) and token(a) < token(?, ?) and token(b) > token(?, ?) ",
+                             0, "a", 0, "d", 0, "a");
+        assertInvalidMessage("The token function arguments must be in the partition key order: a, b",
+                             "SELECT * FROM %s WHERE token(b, a) > token(0, 'c')");
+        assertInvalidMessage("The token() function must be applied to all partition key components or none of them",
+                             "SELECT * FROM %s WHERE token(a, b) > token(?, ?) and token(b) < token(?, ?)", 0, "a", 0, "a");
+        assertInvalidMessage("The token() function must be applied to all partition key components or none of them",
+                             "SELECT * FROM %s WHERE token(a) > token(?, ?) and token(b) > token(?, ?)", 0, "a", 0, "a");
+    }
+
+    @Test
+    public void testSingleColumnPartitionKeyWithTokenNonTokenRestrictionsMix() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int primary key, b int)");
+
+        execute("INSERT INTO %s (a, b) VALUES (0, 0);");
+        execute("INSERT INTO %s (a, b) VALUES (1, 1);");
+        execute("INSERT INTO %s (a, b) VALUES (2, 2);");
+        execute("INSERT INTO %s (a, b) VALUES (3, 3);");
+        execute("INSERT INTO %s (a, b) VALUES (4, 4);");
+        assertRows(execute("SELECT * FROM %s WHERE a IN (?, ?);", 1, 3),
+                   row(1, 1),
+                   row(3, 3));
+        assertRows(execute("SELECT * FROM %s WHERE token(a)> token(?) and token(a) <= token(?);", 1, 3),
+                   row(2, 2),
+                   row(3, 3));
+        assertRows(execute("SELECT * FROM %s WHERE token(a)= token(2);"),
+                   row(2, 2));
+        assertRows(execute("SELECT * FROM %s WHERE token(a) > token(?) AND token(a) <= token(?) AND a IN (?, ?);",
+                           1, 3, 1, 3),
+                   row(3, 3));
+        assertRows(execute("SELECT * FROM %s WHERE token(a) < token(?) AND token(a) >= token(?) AND a IN (?, ?);",
+                           1, 3, 1, 3),
+                   row(3, 3));
+        assertInvalidMessage("Only EQ and IN relation are supported on the partition key (unless you use the token() function)",
+                             "SELECT * FROM %s WHERE token(a) > token(?) AND token(a) <= token(?) AND a > ?;", 1, 3, 1);
+
+        assertRows(execute("SELECT * FROM %s WHERE token(a) > token(?) AND token(a) <= token(?) AND a IN ?;",
+                           1, 3, Arrays.asList(1, 3)),
+                   row(3, 3));
+        assertRows(execute("SELECT * FROM %s WHERE token(a) > token(?) AND a = ?;", 1, 3),
+                   row(3, 3));
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND token(a) > token(?);", 3, 1),
+                   row(3, 3));
+        assertEmpty(execute("SELECT * FROM %s WHERE token(a) > token(?) AND a = ?;", 3, 1));
+        assertEmpty(execute("SELECT * FROM %s WHERE a = ? AND token(a) > token(?);", 1, 3));
+        assertRows(execute("SELECT * FROM %s WHERE token(a) > token(?) AND a IN (?, ?);", 2, 1, 3),
+                   row(3, 3));
+        assertRows(execute("SELECT * FROM %s WHERE token(a) > token(?) AND token(a) < token(?) AND a IN (?, ?) ;", 2, 5, 1, 3),
+                   row(3, 3));
+        assertRows(execute("SELECT * FROM %s WHERE a IN (?, ?) AND token(a) > token(?) AND token(a) < token(?);", 1, 3, 2, 5),
+                   row(3, 3));
+        assertRows(execute("SELECT * FROM %s WHERE token(a) > token(?) AND a IN (?, ?) AND token(a) < token(?);", 2, 1, 3, 5),
+                   row(3, 3));
+        assertEmpty(execute("SELECT * FROM %s WHERE a IN (?, ?) AND token(a) > token(?);", 1, 3, 3));
+        assertRows(execute("SELECT * FROM %s WHERE token(a) <= token(?) AND a = ?;", 2, 2),
+                   row(2, 2));
+        assertEmpty(execute("SELECT * FROM %s WHERE token(a) <= token(?) AND a = ?;", 2, 3));
+        assertEmpty(execute("SELECT * FROM %s WHERE token(a) = token(?) AND a = ?;", 2, 3));
+        assertRows(execute("SELECT * FROM %s WHERE token(a) >= token(?) AND token(a) <= token(?) AND a = ?;", 2, 2, 2),
+                   row(2, 2));
+        assertEmpty(execute("SELECT * FROM %s WHERE token(a) >= token(?) AND token(a) < token(?) AND a = ?;", 2, 2, 2));
+        assertEmpty(execute("SELECT * FROM %s WHERE token(a) > token(?) AND token(a) <= token(?) AND a = ?;", 2, 2, 2));
+        assertEmpty(execute("SELECT * FROM %s WHERE token(a) > token(?) AND token(a) < token(?) AND a = ?;", 2, 2, 2));
+    }
+
+    @Test
+    public void testMultiColumnPartitionKeyWithTokenNonTokenRestrictionsMix() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, primary key((a, b)))");
+
+        execute("INSERT INTO %s (a, b, c) VALUES (0, 0, 0);");
+        execute("INSERT INTO %s (a, b, c) VALUES (0, 1, 1);");
+        execute("INSERT INTO %s (a, b, c) VALUES (0, 2, 2);");
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 0, 3);");
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 1, 4);");
+
+        assertRows(execute("SELECT * FROM %s WHERE token(a, b) > token(?, ?);", 0, 0),
+                   row(0, 1, 1),
+                   row(0, 2, 2),
+                   row(1, 0, 3),
+                   row(1, 1, 4));
+
+        assertRows(execute("SELECT * FROM %s WHERE token(a, b) > token(?, ?) AND a = ? AND b IN (?, ?);",
+                           0, 0, 1, 0, 1),
+                   row(1, 0, 3),
+                   row(1, 1, 4));
+
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND token(a, b) > token(?, ?) AND b IN (?, ?);",
+                           1, 0, 0, 0, 1),
+                   row(1, 0, 3),
+                   row(1, 1, 4));
+
+        assertRows(execute("SELECT * FROM %s WHERE b IN (?, ?) AND token(a, b) > token(?, ?) AND a = ?;",
+                           0, 1, 0, 0, 1),
+                   row(1, 0, 3),
+                   row(1, 1, 4));
+
+        assertEmpty(execute("SELECT * FROM %s WHERE b IN (?, ?) AND token(a, b) > token(?, ?) AND token(a, b) < token(?, ?) AND a = ?;",
+                            0, 1, 0, 0, 0, 0, 1));
+
+        assertEmpty(execute("SELECT * FROM %s WHERE b IN (?, ?) AND token(a, b) > token(?, ?) AND token(a, b) <= token(?, ?) AND a = ?;",
+                            0, 1, 0, 0, 0, 0, 1));
+
+        assertEmpty(execute("SELECT * FROM %s WHERE b IN (?, ?) AND token(a, b) >= token(?, ?) AND token(a, b) < token(?, ?) AND a = ?;",
+                            0, 1, 0, 0, 0, 0, 1));
+
+        assertEmpty(execute("SELECT * FROM %s WHERE b IN (?, ?) AND token(a, b) = token(?, ?) AND a = ?;",
+                            0, 1, 0, 0, 1));
+
+        assertInvalidMessage("Partition key parts: b must be restricted as other parts are",
+                             "SELECT * FROM %s WHERE token(a, b) > token(?, ?) AND a = ?;", 0, 0, 1);
+    }
+
+    @Test
+    public void testMultiColumnPartitionKeyWithIndexAndTokenNonTokenRestrictionsMix() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, primary key((a, b)))");
+        createIndex("CREATE INDEX ON %s(b)");
+        createIndex("CREATE INDEX ON %s(c)");
+
+        execute("INSERT INTO %s (a, b, c) VALUES (0, 0, 0);");
+        execute("INSERT INTO %s (a, b, c) VALUES (0, 1, 1);");
+        execute("INSERT INTO %s (a, b, c) VALUES (0, 2, 2);");
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 0, 3);");
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 1, 4);");
+
+        assertRows(execute("SELECT * FROM %s WHERE b = ?;", 1),
+                   row(0, 1, 1),
+                   row(1, 1, 4));
+
+        assertRows(execute("SELECT * FROM %s WHERE token(a, b) > token(?, ?) AND b = ?;", 0, 0, 1),
+                   row(0, 1, 1),
+                   row(1, 1, 4));
+
+        assertRows(execute("SELECT * FROM %s WHERE b = ? AND token(a, b) > token(?, ?);", 1, 0, 0),
+                   row(0, 1, 1),
+                   row(1, 1, 4));
+
+        assertRows(execute("SELECT * FROM %s WHERE b = ? AND token(a, b) > token(?, ?) and c = ? ALLOW FILTERING;", 1, 0, 0, 4),
+                   row(1, 1, 4));
     }
 
     @Test
@@ -185,11 +341,11 @@
 
         assertInvalid("SELECT content FROM %s WHERE time2 >= 0 AND author='foo'");
 
-        execute("SELECT blog_id, content FROM %s WHERE time1 > 0 AND author='foo'");
-        execute("SELECT blog_id, content FROM %s WHERE time1 = 1 AND author='foo'");
-        execute("SELECT blog_id, content FROM %s WHERE time1 = 1 AND time2 = 0 AND author='foo'");
-        execute("SELECT content FROM %s WHERE time1 = 1 AND time2 = 1 AND author='foo'");
-        execute("SELECT content FROM %s WHERE time1 = 1 AND time2 > 0 AND author='foo'");
+        assertInvalid("SELECT blog_id, content FROM %s WHERE time1 > 0 AND author='foo'");
+        assertInvalid("SELECT blog_id, content FROM %s WHERE time1 = 1 AND author='foo'");
+        assertInvalid("SELECT blog_id, content FROM %s WHERE time1 = 1 AND time2 = 0 AND author='foo'");
+        assertInvalid("SELECT content FROM %s WHERE time1 = 1 AND time2 = 1 AND author='foo'");
+        assertInvalid("SELECT content FROM %s WHERE time1 = 1 AND time2 > 0 AND author='foo'");
     }
 
     /**
@@ -342,4 +498,14 @@
 
         assertInvalid("SELECT * FROM %s WHERE k >= -1 AND k < 1");
     }
+
+    @Test
+    public void testTokenFunctionWithInvalidColumnNames() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY ((a, b), c))");
+        assertInvalidMessage("Undefined name e in where clause ('token(a, e) = token(0, 0)')", "SELECT * FROM %s WHERE token(a, e) = token(0, 0)");
+        assertInvalidMessage("Undefined name e in where clause ('token(a, e) > token(0, 1)')", "SELECT * FROM %s WHERE token(a, e) > token(0, 1)");
+        assertInvalidMessage("Aliases aren't allowed in the where clause ('token(a, e) = token(0, 0)')", "SELECT b AS e FROM %s WHERE token(a, e) = token(0, 0)");
+        assertInvalidMessage("Aliases aren't allowed in the where clause ('token(a, e) > token(0, 1)')", "SELECT b AS e FROM %s WHERE token(a, e) > token(0, 1)");
+    }
 }
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java
index 9896d0a..f8e5a28 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java
@@ -17,12 +17,14 @@
  */
 package org.apache.cassandra.cql3.validation.operations;
 
-import org.junit.Test;
-
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 
 import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.restrictions.StatementRestrictions;
+
+import org.junit.Test;
 
 public class SelectSingleColumnRelationTest extends CQLTester
 {
@@ -34,9 +36,12 @@
         createIndex("CREATE INDEX ON %s (c)");
         createIndex("CREATE INDEX ON %s (d)");
 
-        assertInvalid("SELECT * FROM %s WHERE a = 0 AND b=?", set(0));
-        assertInvalid("SELECT * FROM %s WHERE a = 0 AND c=?", list(0));
-        assertInvalid("SELECT * FROM %s WHERE a = 0 AND d=?", map(0, 0));
+        assertInvalidMessage("Collection column 'b' (set<int>) cannot be restricted by a '=' relation",
+                             "SELECT * FROM %s WHERE a = 0 AND b=?", set(0));
+        assertInvalidMessage("Collection column 'c' (list<int>) cannot be restricted by a '=' relation",
+                             "SELECT * FROM %s WHERE a = 0 AND c=?", list(0));
+        assertInvalidMessage("Collection column 'd' (map<int, int>) cannot be restricted by a '=' relation",
+                             "SELECT * FROM %s WHERE a = 0 AND d=?", map(0, 0));
     }
 
     @Test
@@ -47,39 +52,385 @@
         execute("INSERT INTO %s (a, b, c) VALUES (0, {0}, 0)");
 
         // non-EQ operators
-        assertInvalid("SELECT * FROM %s WHERE c = 0 AND b > ?", set(0));
-        assertInvalid("SELECT * FROM %s WHERE c = 0 AND b >= ?", set(0));
-        assertInvalid("SELECT * FROM %s WHERE c = 0 AND b < ?", set(0));
-        assertInvalid("SELECT * FROM %s WHERE c = 0 AND b <= ?", set(0));
-        assertInvalid("SELECT * FROM %s WHERE c = 0 AND b IN (?)", set(0));
+        assertInvalidMessage("Collection column 'b' (set<int>) cannot be restricted by a '>' relation",
+                             "SELECT * FROM %s WHERE c = 0 AND b > ?", set(0));
+        assertInvalidMessage("Collection column 'b' (set<int>) cannot be restricted by a '>=' relation",
+                             "SELECT * FROM %s WHERE c = 0 AND b >= ?", set(0));
+        assertInvalidMessage("Collection column 'b' (set<int>) cannot be restricted by a '<' relation",
+                             "SELECT * FROM %s WHERE c = 0 AND b < ?", set(0));
+        assertInvalidMessage("Collection column 'b' (set<int>) cannot be restricted by a '<=' relation",
+                             "SELECT * FROM %s WHERE c = 0 AND b <= ?", set(0));
+        assertInvalidMessage("Collection column 'b' (set<int>) cannot be restricted by a 'IN' relation",
+                             "SELECT * FROM %s WHERE c = 0 AND b IN (?)", set(0));
     }
 
     @Test
-    public void testInvalidSliceRestrictionOnPartitionKey() throws Throwable
+    public void testClusteringColumnRelations() throws Throwable
     {
-        createTable("CREATE TABLE %s (a int PRIMARY KEY, b int, c text)");
-        assertInvalidMessage("Only EQ and IN relation are supported on the partition key (unless you use the token() function)",
-                             "SELECT * FROM %s WHERE a >= 1 and a < 4");
-        assertInvalidMessage("Multi-column relations can only be applied to clustering columns: a",
-                             "SELECT * FROM %s WHERE (a) >= (1) and (a) < (4)");
+        createTable("CREATE TABLE %s (a text, b int, c int, d int, primary key(a, b, c))");
+        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 1, 5, 1);
+        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 2, 6, 2);
+        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 3, 7, 3);
+        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "second", 4, 8, 4);
+
+        testSelectQueriesWithClusteringColumnRelations();
     }
 
     @Test
-    public void testInvalidMulticolumnSliceRestrictionOnPartitionKey() throws Throwable
+    public void testClusteringColumnRelationsWithCompactStorage() throws Throwable
     {
-        createTable("CREATE TABLE %s (a int, b int, c text, PRIMARY KEY ((a, b)))");
-        assertInvalidMessage("Multi-column relations can only be applied to clustering columns: a",
-                             "SELECT * FROM %s WHERE (a, b) >= (1, 1) and (a, b) < (4, 1)");
-        assertInvalidMessage("Multi-column relations can only be applied to clustering columns: a",
-                             "SELECT * FROM %s WHERE a >= 1 and (a, b) < (4, 1)");
-        assertInvalidMessage("Multi-column relations can only be applied to clustering columns: a",
-                             "SELECT * FROM %s WHERE b >= 1 and (a, b) < (4, 1)");
-        assertInvalidMessage("Multi-column relations can only be applied to clustering columns: a",
-                             "SELECT * FROM %s WHERE (a, b) >= (1, 1) and (b) < (4)");
-        assertInvalidMessage("Multi-column relations can only be applied to clustering columns: b",
-                             "SELECT * FROM %s WHERE (b) < (4) and (a, b) >= (1, 1)");
-        assertInvalidMessage("Multi-column relations can only be applied to clustering columns: a",
-                             "SELECT * FROM %s WHERE (a, b) >= (1, 1) and a = 1");
+        createTable("CREATE TABLE %s (a text, b int, c int, d int, primary key(a, b, c)) WITH COMPACT STORAGE;");
+        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 1, 5, 1);
+        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 2, 6, 2);
+        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 3, 7, 3);
+        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "second", 4, 8, 4);
+
+        testSelectQueriesWithClusteringColumnRelations();
+    }
+
+    private void testSelectQueriesWithClusteringColumnRelations() throws Throwable
+    {
+        assertRows(execute("select * from %s where a in (?, ?)", "first", "second"),
+                   row("first", 1, 5, 1),
+                   row("first", 2, 6, 2),
+                   row("first", 3, 7, 3),
+                   row("second", 4, 8, 4));
+
+        assertRows(execute("select * from %s where a = ? and b = ? and c in (?, ?)", "first", 2, 6, 7),
+                   row("first", 2, 6, 2));
+
+        assertRows(execute("select * from %s where a = ? and b in (?, ?) and c in (?, ?)", "first", 2, 3, 6, 7),
+                   row("first", 2, 6, 2),
+                   row("first", 3, 7, 3));
+
+        assertRows(execute("select * from %s where a = ? and b in (?, ?) and c in (?, ?)", "first", 3, 2, 7, 6),
+                   row("first", 2, 6, 2),
+                   row("first", 3, 7, 3));
+
+        assertRows(execute("select * from %s where a = ? and c in (?, ?) and b in (?, ?)", "first", 7, 6, 3, 2),
+                   row("first", 2, 6, 2),
+                   row("first", 3, 7, 3));
+
+        assertRows(execute("select c, d from %s where a = ? and c in (?, ?) and b in (?, ?)", "first", 7, 6, 3, 2),
+                   row(6, 2),
+                   row(7, 3));
+
+        assertRows(execute("select c, d from %s where a = ? and c in (?, ?) and b in (?, ?, ?)", "first", 7, 6, 3, 2, 3),
+                   row(6, 2),
+                   row(7, 3));
+
+        assertRows(execute("select * from %s where a = ? and b in (?, ?) and c = ?", "first", 3, 2, 7),
+                   row("first", 3, 7, 3));
+
+        assertRows(execute("select * from %s where a = ? and b in ? and c in ?",
+                           "first", Arrays.asList(3, 2), Arrays.asList(7, 6)),
+                   row("first", 2, 6, 2),
+                   row("first", 3, 7, 3));
+
+        assertInvalidMessage("Invalid null value for column b",
+                             "select * from %s where a = ? and b in ? and c in ?", "first", null, Arrays.asList(7, 6));
+
+        assertRows(execute("select * from %s where a = ? and c >= ? and b in (?, ?)", "first", 6, 3, 2),
+                   row("first", 2, 6, 2),
+                   row("first", 3, 7, 3));
+
+        assertRows(execute("select * from %s where a = ? and c > ? and b in (?, ?)", "first", 6, 3, 2),
+                   row("first", 3, 7, 3));
+
+        assertRows(execute("select * from %s where a = ? and c <= ? and b in (?, ?)", "first", 6, 3, 2),
+                   row("first", 2, 6, 2));
+
+        assertRows(execute("select * from %s where a = ? and c < ? and b in (?, ?)", "first", 7, 3, 2),
+                   row("first", 2, 6, 2));
+//---
+        assertRows(execute("select * from %s where a = ? and c >= ? and c <= ? and b in (?, ?)", "first", 6, 7, 3, 2),
+                   row("first", 2, 6, 2),
+                   row("first", 3, 7, 3));
+
+        assertRows(execute("select * from %s where a = ? and c > ? and c <= ? and b in (?, ?)", "first", 6, 7, 3, 2),
+                   row("first", 3, 7, 3));
+
+        assertEmpty(execute("select * from %s where a = ? and c > ? and c < ? and b in (?, ?)", "first", 6, 7, 3, 2));
+
+        assertInvalidMessage("Column \"c\" cannot be restricted by both an equality and an inequality relation",
+                             "select * from %s where a = ? and c > ? and c = ? and b in (?, ?)", "first", 6, 7, 3, 2);
+
+        assertInvalidMessage("c cannot be restricted by more than one relation if it includes an Equal",
+                             "select * from %s where a = ? and c = ? and c > ?  and b in (?, ?)", "first", 6, 7, 3, 2);
+
+        assertRows(execute("select * from %s where a = ? and c in (?, ?) and b in (?, ?) order by b DESC",
+                           "first", 7, 6, 3, 2),
+                   row("first", 3, 7, 3),
+                   row("first", 2, 6, 2));
+
+        assertInvalidMessage("More than one restriction was found for the start bound on b",
+                             "select * from %s where a = ? and b > ? and b > ?", "first", 6, 3, 2);
+
+        assertInvalidMessage("More than one restriction was found for the end bound on b",
+                             "select * from %s where a = ? and b < ? and b <= ?", "first", 6, 3, 2);
+    }
+
+    @Test
+    public void testPartitionKeyColumnRelations() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a text, b int, c int, d int, primary key((a, b), c))");
+        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 1, 1, 1);
+        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 2, 2, 2);
+        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 3, 3, 3);
+        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 4, 4, 4);
+        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "second", 1, 1, 1);
+        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "second", 4, 4, 4);
+
+        assertRows(execute("select * from %s where a = ? and b = ?", "first", 2),
+                   row("first", 2, 2, 2));
+
+        assertRows(execute("select * from %s where a in (?, ?) and b in (?, ?)", "first", "second", 2, 3),
+                   row("first", 2, 2, 2),
+                   row("first", 3, 3, 3));
+
+        assertRows(execute("select * from %s where a in (?, ?) and b = ?", "first", "second", 4),
+                   row("first", 4, 4, 4),
+                   row("second", 4, 4, 4));
+
+        assertRows(execute("select * from %s where a = ? and b in (?, ?)", "first", 3, 4),
+                   row("first", 3, 3, 3),
+                   row("first", 4, 4, 4));
+
+        assertRows(execute("select * from %s where a in (?, ?) and b in (?, ?)", "first", "second", 1, 4),
+                   row("first", 1, 1, 1),
+                   row("first", 4, 4, 4),
+                   row("second", 1, 1, 1),
+                   row("second", 4, 4, 4));
+
+        assertInvalidMessage("Partition key parts: b must be restricted as other parts are",
+                             "select * from %s where a in (?, ?)", "first", "second");
+        assertInvalidMessage("Partition key parts: b must be restricted as other parts are",
+                             "select * from %s where a = ?", "first");
+        assertInvalidMessage("b cannot be restricted by more than one relation if it includes a IN",
+                             "select * from %s where a = ? AND b IN (?, ?) AND b = ?", "first", 2, 2, 3);
+        assertInvalidMessage("b cannot be restricted by more than one relation if it includes an Equal",
+                             "select * from %s where a = ? AND b = ? AND b IN (?, ?)", "first", 2, 2, 3);
+        assertInvalidMessage("a cannot be restricted by more than one relation if it includes a IN",
+                             "select * from %s where a IN (?, ?) AND a = ? AND b = ?", "first", "second", "first", 3);
+        assertInvalidMessage("a cannot be restricted by more than one relation if it includes an Equal",
+                             "select * from %s where a = ? AND a IN (?, ?) AND b IN (?, ?)", "first", "second", "first", 2, 3);
+    }
+
+    @Test
+    public void testClusteringColumnRelationsWithClusteringOrder() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a text, b int, c int, d int, primary key(a, b, c)) WITH CLUSTERING ORDER BY (b DESC);");
+        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 1, 5, 1);
+        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 2, 6, 2);
+        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "first", 3, 7, 3);
+        execute("insert into %s (a, b, c, d) values (?, ?, ?, ?)", "second", 4, 8, 4);
+
+        assertRows(execute("select * from %s where a = ? and c in (?, ?) and b in (?, ?) order by b DESC",
+                           "first", 7, 6, 3, 2),
+                   row("first", 3, 7, 3),
+                   row("first", 2, 6, 2));
+
+        assertRows(execute("select * from %s where a = ? and c in (?, ?) and b in (?, ?) order by b ASC",
+                           "first", 7, 6, 3, 2),
+                   row("first", 2, 6, 2),
+                   row("first", 3, 7, 3));
+    }
+
+    @Test
+    public void testAllowFilteringWithClusteringColumn() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY (k, c))");
+
+        execute("INSERT INTO %s (k, c, v) VALUES(?, ?, ?)", 1, 2, 1);
+        execute("INSERT INTO %s (k, c, v) VALUES(?, ?, ?)", 1, 3, 2);
+        execute("INSERT INTO %s (k, c, v) VALUES(?, ?, ?)", 2, 2, 3);
+
+        // Don't require filtering, always allowed
+        assertRows(execute("SELECT * FROM %s WHERE k = ?", 1),
+                   row(1, 2, 1),
+                   row(1, 3, 2));
+
+        assertRows(execute("SELECT * FROM %s WHERE k = ? AND c > ?", 1, 2), row(1, 3, 2));
+
+        assertRows(execute("SELECT * FROM %s WHERE k = ? AND c = ?", 1, 2), row(1, 2, 1));
+
+        assertRows(execute("SELECT * FROM %s WHERE k = ? ALLOW FILTERING", 1),
+                   row(1, 2, 1),
+                   row(1, 3, 2));
+
+        assertRows(execute("SELECT * FROM %s WHERE k = ? AND c > ? ALLOW FILTERING", 1, 2), row(1, 3, 2));
+
+        assertRows(execute("SELECT * FROM %s WHERE k = ? AND c = ? ALLOW FILTERING", 1, 2), row(1, 2, 1));
+
+        // Require filtering, allowed only with ALLOW FILTERING
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE c = ?", 2);
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE c > ? AND c <= ?", 2, 4);
+
+        assertRows(execute("SELECT * FROM %s WHERE c = ? ALLOW FILTERING", 2),
+                   row(1, 2, 1),
+                   row(2, 2, 3));
+
+        assertRows(execute("SELECT * FROM %s WHERE c > ? AND c <= ? ALLOW FILTERING", 2, 4), row(1, 3, 2));
+    }
+
+    @Test
+    public void testAllowFilteringWithIndexedColumn() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, a int, b int)");
+        createIndex("CREATE INDEX ON %s(a)");
+
+        execute("INSERT INTO %s(k, a, b) VALUES(?, ?, ?)", 1, 10, 100);
+        execute("INSERT INTO %s(k, a, b) VALUES(?, ?, ?)", 2, 20, 200);
+        execute("INSERT INTO %s(k, a, b) VALUES(?, ?, ?)", 3, 30, 300);
+        execute("INSERT INTO %s(k, a, b) VALUES(?, ?, ?)", 4, 40, 400);
+
+        // Don't require filtering, always allowed
+        assertRows(execute("SELECT * FROM %s WHERE k = ?", 1), row(1, 10, 100));
+        assertRows(execute("SELECT * FROM %s WHERE a = ?", 20), row(2, 20, 200));
+        assertRows(execute("SELECT * FROM %s WHERE k = ? ALLOW FILTERING", 1), row(1, 10, 100));
+        assertRows(execute("SELECT * FROM %s WHERE a = ? ALLOW FILTERING", 20), row(2, 20, 200));
+
+        assertInvalid("SELECT * FROM %s WHERE a = ? AND b = ?");
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND b = ? ALLOW FILTERING", 20, 200), row(2, 20, 200));
+    }
+
+    @Test
+    public void testIndexQueriesOnComplexPrimaryKey() throws Throwable
+    {
+        createTable("CREATE TABLE %s (pk0 int, pk1 int, ck0 int, ck1 int, ck2 int, value int, PRIMARY KEY ((pk0, pk1), ck0, ck1, ck2))");
+
+        createIndex("CREATE INDEX ON %s (ck1)");
+        createIndex("CREATE INDEX ON %s (ck2)");
+        createIndex("CREATE INDEX ON %s (pk0)");
+        createIndex("CREATE INDEX ON %s (ck0)");
+
+        execute("INSERT INTO %s (pk0, pk1, ck0, ck1, ck2, value) VALUES (?, ?, ?, ?, ?, ?)", 0, 1, 2, 3, 4, 5);
+        execute("INSERT INTO %s (pk0, pk1, ck0, ck1, ck2, value) VALUES (?, ?, ?, ?, ?, ?)", 1, 2, 3, 4, 5, 0);
+        execute("INSERT INTO %s (pk0, pk1, ck0, ck1, ck2, value) VALUES (?, ?, ?, ?, ?, ?)", 2, 3, 4, 5, 0, 1);
+        execute("INSERT INTO %s (pk0, pk1, ck0, ck1, ck2, value) VALUES (?, ?, ?, ?, ?, ?)", 3, 4, 5, 0, 1, 2);
+        execute("INSERT INTO %s (pk0, pk1, ck0, ck1, ck2, value) VALUES (?, ?, ?, ?, ?, ?)", 4, 5, 0, 1, 2, 3);
+        execute("INSERT INTO %s (pk0, pk1, ck0, ck1, ck2, value) VALUES (?, ?, ?, ?, ?, ?)", 5, 0, 1, 2, 3, 4);
+
+        assertRows(execute("SELECT value FROM %s WHERE pk0 = 2"), row(1));
+        assertRows(execute("SELECT value FROM %s WHERE ck0 = 0"), row(3));
+        assertRows(execute("SELECT value FROM %s WHERE pk0 = 3 AND pk1 = 4 AND ck1 = 0"), row(2));
+        assertRows(execute("SELECT value FROM %s WHERE pk0 = 5 AND pk1 = 0 AND ck0 = 1 AND ck2 = 3 ALLOW FILTERING"), row(4));
+    }
+
+    @Test
+    public void testIndexOnClusteringColumns() throws Throwable
+    {
+        createTable("CREATE TABLE %s (id1 int, id2 int, author text, time bigint, v1 text, v2 text, PRIMARY KEY ((id1, id2), author, time))");
+        createIndex("CREATE INDEX ON %s(time)");
+        createIndex("CREATE INDEX ON %s(id2)");
+
+        execute("INSERT INTO %s(id1, id2, author, time, v1, v2) VALUES(0, 0, 'bob', 0, 'A', 'A')");
+        execute("INSERT INTO %s(id1, id2, author, time, v1, v2) VALUES(0, 0, 'bob', 1, 'B', 'B')");
+        execute("INSERT INTO %s(id1, id2, author, time, v1, v2) VALUES(0, 1, 'bob', 2, 'C', 'C')");
+        execute("INSERT INTO %s(id1, id2, author, time, v1, v2) VALUES(0, 0, 'tom', 0, 'D', 'D')");
+        execute("INSERT INTO %s(id1, id2, author, time, v1, v2) VALUES(0, 1, 'tom', 1, 'E', 'E')");
+
+        assertRows(execute("SELECT v1 FROM %s WHERE time = 1"), row("B"), row("E"));
+
+        assertRows(execute("SELECT v1 FROM %s WHERE id2 = 1"), row("C"), row("E"));
+
+        assertRows(execute("SELECT v1 FROM %s WHERE id1 = 0 AND id2 = 0 AND author = 'bob' AND time = 0"), row("A"));
+
+        // Test for CASSANDRA-8206
+        execute("UPDATE %s SET v2 = null WHERE id1 = 0 AND id2 = 0 AND author = 'bob' AND time = 1");
+
+        assertRows(execute("SELECT v1 FROM %s WHERE id2 = 0"), row("A"), row("B"), row("D"));
+
+        assertRows(execute("SELECT v1 FROM %s WHERE time = 1"), row("B"), row("E"));
+
+        assertInvalidMessage("IN restrictions are not supported on indexed columns",
+                             "SELECT v1 FROM %s WHERE id2 = 0 and time IN (1, 2) ALLOW FILTERING");
+
+        assertRows(execute("SELECT v1 FROM %s WHERE author > 'ted' AND time = 1 ALLOW FILTERING"), row("E"));
+        assertRows(execute("SELECT v1 FROM %s WHERE author > 'amy' AND author < 'zoe' AND time = 0 ALLOW FILTERING"),
+                           row("A"), row("D"));
+    }
+
+    @Test
+    public void testCompositeIndexWithPrimaryKey() throws Throwable
+    {
+        createTable("CREATE TABLE %s (blog_id int, time1 int, time2 int, author text, content text, PRIMARY KEY (blog_id, time1, time2))");
+
+        createIndex("CREATE INDEX ON %s(author)");
+
+        String req = "INSERT INTO %s (blog_id, time1, time2, author, content) VALUES (?, ?, ?, ?, ?)";
+        execute(req, 1, 0, 0, "foo", "bar1");
+        execute(req, 1, 0, 1, "foo", "bar2");
+        execute(req, 2, 1, 0, "foo", "baz");
+        execute(req, 3, 0, 1, "gux", "qux");
+
+        assertRows(execute("SELECT blog_id, content FROM %s WHERE author='foo'"),
+                   row(1, "bar1"),
+                   row(1, "bar2"),
+                   row(2, "baz"));
+        assertRows(execute("SELECT blog_id, content FROM %s WHERE time1 > 0 AND author='foo' ALLOW FILTERING"), row(2, "baz"));
+        assertRows(execute("SELECT blog_id, content FROM %s WHERE time1 = 1 AND author='foo' ALLOW FILTERING"), row(2, "baz"));
+        assertRows(execute("SELECT blog_id, content FROM %s WHERE time1 = 1 AND time2 = 0 AND author='foo' ALLOW FILTERING"),
+                   row(2, "baz"));
+        assertEmpty(execute("SELECT content FROM %s WHERE time1 = 1 AND time2 = 1 AND author='foo' ALLOW FILTERING"));
+        assertEmpty(execute("SELECT content FROM %s WHERE time1 = 1 AND time2 > 0 AND author='foo' ALLOW FILTERING"));
+
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT content FROM %s WHERE time2 >= 0 AND author='foo'");
+    }
+
+    @Test
+    public void testRangeQueryOnIndex() throws Throwable
+    {
+        createTable("CREATE TABLE %s (id int primary key, row int, setid int);");
+        createIndex("CREATE INDEX ON %s (setid)");
+
+        String q = "INSERT INTO %s (id, row, setid) VALUES (?, ?, ?);";
+        execute(q, 0, 0, 0);
+        execute(q, 1, 1, 0);
+        execute(q, 2, 2, 0);
+        execute(q, 3, 3, 0);
+
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE setid = 0 AND row < 1;");
+        assertRows(execute("SELECT * FROM %s WHERE setid = 0 AND row < 1 ALLOW FILTERING;"), row(0, 0, 0));
+    }
+
+    @Test
+    public void testEmptyIN() throws Throwable
+    {
+        for (String compactOption : new String[] { "", " WITH COMPACT STORAGE" })
+        {
+            createTable("CREATE TABLE %s (k1 int, k2 int, v int, PRIMARY KEY (k1, k2))" + compactOption);
+
+            for (int i = 0; i <= 2; i++)
+                for (int j = 0; j <= 2; j++)
+                    execute("INSERT INTO %s (k1, k2, v) VALUES (?, ?, ?)", i, j, i + j);
+
+            assertEmpty(execute("SELECT v FROM %s WHERE k1 IN ()"));
+            assertEmpty(execute("SELECT v FROM %s WHERE k1 = 0 AND k2 IN ()"));
+        }
+    }
+
+    @Test
+    public void testINWithDuplicateValue() throws Throwable
+    {
+        for (String compactOption : new String[] { "", " WITH COMPACT STORAGE" })
+        {
+            createTable("CREATE TABLE %s (k1 int, k2 int, v int, PRIMARY KEY (k1, k2))" + compactOption);
+            execute("INSERT INTO %s (k1,  k2, v) VALUES (?, ?, ?)", 1, 1, 1);
+
+            assertRows(execute("SELECT * FROM %s WHERE k1 IN (?, ?)", 1, 1),
+                       row(1, 1, 1));
+
+            assertRows(execute("SELECT * FROM %s WHERE k1 IN (?, ?) AND k2 IN (?, ?)", 1, 1, 1, 1),
+                       row(1, 1, 1));
+
+            assertRows(execute("SELECT * FROM %s WHERE k1 = ? AND k2 IN (?, ?)", 1, 1, 1),
+                       row(1, 1, 1));
+        }
     }
 
     @Test
@@ -91,8 +442,7 @@
         for (int i = 0; i < 10000; i++)
             inValues.add(i);
         assertRows(execute("SELECT * FROM %s WHERE k=? AND c IN ?", 0, inValues),
-                row(0, 0, 0)
-        );
+                row(0, 0, 0));
     }
 
     @Test
@@ -112,18 +462,46 @@
 
         execute("INSERT INTO %s (a, b, c, d, e, f) VALUES (?, ?, ?, ?, ?, ?)", 0, 0, 2, 0, 0, 5);
 
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE a = ? AND c = ?", 0, 1);
         assertRows(execute("SELECT * FROM %s WHERE a = ? AND c = ? ALLOW FILTERING", 0, 1),
                    row(0, 0, 1, 0, 0, 3),
                    row(0, 0, 1, 1, 0, 4),
                    row(0, 0, 1, 1, 1, 5));
 
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE a = ? AND c = ? AND d = ?", 0, 1, 1);
         assertRows(execute("SELECT * FROM %s WHERE a = ? AND c = ? AND d = ? ALLOW FILTERING", 0, 1, 1),
                    row(0, 0, 1, 1, 0, 4),
                    row(0, 0, 1, 1, 1, 5));
 
-        assertInvalidMessage("Partition key part b must be restricted since preceding part is",
+        assertInvalidMessage("Partition key parts: b must be restricted as other parts are",
+                             "SELECT * FROM %s WHERE a = ? AND c IN (?) AND  d IN (?) ALLOW FILTERING", 0, 1, 1);
+
+        assertInvalidMessage("Partition key parts: b must be restricted as other parts are",
+                             "SELECT * FROM %s WHERE a = ? AND (c, d) >= (?, ?) ALLOW FILTERING", 0, 1, 1);
+
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE a = ? AND c IN (?) AND f = ?", 0, 1, 5);
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND c IN (?) AND f = ? ALLOW FILTERING", 0, 1, 5),
+                   row(0, 0, 1, 1, 1, 5));
+
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE a = ? AND c IN (?, ?) AND f = ?", 0, 1, 2, 5);
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND c IN (?, ?) AND f = ? ALLOW FILTERING", 0, 1, 2, 5),
+                   row(0, 0, 1, 1, 1, 5),
+                   row(0, 0, 2, 0, 0, 5));
+
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE a = ? AND c IN (?) AND d IN (?) AND f = ?", 0, 1, 0, 3);
+        assertRows(execute("SELECT * FROM %s WHERE a = ? AND c IN (?) AND d IN (?) AND f = ? ALLOW FILTERING", 0, 1, 0, 3),
+                   row(0, 0, 1, 0, 0, 3));
+
+        assertInvalidMessage("Partition key parts: b must be restricted as other parts are",
                              "SELECT * FROM %s WHERE a = ? AND c >= ? ALLOW FILTERING", 0, 1);
 
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE a = ? AND c >= ? AND f = ?", 0, 1, 5);
         assertRows(execute("SELECT * FROM %s WHERE a = ? AND b = ? AND c >= ? AND f = ?", 0, 0, 1, 5),
                    row(0, 0, 1, 1, 1, 5),
                    row(0, 0, 2, 0, 0, 5));
@@ -132,13 +510,103 @@
                    row(0, 0, 1, 1, 1, 5),
                    row(0, 0, 2, 0, 0, 5));
 
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE a = ? AND c = ? AND d >= ? AND f = ?", 0, 1, 1, 5);
+
         assertRows(execute("SELECT * FROM %s WHERE a = ? AND b = ? AND c = ? AND d >= ? AND f = ?", 0, 0, 1, 1, 5),
                    row(0, 0, 1, 1, 1, 5));
 
         assertRows(execute("SELECT * FROM %s WHERE a = ? AND c = ? AND d >= ? AND f = ? ALLOW FILTERING", 0, 1, 1, 5),
                    row(0, 0, 1, 1, 1, 5));
+    }
 
-        assertInvalidMessage("Cannot execute this query as it might involve data filtering and thus may have unpredictable performance. If you want to execute this query despite the performance unpredictability, use ALLOW FILTERING",
-                             "SELECT * FROM %s WHERE a = ? AND d >= ? AND f = ?", 0, 1, 5);
+    @Test
+    public void testFunctionCallWithUnset() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, s text, i int)");
+
+        assertInvalidMessage("Invalid unset value for argument in call to function token",
+                             "SELECT * FROM %s WHERE token(k) >= token(?)", unset());
+        assertInvalidMessage("Invalid unset value for argument in call to function blobasint",
+                             "SELECT * FROM %s WHERE k = blobAsInt(?)", unset());
+    }
+
+    @Test
+    public void testLimitWithUnset() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, i int)");
+        execute("INSERT INTO %s (k, i) VALUES (1, 1)");
+        execute("INSERT INTO %s (k, i) VALUES (2, 1)");
+        assertRows(execute("SELECT k FROM %s LIMIT ?", unset()), // treat as 'unlimited'
+                row(1),
+                row(2)
+        );
+    }
+
+    @Test
+    public void testWithUnsetValues() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, i int, j int, s text, PRIMARY KEY(k,i,j))");
+        createIndex("CREATE INDEX s_index ON %s (s)");
+        // partition key
+        assertInvalidMessage("Invalid unset value for column k", "SELECT * from %s WHERE k = ?", unset());
+        assertInvalidMessage("Invalid unset value for column k", "SELECT * from %s WHERE k IN ?", unset());
+        assertInvalidMessage("Invalid unset value for column k", "SELECT * from %s WHERE k IN(?)", unset());
+        assertInvalidMessage("Invalid unset value for column k", "SELECT * from %s WHERE k IN(?,?)", 1, unset());
+        // clustering column
+        assertInvalidMessage("Invalid unset value for column i", "SELECT * from %s WHERE k = 1 AND i = ?", unset());
+        assertInvalidMessage("Invalid unset value for column i", "SELECT * from %s WHERE k = 1 AND i IN ?", unset());
+        assertInvalidMessage("Invalid unset value for column i", "SELECT * from %s WHERE k = 1 AND i IN(?)", unset());
+        assertInvalidMessage("Invalid unset value for column i", "SELECT * from %s WHERE k = 1 AND i IN(?,?)", 1, unset());
+        assertInvalidMessage("Invalid unset value for column i", "SELECT * from %s WHERE i = ? ALLOW FILTERING", unset());
+        // indexed column
+        assertInvalidMessage("Unsupported unset value for column s", "SELECT * from %s WHERE s = ?", unset());
+        // range
+        assertInvalidMessage("Invalid unset value for column i", "SELECT * from %s WHERE k = 1 AND i > ?", unset());
+    }
+
+    @Test
+    public void testInvalidSliceRestrictionOnPartitionKey() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b int, c text)");
+        assertInvalidMessage("Only EQ and IN relation are supported on the partition key (unless you use the token() function)",
+                             "SELECT * FROM %s WHERE a >= 1 and a < 4");
+        assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: a",
+                             "SELECT * FROM %s WHERE (a) >= (1) and (a) < (4)");
+    }
+
+    @Test
+    public void testInvalidMulticolumnSliceRestrictionOnPartitionKey() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c text, PRIMARY KEY ((a, b)))");
+        assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: a",
+                             "SELECT * FROM %s WHERE (a, b) >= (1, 1) and (a, b) < (4, 1)");
+        assertInvalidMessage("Only EQ and IN relation are supported on the partition key (unless you use the token() function)",
+                             "SELECT * FROM %s WHERE a >= 1 and (a, b) < (4, 1)");
+        assertInvalidMessage("Only EQ and IN relation are supported on the partition key (unless you use the token() function)",
+                             "SELECT * FROM %s WHERE b >= 1 and (a, b) < (4, 1)");
+        assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: a",
+                             "SELECT * FROM %s WHERE (a, b) >= (1, 1) and (b) < (4)");
+        assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: b",
+                             "SELECT * FROM %s WHERE (b) < (4) and (a, b) >= (1, 1)");
+        assertInvalidMessage("Multi-column relations can only be applied to clustering columns but was applied to: a",
+                             "SELECT * FROM %s WHERE (a, b) >= (1, 1) and a = 1");
+    }
+
+    @Test
+    public void testInvalidColumnNames() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c map<int, int>, PRIMARY KEY (a, b))");
+        assertInvalidMessage("Undefined name d in where clause ('d = 0')", "SELECT * FROM %s WHERE d = 0");
+        assertInvalidMessage("Undefined name d in where clause ('d IN [0, 1]')", "SELECT * FROM %s WHERE d IN (0, 1)");
+        assertInvalidMessage("Undefined name d in where clause ('d > 0')", "SELECT * FROM %s WHERE d > 0 and d <= 2");
+        assertInvalidMessage("Undefined name d in where clause ('d CONTAINS 0')", "SELECT * FROM %s WHERE d CONTAINS 0");
+        assertInvalidMessage("Undefined name d in where clause ('d CONTAINS KEY 0')", "SELECT * FROM %s WHERE d CONTAINS KEY 0");
+        assertInvalidMessage("Aliases aren't allowed in the where clause ('d = 0')", "SELECT a AS d FROM %s WHERE d = 0");
+        assertInvalidMessage("Aliases aren't allowed in the where clause ('d IN [0, 1]')", "SELECT b AS d FROM %s WHERE d IN (0, 1)");
+        assertInvalidMessage("Aliases aren't allowed in the where clause ('d > 0')", "SELECT b AS d FROM %s WHERE d > 0 and d <= 2");
+        assertInvalidMessage("Aliases aren't allowed in the where clause ('d CONTAINS 0')", "SELECT c AS d FROM %s WHERE d CONTAINS 0");
+        assertInvalidMessage("Aliases aren't allowed in the where clause ('d CONTAINS KEY 0')", "SELECT c AS d FROM %s WHERE d CONTAINS KEY 0");
+        assertInvalidMessage("Undefined name d in selection clause", "SELECT d FROM %s WHERE a = 0");
     }
 }
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java
index cef4635..ac1ba4c 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java
@@ -20,18 +20,18 @@
 import java.nio.ByteBuffer;
 import java.util.UUID;
 
-import org.junit.BeforeClass;
 import org.junit.Test;
 
 import junit.framework.Assert;
-import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.UntypedResultSet;
-import org.apache.cassandra.cql3.CQLTester;
-import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.cql3.restrictions.StatementRestrictions;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 
+import org.apache.cassandra.cql3.CQLTester;
+
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+
 import static org.apache.cassandra.utils.ByteBufferUtil.EMPTY_BYTE_BUFFER;
 import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
 
@@ -40,11 +40,6 @@
  */
 public class SelectTest extends CQLTester
 {
-    @BeforeClass
-    public static void setUp()
-    {
-        DatabaseDescriptor.setPartitioner(new Murmur3Partitioner());
-    }
 
     @Test
     public void testSingleClustering() throws Throwable
@@ -338,7 +333,7 @@
         for (int i = 0; i < 10; i++)
             execute("INSERT INTO %s (k, c1, c2, v) VALUES (0, 0, ?, ?)", i, i);
 
-        assertInvalid("SELECT v FROM %s WHERE k = 0 AND c1 IN (5, 2, 8) AND c2 = 3");
+        assertEmpty(execute("SELECT v FROM %s WHERE k = 0 AND c1 IN (5, 2, 8) AND c2 = 3"));
 
         assertRows(execute("SELECT v FROM %s WHERE k = 0 AND c1 = 0 AND c2 IN (5, 2, 8)"),
                    row(2), row(5), row(8));
@@ -399,7 +394,14 @@
                    row("test", 5, set("lmn"))
         );
 
-        assertInvalid("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ? AND categories CONTAINS ?", "xyz", "lmn", "notPresent");
+        assertInvalidMessage("Unsupported null value for column categories",
+                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ?", "test", 5, null);
+
+        assertInvalidMessage("Unsupported unset value for column categories",
+                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ?", "test", 5, unset());
+
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE account = ? AND categories CONTAINS ? AND categories CONTAINS ?", "xyz", "lmn", "notPresent");
         assertEmpty(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ? AND categories CONTAINS ? ALLOW FILTERING", "xyz", "lmn", "notPresent"));
     }
 
@@ -425,8 +427,15 @@
                    row("test", 5, list("lmn"))
         );
 
-        assertInvalid("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ? AND categories CONTAINS ?",
-                      "test", 5, "lmn", "notPresent");
+        assertInvalidMessage("Unsupported null value for column categories",
+                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ?", "test", 5, null);
+
+        assertInvalidMessage("Unsupported unset value for column categories",
+                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ?", "test", 5, unset());
+
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ? AND categories CONTAINS ?",
+                             "test", 5, "lmn", "notPresent");
         assertEmpty(execute("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ? AND categories CONTAINS ? ALLOW FILTERING",
                             "test", 5, "lmn", "notPresent"));
     }
@@ -470,13 +479,21 @@
                    row("test", 5, map("lmn", "foo"))
         );
 
-        assertInvalid("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS KEY ? AND categories CONTAINS KEY ?",
-                      "test", 5, "lmn", "notPresent");
+        assertInvalidMessage("Unsupported null value for column categories",
+                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS KEY ?", "test", 5, null);
+
+        assertInvalidMessage("Unsupported unset value for column categories",
+                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS KEY ?", "test", 5, unset());
+
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS KEY ? AND categories CONTAINS KEY ?",
+                             "test", 5, "lmn", "notPresent");
         assertEmpty(execute("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS KEY ? AND categories CONTAINS KEY ? ALLOW FILTERING",
                             "test", 5, "lmn", "notPresent"));
 
-        assertInvalid("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS KEY ? AND categories CONTAINS ?",
-                      "test", 5, "lmn", "foo");
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS KEY ? AND categories CONTAINS ?",
+                             "test", 5, "lmn", "foo");
     }
 
     @Test
@@ -501,8 +518,15 @@
                    row("test", 5, map("lmn", "foo"))
         );
 
-        assertInvalid("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ? AND categories CONTAINS ?"
-                     , "test", 5, "foo", "notPresent");
+        assertInvalidMessage("Unsupported null value for column categories",
+                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ?", "test", 5, null);
+
+        assertInvalidMessage("Unsupported unset value for column categories",
+                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ?", "test", 5, unset());
+
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ? AND categories CONTAINS ?"
+                            , "test", 5, "foo", "notPresent");
 
         assertEmpty(execute("SELECT * FROM %s WHERE account = ? AND id = ? AND categories CONTAINS ? AND categories CONTAINS ? ALLOW FILTERING"
                            , "test", 5, "foo", "notPresent"));
@@ -588,7 +612,8 @@
         execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 5, map("lmn", "foo"));
         execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 6, map("lmn", "foo2"));
 
-        assertInvalid("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?", "test", "foo");
+        assertInvalidMessage("Predicates on non-primary-key columns (categories) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?", "test", "foo");
 
         assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS KEY ?", "test", "lmn"),
                    row("test", 5, map("lmn", "foo")),
@@ -610,7 +635,8 @@
         execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 5, map("lmn", "foo"));
         execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 6, map("lmn2", "foo"));
 
-        assertInvalid("SELECT * FROM %s WHERE account = ? AND categories CONTAINS KEY ?", "test", "lmn");
+        assertInvalidMessage("Predicates on non-primary-key columns (categories) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE account = ? AND categories CONTAINS KEY ?", "test", "lmn");
 
         assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?", "test", "foo"),
                    row("test", 5, map("lmn", "foo")),
@@ -982,6 +1008,65 @@
     }
 
     /**
+     * Migrated from cql_tests.py:TestCQL.multi_in_test()
+     */
+    @Test
+    public void testMultiSelects() throws Throwable
+    {
+        doTestVariousSelects(false);
+    }
+
+    /**
+     * Migrated from cql_tests.py:TestCQL.multi_in_compact_test()
+     */
+    @Test
+    public void testMultiSelectsCompactStorage() throws Throwable
+    {
+        doTestVariousSelects(true);
+    }
+
+
+    public void doTestVariousSelects(boolean compact) throws Throwable
+    {
+        createTable(
+                   "CREATE TABLE %s (group text, zipcode text, state text, fips_regions int, city text, PRIMARY KEY (group, zipcode, state, fips_regions))"
+                   + (compact
+                      ? " WITH COMPACT STORAGE"
+                      : ""));
+
+        String str = "INSERT INTO %s (group, zipcode, state, fips_regions, city) VALUES (?, ?, ?, ?, ?)";
+        execute(str, "test", "06029", "CT", 9, "Ellington");
+        execute(str, "test", "06031", "CT", 9, "Falls Village");
+        execute(str, "test", "06902", "CT", 9, "Stamford");
+        execute(str, "test", "06927", "CT", 9, "Stamford");
+        execute(str, "test", "10015", "NY", 36, "New York");
+        execute(str, "test", "07182", "NJ", 34, "Newark");
+        execute(str, "test", "73301", "TX", 48, "Austin");
+        execute(str, "test", "94102", "CA", 06, "San Francisco");
+
+        execute(str, "test2", "06029", "CT", 9, "Ellington");
+        execute(str, "test2", "06031", "CT", 9, "Falls Village");
+        execute(str, "test2", "06902", "CT", 9, "Stamford");
+        execute(str, "test2", "06927", "CT", 9, "Stamford");
+        execute(str, "test2", "10015", "NY", 36, "New York");
+        execute(str, "test2", "07182", "NJ", 34, "Newark");
+        execute(str, "test2", "73301", "TX", 48, "Austin");
+        execute(str, "test2", "94102", "CA", 06, "San Francisco");
+
+        assertRowCount(execute("select zipcode from %s"), 16);
+        assertRowCount(execute("select zipcode from %s where group='test'"), 8);
+        assertInvalid("select zipcode from %s where zipcode='06902'");
+        assertRowCount(execute("select zipcode from %s where zipcode='06902' ALLOW FILTERING"), 2);
+        assertRowCount(execute("select zipcode from %s where group='test' and zipcode='06902'"), 1);
+        assertRowCount(execute("select zipcode from %s where group='test' and zipcode IN ('06902','73301','94102')"), 3);
+        assertRowCount(execute("select zipcode from %s where group='test' AND zipcode IN ('06902','73301','94102') and state IN ('CT','CA')"), 2);
+        assertRowCount(execute("select zipcode from %s where group='test' AND zipcode IN ('06902','73301','94102') and state IN ('CT','CA') and fips_regions = 9"), 1);
+        assertRowCount(execute("select zipcode from %s where group='test' AND zipcode IN ('06902','73301','94102') and state IN ('CT','CA') ORDER BY zipcode DESC"), 2);
+        assertRowCount(execute("select zipcode from %s where group='test' AND zipcode IN ('06902','73301','94102') and state IN ('CT','CA') and fips_regions > 0"), 2);
+        assertEmpty(execute("select zipcode from %s where group='test' AND zipcode IN ('06902','73301','94102') and state IN ('CT','CA') and fips_regions < 0"));
+    }
+
+    /**
      * Migrated from cql_tests.py:TestCQL.multi_in_compact_non_composite_test()
      */
     @Test
@@ -1171,6 +1256,51 @@
         Assert.assertEquals(9, rows.length);
     }
 
+    @Test
+    public void testSelectDistinctWithWhereClause() throws Throwable {
+        createTable("CREATE TABLE %s (k int, a int, b int, PRIMARY KEY (k, a))");
+        createIndex("CREATE INDEX ON %s (b)");
+
+        for (int i = 0; i < 10; i++)
+        {
+            execute("INSERT INTO %s (k, a, b) VALUES (?, ?, ?)", i, i, i);
+            execute("INSERT INTO %s (k, a, b) VALUES (?, ?, ?)", i, i * 10, i * 10);
+        }
+
+        String distinctQueryErrorMsg = "SELECT DISTINCT with WHERE clause only supports restriction by partition key.";
+        assertInvalidMessage(distinctQueryErrorMsg,
+                             "SELECT DISTINCT k FROM %s WHERE a >= 80 ALLOW FILTERING");
+
+        assertInvalidMessage(distinctQueryErrorMsg,
+                             "SELECT DISTINCT k FROM %s WHERE k IN (1, 2, 3) AND a = 10");
+
+        assertInvalidMessage(distinctQueryErrorMsg,
+                             "SELECT DISTINCT k FROM %s WHERE b = 5");
+
+        assertRows(execute("SELECT DISTINCT k FROM %s WHERE k = 1"),
+                   row(1));
+        assertRows(execute("SELECT DISTINCT k FROM %s WHERE k IN (5, 6, 7)"),
+                   row(5),
+                   row(6),
+                   row(7));
+
+        // With static columns
+        createTable("CREATE TABLE %s (k int, a int, s int static, b int, PRIMARY KEY (k, a))");
+        createIndex("CREATE INDEX ON %s (b)");
+        for (int i = 0; i < 10; i++)
+        {
+            execute("INSERT INTO %s (k, a, b, s) VALUES (?, ?, ?, ?)", i, i, i, i);
+            execute("INSERT INTO %s (k, a, b, s) VALUES (?, ?, ?, ?)", i, i * 10, i * 10, i * 10);
+        }
+
+        assertRows(execute("SELECT DISTINCT s FROM %s WHERE k = 5"),
+                   row(50));
+        assertRows(execute("SELECT DISTINCT s FROM %s WHERE k IN (5, 6, 7)"),
+                   row(50),
+                   row(60),
+                   row(70));
+    }
+
     /**
      * Migrated from cql_tests.py:TestCQL.bug_6327_test()
      */
@@ -1201,7 +1331,7 @@
         execute("insert into %s (field1, field2, field3) values ('hola', now(), false)");
 
         assertRows(execute("select count(*) from %s where field3 = false limit 1"),
-                   row(1L));
+                   row(2L));
     }
 
     /**
@@ -1240,6 +1370,891 @@
     }
 
     @Test
+    public void testAlias() throws Throwable
+    {
+        createTable("CREATE TABLE %s (id int PRIMARY KEY, name text)");
+
+        for (int i = 0; i < 5; i++)
+            execute("INSERT INTO %s (id, name) VALUES (?, ?) USING TTL 10 AND TIMESTAMP 0", i, Integer.toString(i));
+
+        assertInvalidMessage("Aliases aren't allowed in the where clause",
+                             "SELECT id AS user_id, name AS user_name FROM %s WHERE user_id = 0");
+
+        // test that select throws a meaningful exception for aliases in order by clause
+        assertInvalidMessage("Aliases are not allowed in order by clause",
+                             "SELECT id AS user_id, name AS user_name FROM %s WHERE id IN (0) ORDER BY user_name");
+
+    }
+
+    @Test
+    public void testFilteringWithoutIndices() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, e map<int, int>, PRIMARY KEY (a, b))");
+
+        // Checks filtering
+        assertInvalidMessage("Predicates on non-primary-key columns (c, d) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c = 1 AND d = 2 ALLOW FILTERING");
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE a = 1 AND b = 1 AND c = 2 ALLOW FILTERING");
+        assertInvalidMessage("IN predicates on non-primary-key columns (c) is not yet supported",
+                             "SELECT * FROM %s WHERE a IN (1, 2) AND c IN (2, 3) ALLOW FILTERING");
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c > 2 ALLOW FILTERING");
+        assertInvalidMessage("Predicates on non-primary-key columns (e) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE e CONTAINS 1 ALLOW FILTERING");
+        assertInvalidMessage("Predicates on non-primary-key columns (e) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE e CONTAINS KEY 1 ALLOW FILTERING");
+
+        // Checks filtering with null
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c = null ALLOW FILTERING");
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c > null ALLOW FILTERING");
+        assertInvalidMessage("Predicates on non-primary-key columns (e) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE e CONTAINS null ALLOW FILTERING");
+        assertInvalidMessage("Predicates on non-primary-key columns (e) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE e CONTAINS KEY null ALLOW FILTERING");
+
+        // Checks filtering with unset
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c = ? ALLOW FILTERING", unset());
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c > ? ALLOW FILTERING", unset());
+        assertInvalidMessage("Predicates on non-primary-key columns (e) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE e CONTAINS ? ALLOW FILTERING", unset());
+        assertInvalidMessage("Predicates on non-primary-key columns (e) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE e CONTAINS KEY ? ALLOW FILTERING", unset());
+
+        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY(a)) WITH COMPACT STORAGE");
+
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 2, 4)");
+        execute("INSERT INTO %s (a, b, c) VALUES (2, 2, 8)");
+        execute("INSERT INTO %s (a, b, c) VALUES (3, 6, 4)");
+
+        assertRows(execute("SELECT * FROM %s WHERE c = 4 ALLOW FILTERING"),
+                   row(1, 2, 4),
+                   row(3, 6, 4));
+
+        // Checks filtering with null
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE, "SELECT * FROM %s WHERE c = null");
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE, "SELECT * FROM %s WHERE c > null");
+        assertInvalidMessage("Unsupported null value for column c", "SELECT * FROM %s WHERE c = null ALLOW FILTERING");
+        assertInvalidMessage("Unsupported null value for column c", "SELECT * FROM %s WHERE c > null ALLOW FILTERING");
+
+        // Checks filtering with unset
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE c = ?", unset());
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE c > ?", unset());
+        assertInvalidMessage("Unsupported unset value for column c",
+                             "SELECT * FROM %s WHERE c = ? ALLOW FILTERING", unset());
+        assertInvalidMessage("Unsupported unset value for column c",
+                             "SELECT * FROM %s WHERE c > ? ALLOW FILTERING", unset());
+    }
+
+    @Test
+    public void testFilteringOnStaticColumnWithoutIndices() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, s int static, c int, PRIMARY KEY (a, b))");
+
+        // Checks filtering
+        assertInvalidMessage("Predicates on non-primary-key columns (c, s) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c = 1 AND s = 2 ALLOW FILTERING");
+        assertInvalidMessage("Predicates on non-primary-key columns (s) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE a = 1 AND b = 1 AND s = 2 ALLOW FILTERING");
+        assertInvalidMessage("Predicates on non-primary-key columns (s) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE s > 2 ALLOW FILTERING");
+
+        // Checks filtering with null
+        assertInvalidMessage("Predicates on non-primary-key columns (s) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE s = null ALLOW FILTERING");
+        assertInvalidMessage("Predicates on non-primary-key columns (s) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE s > null ALLOW FILTERING");
+
+        // Checks filtering with unset
+        assertInvalidMessage("Predicates on non-primary-key columns (s) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE s = ? ALLOW FILTERING", unset());
+        assertInvalidMessage("Predicates on non-primary-key columns (s) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE s > ? ALLOW FILTERING", unset());
+    }
+
+    @Test
+    public void testFilteringOnCompactTablesWithoutIndices() throws Throwable
+    {
+        //----------------------------------------------
+        // Test COMPACT table with clustering columns
+        //----------------------------------------------
+        createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b)) WITH COMPACT STORAGE");
+
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 2, 4)");
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 3, 6)");
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 4, 4)");
+        execute("INSERT INTO %s (a, b, c) VALUES (2, 3, 7)");
+
+        // Lets add some tombstones to make sure that filtering handle them properly
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 1, 4)");
+        execute("INSERT INTO %s (a, b, c) VALUES (2, 2, 7)");
+        execute("DELETE FROM %s WHERE a = 1 AND b = 1");
+        execute("DELETE FROM %s WHERE a = 2 AND b = 2");
+
+        flush();
+
+        // Checks filtering
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE a = 1 AND b = 4 AND c = 4 ALLOW FILTERING");
+
+        assertInvalidMessage("IN predicates on non-primary-key columns (c) is not yet supported",
+                             "SELECT * FROM %s WHERE a IN (1, 2) AND c IN (6, 7) ALLOW FILTERING");
+
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c > 4 ALLOW FILTERING");
+
+        // Checks filtering with null
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c = null");
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c = null ALLOW FILTERING");
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c > null");
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c > null ALLOW FILTERING");
+
+        // Checks filtering with unset
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c = ? ALLOW FILTERING",
+                             unset());
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c > ? ALLOW FILTERING",
+                             unset());
+
+        //----------------------------------------------
+        // Test COMPACT table without clustering columns
+        //----------------------------------------------
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b int, c int) WITH COMPACT STORAGE");
+
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 2, 4)");
+        execute("INSERT INTO %s (a, b, c) VALUES (2, 1, 6)");
+        execute("INSERT INTO %s (a, b, c) VALUES (3, 2, 4)");
+        execute("INSERT INTO %s (a, b, c) VALUES (4, 1, 7)");
+
+        // Lets add some tombstones to make sure that filtering handle them properly
+        execute("INSERT INTO %s (a, b, c) VALUES (0, 1, 4)");
+        execute("INSERT INTO %s (a, b, c) VALUES (5, 2, 7)");
+        execute("DELETE FROM %s WHERE a = 0");
+        execute("DELETE FROM %s WHERE a = 5");
+
+        flush();
+
+        // Checks filtering
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE a = 1 AND b = 4 AND c = 4");
+
+        assertRows(execute("SELECT * FROM %s WHERE a = 1 AND b = 2 AND c = 4 ALLOW FILTERING"),
+                   row(1, 2, 4));
+
+        assertInvalidMessage("IN predicates on non-primary-key columns (c) is not yet supported",
+                             "SELECT * FROM %s WHERE a IN (1, 2) AND c IN (6, 7)");
+
+        assertInvalidMessage("IN predicates on non-primary-key columns (c) is not yet supported",
+                             "SELECT * FROM %s WHERE a IN (1, 2) AND c IN (6, 7) ALLOW FILTERING");
+
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE c > 4");
+
+        assertRows(execute("SELECT * FROM %s WHERE c > 4 ALLOW FILTERING"),
+                   row(2, 1, 6),
+                   row(4, 1, 7));
+
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE b < 3 AND c <= 4");
+
+        assertRows(execute("SELECT * FROM %s WHERE b < 3 AND c <= 4 ALLOW FILTERING"),
+                   row(1, 2, 4),
+                   row(3, 2, 4));
+
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE c >= 3 AND c <= 6");
+
+        assertRows(execute("SELECT * FROM %s WHERE c >= 3 AND c <= 6 ALLOW FILTERING"),
+                   row(1, 2, 4),
+                   row(2, 1, 6),
+                   row(3, 2, 4));
+
+        // Checks filtering with null
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE c = null");
+        assertInvalidMessage("Unsupported null value for column c",
+                             "SELECT * FROM %s WHERE c = null ALLOW FILTERING");
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE c > null");
+        assertInvalidMessage("Unsupported null value for column c",
+                             "SELECT * FROM %s WHERE c > null ALLOW FILTERING");
+
+        // Checks filtering with unset
+        assertInvalidMessage("Unsupported unset value for column c",
+                             "SELECT * FROM %s WHERE c = ? ALLOW FILTERING",
+                             unset());
+        assertInvalidMessage("Unsupported unset value for column c",
+                             "SELECT * FROM %s WHERE c > ? ALLOW FILTERING",
+                             unset());
+    }
+
+    @Test
+    public void testFilteringWithoutIndicesWithCollections() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c list<int>, d set<int>, e map<int, int>, PRIMARY KEY (a, b))");
+
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 2, [1, 6], {2, 12}, {1: 6})");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 3, [3, 2], {6, 4}, {3: 2})");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 4, [1, 2], {2, 4}, {1: 2})");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (2, 3, [3, 6], {6, 12}, {3: 6})");
+
+        flush();
+
+        // Checks filtering for lists
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c CONTAINS 2 ALLOW FILTERING");
+
+        // Checks filtering for sets
+        assertInvalidMessage("Predicates on non-primary-key columns (d) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE d CONTAINS 4 ALLOW FILTERING");
+
+        // Checks filtering for maps
+        assertInvalidMessage("Predicates on non-primary-key columns (e) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE e CONTAINS 2 ALLOW FILTERING");
+
+        assertInvalidMessage("Predicates on non-primary-key columns (e) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE e CONTAINS KEY 2 ALLOW FILTERING");
+
+        // Checks filtering with null
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c CONTAINS null ALLOW FILTERING");
+        assertInvalidMessage("Predicates on non-primary-key columns (d) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE d CONTAINS null ALLOW FILTERING");
+        assertInvalidMessage("Predicates on non-primary-key columns (e) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE e CONTAINS null ALLOW FILTERING");
+        assertInvalidMessage("Predicates on non-primary-key columns (e) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE e CONTAINS KEY null ALLOW FILTERING");
+        assertInvalidMessage("Predicates on non-primary-key columns (e) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE e[null] = 2 ALLOW FILTERING");
+        assertInvalidMessage("Predicates on non-primary-key columns (e) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE e[1] = null ALLOW FILTERING");
+
+        // Checks filtering with unset
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c CONTAINS ? ALLOW FILTERING",
+                             unset());
+        assertInvalidMessage("Predicates on non-primary-key columns (d) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE d CONTAINS ? ALLOW FILTERING",
+                             unset());
+        assertInvalidMessage("Predicates on non-primary-key columns (e) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE e CONTAINS ? ALLOW FILTERING",
+                             unset());
+        assertInvalidMessage("Predicates on non-primary-key columns (e) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE e CONTAINS KEY ? ALLOW FILTERING",
+                             unset());
+        assertInvalidMessage("Predicates on non-primary-key columns (e) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE e[?] = 2 ALLOW FILTERING",
+                             unset());
+        assertInvalidMessage("Predicates on non-primary-key columns (e) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE e[1] = ? ALLOW FILTERING",
+                             unset());
+    }
+
+    @Test
+    public void testFilteringWithoutIndicesWithFrozenCollections() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c frozen<list<int>>, d frozen<set<int>>, e frozen<map<int, int>>, PRIMARY KEY (a, b))");
+
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 2, [1, 6], {2, 12}, {1: 6})");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 3, [3, 2], {6, 4}, {3: 2})");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (1, 4, [1, 2], {2, 4}, {1: 2})");
+        execute("INSERT INTO %s (a, b, c, d, e) VALUES (2, 3, [3, 6], {6, 12}, {3: 6})");
+
+        flush();
+
+        // Checks filtering for lists
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c = [3, 2] ALLOW FILTERING");
+
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c > [1, 5] AND c < [3, 6] ALLOW FILTERING");
+
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c CONTAINS 2 ALLOW FILTERING");
+
+        // Checks filtering for sets
+        assertInvalidMessage("Predicates on non-primary-key columns (d) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE d = {6, 4} ALLOW FILTERING");
+
+        assertInvalidMessage("Predicates on non-primary-key columns (d) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE d > {4, 5} AND d < {6} ALLOW FILTERING");
+
+        assertInvalidMessage("Predicates on non-primary-key columns (d) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE d CONTAINS 4 ALLOW FILTERING");
+
+        // Checks filtering for maps
+        assertInvalidMessage("Predicates on non-primary-key columns (e) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE e = {1 : 2} ALLOW FILTERING");
+
+        assertInvalidMessage("Predicates on non-primary-key columns (e) are not yet supported for non secondary index queries",
+                "SELECT * FROM %s WHERE e > {1 : 4} AND e < {3 : 6} ALLOW FILTERING");
+
+        assertInvalidMessage("Predicates on non-primary-key columns (e) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE e CONTAINS 2 ALLOW FILTERING");
+
+        assertInvalidMessage("Map-entry equality predicates on frozen map column e are not supported",
+                             "SELECT * FROM %s WHERE e[1] = 6 ALLOW FILTERING");
+
+        // Checks filtering with null
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c = null ALLOW FILTERING");
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c CONTAINS null ALLOW FILTERING");
+        assertInvalidMessage("Predicates on non-primary-key columns (d) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE d = null ALLOW FILTERING");
+        assertInvalidMessage("Predicates on non-primary-key columns (d) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE d CONTAINS null ALLOW FILTERING");
+        assertInvalidMessage("Predicates on non-primary-key columns (e) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE e = null ALLOW FILTERING");
+        assertInvalidMessage("Predicates on non-primary-key columns (e) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE e CONTAINS null ALLOW FILTERING");
+        assertInvalidMessage("Predicates on non-primary-key columns (e) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE e CONTAINS KEY null ALLOW FILTERING");
+        assertInvalidMessage("Map-entry equality predicates on frozen map column e are not supported",
+                             "SELECT * FROM %s WHERE e[null] = 2 ALLOW FILTERING");
+        assertInvalidMessage("Map-entry equality predicates on frozen map column e are not supported",
+                             "SELECT * FROM %s WHERE e[1] = null ALLOW FILTERING");
+
+        // Checks filtering with unset
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c = ? ALLOW FILTERING",
+                             unset());
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c CONTAINS ? ALLOW FILTERING",
+                             unset());
+        assertInvalidMessage("Predicates on non-primary-key columns (d) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE d = ? ALLOW FILTERING",
+                             unset());
+        assertInvalidMessage("Predicates on non-primary-key columns (d) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE d CONTAINS ? ALLOW FILTERING",
+                             unset());
+        assertInvalidMessage("Predicates on non-primary-key columns (e) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE e = ? ALLOW FILTERING",
+                             unset());
+        assertInvalidMessage("Predicates on non-primary-key columns (e) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE e CONTAINS ? ALLOW FILTERING",
+                             unset());
+        assertInvalidMessage("Predicates on non-primary-key columns (e) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE e CONTAINS KEY ? ALLOW FILTERING",
+                             unset());
+        assertInvalidMessage("Map-entry equality predicates on frozen map column e are not supported",
+                             "SELECT * FROM %s WHERE e[?] = 2 ALLOW FILTERING",
+                             unset());
+        assertInvalidMessage("Map-entry equality predicates on frozen map column e are not supported",
+                             "SELECT * FROM %s WHERE e[1] = ? ALLOW FILTERING",
+                             unset());
+    }
+
+    @Test
+    public void testFilteringOnCompactTablesWithoutIndicesAndWithLists() throws Throwable
+    {
+        //----------------------------------------------
+        // Test COMPACT table with clustering columns
+        //----------------------------------------------
+        createTable("CREATE TABLE %s (a int, b int, c frozen<list<int>>, PRIMARY KEY (a, b)) WITH COMPACT STORAGE");
+
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 2, [4, 2])");
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 3, [6, 2])");
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 4, [4, 1])");
+        execute("INSERT INTO %s (a, b, c) VALUES (2, 3, [7, 1])");
+
+        flush();
+
+        // Checks filtering
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE a = 1 AND b = 4 AND c = [4, 1] ALLOW FILTERING");
+
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c > [4, 2] ALLOW FILTERING");
+
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE b <= 3 AND c < [6, 2] ALLOW FILTERING");
+
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c CONTAINS 2 ALLOW FILTERING");
+
+        assertInvalidMessage("Cannot use CONTAINS KEY on non-map column c",
+                             "SELECT * FROM %s WHERE c CONTAINS KEY 2 ALLOW FILTERING");
+
+        // Checks filtering with null
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c = null");
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c = null ALLOW FILTERING");
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c > null");
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c > null ALLOW FILTERING");
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c CONTAINS null");
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c CONTAINS null ALLOW FILTERING");
+
+        // Checks filtering with unset
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c = ? ALLOW FILTERING",
+                             unset());
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c > ? ALLOW FILTERING",
+                             unset());
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c CONTAINS ? ALLOW FILTERING",
+                             unset());
+
+        //----------------------------------------------
+        // Test COMPACT table without clustering columns
+        //----------------------------------------------
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b int, c frozen<list<int>>) WITH COMPACT STORAGE");
+
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 2, [4, 2])");
+        execute("INSERT INTO %s (a, b, c) VALUES (2, 1, [6, 2])");
+        execute("INSERT INTO %s (a, b, c) VALUES (3, 2, [4, 1])");
+        execute("INSERT INTO %s (a, b, c) VALUES (4, 1, [7, 1])");
+
+        flush();
+
+        // Checks filtering
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE a = 1 AND b = 2 AND c = [4, 2]");
+
+        assertRows(execute("SELECT * FROM %s WHERE a = 1 AND b = 2 AND c = [4, 2] ALLOW FILTERING"),
+                   row(1, 2, list(4, 2)));
+
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE c > [4, 2]");
+
+        assertRows(execute("SELECT * FROM %s WHERE c > [4, 2] ALLOW FILTERING"),
+                   row(2, 1, list(6, 2)),
+                   row(4, 1, list(7, 1)));
+
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE b < 3 AND c <= [4, 2]");
+
+        assertRows(execute("SELECT * FROM %s WHERE b < 3 AND c <= [4, 2] ALLOW FILTERING"),
+                   row(1, 2, list(4, 2)),
+                   row(3, 2, list(4, 1)));
+
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE c >= [4, 3] AND c <= [7]");
+
+        assertRows(execute("SELECT * FROM %s WHERE c >= [4, 3] AND c <= [7] ALLOW FILTERING"),
+                   row(2, 1, list(6, 2)));
+
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                "SELECT * FROM %s WHERE c CONTAINS 2");
+
+        assertRows(execute("SELECT * FROM %s WHERE c CONTAINS 2 ALLOW FILTERING"),
+                   row(1, 2, list(4, 2)),
+                   row(2, 1, list(6, 2)));
+
+        assertInvalidMessage("Cannot use CONTAINS KEY on non-map column c",
+                             "SELECT * FROM %s WHERE c CONTAINS KEY 2 ALLOW FILTERING");
+
+        assertRows(execute("SELECT * FROM %s WHERE c CONTAINS 2 AND c CONTAINS 6 ALLOW FILTERING"),
+                   row(2, 1, list(6, 2)));
+
+        // Checks filtering with null
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE c = null");
+        assertInvalidMessage("Unsupported null value for column c",
+                             "SELECT * FROM %s WHERE c = null ALLOW FILTERING");
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE c > null");
+        assertInvalidMessage("Unsupported null value for column c",
+                             "SELECT * FROM %s WHERE c > null ALLOW FILTERING");
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE c CONTAINS null");
+        assertInvalidMessage("Unsupported null value for column c",
+                             "SELECT * FROM %s WHERE c CONTAINS null ALLOW FILTERING");
+
+        // Checks filtering with unset
+        assertInvalidMessage("Unsupported unset value for column c",
+                             "SELECT * FROM %s WHERE c = ? ALLOW FILTERING",
+                             unset());
+        assertInvalidMessage("Unsupported unset value for column c",
+                             "SELECT * FROM %s WHERE c > ? ALLOW FILTERING",
+                             unset());
+        assertInvalidMessage("Unsupported unset value for column c",
+                             "SELECT * FROM %s WHERE c CONTAINS ? ALLOW FILTERING",
+                             unset());
+    }
+
+    @Test
+    public void testFilteringOnCompactTablesWithoutIndicesAndWithSets() throws Throwable
+    {
+        //----------------------------------------------
+        // Test COMPACT table with clustering columns
+        //----------------------------------------------
+        createTable("CREATE TABLE %s (a int, b int, c frozen<set<int>>, PRIMARY KEY (a, b)) WITH COMPACT STORAGE");
+
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 2, {4, 2})");
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 3, {6, 2})");
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 4, {4, 1})");
+        execute("INSERT INTO %s (a, b, c) VALUES (2, 3, {7, 1})");
+
+        flush();
+
+        // Checks filtering
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE a = 1 AND b = 4 AND c = {4, 1} ALLOW FILTERING");
+
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c > {4, 2} ALLOW FILTERING");
+
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c >= {4, 2} AND c <= {6, 4} ALLOW FILTERING");
+
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c CONTAINS 2 ALLOW FILTERING");
+
+        assertInvalidMessage("Cannot use CONTAINS KEY on non-map column c",
+                             "SELECT * FROM %s WHERE c CONTAINS KEY 2 ALLOW FILTERING");
+
+        // Checks filtering with null
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c = null");
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c = null ALLOW FILTERING");
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c > null");
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c > null ALLOW FILTERING");
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c CONTAINS null");
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c CONTAINS null ALLOW FILTERING");
+
+        // Checks filtering with unset
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c = ? ALLOW FILTERING",
+                             unset());
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c > ? ALLOW FILTERING",
+                             unset());
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c CONTAINS ? ALLOW FILTERING",
+                             unset());
+
+        //----------------------------------------------
+        // Test COMPACT table without clustering columns
+        //----------------------------------------------
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b int, c frozen<set<int>>) WITH COMPACT STORAGE");
+
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 2, {4, 2})");
+        execute("INSERT INTO %s (a, b, c) VALUES (2, 1, {6, 2})");
+        execute("INSERT INTO %s (a, b, c) VALUES (3, 2, {4, 1})");
+        execute("INSERT INTO %s (a, b, c) VALUES (4, 1, {7, 1})");
+
+        flush();
+
+        // Checks filtering
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE a = 1 AND b = 2 AND c = {4, 2}");
+
+        assertRows(execute("SELECT * FROM %s WHERE a = 1 AND b = 2 AND c = {4, 2} ALLOW FILTERING"),
+                   row(1, 2, set(4, 2)));
+
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE c > {4, 2}");
+
+        assertRows(execute("SELECT * FROM %s WHERE c > {4, 2} ALLOW FILTERING"),
+                   row(2, 1, set(6, 2)));
+
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE b < 3 AND c <= {4, 2}");
+
+        assertRows(execute("SELECT * FROM %s WHERE b < 3 AND c <= {4, 2} ALLOW FILTERING"),
+                   row(1, 2, set(4, 2)),
+                   row(4, 1, set(1, 7)),
+                   row(3, 2, set(4, 1)));
+
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE c >= {4, 3} AND c <= {7}");
+
+        assertRows(execute("SELECT * FROM %s WHERE c >= {5, 2} AND c <= {7} ALLOW FILTERING"),
+                   row(2, 1, set(6, 2)));
+
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                "SELECT * FROM %s WHERE c CONTAINS 2");
+
+        assertRows(execute("SELECT * FROM %s WHERE c CONTAINS 2 ALLOW FILTERING"),
+                   row(1, 2, set(4, 2)),
+                   row(2, 1, set(6, 2)));
+
+        assertInvalidMessage("Cannot use CONTAINS KEY on non-map column c",
+                             "SELECT * FROM %s WHERE c CONTAINS KEY 2 ALLOW FILTERING");
+
+        assertRows(execute("SELECT * FROM %s WHERE c CONTAINS 2 AND c CONTAINS 6 ALLOW FILTERING"),
+                   row(2, 1, set(6, 2)));
+
+        // Checks filtering with null
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE c = null");
+        assertInvalidMessage("Unsupported null value for column c",
+                             "SELECT * FROM %s WHERE c = null ALLOW FILTERING");
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE c > null");
+        assertInvalidMessage("Unsupported null value for column c",
+                             "SELECT * FROM %s WHERE c > null ALLOW FILTERING");
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE c CONTAINS null");
+        assertInvalidMessage("Unsupported null value for column c",
+                             "SELECT * FROM %s WHERE c CONTAINS null ALLOW FILTERING");
+
+        // Checks filtering with unset
+        assertInvalidMessage("Unsupported unset value for column c",
+                             "SELECT * FROM %s WHERE c = ? ALLOW FILTERING",
+                             unset());
+        assertInvalidMessage("Unsupported unset value for column c",
+                             "SELECT * FROM %s WHERE c > ? ALLOW FILTERING",
+                             unset());
+        assertInvalidMessage("Unsupported unset value for column c",
+                             "SELECT * FROM %s WHERE c CONTAINS ? ALLOW FILTERING",
+                             unset());
+    }
+
+    @Test
+    public void testIndexQueryWithValueOver64K() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c blob, PRIMARY KEY (a, b))");
+        createIndex("CREATE INDEX test ON %s (c)");
+
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 0, bytes(1));
+        execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 1, bytes(2));
+
+        assertInvalidMessage("Index expression values may not be larger than 64K",
+                             "SELECT * FROM %s WHERE c = ?  ALLOW FILTERING", TOO_BIG);
+    }
+
+    @Test
+    public void testFilteringOnCompactTablesWithoutIndicesAndWithMaps() throws Throwable
+    {
+        //----------------------------------------------
+        // Test COMPACT table with clustering columns
+        //----------------------------------------------
+        createTable("CREATE TABLE %s (a int, b int, c frozen<map<int, int>>, PRIMARY KEY (a, b)) WITH COMPACT STORAGE");
+
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 2, {4 : 2})");
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 3, {6 : 2})");
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 4, {4 : 1})");
+        execute("INSERT INTO %s (a, b, c) VALUES (2, 3, {7 : 1})");
+
+        flush();
+
+        // Checks filtering
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE a = 1 AND b = 4 AND c = {4 : 1} ALLOW FILTERING");
+
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c > {4 : 2} ALLOW FILTERING");
+
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE b <= 3 AND c < {6 : 2} ALLOW FILTERING");
+
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c CONTAINS 2 ALLOW FILTERING");
+
+        // Checks filtering with null
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c = null");
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c = null ALLOW FILTERING");
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c > null");
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c > null ALLOW FILTERING");
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c CONTAINS null");
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c CONTAINS null ALLOW FILTERING");
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c CONTAINS KEY null ALLOW FILTERING");
+
+        // Checks filtering with unset
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c = ? ALLOW FILTERING",
+                             unset());
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c > ? ALLOW FILTERING",
+                             unset());
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c CONTAINS ? ALLOW FILTERING",
+                             unset());
+        assertInvalidMessage("Predicates on non-primary-key columns (c) are not yet supported for non secondary index queries",
+                             "SELECT * FROM %s WHERE c CONTAINS KEY ? ALLOW FILTERING",
+                             unset());
+
+        //----------------------------------------------
+        // Test COMPACT table without clustering columns
+        //----------------------------------------------
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b int, c frozen<map<int, int>>) WITH COMPACT STORAGE");
+
+        execute("INSERT INTO %s (a, b, c) VALUES (1, 2, {4 : 2})");
+        execute("INSERT INTO %s (a, b, c) VALUES (2, 1, {6 : 2})");
+        execute("INSERT INTO %s (a, b, c) VALUES (3, 2, {4 : 1})");
+        execute("INSERT INTO %s (a, b, c) VALUES (4, 1, {7 : 1})");
+
+        flush();
+
+        // Checks filtering
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE a = 1 AND b = 2 AND c = {4 : 2}");
+
+        assertRows(execute("SELECT * FROM %s WHERE a = 1 AND b = 2 AND c = {4 : 2} ALLOW FILTERING"),
+                   row(1, 2, map(4, 2)));
+
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE c > {4 : 2}");
+
+        assertRows(execute("SELECT * FROM %s WHERE c > {4 : 2} ALLOW FILTERING"),
+                   row(2, 1, map(6, 2)),
+                   row(4, 1, map(7, 1)));
+
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE b < 3 AND c <= {4 : 2}");
+
+        assertRows(execute("SELECT * FROM %s WHERE b < 3 AND c <= {4 : 2} ALLOW FILTERING"),
+                   row(1, 2, map(4, 2)),
+                   row(3, 2, map(4, 1)));
+
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE c >= {4 : 3} AND c <= {7 : 1}");
+
+        assertRows(execute("SELECT * FROM %s WHERE c >= {5 : 2} AND c <= {7 : 0} ALLOW FILTERING"),
+                   row(2, 1, map(6, 2)));
+
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                "SELECT * FROM %s WHERE c CONTAINS 2");
+
+        assertRows(execute("SELECT * FROM %s WHERE c CONTAINS 2 ALLOW FILTERING"),
+                   row(1, 2, map(4, 2)),
+                   row(2, 1, map(6, 2)));
+
+        assertRows(execute("SELECT * FROM %s WHERE c CONTAINS KEY 4 ALLOW FILTERING"),
+                   row(1, 2, map(4, 2)),
+                   row(3, 2, map(4, 1)));
+
+        assertRows(execute("SELECT * FROM %s WHERE c CONTAINS 2 AND c CONTAINS KEY 6 ALLOW FILTERING"),
+                   row(2, 1, map(6, 2)));
+
+        // Checks filtering with null
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE c = null");
+        assertInvalidMessage("Unsupported null value for column c",
+                             "SELECT * FROM %s WHERE c = null ALLOW FILTERING");
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE c > null");
+        assertInvalidMessage("Unsupported null value for column c",
+                             "SELECT * FROM %s WHERE c > null ALLOW FILTERING");
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE c CONTAINS null");
+        assertInvalidMessage("Unsupported null value for column c",
+                             "SELECT * FROM %s WHERE c CONTAINS null ALLOW FILTERING");
+        assertInvalidMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE,
+                             "SELECT * FROM %s WHERE c CONTAINS KEY null");
+        assertInvalidMessage("Unsupported null value for column c",
+                             "SELECT * FROM %s WHERE c CONTAINS KEY null ALLOW FILTERING");
+
+        // Checks filtering with unset
+        assertInvalidMessage("Unsupported unset value for column c",
+                             "SELECT * FROM %s WHERE c = ? ALLOW FILTERING",
+                             unset());
+        assertInvalidMessage("Unsupported unset value for column c",
+                             "SELECT * FROM %s WHERE c > ? ALLOW FILTERING",
+                             unset());
+        assertInvalidMessage("Unsupported unset value for column c",
+                             "SELECT * FROM %s WHERE c CONTAINS ? ALLOW FILTERING",
+                             unset());
+        assertInvalidMessage("Unsupported unset value for column c",
+                             "SELECT * FROM %s WHERE c CONTAINS KEY ? ALLOW FILTERING",
+                             unset());
+    }
+
+    /**
+     * Check select with and without compact storage, with different column
+     * order. See CASSANDRA-10988
+     */
+    @Test
+    public void testClusteringOrderWithSlice() throws Throwable
+    {
+        for (String compactOption : new String[] { "", " COMPACT STORAGE AND" })
+        {
+            // non-compound, ASC order
+            createTable("CREATE TABLE %s (a text, b int, PRIMARY KEY (a, b)) WITH" +
+                        compactOption +
+                        " CLUSTERING ORDER BY (b ASC)");
+
+            execute("INSERT INTO %s (a, b) VALUES ('a', 2)");
+            execute("INSERT INTO %s (a, b) VALUES ('a', 3)");
+            assertRows(execute("SELECT * FROM %s WHERE a = 'a' AND b > 0"),
+                       row("a", 2),
+                       row("a", 3));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = 'a' AND b > 0 ORDER BY b DESC"),
+                       row("a", 3),
+                       row("a", 2));
+
+            // non-compound, DESC order
+            createTable("CREATE TABLE %s (a text, b int, PRIMARY KEY (a, b)) WITH" +
+                        compactOption +
+                        " CLUSTERING ORDER BY (b DESC)");
+
+            execute("INSERT INTO %s (a, b) VALUES ('a', 2)");
+            execute("INSERT INTO %s (a, b) VALUES ('a', 3)");
+            assertRows(execute("SELECT * FROM %s WHERE a = 'a' AND b > 0"),
+                       row("a", 3),
+                       row("a", 2));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = 'a' AND b > 0 ORDER BY b ASC"),
+                       row("a", 2),
+                       row("a", 3));
+
+            // compound, first column DESC order
+            createTable("CREATE TABLE %s (a text, b int, c int, PRIMARY KEY (a, b, c)) WITH" +
+                        compactOption +
+                        " CLUSTERING ORDER BY (b DESC)"
+            );
+
+            execute("INSERT INTO %s (a, b, c) VALUES ('a', 2, 4)");
+            execute("INSERT INTO %s (a, b, c) VALUES ('a', 3, 5)");
+            assertRows(execute("SELECT * FROM %s WHERE a = 'a' AND b > 0"),
+                       row("a", 3, 5),
+                       row("a", 2, 4));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = 'a' AND b > 0 ORDER BY b ASC"),
+                       row("a", 2, 4),
+                       row("a", 3, 5));
+
+            // compound, mixed order
+            createTable("CREATE TABLE %s (a text, b int, c int, PRIMARY KEY (a, b, c)) WITH" +
+                        compactOption +
+                        " CLUSTERING ORDER BY (b ASC, c DESC)"
+            );
+
+            execute("INSERT INTO %s (a, b, c) VALUES ('a', 2, 4)");
+            execute("INSERT INTO %s (a, b, c) VALUES ('a', 3, 5)");
+            assertRows(execute("SELECT * FROM %s WHERE a = 'a' AND b > 0"),
+                       row("a", 2, 4),
+                       row("a", 3, 5));
+
+            assertRows(execute("SELECT * FROM %s WHERE a = 'a' AND b > 0 ORDER BY b ASC"),
+                       row("a", 2, 4),
+                       row("a", 3, 5));
+        }
+    }
+
+    @Test
     public void testOverlyLargeSelectPK() throws Throwable
     {
         createTable("CREATE TABLE %s (a text, b text, PRIMARY KEY ((a), b))");
@@ -1297,19 +2312,16 @@
         assertRows(execute("SELECT * FROM %s WHERE pk = 1 AND  c1 > 1 AND c2 > 2 AND c3 = 3 AND v = 3 ALLOW FILTERING;"),
                    row(1, 3, 3, 3, 3));
 
-        assertRows(execute("SELECT * FROM %s WHERE pk = 1 AND  (c1, c2) > (1, 3) AND c3 = 3 AND v = 3 ALLOW FILTERING;"),
-                   row(1, 3, 3, 3, 3));
+        assertRows(execute("SELECT * FROM %s WHERE pk = 1 AND  c1 IN(0,1,2) AND c2 = 1 AND v = 3 ALLOW FILTERING;"),
+                   row(1, 1, 1, 3, 3));
 
-        assertInvalidMessage("Clustering column \"c1\" cannot be restricted by an IN relation",
-                             "SELECT * FROM %s WHERE pk = 1 AND  c1 IN(0,1,2) AND c2 = 1 AND v = 3 ALLOW FILTERING;");
+        assertRows(execute("SELECT * FROM %s WHERE pk = 1 AND  c1 IN(0,1,2) AND c2 = 1 AND v = 3"),
+                   row(1, 1, 1, 3, 3));
 
-        assertInvalidMessage("Clustering column \"c1\" cannot be restricted by an IN relation",
-                             "SELECT * FROM %s WHERE pk = 1 AND  c1 IN(0,1,2) AND c2 = 1 AND v = 3");
-
-        assertInvalidMessage("PRIMARY KEY column \"c2\" cannot be restricted (preceding column \"c1\" is restricted by a non-EQ relation)",
+        assertInvalidMessage("Clustering column \"c2\" cannot be restricted (preceding column \"c1\" is restricted by a non-EQ relation)",
                              "SELECT * FROM %s WHERE pk = 1 AND  c1 > 0 AND c1 < 5 AND c2 = 1 ALLOW FILTERING;");
 
-        assertInvalidMessage("PRIMARY KEY column \"c2\" cannot be restricted (preceding column \"c1\" is not restricted)",
+        assertInvalidMessage("PRIMARY KEY column \"c2\" cannot be restricted as preceding column \"c1\" is not restricted",
                              "SELECT * FROM %s WHERE pk = 1 AND  c2 = 1 ALLOW FILTERING;");
     }
 
@@ -1317,7 +2329,7 @@
     public void testIndexQueryWithCompositePartitionKey() throws Throwable
     {
         createTable("CREATE TABLE %s (p1 int, p2 int, v int, PRIMARY KEY ((p1, p2)))");
-        assertInvalidMessage("Partition key part p2 must be restricted since preceding part is",
+        assertInvalidMessage("Partition key parts: p2 must be restricted as other parts are",
                              "SELECT * FROM %s WHERE p1 = 1 AND v = 3 ALLOW FILTERING");
 
         createIndex("CREATE INDEX ON %s(v)");
@@ -1397,10 +2409,6 @@
                 assertInvalidMessage("Missing PRIMARY KEY part c",
                                      "INSERT INTO %s (pk, c, v) VALUES (?, ?, ?)",
                                      bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("4"));
-
-                // Test restrictions on non-primary key value
-                assertInvalidMessage("Predicates on the non-primary-key column (v) of a COMPACT table are not yet supported",
-                                     "SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND v = textAsBlob('') ALLOW FILTERING;");
             }
             else
             {
@@ -1455,12 +2463,12 @@
                     assertEmpty(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND (c) < (textAsBlob(''));"));
 
                     assertEmpty(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c >= textAsBlob('') AND c < textAsBlob('');"));
-
-                    // Test restrictions on non-primary key value
-                    assertInvalidMessage("No secondary indexes on the restricted columns support the provided operators",
-                                         "SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND v = textAsBlob('') ALLOW FILTERING;");
                 }
             }
+
+            // Test restrictions on non-primary key value
+            assertInvalidMessage("Predicates on non-primary-key columns (v) are not yet supported for non secondary index queries",
+                                 "SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND v = textAsBlob('') ALLOW FILTERING;");
         }
     }
 
@@ -1484,10 +2492,11 @@
 
                 assertEmpty(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND (c1, c2) = (textAsBlob('1'), textAsBlob(''));"));
 
-                assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c1 = textAsBlob('1') AND c2 IN (textAsBlob(''), textAsBlob('1'));"),
+                assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c1 IN (textAsBlob(''), textAsBlob('1')) AND c2 = textAsBlob('1');"),
                            row(bytes("foo123"), bytes("1"), bytes("1"), bytes("1")));
 
-                assertEmpty(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c1 = textAsBlob('') AND c2 = textAsBlob('1');"));
+                assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c1 = textAsBlob('1') AND c2 IN (textAsBlob(''), textAsBlob('1'));"),
+                           row(bytes("foo123"), bytes("1"), bytes("1"), bytes("1")));
 
                 assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND (c1, c2) IN ((textAsBlob(''), textAsBlob('1')), (textAsBlob('1'), textAsBlob('1')));"),
                            row(bytes("foo123"), bytes("1"), bytes("1"), bytes("1")));
@@ -1530,10 +2539,8 @@
                 assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND (c1, c2) = (textAsBlob(''), textAsBlob('1'));"),
                            row(bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("1"), bytes("4")));
 
-                assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c1 = textAsBlob('') AND c2 IN (textAsBlob(''), textAsBlob('1'));"),
-                           row(bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("1"), bytes("4")));
-
-                assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c1 = textAsBlob('1') AND c2 IN (textAsBlob(''), textAsBlob('1'));"),
+                assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c1 IN (textAsBlob(''), textAsBlob('1')) AND c2 = textAsBlob('1');"),
+                           row(bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("1"), bytes("4")),
                            row(bytes("foo123"), bytes("1"), bytes("1"), bytes("1")));
 
                 assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND (c1, c2) IN ((textAsBlob(''), textAsBlob('1')), (textAsBlob('1'), textAsBlob('1')));"),
@@ -1593,7 +2600,6 @@
                 assertEmpty(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c < textAsBlob('')" + orderingClause));
 
                 assertEmpty(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c <= textAsBlob('')" + orderingClause));
-
             }
 
             if (options.contains("COMPACT"))
@@ -1680,10 +2686,8 @@
                 if (flush)
                     flush();
 
-                assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c1 = textAsBlob('1') AND c2 IN (textAsBlob(''), textAsBlob('1'))" + orderingClause),
-                           row(bytes("foo123"), bytes("1"), bytes("1"), bytes("1")));
-
-                assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c1 = textAsBlob('') AND c2 IN (textAsBlob(''), textAsBlob('1'))" + orderingClause),
+                assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c1 IN (textAsBlob(''), textAsBlob('1')) AND c2 = textAsBlob('1')" + orderingClause),
+                           row(bytes("foo123"), bytes("1"), bytes("1"), bytes("1")),
                            row(bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("1"), bytes("4")));
 
                 assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND (c1, c2) IN ((textAsBlob(''), textAsBlob('1')), (textAsBlob('1'), textAsBlob('1')))" + orderingClause),
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/UpdateTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/UpdateTest.java
index 19855d8..c5d153f 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/UpdateTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/UpdateTest.java
@@ -56,4 +56,31 @@
         assertRows(execute("SELECT * FROM %s"),
                    row(ByteBufferUtil.bytes("ɸ"), ByteBufferUtil.bytes("ɸ"), set(ByteBufferUtil.bytes("ɸ")), ByteBufferUtil.bytes("ɸ")));
     }
+
+    @Test
+    public void testTypeCasts() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, t text, a ascii, d double, i int)");
+
+        // The followings is fine
+        execute("UPDATE %s SET t = 'foo' WHERE k = ?", 0);
+        execute("UPDATE %s SET t = (ascii)'foo' WHERE k = ?", 0);
+        execute("UPDATE %s SET t = (text)(ascii)'foo' WHERE k = ?", 0);
+        execute("UPDATE %s SET a = 'foo' WHERE k = ?", 0);
+        execute("UPDATE %s SET a = (ascii)'foo' WHERE k = ?", 0);
+
+        // But trying to put some explicitely type-casted text into an ascii
+        // column should be rejected (even though the text is actually ascci)
+        assertInvalid("UPDATE %s SET a = (text)'foo' WHERE k = ?", 0);
+
+        // This is also fine because integer constants works for both integer and float types
+        execute("UPDATE %s SET i = 3 WHERE k = ?", 0);
+        execute("UPDATE %s SET i = (int)3 WHERE k = ?", 0);
+        execute("UPDATE %s SET d = 3 WHERE k = ?", 0);
+        execute("UPDATE %s SET d = (double)3 WHERE k = ?", 0);
+
+        // But values for ints and doubles are not truly compatible (their binary representation differs)
+        assertInvalid("UPDATE %s SET d = (int)3 WHERE k = ?", 0);
+        assertInvalid("UPDATE %s SET i = (double)3 WHERE k = ?", 0);
+    }
 }
diff --git a/test/unit/org/apache/cassandra/db/ArrayBackedSortedColumnsTest.java b/test/unit/org/apache/cassandra/db/ArrayBackedSortedColumnsTest.java
index 18851d4..0fdabe9 100644
--- a/test/unit/org/apache/cassandra/db/ArrayBackedSortedColumnsTest.java
+++ b/test/unit/org/apache/cassandra/db/ArrayBackedSortedColumnsTest.java
@@ -21,6 +21,8 @@
 package org.apache.cassandra.db;
 
 import java.util.*;
+
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import static org.junit.Assert.*;
@@ -29,15 +31,32 @@
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.db.filter.ColumnSlice;
 import org.apache.cassandra.db.marshal.Int32Type;
-import org.apache.cassandra.utils.BatchRemoveIterator;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.SearchIterator;
+import org.apache.cassandra.utils.BatchRemoveIterator;
 
-public class ArrayBackedSortedColumnsTest extends SchemaLoader
+public class ArrayBackedSortedColumnsTest
 {
+    private static final String KEYSPACE1 = "ArrayBackedSortedColumnsTest";
+    private static final String CF_STANDARD1 = "Standard1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1));
+    }
+
     @Test
     public void testAdd()
     {
@@ -47,7 +66,7 @@
 
     private CFMetaData metadata()
     {
-        return Schema.instance.getCFMetaData("Keyspace1", "Standard1");
+        return Schema.instance.getCFMetaData(KEYSPACE1, CF_STANDARD1);
     }
 
     private void testAddInternal(boolean reversed)
@@ -216,6 +235,43 @@
         assertSame(map.iterator(), map.iterator(ColumnSlice.ALL_COLUMNS_ARRAY));
     }
 
+    @Test
+    public void testSearchIterator()
+    {
+        CellNameType type = new SimpleDenseCellNameType(Int32Type.instance);
+        ColumnFamily map = ArrayBackedSortedColumns.factory.create(metadata(), false);
+
+        int[] values = new int[]{ 1, 2, 3, 5, 9, 15, 21, 22 };
+
+        for (int i = 0; i < values.length; ++i)
+            map.addColumn(new BufferCell(type.makeCellName(values[i])));
+
+        SearchIterator<CellName, Cell> iter = map.searchIterator();
+        for (int i = 0 ; i < values.length ; i++)
+            assertSame(values[i], iter.next(type.makeCellName(values[i])));
+
+        iter = map.searchIterator();
+        for (int i = 0 ; i < values.length ; i+=2)
+            assertSame(values[i], iter.next(type.makeCellName(values[i])));
+
+        iter = map.searchIterator();
+        for (int i = 0 ; i < values.length ; i+=4)
+            assertSame(values[i], iter.next(type.makeCellName(values[i])));
+
+        iter = map.searchIterator();
+        for (int i = 0 ; i < values.length ; i+=1)
+        {
+            if (i % 2 == 0)
+            {
+                Cell cell = iter.next(type.makeCellName(values[i] - 1));
+                if (i > 0 && values[i - 1] == values[i] - 1)
+                    assertSame(values[i - 1], cell);
+                else
+                    assertNull(cell);
+            }
+        }
+    }
+
     private <T> void assertSame(Iterable<T> c1, Iterable<T> c2)
     {
         assertSame(c1.iterator(), c2.iterator());
@@ -229,6 +285,11 @@
             fail("The collection don't have the same size");
     }
 
+    private void assertSame(int name, Cell cell)
+    {
+        int value = ByteBufferUtil.toInt(cell.name().toByteBuffer());
+        assert name == value : "Expected " + name + " but got " + value;
+    }
     private void assertSame(int[] names, Iterator<Cell> iter)
     {
         for (int name : names)
diff --git a/test/unit/org/apache/cassandra/db/BatchlogManagerTest.java b/test/unit/org/apache/cassandra/db/BatchlogManagerTest.java
index 13b7150..70d1d0c 100644
--- a/test/unit/org/apache/cassandra/db/BatchlogManagerTest.java
+++ b/test/unit/org/apache/cassandra/db/BatchlogManagerTest.java
@@ -23,17 +23,21 @@
 import java.util.concurrent.ExecutionException;
 
 import com.google.common.collect.Lists;
+import org.junit.BeforeClass;
 import org.junit.Before;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.db.composites.CellNameType;
 import org.apache.cassandra.db.commitlog.ReplayPosition;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.locator.TokenMetadata;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.UUIDGen;
@@ -43,8 +47,25 @@
 
 import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
 
-public class BatchlogManagerTest extends SchemaLoader
+public class BatchlogManagerTest
 {
+    private static final String KEYSPACE1 = "BatchlogManagerTest1";
+    private static final String CF_STANDARD1 = "Standard1";
+    private static final String CF_STANDARD2 = "Standard2";
+    private static final String CF_STANDARD3 = "Standard3";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                SimpleStrategy.class,
+                KSMetaData.optsWithRF(1),
+                SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1),
+                SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD2),
+                SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD3));
+    }
+
     @Before
     public void setUp() throws Exception
     {
@@ -62,10 +83,10 @@
 
         // Generate 1000 mutations and put them all into the batchlog.
         // Half (500) ready to be replayed, half not.
-        CellNameType comparator = Keyspace.open("Keyspace1").getColumnFamilyStore("Standard1").metadata.comparator;
+        CellNameType comparator = Keyspace.open(KEYSPACE1).getColumnFamilyStore("Standard1").metadata.comparator;
         for (int i = 0; i < 1000; i++)
         {
-            Mutation mutation = new Mutation("Keyspace1", bytes(i));
+            Mutation mutation = new Mutation(KEYSPACE1, bytes(i));
             mutation.add("Standard1", comparator.makeCellName(bytes(i)), bytes(i), System.currentTimeMillis());
 
             long timestamp = i < 500
@@ -76,11 +97,11 @@
                                                    UUIDGen.getTimeUUID(),
                                                    MessagingService.current_version,
                                                    timestamp)
-                           .apply();
+                           .applyUnsafe();
         }
 
         // Flush the batchlog to disk (see CASSANDRA-6822).
-        Keyspace.open(Keyspace.SYSTEM_KS).getColumnFamilyStore(SystemKeyspace.BATCHLOG_CF).forceBlockingFlush();
+        Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStore(SystemKeyspace.BATCHLOG).forceBlockingFlush();
 
         assertEquals(1000, BatchlogManager.instance.countAllBatches() - initialAllBatches);
         assertEquals(0, BatchlogManager.instance.getTotalBatchesReplayed() - initialReplayedBatches);
@@ -94,7 +115,7 @@
 
         for (int i = 0; i < 1000; i++)
         {
-            UntypedResultSet result = QueryProcessor.executeInternal(String.format("SELECT * FROM \"Keyspace1\".\"Standard1\" WHERE key = intAsBlob(%d)", i));
+            UntypedResultSet result = QueryProcessor.executeInternal(String.format("SELECT * FROM \"%s\".\"%s\" WHERE key = intAsBlob(%d)", KEYSPACE1, CF_STANDARD1, i));
             if (i < 500)
             {
                 assertEquals(bytes(i), result.one().getBytes("key"));
@@ -108,23 +129,23 @@
         }
 
         // Ensure that no stray mutations got somehow applied.
-        UntypedResultSet result = QueryProcessor.executeInternal(String.format("SELECT count(*) FROM \"Keyspace1\".\"Standard1\""));
+        UntypedResultSet result = QueryProcessor.executeInternal(String.format("SELECT count(*) FROM \"%s\".\"%s\"", KEYSPACE1, CF_STANDARD1));
         assertEquals(500, result.one().getLong("count"));
     }
 
     @Test
     public void testTruncatedReplay() throws InterruptedException, ExecutionException
     {
-        CellNameType comparator2 = Keyspace.open("Keyspace1").getColumnFamilyStore("Standard2").metadata.comparator;
-        CellNameType comparator3 = Keyspace.open("Keyspace1").getColumnFamilyStore("Standard3").metadata.comparator;
+        CellNameType comparator2 = Keyspace.open(KEYSPACE1).getColumnFamilyStore("Standard2").metadata.comparator;
+        CellNameType comparator3 = Keyspace.open(KEYSPACE1).getColumnFamilyStore("Standard3").metadata.comparator;
         // Generate 2000 mutations (1000 batchlog entries) and put them all into the batchlog.
         // Each batchlog entry with a mutation for Standard2 and Standard3.
         // In the middle of the process, 'truncate' Standard2.
         for (int i = 0; i < 1000; i++)
         {
-            Mutation mutation1 = new Mutation("Keyspace1", bytes(i));
+            Mutation mutation1 = new Mutation(KEYSPACE1, bytes(i));
             mutation1.add("Standard2", comparator2.makeCellName(bytes(i)), bytes(i), 0);
-            Mutation mutation2 = new Mutation("Keyspace1", bytes(i));
+            Mutation mutation2 = new Mutation(KEYSPACE1, bytes(i));
             mutation2.add("Standard3", comparator3.makeCellName(bytes(i)), bytes(i), 0);
             List<Mutation> mutations = Lists.newArrayList(mutation1, mutation2);
 
@@ -132,7 +153,7 @@
             long timestamp = System.currentTimeMillis() - DatabaseDescriptor.getWriteRpcTimeout() * 2;
 
             if (i == 500)
-                SystemKeyspace.saveTruncationRecord(Keyspace.open("Keyspace1").getColumnFamilyStore("Standard2"),
+                SystemKeyspace.saveTruncationRecord(Keyspace.open(KEYSPACE1).getColumnFamilyStore("Standard2"),
                                                     timestamp,
                                                     ReplayPosition.NONE);
 
@@ -146,11 +167,11 @@
                                                    UUIDGen.getTimeUUID(),
                                                    MessagingService.current_version,
                                                    timestamp * 1000)
-                           .apply();
+                           .applyUnsafe();
         }
 
         // Flush the batchlog to disk (see CASSANDRA-6822).
-        Keyspace.open(Keyspace.SYSTEM_KS).getColumnFamilyStore(SystemKeyspace.BATCHLOG_CF).forceFlush();
+        Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStore(SystemKeyspace.BATCHLOG).forceFlush();
 
         // Force batchlog replay and wait for it to complete.
         BatchlogManager.instance.startBatchlogReplay().get();
@@ -158,7 +179,7 @@
         // We should see half of Standard2-targeted mutations written after the replay and all of Standard3 mutations applied.
         for (int i = 0; i < 1000; i++)
         {
-            UntypedResultSet result = QueryProcessor.executeInternal(String.format("SELECT * FROM \"Keyspace1\".\"Standard2\" WHERE key = intAsBlob(%d)", i));
+            UntypedResultSet result = QueryProcessor.executeInternal(String.format("SELECT * FROM \"%s\".\"%s\" WHERE key = intAsBlob(%d)", KEYSPACE1, CF_STANDARD2,i));
             if (i >= 500)
             {
                 assertEquals(bytes(i), result.one().getBytes("key"));
@@ -173,7 +194,7 @@
 
         for (int i = 0; i < 1000; i++)
         {
-            UntypedResultSet result = QueryProcessor.executeInternal(String.format("SELECT * FROM \"Keyspace1\".\"Standard3\" WHERE key = intAsBlob(%d)", i));
+            UntypedResultSet result = QueryProcessor.executeInternal(String.format("SELECT * FROM \"%s\".\"%s\" WHERE key = intAsBlob(%d)", KEYSPACE1, CF_STANDARD3, i));
             assertEquals(bytes(i), result.one().getBytes("key"));
             assertEquals(bytes(i), result.one().getBytes("column1"));
             assertEquals(bytes(i), result.one().getBytes("value"));
diff --git a/test/unit/org/apache/cassandra/db/CleanupTest.java b/test/unit/org/apache/cassandra/db/CleanupTest.java
index 7f54ed7..5777af4 100644
--- a/test/unit/org/apache/cassandra/db/CleanupTest.java
+++ b/test/unit/org/apache/cassandra/db/CleanupTest.java
@@ -30,18 +30,22 @@
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.cql3.Operator;
+import org.apache.cassandra.db.filter.IDiskAtomFilter;
 import org.apache.cassandra.db.columniterator.IdentityQueryFilter;
 import org.apache.cassandra.db.compaction.CompactionManager;
-import org.apache.cassandra.db.filter.IDiskAtomFilter;
 import org.apache.cassandra.db.index.SecondaryIndex;
-import org.apache.cassandra.dht.BytesToken;
+import org.apache.cassandra.dht.ByteOrderedPartitioner.BytesToken;
 import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.locator.TokenMetadata;
 import org.apache.cassandra.service.StorageService;
@@ -49,10 +53,10 @@
 
 import static org.junit.Assert.assertEquals;
 
-public class CleanupTest extends SchemaLoader
+public class CleanupTest
 {
     public static final int LOOPS = 200;
-    public static final String KEYSPACE1 = "Keyspace1";
+    public static final String KEYSPACE1 = "CleanupTest1";
     public static final String CF1 = "Indexed1";
     public static final String CF2 = "Standard1";
     public static final ByteBuffer COLUMN = ByteBufferUtil.bytes("birthdate");
@@ -63,6 +67,17 @@
         VALUE.flip();
     }
 
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF2),
+                                    SchemaLoader.indexCFMD(KEYSPACE1, CF1, true));
+    }
+
     @Test
     public void testCleanup() throws ExecutionException, InterruptedException
     {
diff --git a/test/unit/org/apache/cassandra/db/CollationControllerTest.java b/test/unit/org/apache/cassandra/db/CollationControllerTest.java
index 22c60b8..c227816 100644
--- a/test/unit/org/apache/cassandra/db/CollationControllerTest.java
+++ b/test/unit/org/apache/cassandra/db/CollationControllerTest.java
@@ -18,50 +18,69 @@
 */
 package org.apache.cassandra.db;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.filter.QueryFilter;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
 import static org.junit.Assert.assertEquals;
 
-public class CollationControllerTest extends SchemaLoader
+public class CollationControllerTest
 {
+    private static final String KEYSPACE1 = "CollationControllerTest";
+    private static final String CF = "Standard1";
+    private static final String CFGCGRACE = "StandardGCGS0";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CFGCGRACE).gcGraceSeconds(0));
+    }
+
     @Test
     public void getTopLevelColumnsSkipsSSTablesModifiedBeforeRowDelete() 
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
         Mutation rm;
         DecoratedKey dk = Util.dk("key1");
         
         // add data
         rm = new Mutation(keyspace.getName(), dk.getKey());
         rm.add(cfs.name, Util.cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
         
         // remove
         rm = new Mutation(keyspace.getName(), dk.getKey());
         rm.delete(cfs.name, 10);
-        rm.apply();
+        rm.applyUnsafe();
         
         // add another mutation because sstable maxtimestamp isn't set
         // correctly during flush if the most recent mutation is a row delete
         rm = new Mutation(keyspace.getName(), Util.dk("key2").getKey());
         rm.add(cfs.name, Util.cellname("Column1"), ByteBufferUtil.bytes("zxcv"), 20);
-        rm.apply();
+        rm.applyUnsafe();
         
         cfs.forceBlockingFlush();
 
         // add yet one more mutation
         rm = new Mutation(keyspace.getName(), dk.getKey());
         rm.add(cfs.name, Util.cellname("Column1"), ByteBufferUtil.bytes("foobar"), 30);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         // A NamesQueryFilter goes down one code path (through collectTimeOrderedData())
@@ -83,8 +102,8 @@
     @Test
     public void ensureTombstonesAppliedAfterGCGS()
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("StandardGCGS0");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CFGCGRACE);
         cfs.disableAutoCompaction();
 
         Mutation rm;
@@ -94,13 +113,13 @@
         // add data
         rm = new Mutation(keyspace.getName(), dk.getKey());
         rm.add(cfs.name, cellName, ByteBufferUtil.bytes("asdf"), 0);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         // remove
         rm = new Mutation(keyspace.getName(), dk.getKey());
         rm.delete(cfs.name, cellName, 0);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         // use "realistic" query times since we'll compare these numbers to the local deletion time of the tombstone
diff --git a/test/unit/org/apache/cassandra/db/ColumnFamilyMetricTest.java b/test/unit/org/apache/cassandra/db/ColumnFamilyMetricTest.java
index 9181dc0..2d89e09 100644
--- a/test/unit/org/apache/cassandra/db/ColumnFamilyMetricTest.java
+++ b/test/unit/org/apache/cassandra/db/ColumnFamilyMetricTest.java
@@ -20,34 +20,49 @@
 import java.nio.ByteBuffer;
 import java.util.Collection;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
-
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
+import com.google.common.base.Supplier;
+
 import static org.junit.Assert.assertEquals;
 import static org.apache.cassandra.Util.cellname;
 
-public class ColumnFamilyMetricTest extends SchemaLoader
+public class ColumnFamilyMetricTest
 {
+    @BeforeClass
+    public static void defineSchema() throws Exception
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace("Keyspace1",
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD("Keyspace1", "Standard2"));
+    }
+
     @Test
     public void testSizeMetric()
     {
         Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
+        final ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard2");
         store.disableAutoCompaction();
 
         store.truncateBlocking();
 
-        assertEquals(0, store.metric.liveDiskSpaceUsed.count());
-        assertEquals(0, store.metric.totalDiskSpaceUsed.count());
+        assertEquals(0, store.metric.liveDiskSpaceUsed.getCount());
+        assertEquals(0, store.metric.totalDiskSpaceUsed.getCount());
 
         for (int j = 0; j < 10; j++)
         {
             ByteBuffer key = ByteBufferUtil.bytes(String.valueOf(j));
             Mutation rm = new Mutation("Keyspace1", key);
-            rm.add("Standard1", cellname("0"), ByteBufferUtil.EMPTY_BYTE_BUFFER, j);
+            rm.add("Standard2", cellname("0"), ByteBufferUtil.EMPTY_BYTE_BUFFER, j);
             rm.apply();
         }
         store.forceBlockingFlush();
@@ -59,16 +74,59 @@
         }
 
         // size metrics should show the sum of all SSTable sizes
-        assertEquals(size, store.metric.liveDiskSpaceUsed.count());
-        assertEquals(size, store.metric.totalDiskSpaceUsed.count());
+        assertEquals(size, store.metric.liveDiskSpaceUsed.getCount());
+        assertEquals(size, store.metric.totalDiskSpaceUsed.getCount());
 
         store.truncateBlocking();
 
         // after truncate, size metrics should be down to 0
-        assertEquals(0, store.metric.liveDiskSpaceUsed.count());
-        assertEquals(0, store.metric.totalDiskSpaceUsed.count());
+        Util.spinAssertEquals(
+                0L,
+                new Supplier<Object>()
+                {
+                    public Long get()
+                    {
+                        return store.metric.liveDiskSpaceUsed.getCount();
+                    }
+                },
+                30);
+        Util.spinAssertEquals(
+                0L,
+                new Supplier<Object>()
+                {
+                    public Long get()
+                    {
+                        return store.metric.totalDiskSpaceUsed.getCount();
+                    }
+                },
+                30);
 
         store.enableAutoCompaction();
     }
 
+    @Test
+    public void testColUpdateTimeDeltaFiltering()
+    {
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard2");
+
+        // This confirms another test/set up did not overflow the histogram
+        store.metric.colUpdateTimeDeltaHistogram.cf.getSnapshot().get999thPercentile();
+
+        ByteBuffer key = ByteBufferUtil.bytes(4242);
+        Mutation m = new Mutation("Keyspace1", key);
+        m.add("Standard2", cellname("0"), ByteBufferUtil.bytes("0"), 0);
+        m.apply();
+
+        // The histogram should not have overflowed on the first write
+        store.metric.colUpdateTimeDeltaHistogram.cf.getSnapshot().get999thPercentile();
+
+        m = new Mutation("Keyspace1", key);
+        // smallest time delta that would overflow the histogram if unfiltered
+        m.add("Standard2", cellname("0"), ByteBufferUtil.bytes("1"), 18165375903307L);
+        m.apply();
+
+        // CASSANDRA-11117 - update with large timestamp delta should not overflow the histogram
+        store.metric.colUpdateTimeDeltaHistogram.cf.getSnapshot().get999thPercentile();
+    }
 }
diff --git a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
index 35814f0..b21d3fa 100644
--- a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
+++ b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
@@ -22,7 +22,20 @@
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.charset.CharacterCodingException;
-import java.util.*;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.UUID;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
@@ -30,8 +43,15 @@
 import com.google.common.base.Function;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Sets;
+
+import org.apache.cassandra.db.index.PerRowSecondaryIndexTest;
+import org.apache.cassandra.io.sstable.*;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
 import org.apache.commons.lang3.ArrayUtils;
 import org.apache.commons.lang3.StringUtils;
+import org.junit.Assume;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 
@@ -41,22 +61,40 @@
 import org.apache.cassandra.config.*;
 import org.apache.cassandra.cql3.Operator;
 import org.apache.cassandra.db.columniterator.IdentityQueryFilter;
-import org.apache.cassandra.db.composites.*;
-import org.apache.cassandra.db.filter.*;
-import org.apache.cassandra.db.index.PerRowSecondaryIndexTest;
+import org.apache.cassandra.db.composites.CellName;
+import org.apache.cassandra.db.composites.CellNameType;
+import org.apache.cassandra.db.composites.CellNames;
+import org.apache.cassandra.db.composites.Composites;
+import org.apache.cassandra.db.filter.ColumnSlice;
+import org.apache.cassandra.db.filter.ExtendedFilter;
+import org.apache.cassandra.db.filter.IDiskAtomFilter;
+import org.apache.cassandra.db.filter.NamesQueryFilter;
+import org.apache.cassandra.db.filter.QueryFilter;
+import org.apache.cassandra.db.filter.SliceQueryFilter;
 import org.apache.cassandra.db.index.SecondaryIndex;
+import org.apache.cassandra.db.marshal.IntegerType;
 import org.apache.cassandra.db.marshal.LexicalUUIDType;
 import org.apache.cassandra.db.marshal.LongType;
-import org.apache.cassandra.dht.*;
-import org.apache.cassandra.io.sstable.*;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.Bounds;
+import org.apache.cassandra.dht.ExcludingBounds;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.IncludingExcludingBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.metrics.ClearableHistogram;
 import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.thrift.SlicePredicate;
 import org.apache.cassandra.thrift.SliceRange;
 import org.apache.cassandra.thrift.ThriftValidation;
-import org.apache.cassandra.utils.*;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.WrappedRunnable;
 
 import static org.apache.cassandra.Util.cellname;
 import static org.apache.cassandra.Util.column;
@@ -70,9 +108,25 @@
 import static org.junit.Assert.assertTrue;
 
 @RunWith(OrderedJUnit4ClassRunner.class)
-public class ColumnFamilyStoreTest extends SchemaLoader
+public class ColumnFamilyStoreTest
 {
     static byte[] bytes1, bytes2;
+    public static final String KEYSPACE1 = "ColumnFamilyStoreTest1";
+    public static final String KEYSPACE2 = "ColumnFamilyStoreTest2";
+    public static final String KEYSPACE3 = "ColumnFamilyStoreTest3";
+    public static final String KEYSPACE4 = "PerRowSecondaryIndex";
+    public static final String CF_STANDARD1 = "Standard1";
+    public static final String CF_STANDARD2 = "Standard2";
+    public static final String CF_STANDARD3 = "Standard3";
+    public static final String CF_STANDARD4 = "Standard4";
+    public static final String CF_STANDARD5 = "Standard5";
+    public static final String CF_STANDARD6 = "Standard6";
+    public static final String CF_STANDARDINT = "StandardInteger1";
+    public static final String CF_SUPER1 = "Super1";
+    public static final String CF_SUPER6 = "Super6";
+    public static final String CF_INDEX1 = "Indexed1";
+    public static final String CF_INDEX2 = "Indexed2";
+    public static final String CF_INDEX3 = "Indexed3";
 
     static
     {
@@ -83,67 +137,102 @@
         random.nextBytes(bytes2);
     }
 
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD2),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD3),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD4),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD5),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD6),
+                                    SchemaLoader.indexCFMD(KEYSPACE1, CF_INDEX1, true),
+                                    SchemaLoader.indexCFMD(KEYSPACE1, CF_INDEX2, false),
+                                    SchemaLoader.superCFMD(KEYSPACE1, CF_SUPER1, LongType.instance),
+                                    SchemaLoader.superCFMD(KEYSPACE1, CF_SUPER6, LexicalUUIDType.instance, UTF8Type.instance),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARDINT, IntegerType.instance));
+        SchemaLoader.createKeyspace(KEYSPACE2,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE2, CF_STANDARD1),
+                                    SchemaLoader.indexCFMD(KEYSPACE2, CF_INDEX1, true),
+                                    SchemaLoader.compositeIndexCFMD(KEYSPACE2, CF_INDEX2, true),
+                                    SchemaLoader.compositeIndexCFMD(KEYSPACE2, CF_INDEX3, true).gcGraceSeconds(0));
+        SchemaLoader.createKeyspace(KEYSPACE3,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(5),
+                                    SchemaLoader.indexCFMD(KEYSPACE3, CF_INDEX1, true));
+        SchemaLoader.createKeyspace(KEYSPACE4,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.perRowIndexedCFMD(KEYSPACE4, "Indexed1"));
+    }
+
     @Test
     // create two sstables, and verify that we only deserialize data from the most recent one
     public void testTimeSortedQuery()
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_STANDARD1);
         cfs.truncateBlocking();
 
         Mutation rm;
-        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("key1"));
-        rm.add("Standard1", cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
-        rm.apply();
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("key1"));
+        rm.add(CF_STANDARD1, cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
-        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("key1"));
-        rm.add("Standard1", cellname("Column1"), ByteBufferUtil.bytes("asdf"), 1);
-        rm.apply();
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("key1"));
+        rm.add(CF_STANDARD1, cellname("Column1"), ByteBufferUtil.bytes("asdf"), 1);
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
-        cfs.getRecentSSTablesPerReadHistogram(); // resets counts
+        ((ClearableHistogram)cfs.metric.sstablesPerReadHistogram.cf).clear(); // resets counts
         cfs.getColumnFamily(Util.namesQueryFilter(cfs, Util.dk("key1"), "Column1"));
-        assertEquals(1, cfs.getRecentSSTablesPerReadHistogram()[0]);
+        assertEquals(1, cfs.metric.sstablesPerReadHistogram.cf.getCount());
     }
 
     @Test
     public void testGetColumnWithWrongBF()
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_STANDARD1);
         cfs.truncateBlocking();
 
         List<Mutation> rms = new LinkedList<>();
-        Mutation rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("key1"));
-        rm.add("Standard1", cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
-        rm.add("Standard1", cellname("Column2"), ByteBufferUtil.bytes("asdf"), 0);
+        Mutation rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("key1"));
+        rm.add(CF_STANDARD1, cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
+        rm.add(CF_STANDARD1, cellname("Column2"), ByteBufferUtil.bytes("asdf"), 0);
         rms.add(rm);
         Util.writeColumnFamily(rms);
 
         List<SSTableReader> ssTables = keyspace.getAllSSTables();
         assertEquals(1, ssTables.size());
         ssTables.get(0).forceFilterFailures();
-        ColumnFamily cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(Util.dk("key2"), "Standard1", System.currentTimeMillis()));
+        ColumnFamily cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(Util.dk("key2"), CF_STANDARD1, System.currentTimeMillis()));
         assertNull(cf);
     }
 
     @Test
     public void testEmptyRow() throws Exception
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        final ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard2");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        final ColumnFamilyStore store = keyspace.getColumnFamilyStore(CF_STANDARD2);
         Mutation rm;
 
-        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("key1"));
-        rm.delete("Standard2", System.currentTimeMillis());
-        rm.apply();
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("key1"));
+        rm.delete(CF_STANDARD2, System.currentTimeMillis());
+        rm.applyUnsafe();
 
         Runnable r = new WrappedRunnable()
         {
             public void runMayThrow() throws IOException
             {
-                QueryFilter sliceFilter = QueryFilter.getSliceFilter(Util.dk("key1"), "Standard2", Composites.EMPTY, Composites.EMPTY, false, 1, System.currentTimeMillis());
+                QueryFilter sliceFilter = QueryFilter.getSliceFilter(Util.dk("key1"), CF_STANDARD2, Composites.EMPTY, Composites.EMPTY, false, 1, System.currentTimeMillis());
                 ColumnFamily cf = store.getColumnFamily(sliceFilter);
                 assertTrue(cf.isMarkedForDelete());
                 assertFalse(cf.hasColumns());
@@ -159,6 +248,38 @@
     }
 
     @Test
+    public void testFilterWithNullCF() throws Exception
+    {
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_STANDARD1);
+        final Row row = new Row(Util.dk("key1"), null);
+
+        ColumnFamilyStore.AbstractScanIterator iterator = new ColumnFamilyStore.AbstractScanIterator()
+        {
+            Iterator<Row> it = Collections.singletonList(row).iterator();
+
+            protected Row computeNext()
+            {
+                return it.hasNext() ? it.next() : endOfData();
+            }
+
+            @Override
+            public void close()
+            {
+            }
+        };
+
+        ExtendedFilter filter = ExtendedFilter.create(
+                cfs,
+                DataRange.allData(DatabaseDescriptor.getPartitioner()), null, 1, true, System.currentTimeMillis());
+
+        List<Row> list = cfs.filter(iterator, filter);
+        assert 1 == list.size();
+        assert list.get(0).key == row.key;
+        assert null == list.get(0).cf;
+    }
+
+    @Test
     public void testSkipStartKey()
     {
         ColumnFamilyStore cfs = insertKey1Key2();
@@ -175,30 +296,30 @@
     @Test
     public void testIndexScan()
     {
-        ColumnFamilyStore cfs = Keyspace.open("Keyspace1").getColumnFamilyStore("Indexed1");
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_INDEX1);
         Mutation rm;
         CellName nobirthdate = cellname("notbirthdate");
         CellName birthdate = cellname("birthdate");
 
-        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("k1"));
-        rm.add("Indexed1", nobirthdate, ByteBufferUtil.bytes(1L), 0);
-        rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(1L), 0);
-        rm.apply();
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("k1"));
+        rm.add(CF_INDEX1, nobirthdate, ByteBufferUtil.bytes(1L), 0);
+        rm.add(CF_INDEX1, birthdate, ByteBufferUtil.bytes(1L), 0);
+        rm.applyUnsafe();
 
-        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("k2"));
-        rm.add("Indexed1", nobirthdate, ByteBufferUtil.bytes(2L), 0);
-        rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(2L), 0);
-        rm.apply();
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("k2"));
+        rm.add(CF_INDEX1, nobirthdate, ByteBufferUtil.bytes(2L), 0);
+        rm.add(CF_INDEX1, birthdate, ByteBufferUtil.bytes(2L), 0);
+        rm.applyUnsafe();
 
-        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("k3"));
-        rm.add("Indexed1", nobirthdate, ByteBufferUtil.bytes(2L), 0);
-        rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(1L), 0);
-        rm.apply();
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("k3"));
+        rm.add(CF_INDEX1, nobirthdate, ByteBufferUtil.bytes(2L), 0);
+        rm.add(CF_INDEX1, birthdate, ByteBufferUtil.bytes(1L), 0);
+        rm.applyUnsafe();
 
-        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("k4aaaa"));
-        rm.add("Indexed1", nobirthdate, ByteBufferUtil.bytes(2L), 0);
-        rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(3L), 0);
-        rm.apply();
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("k4aaaa"));
+        rm.add(CF_INDEX1, nobirthdate, ByteBufferUtil.bytes(2L), 0);
+        rm.add(CF_INDEX1, birthdate, ByteBufferUtil.bytes(3L), 0);
+        rm.applyUnsafe();
 
         // basic single-expression query
         IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), Operator.EQ, ByteBufferUtil.bytes(1L));
@@ -260,12 +381,12 @@
     public void testLargeScan()
     {
         Mutation rm;
-        ColumnFamilyStore cfs = Keyspace.open("Keyspace1").getColumnFamilyStore("Indexed1");
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_INDEX1);
         for (int i = 0; i < 100; i++)
         {
-            rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("key" + i));
-            rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(34L), 0);
-            rm.add("Indexed1", cellname("notbirthdate"), ByteBufferUtil.bytes((long) (i % 2)), 0);
+            rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("key" + i));
+            rm.add(CF_INDEX1, cellname("birthdate"), ByteBufferUtil.bytes(34L), 0);
+            rm.add(CF_INDEX1, cellname("notbirthdate"), ByteBufferUtil.bytes((long) (i % 2)), 0);
             rm.applyUnsafe();
         }
 
@@ -288,12 +409,12 @@
     @Test
     public void testIndexDeletions() throws IOException
     {
-        ColumnFamilyStore cfs = Keyspace.open("Keyspace3").getColumnFamilyStore("Indexed1");
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE3).getColumnFamilyStore(CF_INDEX1);
         Mutation rm;
 
-        rm = new Mutation("Keyspace3", ByteBufferUtil.bytes("k1"));
-        rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(1L), 0);
-        rm.apply();
+        rm = new Mutation(KEYSPACE3, ByteBufferUtil.bytes("k1"));
+        rm.add(CF_INDEX1, cellname("birthdate"), ByteBufferUtil.bytes(1L), 0);
+        rm.applyUnsafe();
 
         IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), Operator.EQ, ByteBufferUtil.bytes(1L));
         List<IndexExpression> clause = Arrays.asList(expr);
@@ -305,9 +426,9 @@
         assert "k1".equals( key );
 
         // delete the column directly
-        rm = new Mutation("Keyspace3", ByteBufferUtil.bytes("k1"));
-        rm.delete("Indexed1", cellname("birthdate"), 1);
-        rm.apply();
+        rm = new Mutation(KEYSPACE3, ByteBufferUtil.bytes("k1"));
+        rm.delete(CF_INDEX1, cellname("birthdate"), 1);
+        rm.applyUnsafe();
         rows = cfs.search(range, clause, filter, 100);
         assert rows.isEmpty();
 
@@ -320,59 +441,59 @@
         assert rows.isEmpty();
 
         // resurrect w/ a newer timestamp
-        rm = new Mutation("Keyspace3", ByteBufferUtil.bytes("k1"));
-        rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(1L), 2);
-        rm.apply();
+        rm = new Mutation(KEYSPACE3, ByteBufferUtil.bytes("k1"));
+        rm.add(CF_INDEX1, cellname("birthdate"), ByteBufferUtil.bytes(1L), 2);
+        rm.applyUnsafe();
         rows = cfs.search(range, clause, filter, 100);
         assert rows.size() == 1 : StringUtils.join(rows, ",");
         key = ByteBufferUtil.string(rows.get(0).key.getKey());
         assert "k1".equals( key );
 
         // verify that row and delete w/ older timestamp does nothing
-        rm = new Mutation("Keyspace3", ByteBufferUtil.bytes("k1"));
-        rm.delete("Indexed1", 1);
-        rm.apply();
+        rm = new Mutation(KEYSPACE3, ByteBufferUtil.bytes("k1"));
+        rm.delete(CF_INDEX1, 1);
+        rm.applyUnsafe();
         rows = cfs.search(range, clause, filter, 100);
         assert rows.size() == 1 : StringUtils.join(rows, ",");
         key = ByteBufferUtil.string(rows.get(0).key.getKey());
         assert "k1".equals( key );
 
         // similarly, column delete w/ older timestamp should do nothing
-        rm = new Mutation("Keyspace3", ByteBufferUtil.bytes("k1"));
-        rm.delete("Indexed1", cellname("birthdate"), 1);
-        rm.apply();
+        rm = new Mutation(KEYSPACE3, ByteBufferUtil.bytes("k1"));
+        rm.delete(CF_INDEX1, cellname("birthdate"), 1);
+        rm.applyUnsafe();
         rows = cfs.search(range, clause, filter, 100);
         assert rows.size() == 1 : StringUtils.join(rows, ",");
         key = ByteBufferUtil.string(rows.get(0).key.getKey());
         assert "k1".equals( key );
 
         // delete the entire row (w/ newer timestamp this time)
-        rm = new Mutation("Keyspace3", ByteBufferUtil.bytes("k1"));
-        rm.delete("Indexed1", 3);
-        rm.apply();
+        rm = new Mutation(KEYSPACE3, ByteBufferUtil.bytes("k1"));
+        rm.delete(CF_INDEX1, 3);
+        rm.applyUnsafe();
         rows = cfs.search(range, clause, filter, 100);
         assert rows.isEmpty() : StringUtils.join(rows, ",");
 
         // make sure obsolete mutations don't generate an index entry
-        rm = new Mutation("Keyspace3", ByteBufferUtil.bytes("k1"));
-        rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(1L), 3);
-        rm.apply();
+        rm = new Mutation(KEYSPACE3, ByteBufferUtil.bytes("k1"));
+        rm.add(CF_INDEX1, cellname("birthdate"), ByteBufferUtil.bytes(1L), 3);
+        rm.applyUnsafe();
         rows = cfs.search(range, clause, filter, 100);
         assert rows.isEmpty() : StringUtils.join(rows, ",");
 
         // try insert followed by row delete in the same mutation
-        rm = new Mutation("Keyspace3", ByteBufferUtil.bytes("k1"));
-        rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(1L), 1);
-        rm.delete("Indexed1", 2);
-        rm.apply();
+        rm = new Mutation(KEYSPACE3, ByteBufferUtil.bytes("k1"));
+        rm.add(CF_INDEX1, cellname("birthdate"), ByteBufferUtil.bytes(1L), 1);
+        rm.delete(CF_INDEX1, 2);
+        rm.applyUnsafe();
         rows = cfs.search(range, clause, filter, 100);
         assert rows.isEmpty() : StringUtils.join(rows, ",");
 
         // try row delete followed by insert in the same mutation
-        rm = new Mutation("Keyspace3", ByteBufferUtil.bytes("k1"));
-        rm.delete("Indexed1", 3);
-        rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(1L), 4);
-        rm.apply();
+        rm = new Mutation(KEYSPACE3, ByteBufferUtil.bytes("k1"));
+        rm.delete(CF_INDEX1, 3);
+        rm.add(CF_INDEX1, cellname("birthdate"), ByteBufferUtil.bytes(1L), 4);
+        rm.applyUnsafe();
         rows = cfs.search(range, clause, filter, 100);
         assert rows.size() == 1 : StringUtils.join(rows, ",");
         key = ByteBufferUtil.string(rows.get(0).key.getKey());
@@ -382,18 +503,18 @@
     @Test
     public void testIndexUpdate() throws IOException
     {
-        Keyspace keyspace = Keyspace.open("Keyspace2");
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Indexed1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE2);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_INDEX1);
         CellName birthdate = cellname("birthdate");
 
         // create a row and update the birthdate value, test that the index query fetches the new version
         Mutation rm;
-        rm = new Mutation("Keyspace2", ByteBufferUtil.bytes("k1"));
-        rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(1L), 1);
-        rm.apply();
-        rm = new Mutation("Keyspace2", ByteBufferUtil.bytes("k1"));
-        rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(2L), 2);
-        rm.apply();
+        rm = new Mutation(KEYSPACE2, ByteBufferUtil.bytes("k1"));
+        rm.add(CF_INDEX1, birthdate, ByteBufferUtil.bytes(1L), 1);
+        rm.applyUnsafe();
+        rm = new Mutation(KEYSPACE2, ByteBufferUtil.bytes("k1"));
+        rm.add(CF_INDEX1, birthdate, ByteBufferUtil.bytes(2L), 2);
+        rm.applyUnsafe();
 
         IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), Operator.EQ, ByteBufferUtil.bytes(1L));
         List<IndexExpression> clause = Arrays.asList(expr);
@@ -404,16 +525,16 @@
 
         expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), Operator.EQ, ByteBufferUtil.bytes(2L));
         clause = Arrays.asList(expr);
-        rows = keyspace.getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
+        rows = keyspace.getColumnFamilyStore(CF_INDEX1).search(range, clause, filter, 100);
         String key = ByteBufferUtil.string(rows.get(0).key.getKey());
         assert "k1".equals( key );
 
         // update the birthdate value with an OLDER timestamp, and test that the index ignores this
-        rm = new Mutation("Keyspace2", ByteBufferUtil.bytes("k1"));
-        rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(3L), 0);
-        rm.apply();
+        rm = new Mutation(KEYSPACE2, ByteBufferUtil.bytes("k1"));
+        rm.add(CF_INDEX1, birthdate, ByteBufferUtil.bytes(3L), 0);
+        rm.applyUnsafe();
 
-        rows = keyspace.getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
+        rows = keyspace.getColumnFamilyStore(CF_INDEX1).search(range, clause, filter, 100);
         key = ByteBufferUtil.string(rows.get(0).key.getKey());
         assert "k1".equals( key );
 
@@ -423,13 +544,13 @@
     public void testIndexUpdateOverwritingExpiringColumns() throws Exception
     {
         // see CASSANDRA-7268
-        Keyspace keyspace = Keyspace.open("Keyspace2");
+        Keyspace keyspace = Keyspace.open(KEYSPACE2);
 
         // create a row and update the birthdate value with an expiring column
         Mutation rm;
-        rm = new Mutation("Keyspace2", ByteBufferUtil.bytes("k100"));
+        rm = new Mutation(KEYSPACE2, ByteBufferUtil.bytes("k100"));
         rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(100L), 1, 1000);
-        rm.apply();
+        rm.applyUnsafe();
 
         IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), Operator.EQ, ByteBufferUtil.bytes(100L));
         List<IndexExpression> clause = Arrays.asList(expr);
@@ -442,17 +563,17 @@
         TimeUnit.SECONDS.sleep(1);
 
         // now overwrite with the same name/value/ttl, but the local expiry time will be different
-        rm = new Mutation("Keyspace2", ByteBufferUtil.bytes("k100"));
+        rm = new Mutation(KEYSPACE2, ByteBufferUtil.bytes("k100"));
         rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(100L), 1, 1000);
-        rm.apply();
+        rm.applyUnsafe();
 
         rows = keyspace.getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
         assertEquals(1, rows.size());
 
         // check that modifying the indexed value using the same timestamp behaves as expected
-        rm = new Mutation("Keyspace2", ByteBufferUtil.bytes("k101"));
+        rm = new Mutation(KEYSPACE2, ByteBufferUtil.bytes("k101"));
         rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(101L), 1, 1000);
-        rm.apply();
+        rm.applyUnsafe();
 
         expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), Operator.EQ, ByteBufferUtil.bytes(101L));
         clause = Arrays.asList(expr);
@@ -460,9 +581,9 @@
         assertEquals(1, rows.size());
 
         TimeUnit.SECONDS.sleep(1);
-        rm = new Mutation("Keyspace2", ByteBufferUtil.bytes("k101"));
+        rm = new Mutation(KEYSPACE2, ByteBufferUtil.bytes("k101"));
         rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(102L), 1, 1000);
-        rm.apply();
+        rm.applyUnsafe();
         // search for the old value
         rows = keyspace.getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
         assertEquals(0, rows.size());
@@ -476,15 +597,15 @@
     @Test
     public void testDeleteOfInconsistentValuesInKeysIndex() throws Exception
     {
-        String keySpace = "Keyspace2";
-        String cfName = "Indexed1";
+        String keySpace = KEYSPACE2;
+        String cfName = CF_INDEX1;
 
         Keyspace keyspace = Keyspace.open(keySpace);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
         cfs.truncateBlocking();
 
         ByteBuffer rowKey = ByteBufferUtil.bytes("k1");
-        CellName colName = cellname("birthdate"); 
+        CellName colName = cellname("birthdate");
         ByteBuffer val1 = ByteBufferUtil.bytes(1L);
         ByteBuffer val2 = ByteBufferUtil.bytes(2L);
 
@@ -492,7 +613,7 @@
         Mutation rm;
         rm = new Mutation(keySpace, rowKey);
         rm.add(cfName, colName, val1, 0);
-        rm.apply();
+        rm.applyUnsafe();
         IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), Operator.EQ, val1);
         List<IndexExpression> clause = Arrays.asList(expr);
         IDiskAtomFilter filter = new IdentityQueryFilter();
@@ -539,8 +660,8 @@
     @Test
     public void testDeleteOfInconsistentValuesFromCompositeIndex() throws Exception
     {
-        String keySpace = "Keyspace2";
-        String cfName = "Indexed2";
+        String keySpace = KEYSPACE2;
+        String cfName = CF_INDEX2;
 
         Keyspace keyspace = Keyspace.open(keySpace);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
@@ -548,7 +669,7 @@
 
         ByteBuffer rowKey = ByteBufferUtil.bytes("k1");
         ByteBuffer clusterKey = ByteBufferUtil.bytes("ck1");
-        ByteBuffer colName = ByteBufferUtil.bytes("col1"); 
+        ByteBuffer colName = ByteBufferUtil.bytes("col1");
 
         CellNameType baseComparator = cfs.getComparator();
         CellName compositeName = baseComparator.makeCellName(clusterKey, colName);
@@ -560,7 +681,7 @@
         Mutation rm;
         rm = new Mutation(keySpace, rowKey);
         rm.add(cfName, compositeName, val1, 0);
-        rm.apply();
+        rm.applyUnsafe();
 
         // test that the index query fetches this version
         IndexExpression expr = new IndexExpression(colName, Operator.EQ, val1);
@@ -612,8 +733,8 @@
     @Test
     public void testDeleteCompositeIndex() throws Exception
     {
-        String keySpace = "Keyspace2";
-        String cfName = "Indexed3"; // has gcGrace 0
+        String keySpace = KEYSPACE2;
+        String cfName = CF_INDEX3; // has gcGrace 0
 
         Keyspace keyspace = Keyspace.open(keySpace);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
@@ -632,12 +753,12 @@
         Mutation rm;
         rm = new Mutation(keySpace, rowKey);
         rm.add(cfName, compositeName, val1, 0);
-        rm.apply();
+        rm.applyUnsafe();
 
         // Now delete the value and flush too.
         rm = new Mutation(keySpace, rowKey);
         rm.delete(cfName, 1);
-        rm.apply();
+        rm.applyUnsafe();
 
         // We want the data to be gcable, but even if gcGrace == 0, we still need to wait 1 second
         // since we won't gc on a tie.
@@ -658,31 +779,31 @@
     @Test
     public void testIndexScanWithLimitOne()
     {
-        ColumnFamilyStore cfs = Keyspace.open("Keyspace1").getColumnFamilyStore("Indexed1");
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_INDEX1);
         Mutation rm;
 
         CellName nobirthdate = cellname("notbirthdate");
         CellName birthdate = cellname("birthdate");
 
-        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("kk1"));
-        rm.add("Indexed1", nobirthdate, ByteBufferUtil.bytes(1L), 0);
-        rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(1L), 0);
-        rm.apply();
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("kk1"));
+        rm.add(CF_INDEX1, nobirthdate, ByteBufferUtil.bytes(1L), 0);
+        rm.add(CF_INDEX1, birthdate, ByteBufferUtil.bytes(1L), 0);
+        rm.applyUnsafe();
 
-        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("kk2"));
-        rm.add("Indexed1", nobirthdate, ByteBufferUtil.bytes(2L), 0);
-        rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(1L), 0);
-        rm.apply();
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("kk2"));
+        rm.add(CF_INDEX1, nobirthdate, ByteBufferUtil.bytes(2L), 0);
+        rm.add(CF_INDEX1, birthdate, ByteBufferUtil.bytes(1L), 0);
+        rm.applyUnsafe();
 
-        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("kk3"));
-        rm.add("Indexed1", nobirthdate, ByteBufferUtil.bytes(2L), 0);
-        rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(1L), 0);
-        rm.apply();
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("kk3"));
+        rm.add(CF_INDEX1, nobirthdate, ByteBufferUtil.bytes(2L), 0);
+        rm.add(CF_INDEX1, birthdate, ByteBufferUtil.bytes(1L), 0);
+        rm.applyUnsafe();
 
-        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("kk4"));
-        rm.add("Indexed1", nobirthdate, ByteBufferUtil.bytes(2L), 0);
-        rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(1L), 0);
-        rm.apply();
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("kk4"));
+        rm.add(CF_INDEX1, nobirthdate, ByteBufferUtil.bytes(2L), 0);
+        rm.add(CF_INDEX1, birthdate, ByteBufferUtil.bytes(1L), 0);
+        rm.applyUnsafe();
 
         // basic single-expression query
         IndexExpression expr1 = new IndexExpression(ByteBufferUtil.bytes("birthdate"), Operator.EQ, ByteBufferUtil.bytes(1L));
@@ -699,14 +820,14 @@
     @Test
     public void testIndexCreate() throws IOException, InterruptedException, ExecutionException
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Indexed2");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_INDEX2);
 
         // create a row and update the birthdate value, test that the index query fetches the new version
         Mutation rm;
-        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("k1"));
-        rm.add("Indexed2", cellname("birthdate"), ByteBufferUtil.bytes(1L), 1);
-        rm.apply();
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("k1"));
+        rm.add(CF_INDEX2, cellname("birthdate"), ByteBufferUtil.bytes(1L), 1);
+        rm.applyUnsafe();
 
         ColumnDefinition old = cfs.metadata.getColumnDefinition(ByteBufferUtil.bytes("birthdate"));
         ColumnDefinition cd = ColumnDefinition.regularDef(cfs.metadata, old.name.bytes, old.type, null).setIndex("birthdate_index", IndexType.KEYS, null);
@@ -733,7 +854,7 @@
         IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), Operator.EQ, ByteBufferUtil.bytes(1L));
         List<IndexExpression> clause = Arrays.asList(expr);
         IDiskAtomFilter filter = new IdentityQueryFilter();
-        List<Row> rows = keyspace.getColumnFamilyStore("Indexed2").search(Util.range("", ""), clause, filter, 100);
+        List<Row> rows = keyspace.getColumnFamilyStore(CF_INDEX2).search(Util.range("", ""), clause, filter, 100);
         assert rows.size() == 1 : StringUtils.join(rows, ",");
         assertEquals("k1", ByteBufferUtil.string(rows.get(0).key.getKey()));
     }
@@ -741,27 +862,27 @@
     @Test
     public void testCassandra6778() throws CharacterCodingException
     {
-        String cfname = "StandardInteger1";
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        String cfname = CF_STANDARDINT;
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);
 
         // insert two columns that represent the same integer but have different binary forms (the
         // second one is padded with extra zeros)
-        Mutation rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("k1"));
+        Mutation rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("k1"));
         CellName column1 = cellname(ByteBuffer.wrap(new byte[]{1}));
         rm.add(cfname, column1, ByteBufferUtil.bytes("data1"), 1);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
-        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("k1"));
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("k1"));
         CellName column2 = cellname(ByteBuffer.wrap(new byte[]{0, 0, 1}));
         rm.add(cfname, column2, ByteBufferUtil.bytes("data2"), 2);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         // fetch by the first column name; we should get the second version of the column value
         SliceByNamesReadCommand cmd = new SliceByNamesReadCommand(
-            "Keyspace1", ByteBufferUtil.bytes("k1"), cfname, System.currentTimeMillis(),
+            KEYSPACE1, ByteBufferUtil.bytes("k1"), cfname, System.currentTimeMillis(),
             new NamesQueryFilter(FBUtilities.singleton(column1, cfs.getComparator())));
 
         ColumnFamily cf = cmd.getRow(keyspace).cf;
@@ -772,7 +893,7 @@
 
         // fetch by the second column name; we should get the second version of the column value
         cmd = new SliceByNamesReadCommand(
-            "Keyspace1", ByteBufferUtil.bytes("k1"), cfname, System.currentTimeMillis(),
+            KEYSPACE1, ByteBufferUtil.bytes("k1"), cfname, System.currentTimeMillis(),
             new NamesQueryFilter(FBUtilities.singleton(column2, cfs.getComparator())));
 
         cf = cmd.getRow(keyspace).cf;
@@ -798,8 +919,8 @@
     @Test
     public void testDeleteSuperRowSticksAfterFlush() throws Throwable
     {
-        String keyspaceName = "Keyspace1";
-        String cfName= "Super1";
+        String keyspaceName = KEYSPACE1;
+        String cfName= CF_SUPER1;
         ByteBuffer scfName = ByteBufferUtil.bytes("SuperDuper");
         Keyspace keyspace = Keyspace.open(keyspaceName);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
@@ -830,7 +951,7 @@
         // delete
         Mutation rm = new Mutation(keyspace.getName(), key.getKey());
         rm.deleteRange(cfName, SuperColumns.startOf(scfName), SuperColumns.endOf(scfName), 2);
-        rm.apply();
+        rm.applyUnsafe();
 
         // verify delete.
         assertRowAndColCount(1, 0, false, cfs.getRangeSlice(Util.range("f", "g"), null, ThriftValidation.asIFilter(sp, cfs.metadata, scfName), 100));
@@ -883,7 +1004,7 @@
         for (Cell col : cols)
             cf.addColumn(col.withUpdatedName(CellNames.compositeDense(scfName, col.name().toByteBuffer())));
         Mutation rm = new Mutation(cfs.keyspace.getName(), key.getKey(), cf);
-        rm.apply();
+        rm.applyUnsafe();
     }
 
     private static void putColsStandard(ColumnFamilyStore cfs, DecoratedKey key, Cell... cols) throws Throwable
@@ -892,15 +1013,15 @@
         for (Cell col : cols)
             cf.addColumn(col);
         Mutation rm = new Mutation(cfs.keyspace.getName(), key.getKey(), cf);
-        rm.apply();
+        rm.applyUnsafe();
     }
 
     @Test
     public void testDeleteStandardRowSticksAfterFlush() throws Throwable
     {
         // test to make sure flushing after a delete doesn't resurrect delted cols.
-        String keyspaceName = "Keyspace1";
-        String cfName = "Standard1";
+        String keyspaceName = KEYSPACE1;
+        String cfName = CF_STANDARD1;
         Keyspace keyspace = Keyspace.open(keyspaceName);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
         DecoratedKey key = Util.dk("f-flush-resurrection");
@@ -925,7 +1046,7 @@
         // delete (from sstable and memtable)
         Mutation rm = new Mutation(keyspace.getName(), key.getKey());
         rm.delete(cfs.name, 2);
-        rm.apply();
+        rm.applyUnsafe();
 
         // verify delete
         assertRowAndColCount(1, 0, true, cfs.getRangeSlice(Util.range("f", "g"), null, ThriftValidation.asIFilter(sp, cfs.metadata, null), 100));
@@ -954,16 +1075,16 @@
 
     private ColumnFamilyStore insertKey1Key2()
     {
-        ColumnFamilyStore cfs = Keyspace.open("Keyspace2").getColumnFamilyStore("Standard1");
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE2).getColumnFamilyStore(CF_STANDARD1);
         List<Mutation> rms = new LinkedList<>();
         Mutation rm;
-        rm = new Mutation("Keyspace2", ByteBufferUtil.bytes("key1"));
-        rm.add("Standard1", cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
+        rm = new Mutation(KEYSPACE2, ByteBufferUtil.bytes("key1"));
+        rm.add(CF_STANDARD1, cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
         rms.add(rm);
         Util.writeColumnFamily(rms);
 
-        rm = new Mutation("Keyspace2", ByteBufferUtil.bytes("key2"));
-        rm.add("Standard1", cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
+        rm = new Mutation(KEYSPACE2, ByteBufferUtil.bytes("key2"));
+        rm.add(CF_STANDARD1, cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
         rms.add(rm);
         return Util.writeColumnFamily(rms);
     }
@@ -975,8 +1096,8 @@
 
         for (int version = 1; version <= 2; ++version)
         {
-            Descriptor existing = new Descriptor(cfs.directories.getDirectoryForNewSSTables(), "Keyspace2", "Standard1", version, Descriptor.Type.FINAL);
-            Descriptor desc = new Descriptor(Directories.getBackupsDirectory(existing), "Keyspace2", "Standard1", version, Descriptor.Type.FINAL);
+            Descriptor existing = new Descriptor(cfs.directories.getDirectoryForNewSSTables(), KEYSPACE2, CF_STANDARD1, version, Descriptor.Type.FINAL);
+            Descriptor desc = new Descriptor(Directories.getBackupsDirectory(existing), KEYSPACE2, CF_STANDARD1, version, Descriptor.Type.FINAL);
             for (Component c : new Component[]{ Component.DATA, Component.PRIMARY_INDEX, Component.FILTER, Component.STATS })
                 assertTrue("can not find backedup file:" + desc.filenameFor(c), new File(desc.filenameFor(c)).exists());
         }
@@ -986,8 +1107,8 @@
     @Test
     public void testSliceByNamesCommandOnUUIDTypeSCF() throws Throwable
     {
-        String keyspaceName = "Keyspace1";
-        String cfName = "Super6";
+        String keyspaceName = KEYSPACE1;
+        String cfName = CF_SUPER6;
         ByteBuffer superColName = LexicalUUIDType.instance.fromString("a4ed3562-0e8e-4b41-bdfd-c45a2774682d");
         Keyspace keyspace = Keyspace.open(keyspaceName);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
@@ -1017,13 +1138,13 @@
     @Test
     public void testSliceByNamesCommandOldMetadata() throws Throwable
     {
-        String keyspaceName = "Keyspace1";
-        String cfName= "Standard1";
+        String keyspaceName = KEYSPACE1;
+        String cfName= CF_STANDARD6;
         DecoratedKey key = Util.dk("slice-name-old-metadata");
         CellName cname = cellname("c1");
         Keyspace keyspace = Keyspace.open(keyspaceName);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
-        cfs.clearUnsafe();
+        cfs.truncateBlocking();
 
         // Create a cell a 'high timestamp'
         putColsStandard(cfs, key, new BufferCell(cname, ByteBufferUtil.bytes("a"), 2));
@@ -1035,7 +1156,11 @@
         cfs.clearUnsafe();
         assertEquals(0, cfs.getSSTables().size());
 
-        new File(ssTables.iterator().next().descriptor.filenameFor(Component.STATS)).delete();
+        SSTableReader sstable = ssTables.iterator().next();
+        File statsFile = new File(sstable.descriptor.filenameFor(Component.STATS));
+        assert statsFile.exists();
+        boolean deleted = statsFile.delete();
+        assert deleted : "Cannot delete " + statsFile;
         cfs.loadNewSSTables();
 
         // Add another cell with a lower timestamp
@@ -1047,7 +1172,7 @@
         Cell cell = cf.getColumn(cname);
         assert cell.value().equals(ByteBufferUtil.bytes("a")) : "expecting a, got " + ByteBufferUtil.string(cell.value());
 
-        Keyspace.clear("Keyspace1"); // CASSANDRA-7195
+        Keyspace.clear(KEYSPACE1); // CASSANDRA-7195
     }
 
     private static void assertTotalColCount(Collection<Row> rows, int expectedCount)
@@ -1064,8 +1189,8 @@
     @Test
     public void testRangeSliceColumnsLimit() throws Throwable
     {
-        String keyspaceName = "Keyspace1";
-        String cfName = "Standard1";
+        String keyspaceName = KEYSPACE1;
+        String cfName = CF_STANDARD1;
         Keyspace keyspace = Keyspace.open(keyspaceName);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
         cfs.clearUnsafe();
@@ -1180,8 +1305,8 @@
     @Test
     public void testRangeSlicePaging() throws Throwable
     {
-        String keyspaceName = "Keyspace1";
-        String cfName = "Standard1";
+        String keyspaceName = KEYSPACE1;
+        String cfName = CF_STANDARD1;
         Keyspace keyspace = Keyspace.open(keyspaceName);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
         cfs.clearUnsafe();
@@ -1316,8 +1441,8 @@
     @Test
     public void testRangeSliceInclusionExclusion() throws Throwable
     {
-        String keyspaceName = "Keyspace1";
-        String cfName = "Standard1";
+        String keyspaceName = KEYSPACE1;
+        String cfName = CF_STANDARD1;
         Keyspace keyspace = Keyspace.open(keyspaceName);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
         cfs.clearUnsafe();
@@ -1370,17 +1495,17 @@
     public void testKeysSearcher() throws Exception
     {
         // Create secondary index and flush to disk
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Indexed1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore(CF_INDEX1);
 
         store.truncateBlocking();
 
         for (int i = 0; i < 10; i++)
         {
             ByteBuffer key = ByteBufferUtil.bytes(String.valueOf("k" + i));
-            Mutation rm = new Mutation("Keyspace1", key);
-            rm.add("Indexed1", cellname("birthdate"), LongType.instance.decompose(1L), System.currentTimeMillis());
-            rm.apply();
+            Mutation rm = new Mutation(KEYSPACE1, key);
+            rm.add(CF_INDEX1, cellname("birthdate"), LongType.instance.decompose(1L), System.currentTimeMillis());
+            rm.applyUnsafe();
         }
 
         store.forceBlockingFlush();
@@ -1412,8 +1537,8 @@
                 new ColumnSlice(cellname("colE"), cellname("colC")),
                 new ColumnSlice(cellname("colA"), Composites.EMPTY) };
 
-        String tableName = "Keyspace1";
-        String cfName = "Standard1";
+        String tableName = KEYSPACE1;
+        String cfName = CF_STANDARD1;
         Keyspace table = Keyspace.open(tableName);
         ColumnFamilyStore cfs = table.getColumnFamilyStore(cfName);
         cfs.clearUnsafe();
@@ -1444,26 +1569,24 @@
     @Test
     public void testClearEphemeralSnapshots() throws Throwable
     {
-        Mutation rm;
-        ColumnFamilyStore cfs = Keyspace.open("Keyspace3").getColumnFamilyStore("Indexed1");
-        for (int i = 0; i < 100; i++)
-        {
-            rm = new Mutation("Keyspace3", ByteBufferUtil.bytes("key" + i));
-            rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(34L), 0);
-            rm.add("Indexed1", cellname("notbirthdate"), ByteBufferUtil.bytes((long) (i % 2)), 0);
-            rm.applyUnsafe();
-        }
+        // We don't do snapshot-based repair on Windows so we don't have ephemeral snapshots from repair that need clearing.
+        // This test will fail as we'll revert to the WindowsFailedSnapshotTracker and counts will be off, but since we
+        // don't do snapshot-based repair on Windows, we just skip this test.
+        Assume.assumeTrue(!FBUtilities.isWindows());
+
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_INDEX1);
 
         //cleanup any previous test gargbage
         cfs.clearSnapshot("");
 
-        Cell[] cols = new Cell[5];
-        for (int i = 0; i < 5; i++)
-            cols[i] = column("c" + i, "value", 1);
-
-        putColsStandard(cfs, Util.dk("a"), cols[0], cols[1], cols[2], cols[3], cols[4]);
-        putColsStandard(cfs, Util.dk("b"), cols[0], cols[1]);
-        putColsStandard(cfs, Util.dk("c"), cols[0], cols[1], cols[2], cols[3]);
+        Mutation rm;
+        for (int i = 0; i < 100; i++)
+        {
+            rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("key" + i));
+            rm.add(CF_INDEX1, cellname("birthdate"), ByteBufferUtil.bytes(34L), 0);
+            rm.add(CF_INDEX1, cellname("notbirthdate"), ByteBufferUtil.bytes((long) (i % 2)), 0);
+            rm.applyUnsafe();
+        }
 
         cfs.snapshot("nonEphemeralSnapshot", null, false);
         cfs.snapshot("ephemeralSnapshot", null, true);
@@ -1503,8 +1626,8 @@
                 new ColumnSlice(cellname("colE"), cellname("colC")),
                 new ColumnSlice(cellname("colA"), Composites.EMPTY) };
 
-        String tableName = "Keyspace1";
-        String cfName = "Standard1";
+        String tableName = KEYSPACE1;
+        String cfName = CF_STANDARD1;
         Keyspace table = Keyspace.open(tableName);
         ColumnFamilyStore cfs = table.getColumnFamilyStore(cfName);
         cfs.clearUnsafe();
@@ -1552,8 +1675,8 @@
                 new ColumnSlice(cellname("colE"), cellname("colC")),
                 new ColumnSlice(cellname("colA"), Composites.EMPTY) };
 
-        String tableName = "Keyspace1";
-        String cfName = "Standard1";
+        String tableName = KEYSPACE1;
+        String cfName = CF_STANDARD1;
         Keyspace table = Keyspace.open(tableName);
         ColumnFamilyStore cfs = table.getColumnFamilyStore(cfName);
         cfs.clearUnsafe();
@@ -1602,8 +1725,8 @@
                 new ColumnSlice(cellname("colE"), cellname("colC")),
                 new ColumnSlice(cellname("colA"), Composites.EMPTY) };
 
-        String tableName = "Keyspace1";
-        String cfName = "Standard1";
+        String tableName = KEYSPACE1;
+        String cfName = CF_STANDARD1;
         Keyspace table = Keyspace.open(tableName);
         ColumnFamilyStore cfs = table.getColumnFamilyStore(cfName);
         cfs.clearUnsafe();
@@ -1650,8 +1773,8 @@
                 new ColumnSlice(cellname("colE"), cellname("colC")),
                 new ColumnSlice(cellname("colA"), Composites.EMPTY) };
 
-        String keyspaceName = "Keyspace1";
-        String cfName = "Standard1";
+        String keyspaceName = KEYSPACE1;
+        String cfName = CF_STANDARD1;
         Keyspace keyspace = Keyspace.open(keyspaceName);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
         cfs.clearUnsafe();
@@ -1717,8 +1840,8 @@
     @Test
     public void testRemoveUnfinishedCompactionLeftovers() throws Throwable
     {
-        String ks = "Keyspace1";
-        String cf = "Standard3"; // should be empty
+        String ks = KEYSPACE1;
+        String cf = CF_STANDARD3; // should be empty
 
         final CFMetaData cfmeta = Schema.instance.getCFMetaData(ks, cf);
         Directories dir = new Directories(cfmeta);
@@ -1744,12 +1867,12 @@
             {
                 MetadataCollector collector = new MetadataCollector(cfmeta.comparator);
                 collector.addAncestor(sstable1.descriptor.generation); // add ancestor from previously written sstable
-                return new SSTableWriter(makeFilename(directory, metadata.ksName, metadata.cfName),
-                                         0,
-                                         ActiveRepairService.UNREPAIRED_SSTABLE,
-                                         metadata,
-                                         StorageService.getPartitioner(),
-                                         collector);
+                return SSTableWriter.create(createDescriptor(directory, metadata.ksName, metadata.cfName, DatabaseDescriptor.getSSTableFormat()),
+                        0L,
+                        ActiveRepairService.UNREPAIRED_SSTABLE,
+                        metadata,
+                        DatabaseDescriptor.getPartitioner(),
+                        collector);
             }
         };
         writer.newRow(key);
@@ -1786,8 +1909,8 @@
     @Test
     public void testFailedToRemoveUnfinishedCompactionLeftovers() throws Throwable
     {
-        final String ks = "Keyspace1";
-        final String cf = "Standard4"; // should be empty
+        final String ks = KEYSPACE1;
+        final String cf = CF_STANDARD4; // should be empty
 
         final CFMetaData cfmeta = Schema.instance.getCFMetaData(ks, cf);
         Directories dir = new Directories(cfmeta);
@@ -1804,12 +1927,12 @@
                 for (int ancestor : ancestors)
                     collector.addAncestor(ancestor);
                 String file = new Descriptor(directory, ks, cf, 3, Descriptor.Type.TEMP).filenameFor(Component.DATA);
-                return new SSTableWriter(file,
-                                         0,
-                                         ActiveRepairService.UNREPAIRED_SSTABLE,
-                                         metadata,
-                                         StorageService.getPartitioner(),
-                                         collector);
+                return SSTableWriter.create(Descriptor.fromFilename(file),
+                        0L,
+                        ActiveRepairService.UNREPAIRED_SSTABLE,
+                        metadata,
+                        StorageService.getPartitioner(),
+                        collector);
             }
         };
         writer.newRow(key);
@@ -1838,8 +1961,8 @@
     @Test
     public void testLoadNewSSTablesAvoidsOverwrites() throws Throwable
     {
-        String ks = "Keyspace1";
-        String cf = "Standard1";
+        String ks = KEYSPACE1;
+        String cf = CF_STANDARD5;
         ColumnFamilyStore cfs = Keyspace.open(ks).getColumnFamilyStore(cf);
         cfs.truncateBlocking();
         SSTableDeletingTask.waitForDeletions();
@@ -1923,8 +2046,8 @@
 
     private ColumnFamilyStore prepareMultiRangeSlicesTest(int valueSize, boolean flush) throws Throwable
     {
-        String keyspaceName = "Keyspace1";
-        String cfName = "Standard1";
+        String keyspaceName = KEYSPACE1;
+        String cfName = CF_STANDARD1;
         Keyspace keyspace = Keyspace.open(keyspaceName);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
         cfs.clearUnsafe();
@@ -2187,7 +2310,7 @@
     {
         DecoratedKey ROW = Util.dk(rowKey);
         System.err.println("Original:");
-        ColumnFamily cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(ROW, "Standard1", System.currentTimeMillis()));
+        ColumnFamily cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(ROW, CF_STANDARD1, System.currentTimeMillis()));
         System.err.println("Row key: " + rowKey + " Cols: "
                 + Iterables.transform(cf.getSortedColumns(), new Function<Cell, String>()
                 {
@@ -2206,31 +2329,31 @@
         });
         System.err.println("Row key: " + rowKey + " Cols: " + transformed);
     }
-    
+
     @Test
     public void testRebuildSecondaryIndex() throws IOException
     {
         CellName indexedCellName = cellname("indexed");
         Mutation rm;
 
-        rm = new Mutation("PerRowSecondaryIndex", ByteBufferUtil.bytes("k1"));
+        rm = new Mutation(KEYSPACE4, ByteBufferUtil.bytes("k1"));
         rm.add("Indexed1", indexedCellName, ByteBufferUtil.bytes("foo"), 1);
 
         rm.apply();
         assertTrue(Arrays.equals("k1".getBytes(), PerRowSecondaryIndexTest.TestIndex.LAST_INDEXED_KEY.array()));
-        
+
         Keyspace.open("PerRowSecondaryIndex").getColumnFamilyStore("Indexed1").forceBlockingFlush();
-        
+
         PerRowSecondaryIndexTest.TestIndex.reset();
-        
+
         ColumnFamilyStore.rebuildSecondaryIndex("PerRowSecondaryIndex", "Indexed1", PerRowSecondaryIndexTest.TestIndex.class.getSimpleName());
         assertTrue(Arrays.equals("k1".getBytes(), PerRowSecondaryIndexTest.TestIndex.LAST_INDEXED_KEY.array()));
-        
+
         PerRowSecondaryIndexTest.TestIndex.reset();
         PerRowSecondaryIndexTest.TestIndex.ACTIVE = false;
         ColumnFamilyStore.rebuildSecondaryIndex("PerRowSecondaryIndex", "Indexed1", PerRowSecondaryIndexTest.TestIndex.class.getSimpleName());
         assertNull(PerRowSecondaryIndexTest.TestIndex.LAST_INDEXED_KEY);
-        
+
         PerRowSecondaryIndexTest.TestIndex.reset();
     }
 }
diff --git a/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java b/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java
index 69a851e..72ddd40 100644
--- a/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java
+++ b/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java
@@ -25,13 +25,20 @@
 import java.util.TreeMap;
 
 import com.google.common.collect.Iterables;
+import org.apache.cassandra.config.CFMetaData;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.context.CounterContext;
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.CounterColumnType;
 import org.apache.cassandra.io.sstable.ColumnStats;
 import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.CounterId;
@@ -45,9 +52,24 @@
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 
-public class ColumnFamilyTest extends SchemaLoader
+public class ColumnFamilyTest
 {
     static int version = MessagingService.current_version;
+    private static final String KEYSPACE1 = "Keyspace1";
+    private static final String CF_STANDARD1 = "Standard1";
+    private static final String CF_COUNTER1 = "Counter1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_COUNTER1)
+                                                .defaultValidator(CounterColumnType.instance));
+    }
 
     // TODO test SuperColumns more
 
@@ -56,7 +78,7 @@
     {
         ColumnFamily cf;
 
-        cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, CF_STANDARD1);
         cf.addColumn(column("C", "v", 1));
         DataOutputBuffer bufOut = new DataOutputBuffer();
         ColumnFamily.serializer.serialize(cf, bufOut, version);
@@ -64,7 +86,7 @@
         ByteArrayInputStream bufIn = new ByteArrayInputStream(bufOut.getData(), 0, bufOut.getLength());
         cf = ColumnFamily.serializer.deserialize(new DataInputStream(bufIn), version);
         assert cf != null;
-        assert cf.metadata().cfName.equals("Standard1");
+        assert cf.metadata().cfName.equals(CF_STANDARD1);
         assert cf.getSortedColumns().size() == 1;
     }
 
@@ -80,7 +102,7 @@
         }
 
         // write
-        cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, CF_STANDARD1);
         DataOutputBuffer bufOut = new DataOutputBuffer();
         for (String cName : map.navigableKeySet())
         {
@@ -102,7 +124,7 @@
     @Test
     public void testGetColumnCount()
     {
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, CF_STANDARD1);
 
         cf.addColumn(column("col1", "", 1));
         cf.addColumn(column("col2", "", 2));
@@ -115,8 +137,8 @@
     @Test
     public void testDigest()
     {
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
-        ColumnFamily cf2 = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, CF_STANDARD1);
+        ColumnFamily cf2 = ArrayBackedSortedColumns.factory.create(KEYSPACE1, CF_STANDARD1);
 
         ByteBuffer digest = ColumnFamily.digest(cf);
 
@@ -157,7 +179,7 @@
     @Test
     public void testTimestamp()
     {
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, CF_STANDARD1);
 
         cf.addColumn(column("col1", "val1", 2));
         cf.addColumn(column("col1", "val2", 2)); // same timestamp, new value
@@ -169,9 +191,9 @@
     @Test
     public void testMergeAndAdd()
     {
-        ColumnFamily cf_new = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
-        ColumnFamily cf_old = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
-        ColumnFamily cf_result = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf_new = ArrayBackedSortedColumns.factory.create(KEYSPACE1, CF_STANDARD1);
+        ColumnFamily cf_old = ArrayBackedSortedColumns.factory.create(KEYSPACE1, CF_STANDARD1);
+        ColumnFamily cf_result = ArrayBackedSortedColumns.factory.create(KEYSPACE1, CF_STANDARD1);
         ByteBuffer val = ByteBufferUtil.bytes("sample value");
         ByteBuffer val2 = ByteBufferUtil.bytes("x value ");
 
@@ -207,7 +229,7 @@
         long timestamp = System.currentTimeMillis();
         int localDeletionTime = (int) (System.currentTimeMillis() / 1000);
 
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, CF_STANDARD1);
         cf.delete(new DeletionInfo(timestamp, localDeletionTime));
         ColumnStats stats = cf.getColumnStats();
         assertEquals(timestamp, stats.maxTimestamp);
@@ -240,14 +262,14 @@
         assertTrue(counter.reconcile(tombstone) == tombstone);
 
         // check that a range tombstone overrides the counter cell, even with a lower timestamp than the counter
-        ColumnFamily cf0 = ArrayBackedSortedColumns.factory.create("Keyspace1", "Counter1");
+        ColumnFamily cf0 = ArrayBackedSortedColumns.factory.create(KEYSPACE1, CF_COUNTER1);
         cf0.addColumn(counter);
         cf0.delete(new RangeTombstone(cellname("counter0"), cellname("counter2"), 0L, (int) (System.currentTimeMillis() / 1000)));
         assertTrue(cf0.deletionInfo().isDeleted(counter));
         assertTrue(cf0.deletionInfo().inOrderTester(false).isDeleted(counter));
 
         // check that a top-level deletion info overrides the counter cell, even with a lower timestamp than the counter
-        ColumnFamily cf1 = ArrayBackedSortedColumns.factory.create("Keyspace1", "Counter1");
+        ColumnFamily cf1 = ArrayBackedSortedColumns.factory.create(KEYSPACE1, CF_COUNTER1);
         cf1.addColumn(counter);
         cf1.delete(new DeletionInfo(0L, (int) (System.currentTimeMillis() / 1000)));
         assertTrue(cf1.deletionInfo().isDeleted(counter));
diff --git a/test/unit/org/apache/cassandra/db/CommitLogTest.java b/test/unit/org/apache/cassandra/db/CommitLogTest.java
deleted file mode 100644
index 1c3daab..0000000
--- a/test/unit/org/apache/cassandra/db/CommitLogTest.java
+++ /dev/null
@@ -1,484 +0,0 @@
-/*
-* 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.db;
-
-import java.io.*;
-import java.nio.ByteBuffer;
-import java.util.UUID;
-import java.util.concurrent.ExecutionException;
-import java.util.zip.CRC32;
-import java.util.zip.Checksum;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.Util;
-import org.apache.cassandra.config.Config;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.commitlog.CommitLog;
-import org.apache.cassandra.db.commitlog.CommitLogDescriptor;
-import org.apache.cassandra.db.commitlog.CommitLogSegmentManager;
-import org.apache.cassandra.db.commitlog.ReplayPosition;
-import org.apache.cassandra.db.commitlog.CommitLogSegment;
-import org.apache.cassandra.db.composites.CellName;
-import org.apache.cassandra.db.composites.CellNameType;
-import org.apache.cassandra.db.filter.NamesQueryFilter;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.gms.Gossiper;
-import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.service.CassandraDaemon;
-import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.JVMStabilityInspector;
-import org.apache.cassandra.utils.KillerForTests;
-
-import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
-
-public class CommitLogTest extends SchemaLoader
-{
-    @Test
-    public void testRecoveryWithEmptyLog() throws Exception
-    {
-        CommitLog.instance.recover(new File[]{ tmpFile() });
-    }
-
-    @Test
-    public void testRecoveryWithShortLog() throws Exception
-    {
-        // force EOF while reading log
-        testRecoveryWithBadSizeArgument(100, 10);
-    }
-
-    @Test
-    public void testRecoveryWithShortSize() throws Exception
-    {
-        testRecovery(new byte[2]);
-    }
-
-    @Test
-    public void testRecoveryWithShortCheckSum() throws Exception
-    {
-        testRecovery(new byte[6]);
-    }
-
-    @Test
-    public void testRecoveryWithGarbageLog() throws Exception
-    {
-        byte[] garbage = new byte[100];
-        (new java.util.Random()).nextBytes(garbage);
-        testRecovery(garbage);
-    }
-
-    @Test
-    public void testRecoveryWithBadSizeChecksum() throws Exception
-    {
-        Checksum checksum = new CRC32();
-        checksum.update(100);
-        testRecoveryWithBadSizeArgument(100, 100, ~checksum.getValue());
-    }
-
-    @Test
-    public void testRecoveryWithZeroSegmentSizeArgument() throws Exception
-    {
-        // many different combinations of 4 bytes (garbage) will be read as zero by readInt()
-        testRecoveryWithBadSizeArgument(0, 10); // zero size, but no EOF
-    }
-
-    @Test
-    public void testRecoveryWithNegativeSizeArgument() throws Exception
-    {
-        // garbage from a partial/bad flush could be read as a negative size even if there is no EOF
-        testRecoveryWithBadSizeArgument(-10, 10); // negative size, but no EOF
-    }
-
-    @Test
-    public void testDontDeleteIfDirty() throws Exception
-    {
-        CommitLog.instance.resetUnsafe();
-        // Roughly 32 MB mutation
-        Mutation rm = new Mutation("Keyspace1", bytes("k"));
-        rm.add("Standard1", Util.cellname("c1"), ByteBuffer.allocate(DatabaseDescriptor.getCommitLogSegmentSize()/4), 0);
-
-        // Adding it 5 times
-        CommitLog.instance.add(rm);
-        CommitLog.instance.add(rm);
-        CommitLog.instance.add(rm);
-        CommitLog.instance.add(rm);
-        CommitLog.instance.add(rm);
-
-        // Adding new mutation on another CF
-        Mutation rm2 = new Mutation("Keyspace1", bytes("k"));
-        rm2.add("Standard2", Util.cellname("c1"), ByteBuffer.allocate(4), 0);
-        CommitLog.instance.add(rm2);
-
-        assert CommitLog.instance.activeSegments() == 2 : "Expecting 2 segments, got " + CommitLog.instance.activeSegments();
-
-        UUID cfid2 = rm2.getColumnFamilyIds().iterator().next();
-        CommitLog.instance.discardCompletedSegments(cfid2, CommitLog.instance.getContext());
-
-        // Assert we still have both our segment
-        assert CommitLog.instance.activeSegments() == 2 : "Expecting 2 segments, got " + CommitLog.instance.activeSegments();
-    }
-
-    @Test
-    public void testDeleteIfNotDirty() throws Exception
-    {
-        DatabaseDescriptor.getCommitLogSegmentSize();
-        CommitLog.instance.resetUnsafe();
-        // Roughly 32 MB mutation
-        Mutation rm = new Mutation("Keyspace1", bytes("k"));
-        rm.add("Standard1", Util.cellname("c1"), ByteBuffer.allocate((DatabaseDescriptor.getCommitLogSegmentSize()/4) - 1), 0);
-
-        // Adding it twice (won't change segment)
-        CommitLog.instance.add(rm);
-        CommitLog.instance.add(rm);
-
-        assert CommitLog.instance.activeSegments() == 1 : "Expecting 1 segment, got " + CommitLog.instance.activeSegments();
-
-        // "Flush": this won't delete anything
-        UUID cfid1 = rm.getColumnFamilyIds().iterator().next();
-        CommitLog.instance.sync(true);
-        CommitLog.instance.discardCompletedSegments(cfid1, CommitLog.instance.getContext());
-
-        assert CommitLog.instance.activeSegments() == 1 : "Expecting 1 segment, got " + CommitLog.instance.activeSegments();
-
-        // Adding new mutation on another CF, large enough (including CL entry overhead) that a new segment is created
-        Mutation rm2 = new Mutation("Keyspace1", bytes("k"));
-        rm2.add("Standard2", Util.cellname("c1"), ByteBuffer.allocate((DatabaseDescriptor.getCommitLogSegmentSize()/2) - 100), 0);
-        CommitLog.instance.add(rm2);
-        // also forces a new segment, since each entry-with-overhead is just under half the CL size
-        CommitLog.instance.add(rm2);
-        CommitLog.instance.add(rm2);
-
-        assert CommitLog.instance.activeSegments() == 3 : "Expecting 3 segments, got " + CommitLog.instance.activeSegments();
-
-
-        // "Flush" second cf: The first segment should be deleted since we
-        // didn't write anything on cf1 since last flush (and we flush cf2)
-
-        UUID cfid2 = rm2.getColumnFamilyIds().iterator().next();
-        CommitLog.instance.discardCompletedSegments(cfid2, CommitLog.instance.getContext());
-
-        // Assert we still have both our segment
-        assert CommitLog.instance.activeSegments() == 1 : "Expecting 1 segment, got " + CommitLog.instance.activeSegments();
-    }
-
-    private static int getMaxRecordDataSize(String keyspace, ByteBuffer key, String table, CellName column)
-    {
-        Mutation rm = new Mutation("Keyspace1", bytes("k"));
-        rm.add("Standard1", Util.cellname("c1"), ByteBuffer.allocate(0), 0);
-
-        int max = (DatabaseDescriptor.getCommitLogSegmentSize() / 2);
-        max -= CommitLogSegment.ENTRY_OVERHEAD_SIZE; // log entry overhead
-        return max - (int) Mutation.serializer.serializedSize(rm, MessagingService.current_version);
-    }
-
-    private static int getMaxRecordDataSize()
-    {
-        return getMaxRecordDataSize("Keyspace1", bytes("k"), "Standard1", Util.cellname("c1"));
-    }
-
-    // CASSANDRA-3615
-    @Test
-    public void testEqualRecordLimit() throws Exception
-    {
-        CommitLog.instance.resetUnsafe();
-
-        Mutation rm = new Mutation("Keyspace1", bytes("k"));
-        rm.add("Standard1", Util.cellname("c1"), ByteBuffer.allocate(getMaxRecordDataSize()), 0);
-        CommitLog.instance.add(rm);
-    }
-
-    @Test
-    public void testExceedRecordLimit() throws Exception
-    {
-        CommitLog.instance.resetUnsafe();
-        try
-        {
-            Mutation rm = new Mutation("Keyspace1", bytes("k"));
-            rm.add("Standard1", Util.cellname("c1"), ByteBuffer.allocate(1 + getMaxRecordDataSize()), 0);
-            CommitLog.instance.add(rm);
-            throw new AssertionError("mutation larger than limit was accepted");
-        }
-        catch (IllegalArgumentException e)
-        {
-            // IAE is thrown on too-large mutations
-        }
-    }
-
-    protected void testRecoveryWithBadSizeArgument(int size, int dataSize) throws Exception
-    {
-        Checksum checksum = new CRC32();
-        checksum.update(size);
-        testRecoveryWithBadSizeArgument(size, dataSize, checksum.getValue());
-    }
-
-    protected void testRecoveryWithBadSizeArgument(int size, int dataSize, long checksum) throws Exception
-    {
-        ByteArrayOutputStream out = new ByteArrayOutputStream();
-        DataOutputStream dout = new DataOutputStream(out);
-        dout.writeInt(size);
-        dout.writeLong(checksum);
-        dout.write(new byte[dataSize]);
-        dout.close();
-        testRecovery(out.toByteArray());
-    }
-
-    protected File tmpFile() throws IOException
-    {
-        File logFile = File.createTempFile("CommitLog-" + CommitLogDescriptor.current_version + "-", ".log");
-        logFile.deleteOnExit();
-        assert logFile.length() == 0;
-        return logFile;
-    }
-
-    protected void testRecovery(byte[] logData) throws Exception
-    {
-        File logFile = tmpFile();
-        try (OutputStream lout = new FileOutputStream(logFile))
-        {
-            lout.write(logData);
-            //statics make it annoying to test things correctly
-            CommitLog.instance.recover(new File[]{ logFile }); //CASSANDRA-1119 / CASSANDRA-1179 throw on failure*/
-        }
-    }
-
-    @Test
-    public void testVersions()
-    {
-        Assert.assertTrue(CommitLogDescriptor.isValid("CommitLog-1340512736956320000.log"));
-        Assert.assertTrue(CommitLogDescriptor.isValid("CommitLog-2-1340512736956320000.log"));
-        Assert.assertFalse(CommitLogDescriptor.isValid("CommitLog--1340512736956320000.log"));
-        Assert.assertFalse(CommitLogDescriptor.isValid("CommitLog--2-1340512736956320000.log"));
-        Assert.assertFalse(CommitLogDescriptor.isValid("CommitLog-2-1340512736956320000-123.log"));
-
-        Assert.assertEquals(1340512736956320000L, CommitLogDescriptor.fromFileName("CommitLog-2-1340512736956320000.log").id);
-
-        Assert.assertEquals(MessagingService.current_version, new CommitLogDescriptor(1340512736956320000L).getMessagingVersion());
-        String newCLName = "CommitLog-" + CommitLogDescriptor.current_version + "-1340512736956320000.log";
-        Assert.assertEquals(MessagingService.current_version, CommitLogDescriptor.fromFileName(newCLName).getMessagingVersion());
-    }
-
-    @Test
-    public void testCommitFailurePolicy_stop() throws ConfigurationException
-    {
-        CassandraDaemon daemon = new CassandraDaemon();
-        daemon.completeSetup(); //startup must be completed, otherwise commit log failure must kill JVM regardless of failure policy
-        StorageService.instance.registerDaemon(daemon);
-
-        // Need storage service active so stop policy can shutdown gossip
-        StorageService.instance.initServer();
-        Assert.assertTrue(Gossiper.instance.isEnabled());
-
-        Config.CommitFailurePolicy oldPolicy = DatabaseDescriptor.getCommitFailurePolicy();
-        try
-        {
-            DatabaseDescriptor.setCommitFailurePolicy(Config.CommitFailurePolicy.stop);
-            CommitLog.handleCommitError("Test stop error", new Throwable());
-            Assert.assertFalse(Gossiper.instance.isEnabled());
-        }
-        finally
-        {
-            DatabaseDescriptor.setCommitFailurePolicy(oldPolicy);
-        }
-    }
-
-    @Test
-    public void testCommitFailurePolicy_die()
-    {
-        CassandraDaemon daemon = new CassandraDaemon();
-        daemon.completeSetup(); //startup must be completed, otherwise commit log failure must kill JVM regardless of failure policy
-        StorageService.instance.registerDaemon(daemon);
-
-        KillerForTests killerForTests = new KillerForTests();
-        JVMStabilityInspector.Killer originalKiller = JVMStabilityInspector.replaceKiller(killerForTests);
-        Config.CommitFailurePolicy oldPolicy = DatabaseDescriptor.getCommitFailurePolicy();
-        try
-        {
-            DatabaseDescriptor.setCommitFailurePolicy(Config.CommitFailurePolicy.die);
-            CommitLog.handleCommitError("Testing die policy", new Throwable());
-            Assert.assertTrue(killerForTests.wasKilled());
-            Assert.assertFalse(killerForTests.wasKilledQuietly()); //only killed quietly on startup failure
-        }
-        finally
-        {
-            DatabaseDescriptor.setCommitFailurePolicy(oldPolicy);
-            JVMStabilityInspector.replaceKiller(originalKiller);
-        }
-    }
-
-    @Test
-    public void testCommitFailurePolicy_mustDieIfNotStartedUp()
-    {
-        //startup was not completed successfuly (since method completeSetup() was not called)
-        CassandraDaemon daemon = new CassandraDaemon();
-        StorageService.instance.registerDaemon(daemon);
-
-        KillerForTests killerForTests = new KillerForTests();
-        JVMStabilityInspector.Killer originalKiller = JVMStabilityInspector.replaceKiller(killerForTests);
-        Config.CommitFailurePolicy oldPolicy = DatabaseDescriptor.getCommitFailurePolicy();
-        try
-        {
-            //even though policy is ignore, JVM must die because Daemon has not finished initializing
-            DatabaseDescriptor.setCommitFailurePolicy(Config.CommitFailurePolicy.ignore);
-            CommitLog.handleCommitError("Testing die policy", new Throwable());
-            Assert.assertTrue(killerForTests.wasKilled());
-            Assert.assertTrue(killerForTests.wasKilledQuietly()); //killed quietly due to startup failure
-        }
-        finally
-        {
-            DatabaseDescriptor.setCommitFailurePolicy(oldPolicy);
-            JVMStabilityInspector.replaceKiller(originalKiller);
-        }
-    }
-
-    @Test
-    public void testCommitLogFailureBeforeInitialization_mustKillJVM() throws Exception
-    {
-        //startup was not completed successfuly (since method completeSetup() was not called)
-        CassandraDaemon daemon = new CassandraDaemon();
-        StorageService.instance.registerDaemon(daemon);
-
-        //let's make the commit log directory non-writable
-        File commitLogDir = new File(DatabaseDescriptor.getCommitLogLocation());
-        commitLogDir.setWritable(false);
-
-        KillerForTests killerForTests = new KillerForTests();
-        JVMStabilityInspector.Killer originalKiller = JVMStabilityInspector.replaceKiller(killerForTests);
-        Config.CommitFailurePolicy oldPolicy = DatabaseDescriptor.getCommitFailurePolicy();
-        try
-        {
-            DatabaseDescriptor.setCommitFailurePolicy(Config.CommitFailurePolicy.ignore);
-
-            //now let's create a commit log segment manager and wait for it to fail
-            new CommitLogSegmentManager();
-
-            //busy wait since commitlogsegmentmanager spawns another thread
-            int retries = 0;
-            while (!killerForTests.wasKilled() && retries++ < 5)
-                Thread.sleep(10);
-
-            //since failure was before CassandraDaemon startup, the JVM must be killed
-            Assert.assertTrue(killerForTests.wasKilled());
-            Assert.assertTrue(killerForTests.wasKilledQuietly()); //killed quietly due to startup failure
-        }
-        finally
-        {
-            DatabaseDescriptor.setCommitFailurePolicy(oldPolicy);
-            JVMStabilityInspector.replaceKiller(originalKiller);
-            commitLogDir.setWritable(true);
-        }
-    }
-
-    @Test
-    public void testCommitLogFailureAfterInitialization_mustRespectFailurePolicy() throws Exception
-    {
-        //startup was not completed successfuly (since method completeSetup() was not called)
-        CassandraDaemon daemon = new CassandraDaemon();
-        daemon.completeSetup(); //startup must be completed, otherwise commit log failure must kill JVM regardless of failure policy
-        StorageService.instance.registerDaemon(daemon);
-
-        //let's make the commit log directory non-writable
-        File commitLogDir = new File(DatabaseDescriptor.getCommitLogLocation());
-        commitLogDir.setWritable(false);
-
-        KillerForTests killerForTests = new KillerForTests();
-        JVMStabilityInspector.Killer originalKiller = JVMStabilityInspector.replaceKiller(killerForTests);
-        Config.CommitFailurePolicy oldPolicy = DatabaseDescriptor.getCommitFailurePolicy();
-        try
-        {
-            DatabaseDescriptor.setCommitFailurePolicy(Config.CommitFailurePolicy.ignore);
-
-            //now let's create a commit log segment manager and wait for it to fail
-            new CommitLogSegmentManager();
-
-            //wait commit log segment manager thread to execute
-            Thread.sleep(50);
-
-            //error policy is set to IGNORE, so JVM must not be killed if error ocurs after startup
-            Assert.assertFalse(killerForTests.wasKilled());
-        }
-        finally
-        {
-            DatabaseDescriptor.setCommitFailurePolicy(oldPolicy);
-            JVMStabilityInspector.replaceKiller(originalKiller);
-            commitLogDir.setWritable(true);
-        }
-    }
-
-    @Test
-    public void testTruncateWithoutSnapshot()  throws ExecutionException, InterruptedException
-    {
-        CommitLog.instance.resetUnsafe();
-        boolean prev = DatabaseDescriptor.isAutoSnapshot();
-        DatabaseDescriptor.setAutoSnapshot(false);
-        ColumnFamilyStore cfs1 = Keyspace.open("Keyspace1").getColumnFamilyStore("Standard1");
-        ColumnFamilyStore cfs2 = Keyspace.open("Keyspace1").getColumnFamilyStore("Standard2");
-
-        final Mutation rm1 = new Mutation("Keyspace1", bytes("k"));
-        rm1.add("Standard1", Util.cellname("c1"), ByteBuffer.allocate(100), 0);
-        rm1.apply();
-        cfs1.truncateBlocking();
-        DatabaseDescriptor.setAutoSnapshot(prev);
-        final Mutation rm2 = new Mutation("Keyspace1", bytes("k"));
-        rm2.add("Standard2", Util.cellname("c1"), ByteBuffer.allocate(DatabaseDescriptor.getCommitLogSegmentSize() / 4), 0);
-
-        for (int i = 0 ; i < 5 ; i++)
-            CommitLog.instance.add(rm2);
-
-        Assert.assertEquals(2, CommitLog.instance.activeSegments());
-        ReplayPosition position = CommitLog.instance.getContext();
-        for (Keyspace ks : Keyspace.system())
-            for (ColumnFamilyStore syscfs : ks.getColumnFamilyStores())
-                CommitLog.instance.discardCompletedSegments(syscfs.metadata.cfId, position);
-        CommitLog.instance.discardCompletedSegments(cfs2.metadata.cfId, position);
-        Assert.assertEquals(1, CommitLog.instance.activeSegments());
-    }
-
-    @Test
-    public void testTruncateWithoutSnapshotNonDurable()  throws ExecutionException, InterruptedException
-    {
-        CommitLog.instance.resetUnsafe();
-        boolean prevAutoSnapshot = DatabaseDescriptor.isAutoSnapshot();
-        DatabaseDescriptor.setAutoSnapshot(false);
-        Keyspace notDurableKs = Keyspace.open("NoCommitlogSpace");
-        Assert.assertFalse(notDurableKs.getMetadata().durableWrites);
-        ColumnFamilyStore cfs = notDurableKs.getColumnFamilyStore("Standard1");
-        CellNameType type = notDurableKs.getColumnFamilyStore("Standard1").getComparator();
-        Mutation rm;
-        DecoratedKey dk = Util.dk("key1");
-
-        // add data
-        rm = new Mutation("NoCommitlogSpace", dk.getKey());
-        rm.add("Standard1", Util.cellname("Column1"), ByteBufferUtil.bytes("abcd"), 0);
-        rm.apply();
-
-        ReadCommand command = new SliceByNamesReadCommand("NoCommitlogSpace", dk.getKey(), "Standard1", System.currentTimeMillis(), new NamesQueryFilter(FBUtilities.singleton(Util.cellname("Column1"), type)));
-        Row row = command.getRow(notDurableKs);
-        Cell col = row.cf.getColumn(Util.cellname("Column1"));
-        Assert.assertEquals(col.value(), ByteBuffer.wrap("abcd".getBytes()));
-        cfs.truncateBlocking();
-        DatabaseDescriptor.setAutoSnapshot(prevAutoSnapshot);
-        row = command.getRow(notDurableKs);
-        Assert.assertEquals(null, row.cf);
-    }
-}
diff --git a/test/unit/org/apache/cassandra/db/CounterCacheTest.java b/test/unit/org/apache/cassandra/db/CounterCacheTest.java
index 542358d..ed7921e 100644
--- a/test/unit/org/apache/cassandra/db/CounterCacheTest.java
+++ b/test/unit/org/apache/cassandra/db/CounterCacheTest.java
@@ -21,13 +21,18 @@
 import java.util.concurrent.ExecutionException;
 
 import org.junit.AfterClass;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.db.marshal.CounterColumnType;
 import org.apache.cassandra.dht.Bounds;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.exceptions.WriteTimeoutException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.service.CacheService;
 import org.apache.cassandra.utils.FBUtilities;
 
@@ -37,21 +42,31 @@
 import static org.apache.cassandra.Util.cellname;
 import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
 
-public class CounterCacheTest extends SchemaLoader
+public class CounterCacheTest
 {
-    private static final String KS = "CounterCacheSpace";
+    private static final String KEYSPACE1 = "CounterCacheTest";
     private static final String CF = "Counter1";
 
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF).defaultValidator(CounterColumnType.instance));
+    }
+
     @AfterClass
     public static void cleanup()
     {
-        cleanupSavedCaches();
+        SchemaLoader.cleanupSavedCaches();
     }
 
     @Test
     public void testReadWrite()
     {
-        ColumnFamilyStore cfs = Keyspace.open(KS).getColumnFamilyStore(CF);
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF);
         cfs.truncateBlocking();
         CacheService.instance.invalidateCounterCache();
 
@@ -76,7 +91,7 @@
     @Test
     public void testCounterCacheInvalidate()
     {
-        ColumnFamilyStore cfs = Keyspace.open(KS).getColumnFamilyStore(CF);
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF);
         cfs.truncateBlocking();
         CacheService.instance.invalidateCounterCache();
 
@@ -118,15 +133,15 @@
     @Test
     public void testSaveLoad() throws ExecutionException, InterruptedException, WriteTimeoutException
     {
-        ColumnFamilyStore cfs = Keyspace.open(KS).getColumnFamilyStore(CF);
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF);
         cfs.truncateBlocking();
         CacheService.instance.invalidateCounterCache();
 
         ColumnFamily cells = ArrayBackedSortedColumns.factory.create(cfs.metadata);
         cells.addColumn(new BufferCounterUpdateCell(cellname(1), 1L, FBUtilities.timestampMicros()));
         cells.addColumn(new BufferCounterUpdateCell(cellname(2), 2L, FBUtilities.timestampMicros()));
-        new CounterMutation(new Mutation(KS, bytes(1), cells), ConsistencyLevel.ONE).apply();
-        new CounterMutation(new Mutation(KS, bytes(2), cells), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new Mutation(KEYSPACE1, bytes(1), cells), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new Mutation(KEYSPACE1, bytes(2), cells), ConsistencyLevel.ONE).apply();
 
         // flush the counter cache and invalidate
         CacheService.instance.counterCache.submitWrite(Integer.MAX_VALUE).get();
@@ -145,22 +160,22 @@
     @Test
     public void testDroppedSaveLoad() throws ExecutionException, InterruptedException, WriteTimeoutException
     {
-        ColumnFamilyStore cfs = Keyspace.open(KS).getColumnFamilyStore(CF);
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF);
         cfs.truncateBlocking();
         CacheService.instance.invalidateCounterCache();
 
         ColumnFamily cells = ArrayBackedSortedColumns.factory.create(cfs.metadata);
         cells.addColumn(new BufferCounterUpdateCell(cellname(1), 1L, FBUtilities.timestampMicros()));
         cells.addColumn(new BufferCounterUpdateCell(cellname(2), 2L, FBUtilities.timestampMicros()));
-        new CounterMutation(new Mutation(KS, bytes(1), cells), ConsistencyLevel.ONE).apply();
-        new CounterMutation(new Mutation(KS, bytes(2), cells), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new Mutation(KEYSPACE1, bytes(1), cells), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new Mutation(KEYSPACE1, bytes(2), cells), ConsistencyLevel.ONE).apply();
 
         // flush the counter cache and invalidate
         CacheService.instance.counterCache.submitWrite(Integer.MAX_VALUE).get();
         CacheService.instance.invalidateCounterCache();
         assertEquals(0, CacheService.instance.counterCache.size());
 
-        Keyspace ks = Schema.instance.removeKeyspaceInstance(KS);
+        Keyspace ks = Schema.instance.removeKeyspaceInstance(KEYSPACE1);
 
         try
         {
@@ -177,15 +192,15 @@
     @Test
     public void testDisabledSaveLoad() throws ExecutionException, InterruptedException, WriteTimeoutException
     {
-        ColumnFamilyStore cfs = Keyspace.open(KS).getColumnFamilyStore(CF);
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF);
         cfs.truncateBlocking();
         CacheService.instance.invalidateCounterCache();
 
         ColumnFamily cells = ArrayBackedSortedColumns.factory.create(cfs.metadata);
         cells.addColumn(new BufferCounterUpdateCell(cellname(1), 1L, FBUtilities.timestampMicros()));
         cells.addColumn(new BufferCounterUpdateCell(cellname(2), 2L, FBUtilities.timestampMicros()));
-        new CounterMutation(new Mutation(KS, bytes(1), cells), ConsistencyLevel.ONE).apply();
-        new CounterMutation(new Mutation(KS, bytes(2), cells), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new Mutation(KEYSPACE1, bytes(1), cells), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new Mutation(KEYSPACE1, bytes(2), cells), ConsistencyLevel.ONE).apply();
 
         // flush the counter cache and invalidate
         CacheService.instance.counterCache.submitWrite(Integer.MAX_VALUE).get();
diff --git a/test/unit/org/apache/cassandra/db/CounterCellTest.java b/test/unit/org/apache/cassandra/db/CounterCellTest.java
index 21826d2..5d4b8a8 100644
--- a/test/unit/org/apache/cassandra/db/CounterCellTest.java
+++ b/test/unit/org/apache/cassandra/db/CounterCellTest.java
@@ -29,8 +29,8 @@
 import org.junit.Assert;
 import org.junit.Test;
 
-import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.composites.CellNameType;
 import org.apache.cassandra.db.composites.SimpleDenseCellNameType;
 import org.apache.cassandra.db.context.CounterContext;
@@ -41,7 +41,7 @@
 import static org.apache.cassandra.Util.cellname;
 import static org.apache.cassandra.db.context.CounterContext.ContextState;
 
-public class CounterCellTest extends SchemaLoader
+public class CounterCellTest
 {
     private static final CounterContext cc = new CounterContext();
 
@@ -58,6 +58,8 @@
         countLength   = 8; // size of long
 
         stepLength    = idLength + clockLength + countLength;
+        // TODO: CounterId accesses SystemKespace to get local host ID, so need to mark as daemon initialized
+        DatabaseDescriptor.setDaemonInitialized();
     }
 
     @Test
diff --git a/test/unit/org/apache/cassandra/db/CounterMutationTest.java b/test/unit/org/apache/cassandra/db/CounterMutationTest.java
index 431531c..0aa33c5 100644
--- a/test/unit/org/apache/cassandra/db/CounterMutationTest.java
+++ b/test/unit/org/apache/cassandra/db/CounterMutationTest.java
@@ -19,12 +19,17 @@
 
 import java.nio.ByteBuffer;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.db.context.CounterContext;
 import org.apache.cassandra.db.filter.QueryFilter;
+import org.apache.cassandra.db.marshal.CounterColumnType;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.WriteTimeoutException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.FBUtilities;
 
 import static org.junit.Assert.assertEquals;
@@ -34,36 +39,47 @@
 import static org.apache.cassandra.Util.dk;
 import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
 
-public class CounterMutationTest extends SchemaLoader
+public class CounterMutationTest
 {
-    private static final String KS = "CounterCacheSpace";
+    private static final String KEYSPACE1 = "CounterMutationTest";
     private static final String CF1 = "Counter1";
     private static final String CF2 = "Counter2";
 
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF1).defaultValidator(CounterColumnType.instance),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF2).defaultValidator(CounterColumnType.instance));
+    }
+
     @Test
     public void testSingleCell() throws WriteTimeoutException
     {
-        ColumnFamilyStore cfs = Keyspace.open(KS).getColumnFamilyStore(CF1);
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF1);
         cfs.truncateBlocking();
 
         // Do the initial update (+1)
         ColumnFamily cells = ArrayBackedSortedColumns.factory.create(cfs.metadata);
         cells.addCounter(cellname(1), 1L);
-        new CounterMutation(new Mutation(KS, bytes(1), cells), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new Mutation(KEYSPACE1, bytes(1), cells), ConsistencyLevel.ONE).apply();
         ColumnFamily current = cfs.getColumnFamily(QueryFilter.getIdentityFilter(dk(bytes(1)), CF1, System.currentTimeMillis()));
         assertEquals(1L, CounterContext.instance().total(current.getColumn(cellname(1)).value()));
 
         // Make another increment (+2)
         cells = ArrayBackedSortedColumns.factory.create(cfs.metadata);
         cells.addCounter(cellname(1), 2L);
-        new CounterMutation(new Mutation(KS, bytes(1), cells), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new Mutation(KEYSPACE1, bytes(1), cells), ConsistencyLevel.ONE).apply();
         current = cfs.getColumnFamily(QueryFilter.getIdentityFilter(dk(bytes(1)), CF1, System.currentTimeMillis()));
         assertEquals(3L, CounterContext.instance().total(current.getColumn(cellname(1)).value()));
 
         // Decrement to 0 (-3)
         cells = ArrayBackedSortedColumns.factory.create(cfs.metadata);
         cells.addCounter(cellname(1), -3L);
-        new CounterMutation(new Mutation(KS, bytes(1), cells), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new Mutation(KEYSPACE1, bytes(1), cells), ConsistencyLevel.ONE).apply();
         current = cfs.getColumnFamily(QueryFilter.getIdentityFilter(dk(bytes(1)), CF1, System.currentTimeMillis()));
         assertEquals(0L, CounterContext.instance().total(current.getColumn(cellname(1)).value()));
         assertEquals(ClockAndCount.create(3L, 0L), cfs.getCachedCounter(bytes(1), cellname(1)));
@@ -72,14 +88,14 @@
     @Test
     public void testTwoCells() throws WriteTimeoutException
     {
-        ColumnFamilyStore cfs = Keyspace.open(KS).getColumnFamilyStore(CF1);
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF1);
         cfs.truncateBlocking();
 
         // Do the initial update (+1, -1)
         ColumnFamily cells = ArrayBackedSortedColumns.factory.create(cfs.metadata);
         cells.addCounter(cellname(1), 1L);
         cells.addCounter(cellname(2), -1L);
-        new CounterMutation(new Mutation(KS, bytes(1), cells), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new Mutation(KEYSPACE1, bytes(1), cells), ConsistencyLevel.ONE).apply();
         ColumnFamily current = cfs.getColumnFamily(QueryFilter.getIdentityFilter(dk(bytes(1)), CF1, System.currentTimeMillis()));
         assertEquals(1L, CounterContext.instance().total(current.getColumn(cellname(1)).value()));
         assertEquals(-1L, CounterContext.instance().total(current.getColumn(cellname(2)).value()));
@@ -88,7 +104,7 @@
         cells = ArrayBackedSortedColumns.factory.create(cfs.metadata);
         cells.addCounter(cellname(1), 2L);
         cells.addCounter(cellname(2), -2L);
-        new CounterMutation(new Mutation(KS, bytes(1), cells), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new Mutation(KEYSPACE1, bytes(1), cells), ConsistencyLevel.ONE).apply();
         current = cfs.getColumnFamily(QueryFilter.getIdentityFilter(dk(bytes(1)), CF1, System.currentTimeMillis()));
         assertEquals(3L, CounterContext.instance().total(current.getColumn(cellname(1)).value()));
 
@@ -96,7 +112,7 @@
         cells = ArrayBackedSortedColumns.factory.create(cfs.metadata);
         cells.addCounter(cellname(1), -3L);
         cells.addCounter(cellname(2), 3L);
-        new CounterMutation(new Mutation(KS, bytes(1), cells), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new Mutation(KEYSPACE1, bytes(1), cells), ConsistencyLevel.ONE).apply();
         current = cfs.getColumnFamily(QueryFilter.getIdentityFilter(dk(bytes(1)), CF1, System.currentTimeMillis()));
         assertEquals(0L, CounterContext.instance().total(current.getColumn(cellname(1)).value()));
         assertEquals(0L, CounterContext.instance().total(current.getColumn(cellname(2)).value()));
@@ -109,8 +125,8 @@
     @Test
     public void testBatch() throws WriteTimeoutException
     {
-        ColumnFamilyStore cfs1 = Keyspace.open(KS).getColumnFamilyStore(CF1);
-        ColumnFamilyStore cfs2 = Keyspace.open(KS).getColumnFamilyStore(CF2);
+        ColumnFamilyStore cfs1 = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF1);
+        ColumnFamilyStore cfs2 = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF2);
 
         cfs1.truncateBlocking();
         cfs2.truncateBlocking();
@@ -124,7 +140,7 @@
         cells2.addCounter(cellname(1), 2L);
         cells2.addCounter(cellname(2), -2L);
 
-        Mutation mutation = new Mutation(KS, bytes(1));
+        Mutation mutation = new Mutation(KEYSPACE1, bytes(1));
         mutation.add(cells1);
         mutation.add(cells2);
 
@@ -149,14 +165,14 @@
     @Test
     public void testDeletes() throws WriteTimeoutException
     {
-        ColumnFamilyStore cfs = Keyspace.open(KS).getColumnFamilyStore(CF1);
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF1);
         cfs.truncateBlocking();
 
         // Do the initial update (+1, -1)
         ColumnFamily cells = ArrayBackedSortedColumns.factory.create(cfs.metadata);
         cells.addCounter(cellname(1), 1L);
         cells.addCounter(cellname(2), 1L);
-        new CounterMutation(new Mutation(KS, bytes(1), cells), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new Mutation(KEYSPACE1, bytes(1), cells), ConsistencyLevel.ONE).apply();
         ColumnFamily current = cfs.getColumnFamily(QueryFilter.getIdentityFilter(dk(bytes(1)), CF1, System.currentTimeMillis()));
         assertEquals(1L, CounterContext.instance().total(current.getColumn(cellname(1)).value()));
         assertEquals(1L, CounterContext.instance().total(current.getColumn(cellname(2)).value()));
@@ -165,7 +181,7 @@
         cells = ArrayBackedSortedColumns.factory.create(cfs.metadata);
         cells.addTombstone(cellname(1), (int) System.currentTimeMillis() / 1000, FBUtilities.timestampMicros());
         cells.addCounter(cellname(2), 1L);
-        new CounterMutation(new Mutation(KS, bytes(1), cells), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new Mutation(KEYSPACE1, bytes(1), cells), ConsistencyLevel.ONE).apply();
         current = cfs.getColumnFamily(QueryFilter.getIdentityFilter(dk(bytes(1)), CF1, System.currentTimeMillis()));
         assertNull(current.getColumn(cellname(1)));
         assertEquals(2L, CounterContext.instance().total(current.getColumn(cellname(2)).value()));
@@ -173,12 +189,12 @@
         // Increment the first counter, make sure it's still shadowed by the tombstone
         cells = ArrayBackedSortedColumns.factory.create(cfs.metadata);
         cells.addCounter(cellname(1), 1L);
-        new CounterMutation(new Mutation(KS, bytes(1), cells), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new Mutation(KEYSPACE1, bytes(1), cells), ConsistencyLevel.ONE).apply();
         current = cfs.getColumnFamily(QueryFilter.getIdentityFilter(dk(bytes(1)), CF1, System.currentTimeMillis()));
         assertNull(current.getColumn(cellname(1)));
 
         // Get rid of the complete partition
-        Mutation mutation = new Mutation(KS, bytes(1));
+        Mutation mutation = new Mutation(KEYSPACE1, bytes(1));
         mutation.delete(CF1, FBUtilities.timestampMicros());
         new CounterMutation(mutation, ConsistencyLevel.ONE).apply();
         current = cfs.getColumnFamily(QueryFilter.getIdentityFilter(dk(bytes(1)), CF1, System.currentTimeMillis()));
@@ -189,7 +205,7 @@
         cells = ArrayBackedSortedColumns.factory.create(cfs.metadata);
         cells.addCounter(cellname(1), 1L);
         cells.addCounter(cellname(2), 1L);
-        new CounterMutation(new Mutation(KS, bytes(1), cells), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new Mutation(KEYSPACE1, bytes(1), cells), ConsistencyLevel.ONE).apply();
         current = cfs.getColumnFamily(QueryFilter.getIdentityFilter(dk(bytes(1)), CF1, System.currentTimeMillis()));
         assertNull(current.getColumn(cellname(1)));
         assertNull(current.getColumn(cellname(2)));
@@ -198,7 +214,7 @@
     @Test
     public void testDuplicateCells() throws WriteTimeoutException
     {
-        ColumnFamilyStore cfs = Keyspace.open(KS).getColumnFamilyStore(CF1);
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF1);
         cfs.truncateBlocking();
 
         ColumnFamily cells = ArrayBackedSortedColumns.factory.create(cfs.metadata);
@@ -206,7 +222,7 @@
         cells.addCounter(cellname(1), 2L);
         cells.addCounter(cellname(1), 3L);
         cells.addCounter(cellname(1), 4L);
-        new CounterMutation(new Mutation(KS, bytes(1), cells), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new Mutation(KEYSPACE1, bytes(1), cells), ConsistencyLevel.ONE).apply();
 
         ColumnFamily current = cfs.getColumnFamily(QueryFilter.getIdentityFilter(dk(bytes(1)), CF1, System.currentTimeMillis()));
         ByteBuffer context = current.getColumn(cellname(1)).value();
diff --git a/test/unit/org/apache/cassandra/db/DataTrackerTest.java b/test/unit/org/apache/cassandra/db/DataTrackerTest.java
deleted file mode 100644
index b557ee3..0000000
--- a/test/unit/org/apache/cassandra/db/DataTrackerTest.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
-* 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.db;
-
-import java.nio.ByteBuffer;
-import java.util.HashSet;
-import java.util.Set;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
-import org.junit.Test;
-
-import junit.framework.Assert;
-import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.Util;
-import org.apache.cassandra.io.sstable.SSTableReader;
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-public class DataTrackerTest extends SchemaLoader
-{
-    private static final String KEYSPACE = "Keyspace1";
-    private static final String CF = "Standard1";
-
-    @Test
-    public void testCompactOnlyCorrectInstance()
-    {
-        Keyspace keyspace = Keyspace.open(KEYSPACE);
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
-        cfs.truncateBlocking();
-        for (int j = 0; j < 100; j ++)
-        {
-            ByteBuffer key = ByteBufferUtil.bytes(String.valueOf(j));
-            Mutation rm = new Mutation(KEYSPACE, key);
-            rm.add(CF, Util.cellname("0"), ByteBufferUtil.EMPTY_BYTE_BUFFER, j);
-            rm.apply();
-        }
-        cfs.forceBlockingFlush();
-        Set<SSTableReader> sstables = new HashSet<>(cfs.getSSTables());
-        assert sstables.size() == 1;
-        SSTableReader reader = Iterables.getFirst(sstables, null);
-        SSTableReader reader2 = reader.cloneAsShadowed(new Runnable() { public void run() { } });
-        Assert.assertFalse(cfs.getDataTracker().markCompacting(ImmutableList.of(reader2)));
-        Assert.assertTrue(cfs.getDataTracker().markCompacting(ImmutableList.of(reader)));
-        cfs.getDataTracker().replaceWithNewInstances(ImmutableList.of(reader), ImmutableList.of(reader2));
-        cfs.getDataTracker().unmarkCompacting(ImmutableList.of(reader));
-        cfs.truncateBlocking();
-    }
-
-}
diff --git a/test/unit/org/apache/cassandra/db/DirectoriesTest.java b/test/unit/org/apache/cassandra/db/DirectoriesTest.java
index 4267c1f..bc5b7bf 100644
--- a/test/unit/org/apache/cassandra/db/DirectoriesTest.java
+++ b/test/unit/org/apache/cassandra/db/DirectoriesTest.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -17,16 +17,8 @@
  */
 package org.apache.cassandra.db;
 
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.IdentityHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
+import java.io.*;
+import java.util.*;
 import java.util.concurrent.Callable;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
@@ -47,6 +39,7 @@
 import org.apache.cassandra.service.DefaultFSErrorHandler;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.utils.Pair;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -61,7 +54,11 @@
     private static final String[] CFS = new String[] { "cf1", "ks" };
 
     private static final Set<CFMetaData> CFM = new HashSet<>(CFS.length);
-    private static Map<String, List<File>> files = new HashMap<String, List<File>>();
+
+    private static final CFMetaData PARENT_CFM = new CFMetaData(KS, "cf", ColumnFamilyType.Standard, null);
+    private static final CFMetaData INDEX_CFM = new CFMetaData(KS, "cf.idx", ColumnFamilyType.Standard, null, PARENT_CFM.cfId);
+
+    private static final Map<String, List<File>> files = new HashMap<>();
 
     @BeforeClass
     public static void beforeClass() throws IOException
@@ -124,11 +121,23 @@
     private static File cfDir(CFMetaData metadata)
     {
         String cfId = ByteBufferUtil.bytesToHex(ByteBufferUtil.bytes(metadata.cfId));
-        return new File(tempDataDir, metadata.ksName + File.separator + metadata.cfName + "-" + cfId);
+        int idx = metadata.cfName.indexOf(Directories.SECONDARY_INDEX_NAME_SEPARATOR);
+        if (idx >= 0)
+        {
+            // secondary index
+            return new File(tempDataDir,
+                            metadata.ksName + File.separator +
+                            metadata.cfName.substring(0, idx) + '-' + cfId + File.separator +
+                            metadata.cfName.substring(idx));
+        }
+        else
+        {
+            return new File(tempDataDir, metadata.ksName + File.separator + metadata.cfName + '-' + cfId);
+        }
     }
 
     @Test
-    public void testStandardDirs()
+    public void testStandardDirs() throws IOException
     {
         for (CFMetaData cfm : CFM)
         {
@@ -137,14 +146,77 @@
 
             Descriptor desc = new Descriptor(cfDir(cfm), KS, cfm.cfName, 1, Descriptor.Type.FINAL);
             File snapshotDir = new File(cfDir(cfm),  File.separator + Directories.SNAPSHOT_SUBDIR + File.separator + "42");
-            assertEquals(snapshotDir, Directories.getSnapshotDirectory(desc, "42"));
+            assertEquals(snapshotDir.getCanonicalFile(), Directories.getSnapshotDirectory(desc, "42"));
 
             File backupsDir = new File(cfDir(cfm),  File.separator + Directories.BACKUPS_SUBDIR);
-            assertEquals(backupsDir, Directories.getBackupsDirectory(desc));
+            assertEquals(backupsDir.getCanonicalFile(), Directories.getBackupsDirectory(desc));
         }
     }
 
     @Test
+    public void testSecondaryIndexDirectories()
+    {
+        Directories parentDirectories = new Directories(PARENT_CFM);
+        Directories indexDirectories = new Directories(INDEX_CFM);
+        // secondary index has its own directory
+        for (File dir : indexDirectories.getCFDirectories())
+        {
+            assertEquals(cfDir(INDEX_CFM), dir);
+        }
+        Descriptor parentDesc = new Descriptor(parentDirectories.getDirectoryForNewSSTables(), KS, PARENT_CFM.cfName, 0, Descriptor.Type.FINAL);
+        Descriptor indexDesc = new Descriptor(indexDirectories.getDirectoryForNewSSTables(), KS, INDEX_CFM.cfName, 0, Descriptor.Type.FINAL);
+
+        // snapshot dir should be created under its parent's
+        File parentSnapshotDirectory = Directories.getSnapshotDirectory(parentDesc, "test");
+        File indexSnapshotDirectory = Directories.getSnapshotDirectory(indexDesc, "test");
+        assertEquals(parentSnapshotDirectory, indexSnapshotDirectory.getParentFile());
+
+        // check if snapshot directory exists
+        parentSnapshotDirectory.mkdirs();
+        assertTrue(parentDirectories.snapshotExists("test"));
+        assertTrue(indexDirectories.snapshotExists("test"));
+
+        // check their creation time
+        assertEquals(parentDirectories.snapshotCreationTime("test"),
+                     indexDirectories.snapshotCreationTime("test"));
+
+        // check true snapshot size
+        Descriptor parentSnapshot = new Descriptor(parentSnapshotDirectory, KS, PARENT_CFM.cfName, 0, Descriptor.Type.FINAL);
+        createFile(parentSnapshot.filenameFor(Component.DATA), 30);
+        Descriptor indexSnapshot = new Descriptor(indexSnapshotDirectory, KS, INDEX_CFM.cfName, 0, Descriptor.Type.FINAL);
+        createFile(indexSnapshot.filenameFor(Component.DATA), 40);
+
+        assertEquals(30, parentDirectories.trueSnapshotsSize());
+        assertEquals(40, indexDirectories.trueSnapshotsSize());
+
+        // check snapshot details
+        Map<String, Pair<Long, Long>> parentSnapshotDetail = parentDirectories.getSnapshotDetails();
+        assertTrue(parentSnapshotDetail.containsKey("test"));
+        assertEquals(30L, parentSnapshotDetail.get("test").right.longValue());
+
+        Map<String, Pair<Long, Long>> indexSnapshotDetail = indexDirectories.getSnapshotDetails();
+        assertTrue(indexSnapshotDetail.containsKey("test"));
+        assertEquals(40L, indexSnapshotDetail.get("test").right.longValue());
+
+        // check backup directory
+        File parentBackupDirectory = Directories.getBackupsDirectory(parentDesc);
+        File indexBackupDirectory = Directories.getBackupsDirectory(indexDesc);
+        assertEquals(parentBackupDirectory, indexBackupDirectory.getParentFile());
+    }
+
+    private File createFile(String fileName, int size)
+    {
+        File newFile = new File(fileName);
+        try (FileOutputStream writer = new FileOutputStream(newFile))
+        {
+            writer.write(new byte[size]);
+            writer.flush();
+        }
+        catch (IOException ignore) {}
+        return newFile;
+    }
+
+    @Test
     public void testSSTableLister()
     {
         for (CFMetaData cfm : CFM)
@@ -182,7 +254,7 @@
             {
                 if (f.getPath().contains(Directories.SNAPSHOT_SUBDIR) || f.getPath().contains(Directories.BACKUPS_SUBDIR))
                     assert !listed.contains(f) : f + " should not be listed";
-                else if (f.getName().contains("-tmp-"))
+                else if (f.getName().contains("tmp-"))
                     assert !listed.contains(f) : f + " should not be listed";
                 else
                     assert listed.contains(f) : f + " is missing";
diff --git a/test/unit/org/apache/cassandra/db/HintedHandOffTest.java b/test/unit/org/apache/cassandra/db/HintedHandOffTest.java
index ad1e6cf..6bb6be2 100644
--- a/test/unit/org/apache/cassandra/db/HintedHandOffTest.java
+++ b/test/unit/org/apache/cassandra/db/HintedHandOffTest.java
@@ -24,36 +24,50 @@
 import java.util.Map;
 import java.util.UUID;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import com.google.common.collect.Iterators;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy;
 import org.apache.cassandra.db.marshal.Int32Type;
 import org.apache.cassandra.db.marshal.UUIDType;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.Pair;
 
 import static org.junit.Assert.assertEquals;
 import static org.apache.cassandra.cql3.QueryProcessor.executeInternal;
 
-public class HintedHandOffTest extends SchemaLoader
+public class HintedHandOffTest
 {
 
-    public static final String KEYSPACE4 = "Keyspace4";
+    public static final String KEYSPACE4 = "HintedHandOffTest4";
     public static final String STANDARD1_CF = "Standard1";
     public static final String COLUMN1 = "column1";
 
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE4,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE4, STANDARD1_CF));
+    }
+
     // Test compaction of hints column family. It shouldn't remove all columns on compaction.
     @Test
     public void testCompactionOfHintsCF() throws Exception
     {
         // prepare hints column family
         Keyspace systemKeyspace = Keyspace.open("system");
-        ColumnFamilyStore hintStore = systemKeyspace.getColumnFamilyStore(SystemKeyspace.HINTS_CF);
+        ColumnFamilyStore hintStore = systemKeyspace.getColumnFamilyStore(SystemKeyspace.HINTS);
         hintStore.clearUnsafe();
         hintStore.metadata.gcGraceSeconds(36000); // 10 hours
         hintStore.setCompactionStrategyClass(SizeTieredCompactionStrategy.class.getCanonicalName());
@@ -67,7 +81,7 @@
                                               System.currentTimeMillis(),
                                               HintedHandOffManager.calculateHintTTL(rm),
                                               Pair.create(InetAddress.getByName("127.0.0.1"), UUID.randomUUID()))
-                                     .apply();
+                                     .applyUnsafe();
 
         // flush data to disk
         hintStore.forceBlockingFlush();
@@ -88,7 +102,7 @@
             HintedHandOffManager.instance.metrics.incrPastWindow(InetAddress.getLocalHost());
         HintedHandOffManager.instance.metrics.log();
 
-        UntypedResultSet rows = executeInternal("SELECT hints_dropped FROM system." + SystemKeyspace.PEER_EVENTS_CF);
+        UntypedResultSet rows = executeInternal("SELECT hints_dropped FROM system." + SystemKeyspace.PEER_EVENTS);
         Map<UUID, Integer> returned = rows.one().getMap("hints_dropped", UUIDType.instance, Int32Type.instance);
         assertEquals(Iterators.getLast(returned.values().iterator()).intValue(), 99);
     }
@@ -97,7 +111,7 @@
     public void testTruncateHints() throws Exception
     {
         Keyspace systemKeyspace = Keyspace.open("system");
-        ColumnFamilyStore hintStore = systemKeyspace.getColumnFamilyStore(SystemKeyspace.HINTS_CF);
+        ColumnFamilyStore hintStore = systemKeyspace.getColumnFamilyStore(SystemKeyspace.HINTS);
         hintStore.clearUnsafe();
 
         // insert 1 hint
@@ -108,7 +122,7 @@
                                               System.currentTimeMillis(),
                                               HintedHandOffManager.calculateHintTTL(rm),
                                               Pair.create(InetAddress.getByName("127.0.0.1"), UUID.randomUUID()))
-                                     .apply();
+                                     .applyUnsafe();
 
         assert getNoOfHints() == 1;
 
@@ -125,7 +139,7 @@
     private int getNoOfHints()
     {
         String req = "SELECT * FROM system.%s";
-        UntypedResultSet resultSet = executeInternal(String.format(req, SystemKeyspace.HINTS_CF));
+        UntypedResultSet resultSet = executeInternal(String.format(req, SystemKeyspace.HINTS));
         return resultSet.size();
     }
 }
diff --git a/test/unit/org/apache/cassandra/db/KeyCacheTest.java b/test/unit/org/apache/cassandra/db/KeyCacheTest.java
index d3328f1..c8caff9 100644
--- a/test/unit/org/apache/cassandra/db/KeyCacheTest.java
+++ b/test/unit/org/apache/cassandra/db/KeyCacheTest.java
@@ -19,6 +19,7 @@
 
 import java.util.Collection;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ExecutionException;
@@ -27,35 +28,52 @@
 import com.google.common.collect.ImmutableList;
 import com.google.common.util.concurrent.Uninterruptibles;
 import org.junit.AfterClass;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.cache.KeyCacheKey;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.concurrent.ScheduledExecutors;
 import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.filter.QueryFilter;
-import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.service.CacheService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 import org.apache.cassandra.utils.concurrent.Refs;
 import static org.junit.Assert.assertEquals;
 
-public class KeyCacheTest extends SchemaLoader
+public class KeyCacheTest
 {
-    private static final String KEYSPACE1 = "KeyCacheSpace";
+    private static final String KEYSPACE1 = "KeyCacheTest1";
     private static final String COLUMN_FAMILY1 = "Standard1";
     private static final String COLUMN_FAMILY2 = "Standard2";
     private static final String COLUMN_FAMILY3 = "Standard3";
 
 
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, COLUMN_FAMILY1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, COLUMN_FAMILY2),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, COLUMN_FAMILY3));
+    }
+
     @AfterClass
     public static void cleanup()
     {
-        cleanupSavedCaches();
+        SchemaLoader.cleanupSavedCaches();
     }
 
     @Test
@@ -70,17 +88,19 @@
         assertKeyCacheSize(0, KEYSPACE1, COLUMN_FAMILY2);
 
         // insert data and force to disk
-        insertData(KEYSPACE1, COLUMN_FAMILY2, 0, 100);
+        SchemaLoader.insertData(KEYSPACE1, COLUMN_FAMILY2, 0, 100);
         store.forceBlockingFlush();
 
         // populate the cache
-        readData(KEYSPACE1, COLUMN_FAMILY2, 0, 100);
+        SchemaLoader.readData(KEYSPACE1, COLUMN_FAMILY2, 0, 100);
         assertKeyCacheSize(100, KEYSPACE1, COLUMN_FAMILY2);
 
         // really? our caches don't implement the map interface? (hence no .addAll)
         Map<KeyCacheKey, RowIndexEntry> savedMap = new HashMap<KeyCacheKey, RowIndexEntry>();
-        for (KeyCacheKey k : CacheService.instance.keyCache.getKeySet())
+        for (Iterator<KeyCacheKey> iter = CacheService.instance.keyCache.keyIterator();
+             iter.hasNext();)
         {
+            KeyCacheKey k = iter.next();
             if (k.desc.ksname.equals(KEYSPACE1) && k.desc.cfname.equals(COLUMN_FAMILY2))
                 savedMap.put(k, CacheService.instance.keyCache.get(k));
         }
@@ -120,21 +140,21 @@
         assertKeyCacheSize(0, KEYSPACE1, COLUMN_FAMILY3);
 
         // insert data and force to disk
-        insertData(KEYSPACE1, COLUMN_FAMILY3, 0, 100);
+        SchemaLoader.insertData(KEYSPACE1, COLUMN_FAMILY3, 0, 100);
         store.forceBlockingFlush();
 
         Collection<SSTableReader> firstFlushTables = ImmutableList.copyOf(store.getSSTables());
 
         // populate the cache
-        readData(KEYSPACE1, COLUMN_FAMILY3, 0, 100);
+        SchemaLoader.readData(KEYSPACE1, COLUMN_FAMILY3, 0, 100);
         assertKeyCacheSize(100, KEYSPACE1, COLUMN_FAMILY3);
 
         // insert some new data and force to disk
-        insertData(KEYSPACE1, COLUMN_FAMILY3, 100, 50);
+        SchemaLoader.insertData(KEYSPACE1, COLUMN_FAMILY3, 100, 50);
         store.forceBlockingFlush();
 
         // check that it's fine
-        readData(KEYSPACE1, COLUMN_FAMILY3, 100, 50);
+        SchemaLoader.readData(KEYSPACE1, COLUMN_FAMILY3, 100, 50);
         assertKeyCacheSize(150, KEYSPACE1, COLUMN_FAMILY3);
 
         // force the cache to disk
@@ -179,10 +199,10 @@
         // inserts
         rm = new Mutation(KEYSPACE1, key1.getKey());
         rm.add(COLUMN_FAMILY1, Util.cellname("1"), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
-        rm.apply();
+        rm.applyUnsafe();
         rm = new Mutation(KEYSPACE1, key2.getKey());
         rm.add(COLUMN_FAMILY1, Util.cellname("2"), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
-        rm.apply();
+        rm.applyUnsafe();
 
         // to make sure we have SSTable
         cfs.forceBlockingFlush();
@@ -206,16 +226,18 @@
 
         assertKeyCacheSize(2, KEYSPACE1, COLUMN_FAMILY1);
 
-        Set<SSTableReader> readers = cfs.getDataTracker().getSSTables();
+        Set<SSTableReader> readers = cfs.getTracker().getSSTables();
         Refs<SSTableReader> refs = Refs.tryRef(readers);
         if (refs == null)
             throw new IllegalStateException();
 
         Util.compactAll(cfs, Integer.MAX_VALUE).get();
+        boolean noEarlyOpen = DatabaseDescriptor.getSSTablePreempiveOpenIntervalInMB() < 0;
+
         // after compaction cache should have entries for new SSTables,
         // but since we have kept a reference to the old sstables,
         // if we had 2 keys in cache previously it should become 4
-        assertKeyCacheSize(4, KEYSPACE1, COLUMN_FAMILY1);
+        assertKeyCacheSize(noEarlyOpen ? 2 : 4, KEYSPACE1, COLUMN_FAMILY1);
 
         refs.release();
 
@@ -242,14 +264,16 @@
                                                        10,
                                                        System.currentTimeMillis()));
 
-        assertKeyCacheSize(2, KEYSPACE1, COLUMN_FAMILY1);
+        assertKeyCacheSize(noEarlyOpen ? 4 : 2, KEYSPACE1, COLUMN_FAMILY1);
     }
 
     private void assertKeyCacheSize(int expected, String keyspace, String columnFamily)
     {
         int size = 0;
-        for (KeyCacheKey k : CacheService.instance.keyCache.getKeySet())
+        for (Iterator<KeyCacheKey> iter = CacheService.instance.keyCache.keyIterator();
+             iter.hasNext();)
         {
+            KeyCacheKey k = iter.next();
             if (k.desc.ksname.equals(keyspace) && k.desc.cfname.equals(columnFamily))
                 size++;
         }
diff --git a/test/unit/org/apache/cassandra/db/KeyspaceTest.java b/test/unit/org/apache/cassandra/db/KeyspaceTest.java
index d610563..d9481ca 100644
--- a/test/unit/org/apache/cassandra/db/KeyspaceTest.java
+++ b/test/unit/org/apache/cassandra/db/KeyspaceTest.java
@@ -26,29 +26,66 @@
 import java.io.IOException;
 
 import com.google.common.collect.Iterables;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.commons.lang3.StringUtils;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import static org.junit.Assert.*;
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.filter.QueryFilter;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.CompositeType;
+import org.apache.cassandra.db.marshal.IntegerType;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.metrics.ClearableHistogram;
 import org.apache.cassandra.utils.WrappedRunnable;
 import static org.apache.cassandra.Util.column;
 import static org.apache.cassandra.Util.expiringColumn;
 import static org.apache.cassandra.Util.cellname;
 import org.apache.cassandra.Util;
-import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 
-public class KeyspaceTest extends SchemaLoader
+public class KeyspaceTest
 {
     private static final DecoratedKey TEST_KEY = Util.dk("key1");
     private static final DecoratedKey TEST_SLICE_KEY = Util.dk("key1-slicerange");
 
+    private static final String KEYSPACE1 = "Keyspace1";
+    private static final String CF_STANDARD1 = "Standard1";
+    private static final String CF_STANDARD2 = "Standard2";
+    private static final String CF_STANDARDLONG = "StandardLong1";
+    private static final String CF_STANDARDCOMPOSITE2 = "StandardComposite2";
+
+    private static final String KEYSPACE2 = "Keyspace2";
+    private static final String CF_STANDARD3 = "Standard3";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        AbstractType<?> compositeMaxMin = CompositeType.getInstance(Arrays.asList(new AbstractType<?>[]{BytesType.instance, IntegerType.instance}));
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD2),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARDLONG),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARDCOMPOSITE2, compositeMaxMin));
+        SchemaLoader.createKeyspace(KEYSPACE2,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE2, CF_STANDARD3));
+    }
+
     public static void reTest(ColumnFamilyStore cfs, Runnable verify) throws Exception
     {
         verify.run();
@@ -59,13 +96,13 @@
     @Test
     public void testGetRowNoColumns() throws Throwable
     {
-        final Keyspace keyspace = Keyspace.open("Keyspace2");
+        final Keyspace keyspace = Keyspace.open(KEYSPACE2);
         final ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore("Standard3");
 
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace2", "Standard3");
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE2, "Standard3");
         cf.addColumn(column("col1","val1", 1L));
-        Mutation rm = new Mutation("Keyspace2", TEST_KEY.getKey(), cf);
-        rm.apply();
+        Mutation rm = new Mutation(KEYSPACE2, TEST_KEY.getKey(), cf);
+        rm.applyUnsafe();
 
         Runnable verify = new WrappedRunnable()
         {
@@ -89,15 +126,15 @@
     @Test
     public void testGetRowSingleColumn() throws Throwable
     {
-        final Keyspace keyspace = Keyspace.open("Keyspace1");
+        final Keyspace keyspace = Keyspace.open(KEYSPACE1);
         final ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore("Standard1");
 
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf.addColumn(column("col1","val1", 1L));
         cf.addColumn(column("col2","val2", 1L));
         cf.addColumn(column("col3","val3", 1L));
-        Mutation rm = new Mutation("Keyspace1", TEST_KEY.getKey(), cf);
-        rm.apply();
+        Mutation rm = new Mutation(KEYSPACE1, TEST_KEY.getKey(), cf);
+        rm.applyUnsafe();
 
         Runnable verify = new WrappedRunnable()
         {
@@ -118,16 +155,16 @@
     @Test
     public void testGetRowSliceByRange() throws Throwable
     {
-    	DecoratedKey key = TEST_SLICE_KEY;
-    	Keyspace keyspace = Keyspace.open("Keyspace1");
+        DecoratedKey key = TEST_SLICE_KEY;
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore("Standard1");
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         // First write "a", "b", "c"
         cf.addColumn(column("a", "val1", 1L));
         cf.addColumn(column("b", "val2", 1L));
         cf.addColumn(column("c", "val3", 1L));
-        Mutation rm = new Mutation("Keyspace1", key.getKey(), cf);
-        rm.apply();
+        Mutation rm = new Mutation(KEYSPACE1, key.getKey(), cf);
+        rm.applyUnsafe();
 
         cf = cfStore.getColumnFamily(key, cellname("b"), cellname("c"), false, 100, System.currentTimeMillis());
         assertEquals(2, cf.getColumnCount());
@@ -142,11 +179,11 @@
     @Test
     public void testGetSliceNoMatch() throws Throwable
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard2");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard2");
         cf.addColumn(column("col1", "val1", 1));
-        Mutation rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("row1000"), cf);
-        rm.apply();
+        Mutation rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("row1000"), cf);
+        rm.applyUnsafe();
 
         validateGetSliceNoMatch(keyspace);
         keyspace.getColumnFamilyStore("Standard2").forceBlockingFlush();
@@ -162,18 +199,18 @@
     public void testGetSliceWithCutoff() throws Throwable
     {
         // tests slicing against data from one row in a memtable and then flushed to an sstable
-        final Keyspace keyspace = Keyspace.open("Keyspace1");
+        final Keyspace keyspace = Keyspace.open(KEYSPACE1);
         final ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore("Standard1");
         final DecoratedKey ROW = Util.dk("row4");
         final NumberFormat fmt = new DecimalFormat("000");
 
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         // at this rate, we're getting 78-79 cos/block, assuming the blocks are set to be about 4k.
         // so if we go to 300, we'll get at least 4 blocks, which is plenty for testing.
         for (int i = 0; i < 300; i++)
             cf.addColumn(column("col" + fmt.format(i), "omg!thisisthevalue!"+i, 1L));
-        Mutation rm = new Mutation("Keyspace1", ROW.getKey(), cf);
-        rm.apply();
+        Mutation rm = new Mutation(KEYSPACE1, ROW.getKey(), cf);
+        rm.applyUnsafe();
 
         Runnable verify = new WrappedRunnable()
         {
@@ -219,26 +256,26 @@
     @Test
     public void testReversedWithFlushing()
     {
-        final Keyspace keyspace = Keyspace.open("Keyspace1");
+        final Keyspace keyspace = Keyspace.open(KEYSPACE1);
         final ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("StandardLong1");
         final DecoratedKey ROW = Util.dk("row4");
 
         for (int i = 0; i < 10; i++)
         {
-            ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "StandardLong1");
+            ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "StandardLong1");
             cf.addColumn(new BufferCell(cellname((long)i), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0));
-            Mutation rm = new Mutation("Keyspace1", ROW.getKey(), cf);
-            rm.apply();
+            Mutation rm = new Mutation(KEYSPACE1, ROW.getKey(), cf);
+            rm.applyUnsafe();
         }
 
         cfs.forceBlockingFlush();
 
         for (int i = 10; i < 20; i++)
         {
-            ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "StandardLong1");
+            ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "StandardLong1");
             cf.addColumn(new BufferCell(cellname((long)i), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0));
-            Mutation rm = new Mutation("Keyspace1", ROW.getKey(), cf);
-            rm.apply();
+            Mutation rm = new Mutation(KEYSPACE1, ROW.getKey(), cf);
+            rm.applyUnsafe();
 
             cf = cfs.getColumnFamily(ROW, Composites.EMPTY, Composites.EMPTY, true, 1, System.currentTimeMillis());
             assertEquals(1, Iterables.size(cf.getColumnNames()));
@@ -264,23 +301,23 @@
     public void testGetSliceFromBasic() throws Throwable
     {
         // tests slicing against data from one row in a memtable and then flushed to an sstable
-        final Keyspace keyspace = Keyspace.open("Keyspace1");
+        final Keyspace keyspace = Keyspace.open(KEYSPACE1);
         final ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore("Standard1");
         final DecoratedKey ROW = Util.dk("row1");
 
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf.addColumn(column("col1", "val1", 1L));
         cf.addColumn(column("col3", "val3", 1L));
         cf.addColumn(column("col4", "val4", 1L));
         cf.addColumn(column("col5", "val5", 1L));
         cf.addColumn(column("col7", "val7", 1L));
         cf.addColumn(column("col9", "val9", 1L));
-        Mutation rm = new Mutation("Keyspace1", ROW.getKey(), cf);
-        rm.apply();
+        Mutation rm = new Mutation(KEYSPACE1, ROW.getKey(), cf);
+        rm.applyUnsafe();
 
-        rm = new Mutation("Keyspace1", ROW.getKey());
+        rm = new Mutation(KEYSPACE1, ROW.getKey());
         rm.delete("Standard1", cellname("col4"), 2L);
-        rm.apply();
+        rm.applyUnsafe();
 
         Runnable verify = new WrappedRunnable()
         {
@@ -319,16 +356,16 @@
     public void testGetSliceWithExpiration() throws Throwable
     {
         // tests slicing against data from one row with expiring column in a memtable and then flushed to an sstable
-        final Keyspace keyspace = Keyspace.open("Keyspace1");
+        final Keyspace keyspace = Keyspace.open(KEYSPACE1);
         final ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore("Standard1");
         final DecoratedKey ROW = Util.dk("row5");
 
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf.addColumn(column("col1", "val1", 1L));
         cf.addColumn(expiringColumn("col2", "val2", 1L, 60)); // long enough not to be tombstoned
         cf.addColumn(column("col3", "val3", 1L));
-        Mutation rm = new Mutation("Keyspace1", ROW.getKey(), cf);
-        rm.apply();
+        Mutation rm = new Mutation(KEYSPACE1, ROW.getKey(), cf);
+        rm.applyUnsafe();
 
         Runnable verify = new WrappedRunnable()
         {
@@ -353,27 +390,27 @@
     public void testGetSliceFromAdvanced() throws Throwable
     {
         // tests slicing against data from one row spread across two sstables
-        final Keyspace keyspace = Keyspace.open("Keyspace1");
+        final Keyspace keyspace = Keyspace.open(KEYSPACE1);
         final ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore("Standard1");
         final DecoratedKey ROW = Util.dk("row2");
 
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf.addColumn(column("col1", "val1", 1L));
         cf.addColumn(column("col2", "val2", 1L));
         cf.addColumn(column("col3", "val3", 1L));
         cf.addColumn(column("col4", "val4", 1L));
         cf.addColumn(column("col5", "val5", 1L));
         cf.addColumn(column("col6", "val6", 1L));
-        Mutation rm = new Mutation("Keyspace1", ROW.getKey(), cf);
-        rm.apply();
+        Mutation rm = new Mutation(KEYSPACE1, ROW.getKey(), cf);
+        rm.applyUnsafe();
         cfStore.forceBlockingFlush();
 
-        cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf.addColumn(column("col1", "valx", 2L));
         cf.addColumn(column("col2", "valx", 2L));
         cf.addColumn(column("col3", "valx", 2L));
-        rm = new Mutation("Keyspace1", ROW.getKey(), cf);
-        rm.apply();
+        rm = new Mutation(KEYSPACE1, ROW.getKey(), cf);
+        rm.applyUnsafe();
 
         Runnable verify = new WrappedRunnable()
         {
@@ -402,14 +439,14 @@
     public void testGetSliceFromLarge() throws Throwable
     {
         // tests slicing against 1000 columns in an sstable
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore("Standard1");
         DecoratedKey key = Util.dk("row3");
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         for (int i = 1000; i < 2000; i++)
             cf.addColumn(column("col" + i, ("v" + i), 1L));
-        Mutation rm = new Mutation("Keyspace1", key.getKey(), cf);
-        rm.apply();
+        Mutation rm = new Mutation(KEYSPACE1, key.getKey(), cf);
+        rm.applyUnsafe();
         cfStore.forceBlockingFlush();
 
         validateSliceLarge(cfStore);
@@ -417,7 +454,7 @@
         // compact so we have a big row with more than the minimum index count
         if (cfStore.getSSTables().size() > 1)
         {
-            CompactionManager.instance.performMaximal(cfStore);
+            CompactionManager.instance.performMaximal(cfStore, false);
         }
         // verify that we do indeed have multiple index entries
         SSTableReader sstable = cfStore.getSSTables().iterator().next();
@@ -430,33 +467,33 @@
     @Test
     public void testLimitSSTables() throws CharacterCodingException
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore("Standard1");
         cfStore.disableAutoCompaction();
         DecoratedKey key = Util.dk("row_maxmin");
         for (int j = 0; j < 10; j++)
         {
-            ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+            ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
             for (int i = 1000 + (j*100); i < 1000 + ((j+1)*100); i++)
             {
                 cf.addColumn(column("col" + i, ("v" + i), i));
             }
-            Mutation rm = new Mutation("Keyspace1", key.getKey(), cf);
-            rm.apply();
+            Mutation rm = new Mutation(KEYSPACE1, key.getKey(), cf);
+            rm.applyUnsafe();
             cfStore.forceBlockingFlush();
         }
-        cfStore.metric.sstablesPerReadHistogram.cf.clear();
+        ((ClearableHistogram)cfStore.metric.sstablesPerReadHistogram.cf).clear();
         ColumnFamily cf = cfStore.getColumnFamily(key, Composites.EMPTY, cellname("col1499"), false, 1000, System.currentTimeMillis());
-        assertEquals(cfStore.metric.sstablesPerReadHistogram.cf.max(), 5, 0.1);
+        assertEquals(cfStore.metric.sstablesPerReadHistogram.cf.getSnapshot().getMax(), 5, 0.1);
         int i = 0;
         for (Cell c : cf.getSortedColumns())
         {
             assertEquals(ByteBufferUtil.string(c.name().toByteBuffer()), "col" + (1000 + i++));
         }
         assertEquals(i, 500);
-        cfStore.metric.sstablesPerReadHistogram.cf.clear();
+        ((ClearableHistogram)cfStore.metric.sstablesPerReadHistogram.cf).clear();
         cf = cfStore.getColumnFamily(key, cellname("col1500"), cellname("col2000"), false, 1000, System.currentTimeMillis());
-        assertEquals(cfStore.metric.sstablesPerReadHistogram.cf.max(), 5, 0.1);
+        assertEquals(cfStore.metric.sstablesPerReadHistogram.cf.getSnapshot().getMax(), 5, 0.1);
 
         for (Cell c : cf.getSortedColumns())
         {
@@ -465,9 +502,9 @@
         assertEquals(i, 1000);
 
         // reverse
-        cfStore.metric.sstablesPerReadHistogram.cf.clear();
+        ((ClearableHistogram)cfStore.metric.sstablesPerReadHistogram.cf).clear();
         cf = cfStore.getColumnFamily(key, cellname("col2000"), cellname("col1500"), true, 1000, System.currentTimeMillis());
-        assertEquals(cfStore.metric.sstablesPerReadHistogram.cf.max(), 5, 0.1);
+        assertEquals(cfStore.metric.sstablesPerReadHistogram.cf.getSnapshot().getMax(), 5, 0.1);
         i = 500;
         for (Cell c : cf.getSortedColumns())
         {
@@ -494,7 +531,7 @@
         ---------------------
         then we slice out col1 = a5 and col2 > 85 -> which should let us just check 2 sstables and get 2 columns
          */
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
 
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("StandardComposite2");
         cfs.disableAutoCompaction();
@@ -505,22 +542,22 @@
         {
             for (int i = 0; i < 10; i++)
             {
-                Mutation rm = new Mutation("Keyspace1", key.getKey());
+                Mutation rm = new Mutation(KEYSPACE1, key.getKey());
                 CellName colName = type.makeCellName(ByteBufferUtil.bytes("a" + i), ByteBufferUtil.bytes(j*10 + i));
                 rm.add("StandardComposite2", colName, ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
-                rm.apply();
+                rm.applyUnsafe();
             }
             cfs.forceBlockingFlush();
         }
         Composite start = type.builder().add(ByteBufferUtil.bytes("a5")).add(ByteBufferUtil.bytes(85)).build();
         Composite finish = type.builder().add(ByteBufferUtil.bytes("a5")).build().end();
-        cfs.metric.sstablesPerReadHistogram.cf.clear();
+        ((ClearableHistogram)cfs.metric.sstablesPerReadHistogram.cf).clear();
         ColumnFamily cf = cfs.getColumnFamily(key, start, finish, false, 1000, System.currentTimeMillis());
         int colCount = 0;
         for (Cell c : cf)
             colCount++;
         assertEquals(2, colCount);
-        assertEquals(2, cfs.metric.sstablesPerReadHistogram.cf.max(), 0.1);
+        assertEquals(2, cfs.metric.sstablesPerReadHistogram.cf.getSnapshot().getMax(), 0.1);
     }
 
     private void validateSliceLarge(ColumnFamilyStore cfStore) throws IOException
diff --git a/test/unit/org/apache/cassandra/db/MmapFileTest.java b/test/unit/org/apache/cassandra/db/MmapFileTest.java
new file mode 100644
index 0000000..0c67ff7
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/MmapFileTest.java
@@ -0,0 +1,163 @@
+/*
+ * 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.db;
+
+import java.io.File;
+import java.io.RandomAccessFile;
+import java.lang.management.ManagementFactory;
+import java.nio.MappedByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.StandardOpenOption;
+import javax.management.MBeanServer;
+import javax.management.ObjectName;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import sun.nio.ch.DirectBuffer;
+
+public class MmapFileTest
+{
+    /**
+     * Verifies that {@link sun.misc.Cleaner} works and that mmap'd files can be deleted.
+     */
+    @Test
+    public void testMmapFile() throws Exception
+    {
+        ObjectName bpmName = new ObjectName("java.nio:type=BufferPool,name=mapped");
+
+        MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
+        Long mmapCount = (Long) mbs.getAttribute(bpmName, "Count");
+        Long mmapMemoryUsed = (Long) mbs.getAttribute(bpmName, "MemoryUsed");
+
+        Assert.assertEquals("# of mapped buffers should be 0", Long.valueOf(0L), mmapCount);
+        Assert.assertEquals("amount of mapped memory should be 0", Long.valueOf(0L), mmapMemoryUsed);
+
+        File f1 = File.createTempFile("MmapFileTest1", ".bin");
+        File f2 = File.createTempFile("MmapFileTest2", ".bin");
+        File f3 = File.createTempFile("MmapFileTest2", ".bin");
+
+        try
+        {
+            int size = 1024 * 1024;
+
+            try (RandomAccessFile raf = new RandomAccessFile(f1, "rw"))
+            {
+                raf.setLength(size);
+            }
+
+            try (RandomAccessFile raf = new RandomAccessFile(f2, "rw"))
+            {
+                raf.setLength(size);
+            }
+
+            try (RandomAccessFile raf = new RandomAccessFile(f3, "rw"))
+            {
+                raf.setLength(size);
+            }
+
+            try (FileChannel channel = FileChannel.open(f1.toPath(), StandardOpenOption.WRITE, StandardOpenOption.READ))
+            {
+                MappedByteBuffer buffer = channel.map(FileChannel.MapMode.READ_WRITE, 0, size);
+
+                mmapCount = (Long) mbs.getAttribute(bpmName, "Count");
+                mmapMemoryUsed = (Long) mbs.getAttribute(bpmName, "MemoryUsed");
+                Assert.assertEquals("mapped buffers don't work?", Long.valueOf(1L), mmapCount);
+                Assert.assertTrue("mapped buffers don't work?", mmapMemoryUsed >= size);
+
+                Assert.assertTrue(buffer.isDirect());
+
+                buffer.putInt(42);
+                buffer.putInt(42);
+                buffer.putInt(42);
+                buffer.putInt(42);
+                buffer.putInt(42);
+
+                ((DirectBuffer) buffer).cleaner().clean();
+            }
+
+            mmapCount = (Long) mbs.getAttribute(bpmName, "Count");
+            mmapMemoryUsed = (Long) mbs.getAttribute(bpmName, "MemoryUsed");
+            Assert.assertEquals("# of mapped buffers should be 0", Long.valueOf(0L), mmapCount);
+            Assert.assertEquals("amount of mapped memory should be 0", Long.valueOf(0L), mmapMemoryUsed);
+
+            try (FileChannel channel = FileChannel.open(f2.toPath(), StandardOpenOption.WRITE, StandardOpenOption.READ))
+            {
+                MappedByteBuffer buffer = channel.map(FileChannel.MapMode.READ_WRITE, 0, size);
+
+                // # of mapped buffers is == 1 here - seems that previous direct buffer for 'f1' is deallocated now
+
+                mmapCount = (Long) mbs.getAttribute(bpmName, "Count");
+                mmapMemoryUsed = (Long) mbs.getAttribute(bpmName, "MemoryUsed");
+                Assert.assertEquals("mapped buffers don't work?", Long.valueOf(1L), mmapCount);
+                Assert.assertTrue("mapped buffers don't work?", mmapMemoryUsed >= size);
+
+                Assert.assertTrue(buffer.isDirect());
+
+                buffer.putInt(42);
+                buffer.putInt(42);
+                buffer.putInt(42);
+                buffer.putInt(42);
+                buffer.putInt(42);
+
+                ((DirectBuffer) buffer).cleaner().clean();
+            }
+
+            mmapCount = (Long) mbs.getAttribute(bpmName, "Count");
+            mmapMemoryUsed = (Long) mbs.getAttribute(bpmName, "MemoryUsed");
+            Assert.assertEquals("# of mapped buffers should be 0", Long.valueOf(0L), mmapCount);
+            Assert.assertEquals("amount of mapped memory should be 0", Long.valueOf(0L), mmapMemoryUsed);
+
+            try (FileChannel channel = FileChannel.open(f3.toPath(), StandardOpenOption.WRITE, StandardOpenOption.READ))
+            {
+                MappedByteBuffer buffer = channel.map(FileChannel.MapMode.READ_WRITE, 0, size);
+
+                mmapCount = (Long) mbs.getAttribute(bpmName, "Count");
+                mmapMemoryUsed = (Long) mbs.getAttribute(bpmName, "MemoryUsed");
+                Assert.assertEquals("mapped buffers don't work?", Long.valueOf(1L), mmapCount);
+                Assert.assertTrue("mapped buffers don't work?", mmapMemoryUsed >= size);
+
+                Assert.assertTrue(buffer.isDirect());
+
+                buffer.putInt(42);
+                buffer.putInt(42);
+                buffer.putInt(42);
+                buffer.putInt(42);
+                buffer.putInt(42);
+
+                ((DirectBuffer) buffer).cleaner().clean();
+            }
+
+            mmapCount = (Long) mbs.getAttribute(bpmName, "Count");
+            mmapMemoryUsed = (Long) mbs.getAttribute(bpmName, "MemoryUsed");
+            Assert.assertEquals("# of mapped buffers should be 0", Long.valueOf(0L), mmapCount);
+            Assert.assertEquals("amount of mapped memory should be 0", Long.valueOf(0L), mmapMemoryUsed);
+
+            Assert.assertTrue(f1.delete());
+            Assert.assertTrue(f2.delete());
+            Assert.assertTrue(f3.delete());
+        }
+        finally
+        {
+            Runtime.getRuntime().gc();
+            f1.delete();
+            f2.delete();
+            f3.delete();
+        }
+    }
+}
diff --git a/test/unit/org/apache/cassandra/db/MultitableTest.java b/test/unit/org/apache/cassandra/db/MultitableTest.java
index cc11163..fd04b76 100644
--- a/test/unit/org/apache/cassandra/db/MultitableTest.java
+++ b/test/unit/org/apache/cassandra/db/MultitableTest.java
@@ -20,34 +20,56 @@
  *
  */
 
-import org.apache.cassandra.Util;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import static org.apache.cassandra.db.KeyspaceTest.assertColumns;
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.Util;
 import static org.apache.cassandra.Util.column;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 
-public class MultitableTest extends SchemaLoader
+public class MultitableTest
 {
+    private static final String KEYSPACE1 = "MultitableTest1";
+    private static final String KEYSPACE2 = "MultitableTest2";
+    private static final String CF1 = "Standard1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF1));
+        SchemaLoader.createKeyspace(KEYSPACE2,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE2, CF1));
+    }
+
     @Test
     public void testSameCFs()
     {
-        Keyspace keyspace1 = Keyspace.open("Keyspace1");
-        Keyspace keyspace2 = Keyspace.open("Keyspace2");
+        Keyspace keyspace1 = Keyspace.open(KEYSPACE1);
+        Keyspace keyspace2 = Keyspace.open(KEYSPACE2);
 
         Mutation rm;
         DecoratedKey dk = Util.dk("keymulti");
         ColumnFamily cf;
 
-        cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf.addColumn(column("col1", "val1", 1L));
-        rm = new Mutation("Keyspace1", dk.getKey(), cf);
-        rm.apply();
+        rm = new Mutation(KEYSPACE1, dk.getKey(), cf);
+        rm.applyUnsafe();
 
-        cf = ArrayBackedSortedColumns.factory.create("Keyspace2", "Standard1");
+        cf = ArrayBackedSortedColumns.factory.create(KEYSPACE2, "Standard1");
         cf.addColumn(column("col2", "val2", 1L));
-        rm = new Mutation("Keyspace2", dk.getKey(), cf);
-        rm.apply();
+        rm = new Mutation(KEYSPACE2, dk.getKey(), cf);
+        rm.applyUnsafe();
 
         keyspace1.getColumnFamilyStore("Standard1").forceBlockingFlush();
         keyspace2.getColumnFamilyStore("Standard1").forceBlockingFlush();
diff --git a/test/unit/org/apache/cassandra/db/NameSortTest.java b/test/unit/org/apache/cassandra/db/NameSortTest.java
index 6bd71c7..c4361d8 100644
--- a/test/unit/org/apache/cassandra/db/NameSortTest.java
+++ b/test/unit/org/apache/cassandra/db/NameSortTest.java
@@ -26,12 +26,32 @@
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
-public class NameSortTest extends SchemaLoader
+public class NameSortTest
 {
+    private static final String KEYSPACE1 = "NameSortTest";
+    private static final String CF = "Standard1";
+    private static final String CFSUPER = "Super1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF),
+                                    SchemaLoader.superCFMD(KEYSPACE1, CFSUPER, LongType.instance));
+    }
+
     @Test
     public void testNameSort1() throws IOException
     {
@@ -55,7 +75,7 @@
 
     private void testNameSort(int N) throws IOException
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
 
         for (int i = 0; i < N; ++i)
         {
@@ -66,7 +86,7 @@
             for (int j = 0; j < 8; ++j)
             {
                 ByteBuffer bytes = j % 2 == 0 ? ByteBufferUtil.bytes("a") : ByteBufferUtil.bytes("b");
-                rm = new Mutation("Keyspace1", key);
+                rm = new Mutation(KEYSPACE1, key);
                 rm.add("Standard1", Util.cellname("Cell-" + j), bytes, j);
                 rm.applyUnsafe();
             }
@@ -74,11 +94,11 @@
             // super
             for (int j = 0; j < 8; ++j)
             {
-                rm = new Mutation("Keyspace1", key);
+                rm = new Mutation(KEYSPACE1, key);
                 for (int k = 0; k < 4; ++k)
                 {
                     String value = (j + k) % 2 == 0 ? "a" : "b";
-                    addMutation(rm, "Super1", "SuperColumn-" + j, k, value, k);
+                    addMutation(rm, CFSUPER, "SuperColumn-" + j, k, value, k);
                 }
                 rm.applyUnsafe();
             }
@@ -87,7 +107,7 @@
         validateNameSort(keyspace, N);
 
         keyspace.getColumnFamilyStore("Standard1").forceBlockingFlush();
-        keyspace.getColumnFamilyStore("Super1").forceBlockingFlush();
+        keyspace.getColumnFamilyStore(CFSUPER).forceBlockingFlush();
         validateNameSort(keyspace, N);
     }
 
diff --git a/test/unit/org/apache/cassandra/db/NativeCellTest.java b/test/unit/org/apache/cassandra/db/NativeCellTest.java
index 70b7b87..4145a91 100644
--- a/test/unit/org/apache/cassandra/db/NativeCellTest.java
+++ b/test/unit/org/apache/cassandra/db/NativeCellTest.java
@@ -32,6 +32,7 @@
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.composites.CellNameType;
@@ -128,6 +129,8 @@
         {
             throw new AssertionError();
         }
+        // TODO: CounterId accesses SystemKespace to get local host ID, so need to mark as daemon initialized
+        DatabaseDescriptor.setDaemonInitialized();
     }
 
     @Test
diff --git a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
index 3292422..bff0ddf 100644
--- a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
+++ b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
@@ -18,7 +18,6 @@
 */
 package org.apache.cassandra.db;
 
-import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -27,18 +26,22 @@
 import java.util.SortedSet;
 import java.util.TreeSet;
 import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
 
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Iterators;
+
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.*;
 import org.apache.cassandra.Util;
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.IndexType;
 import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
 import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.db.compaction.LeveledCompactionStrategy;
 import org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.composites.CellNames;
@@ -51,22 +54,33 @@
 import org.apache.cassandra.db.index.SecondaryIndex;
 import org.apache.cassandra.db.index.SecondaryIndexSearcher;
 import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.db.marshal.IntegerType;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.concurrent.OpOrder;
-import org.apache.cassandra.utils.memory.MemtableAllocator;
 
 import static org.apache.cassandra.Util.dk;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-public class RangeTombstoneTest extends SchemaLoader
+public class RangeTombstoneTest
 {
-    private static final String KSNAME = "Keyspace1";
+    private static final String KSNAME = "RangeTombstoneTest";
     private static final String CFNAME = "StandardInteger1";
 
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KSNAME,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KSNAME, CFNAME, IntegerType.instance));
+    }
+
     @Test
     public void simpleQueryWithRangeTombstoneTest() throws Exception
     {
@@ -81,25 +95,25 @@
         rm = new Mutation(KSNAME, ByteBufferUtil.bytes(key));
         for (int i = 0; i < 40; i += 2)
             add(rm, i, 0);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         rm = new Mutation(KSNAME, ByteBufferUtil.bytes(key));
         cf = rm.addOrGet(CFNAME);
         delete(cf, 10, 22, 1);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         rm = new Mutation(KSNAME, ByteBufferUtil.bytes(key));
         for (int i = 1; i < 40; i += 2)
             add(rm, i, 2);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         rm = new Mutation(KSNAME, ByteBufferUtil.bytes(key));
         cf = rm.addOrGet(CFNAME);
         delete(cf, 19, 27, 3);
-        rm.apply();
+        rm.applyUnsafe();
         // We don't flush to test with both a range tomsbtone in memtable and in sstable
 
         // Queries by name
@@ -113,17 +127,17 @@
         cf = cfs.getColumnFamily(QueryFilter.getNamesFilter(dk(key), CFNAME, columns, System.currentTimeMillis()));
 
         for (int i : live)
-            assert isLive(cf, cf.getColumn(b(i))) : "Cell " + i + " should be live";
+            assertTrue("Cell " + i + " should be live", isLive(cf, cf.getColumn(b(i))));
         for (int i : dead)
-            assert !isLive(cf, cf.getColumn(b(i))) : "Cell " + i + " shouldn't be live";
+            assertTrue("Cell " + i + " shouldn't be live", !isLive(cf, cf.getColumn(b(i))));
 
         // Queries by slices
         cf = cfs.getColumnFamily(QueryFilter.getSliceFilter(dk(key), CFNAME, b(7), b(30), false, Integer.MAX_VALUE, System.currentTimeMillis()));
 
         for (int i : new int[]{ 7, 8, 9, 11, 13, 15, 17, 28, 29, 30 })
-            assert isLive(cf, cf.getColumn(b(i))) : "Cell " + i + " should be live";
+            assertTrue("Cell " + i + " should be live", isLive(cf, cf.getColumn(b(i))));
         for (int i : new int[]{ 10, 12, 14, 16, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27 })
-            assert !isLive(cf, cf.getColumn(b(i))) : "Cell " + i + " shouldn't be live";
+            assertTrue("Cell " + i + " shouldn't be live", !isLive(cf, cf.getColumn(b(i))));
     }
 
     @Test
@@ -141,17 +155,17 @@
         rm = new Mutation(KSNAME, ByteBufferUtil.bytes(key));
         for (int i = 0; i < 40; i += 2)
             add(rm, i, 0);
-        rm.apply();
+        rm.applyUnsafe();
 
         rm = new Mutation(KSNAME, ByteBufferUtil.bytes(key));
         cf = rm.addOrGet(CFNAME);
         delete(cf, 5, 10, 1);
-        rm.apply();
+        rm.applyUnsafe();
 
         rm = new Mutation(KSNAME, ByteBufferUtil.bytes(key));
         cf = rm.addOrGet(CFNAME);
         delete(cf, 15, 20, 2);
-        rm.apply();
+        rm.applyUnsafe();
 
         cf = cfs.getColumnFamily(QueryFilter.getSliceFilter(dk(key), CFNAME, b(11), b(14), false, Integer.MAX_VALUE, System.currentTimeMillis()));
         Collection<RangeTombstone> rt = rangeTombstones(cf);
@@ -322,7 +336,7 @@
     }
 
     @Test
-    public void test7810() throws ExecutionException, InterruptedException, IOException
+    public void test7810() throws ExecutionException, InterruptedException
     {
         Keyspace ks = Keyspace.open(KSNAME);
         ColumnFamilyStore cfs = ks.getColumnFamilyStore(CFNAME);
@@ -370,7 +384,7 @@
     }
 
     @Test
-    public void test7808_2() throws ExecutionException, InterruptedException, IOException
+    public void test7808_2() throws ExecutionException, InterruptedException
     {
         Keyspace ks = Keyspace.open(KSNAME);
         ColumnFamilyStore cfs = ks.getColumnFamilyStore(CFNAME);
@@ -414,46 +428,46 @@
         rm = new Mutation(KSNAME, ByteBufferUtil.bytes(key));
         for (int i = 0; i < 20; i++)
             add(rm, i, 0);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         rm = new Mutation(KSNAME, ByteBufferUtil.bytes(key));
         cf = rm.addOrGet(CFNAME);
         delete(cf, 5, 15, 1);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         rm = new Mutation(KSNAME, ByteBufferUtil.bytes(key));
         cf = rm.addOrGet(CFNAME);
         delete(cf, 5, 10, 1);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         rm = new Mutation(KSNAME, ByteBufferUtil.bytes(key));
         cf = rm.addOrGet(CFNAME);
         delete(cf, 5, 8, 2);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(dk(key), CFNAME, System.currentTimeMillis()));
 
         for (int i = 0; i < 5; i++)
-            assert isLive(cf, cf.getColumn(b(i))) : "Cell " + i + " should be live";
+            assertTrue("Cell " + i + " should be live", isLive(cf, cf.getColumn(b(i))));
         for (int i = 16; i < 20; i++)
-            assert isLive(cf, cf.getColumn(b(i))) : "Cell " + i + " should be live";
+            assertTrue("Cell " + i + " should be live", isLive(cf, cf.getColumn(b(i))));
         for (int i = 5; i <= 15; i++)
-            assert !isLive(cf, cf.getColumn(b(i))) : "Cell " + i + " shouldn't be live";
+            assertTrue("Cell " + i + " shouldn't be live", !isLive(cf, cf.getColumn(b(i))));
 
         // Compact everything and re-test
-        CompactionManager.instance.performMaximal(cfs);
+        CompactionManager.instance.performMaximal(cfs, false);
         cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(dk(key), CFNAME, System.currentTimeMillis()));
 
         for (int i = 0; i < 5; i++)
-            assert isLive(cf, cf.getColumn(b(i))) : "Cell " + i + " should be live";
+            assertTrue("Cell " + i + " should be live", isLive(cf, cf.getColumn(b(i))));
         for (int i = 16; i < 20; i++)
-            assert isLive(cf, cf.getColumn(b(i))) : "Cell " + i + " should be live";
+            assertTrue("Cell " + i + " should be live", isLive(cf, cf.getColumn(b(i))));
         for (int i = 5; i <= 15; i++)
-            assert !isLive(cf, cf.getColumn(b(i))) : "Cell " + i + " shouldn't be live";
+            assertTrue("Cell " + i + " shouldn't be live", !isLive(cf, cf.getColumn(b(i))));
     }
 
     @Test
@@ -469,22 +483,22 @@
 
         rm = new Mutation(KSNAME, ByteBufferUtil.bytes(key));
         add(rm, 2, 0);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         rm = new Mutation(KSNAME, ByteBufferUtil.bytes(key));
         // Deletes everything but without being a row tombstone
         delete(rm.addOrGet(CFNAME), 0, 10, 1);
         add(rm, 1, 2);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         // Get the last value of the row
         cf = cfs.getColumnFamily(QueryFilter.getSliceFilter(dk(key), CFNAME, Composites.EMPTY, Composites.EMPTY, true, 1, System.currentTimeMillis()));
 
-        assert !cf.isEmpty();
+        assertFalse(cf.isEmpty());
         int last = i(cf.getSortedColumns().iterator().next().name());
-        assert last == 1 : "Last column should be column 1 since column 2 has been deleted";
+        assertEquals("Last column should be column 1 since column 2 has been deleted", 1, last);
     }
 
     @Test
@@ -508,38 +522,80 @@
         Mutation rm = new Mutation(KSNAME, key);
         for (int i = 0; i < 10; i += 2)
             add(rm, i, 0);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         rm = new Mutation(KSNAME, key);
         ColumnFamily cf = rm.addOrGet(CFNAME);
         for (int i = 0; i < 10; i += 2)
             delete(cf, 0, 7, 0);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         // there should be 2 sstables
         assertEquals(2, cfs.getSSTables().size());
 
         // compact down to single sstable
-        CompactionManager.instance.performMaximal(cfs);
+        CompactionManager.instance.performMaximal(cfs, false);
         assertEquals(1, cfs.getSSTables().size());
 
         // test the physical structure of the sstable i.e. rt & columns on disk
         SSTableReader sstable = cfs.getSSTables().iterator().next();
-        OnDiskAtomIterator iter = sstable.getScanner().next();
-        int cnt = 0;
-        // after compaction, the first element should be an RT followed by the remaining non-deleted columns
-        while(iter.hasNext())
+        try(ISSTableScanner scanner = sstable.getScanner())
         {
-            OnDiskAtom atom = iter.next();
-            if (cnt == 0)
-                assertTrue(atom instanceof RangeTombstone);
-            if (cnt > 0)
-                assertTrue(atom instanceof Cell);
-            cnt++;
+            OnDiskAtomIterator iter = scanner.next();
+            int cnt = 0;
+            // after compaction, the first element should be an RT followed by the remaining non-deleted columns
+            while (iter.hasNext())
+            {
+                OnDiskAtom atom = iter.next();
+                if (cnt == 0)
+                    assertTrue(atom instanceof RangeTombstone);
+                if (cnt > 0)
+                    assertTrue(atom instanceof Cell);
+                cnt++;
+            }
+            assertEquals(2, cnt);
         }
-        assertEquals(2, cnt);
+    }
+
+    @Test
+    public void testCompactionOfRangeTombstonesCoveredByRowTombstone() throws Exception
+    {
+        long testTimeStamp = 1451606400L; // 01/01/2016 : 00:00:00 GMT
+        Keyspace table = Keyspace.open(KSNAME);
+        ColumnFamilyStore cfs = table.getColumnFamilyStore(CFNAME);
+        ByteBuffer key = ByteBufferUtil.bytes("k4");
+
+        // remove any existing sstables before starting
+        cfs.truncateBlocking();
+        cfs.disableAutoCompaction();
+        cfs.setCompactionStrategyClass(LeveledCompactionStrategy.class.getCanonicalName());
+
+        Mutation rm = new Mutation(KSNAME, key);
+        for (int i = 1; i < 11; i += 2, testTimeStamp += i * 10)
+            add(rm, i, testTimeStamp);
+        rm.apply();
+        cfs.forceBlockingFlush();
+
+        rm = new Mutation(KSNAME, key);
+        ColumnFamily cf = rm.addOrGet(CFNAME);
+
+        // Write the covering row tombstone
+        cf.delete(new DeletionTime(++testTimeStamp, (int) testTimeStamp));
+
+        // Create range tombstones covered by row tombstone above.
+        for (int i = 1; i < 11; i += 2, testTimeStamp -= i * 5)
+            delete(cf, 0, 7, testTimeStamp);
+        rm.apply();
+        cfs.forceBlockingFlush();
+
+        // there should be 2 sstables
+        assertEquals(2, cfs.getSSTables().size());
+
+        // compact down to nothing
+        CompactionManager.instance.performMaximal(cfs, false);
+        assertEquals(0, cfs.getSSTables().size());
     }
 
     @Test
@@ -557,7 +613,10 @@
         {
             ColumnDefinition cd = new ColumnDefinition(cfs.metadata, indexedColumnName, Int32Type.instance, null, ColumnDefinition.Kind.REGULAR);
             cd.setIndex("test_index", IndexType.CUSTOM, ImmutableMap.of(SecondaryIndex.CUSTOM_INDEX_OPTION_NAME, TestIndex.class.getName()));
-            cfs.indexManager.addIndexedColumn(cd);
+            Future<?> rebuild = cfs.indexManager.addIndexedColumn(cd);
+            // If rebuild there is, wait for the rebuild to finish so it doesn't race with the following insertions
+            if (rebuild != null)
+                rebuild.get();
         }
 
         TestIndex index = ((TestIndex)cfs.indexManager.getIndexForColumn(indexedColumnName));
@@ -565,18 +624,18 @@
 
         Mutation rm = new Mutation(KSNAME, key);
         add(rm, 1, 0);
-        rm.apply();
+        rm.applyUnsafe();
 
         // add a RT which hides the column we just inserted
         rm = new Mutation(KSNAME, key);
         ColumnFamily cf = rm.addOrGet(CFNAME);
         delete(cf, 0, 1, 1);
-        rm.apply();
+        rm.applyUnsafe();
 
         // now re-insert that column
         rm = new Mutation(KSNAME, key);
         add(rm, 1, 2);
-        rm.apply();
+        rm.applyUnsafe();
 
         cfs.forceBlockingFlush();
 
@@ -600,7 +659,11 @@
         {
             ColumnDefinition cd = ColumnDefinition.regularDef(cfs.metadata, indexedColumnName, cfs.getComparator().asAbstractType(), 0)
                                                   .setIndex("test_index", IndexType.CUSTOM, ImmutableMap.of(SecondaryIndex.CUSTOM_INDEX_OPTION_NAME, TestIndex.class.getName()));
-            cfs.indexManager.addIndexedColumn(cd);
+            Future<?> rebuild = cfs.indexManager.addIndexedColumn(cd);
+            // If rebuild there is, wait for the rebuild to finish so it doesn't race with the following insertions
+            if (rebuild != null)
+                rebuild.get();
+
         }
 
         TestIndex index = ((TestIndex)cfs.indexManager.getIndexForColumn(indexedColumnName));
@@ -609,20 +672,20 @@
         Mutation rm = new Mutation(KSNAME, key);
         for (int i = 0; i < 10; i++)
             add(rm, i, 0);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         rm = new Mutation(KSNAME, key);
         ColumnFamily cf = rm.addOrGet(CFNAME);
         for (int i = 0; i < 10; i += 2)
             delete(cf, 0, 7, 0);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         // We should have indexed 1 column
         assertEquals(1, index.inserts.size());
 
-        CompactionManager.instance.performMaximal(cfs);
+        CompactionManager.instance.performMaximal(cfs, false);
 
         // compacted down to single sstable
         assertEquals(1, cfs.getSSTables().size());
diff --git a/test/unit/org/apache/cassandra/db/ReadMessageTest.java b/test/unit/org/apache/cassandra/db/ReadMessageTest.java
index 6d19d3a..34f25a1 100644
--- a/test/unit/org/apache/cassandra/db/ReadMessageTest.java
+++ b/test/unit/org/apache/cassandra/db/ReadMessageTest.java
@@ -25,26 +25,51 @@
 import java.util.SortedSet;
 import java.util.TreeSet;
 
+import com.google.common.base.Predicate;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
-import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.db.commitlog.CommitLogTestReplayer;
 import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.db.filter.NamesQueryFilter;
 import org.apache.cassandra.db.filter.SliceQueryFilter;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
 
-public class ReadMessageTest extends SchemaLoader
+public class ReadMessageTest
 {
+    private static final String KEYSPACE1 = "ReadMessageTest1";
+    private static final String KEYSPACENOCOMMIT = "ReadMessageTest_NoCommit";
+    private static final String CF = "Standard1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF));
+        SchemaLoader.createKeyspace(KEYSPACENOCOMMIT,
+                                    false,
+                                    true,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACENOCOMMIT, CF));
+    }
+
     @Test
     public void testMakeReadMessage() throws IOException
     {
-        CellNameType type = Keyspace.open("Keyspace1").getColumnFamilyStore("Standard1").getComparator();
+        CellNameType type = Keyspace.open(KEYSPACE1).getColumnFamilyStore("Standard1").getComparator();
 
         SortedSet<CellName> colList = new TreeSet<CellName>(type);
         colList.add(Util.cellname("col1"));
@@ -54,15 +79,15 @@
         DecoratedKey dk = Util.dk("row1");
         long ts = System.currentTimeMillis();
 
-        rm = new SliceByNamesReadCommand("Keyspace1", dk.getKey(), "Standard1", ts, new NamesQueryFilter(colList));
+        rm = new SliceByNamesReadCommand(KEYSPACE1, dk.getKey(), "Standard1", ts, new NamesQueryFilter(colList));
         rm2 = serializeAndDeserializeReadMessage(rm);
         assert rm2.toString().equals(rm.toString());
 
-        rm = new SliceFromReadCommand("Keyspace1", dk.getKey(), "Standard1", ts, new SliceQueryFilter(Composites.EMPTY, Composites.EMPTY, true, 2));
+        rm = new SliceFromReadCommand(KEYSPACE1, dk.getKey(), "Standard1", ts, new SliceQueryFilter(Composites.EMPTY, Composites.EMPTY, true, 2));
         rm2 = serializeAndDeserializeReadMessage(rm);
         assert rm2.toString().equals(rm.toString());
 
-        rm = new SliceFromReadCommand("Keyspace1", dk.getKey(), "Standard1", ts, new SliceQueryFilter(Util.cellname("a"), Util.cellname("z"), true, 5));
+        rm = new SliceFromReadCommand(KEYSPACE1, dk.getKey(), "Standard1", ts, new SliceQueryFilter(Util.cellname("a"), Util.cellname("z"), true, 5));
         rm2 = serializeAndDeserializeReadMessage(rm);
         assert rm2.toString().equals(rm.toString());
     }
@@ -81,17 +106,17 @@
     @Test
     public void testGetColumn()
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         CellNameType type = keyspace.getColumnFamilyStore("Standard1").getComparator();
         Mutation rm;
         DecoratedKey dk = Util.dk("key1");
 
         // add data
-        rm = new Mutation("Keyspace1", dk.getKey());
+        rm = new Mutation(KEYSPACE1, dk.getKey());
         rm.add("Standard1", Util.cellname("Column1"), ByteBufferUtil.bytes("abcd"), 0);
         rm.apply();
 
-        ReadCommand command = new SliceByNamesReadCommand("Keyspace1", dk.getKey(), "Standard1", System.currentTimeMillis(), new NamesQueryFilter(FBUtilities.singleton(Util.cellname("Column1"), type)));
+        ReadCommand command = new SliceByNamesReadCommand(KEYSPACE1, dk.getKey(), "Standard1", System.currentTimeMillis(), new NamesQueryFilter(FBUtilities.singleton(Util.cellname("Column1"), type)));
         Row row = command.getRow(keyspace);
         Cell col = row.cf.getColumn(Util.cellname("Column1"));
         assertEquals(col.value(), ByteBuffer.wrap("abcd".getBytes()));
@@ -100,89 +125,36 @@
     @Test
     public void testNoCommitLog() throws Exception
     {
-        Mutation rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("row"));
+        Mutation rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("row"));
         rm.add("Standard1", Util.cellname("commit1"), ByteBufferUtil.bytes("abcd"), 0);
         rm.apply();
 
-        rm = new Mutation("NoCommitlogSpace", ByteBufferUtil.bytes("row"));
+        rm = new Mutation(KEYSPACENOCOMMIT, ByteBufferUtil.bytes("row"));
         rm.add("Standard1", Util.cellname("commit2"), ByteBufferUtil.bytes("abcd"), 0);
         rm.apply();
 
+        Checker checker = new Checker();
+        CommitLogTestReplayer.examineCommitLog(checker);
+
+        assertTrue(checker.commitLogMessageFound);
+        assertFalse(checker.noCommitLogMessageFound);
+    }
+
+    static class Checker implements Predicate<Mutation>
+    {
         boolean commitLogMessageFound = false;
         boolean noCommitLogMessageFound = false;
 
-        File commitLogDir = new File(DatabaseDescriptor.getCommitLogLocation());
-
-        byte[] commitBytes = "commit".getBytes("UTF-8");
-
-        for(String filename : commitLogDir.list())
+        public boolean apply(Mutation mutation)
         {
-            BufferedInputStream is = null;
-            try
+            for (ColumnFamily cf : mutation.getColumnFamilies())
             {
-                is = new BufferedInputStream(new FileInputStream(commitLogDir.getAbsolutePath()+File.separator+filename));
-
-                if (!isEmptyCommitLog(is))
-                {
-                    while (findPatternInStream(commitBytes, is))
-                    {
-                        char c = (char)is.read();
-
-                        if (c == '1')
-                            commitLogMessageFound = true;
-                        else if (c == '2')
-                            noCommitLogMessageFound = true;
-                    }
-                }
+                if (cf.getColumn(Util.cellname("commit1")) != null)
+                    commitLogMessageFound = true;
+                if (cf.getColumn(Util.cellname("commit2")) != null)
+                    noCommitLogMessageFound = true;
             }
-            finally
-            {
-                if (is != null)
-                    is.close();
-            }
+            return true;
         }
-
-        assertTrue(commitLogMessageFound);
-        assertFalse(noCommitLogMessageFound);
-    }
-
-    private boolean isEmptyCommitLog(BufferedInputStream is) throws IOException
-    {
-        DataInputStream dis = new DataInputStream(is);
-        byte[] lookahead = new byte[100];
-
-        dis.mark(100);
-        dis.readFully(lookahead);
-        dis.reset();
-
-        for (int i = 0; i < 100; i++)
-        {
-            if (lookahead[i] != 0)
-                return false;
-        }
-
-        return true;
-    }
-
-    private boolean findPatternInStream(byte[] pattern, InputStream is) throws IOException
-    {
-        int patternOffset = 0;
-
-        int b = is.read();
-        while (b != -1)
-        {
-            if (pattern[patternOffset] == ((byte) b))
-            {
-                patternOffset++;
-                if (patternOffset == pattern.length)
-                    return true;
-            }
-            else
-                patternOffset = 0;
-
-            b = is.read();
-        }
-
-        return false;
     }
 }
diff --git a/test/unit/org/apache/cassandra/db/RecoveryManager2Test.java b/test/unit/org/apache/cassandra/db/RecoveryManager2Test.java
index ede3e9b..3beb28e 100644
--- a/test/unit/org/apache/cassandra/db/RecoveryManager2Test.java
+++ b/test/unit/org/apache/cassandra/db/RecoveryManager2Test.java
@@ -21,28 +21,83 @@
  */
 
 
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+
+import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static org.apache.cassandra.Util.column;
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.config.ParameterizedClass;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.commitlog.CommitLog;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.compress.DeflateCompressor;
+import org.apache.cassandra.io.compress.LZ4Compressor;
+import org.apache.cassandra.io.compress.SnappyCompressor;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
-public class RecoveryManager2Test extends SchemaLoader
+@RunWith(Parameterized.class)
+public class RecoveryManager2Test
 {
     private static Logger logger = LoggerFactory.getLogger(RecoveryManager2Test.class);
 
+    private static final String KEYSPACE1 = "RecoveryManager2Test";
+    private static final String CF_STANDARD1 = "Standard1";
+    private static final String CF_STANDARD2 = "Standard2";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD2));
+    }
+
+    public RecoveryManager2Test(ParameterizedClass commitLogCompression)
+    {
+        DatabaseDescriptor.setCommitLogCompression(commitLogCompression);
+    }
+
+    @Before
+    public void setUp() throws IOException
+    {
+        CommitLog.instance.resetUnsafe(true);
+    }
+
+    @Parameters()
+    public static Collection<Object[]> generateData()
+    {
+        return Arrays.asList(new Object[][] {
+                { null }, // No compression
+                { new ParameterizedClass(LZ4Compressor.class.getName(), Collections.<String, String>emptyMap()) },
+                { new ParameterizedClass(SnappyCompressor.class.getName(), Collections.<String, String>emptyMap()) },
+                { new ParameterizedClass(DeflateCompressor.class.getName(), Collections.<String, String>emptyMap()) } });
+    }
+
     @Test
     /* test that commit logs do not replay flushed data */
     public void testWithFlush() throws Exception
     {
         // Flush everything that may be in the commit log now to start fresh
-        FBUtilities.waitOnFutures(Keyspace.open(Keyspace.SYSTEM_KS).flush());
+        FBUtilities.waitOnFutures(Keyspace.open(SystemKeyspace.NAME).flush());
 
         CompactionManager.instance.disableAutoCompaction();
 
@@ -55,7 +110,7 @@
             insertRow("Standard1", key);
         }
 
-        Keyspace keyspace1 = Keyspace.open("Keyspace1");
+        Keyspace keyspace1 = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore cfs = keyspace1.getColumnFamilyStore("Standard1");
         logger.debug("forcing flush");
         cfs.forceBlockingFlush();
@@ -63,16 +118,15 @@
         logger.debug("begin manual replay");
         // replay the commit log (nothing on Standard1 should be replayed since everything was flushed, so only the row on Standard2
         // will be replayed)
-        CommitLog.instance.resetUnsafe();
-        int replayed = CommitLog.instance.recover();
+        int replayed = CommitLog.instance.resetUnsafe(false);
         assert replayed == 1 : "Expecting only 1 replayed mutation, got " + replayed;
     }
 
     private void insertRow(String cfname, String key) 
     {
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", cfname);
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, cfname);
         cf.addColumn(column("col1", "val1", 1L));
-        Mutation rm = new Mutation("Keyspace1", ByteBufferUtil.bytes(key), cf);
+        Mutation rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes(key), cf);
         rm.apply();
     }
 }
diff --git a/test/unit/org/apache/cassandra/db/RecoveryManager3Test.java b/test/unit/org/apache/cassandra/db/RecoveryManager3Test.java
index c9bc86a..2dd7eae 100644
--- a/test/unit/org/apache/cassandra/db/RecoveryManager3Test.java
+++ b/test/unit/org/apache/cassandra/db/RecoveryManager3Test.java
@@ -23,38 +23,95 @@
 
 import java.io.File;
 import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
 
+import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.config.ParameterizedClass;
 import org.apache.cassandra.db.commitlog.CommitLog;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.compress.DeflateCompressor;
+import org.apache.cassandra.io.compress.LZ4Compressor;
+import org.apache.cassandra.io.compress.SnappyCompressor;
 import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.locator.SimpleStrategy;
 
 import static org.apache.cassandra.Util.column;
 import static org.apache.cassandra.db.KeyspaceTest.assertColumns;
 
-public class RecoveryManager3Test extends SchemaLoader
+@RunWith(Parameterized.class)
+public class RecoveryManager3Test
 {
+    private static final String KEYSPACE1 = "RecoveryManager3Test1";
+    private static final String CF_STANDARD1 = "Standard1";
+
+    private static final String KEYSPACE2 = "RecoveryManager3Test2";
+    private static final String CF_STANDARD3 = "Standard3";
+
+    public RecoveryManager3Test(ParameterizedClass commitLogCompression)
+    {
+        DatabaseDescriptor.setCommitLogCompression(commitLogCompression);
+    }
+
+    @Before
+    public void setUp() throws IOException
+    {
+        CommitLog.instance.resetUnsafe(true);
+    }
+
+    @Parameters()
+    public static Collection<Object[]> generateData()
+    {
+        return Arrays.asList(new Object[][] {
+                { null }, // No compression
+                { new ParameterizedClass(LZ4Compressor.class.getName(), Collections.<String, String>emptyMap()) },
+                { new ParameterizedClass(SnappyCompressor.class.getName(), Collections.<String, String>emptyMap()) },
+                { new ParameterizedClass(DeflateCompressor.class.getName(), Collections.<String, String>emptyMap()) } });
+    }
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1));
+        SchemaLoader.createKeyspace(KEYSPACE2,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE2, CF_STANDARD3));
+    }
+
     @Test
     public void testMissingHeader() throws IOException
     {
-        Keyspace keyspace1 = Keyspace.open("Keyspace1");
-        Keyspace keyspace2 = Keyspace.open("Keyspace2");
+        Keyspace keyspace1 = Keyspace.open(KEYSPACE1);
+        Keyspace keyspace2 = Keyspace.open(KEYSPACE2);
 
         Mutation rm;
         DecoratedKey dk = Util.dk("keymulti");
         ColumnFamily cf;
 
-        cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf.addColumn(column("col1", "val1", 1L));
-        rm = new Mutation("Keyspace1", dk.getKey(), cf);
+        rm = new Mutation(KEYSPACE1, dk.getKey(), cf);
         rm.apply();
 
-        cf = ArrayBackedSortedColumns.factory.create("Keyspace2", "Standard3");
+        cf = ArrayBackedSortedColumns.factory.create(KEYSPACE2, "Standard3");
         cf.addColumn(column("col2", "val2", 1L));
-        rm = new Mutation("Keyspace2", dk.getKey(), cf);
+        rm = new Mutation(KEYSPACE2, dk.getKey(), cf);
         rm.apply();
 
         keyspace1.getColumnFamilyStore("Standard1").clearUnsafe();
@@ -67,8 +124,7 @@
                 FileUtils.deleteWithConfirm(file);
         }
 
-        CommitLog.instance.resetUnsafe(); // disassociate segments from live CL
-        CommitLog.instance.recover();
+        CommitLog.instance.resetUnsafe(false); // disassociate segments from live CL
 
         assertColumns(Util.getColumnFamily(keyspace1, dk, "Standard1"), "col1");
         assertColumns(Util.getColumnFamily(keyspace2, dk, "Standard3"), "col2");
diff --git a/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java b/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java
index 460c267..5676b99 100644
--- a/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java
+++ b/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java
@@ -19,59 +19,108 @@
 package org.apache.cassandra.db;
 
 import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
 import java.util.Date;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
-import org.apache.cassandra.Util;
 import org.junit.Assert;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.Util;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.config.ParameterizedClass;
 import org.apache.cassandra.db.commitlog.CommitLog;
 import org.apache.cassandra.db.commitlog.CommitLogArchiver;
+import org.apache.cassandra.db.marshal.CounterColumnType;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.compress.DeflateCompressor;
+import org.apache.cassandra.io.compress.LZ4Compressor;
+import org.apache.cassandra.io.compress.SnappyCompressor;
+import org.apache.cassandra.locator.SimpleStrategy;
 
+import static org.apache.cassandra.Util.cellname;
 import static org.apache.cassandra.Util.column;
 import static org.apache.cassandra.db.KeyspaceTest.assertColumns;
-import static org.apache.cassandra.Util.cellname;
 
-@RunWith(OrderedJUnit4ClassRunner.class)
-public class RecoveryManagerTest extends SchemaLoader
+@RunWith(Parameterized.class)
+public class RecoveryManagerTest
 {
+    private static final String KEYSPACE1 = "RecoveryManagerTest1";
+    private static final String CF_STANDARD1 = "Standard1";
+    private static final String CF_COUNTER1 = "Counter1";
+
+    private static final String KEYSPACE2 = "RecoveryManagerTest2";
+    private static final String CF_STANDARD3 = "Standard3";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_COUNTER1).defaultValidator(CounterColumnType.instance));
+        SchemaLoader.createKeyspace(KEYSPACE2,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE2, CF_STANDARD3));
+    }
+
+    public RecoveryManagerTest(ParameterizedClass commitLogCompression)
+    {
+        DatabaseDescriptor.setCommitLogCompression(commitLogCompression);
+    }
+
+    @Parameters()
+    public static Collection<Object[]> generateData()
+    {
+        return Arrays.asList(new Object[][] {
+                { null }, // No compression
+                { new ParameterizedClass(LZ4Compressor.class.getName(), Collections.<String, String>emptyMap()) },
+                { new ParameterizedClass(SnappyCompressor.class.getName(), Collections.<String, String>emptyMap()) },
+                { new ParameterizedClass(DeflateCompressor.class.getName(), Collections.<String, String>emptyMap()) } });
+    }
+
     @Test
     public void testNothingToRecover() throws IOException
     {
-        CommitLog.instance.resetUnsafe();
-        CommitLog.instance.recover();
+        CommitLog.instance.resetUnsafe(true);
     }
 
     @Test
     public void testOne() throws IOException
     {
-        CommitLog.instance.resetUnsafe();
-        Keyspace keyspace1 = Keyspace.open("Keyspace1");
-        Keyspace keyspace2 = Keyspace.open("Keyspace2");
+        CommitLog.instance.resetUnsafe(true);
+        Keyspace keyspace1 = Keyspace.open(KEYSPACE1);
+        Keyspace keyspace2 = Keyspace.open(KEYSPACE2);
 
         Mutation rm;
         DecoratedKey dk = Util.dk("keymulti");
         ColumnFamily cf;
 
-        cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf.addColumn(column("col1", "val1", 1L));
-        rm = new Mutation("Keyspace1", dk.getKey(), cf);
+        rm = new Mutation(KEYSPACE1, dk.getKey(), cf);
         rm.apply();
 
-        cf = ArrayBackedSortedColumns.factory.create("Keyspace2", "Standard3");
+        cf = ArrayBackedSortedColumns.factory.create(KEYSPACE2, "Standard3");
         cf.addColumn(column("col2", "val2", 1L));
-        rm = new Mutation("Keyspace2", dk.getKey(), cf);
+        rm = new Mutation(KEYSPACE2, dk.getKey(), cf);
         rm.apply();
 
         keyspace1.getColumnFamilyStore("Standard1").clearUnsafe();
         keyspace2.getColumnFamilyStore("Standard3").clearUnsafe();
 
-        CommitLog.instance.resetUnsafe(); // disassociate segments from live CL
-        CommitLog.instance.recover();
+        CommitLog.instance.resetUnsafe(false); // disassociate segments from live CL
 
         assertColumns(Util.getColumnFamily(keyspace1, dk, "Standard1"), "col1");
         assertColumns(Util.getColumnFamily(keyspace2, dk, "Standard3"), "col2");
@@ -80,8 +129,8 @@
     @Test
     public void testRecoverCounter() throws IOException
     {
-        CommitLog.instance.resetUnsafe();
-        Keyspace keyspace1 = Keyspace.open("Keyspace1");
+        CommitLog.instance.resetUnsafe(true);
+        Keyspace keyspace1 = Keyspace.open(KEYSPACE1);
 
         Mutation rm;
         DecoratedKey dk = Util.dk("key");
@@ -89,16 +138,15 @@
 
         for (int i = 0; i < 10; ++i)
         {
-            cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Counter1");
+            cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Counter1");
             cf.addColumn(BufferCounterCell.createLocal(cellname("col"), 1L, 1L, Long.MIN_VALUE));
-            rm = new Mutation("Keyspace1", dk.getKey(), cf);
+            rm = new Mutation(KEYSPACE1, dk.getKey(), cf);
             rm.apply();
         }
 
         keyspace1.getColumnFamilyStore("Counter1").clearUnsafe();
 
-        CommitLog.instance.resetUnsafe(); // disassociate segments from live CL
-        CommitLog.instance.recover();
+        CommitLog.instance.resetUnsafe(false); // disassociate segments from live CL
 
         cf = Util.getColumnFamily(keyspace1, dk, "Counter1");
 
@@ -112,23 +160,22 @@
     @Test
     public void testRecoverPIT() throws Exception
     {
-        CommitLog.instance.resetUnsafe();
+        CommitLog.instance.resetUnsafe(true);
         Date date = CommitLogArchiver.format.parse("2112:12:12 12:12:12");
         long timeMS = date.getTime() - 5000;
 
-        Keyspace keyspace1 = Keyspace.open("Keyspace1");
+        Keyspace keyspace1 = Keyspace.open(KEYSPACE1);
         DecoratedKey dk = Util.dk("dkey");
         for (int i = 0; i < 10; ++i)
         {
             long ts = TimeUnit.MILLISECONDS.toMicros(timeMS + (i * 1000));
-            ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+            ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
             cf.addColumn(column("name-" + i, "value", ts));
-            Mutation rm = new Mutation("Keyspace1", dk.getKey(), cf);
+            Mutation rm = new Mutation(KEYSPACE1, dk.getKey(), cf);
             rm.apply();
         }
         keyspace1.getColumnFamilyStore("Standard1").clearUnsafe();
-        CommitLog.instance.resetUnsafe(); // disassociate segments from live CL
-        CommitLog.instance.recover();
+        CommitLog.instance.resetUnsafe(false); // disassociate segments from live CL
 
         ColumnFamily cf = Util.getColumnFamily(keyspace1, dk, "Standard1");
         Assert.assertEquals(6, cf.getColumnCount());
@@ -138,11 +185,11 @@
     @Test
     public void testRecoverPITUnordered() throws Exception
     {
-        CommitLog.instance.resetUnsafe();
+        CommitLog.instance.resetUnsafe(true);
         Date date = CommitLogArchiver.format.parse("2112:12:12 12:12:12");
         long timeMS = date.getTime();
 
-        Keyspace keyspace1 = Keyspace.open("Keyspace1");
+        Keyspace keyspace1 = Keyspace.open(KEYSPACE1);
         DecoratedKey dk = Util.dk("dkey");
 
         // Col 0 and 9 are the only ones to be recovered
@@ -154,9 +201,9 @@
             else
                 ts = TimeUnit.MILLISECONDS.toMicros(timeMS + (i * 1000));
 
-            ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+            ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
             cf.addColumn(column("name-" + i, "value", ts));
-            Mutation rm = new Mutation("Keyspace1", dk.getKey(), cf);
+            Mutation rm = new Mutation(KEYSPACE1, dk.getKey(), cf);
             rm.apply();
         }
 
@@ -164,8 +211,7 @@
         Assert.assertEquals(10, cf.getColumnCount());
 
         keyspace1.getColumnFamilyStore("Standard1").clearUnsafe();
-        CommitLog.instance.resetUnsafe(); // disassociate segments from live CL
-        CommitLog.instance.recover();
+        CommitLog.instance.resetUnsafe(false); // disassociate segments from live CL
 
         cf = Util.getColumnFamily(keyspace1, dk, "Standard1");
         Assert.assertEquals(2, cf.getColumnCount());
diff --git a/test/unit/org/apache/cassandra/db/RecoveryManagerTruncateTest.java b/test/unit/org/apache/cassandra/db/RecoveryManagerTruncateTest.java
index 817b8e9..769316f 100644
--- a/test/unit/org/apache/cassandra/db/RecoveryManagerTruncateTest.java
+++ b/test/unit/org/apache/cassandra/db/RecoveryManagerTruncateTest.java
@@ -22,78 +22,130 @@
 import static org.junit.Assert.*;
 
 import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+
+import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.config.ParameterizedClass;
 import org.apache.cassandra.db.commitlog.CommitLog;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.compress.DeflateCompressor;
+import org.apache.cassandra.io.compress.LZ4Compressor;
+import org.apache.cassandra.io.compress.SnappyCompressor;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 /**
  * Test for the truncate operation.
  */
-public class RecoveryManagerTruncateTest extends SchemaLoader
+@RunWith(Parameterized.class)
+public class RecoveryManagerTruncateTest
 {
+    private static final String KEYSPACE1 = "RecoveryManagerTruncateTest";
+    private static final String CF_STANDARD1 = "Standard1";
+    private static final String CF_STANDARD2 = "Standard2";
+
+    public RecoveryManagerTruncateTest(ParameterizedClass commitLogCompression)
+    {
+        DatabaseDescriptor.setCommitLogCompression(commitLogCompression);
+    }
+
+    @Before
+    public void setUp() throws IOException
+    {
+        CommitLog.instance.resetUnsafe(true);
+    }
+
+    @Parameters()
+    public static Collection<Object[]> generateData()
+    {
+        return Arrays.asList(new Object[][] {
+                { null }, // No compression
+                { new ParameterizedClass(LZ4Compressor.class.getName(), Collections.<String, String>emptyMap()) },
+                { new ParameterizedClass(SnappyCompressor.class.getName(), Collections.<String, String>emptyMap()) },
+                { new ParameterizedClass(DeflateCompressor.class.getName(), Collections.<String, String>emptyMap()) } });
+    }
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD2));
+    }
+
     @Test
     public void testTruncate() throws IOException
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_STANDARD1);
 
         Mutation rm;
         ColumnFamily cf;
 
         // add a single cell
-        cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, CF_STANDARD1);
         cf.addColumn(column("col1", "val1", 1L));
-        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("keymulti"), cf);
-        rm.apply();
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("keymulti"), cf);
+        rm.applyUnsafe();
         long time = System.currentTimeMillis();
 
         // Make sure data was written
-        assertNotNull(getFromTable(keyspace, "Standard1", "keymulti", "col1"));
+        assertNotNull(getFromTable(keyspace, CF_STANDARD1, "keymulti", "col1"));
 
         // and now truncate it
         cfs.truncateBlocking();
-        CommitLog.instance.resetUnsafe();
-        CommitLog.instance.recover();
+        CommitLog.instance.resetUnsafe(false);
 
         // and validate truncation.
-        assertNull(getFromTable(keyspace, "Standard1", "keymulti", "col1"));
+        assertNull(getFromTable(keyspace, CF_STANDARD1, "keymulti", "col1"));
         assertTrue(SystemKeyspace.getTruncatedAt(cfs.metadata.cfId) > time);
     }
 
     @Test
     public void testTruncatePointInTime() throws IOException
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_STANDARD1);
 
         Mutation rm;
         ColumnFamily cf;
 
         // add a single cell
-        cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, CF_STANDARD1);
         cf.addColumn(column("col2", "val1", 1L));
-        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("keymulti"), cf);
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("keymulti"), cf);
         rm.apply();
 
         // Make sure data was written
         long time = System.currentTimeMillis();
-        assertNotNull(getFromTable(keyspace, "Standard1", "keymulti", "col2"));
+        assertNotNull(getFromTable(keyspace, CF_STANDARD1, "keymulti", "col2"));
 
         // and now truncate it
         cfs.truncateBlocking();
 
         // verify truncation
-        assertNull(getFromTable(keyspace, "Standard1", "keymulti", "col2"));
+        assertNull(getFromTable(keyspace, CF_STANDARD1, "keymulti", "col2"));
 
         try
         {
             // Restore to point in time.
             CommitLog.instance.archiver.restorePointInTime = time;
-            CommitLog.instance.resetUnsafe();
-            CommitLog.instance.recover();
+            CommitLog.instance.resetUnsafe(false);
         }
         finally
         {
@@ -101,7 +153,7 @@
         }
 
         // Validate pre-truncation data was restored.
-        assertNotNull(getFromTable(keyspace, "Standard1", "keymulti", "col2"));
+        assertNotNull(getFromTable(keyspace, CF_STANDARD1, "keymulti", "col2"));
         // And that we don't have a truncation record after restore time.
         assertFalse(SystemKeyspace.getTruncatedAt(cfs.metadata.cfId) > time);
     }
@@ -109,45 +161,44 @@
     @Test
     public void testTruncatePointInTimeReplayList() throws IOException
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore cfs1 = keyspace.getColumnFamilyStore("Standard1");
-        ColumnFamilyStore cfs2 = keyspace.getColumnFamilyStore("Standard2");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore cfs1 = keyspace.getColumnFamilyStore(CF_STANDARD1);
+        ColumnFamilyStore cfs2 = keyspace.getColumnFamilyStore(CF_STANDARD2);
 
         Mutation rm;
         ColumnFamily cf;
 
         // add a single cell
-        cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, CF_STANDARD1);
         cf.addColumn(column("col3", "val1", 1L));
-        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("keymulti"), cf);
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("keymulti"), cf);
         rm.apply();
 
         // add a single cell
-        cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard2");
+        cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, CF_STANDARD2);
         cf.addColumn(column("col4", "val1", 1L));
-        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("keymulti"), cf);
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("keymulti"), cf);
         rm.apply();
 
         // Make sure data was written
         long time = System.currentTimeMillis();
-        assertNotNull(getFromTable(keyspace, "Standard1", "keymulti", "col3"));
-        assertNotNull(getFromTable(keyspace, "Standard2", "keymulti", "col4"));
+        assertNotNull(getFromTable(keyspace, CF_STANDARD1, "keymulti", "col3"));
+        assertNotNull(getFromTable(keyspace, CF_STANDARD2, "keymulti", "col4"));
 
         // and now truncate it
         cfs1.truncateBlocking();
         cfs2.truncateBlocking();
 
         // verify truncation
-        assertNull(getFromTable(keyspace, "Standard1", "keymulti", "col3"));
-        assertNull(getFromTable(keyspace, "Standard2", "keymulti", "col4"));
+        assertNull(getFromTable(keyspace, CF_STANDARD1, "keymulti", "col3"));
+        assertNull(getFromTable(keyspace, CF_STANDARD2, "keymulti", "col4"));
 
         try
         {
             // Restore to point in time.
             CommitLog.instance.archiver.restorePointInTime = time;
-            System.setProperty("cassandra.replayList", "Keyspace1.Standard1");
-            CommitLog.instance.resetUnsafe();
-            CommitLog.instance.recover();
+            System.setProperty("cassandra.replayList", KEYSPACE1 + "." + CF_STANDARD1);
+            CommitLog.instance.resetUnsafe(false);
         }
         finally
         {
@@ -156,9 +207,9 @@
         }
 
         // Validate pre-truncation data was restored.
-        assertNotNull(getFromTable(keyspace, "Standard1", "keymulti", "col3"));
+        assertNotNull(getFromTable(keyspace, CF_STANDARD1, "keymulti", "col3"));
         // But only on the replayed table.
-        assertNull(getFromTable(keyspace, "Standard2", "keymulti", "col4"));
+        assertNull(getFromTable(keyspace, CF_STANDARD2, "keymulti", "col4"));
 
         // And that we have the correct truncation records.
         assertFalse(SystemKeyspace.getTruncatedAt(cfs1.metadata.cfId) > time);
diff --git a/test/unit/org/apache/cassandra/db/RemoveCellTest.java b/test/unit/org/apache/cassandra/db/RemoveCellTest.java
index 77ff02d..1edb964 100644
--- a/test/unit/org/apache/cassandra/db/RemoveCellTest.java
+++ b/test/unit/org/apache/cassandra/db/RemoveCellTest.java
@@ -18,6 +18,7 @@
 */
 package org.apache.cassandra.db;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import static org.junit.Assert.assertFalse;
@@ -25,29 +26,45 @@
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.filter.QueryFilter;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
-public class RemoveCellTest extends SchemaLoader
+public class RemoveCellTest
 {
+    private static final String KEYSPACE1 = "RemoveCellTest";
+    private static final String CF_STANDARD1 = "Standard1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1));
+    }
+
     @Test
     public void testRemoveColumn()
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
         Mutation rm;
         DecoratedKey dk = Util.dk("key1");
 
         // add data
-        rm = new Mutation("Keyspace1", dk.getKey());
+        rm = new Mutation(KEYSPACE1, dk.getKey());
         rm.add("Standard1", Util.cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
-        rm.apply();
+        rm.applyUnsafe();
         store.forceBlockingFlush();
 
         // remove
-        rm = new Mutation("Keyspace1", dk.getKey());
+        rm = new Mutation(KEYSPACE1, dk.getKey());
         rm.delete("Standard1", Util.cellname("Column1"), 1);
-        rm.apply();
+        rm.applyUnsafe();
 
         ColumnFamily retrieved = store.getColumnFamily(Util.namesQueryFilter(store, dk, "Column1"));
         assertFalse(retrieved.getColumn(Util.cellname("Column1")).isLive());
diff --git a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyTest.java b/test/unit/org/apache/cassandra/db/RemoveColumnFamilyTest.java
index 09eed71..fec8711 100644
--- a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyTest.java
+++ b/test/unit/org/apache/cassandra/db/RemoveColumnFamilyTest.java
@@ -18,6 +18,7 @@
 */
 package org.apache.cassandra.db;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import static org.junit.Assert.assertNull;
@@ -25,28 +26,44 @@
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 
-public class RemoveColumnFamilyTest extends SchemaLoader
+public class RemoveColumnFamilyTest
 {
+    private static final String KEYSPACE1 = "RemoveColumnFamilyTest";
+    private static final String CF_STANDARD1 = "Standard1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1));
+    }
+
     @Test
     public void testRemoveColumnFamily()
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
         Mutation rm;
         DecoratedKey dk = Util.dk("key1");
 
         // add data
-        rm = new Mutation("Keyspace1", dk.getKey());
+        rm = new Mutation(KEYSPACE1, dk.getKey());
         rm.add("Standard1", Util.cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
-        rm.apply();
+        rm.applyUnsafe();
 
         // remove
-        rm = new Mutation("Keyspace1", dk.getKey());
+        rm = new Mutation(KEYSPACE1, dk.getKey());
         rm.delete("Standard1", 1);
-        rm.apply();
+        rm.applyUnsafe();
 
         ColumnFamily retrieved = store.getColumnFamily(QueryFilter.getIdentityFilter(dk, "Standard1", System.currentTimeMillis()));
         assert retrieved.isMarkedForDelete();
diff --git a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush1Test.java b/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush1Test.java
index f898f16..72827d0 100644
--- a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush1Test.java
+++ b/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush1Test.java
@@ -18,6 +18,7 @@
 */
 package org.apache.cassandra.db;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import static org.junit.Assert.assertNull;
@@ -25,30 +26,46 @@
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 
-public class RemoveColumnFamilyWithFlush1Test extends SchemaLoader
+public class RemoveColumnFamilyWithFlush1Test
 {
+    private static final String KEYSPACE1 = "RemoveColumnFamilyWithFlush1Test";
+    private static final String CF_STANDARD1 = "Standard1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1));
+    }
+
     @Test
     public void testRemoveColumnFamilyWithFlush1()
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
         Mutation rm;
         DecoratedKey dk = Util.dk("key1");
 
         // add data
-        rm = new Mutation("Keyspace1", dk.getKey());
+        rm = new Mutation(KEYSPACE1, dk.getKey());
         rm.add("Standard1", Util.cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
         rm.add("Standard1", Util.cellname("Column2"), ByteBufferUtil.bytes("asdf"), 0);
-        rm.apply();
+        rm.applyUnsafe();
         store.forceBlockingFlush();
 
         // remove
-        rm = new Mutation("Keyspace1", dk.getKey());
+        rm = new Mutation(KEYSPACE1, dk.getKey());
         rm.delete("Standard1", 1);
-        rm.apply();
+        rm.applyUnsafe();
 
         ColumnFamily retrieved = store.getColumnFamily(QueryFilter.getIdentityFilter(dk, "Standard1", System.currentTimeMillis()));
         assert retrieved.isMarkedForDelete();
diff --git a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush2Test.java b/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush2Test.java
index 1e910ad..ef7f7f2 100644
--- a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush2Test.java
+++ b/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush2Test.java
@@ -18,6 +18,7 @@
 */
 package org.apache.cassandra.db;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import static org.junit.Assert.assertNull;
@@ -25,27 +26,43 @@
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 
-public class RemoveColumnFamilyWithFlush2Test extends SchemaLoader
+public class RemoveColumnFamilyWithFlush2Test
 {
+    private static final String KEYSPACE1 = "RemoveColumnFamilyWithFlush2Test";
+    private static final String CF_STANDARD1 = "Standard1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1));
+    }
+
     @Test
     public void testRemoveColumnFamilyWithFlush2()
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
         Mutation rm;
         DecoratedKey dk = Util.dk("key1");
 
         // add data
-        rm = new Mutation("Keyspace1", dk.getKey());
+        rm = new Mutation(KEYSPACE1, dk.getKey());
         rm.add("Standard1", Util.cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
-        rm.apply();
+        rm.applyUnsafe();
         // remove
-        rm = new Mutation("Keyspace1", dk.getKey());
+        rm = new Mutation(KEYSPACE1, dk.getKey());
         rm.delete("Standard1", 1);
-        rm.apply();
+        rm.applyUnsafe();
         store.forceBlockingFlush();
 
         ColumnFamily retrieved = store.getColumnFamily(QueryFilter.getIdentityFilter(dk, "Standard1", System.currentTimeMillis()));
diff --git a/test/unit/org/apache/cassandra/db/RemoveSubCellTest.java b/test/unit/org/apache/cassandra/db/RemoveSubCellTest.java
index cec1bce..3fa5c2f 100644
--- a/test/unit/org/apache/cassandra/db/RemoveSubCellTest.java
+++ b/test/unit/org/apache/cassandra/db/RemoveSubCellTest.java
@@ -21,12 +21,17 @@
 import java.nio.ByteBuffer;
 import java.util.concurrent.TimeUnit;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.db.filter.QueryFilter;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 
 import static org.apache.cassandra.Util.getBytes;
 import org.apache.cassandra.Util;
@@ -36,27 +41,40 @@
 import com.google.common.util.concurrent.Uninterruptibles;
 
 
-public class RemoveSubCellTest extends SchemaLoader
+public class RemoveSubCellTest
 {
+    private static final String KEYSPACE1 = "RemoveSubCellTest";
+    private static final String CF_SUPER1 = "Super1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.superCFMD(KEYSPACE1, CF_SUPER1, LongType.instance));
+    }
+
     @Test
     public void testRemoveSubColumn()
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore("Super1");
         Mutation rm;
         DecoratedKey dk = Util.dk("key1");
 
         // add data
-        rm = new Mutation("Keyspace1", dk.getKey());
+        rm = new Mutation(KEYSPACE1, dk.getKey());
         Util.addMutation(rm, "Super1", "SC1", 1, "asdf", 0);
-        rm.apply();
+        rm.applyUnsafe();
         store.forceBlockingFlush();
 
         CellName cname = CellNames.compositeDense(ByteBufferUtil.bytes("SC1"), getBytes(1L));
         // remove
-        rm = new Mutation("Keyspace1", dk.getKey());
+        rm = new Mutation(KEYSPACE1, dk.getKey());
         rm.delete("Super1", cname, 1);
-        rm.apply();
+        rm.applyUnsafe();
 
         ColumnFamily retrieved = store.getColumnFamily(QueryFilter.getIdentityFilter(dk, "Super1", System.currentTimeMillis()));
         assertFalse(retrieved.getColumn(cname).isLive());
@@ -66,23 +84,23 @@
     @Test
     public void testRemoveSubColumnAndContainer()
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore("Super1");
         Mutation rm;
         DecoratedKey dk = Util.dk("key2");
 
         // add data
-        rm = new Mutation("Keyspace1", dk.getKey());
+        rm = new Mutation(KEYSPACE1, dk.getKey());
         Util.addMutation(rm, "Super1", "SC1", 1, "asdf", 0);
-        rm.apply();
+        rm.applyUnsafe();
         store.forceBlockingFlush();
 
         // remove the SC
         ByteBuffer scName = ByteBufferUtil.bytes("SC1");
         CellName cname = CellNames.compositeDense(scName, getBytes(1L));
-        rm = new Mutation("Keyspace1", dk.getKey());
+        rm = new Mutation(KEYSPACE1, dk.getKey());
         rm.deleteRange("Super1", SuperColumns.startOf(scName), SuperColumns.endOf(scName), 1);
-        rm.apply();
+        rm.applyUnsafe();
 
         // Mark current time and make sure the next insert happens at least
         // one second after the previous one (since gc resolution is the second)
@@ -90,9 +108,9 @@
         Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS);
 
         // remove the column itself
-        rm = new Mutation("Keyspace1", dk.getKey());
+        rm = new Mutation(KEYSPACE1, dk.getKey());
         rm.delete("Super1", cname, 2);
-        rm.apply();
+        rm.applyUnsafe();
 
         ColumnFamily retrieved = store.getColumnFamily(filter);
         assertFalse(retrieved.getColumn(cname).isLive());
diff --git a/test/unit/org/apache/cassandra/db/RowCacheTest.java b/test/unit/org/apache/cassandra/db/RowCacheTest.java
index 714d95d..332714fb 100644
--- a/test/unit/org/apache/cassandra/db/RowCacheTest.java
+++ b/test/unit/org/apache/cassandra/db/RowCacheTest.java
@@ -27,35 +27,57 @@
 
 import com.google.common.collect.Lists;
 import org.junit.AfterClass;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.cache.CachingOptions;
 import org.apache.cassandra.cache.RowCacheKey;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.filter.QueryFilter;
+import org.apache.cassandra.db.marshal.IntegerType;
 import org.apache.cassandra.dht.Bounds;
-import org.apache.cassandra.dht.BytesToken;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.dht.ByteOrderedPartitioner.BytesToken;
 import org.apache.cassandra.locator.TokenMetadata;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.metrics.ClearableHistogram;
 import org.apache.cassandra.service.CacheService;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.ByteBufferUtil;
-
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
-public class RowCacheTest extends SchemaLoader
+public class RowCacheTest
 {
-    private String KEYSPACE = "RowCacheSpace";
-    private String COLUMN_FAMILY = "CachedCF";
+    private static final String KEYSPACE_CACHED = "RowCacheTest";
+    private static final String CF_CACHED = "CachedCF";
+    private static final String CF_CACHEDINT = "CachedIntCF";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        System.setProperty("org.caffinitas.ohc.segmentCount", "16");
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE_CACHED,
+                SimpleStrategy.class,
+                KSMetaData.optsWithRF(1),
+                SchemaLoader.standardCFMD(KEYSPACE_CACHED, CF_CACHED).caching(CachingOptions.ALL),
+                SchemaLoader.standardCFMD(KEYSPACE_CACHED, CF_CACHEDINT)
+                            .defaultValidator(IntegerType.instance)
+                            .caching(new CachingOptions(new CachingOptions.KeyCache(CachingOptions.KeyCache.Type.ALL),
+                                     new CachingOptions.RowCache(CachingOptions.RowCache.Type.HEAD, 100))));
+    }
 
     @AfterClass
     public static void cleanup()
     {
-        cleanupSavedCaches();
+        SchemaLoader.cleanupSavedCaches();
     }
 
     @Test
@@ -63,8 +85,8 @@
     {
         CompactionManager.instance.disableAutoCompaction();
 
-        Keyspace keyspace = Keyspace.open(KEYSPACE);
-        ColumnFamilyStore cachedStore  = keyspace.getColumnFamilyStore(COLUMN_FAMILY);
+        Keyspace keyspace = Keyspace.open(KEYSPACE_CACHED);
+        ColumnFamilyStore cachedStore  = keyspace.getColumnFamilyStore(CF_CACHED);
 
         // empty the row cache
         CacheService.instance.invalidateRowCache();
@@ -73,7 +95,7 @@
         CacheService.instance.setRowCacheCapacityInMB(1);
 
         // inserting 100 rows into both column families
-        insertData(KEYSPACE, COLUMN_FAMILY, 0, 100);
+        SchemaLoader.insertData(KEYSPACE_CACHED, CF_CACHED, 0, 100);
 
         // now reading rows one by one and checking if row change grows
         for (int i = 0; i < 100; i++)
@@ -96,7 +118,7 @@
         }
 
         // insert 10 more keys
-        insertData(KEYSPACE, COLUMN_FAMILY, 100, 10);
+        SchemaLoader.insertData(KEYSPACE_CACHED, CF_CACHED, 100, 10);
 
         for (int i = 100; i < 110; i++)
         {
@@ -143,10 +165,10 @@
         CacheService.instance.setRowCacheCapacityInMB(1);
         rowCacheLoad(100, Integer.MAX_VALUE, 1000);
 
-        ColumnFamilyStore store = Keyspace.open(KEYSPACE).getColumnFamilyStore(COLUMN_FAMILY);
-        assertEquals(CacheService.instance.rowCache.getKeySet().size(), 100);
+        ColumnFamilyStore store = Keyspace.open(KEYSPACE_CACHED).getColumnFamilyStore(CF_CACHED);
+        assertEquals(CacheService.instance.rowCache.size(), 100);
         store.cleanupCache();
-        assertEquals(CacheService.instance.rowCache.getKeySet().size(), 100);
+        assertEquals(CacheService.instance.rowCache.size(), 100);
         TokenMetadata tmd = StorageService.instance.getTokenMetadata();
         byte[] tk1, tk2;
         tk1 = "key1000".getBytes();
@@ -154,7 +176,7 @@
         tmd.updateNormalToken(new BytesToken(tk1), InetAddress.getByName("127.0.0.1"));
         tmd.updateNormalToken(new BytesToken(tk2), InetAddress.getByName("127.0.0.2"));
         store.cleanupCache();
-        assertEquals(CacheService.instance.rowCache.getKeySet().size(), 50);
+        assertEquals(50, CacheService.instance.rowCache.size());
         CacheService.instance.setRowCacheCapacityInMB(0);
     }
 
@@ -165,8 +187,8 @@
         CacheService.instance.setRowCacheCapacityInMB(1);
         rowCacheLoad(100, Integer.MAX_VALUE, 1000);
 
-        ColumnFamilyStore store = Keyspace.open(KEYSPACE).getColumnFamilyStore(COLUMN_FAMILY);
-        assertEquals(CacheService.instance.rowCache.getKeySet().size(), 100);
+        ColumnFamilyStore store = Keyspace.open(KEYSPACE_CACHED).getColumnFamilyStore(CF_CACHED);
+        assertEquals(CacheService.instance.rowCache.size(), 100);
 
         //construct 5 ranges of 20 elements each
         ArrayList<Bounds<Token>> subranges = getBounds(20);
@@ -177,17 +199,17 @@
         assertEquals(60, invalidatedKeys);
 
         //now there should be only 40 cached entries left
-        assertEquals(40, CacheService.instance.rowCache.getKeySet().size());
+        assertEquals(CacheService.instance.rowCache.size(), 40);
         CacheService.instance.setRowCacheCapacityInMB(0);
     }
 
     private ArrayList<Bounds<Token>> getBounds(int nElements)
     {
-        ColumnFamilyStore store = Keyspace.open(KEYSPACE).getColumnFamilyStore(COLUMN_FAMILY);
+        ColumnFamilyStore store = Keyspace.open(KEYSPACE_CACHED).getColumnFamilyStore(CF_CACHED);
         TreeSet<DecoratedKey> orderedKeys = new TreeSet<>();
 
-        for (RowCacheKey key : CacheService.instance.rowCache.getKeySet())
-            orderedKeys.add(store.partitioner.decorateKey(ByteBuffer.wrap(key.key)));
+        for(Iterator<RowCacheKey> it = CacheService.instance.rowCache.keyIterator();it.hasNext();)
+            orderedKeys.add(store.partitioner.decorateKey(ByteBuffer.wrap(it.next().key)));
 
         ArrayList<Bounds<Token>> boundsToInvalidate = new ArrayList<>();
         Iterator<DecoratedKey> iterator = orderedKeys.iterator();
@@ -217,7 +239,7 @@
         CacheService.instance.setRowCacheCapacityInMB(1);
         rowCacheLoad(100, 50, 0);
         CacheService.instance.rowCache.submitWrite(Integer.MAX_VALUE).get();
-        Keyspace instance = Schema.instance.removeKeyspaceInstance(KEYSPACE);
+        Keyspace instance = Schema.instance.removeKeyspaceInstance(KEYSPACE_CACHED);
         try
         {
             CacheService.instance.rowCache.size();
@@ -251,11 +273,11 @@
     {
         CompactionManager.instance.disableAutoCompaction();
 
-        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        Keyspace keyspace = Keyspace.open(KEYSPACE_CACHED);
         String cf = "CachedIntCF";
         ColumnFamilyStore cachedStore  = keyspace.getColumnFamilyStore(cf);
-        long startRowCacheHits = cachedStore.metric.rowCacheHit.count();
-        long startRowCacheOutOfRange = cachedStore.metric.rowCacheHitOutOfRange.count();
+        long startRowCacheHits = cachedStore.metric.rowCacheHit.getCount();
+        long startRowCacheOutOfRange = cachedStore.metric.rowCacheHitOutOfRange.getCount();
         // empty the row cache
         CacheService.instance.invalidateRowCache();
 
@@ -265,41 +287,41 @@
         ByteBuffer key = ByteBufferUtil.bytes("rowcachekey");
         DecoratedKey dk = cachedStore.partitioner.decorateKey(key);
         RowCacheKey rck = new RowCacheKey(cachedStore.metadata.ksAndCFName, dk);
-        Mutation mutation = new Mutation(KEYSPACE, key);
+        Mutation mutation = new Mutation(KEYSPACE_CACHED, key);
         for (int i = 0; i < 200; i++)
             mutation.add(cf, Util.cellname(i), ByteBufferUtil.bytes("val" + i), System.currentTimeMillis());
         mutation.applyUnsafe();
 
         // populate row cache, we should not get a row cache hit;
         cachedStore.getColumnFamily(QueryFilter.getSliceFilter(dk, cf,
-                                                               Composites.EMPTY,
-                                                               Composites.EMPTY,
-                                                               false, 10, System.currentTimeMillis()));
-        assertEquals(startRowCacheHits, cachedStore.metric.rowCacheHit.count());
+                                                                Composites.EMPTY,
+                                                                Composites.EMPTY,
+                                                                false, 10, System.currentTimeMillis()));
+        assertEquals(startRowCacheHits, cachedStore.metric.rowCacheHit.getCount());
 
         // do another query, limit is 20, which is < 100 that we cache, we should get a hit and it should be in range
         cachedStore.getColumnFamily(QueryFilter.getSliceFilter(dk, cf,
                                                                 Composites.EMPTY,
                                                                 Composites.EMPTY,
                                                                 false, 20, System.currentTimeMillis()));
-        assertEquals(++startRowCacheHits, cachedStore.metric.rowCacheHit.count());
-        assertEquals(startRowCacheOutOfRange, cachedStore.metric.rowCacheHitOutOfRange.count());
+        assertEquals(++startRowCacheHits, cachedStore.metric.rowCacheHit.getCount());
+        assertEquals(startRowCacheOutOfRange, cachedStore.metric.rowCacheHitOutOfRange.getCount());
 
         // get a slice from 95 to 105, 95->99 are in cache, we should not get a hit and then row cache is out of range
         cachedStore.getColumnFamily(QueryFilter.getSliceFilter(dk, cf,
                                                                CellNames.simpleDense(ByteBufferUtil.bytes(95)),
                                                                CellNames.simpleDense(ByteBufferUtil.bytes(105)),
                                                                false, 10, System.currentTimeMillis()));
-        assertEquals(startRowCacheHits, cachedStore.metric.rowCacheHit.count());
-        assertEquals(++startRowCacheOutOfRange, cachedStore.metric.rowCacheHitOutOfRange.count());
+        assertEquals(startRowCacheHits, cachedStore.metric.rowCacheHit.getCount());
+        assertEquals(++startRowCacheOutOfRange, cachedStore.metric.rowCacheHitOutOfRange.getCount());
 
         // get a slice with limit > 100, we should get a hit out of range.
         cachedStore.getColumnFamily(QueryFilter.getSliceFilter(dk, cf,
                                                                Composites.EMPTY,
                                                                Composites.EMPTY,
                                                                false, 101, System.currentTimeMillis()));
-        assertEquals(startRowCacheHits, cachedStore.metric.rowCacheHit.count());
-        assertEquals(++startRowCacheOutOfRange, cachedStore.metric.rowCacheHitOutOfRange.count());
+        assertEquals(startRowCacheHits, cachedStore.metric.rowCacheHit.getCount());
+        assertEquals(++startRowCacheOutOfRange, cachedStore.metric.rowCacheHitOutOfRange.getCount());
 
 
         CacheService.instance.invalidateRowCache();
@@ -309,7 +331,7 @@
                                                                 Composites.EMPTY,
                                                                 Composites.EMPTY,
                                                                 false, 105, System.currentTimeMillis()));
-        assertEquals(startRowCacheHits, cachedStore.metric.rowCacheHit.count());
+        assertEquals(startRowCacheHits, cachedStore.metric.rowCacheHit.getCount());
         // validate the stuff in cache;
         ColumnFamily cachedCf = (ColumnFamily)CacheService.instance.rowCache.get(rck);
         assertEquals(cachedCf.getColumnCount(), 100);
@@ -325,8 +347,8 @@
     {
         CompactionManager.instance.disableAutoCompaction();
 
-        Keyspace keyspace = Keyspace.open(KEYSPACE);
-        ColumnFamilyStore cachedStore  = keyspace.getColumnFamilyStore(COLUMN_FAMILY);
+        Keyspace keyspace = Keyspace.open(KEYSPACE_CACHED);
+        ColumnFamilyStore cachedStore  = keyspace.getColumnFamilyStore(CF_CACHED);
 
         // empty the row cache
         CacheService.instance.invalidateRowCache();
@@ -335,13 +357,12 @@
         CacheService.instance.setRowCacheCapacityInMB(1);
 
         // inserting 100 rows into both column families
-        insertData(KEYSPACE, COLUMN_FAMILY, 0, 100);
+        SchemaLoader.insertData(KEYSPACE_CACHED, CF_CACHED, 0, 100);
 
         //force flush for confidence that SSTables exists
         cachedStore.forceBlockingFlush();
 
-        // clear SSTablePerReadHistogram
-        cachedStore.metric.sstablesPerReadHistogram.cf.clear();
+        ((ClearableHistogram)cachedStore.metric.sstablesPerReadHistogram.cf).clear();
 
         for (int i = 0; i < 100; i++)
         {
@@ -349,46 +370,47 @@
 
             cachedStore.getColumnFamily(key, Composites.EMPTY, Composites.EMPTY, false, 1, System.currentTimeMillis());
 
-            long count_before = cachedStore.metric.sstablesPerReadHistogram.cf.count();
+            long count_before = cachedStore.metric.sstablesPerReadHistogram.cf.getCount();
             cachedStore.getColumnFamily(key, Composites.EMPTY, Composites.EMPTY, false, 1, System.currentTimeMillis());
 
             // check that SSTablePerReadHistogram has been updated by zero,
             // so count has been increased and in a 1/2 of requests there were zero read SSTables
-            long count_after = cachedStore.metric.sstablesPerReadHistogram.cf.count();
-            double belowMedian_after = cachedStore.metric.sstablesPerReadHistogram.cf.getSnapshot().getValue(0.49);
-            double mean_after = cachedStore.metric.sstablesPerReadHistogram.cf.mean();
+            long count_after = cachedStore.metric.sstablesPerReadHistogram.cf.getCount();
+            double belowMedian = cachedStore.metric.sstablesPerReadHistogram.cf.getSnapshot().getValue(0.49D);
+            double mean_after = cachedStore.metric.sstablesPerReadHistogram.cf.getSnapshot().getMean();
             assertEquals("SSTablePerReadHistogram should be updated even key found in row cache", count_before + 1, count_after);
             assertTrue("In half of requests we have not touched SSTables, " +
-                       "so 49 percentile value (" + belowMedian_after + ") must be strongly less than 0.9", belowMedian_after < 0.9D);
+                       "so 49 percentile (" + belowMedian + ") must be strongly less than 0.9", belowMedian < 0.9D);
             assertTrue("In half of requests we have not touched SSTables, " +
                        "so mean value (" + mean_after + ") must be strongly less than 1, but greater than 0", mean_after < 0.999D && mean_after > 0.001D);
         }
 
-        assertEquals("Min value of SSTablesPerRead should be zero", 0, cachedStore.metric.sstablesPerReadHistogram.cf.min(), 0.01);
+        assertEquals("Min value of SSTablesPerRead should be zero", 0, cachedStore.metric.sstablesPerReadHistogram.cf.getSnapshot().getMin());
 
         CacheService.instance.setRowCacheCapacityInMB(0);
     }
 
-
     public void rowCacheLoad(int totalKeys, int keysToSave, int offset) throws Exception
     {
         CompactionManager.instance.disableAutoCompaction();
 
+        ColumnFamilyStore store = Keyspace.open(KEYSPACE_CACHED).getColumnFamilyStore(CF_CACHED);
+
         // empty the cache
         CacheService.instance.invalidateRowCache();
-        assert CacheService.instance.rowCache.size() == 0;
+        assertEquals(0, CacheService.instance.rowCache.size());
 
         // insert data and fill the cache
-        insertData(KEYSPACE, COLUMN_FAMILY, offset, totalKeys);
-        readData(KEYSPACE, COLUMN_FAMILY, offset, totalKeys);
-        assert CacheService.instance.rowCache.size() == totalKeys;
+        SchemaLoader.insertData(KEYSPACE_CACHED, CF_CACHED, offset, totalKeys);
+        SchemaLoader.readData(KEYSPACE_CACHED, CF_CACHED, offset, totalKeys);
+        assertEquals(totalKeys, CacheService.instance.rowCache.size());
 
         // force the cache to disk
         CacheService.instance.rowCache.submitWrite(keysToSave).get();
 
         // empty the cache again to make sure values came from disk
         CacheService.instance.invalidateRowCache();
-        assert CacheService.instance.rowCache.size() == 0;
-        assert CacheService.instance.rowCache.loadSaved() == (keysToSave == Integer.MAX_VALUE ? totalKeys : keysToSave);
+        assertEquals(0, CacheService.instance.rowCache.size());
+        assertEquals(keysToSave == Integer.MAX_VALUE ? totalKeys : keysToSave, CacheService.instance.rowCache.loadSaved());
     }
 }
diff --git a/test/unit/org/apache/cassandra/db/RowIndexEntryTest.java b/test/unit/org/apache/cassandra/db/RowIndexEntryTest.java
index ce58e11..30267d9 100644
--- a/test/unit/org/apache/cassandra/db/RowIndexEntryTest.java
+++ b/test/unit/org/apache/cassandra/db/RowIndexEntryTest.java
@@ -18,14 +18,19 @@
 package org.apache.cassandra.db;
 
 import java.io.IOException;
+import java.util.Collections;
 
 import junit.framework.Assert;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.composites.CellNames;
 import org.apache.cassandra.db.composites.SimpleDenseCellNameType;
 import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.io.sstable.IndexHelper;
 import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.junit.Test;
@@ -35,16 +40,21 @@
     @Test
     public void testSerializedSize() throws IOException
     {
-        final RowIndexEntry simple = new RowIndexEntry(123);
+        final RowIndexEntry<IndexHelper.IndexInfo> simple = new RowIndexEntry<>(123);
 
         DataOutputBuffer buffer = new DataOutputBuffer();
-        RowIndexEntry.Serializer serializer = new RowIndexEntry.Serializer(new SimpleDenseCellNameType(UTF8Type.instance));
+        RowIndexEntry.Serializer serializer = new RowIndexEntry.Serializer(new IndexHelper.IndexInfo.Serializer(new SimpleDenseCellNameType(UTF8Type.instance)));
 
         serializer.serialize(simple, buffer);
 
         Assert.assertEquals(buffer.getLength(), serializer.serializedSize(simple));
 
         buffer = new DataOutputBuffer();
+        Schema.instance.setKeyspaceDefinition(KSMetaData.newKeyspace("Keyspace1",
+                                                                     SimpleStrategy.class,
+                                                                     Collections.<String,String>emptyMap(),
+                                                                     false,
+                                                                     Collections.singleton(standardCFMD("Keyspace1", "Standard1"))));
         ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
         ColumnIndex columnIndex = new ColumnIndex.Builder(cf, ByteBufferUtil.bytes("a"), new DataOutputBuffer())
         {{
@@ -62,7 +72,7 @@
 
         }}.build();
 
-        RowIndexEntry withIndex = RowIndexEntry.create(0xdeadbeef, DeletionTime.LIVE, columnIndex);
+        RowIndexEntry<IndexHelper.IndexInfo> withIndex = RowIndexEntry.create(0xdeadbeef, DeletionTime.LIVE, columnIndex);
 
         serializer.serialize(withIndex, buffer);
         Assert.assertEquals(buffer.getLength(), serializer.serializedSize(withIndex));
diff --git a/test/unit/org/apache/cassandra/db/RowIterationTest.java b/test/unit/org/apache/cassandra/db/RowIterationTest.java
index 245a6a7..ee7bf1a 100644
--- a/test/unit/org/apache/cassandra/db/RowIterationTest.java
+++ b/test/unit/org/apache/cassandra/db/RowIterationTest.java
@@ -25,20 +25,36 @@
 
 import org.apache.cassandra.Util;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.composites.*;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.FBUtilities;
 import static org.junit.Assert.assertEquals;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 
-public class RowIterationTest extends SchemaLoader
+public class RowIterationTest
 {
-    public static final String KEYSPACE1 = "Keyspace2";
+    public static final String KEYSPACE1 = "RowIterationTest";
     public static final InetAddress LOCAL = FBUtilities.getBroadcastAddress();
 
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, "Standard3"),
+                                    SchemaLoader.superCFMD(KEYSPACE1, "Super3", LongType.instance));
+    }
+
     @Test
     public void testRowIteration()
     {
@@ -51,7 +67,7 @@
             DecoratedKey key = Util.dk(String.valueOf(i));
             Mutation rm = new Mutation(KEYSPACE1, key.getKey());
             rm.add("Super3", CellNames.compositeDense(ByteBufferUtil.bytes("sc"), ByteBufferUtil.bytes(String.valueOf(i))), ByteBuffer.wrap(new byte[ROWS_PER_SSTABLE * 10 - i * 2]), i);
-            rm.apply();
+            rm.applyUnsafe();
             inserted.add(key);
         }
         store.forceBlockingFlush();
@@ -70,7 +86,7 @@
         Mutation rm = new Mutation(KEYSPACE1, key.getKey());
         rm.delete(CF_NAME, 0);
         rm.add(CF_NAME, Util.cellname("c"), ByteBufferUtil.bytes("values"), 0L);
-        rm.apply();
+        rm.applyUnsafe();
         store.forceBlockingFlush();
 
         // Delete row in second sstable with higher timestamp
@@ -79,10 +95,10 @@
         rm.add(CF_NAME, Util.cellname("c"), ByteBufferUtil.bytes("values"), 1L);
         DeletionInfo delInfo2 = rm.getColumnFamilies().iterator().next().deletionInfo();
         assert delInfo2.getTopLevelDeletion().markedForDeleteAt == 1L;
-        rm.apply();
+        rm.applyUnsafe();
         store.forceBlockingFlush();
 
-        ColumnFamily cf = Util.getRangeSlice(store).iterator().next().cf;
+        ColumnFamily cf = Util.getRangeSlice(store).get(0).cf;
         assert cf.deletionInfo().equals(delInfo2);
     }
 
@@ -97,10 +113,10 @@
         // Delete a row in first sstable
         Mutation rm = new Mutation(KEYSPACE1, key.getKey());
         rm.delete(CF_NAME, 0);
-        rm.apply();
+        rm.applyUnsafe();
         store.forceBlockingFlush();
 
-        ColumnFamily cf = Util.getRangeSlice(store).iterator().next().cf;
+        ColumnFamily cf = Util.getRangeSlice(store).get(0).cf;
         assert cf != null;
     }
 }
diff --git a/test/unit/org/apache/cassandra/db/RowTest.java b/test/unit/org/apache/cassandra/db/RowTest.java
index 22e112e..910f9e1 100644
--- a/test/unit/org/apache/cassandra/db/RowTest.java
+++ b/test/unit/org/apache/cassandra/db/RowTest.java
@@ -22,10 +22,14 @@
 import java.util.concurrent.TimeUnit;
 
 import com.google.common.util.concurrent.Uninterruptibles;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.composites.CellNames;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 import static org.apache.cassandra.Util.column;
@@ -34,15 +38,28 @@
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-public class RowTest extends SchemaLoader
+public class RowTest
 {
+    private static final String KEYSPACE1 = "RowTest";
+    private static final String CF_STANDARD1 = "Standard1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1));
+    }
+
     @Test
     public void testDiffColumnFamily()
     {
-        ColumnFamily cf1 = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf1 = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf1.addColumn(column("one", "onev", 0));
 
-        ColumnFamily cf2 = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf2 = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         DeletionInfo delInfo = new DeletionInfo(0, 0);
         cf2.delete(delInfo);
 
@@ -86,10 +103,10 @@
     @Test
     public void testResolve()
     {
-        ColumnFamily cf1 = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf1 = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf1.addColumn(column("one", "A", 0));
 
-        ColumnFamily cf2 = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf2 = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf2.addColumn(column("one", "B", 1));
         cf2.addColumn(column("two", "C", 1));
 
diff --git a/test/unit/org/apache/cassandra/db/ScrubTest.java b/test/unit/org/apache/cassandra/db/ScrubTest.java
index 4efd082..4cca7ff 100644
--- a/test/unit/org/apache/cassandra/db/ScrubTest.java
+++ b/test/unit/org/apache/cassandra/db/ScrubTest.java
@@ -20,23 +20,31 @@
  *
  */
 
-import java.io.*;
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
 import java.nio.ByteBuffer;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
+import java.util.*;
 import java.util.concurrent.ExecutionException;
 
+import java.io.File;
+import java.io.IOError;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.db.composites.CellNameType;
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.CounterColumnType;
+import org.apache.cassandra.db.marshal.UUIDType;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.db.marshal.CompositeType;
-import org.apache.cassandra.db.marshal.LongType;
-import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.exceptions.RequestExecutionException;
 import org.apache.cassandra.io.compress.CompressionMetadata;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.UUIDGen;
 
 import org.apache.commons.lang3.StringUtils;
@@ -45,43 +53,70 @@
 import org.junit.Test;
 import org.junit.runner.RunWith;
 
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
-import org.apache.cassandra.cql3.UntypedResultSet;
-import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.Util;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.cql3.Operator;
+import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.db.columniterator.IdentityQueryFilter;
 import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.compaction.Scrubber;
+import org.apache.cassandra.db.index.SecondaryIndex;
+import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.exceptions.WriteTimeoutException;
 import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.io.sstable.SSTableRewriter;
+import org.apache.cassandra.OrderedJUnit4ClassRunner;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.Util;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
+import static org.junit.Assert.*;
+import static org.junit.Assume.assumeTrue;
+
 import static org.apache.cassandra.Util.cellname;
 import static org.apache.cassandra.Util.column;
-import static junit.framework.Assert.assertNotNull;
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-import static org.junit.Assume.assumeTrue;
 
 @RunWith(OrderedJUnit4ClassRunner.class)
-public class ScrubTest extends SchemaLoader
+public class ScrubTest
 {
-    public String KEYSPACE = "Keyspace1";
-    public String CF = "Standard1";
-    public String CF3 = "Standard2";
-    public String COUNTER_CF = "Counter1";
-    private static Integer COMPRESSION_CHUNK_LENGTH = 4096;
+    public static final String KEYSPACE = "Keyspace1";
+    public static final String CF = "Standard1";
+    public static final String CF2 = "Standard2";
+    public static final String CF3 = "Standard3";
+    public static final String CFI1 = "StandardInteger1";
+    public static final String COUNTER_CF = "Counter1";
+    public static final String CF_UUID = "UUIDKeys";
+    public static final String CF_INDEX1 = "Indexed1";
+    public static final String CF_INDEX2 = "Indexed2";
+
+    public static final String COL_KEYS_INDEX = "birthdate";
+    public static final String COL_COMPOSITES_INDEX = "col1";
+    public static final String COL_NON_INDEX = "notanindexcol";
+
+    public static final Integer COMPRESSION_CHUNK_LENGTH = 4096;
 
     @BeforeClass
-    public static void loadSchema() throws ConfigurationException
+    public static void defineSchema() throws ConfigurationException
     {
-        loadSchema(COMPRESSION_CHUNK_LENGTH);
+        SchemaLoader.loadSchema();
+        SchemaLoader.createKeyspace(KEYSPACE,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF2),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF3),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CFI1),
+                                    SchemaLoader.standardCFMD(KEYSPACE, COUNTER_CF)
+                                                .defaultValidator(CounterColumnType.instance)
+                                                .compressionParameters(SchemaLoader.getCompressionParameters(COMPRESSION_CHUNK_LENGTH)),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF_UUID).keyValidator(UUIDType.instance),
+                                    SchemaLoader.indexCFMD(KEYSPACE, CF_INDEX1, true),
+                                    SchemaLoader.compositeIndexCFMD(KEYSPACE, CF_INDEX2, true));
     }
 
     @Test
@@ -122,6 +157,7 @@
 
         List<Row> rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), numPartitions*10);
         assertEquals(numPartitions, rows.size());
+
         assertEquals(1, cfs.getSSTables().size());
 
         SSTableReader sstable = cfs.getSSTables().iterator().next();
@@ -130,19 +166,21 @@
         overrideWithGarbage(sstable, ByteBufferUtil.bytes("0"), ByteBufferUtil.bytes("1"));
 
         // with skipCorrupted == false, the scrub is expected to fail
-        Scrubber scrubber = new Scrubber(cfs, sstable, false, false, true);
-        try
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(Arrays.asList(sstable), OperationType.SCRUB);
+             Scrubber scrubber = new Scrubber(cfs, txn, false, true);)
         {
             scrubber.scrub();
             fail("Expected a CorruptSSTableException to be thrown");
         }
         catch (IOError err) {}
 
-        // with skipCorrupted == true, the corrupt row will be skipped
+        // with skipCorrupted == true, the corrupt rows will be skipped
         Scrubber.ScrubResult scrubResult;
-        scrubber = new Scrubber(cfs, sstable, true, false, true);
-        scrubResult = scrubber.scrubWithResult();
-        scrubber.close();
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(Arrays.asList(sstable), OperationType.SCRUB);
+             Scrubber scrubber = new Scrubber(cfs, txn, true, true);)
+        {
+            scrubResult = scrubber.scrubWithResult();
+        }
 
         assertNotNull(scrubResult);
 
@@ -188,20 +226,24 @@
         overrideWithGarbage(sstable, ByteBufferUtil.bytes("0"), ByteBufferUtil.bytes("1"));
 
         // with skipCorrupted == false, the scrub is expected to fail
-        Scrubber scrubber = new Scrubber(cfs, sstable, false, false, true);
-        try
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(Arrays.asList(sstable), OperationType.SCRUB);
+             Scrubber scrubber = new Scrubber(cfs, txn, false, true))
         {
+            // with skipCorrupted == true, the corrupt row will be skipped
             scrubber.scrub();
             fail("Expected a CorruptSSTableException to be thrown");
         }
         catch (IOError err) {}
 
-        // with skipCorrupted == true, the corrupt row will be skipped
-        scrubber = new Scrubber(cfs, sstable, true, false, true);
-        scrubber.scrub();
-        scrubber.close();
-        assertEquals(1, cfs.getSSTables().size());
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(Arrays.asList(sstable), OperationType.SCRUB);
+             Scrubber scrubber = new Scrubber(cfs, txn, true, true))
+        {
+            // with skipCorrupted == true, the corrupt row will be skipped
+            scrubber.scrub();
+            scrubber.close();
+        }
 
+        assertEquals(1, cfs.getSSTables().size());
         // verify that we can read all of the rows, and there is now one less row
         rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
         assertEquals(1, rows.size());
@@ -236,14 +278,29 @@
     }
 
     @Test
+    public void testScrubCorruptedCounterRowNoEarlyOpen() throws IOException, WriteTimeoutException
+    {
+        boolean oldDisabledVal = SSTableRewriter.disableEarlyOpeningForTests;
+        try
+        {
+            SSTableRewriter.disableEarlyOpeningForTests = true;
+            testScrubCorruptedCounterRow();
+        }
+        finally
+        {
+            SSTableRewriter.disableEarlyOpeningForTests = oldDisabledVal;
+        }
+    }
+
+    @Test
     public void testScrubDeletedRow() throws ExecutionException, InterruptedException
     {
         CompactionManager.instance.disableAutoCompaction();
         Keyspace keyspace = Keyspace.open(KEYSPACE);
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF3);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF2);
         cfs.clearUnsafe();
 
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE, CF3);
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE, CF2);
         cf.delete(new DeletionInfo(0, 1)); // expired tombstone
         Mutation rm = new Mutation(KEYSPACE, ByteBufferUtil.bytes(1), cf);
         rm.applyUnsafe();
@@ -305,7 +362,7 @@
     {
         CompactionManager.instance.disableAutoCompaction();
         Keyspace keyspace = Keyspace.open(KEYSPACE);
-        String columnFamily = "Standard3";
+        String columnFamily = CF3;
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(columnFamily);
         cfs.clearUnsafe();
 
@@ -314,7 +371,7 @@
          * The test also assumes an ordered partitioner.
          *
         ColumnFamily cf = ArrayBackedSortedColumns.factory.create(cfs.metadata);
-        cf.addColumn(new Cell(ByteBufferUtil.bytes("someName"), ByteBufferUtil.bytes("someValue"), 0L));
+        cf.addColumn(new BufferCell(ByteBufferUtil.bytes("someName"), ByteBufferUtil.bytes("someValue"), 0L));
 
         SSTableWriter writer = new SSTableWriter(cfs.getTempSSTablePath(new File(System.getProperty("corrupt-sstable-root"))),
                                                  cfs.metadata.getIndexInterval(),
@@ -327,14 +384,15 @@
         writer.append(Util.dk("c"), cf);
         writer.append(Util.dk("y"), cf);
         writer.append(Util.dk("d"), cf);
-        writer.closeAndOpenReader();
+        writer.finish();
         */
 
         String root = System.getProperty("corrupt-sstable-root");
         assert root != null;
+
         File rootDir = new File(root);
         assert rootDir.isDirectory();
-        Descriptor desc = new Descriptor(new Descriptor.Version("jb"), rootDir, KEYSPACE, columnFamily, 1, Descriptor.Type.FINAL);
+        Descriptor desc = new Descriptor("jb", rootDir, KEYSPACE, columnFamily, 1, Descriptor.Type.FINAL, SSTableFormat.Type.LEGACY);
         CFMetaData metadata = Schema.instance.getCFMetaData(desc.ksname, desc.cfname);
 
         try
@@ -353,11 +411,16 @@
         components.add(Component.STATS);
         components.add(Component.SUMMARY);
         components.add(Component.TOC);
-        SSTableReader sstable = SSTableReader.openNoValidation(desc, components, metadata);
 
-        Scrubber scrubber = new Scrubber(cfs, sstable, false, true, true);
-        scrubber.scrub();
+        SSTableReader sstable = SSTableReader.openNoValidation(desc, components, cfs);
+        if (sstable.last.compareTo(sstable.first) < 0)
+            sstable.last = sstable.first;
 
+        try (LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.SCRUB, sstable);
+             Scrubber scrubber = new Scrubber(cfs, txn, false, true);)
+        {
+            scrubber.scrub();
+        }
         cfs.loadNewSSTables();
         List<Row> rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
         assert isRowOrdered(rows) : "Scrub failed: " + rows;
@@ -370,7 +433,7 @@
         // Table is created by StreamingTransferTest.testTransferRangeTombstones with CASSANDRA-10791 fix disabled.
         CompactionManager.instance.disableAutoCompaction();
         Keyspace keyspace = Keyspace.open(KEYSPACE);
-        String columnFamily = "StandardInteger1";
+        String columnFamily = CFI1;
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(columnFamily);
         cfs.clearUnsafe();
 
@@ -378,8 +441,7 @@
         assert root != null;
         File rootDir = new File(root);
         assert rootDir.isDirectory();
-        Descriptor desc = new Descriptor(new Descriptor.Version("ka"), rootDir, KEYSPACE, columnFamily, 2, Descriptor.Type.FINAL);
-        CFMetaData metadata = Schema.instance.getCFMetaData(desc.ksname, desc.cfname);
+        Descriptor desc = new Descriptor("ka", rootDir, KEYSPACE, columnFamily, 2, Descriptor.Type.FINAL, SSTableFormat.Type.LEGACY);
 
         // open without validation for scrubbing
         Set<Component> components = new HashSet<>();
@@ -389,10 +451,13 @@
         components.add(Component.STATS);
         components.add(Component.SUMMARY);
         components.add(Component.TOC);
-        SSTableReader sstable = SSTableReader.openNoValidation(desc, components, metadata);
+        SSTableReader sstable = SSTableReader.openNoValidation(desc, components, cfs);
 
-        Scrubber scrubber = new Scrubber(cfs, sstable, false, true, true);
-        scrubber.scrub();
+        try (LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.SCRUB, sstable);
+             Scrubber scrubber = new Scrubber(cfs, txn, false, true);)
+        {
+            scrubber.scrub();
+        }
 
         cfs.loadNewSSTables();
         assertEquals(7, countCells(cfs));
@@ -470,7 +535,6 @@
         for (int i = 0; i < rowsPerSSTable; i++)
         {
             String key = String.valueOf(i);
-            // create a row and update the birthdate value, test that the index query fetches the new version
             ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE, CF);
             cf.addColumn(column("c1", "1", 1L));
             cf.addColumn(column("c2", "2", 1L));
@@ -481,6 +545,31 @@
         cfs.forceBlockingFlush();
     }
 
+    private void fillIndexCF(ColumnFamilyStore cfs, boolean composite, long ... values)
+    {
+        assertTrue(values.length % 2 == 0);
+        for (int i = 0; i < values.length; i +=2)
+        {
+            String key = String.valueOf(i);
+            ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE, cfs.name);
+            if (composite)
+            {
+                String clusterKey = "c" + key;
+                cf.addColumn(column(clusterKey, COL_COMPOSITES_INDEX, values[i], 1L));
+                cf.addColumn(column(clusterKey, COL_NON_INDEX, values[i + 1], 1L));
+            }
+            else
+            {
+                cf.addColumn(column(COL_KEYS_INDEX, values[i], 1L));
+                cf.addColumn(column(COL_NON_INDEX, values[i + 1], 1L));
+            }
+            Mutation rm = new Mutation(KEYSPACE, ByteBufferUtil.bytes(key), cf);
+            rm.applyUnsafe();
+        }
+
+        cfs.forceBlockingFlush();
+    }
+
     protected void fillCounterCF(ColumnFamilyStore cfs, int rowsPerSSTable) throws WriteTimeoutException
     {
         for (int i = 0; i < rowsPerSSTable; i++)
@@ -499,12 +588,12 @@
     @Test
     public void testScrubColumnValidation() throws InterruptedException, RequestExecutionException, ExecutionException
     {
-        QueryProcessor.process("CREATE TABLE \"Keyspace1\".test_compact_static_columns (a bigint, b timeuuid, c boolean static, d text, PRIMARY KEY (a, b))", ConsistencyLevel.ONE);
+        QueryProcessor.process(String.format("CREATE TABLE \"%s\".test_compact_static_columns (a bigint, b timeuuid, c boolean static, d text, PRIMARY KEY (a, b))", KEYSPACE), ConsistencyLevel.ONE);
 
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("test_compact_static_columns");
 
-        QueryProcessor.executeInternal("INSERT INTO \"Keyspace1\".test_compact_static_columns (a, b, c, d) VALUES (123, c3db07e8-b602-11e3-bc6b-e0b9a54a6d93, true, 'foobar')");
+        QueryProcessor.executeInternal(String.format("INSERT INTO \"%s\".test_compact_static_columns (a, b, c, d) VALUES (123, c3db07e8-b602-11e3-bc6b-e0b9a54a6d93, true, 'foobar')", KEYSPACE));
         cfs.forceBlockingFlush();
         CompactionManager.instance.performScrub(cfs, false, true, 2);
 
@@ -525,12 +614,12 @@
     @Test
     public void testColumnNameEqualToDefaultKeyAlias() throws ExecutionException, InterruptedException
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("UUIDKeys");
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_UUID);
 
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "UUIDKeys");
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE, CF_UUID);
         cf.addColumn(column(CFMetaData.DEFAULT_KEY_ALIAS, "not a uuid", 1L));
-        Mutation mutation = new Mutation("Keyspace1", ByteBufferUtil.bytes(UUIDGen.getTimeUUID()), cf);
+        Mutation mutation = new Mutation(KEYSPACE, ByteBufferUtil.bytes(UUIDGen.getTimeUUID()), cf);
         mutation.applyUnsafe();
         cfs.forceBlockingFlush();
         CompactionManager.instance.performScrub(cfs, false, true, 2);
@@ -545,19 +634,19 @@
     @Test
     public void testValidationCompactStorage() throws Exception
     {
-        QueryProcessor.process("CREATE TABLE \"Keyspace1\".test_compact_dynamic_columns (a int, b text, c text, PRIMARY KEY (a, b)) WITH COMPACT STORAGE", ConsistencyLevel.ONE);
+        QueryProcessor.process(String.format("CREATE TABLE \"%s\".test_compact_dynamic_columns (a int, b text, c text, PRIMARY KEY (a, b)) WITH COMPACT STORAGE", KEYSPACE), ConsistencyLevel.ONE);
 
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("test_compact_dynamic_columns");
 
-        QueryProcessor.executeInternal("INSERT INTO \"Keyspace1\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'a', 'foo')");
-        QueryProcessor.executeInternal("INSERT INTO \"Keyspace1\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'b', 'bar')");
-        QueryProcessor.executeInternal("INSERT INTO \"Keyspace1\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'c', 'boo')");
+        QueryProcessor.executeInternal(String.format("INSERT INTO \"%s\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'a', 'foo')", KEYSPACE));
+        QueryProcessor.executeInternal(String.format("INSERT INTO \"%s\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'b', 'bar')", KEYSPACE));
+        QueryProcessor.executeInternal(String.format("INSERT INTO \"%s\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'c', 'boo')", KEYSPACE));
         cfs.forceBlockingFlush();
         CompactionManager.instance.performScrub(cfs, true, true, 2);
 
         // Scrub is silent, but it will remove broken records. So reading everything back to make sure nothing to "scrubbed away"
-        UntypedResultSet rs = QueryProcessor.executeInternal("SELECT * FROM \"Keyspace1\".test_compact_dynamic_columns");
+        UntypedResultSet rs = QueryProcessor.executeInternal(String.format("SELECT * FROM \"%s\".test_compact_dynamic_columns", KEYSPACE));
         assertEquals(3, rs.size());
 
         Iterator<UntypedResultSet.Row> iter = rs.iterator();
@@ -565,4 +654,129 @@
         assertEquals("bar", iter.next().getString("c"));
         assertEquals("boo", iter.next().getString("c"));
     }
+
+    @Test /* CASSANDRA-5174 */
+    public void testScrubKeysIndex_preserveOrder() throws IOException, ExecutionException, InterruptedException
+    {
+        //If the partitioner preserves the order then SecondaryIndex uses BytesType comparator,
+        // otherwise it uses LocalByPartitionerType
+        setKeyComparator(BytesType.instance);
+        testScrubIndex(CF_INDEX1, COL_KEYS_INDEX, false, true);
+    }
+
+    @Test /* CASSANDRA-5174 */
+    public void testScrubCompositeIndex_preserveOrder() throws IOException, ExecutionException, InterruptedException
+    {
+        setKeyComparator(BytesType.instance);
+        testScrubIndex(CF_INDEX2, COL_COMPOSITES_INDEX, true, true);
+    }
+
+    @Test /* CASSANDRA-5174 */
+    public void testScrubKeysIndex() throws IOException, ExecutionException, InterruptedException
+    {
+        setKeyComparator(new LocalByPartionerType(StorageService.getPartitioner()));
+        testScrubIndex(CF_INDEX1, COL_KEYS_INDEX, false, true);
+    }
+
+    @Test /* CASSANDRA-5174 */
+    public void testScrubCompositeIndex() throws IOException, ExecutionException, InterruptedException
+    {
+        setKeyComparator(new LocalByPartionerType(StorageService.getPartitioner()));
+        testScrubIndex(CF_INDEX2, COL_COMPOSITES_INDEX, true, true);
+    }
+
+    @Test /* CASSANDRA-5174 */
+    public void testFailScrubKeysIndex() throws IOException, ExecutionException, InterruptedException
+    {
+        testScrubIndex(CF_INDEX1, COL_KEYS_INDEX, false, false);
+    }
+
+    @Test /* CASSANDRA-5174 */
+    public void testFailScrubCompositeIndex() throws IOException, ExecutionException, InterruptedException
+    {
+        testScrubIndex(CF_INDEX2, COL_COMPOSITES_INDEX, true, false);
+    }
+
+    @Test /* CASSANDRA-5174 */
+    public void testScrubTwice() throws IOException, ExecutionException, InterruptedException
+    {
+        testScrubIndex(CF_INDEX1, COL_KEYS_INDEX, false, true, true);
+    }
+
+    /** The SecondaryIndex class is used for custom indexes so to avoid
+     * making a public final field into a private field with getters
+     * and setters, we resort to this hack in order to test it properly
+     * since it can have two values which influence the scrubbing behavior.
+     * @param comparator - the key comparator we want to test
+     */
+    private void setKeyComparator(AbstractType<?> comparator)
+    {
+        try
+        {
+            Field keyComparator = SecondaryIndex.class.getDeclaredField("keyComparator");
+            keyComparator.setAccessible(true);
+            int modifiers = keyComparator.getModifiers();
+            Field modifierField = keyComparator.getClass().getDeclaredField("modifiers");
+            modifiers = modifiers & ~Modifier.FINAL;
+            modifierField.setAccessible(true);
+            modifierField.setInt(keyComparator, modifiers);
+
+            keyComparator.set(null, comparator);
+        }
+        catch (Exception ex)
+        {
+            fail("Failed to change key comparator in secondary index : " + ex.getMessage());
+            ex.printStackTrace();
+        }
+    }
+
+    private void testScrubIndex(String cfName, String colName, boolean composite, boolean ... scrubs)
+            throws IOException, ExecutionException, InterruptedException
+    {
+        CompactionManager.instance.disableAutoCompaction();
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
+        cfs.clearUnsafe();
+
+        int numRows = 1000;
+        long[] colValues = new long [numRows * 2]; // each row has two columns
+        for (int i = 0; i < colValues.length; i+=2)
+        {
+            colValues[i] = (i % 4 == 0 ? 1L : 2L); // index column
+            colValues[i+1] = 3L; //other column
+        }
+        fillIndexCF(cfs, composite, colValues);
+
+        // check index
+        IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes(colName), Operator.EQ, ByteBufferUtil.bytes(1L));
+        List<Row> rows = cfs.search(Util.range("", ""), Arrays.asList(expr), new IdentityQueryFilter(), numRows);
+        assertNotNull(rows);
+        assertEquals(numRows / 2, rows.size());
+
+        // scrub index
+        Set<ColumnFamilyStore> indexCfss = cfs.indexManager.getIndexesBackedByCfs();
+        assertTrue(indexCfss.size() == 1);
+        for(ColumnFamilyStore indexCfs : indexCfss)
+        {
+            for (int i = 0; i < scrubs.length; i++)
+            {
+                boolean failure = !scrubs[i];
+                if (failure)
+                { //make sure the next scrub fails
+                    overrideWithGarbage(indexCfs.getSSTables().iterator().next(), ByteBufferUtil.bytes(1L), ByteBufferUtil.bytes(2L));
+                }
+                CompactionManager.AllSSTableOpStatus result = indexCfs.scrub(false, false, true, true, 0);
+                assertEquals(failure ?
+                             CompactionManager.AllSSTableOpStatus.ABORTED :
+                             CompactionManager.AllSSTableOpStatus.SUCCESSFUL,
+                                result);
+            }
+        }
+
+
+        // check index is still working
+        rows = cfs.search(Util.range("", ""), Arrays.asList(expr), new IdentityQueryFilter(), numRows);
+        assertNotNull(rows);
+        assertEquals(numRows / 2, rows.size());
+    }
 }
diff --git a/test/unit/org/apache/cassandra/db/SerializationsTest.java b/test/unit/org/apache/cassandra/db/SerializationsTest.java
index 0dd6b8f..a280448 100644
--- a/test/unit/org/apache/cassandra/db/SerializationsTest.java
+++ b/test/unit/org/apache/cassandra/db/SerializationsTest.java
@@ -19,23 +19,28 @@
 package org.apache.cassandra.db;
 
 import org.apache.cassandra.AbstractSerializationsTester;
+import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.db.filter.*;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.LongType;
 import org.apache.cassandra.dht.AbstractBounds;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.io.util.DataOutputStreamAndChannel;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.util.DataOutputStreamPlus;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.net.CallbackInfo;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.ByteBufferUtil;
-
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import java.io.DataInputStream;
@@ -47,6 +52,7 @@
 {
     Statics statics = new Statics();
 
+    private static final String KEYSPACE1 = "Keyspace1";
     private ByteBuffer startCol = ByteBufferUtil.bytes("Start");
     private ByteBuffer stopCol = ByteBufferUtil.bytes("Stop");
     private Composite emptyCol = Composites.EMPTY;
@@ -56,10 +62,21 @@
     public SliceQueryFilter nonEmptyRangePred = new SliceQueryFilter(CellNames.simpleDense(startCol), CellNames.simpleDense(stopCol), true, 100);
     public SliceQueryFilter nonEmptyRangeSCPred = new SliceQueryFilter(CellNames.compositeDense(statics.SC, startCol), CellNames.compositeDense(statics.SC, stopCol), true, 100);
 
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, "Standard1"),
+                                    SchemaLoader.superCFMD(KEYSPACE1, "Super1", LongType.instance));
+    }
+
     private void testRangeSliceCommandWrite() throws IOException
     {
         IPartitioner part = StorageService.getPartitioner();
-        AbstractBounds<RowPosition> bounds = new Range<Token>(part.getRandomToken(), part.getRandomToken()).toRowBounds();
+        AbstractBounds<RowPosition> bounds = Range.makeRowRange(part.getRandomToken(), part.getRandomToken());
 
         RangeSliceCommand namesCmd = new RangeSliceCommand(statics.KS, "Standard1", statics.readTs, namesPred, bounds, 100);
         MessageOut<RangeSliceCommand> namesCmdMsg = namesCmd.createMessage();
@@ -74,7 +91,7 @@
         RangeSliceCommand regRangeCmdSup = new RangeSliceCommand(statics.KS, "Super1", statics.readTs, nonEmptyRangeSCPred, bounds, 100);
         MessageOut<RangeSliceCommand> regRangeCmdSupMsg = regRangeCmdSup.createMessage();
 
-        DataOutputStreamAndChannel out = getOutput("db.RangeSliceCommand.bin");
+        DataOutputStreamPlus out = getOutput("db.RangeSliceCommand.bin");
         namesCmdMsg.serialize(out, getVersion());
         emptyRangeCmdMsg.serialize(out, getVersion());
         regRangeCmdMsg.serialize(out, getVersion());
@@ -109,7 +126,7 @@
         SliceByNamesReadCommand standardCmd = new SliceByNamesReadCommand(statics.KS, statics.Key, statics.StandardCF, statics.readTs, namesPred);
         SliceByNamesReadCommand superCmd = new SliceByNamesReadCommand(statics.KS, statics.Key, statics.SuperCF, statics.readTs, namesSCPred);
 
-        DataOutputStreamAndChannel out = getOutput("db.SliceByNamesReadCommand.bin");
+        DataOutputStreamPlus out = getOutput("db.SliceByNamesReadCommand.bin");
         SliceByNamesReadCommand.serializer.serialize(standardCmd, out, getVersion());
         SliceByNamesReadCommand.serializer.serialize(superCmd, out, getVersion());
         ReadCommand.serializer.serialize(standardCmd, out, getVersion());
@@ -143,8 +160,8 @@
     {
         SliceFromReadCommand standardCmd = new SliceFromReadCommand(statics.KS, statics.Key, statics.StandardCF, statics.readTs, nonEmptyRangePred);
         SliceFromReadCommand superCmd = new SliceFromReadCommand(statics.KS, statics.Key, statics.SuperCF, statics.readTs, nonEmptyRangeSCPred);
-        
-        DataOutputStreamAndChannel out = getOutput("db.SliceFromReadCommand.bin");
+
+        DataOutputStreamPlus out = getOutput("db.SliceFromReadCommand.bin");
         SliceFromReadCommand.serializer.serialize(standardCmd, out, getVersion());
         SliceFromReadCommand.serializer.serialize(superCmd, out, getVersion());
         ReadCommand.serializer.serialize(standardCmd, out, getVersion());
@@ -177,7 +194,7 @@
 
     private void testRowWrite() throws IOException
     {
-        DataOutputStreamAndChannel out = getOutput("db.Row.bin");
+        DataOutputStreamPlus out = getOutput("db.Row.bin");
         Row.serializer.serialize(statics.StandardRow, out, getVersion());
         Row.serializer.serialize(statics.SuperRow, out, getVersion());
         Row.serializer.serialize(statics.NullRow, out, getVersion());
@@ -214,7 +231,7 @@
         mods.put(statics.SuperCf.metadata().cfId, statics.SuperCf);
         Mutation mixedRm = new Mutation(statics.KS, statics.Key, mods);
 
-        DataOutputStreamAndChannel out = getOutput("db.RowMutation.bin");
+        DataOutputStreamPlus out = getOutput("db.RowMutation.bin");
         Mutation.serializer.serialize(standardRowRm, out, getVersion());
         Mutation.serializer.serialize(superRowRm, out, getVersion());
         Mutation.serializer.serialize(standardRm, out, getVersion());
@@ -263,7 +280,7 @@
         Truncation tr = new Truncation(statics.KS, "Doesn't Really Matter");
         TruncateResponse aff = new TruncateResponse(statics.KS, "Doesn't Matter Either", true);
         TruncateResponse neg = new TruncateResponse(statics.KS, "Still Doesn't Matter", false);
-        DataOutputStreamAndChannel out = getOutput("db.Truncation.bin");
+        DataOutputStreamPlus out = getOutput("db.Truncation.bin");
         Truncation.serializer.serialize(tr, out, getVersion());
         TruncateResponse.serializer.serialize(aff, out, getVersion());
         TruncateResponse.serializer.serialize(neg, out, getVersion());
@@ -293,8 +310,8 @@
         assert MessageIn.read(in, getVersion(), -1) != null;
 
         // set up some fake callbacks so deserialization knows that what it's deserializing is a TruncateResponse
-        MessagingService.instance().setCallbackForTests(1, new CallbackInfo(null, null, TruncateResponse.serializer));
-        MessagingService.instance().setCallbackForTests(2, new CallbackInfo(null, null, TruncateResponse.serializer));
+        MessagingService.instance().setCallbackForTests(1, new CallbackInfo(null, null, TruncateResponse.serializer, false));
+        MessagingService.instance().setCallbackForTests(2, new CallbackInfo(null, null, TruncateResponse.serializer, false));
 
         assert MessageIn.read(in, getVersion(), 1) != null;
         assert MessageIn.read(in, getVersion(), 2) != null;
@@ -305,7 +322,7 @@
     {
         WriteResponse aff = new WriteResponse();
         WriteResponse neg = new WriteResponse();
-        DataOutputStreamAndChannel out = getOutput("db.WriteResponse.bin");
+        DataOutputStreamPlus out = getOutput("db.WriteResponse.bin");
         WriteResponse.serializer.serialize(aff, out, getVersion());
         WriteResponse.serializer.serialize(neg, out, getVersion());
         out.close();
@@ -339,7 +356,7 @@
 
     private static class Statics
     {
-        private final String KS = "Keyspace1";
+        private final String KS = KEYSPACE1;
         private final ByteBuffer Key = ByteBufferUtil.bytes("Key01");
         private final SortedSet<CellName> NamedCols = new TreeSet<CellName>(new SimpleDenseCellNameType(BytesType.instance))
         {{
diff --git a/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java b/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
index 3eebe98..c3047b8 100644
--- a/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
+++ b/test/unit/org/apache/cassandra/db/SystemKeyspaceTest.java
@@ -1,6 +1,4 @@
-package org.apache.cassandra.db;
 /*
- *
  * 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
@@ -9,35 +7,47 @@
  * "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
+ *     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.
- *
+ * 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.db;
 
-
+import java.io.IOException;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Collection;
-import java.util.List;
-import java.util.UUID;
+import java.util.*;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
-import org.apache.cassandra.dht.BytesToken;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.dht.ByteOrderedPartitioner.BytesToken;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.CassandraVersion;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 public class SystemKeyspaceTest
 {
+    @BeforeClass
+    public static void prepSnapshotTracker()
+    {
+        DatabaseDescriptor.setDaemonInitialized();
+
+        if (FBUtilities.isWindows())
+            WindowsFailedSnapshotTracker.deleteOldSnapshots();
+    }
+
     @Test
     public void testLocalTokens()
     {
@@ -77,4 +87,100 @@
         UUID secondId = SystemKeyspace.getLocalHostId();
         assert firstId.equals(secondId) : String.format("%s != %s%n", firstId.toString(), secondId.toString());
     }
+
+    private void assertDeletedOrDeferred(int expectedCount)
+    {
+        if (FBUtilities.isWindows())
+            assertEquals(expectedCount, getDeferredDeletionCount());
+        else
+            assertTrue(getSystemSnapshotFiles().isEmpty());
+    }
+
+    private int getDeferredDeletionCount()
+    {
+        try
+        {
+            Class c = Class.forName("java.io.DeleteOnExitHook");
+            LinkedHashSet<String> files = (LinkedHashSet<String>)FBUtilities.getProtectedField(c, "files").get(c);
+            return files.size();
+        }
+        catch (Exception e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Test
+    public void snapshotSystemKeyspaceIfUpgrading() throws IOException
+    {
+        // First, check that in the absence of any previous installed version, we don't create snapshots
+        for (ColumnFamilyStore cfs : Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStores())
+            cfs.clearUnsafe();
+        Keyspace.clearSnapshot(null, SystemKeyspace.NAME);
+
+        int baseline = getDeferredDeletionCount();
+
+        SystemKeyspace.snapshotOnVersionChange();
+        assertDeletedOrDeferred(baseline);
+
+        // now setup system.local as if we're upgrading from a previous version
+        setupReleaseVersion(getOlderVersionString());
+        Keyspace.clearSnapshot(null, SystemKeyspace.NAME);
+        assertDeletedOrDeferred(baseline);
+
+        // Compare versions again & verify that snapshots were created for all tables in the system ks
+        SystemKeyspace.snapshotOnVersionChange();
+        assertEquals(SystemKeyspace.definition().cfMetaData().size(), getSystemSnapshotFiles().size());
+
+        // clear out the snapshots & set the previous recorded version equal to the latest, we shouldn't
+        // see any new snapshots created this time.
+        Keyspace.clearSnapshot(null, SystemKeyspace.NAME);
+        setupReleaseVersion(FBUtilities.getReleaseVersionString());
+
+        SystemKeyspace.snapshotOnVersionChange();
+
+        // snapshotOnVersionChange for upgrade case will open a SSTR when the CFS is flushed. On Windows, we won't be
+        // able to delete hard-links to that file while segments are memory-mapped, so they'll be marked for deferred deletion.
+        // 10 files expected.
+        assertDeletedOrDeferred(baseline + 10);
+
+        Keyspace.clearSnapshot(null, SystemKeyspace.NAME);
+    }
+
+    private String getOlderVersionString()
+    {
+        String version = FBUtilities.getReleaseVersionString();
+        CassandraVersion semver = new CassandraVersion(version.contains("-") ? version.substring(0, version.indexOf('-'))
+                                                                           : version);
+        return (String.format("%s.%s.%s", semver.major - 1, semver.minor, semver.patch));
+    }
+
+    private Set<String> getSystemSnapshotFiles()
+    {
+        Set<String> snapshottedTableNames = new HashSet<>();
+        for (ColumnFamilyStore cfs : Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStores())
+        {
+            if (!cfs.getSnapshotDetails().isEmpty())
+                snapshottedTableNames.add(cfs.getColumnFamilyName());
+        }
+        return snapshottedTableNames;
+    }
+
+    private void setupReleaseVersion(String version)
+    {
+        // besides the release_version, we also need to insert the cluster_name or the check
+        // in SystemKeyspace.checkHealth were we verify it matches DatabaseDescriptor will fail
+        QueryProcessor.executeInternal(String.format("INSERT INTO system.local(key, release_version, cluster_name) " +
+                                                     "VALUES ('local', '%s', '%s')",
+                                                     version,
+                                                     DatabaseDescriptor.getClusterName()));
+        String r = readLocalVersion();
+        assertEquals(String.format("Expected %s, got %s", version, r), version, r);
+    }
+
+    private String readLocalVersion()
+    {
+        UntypedResultSet rs = QueryProcessor.executeInternal("SELECT release_version FROM system.local WHERE key='local'");
+        return rs.isEmpty() || !rs.one().has("release_version") ? null : rs.one().getString("release_version");
+    }
 }
diff --git a/test/unit/org/apache/cassandra/db/TimeSortTest.java b/test/unit/org/apache/cassandra/db/TimeSortTest.java
index 80c0ff1..1d9fb10 100644
--- a/test/unit/org/apache/cassandra/db/TimeSortTest.java
+++ b/test/unit/org/apache/cassandra/db/TimeSortTest.java
@@ -21,6 +21,7 @@
 import java.io.IOException;
 import java.util.*;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 import static org.junit.Assert.assertEquals;
 
@@ -29,29 +30,45 @@
 import static org.apache.cassandra.Util.getBytes;
 import org.apache.cassandra.Util;
 
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.db.filter.QueryFilter;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 
-public class TimeSortTest extends SchemaLoader
+public class TimeSortTest
 {
+    private static final String KEYSPACE1 = "TimeSortTest";
+    private static final String CF_STANDARD1 = "StandardLong1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1));
+    }
+
     @Test
     public void testMixedSources()
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore("StandardLong1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore(CF_STANDARD1);
         Mutation rm;
         DecoratedKey key = Util.dk("key0");
 
-        rm = new Mutation("Keyspace1", key.getKey());
-        rm.add("StandardLong1", cellname(100), ByteBufferUtil.bytes("a"), 100);
-        rm.apply();
+        rm = new Mutation(KEYSPACE1, key.getKey());
+        rm.add(CF_STANDARD1, cellname(100), ByteBufferUtil.bytes("a"), 100);
+        rm.applyUnsafe();
         cfStore.forceBlockingFlush();
 
-        rm = new Mutation("Keyspace1", key.getKey());
-        rm.add("StandardLong1", cellname(0), ByteBufferUtil.bytes("b"), 0);
-        rm.apply();
+        rm = new Mutation(KEYSPACE1, key.getKey());
+        rm.add(CF_STANDARD1, cellname(0), ByteBufferUtil.bytes("b"), 0);
+        rm.applyUnsafe();
 
         ColumnFamily cf = cfStore.getColumnFamily(key, cellname(10), Composites.EMPTY, false, 1000, System.currentTimeMillis());
         Collection<Cell> cells = cf.getSortedColumns();
@@ -61,17 +78,17 @@
     @Test
     public void testTimeSort() throws IOException
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore("StandardLong1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore(CF_STANDARD1);
 
         for (int i = 900; i < 1000; ++i)
         {
-            Mutation rm = new Mutation("Keyspace1", ByteBufferUtil.bytes(Integer.toString(i)));
+            Mutation rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes(Integer.toString(i)));
             for (int j = 0; j < 8; ++j)
             {
-                rm.add("StandardLong1", cellname(j * 2), ByteBufferUtil.bytes("a"), j * 2);
+                rm.add(CF_STANDARD1, cellname(j * 2), ByteBufferUtil.bytes("a"), j * 2);
             }
-            rm.apply();
+            rm.applyUnsafe();
         }
 
         validateTimeSort(keyspace);
@@ -81,17 +98,17 @@
 
         // interleave some new data to test memtable + sstable
         DecoratedKey key = Util.dk("900");
-        Mutation rm = new Mutation("Keyspace1", key.getKey());
+        Mutation rm = new Mutation(KEYSPACE1, key.getKey());
         for (int j = 0; j < 4; ++j)
         {
-            rm.add("StandardLong1", cellname(j * 2 + 1), ByteBufferUtil.bytes("b"), j * 2 + 1);
+            rm.add(CF_STANDARD1, cellname(j * 2 + 1), ByteBufferUtil.bytes("b"), j * 2 + 1);
         }
-        rm.apply();
+        rm.applyUnsafe();
         // and some overwrites
-        rm = new Mutation("Keyspace1", key.getKey());
-        rm.add("StandardLong1", cellname(0), ByteBufferUtil.bytes("c"), 100);
-        rm.add("StandardLong1", cellname(10), ByteBufferUtil.bytes("c"), 100);
-        rm.apply();
+        rm = new Mutation(KEYSPACE1, key.getKey());
+        rm.add(CF_STANDARD1, cellname(0), ByteBufferUtil.bytes("c"), 100);
+        rm.add(CF_STANDARD1, cellname(10), ByteBufferUtil.bytes("c"), 100);
+        rm.applyUnsafe();
 
         // verify
         ColumnFamily cf = cfStore.getColumnFamily(key, cellname(0), Composites.EMPTY, false, 1000, System.currentTimeMillis());
@@ -107,7 +124,7 @@
         TreeSet<CellName> columnNames = new TreeSet<CellName>(cfStore.getComparator());
         columnNames.add(cellname(10));
         columnNames.add(cellname(0));
-        cf = cfStore.getColumnFamily(QueryFilter.getNamesFilter(Util.dk("900"), "StandardLong1", columnNames, System.currentTimeMillis()));
+        cf = cfStore.getColumnFamily(QueryFilter.getNamesFilter(Util.dk("900"), CF_STANDARD1, columnNames, System.currentTimeMillis()));
         assert "c".equals(ByteBufferUtil.string(cf.getColumn(cellname(0)).value()));
         assert "c".equals(ByteBufferUtil.string(cf.getColumn(cellname(10)).value()));
     }
@@ -119,7 +136,7 @@
             DecoratedKey key = Util.dk(Integer.toString(i));
             for (int j = 0; j < 8; j += 3)
             {
-                ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("StandardLong1");
+                ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_STANDARD1);
                 ColumnFamily cf = cfs.getColumnFamily(key, cellname(j * 2), Composites.EMPTY, false, 1000, System.currentTimeMillis());
                 Collection<Cell> cells = cf.getSortedColumns();
                 assert cells.size() == 8 - j;
diff --git a/test/unit/org/apache/cassandra/db/VerifyTest.java b/test/unit/org/apache/cassandra/db/VerifyTest.java
new file mode 100644
index 0000000..27e99ab
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/VerifyTest.java
@@ -0,0 +1,427 @@
+package org.apache.cassandra.db;
+/*
+ *
+ * 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.
+ *
+ */
+
+import com.google.common.base.Charsets;
+import org.apache.cassandra.OrderedJUnit4ClassRunner;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.Util;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.db.columniterator.IdentityQueryFilter;
+import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.db.compaction.Verifier;
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.CounterColumnType;
+import org.apache.cassandra.db.marshal.UUIDType;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.WriteTimeoutException;
+import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.io.compress.*;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.commons.lang3.StringUtils;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import java.io.*;
+import java.nio.file.Files;
+import java.util.HashMap;
+import java.util.List;
+import java.util.zip.Adler32;
+import java.util.zip.CheckedInputStream;
+
+import static org.apache.cassandra.Util.cellname;
+import static org.apache.cassandra.Util.column;
+import static org.junit.Assert.fail;
+
+@RunWith(OrderedJUnit4ClassRunner.class)
+public class VerifyTest
+{
+    public static final String KEYSPACE = "Keyspace1";
+    public static final String CF = "Standard1";
+    public static final String CF2 = "Standard2";
+    public static final String CF3 = "Standard3";
+    public static final String CF4 = "Standard4";
+    public static final String COUNTER_CF = "Counter1";
+    public static final String COUNTER_CF2 = "Counter2";
+    public static final String COUNTER_CF3 = "Counter3";
+    public static final String COUNTER_CF4 = "Counter4";
+    public static final String CORRUPT_CF = "Corrupt1";
+    public static final String CORRUPT_CF2 = "Corrupt2";
+    public static final String CORRUPTCOUNTER_CF = "CounterCorrupt1";
+    public static final String CORRUPTCOUNTER_CF2 = "CounterCorrupt2";
+
+    public static final String CF_UUID = "UUIDKeys";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        CompressionParameters compressionParameters = new CompressionParameters(SnappyCompressor.instance, 32768, new HashMap<String, String>());
+
+        SchemaLoader.loadSchema();
+        SchemaLoader.createKeyspace(KEYSPACE,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF).compressionParameters(compressionParameters),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF2).compressionParameters(compressionParameters),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF3),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF4),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CORRUPT_CF),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CORRUPT_CF2),
+                                    SchemaLoader.standardCFMD(KEYSPACE, COUNTER_CF, BytesType.instance).defaultValidator(CounterColumnType.instance).compressionParameters(compressionParameters),
+                                    SchemaLoader.standardCFMD(KEYSPACE, COUNTER_CF2, BytesType.instance).defaultValidator(CounterColumnType.instance).compressionParameters(compressionParameters),
+                                    SchemaLoader.standardCFMD(KEYSPACE, COUNTER_CF3, BytesType.instance).defaultValidator(CounterColumnType.instance),
+                                    SchemaLoader.standardCFMD(KEYSPACE, COUNTER_CF4, BytesType.instance).defaultValidator(CounterColumnType.instance),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CORRUPTCOUNTER_CF, BytesType.instance).defaultValidator(CounterColumnType.instance),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CORRUPTCOUNTER_CF2, BytesType.instance).defaultValidator(CounterColumnType.instance),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF_UUID).keyValidator(UUIDType.instance));
+    }
+
+
+    @Test
+    public void testVerifyCorrect() throws IOException
+    {
+        CompactionManager.instance.disableAutoCompaction();
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
+
+        fillCF(cfs, KEYSPACE, CF, 2);
+
+        SSTableReader sstable = cfs.getSSTables().iterator().next();
+
+        try(Verifier verifier = new Verifier(cfs, sstable, false))
+        {
+            verifier.verify(false);
+        }
+        catch (CorruptSSTableException err)
+        {
+            fail("Unexpected CorruptSSTableException");
+        }
+    }
+
+    @Test
+    public void testVerifyCounterCorrect() throws IOException
+    {
+        CompactionManager.instance.disableAutoCompaction();
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(COUNTER_CF);
+
+        fillCounterCF(cfs, KEYSPACE, COUNTER_CF, 2);
+
+        SSTableReader sstable = cfs.getSSTables().iterator().next();
+
+        Verifier verifier = new Verifier(cfs, sstable, false);
+        try
+        {
+            verifier.verify(false);
+        }
+        catch (CorruptSSTableException err)
+        {
+            fail("Unexpected CorruptSSTableException");
+        }
+    }
+
+    @Test
+    public void testExtendedVerifyCorrect() throws IOException
+    {
+        CompactionManager.instance.disableAutoCompaction();
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF2);
+
+        fillCF(cfs, KEYSPACE, CF2, 2);
+
+        SSTableReader sstable = cfs.getSSTables().iterator().next();
+
+        Verifier verifier = new Verifier(cfs, sstable, false);
+        try
+        {
+            verifier.verify(true);
+        }
+        catch (CorruptSSTableException err)
+        {
+            fail("Unexpected CorruptSSTableException");
+        }
+    }
+
+    @Test
+    public void testExtendedVerifyCounterCorrect() throws IOException
+    {
+        CompactionManager.instance.disableAutoCompaction();
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(COUNTER_CF2);
+
+        fillCounterCF(cfs, KEYSPACE, COUNTER_CF2, 2);
+
+        SSTableReader sstable = cfs.getSSTables().iterator().next();
+
+        Verifier verifier = new Verifier(cfs, sstable, false);
+        try
+        {
+            verifier.verify(true);
+        }
+        catch (CorruptSSTableException err)
+        {
+            fail("Unexpected CorruptSSTableException");
+        }
+    }
+
+    @Test
+    public void testVerifyCorrectUncompressed() throws IOException
+    {
+        CompactionManager.instance.disableAutoCompaction();
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF3);
+
+        fillCF(cfs, KEYSPACE, CF3, 2);
+
+        SSTableReader sstable = cfs.getSSTables().iterator().next();
+
+        Verifier verifier = new Verifier(cfs, sstable, false);
+        try
+        {
+            verifier.verify(false);
+        }
+        catch (CorruptSSTableException err)
+        {
+            fail("Unexpected CorruptSSTableException");
+        }
+    }
+
+    @Test
+    public void testVerifyCounterCorrectUncompressed() throws IOException
+    {
+        CompactionManager.instance.disableAutoCompaction();
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(COUNTER_CF3);
+
+        fillCounterCF(cfs, KEYSPACE, COUNTER_CF3, 2);
+
+        SSTableReader sstable = cfs.getSSTables().iterator().next();
+
+        Verifier verifier = new Verifier(cfs, sstable, false);
+        try
+        {
+            verifier.verify(false);
+        }
+        catch (CorruptSSTableException err)
+        {
+            fail("Unexpected CorruptSSTableException");
+        }
+    }
+
+    @Test
+    public void testExtendedVerifyCorrectUncompressed() throws IOException
+    {
+        CompactionManager.instance.disableAutoCompaction();
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF4);
+
+        fillCF(cfs, KEYSPACE, CF4, 2);
+
+        SSTableReader sstable = cfs.getSSTables().iterator().next();
+
+        Verifier verifier = new Verifier(cfs, sstable, false);
+        try
+        {
+            verifier.verify(true);
+        }
+        catch (CorruptSSTableException err)
+        {
+            fail("Unexpected CorruptSSTableException");
+        }
+    }
+
+    @Test
+    public void testExtendedVerifyCounterCorrectUncompressed() throws IOException
+    {
+        CompactionManager.instance.disableAutoCompaction();
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(COUNTER_CF4);
+
+        fillCounterCF(cfs, KEYSPACE, COUNTER_CF4, 2);
+
+        SSTableReader sstable = cfs.getSSTables().iterator().next();
+
+        Verifier verifier = new Verifier(cfs, sstable, false);
+        try
+        {
+            verifier.verify(true);
+        }
+        catch (CorruptSSTableException err)
+        {
+            fail("Unexpected CorruptSSTableException");
+        }
+    }
+
+
+    @Test
+    public void testVerifyIncorrectDigest() throws IOException, WriteTimeoutException
+    {
+        CompactionManager.instance.disableAutoCompaction();
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CORRUPT_CF);
+
+        fillCF(cfs, KEYSPACE, CORRUPT_CF, 2);
+
+        List<Row> rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
+
+        SSTableReader sstable = cfs.getSSTables().iterator().next();
+
+
+        RandomAccessFile file = new RandomAccessFile(sstable.descriptor.filenameFor(Component.DIGEST), "rw");
+        Long correctChecksum = Long.parseLong(file.readLine());
+        file.close();
+
+        writeChecksum(++correctChecksum, sstable.descriptor.filenameFor(Component.DIGEST));
+
+        Verifier verifier = new Verifier(cfs, sstable, false);
+        try
+        {
+            verifier.verify(false);
+            fail("Expected a CorruptSSTableException to be thrown");
+        }
+        catch (CorruptSSTableException err) {}
+    }
+
+
+    @Test
+    public void testVerifyCorruptRowCorrectDigest() throws IOException, WriteTimeoutException
+    {
+        CompactionManager.instance.disableAutoCompaction();
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CORRUPT_CF2);
+
+        fillCF(cfs, KEYSPACE, CORRUPT_CF2, 2);
+
+        List<Row> rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
+
+        SSTableReader sstable = cfs.getSSTables().iterator().next();
+
+        // overwrite one row with garbage
+        long row0Start = sstable.getPosition(RowPosition.ForKey.get(ByteBufferUtil.bytes("0"), sstable.partitioner), SSTableReader.Operator.EQ).position;
+        long row1Start = sstable.getPosition(RowPosition.ForKey.get(ByteBufferUtil.bytes("1"), sstable.partitioner), SSTableReader.Operator.EQ).position;
+        long startPosition = row0Start < row1Start ? row0Start : row1Start;
+        long endPosition = row0Start < row1Start ? row1Start : row0Start;
+
+        RandomAccessFile file = new RandomAccessFile(sstable.getFilename(), "rw");
+        file.seek(startPosition);
+        file.writeBytes(StringUtils.repeat('z', (int) 2));
+        file.close();
+
+        // Update the Digest to have the right Checksum
+        writeChecksum(simpleFullChecksum(sstable.getFilename()), sstable.descriptor.filenameFor(Component.DIGEST));
+
+        Verifier verifier = new Verifier(cfs, sstable, false);
+
+        // First a simple verify checking digest, which should succeed
+        try
+        {
+            verifier.verify(false);
+        }
+        catch (CorruptSSTableException err)
+        {
+            fail("Simple verify should have succeeded as digest matched");
+        }
+
+        // Now try extended verify
+        try
+        {
+            verifier.verify(true);
+
+        }
+        catch (CorruptSSTableException err)
+        {
+            return;
+        }
+        fail("Expected a CorruptSSTableException to be thrown");
+
+    }
+
+    protected void fillCF(ColumnFamilyStore cfs, String keyspace, String columnFamily, int rowsPerSSTable)
+    {
+        for (int i = 0; i < rowsPerSSTable; i++)
+        {
+            String key = String.valueOf(i);
+            ColumnFamily cf = ArrayBackedSortedColumns.factory.create(keyspace, columnFamily);
+            cf.addColumn(column("c1", "1", 1L));
+            cf.addColumn(column("c2", "2", 1L));
+            Mutation rm = new Mutation(keyspace, ByteBufferUtil.bytes(key), cf);
+            rm.apply();
+        }
+
+        cfs.forceBlockingFlush();
+    }
+
+    protected void fillCounterCF(ColumnFamilyStore cfs, String keyspace, String columnFamily, int rowsPerSSTable) throws WriteTimeoutException
+    {
+        for (int i = 0; i < rowsPerSSTable; i++)
+        {
+            String key = String.valueOf(i);
+            ColumnFamily cf = ArrayBackedSortedColumns.factory.create(keyspace, columnFamily);
+            Mutation rm = new Mutation(keyspace, ByteBufferUtil.bytes(key), cf);
+            rm.addCounter(columnFamily, cellname("Column1"), 100);
+            CounterMutation cm = new CounterMutation(rm, ConsistencyLevel.ONE);
+            cm.apply();
+        }
+
+        cfs.forceBlockingFlush();
+    }
+
+    protected long simpleFullChecksum(String filename) throws IOException
+    {
+        FileInputStream inputStream = new FileInputStream(filename);
+        Adler32 adlerChecksum = new Adler32();
+        CheckedInputStream cinStream = new CheckedInputStream(inputStream, adlerChecksum);
+        byte[] b = new byte[128];
+        while (cinStream.read(b) >= 0) {
+        }
+        return cinStream.getChecksum().getValue();
+    }
+
+    protected void writeChecksum(long checksum, String filePath)
+    {
+        File outFile = new File(filePath);
+        BufferedWriter out = null;
+        try
+        {
+            out = Files.newBufferedWriter(outFile.toPath(), Charsets.UTF_8);
+            out.write(String.valueOf(checksum));
+            out.flush();
+            out.close();
+        }
+        catch (IOException e)
+        {
+            throw new FSWriteError(e, outFile);
+        }
+        finally
+        {
+            FileUtils.closeQuietly(out);
+        }
+
+    }
+
+}
diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogDescriptorTest.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogDescriptorTest.java
new file mode 100644
index 0000000..8d63959
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogDescriptorTest.java
@@ -0,0 +1,103 @@
+/*
+ * 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.db.commitlog;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+
+import com.google.common.collect.ImmutableMap;
+
+import org.junit.Test;
+
+import org.apache.cassandra.config.ParameterizedClass;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.util.ByteBufferDataInput;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.net.MessagingService;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class CommitLogDescriptorTest
+{
+    @Test
+    public void testVersions()
+    {
+        assertTrue(CommitLogDescriptor.isValid("CommitLog-1340512736956320000.log"));
+        assertTrue(CommitLogDescriptor.isValid("CommitLog-2-1340512736956320000.log"));
+        assertFalse(CommitLogDescriptor.isValid("CommitLog--1340512736956320000.log"));
+        assertFalse(CommitLogDescriptor.isValid("CommitLog--2-1340512736956320000.log"));
+        assertFalse(CommitLogDescriptor.isValid("CommitLog-2-1340512736956320000-123.log"));
+
+        assertEquals(1340512736956320000L, CommitLogDescriptor.fromFileName("CommitLog-2-1340512736956320000.log").id);
+
+        assertEquals(MessagingService.current_version, new CommitLogDescriptor(1340512736956320000L, null).getMessagingVersion());
+        String newCLName = "CommitLog-" + CommitLogDescriptor.current_version + "-1340512736956320000.log";
+        assertEquals(MessagingService.current_version, CommitLogDescriptor.fromFileName(newCLName).getMessagingVersion());
+    }
+
+    private void testDescriptorPersistence(CommitLogDescriptor desc) throws IOException
+    {
+        ByteBuffer buf = ByteBuffer.allocate(1024);
+        CommitLogDescriptor.writeHeader(buf, desc);
+        long length = buf.position();
+        // Put some extra data in the stream.
+        buf.putDouble(0.1);
+        buf.flip();
+        try (FileDataInput input = new ByteBufferDataInput(buf, "input", 0, 0))
+        {
+            CommitLogDescriptor read = CommitLogDescriptor.readHeader(input);
+            assertEquals("Descriptor length", length, input.getFilePointer());
+            assertEquals("Descriptors", desc, read);
+        }
+    }
+
+    @Test
+    public void testDescriptorPersistence() throws IOException
+    {
+        testDescriptorPersistence(new CommitLogDescriptor(11, null));
+        testDescriptorPersistence(new CommitLogDescriptor(CommitLogDescriptor.VERSION_21, 13, null));
+        testDescriptorPersistence(new CommitLogDescriptor(CommitLogDescriptor.VERSION_22, 15, null));
+        testDescriptorPersistence(new CommitLogDescriptor(CommitLogDescriptor.VERSION_22, 17, new ParameterizedClass("LZ4Compressor", null)));
+        testDescriptorPersistence(new CommitLogDescriptor(CommitLogDescriptor.VERSION_22, 19,
+                new ParameterizedClass("StubbyCompressor", ImmutableMap.of("parameter1", "value1", "flag2", "55", "argument3", "null"))));
+    }
+
+    @Test
+    public void testDescriptorInvalidParametersSize() throws IOException
+    {
+        Map<String, String> params = new HashMap<>();
+        for (int i=0; i<6000; ++i)
+            params.put("key"+i, Integer.toString(i, 16));
+        try {
+            CommitLogDescriptor desc = new CommitLogDescriptor(CommitLogDescriptor.VERSION_22,
+                                                               21,
+                                                               new ParameterizedClass("LZ4Compressor", params));
+            ByteBuffer buf = ByteBuffer.allocate(1024000);
+            CommitLogDescriptor.writeHeader(buf, desc);
+            fail("Parameter object too long should fail on writing descriptor.");
+        } catch (ConfigurationException e)
+        {
+            // correct path
+        }
+    }
+}
diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogFailurePolicyTest.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogFailurePolicyTest.java
new file mode 100644
index 0000000..bde8ca3
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogFailurePolicyTest.java
@@ -0,0 +1,140 @@
+/*
+* 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.db.commitlog;
+
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.commitlog.CommitLog;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.gms.Gossiper;
+import org.apache.cassandra.service.CassandraDaemon;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.KillerForTests;
+
+public class CommitLogFailurePolicyTest
+{
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+    }
+
+    @Test
+    public void testCommitFailurePolicy_stop() throws ConfigurationException
+    {
+        CassandraDaemon daemon = new CassandraDaemon();
+        daemon.completeSetup(); //startup must be completed, otherwise commit log failure must kill JVM regardless of failure policy
+        StorageService.instance.registerDaemon(daemon);
+
+        // Need storage service active so stop policy can shutdown gossip
+        StorageService.instance.initServer();
+        Assert.assertTrue(Gossiper.instance.isEnabled());
+
+        Config.CommitFailurePolicy oldPolicy = DatabaseDescriptor.getCommitFailurePolicy();
+        try
+        {
+            DatabaseDescriptor.setCommitFailurePolicy(Config.CommitFailurePolicy.stop);
+            CommitLog.handleCommitError("Test stop error", new Throwable());
+            Assert.assertFalse(Gossiper.instance.isEnabled());
+        }
+        finally
+        {
+            DatabaseDescriptor.setCommitFailurePolicy(oldPolicy);
+        }
+    }
+
+    @Test
+    public void testCommitFailurePolicy_die()
+    {
+        CassandraDaemon daemon = new CassandraDaemon();
+        daemon.completeSetup(); //startup must be completed, otherwise commit log failure must kill JVM regardless of failure policy
+        StorageService.instance.registerDaemon(daemon);
+
+        KillerForTests killerForTests = new KillerForTests();
+        JVMStabilityInspector.Killer originalKiller = JVMStabilityInspector.replaceKiller(killerForTests);
+        Config.CommitFailurePolicy oldPolicy = DatabaseDescriptor.getCommitFailurePolicy();
+        try
+        {
+            DatabaseDescriptor.setCommitFailurePolicy(Config.CommitFailurePolicy.die);
+            CommitLog.handleCommitError("Testing die policy", new Throwable());
+            Assert.assertTrue(killerForTests.wasKilled());
+            Assert.assertFalse(killerForTests.wasKilledQuietly()); //only killed quietly on startup failure
+        }
+        finally
+        {
+            DatabaseDescriptor.setCommitFailurePolicy(oldPolicy);
+            JVMStabilityInspector.replaceKiller(originalKiller);
+        }
+    }
+
+    @Test
+    public void testCommitFailurePolicy_ignore_beforeStartup()
+    {
+        //startup was not completed successfuly (since method completeSetup() was not called)
+        CassandraDaemon daemon = new CassandraDaemon();
+        StorageService.instance.registerDaemon(daemon);
+
+        KillerForTests killerForTests = new KillerForTests();
+        JVMStabilityInspector.Killer originalKiller = JVMStabilityInspector.replaceKiller(killerForTests);
+        Config.CommitFailurePolicy oldPolicy = DatabaseDescriptor.getCommitFailurePolicy();
+        try
+        {
+            DatabaseDescriptor.setCommitFailurePolicy(Config.CommitFailurePolicy.ignore);
+            CommitLog.handleCommitError("Testing ignore policy", new Throwable());
+            //even though policy is ignore, JVM must die because Daemon has not finished initializing
+            Assert.assertTrue(killerForTests.wasKilled());
+            Assert.assertTrue(killerForTests.wasKilledQuietly()); //killed quietly due to startup failure
+        }
+        finally
+        {
+            DatabaseDescriptor.setCommitFailurePolicy(oldPolicy);
+            JVMStabilityInspector.replaceKiller(originalKiller);
+        }
+    }
+
+    @Test
+    public void testCommitFailurePolicy_ignore_afterStartup() throws Exception
+    {
+        CassandraDaemon daemon = new CassandraDaemon();
+        daemon.completeSetup(); //startup must be completed, otherwise commit log failure must kill JVM regardless of failure policy
+        StorageService.instance.registerDaemon(daemon);
+
+        KillerForTests killerForTests = new KillerForTests();
+        JVMStabilityInspector.Killer originalKiller = JVMStabilityInspector.replaceKiller(killerForTests);
+        Config.CommitFailurePolicy oldPolicy = DatabaseDescriptor.getCommitFailurePolicy();
+        try
+        {
+            DatabaseDescriptor.setCommitFailurePolicy(Config.CommitFailurePolicy.ignore);
+            CommitLog.handleCommitError("Testing ignore policy", new Throwable());
+            //error policy is set to IGNORE, so JVM must not be killed if error ocurs after startup
+            Assert.assertFalse(killerForTests.wasKilled());
+        }
+        finally
+        {
+            DatabaseDescriptor.setCommitFailurePolicy(oldPolicy);
+            JVMStabilityInspector.replaceKiller(originalKiller);
+        }
+    }
+}
diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
new file mode 100644
index 0000000..9b63885
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
@@ -0,0 +1,500 @@
+/*
+* 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.db.commitlog;
+
+import java.io.*;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.zip.CRC32;
+import java.util.zip.Checksum;
+
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.Util;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.config.ParameterizedClass;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.commitlog.CommitLogReplayer.CommitLogReplayException;
+import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.db.composites.CellName;
+import org.apache.cassandra.db.composites.CellNameType;
+import org.apache.cassandra.db.filter.NamesQueryFilter;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.compress.DeflateCompressor;
+import org.apache.cassandra.io.compress.LZ4Compressor;
+import org.apache.cassandra.io.compress.SnappyCompressor;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.utils.*;
+
+import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
+
+@RunWith(Parameterized.class)
+public class CommitLogTest
+{
+    private static final String KEYSPACE1 = "CommitLogTest";
+    private static final String KEYSPACE2 = "CommitLogTestNonDurable";
+    private static final String CF1 = "Standard1";
+    private static final String CF2 = "Standard2";
+
+    public CommitLogTest(ParameterizedClass commitLogCompression)
+    {
+        DatabaseDescriptor.setCommitLogCompression(commitLogCompression);
+    }
+
+    @Before
+    public void setUp() throws IOException
+    {
+        CommitLog.instance.resetUnsafe(true);
+    }
+
+    @Parameters()
+    public static Collection<Object[]> generateData()
+    {
+        return Arrays.asList(new Object[][] {
+                { null }, // No compression
+                { new ParameterizedClass(LZ4Compressor.class.getName(), Collections.<String, String>emptyMap()) },
+                { new ParameterizedClass(SnappyCompressor.class.getName(), Collections.<String, String>emptyMap()) },
+                { new ParameterizedClass(DeflateCompressor.class.getName(), Collections.<String, String>emptyMap()) } });
+    }
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF2));
+        SchemaLoader.createKeyspace(KEYSPACE2,
+                                    false,
+                                    true,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF2));
+        CompactionManager.instance.disableAutoCompaction();
+    }
+
+    @Test
+    public void testRecoveryWithEmptyLog() throws Exception
+    {
+        runExpecting(new WrappedRunnable() {
+            public void runMayThrow() throws Exception
+            {
+                CommitLog.instance.recover(new File[]{ tmpFile(CommitLogDescriptor.current_version) });
+            }
+        }, CommitLogReplayException.class);
+    }
+
+    @Test
+    public void testRecoveryWithEmptyLog20() throws Exception
+    {
+        CommitLog.instance.recover(new File[]{ tmpFile(CommitLogDescriptor.VERSION_20) });
+    }
+
+    @Test
+    public void testRecoveryWithZeroLog() throws Exception
+    {
+        testRecovery(new byte[10], null);
+    }
+
+    @Test
+    public void testRecoveryWithShortLog() throws Exception
+    {
+        // force EOF while reading log
+        testRecoveryWithBadSizeArgument(100, 10);
+    }
+
+    @Test
+    public void testRecoveryWithShortPadding() throws Exception
+    {
+        // If we have 0-3 bytes remaining, commitlog replayer
+        // should pass, because there's insufficient room
+        // left in the segment for the legacy size marker.
+        testRecovery(new byte[1], null);
+        testRecovery(new byte[2], null);
+        testRecovery(new byte[3], null);
+    }
+
+    @Test
+    public void testRecoveryWithShortSize() throws Exception
+    {
+        runExpecting(new WrappedRunnable()  {
+            public void runMayThrow() throws Exception {
+                byte[] data = new byte[5];
+                data[3] = 1; // Not a legacy marker, give it a fake (short) size
+                testRecovery(data, CommitLogDescriptor.VERSION_20);
+            }
+        }, CommitLogReplayException.class);
+    }
+
+    @Test
+    public void testRecoveryWithShortCheckSum() throws Exception
+    {
+        byte[] data = new byte[8];
+        data[3] = 10;   // make sure this is not a legacy end marker.
+        testRecovery(data, CommitLogReplayException.class);
+    }
+
+    @Test
+    public void testRecoveryWithShortMutationSize() throws Exception
+    {
+        testRecoveryWithBadSizeArgument(9, 10);
+    }
+
+    private void testRecoveryWithGarbageLog() throws Exception
+    {
+        byte[] garbage = new byte[100];
+        (new java.util.Random()).nextBytes(garbage);
+        testRecovery(garbage, CommitLogDescriptor.current_version);
+    }
+
+    @Test
+    public void testRecoveryWithGarbageLog_fail() throws Exception
+    {
+        runExpecting(new WrappedRunnable() {
+            public void runMayThrow() throws Exception
+            {
+                testRecoveryWithGarbageLog();
+            }
+        }, CommitLogReplayException.class);
+    }
+
+    @Test
+    public void testRecoveryWithGarbageLog_ignoredByProperty() throws Exception
+    {
+        try {
+            System.setProperty(CommitLogReplayer.IGNORE_REPLAY_ERRORS_PROPERTY, "true");
+            testRecoveryWithGarbageLog();
+        } finally {
+            System.clearProperty(CommitLogReplayer.IGNORE_REPLAY_ERRORS_PROPERTY);
+        }
+    }
+
+    @Test
+    public void testRecoveryWithBadSizeChecksum() throws Exception
+    {
+        Checksum checksum = new CRC32();
+        checksum.update(100);
+        testRecoveryWithBadSizeArgument(100, 100, ~checksum.getValue());
+    }
+
+    @Test
+    public void testRecoveryWithNegativeSizeArgument() throws Exception
+    {
+        // garbage from a partial/bad flush could be read as a negative size even if there is no EOF
+        testRecoveryWithBadSizeArgument(-10, 10); // negative size, but no EOF
+    }
+
+    @Test
+    public void testDontDeleteIfDirty() throws Exception
+    {
+        // Roughly 32 MB mutation
+        Mutation rm = new Mutation(KEYSPACE1, bytes("k"));
+        rm.add(CF1, Util.cellname("c1"), ByteBuffer.allocate(DatabaseDescriptor.getCommitLogSegmentSize()/4), 0);
+
+        // Adding it 5 times
+        CommitLog.instance.add(rm);
+        CommitLog.instance.add(rm);
+        CommitLog.instance.add(rm);
+        CommitLog.instance.add(rm);
+        CommitLog.instance.add(rm);
+
+        // Adding new mutation on another CF
+        Mutation rm2 = new Mutation(KEYSPACE1, bytes("k"));
+        rm2.add(CF2, Util.cellname("c1"), ByteBuffer.allocate(4), 0);
+        CommitLog.instance.add(rm2);
+
+        assert CommitLog.instance.activeSegments() == 2 : "Expecting 2 segments, got " + CommitLog.instance.activeSegments();
+
+        UUID cfid2 = rm2.getColumnFamilyIds().iterator().next();
+        CommitLog.instance.discardCompletedSegments(cfid2, CommitLog.instance.getContext());
+
+        // Assert we still have both our segment
+        assert CommitLog.instance.activeSegments() == 2 : "Expecting 2 segments, got " + CommitLog.instance.activeSegments();
+    }
+
+    @Test
+    public void testDeleteIfNotDirty() throws Exception
+    {
+        DatabaseDescriptor.getCommitLogSegmentSize();
+        // Roughly 32 MB mutation
+        Mutation rm = new Mutation(KEYSPACE1, bytes("k"));
+        rm.add(CF1, Util.cellname("c1"), ByteBuffer.allocate((DatabaseDescriptor.getCommitLogSegmentSize()/4) - 1), 0);
+
+        // Adding it twice (won't change segment)
+        CommitLog.instance.add(rm);
+        CommitLog.instance.add(rm);
+
+        assert CommitLog.instance.activeSegments() == 1 : "Expecting 1 segment, got " + CommitLog.instance.activeSegments();
+
+        // "Flush": this won't delete anything
+        UUID cfid1 = rm.getColumnFamilyIds().iterator().next();
+        CommitLog.instance.sync(true);
+        CommitLog.instance.discardCompletedSegments(cfid1, CommitLog.instance.getContext());
+
+        assert CommitLog.instance.activeSegments() == 1 : "Expecting 1 segment, got " + CommitLog.instance.activeSegments();
+
+        // Adding new mutation on another CF, large enough (including CL entry overhead) that a new segment is created
+        Mutation rm2 = new Mutation(KEYSPACE1, bytes("k"));
+        rm2.add(CF2, Util.cellname("c1"), ByteBuffer.allocate((DatabaseDescriptor.getCommitLogSegmentSize()/2) - 200), 0);
+        CommitLog.instance.add(rm2);
+        // also forces a new segment, since each entry-with-overhead is just under half the CL size
+        CommitLog.instance.add(rm2);
+        CommitLog.instance.add(rm2);
+
+        assert CommitLog.instance.activeSegments() == 3 : "Expecting 3 segments, got " + CommitLog.instance.activeSegments();
+
+
+        // "Flush" second cf: The first segment should be deleted since we
+        // didn't write anything on cf1 since last flush (and we flush cf2)
+
+        UUID cfid2 = rm2.getColumnFamilyIds().iterator().next();
+        CommitLog.instance.discardCompletedSegments(cfid2, CommitLog.instance.getContext());
+
+        // Assert we still have both our segment
+        assert CommitLog.instance.activeSegments() == 1 : "Expecting 1 segment, got " + CommitLog.instance.activeSegments();
+    }
+
+    private static int getMaxRecordDataSize(String keyspace, ByteBuffer key, String table, CellName column)
+    {
+        Mutation rm = new Mutation(KEYSPACE1, bytes("k"));
+        rm.add("Standard1", Util.cellname("c1"), ByteBuffer.allocate(0), 0);
+
+        int max = (DatabaseDescriptor.getCommitLogSegmentSize() / 2);
+        max -= CommitLogSegment.ENTRY_OVERHEAD_SIZE; // log entry overhead
+        return max - (int) Mutation.serializer.serializedSize(rm, MessagingService.current_version);
+    }
+
+    private static int getMaxRecordDataSize()
+    {
+        return getMaxRecordDataSize(KEYSPACE1, bytes("k"), CF1, Util.cellname("c1"));
+    }
+
+    // CASSANDRA-3615
+    @Test
+    public void testEqualRecordLimit() throws Exception
+    {
+        Mutation rm = new Mutation(KEYSPACE1, bytes("k"));
+        rm.add(CF1, Util.cellname("c1"), ByteBuffer.allocate(getMaxRecordDataSize()), 0);
+        CommitLog.instance.add(rm);
+    }
+
+    @Test
+    public void testExceedRecordLimit() throws Exception
+    {
+        try
+        {
+            Mutation rm = new Mutation(KEYSPACE1, bytes("k"));
+            rm.add(CF1, Util.cellname("c1"), ByteBuffer.allocate(1 + getMaxRecordDataSize()), 0);
+            CommitLog.instance.add(rm);
+            throw new AssertionError("mutation larger than limit was accepted");
+        }
+        catch (IllegalArgumentException e)
+        {
+            // IAE is thrown on too-large mutations
+        }
+    }
+
+    protected void testRecoveryWithBadSizeArgument(int size, int dataSize) throws Exception
+    {
+        Checksum checksum = new CRC32();
+        checksum.update(size);
+        testRecoveryWithBadSizeArgument(size, dataSize, checksum.getValue());
+    }
+
+    protected void testRecoveryWithBadSizeArgument(int size, int dataSize, long checksum) throws Exception
+    {
+        ByteArrayOutputStream out = new ByteArrayOutputStream();
+        DataOutputStream dout = new DataOutputStream(out);
+        dout.writeInt(size);
+        dout.writeLong(checksum);
+        dout.write(new byte[dataSize]);
+        dout.close();
+        testRecovery(out.toByteArray(), CommitLogReplayException.class);
+    }
+
+    protected File tmpFile(int version) throws IOException
+    {
+        File logFile = File.createTempFile("CommitLog-" + version + "-", ".log");
+        logFile.deleteOnExit();
+        assert logFile.length() == 0;
+        return logFile;
+    }
+
+    protected void testRecovery(byte[] logData, int version) throws Exception
+    {
+        File logFile = tmpFile(version);
+        try (OutputStream lout = new FileOutputStream(logFile))
+        {
+            lout.write(logData);
+            //statics make it annoying to test things correctly
+            CommitLog.instance.recover(logFile.getPath()); //CASSANDRA-1119 / CASSANDRA-1179 throw on failure*/
+        }
+    }
+
+    protected void testRecovery(CommitLogDescriptor desc, byte[] logData) throws Exception
+    {
+        File logFile = tmpFile(desc.version);
+        CommitLogDescriptor fromFile = CommitLogDescriptor.fromFileName(logFile.getName());
+        // Change id to match file.
+        desc = new CommitLogDescriptor(desc.version, fromFile.id, desc.compression);
+        ByteBuffer buf = ByteBuffer.allocate(1024);
+        CommitLogDescriptor.writeHeader(buf, desc);
+        try (OutputStream lout = new FileOutputStream(logFile))
+        {
+            lout.write(buf.array(), 0, buf.position());
+            lout.write(logData);
+            //statics make it annoying to test things correctly
+            CommitLog.instance.recover(logFile.getPath()); //CASSANDRA-1119 / CASSANDRA-1179 throw on failure*/
+        }
+    }
+
+    @Test
+    public void testRecoveryWithIdMismatch() throws Exception
+    {
+        CommitLogDescriptor desc = new CommitLogDescriptor(4, null);
+        final File logFile = tmpFile(desc.version);
+        ByteBuffer buf = ByteBuffer.allocate(1024);
+        CommitLogDescriptor.writeHeader(buf, desc);
+        try (OutputStream lout = new FileOutputStream(logFile))
+        {
+            lout.write(buf.array(), 0, buf.position());
+
+            runExpecting(new WrappedRunnable() {
+                public void runMayThrow() throws Exception
+                {
+                    CommitLog.instance.recover(logFile.getPath()); //CASSANDRA-1119 / CASSANDRA-1179 throw on failure*/
+                }
+            }, CommitLogReplayException.class);
+        }
+    }
+
+    protected void runExpecting(Runnable r, Class<?> expected)
+    {
+        JVMStabilityInspector.Killer originalKiller;
+        KillerForTests killerForTests;
+
+        killerForTests = new KillerForTests();
+        originalKiller = JVMStabilityInspector.replaceKiller(killerForTests);
+
+        Throwable caught = null;
+        try
+        {
+            r.run();
+        }
+        catch (RuntimeException e)
+        {
+            if (expected != e.getCause().getClass())
+                throw new AssertionError("Expected exception " + expected + ", got " + e, e);
+            caught = e;
+        }
+        if (expected != null && caught == null)
+            Assert.fail("Expected exception " + expected + " but call completed successfully.");
+
+        JVMStabilityInspector.replaceKiller(originalKiller);
+        Assert.assertEquals("JVM killed", expected != null, killerForTests.wasKilled());
+    }
+
+    protected void testRecovery(final byte[] logData, Class<?> expected) throws Exception
+    {
+        runExpecting(new WrappedRunnable() {
+            public void runMayThrow() throws Exception
+            {
+                testRecovery(logData, CommitLogDescriptor.VERSION_20);
+            }
+        }, expected);
+        runExpecting(new WrappedRunnable() {
+            public void runMayThrow() throws Exception
+            {
+                testRecovery(new CommitLogDescriptor(4, null), logData);
+            }
+        }, expected);
+    }
+
+    @Test
+    public void testTruncateWithoutSnapshot() throws ExecutionException, InterruptedException, IOException
+    {
+        boolean prev = DatabaseDescriptor.isAutoSnapshot();
+        DatabaseDescriptor.setAutoSnapshot(false);
+        ColumnFamilyStore cfs1 = Keyspace.open(KEYSPACE1).getColumnFamilyStore("Standard1");
+        ColumnFamilyStore cfs2 = Keyspace.open(KEYSPACE1).getColumnFamilyStore("Standard2");
+
+        final Mutation rm1 = new Mutation(KEYSPACE1, bytes("k"));
+        rm1.add("Standard1", Util.cellname("c1"), ByteBuffer.allocate(100), 0);
+        rm1.apply();
+        cfs1.truncateBlocking();
+        DatabaseDescriptor.setAutoSnapshot(prev);
+        final Mutation rm2 = new Mutation(KEYSPACE1, bytes("k"));
+        rm2.add("Standard2", Util.cellname("c1"), ByteBuffer.allocate(DatabaseDescriptor.getCommitLogSegmentSize() / 4), 0);
+
+        for (int i = 0 ; i < 5 ; i++)
+            CommitLog.instance.add(rm2);
+
+        Assert.assertEquals(2, CommitLog.instance.activeSegments());
+        ReplayPosition position = CommitLog.instance.getContext();
+        for (Keyspace ks : Keyspace.system())
+            for (ColumnFamilyStore syscfs : ks.getColumnFamilyStores())
+                CommitLog.instance.discardCompletedSegments(syscfs.metadata.cfId, position);
+        CommitLog.instance.discardCompletedSegments(cfs2.metadata.cfId, position);
+        Assert.assertEquals(1, CommitLog.instance.activeSegments());
+    }
+
+    @Test
+    public void testTruncateWithoutSnapshotNonDurable() throws IOException
+    {
+        boolean prevAutoSnapshot = DatabaseDescriptor.isAutoSnapshot();
+        DatabaseDescriptor.setAutoSnapshot(false);
+        Keyspace notDurableKs = Keyspace.open(KEYSPACE2);
+        Assert.assertFalse(notDurableKs.getMetadata().durableWrites);
+        ColumnFamilyStore cfs = notDurableKs.getColumnFamilyStore("Standard1");
+        CellNameType type = notDurableKs.getColumnFamilyStore("Standard1").getComparator();
+        Mutation rm;
+        DecoratedKey dk = Util.dk("key1");
+
+        // add data
+        rm = new Mutation(KEYSPACE2, dk.getKey());
+        rm.add("Standard1", Util.cellname("Column1"), ByteBufferUtil.bytes("abcd"), 0);
+        rm.apply();
+
+        ReadCommand command = new SliceByNamesReadCommand(KEYSPACE2, dk.getKey(), "Standard1", System.currentTimeMillis(), new NamesQueryFilter(FBUtilities.singleton(Util.cellname("Column1"), type)));
+        Row row = command.getRow(notDurableKs);
+        Cell col = row.cf.getColumn(Util.cellname("Column1"));
+        Assert.assertEquals(col.value(), ByteBuffer.wrap("abcd".getBytes()));
+        cfs.truncateBlocking();
+        DatabaseDescriptor.setAutoSnapshot(prevAutoSnapshot);
+        row = command.getRow(notDurableKs);
+        Assert.assertEquals(null, row.cf);
+    }
+}
diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTestReplayer.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTestReplayer.java
new file mode 100644
index 0000000..0c46061
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTestReplayer.java
@@ -0,0 +1,79 @@
+/*
+* 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.db.commitlog;
+
+import java.io.DataInputStream;
+import java.io.File;
+import java.io.IOException;
+
+import com.google.common.base.Predicate;
+
+import org.junit.Assert;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.ColumnSerializer;
+import org.apache.cassandra.db.Mutation;
+import org.apache.cassandra.io.util.FastByteArrayInputStream;
+
+/**
+ * Utility class for tests needing to examine the commitlog contents.
+ */
+public class CommitLogTestReplayer extends CommitLogReplayer
+{
+    static public void examineCommitLog(Predicate<Mutation> processor) throws IOException
+    {
+        CommitLog.instance.sync(true);
+
+        CommitLogTestReplayer replayer = new CommitLogTestReplayer(CommitLog.instance, processor);
+        File commitLogDir = new File(DatabaseDescriptor.getCommitLogLocation());
+        replayer.recover(commitLogDir.listFiles());
+    }
+
+    final private Predicate<Mutation> processor;
+
+    public CommitLogTestReplayer(CommitLog log, Predicate<Mutation> processor)
+    {
+        this(log, ReplayPosition.NONE, processor);
+    }
+
+    public CommitLogTestReplayer(CommitLog log, ReplayPosition discardedPos, Predicate<Mutation> processor)
+    {
+        super(log, discardedPos, null, ReplayFilter.create());
+        this.processor = processor;
+    }
+
+    @Override
+    void replayMutation(byte[] inputBuffer, int size, final int entryLocation, final CommitLogDescriptor desc)
+    {
+        FastByteArrayInputStream bufIn = new FastByteArrayInputStream(inputBuffer, 0, size);
+        Mutation mutation;
+        try
+        {
+            mutation = Mutation.serializer.deserialize(new DataInputStream(bufIn),
+                                                           desc.getMessagingVersion(),
+                                                           ColumnSerializer.Flag.LOCAL);
+            Assert.assertTrue(processor.apply(mutation));
+        }
+        catch (IOException e)
+        {
+            // Test fails.
+            throw new AssertionError(e);
+        }
+    }
+}
diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogUpgradeTest.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogUpgradeTest.java
new file mode 100644
index 0000000..9de2628
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogUpgradeTest.java
@@ -0,0 +1,208 @@
+package org.apache.cassandra.db.commitlog;
+
+/*
+ *
+ * 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.
+ *
+ */
+
+import java.io.*;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.Properties;
+import java.util.UUID;
+
+import junit.framework.Assert;
+
+import com.google.common.base.Predicate;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.db.Cell;
+import org.apache.cassandra.db.ColumnFamily;
+import org.apache.cassandra.db.Mutation;
+import org.apache.cassandra.db.commitlog.CommitLogReplayer.CommitLogReplayException;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.KillerForTests;
+
+public class CommitLogUpgradeTest
+{
+    static final String DATA_DIR = "test/data/legacy-commitlog/";
+    static final String PROPERTIES_FILE = "hash.txt";
+    static final String CFID_PROPERTY = "cfid";
+    static final String CELLS_PROPERTY = "cells";
+    static final String HASH_PROPERTY = "hash";
+
+    static final String TABLE = "Standard1";
+    static final String KEYSPACE = "Keyspace1";
+    static final String CELLNAME = "name";
+
+    private JVMStabilityInspector.Killer originalKiller;
+    private KillerForTests killerForTests;
+    private boolean shouldBeKilled = false;
+
+    @Before
+    public void prepareToBeKilled()
+    {
+        killerForTests = new KillerForTests();
+        originalKiller = JVMStabilityInspector.replaceKiller(killerForTests);
+    }
+
+    @After
+    public void cleanUp()
+    {
+        JVMStabilityInspector.replaceKiller(originalKiller);
+        Assert.assertEquals("JVM killed", shouldBeKilled, killerForTests.wasKilled());
+    }
+
+    @Test
+    public void test20() throws Exception
+    {
+        testRestore(DATA_DIR + "2.0");
+    }
+
+    @Test
+    public void test21() throws Exception
+    {
+        testRestore(DATA_DIR + "2.1");
+    }
+
+    @Test
+    public void test22_truncated() throws Exception
+    {
+        testRestore(DATA_DIR + "2.2-lz4-truncated");
+    }
+
+    @Test(expected = CommitLogReplayException.class)
+    public void test22_bitrot() throws Exception
+    {
+        shouldBeKilled = true;
+        testRestore(DATA_DIR + "2.2-lz4-bitrot");
+    }
+
+    @Test
+    public void test22_bitrot_ignored() throws Exception
+    {
+        try {
+            System.setProperty(CommitLogReplayer.IGNORE_REPLAY_ERRORS_PROPERTY, "true");
+            testRestore(DATA_DIR + "2.2-lz4-bitrot");
+        } finally {
+            System.clearProperty(CommitLogReplayer.IGNORE_REPLAY_ERRORS_PROPERTY);
+        }
+    }
+
+    @Test(expected = CommitLogReplayException.class)
+    public void test22_bitrot2() throws Exception
+    {
+        shouldBeKilled = true;
+        testRestore(DATA_DIR + "2.2-lz4-bitrot2");
+    }
+
+    @Test
+    public void test22_bitrot2_ignored() throws Exception
+    {
+        try {
+            System.setProperty(CommitLogReplayer.IGNORE_REPLAY_ERRORS_PROPERTY, "true");
+            testRestore(DATA_DIR + "2.2-lz4-bitrot2");
+        } finally {
+            System.clearProperty(CommitLogReplayer.IGNORE_REPLAY_ERRORS_PROPERTY);
+        }
+    }
+
+    @BeforeClass
+    static public void initialize() throws FileNotFoundException, IOException, InterruptedException
+    {
+        SchemaLoader.loadSchema();
+        SchemaLoader.schemaDefinition("");
+    }
+
+    public void testRestore(String location) throws IOException, InterruptedException
+    {
+        Properties prop = new Properties();
+        prop.load(new FileInputStream(new File(location + File.separatorChar + PROPERTIES_FILE)));
+        int hash = Integer.parseInt(prop.getProperty(HASH_PROPERTY));
+        int cells = Integer.parseInt(prop.getProperty(CELLS_PROPERTY));
+
+        String cfidString = prop.getProperty(CFID_PROPERTY);
+        if (cfidString != null)
+        {
+            UUID cfid = UUID.fromString(cfidString);
+            if (Schema.instance.getCF(cfid) == null)
+            {
+                CFMetaData cfm = Schema.instance.getCFMetaData(KEYSPACE, TABLE);
+                Schema.instance.purge(cfm);
+                Schema.instance.load(cfm.copy(cfid));
+            }
+        }
+
+        Hasher hasher = new Hasher();
+        CommitLogTestReplayer replayer = new CommitLogTestReplayer(CommitLog.instance, hasher);
+        File[] files = new File(location).listFiles(new FilenameFilter()
+        {
+            @Override
+            public boolean accept(File dir, String name)
+            {
+                return name.endsWith(".log");
+            }
+        });
+        replayer.recover(files);
+
+        Assert.assertEquals(cells, hasher.cells);
+        Assert.assertEquals(hash, hasher.hash);
+    }
+
+    public static int hash(int hash, ByteBuffer bytes)
+    {
+        int shift = 0;
+        for (int i = 0; i < bytes.limit(); i++)
+        {
+            hash += (bytes.get(i) & 0xFF) << shift;
+            shift = (shift + 8) & 0x1F;
+        }
+        return hash;
+    }
+
+    class Hasher implements Predicate<Mutation>
+    {
+        int hash = 0;
+        int cells = 0;
+
+        @Override
+        public boolean apply(Mutation mutation)
+        {
+            for (ColumnFamily cf : mutation.getColumnFamilies())
+            {
+                for (Cell c : cf.getSortedColumns())
+                {
+                    if (new String(c.name().toByteBuffer().array(), StandardCharsets.UTF_8).startsWith(CELLNAME))
+                    {
+                        hash = hash(hash, c.value());
+                        ++cells;
+                    }
+                }
+            }
+            return true;
+        }
+    }
+}
diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogUpgradeTestMaker.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogUpgradeTestMaker.java
new file mode 100644
index 0000000..175a8d6
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogUpgradeTestMaker.java
@@ -0,0 +1,250 @@
+package org.apache.cassandra.db.commitlog;
+
+/*
+ *
+ * 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.
+ *
+ */
+
+import java.io.*;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import com.google.common.util.concurrent.RateLimiter;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.Util;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.db.Mutation;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.FBUtilities;
+
+import static org.apache.cassandra.db.commitlog.CommitLogUpgradeTest.*;
+
+public class CommitLogUpgradeTestMaker
+{
+    public static ByteBuffer dataSource;
+
+    private static int NUM_THREADS = 4 * Runtime.getRuntime().availableProcessors() - 1;
+    public static int numCells = 1;
+    public static int cellSize = 256;
+    public static int rateLimit = 0;
+    public static int runTimeMs = 1000;
+
+    public static void main(String[] args) throws Exception
+    {
+        try
+        {
+            initialize();
+
+            CommitLogUpgradeTestMaker tester = new CommitLogUpgradeTestMaker();
+            tester.makeLog();
+        }
+        catch (Throwable e)
+        {
+            e.printStackTrace(System.err);
+        }
+        finally
+        {
+            System.exit(0);
+        }
+    }
+
+    volatile boolean stop = false;
+    boolean randomSize = true;
+
+    static public void initialize() throws IOException, ConfigurationException
+    {
+        try (FileInputStream fis = new FileInputStream("CHANGES.txt"))
+        {
+            dataSource = ByteBuffer.allocateDirect((int) fis.getChannel().size());
+            while (dataSource.hasRemaining())
+            {
+                fis.getChannel().read(dataSource);
+            }
+            dataSource.flip();
+        }
+
+        SchemaLoader.loadSchema();
+        SchemaLoader.schemaDefinition("");
+    }
+
+    public void makeLog() throws IOException, InterruptedException
+    {
+        CommitLog commitLog = CommitLog.instance;
+        System.out.format("\nUsing commit log size %dmb, compressor %s, sync %s%s\n",
+                          mb(DatabaseDescriptor.getCommitLogSegmentSize()),
+                          commitLog.configuration.getCompressorName(),
+                          commitLog.executor.getClass().getSimpleName(),
+                          randomSize ? " random size" : "");
+        final List<CommitlogExecutor> threads = new ArrayList<>();
+        ScheduledExecutorService scheduled = startThreads(commitLog, threads);
+
+        Thread.sleep(runTimeMs);
+        stop = true;
+        scheduled.shutdown();
+        scheduled.awaitTermination(2, TimeUnit.SECONDS);
+
+        int hash = 0;
+        int cells = 0;
+        for (CommitlogExecutor t : threads)
+        {
+            t.join();
+            hash += t.hash;
+            cells += t.cells;
+        }
+        commitLog.shutdownBlocking();
+
+        File dataDir = new File(CommitLogUpgradeTest.DATA_DIR + FBUtilities.getReleaseVersionString());
+        System.out.format("Data will be stored in %s\n", dataDir);
+        if (dataDir.exists())
+            FileUtils.deleteRecursive(dataDir);
+
+        dataDir.mkdirs();
+        for (File f : new File(DatabaseDescriptor.getCommitLogLocation()).listFiles())
+            FileUtils.createHardLink(f, new File(dataDir, f.getName()));
+
+        Properties prop = new Properties();
+        prop.setProperty(CFID_PROPERTY, Schema.instance.getId(KEYSPACE, TABLE).toString());
+        prop.setProperty(CELLS_PROPERTY, Integer.toString(cells));
+        prop.setProperty(HASH_PROPERTY, Integer.toString(hash));
+        prop.store(new FileOutputStream(new File(dataDir, PROPERTIES_FILE)),
+                   "CommitLog upgrade test, version " + FBUtilities.getReleaseVersionString());
+        System.out.println("Done");
+    }
+
+    public ScheduledExecutorService startThreads(CommitLog commitLog, final List<CommitlogExecutor> threads)
+    {
+        stop = false;
+        for (int ii = 0; ii < NUM_THREADS; ii++)
+        {
+            final CommitlogExecutor t = new CommitlogExecutor(commitLog);
+            threads.add(t);
+            t.start();
+        }
+
+        final long start = System.currentTimeMillis();
+        Runnable printRunnable = new Runnable()
+        {
+            long lastUpdate = 0;
+
+            public void run()
+            {
+                Runtime runtime = Runtime.getRuntime();
+                long maxMemory = mb(runtime.maxMemory());
+                long allocatedMemory = mb(runtime.totalMemory());
+                long freeMemory = mb(runtime.freeMemory());
+                long temp = 0;
+                long sz = 0;
+                for (CommitlogExecutor cle : threads)
+                {
+                    temp += cle.counter.get();
+                    sz += cle.dataSize;
+                }
+                double time = (System.currentTimeMillis() - start) / 1000.0;
+                double avg = (temp / time);
+                System.out.println(
+                        String.format("second %d mem max %dmb allocated %dmb free %dmb mutations %d since start %d avg %.3f transfer %.3fmb",
+                                      ((System.currentTimeMillis() - start) / 1000),
+                                      maxMemory,
+                                      allocatedMemory,
+                                      freeMemory,
+                                      (temp - lastUpdate),
+                                      lastUpdate,
+                                      avg,
+                                      mb(sz / time)));
+                lastUpdate = temp;
+            }
+        };
+        ScheduledExecutorService scheduled = Executors.newScheduledThreadPool(1);
+        scheduled.scheduleAtFixedRate(printRunnable, 1, 1, TimeUnit.SECONDS);
+        return scheduled;
+    }
+
+    private static long mb(long maxMemory)
+    {
+        return maxMemory / (1024 * 1024);
+    }
+
+    private static double mb(double maxMemory)
+    {
+        return maxMemory / (1024 * 1024);
+    }
+
+    public static ByteBuffer randomBytes(int quantity, ThreadLocalRandom tlr)
+    {
+        ByteBuffer slice = ByteBuffer.allocate(quantity);
+        ByteBuffer source = dataSource.duplicate();
+        source.position(tlr.nextInt(source.capacity() - quantity));
+        source.limit(source.position() + quantity);
+        slice.put(source);
+        slice.flip();
+        return slice;
+    }
+
+    public class CommitlogExecutor extends Thread
+    {
+        final AtomicLong counter = new AtomicLong();
+        int hash = 0;
+        int cells = 0;
+        int dataSize = 0;
+        final CommitLog commitLog;
+
+        volatile ReplayPosition rp;
+
+        public CommitlogExecutor(CommitLog commitLog)
+        {
+            this.commitLog = commitLog;
+        }
+
+        public void run()
+        {
+            RateLimiter rl = rateLimit != 0 ? RateLimiter.create(rateLimit) : null;
+            final ThreadLocalRandom tlr = ThreadLocalRandom.current();
+            while (!stop)
+            {
+                if (rl != null)
+                    rl.acquire();
+                String ks = KEYSPACE;
+                ByteBuffer key = randomBytes(16, tlr);
+                Mutation mutation = new Mutation(ks, key);
+
+                for (int ii = 0; ii < numCells; ii++)
+                {
+                    int sz = randomSize ? tlr.nextInt(cellSize) : cellSize;
+                    ByteBuffer bytes = randomBytes(sz, tlr);
+                    mutation.add(TABLE, Util.cellname(CELLNAME + ii), bytes, System.currentTimeMillis());
+                    hash = hash(hash, bytes);
+                    ++cells;
+                    dataSize += sz;
+                }
+                rp = commitLog.add(mutation);
+                counter.incrementAndGet();
+            }
+        }
+    }
+}
diff --git a/test/unit/org/apache/cassandra/db/commitlog/SnapshotDeletingTest.java b/test/unit/org/apache/cassandra/db/commitlog/SnapshotDeletingTest.java
new file mode 100644
index 0000000..99cf72d
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/commitlog/SnapshotDeletingTest.java
@@ -0,0 +1,109 @@
+/*
+ * 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.db.commitlog;
+
+import org.junit.Assume;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.Mutation;
+import org.apache.cassandra.db.WindowsFailedSnapshotTracker;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.sstable.SnapshotDeletingTask;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.service.GCInspector;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+
+public class SnapshotDeletingTest
+{
+    private static final String KEYSPACE1 = "Keyspace1";
+    private static final String CF_STANDARD1 = "CF_STANDARD1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1));
+    }
+
+    @Test
+    public void testSnapshotDeletionFailure() throws Exception
+    {
+        Assume.assumeTrue(FBUtilities.isWindows());
+
+        GCInspector.register();
+
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore(CF_STANDARD1);
+        store.clearUnsafe();
+
+        // Needed to init the output file where we print failed snapshots. This is called on node startup.
+        WindowsFailedSnapshotTracker.deleteOldSnapshots();
+
+        populate(10000);
+        store.snapshot("snapshot1");
+
+        // Confirm snapshot deletion fails. Sleep for a bit just to make sure the SnapshotDeletingTask has
+        // time to run and fail.
+        Thread.sleep(500);
+        store.clearSnapshot("snapshot1");
+        assertEquals(1, SnapshotDeletingTask.pendingDeletionCount());
+
+        // Compact the cf and confirm that the executor's after hook calls rescheduleDeletion
+        populate(20000);
+        store.forceBlockingFlush();
+        store.forceMajorCompaction();
+
+        long start = System.currentTimeMillis();
+        while (System.currentTimeMillis() - start < 1000 && SnapshotDeletingTask.pendingDeletionCount() > 0)
+        {
+            Thread.yield();
+        }
+
+        assertEquals(0, SnapshotDeletingTask.pendingDeletionCount());
+    }
+
+    private long populate(int rowCount)
+    {
+        long timestamp = System.currentTimeMillis();
+        for (int i = 0; i <= rowCount; i++)
+        {
+            DecoratedKey key = Util.dk(Integer.toString(i));
+            Mutation rm = new Mutation(KEYSPACE1, key.getKey());
+            for (int j = 0; j < 10; j++)
+                rm.add(CF_STANDARD1,  Util.cellname(Integer.toString(j)),
+                       ByteBufferUtil.EMPTY_BYTE_BUFFER,
+                       timestamp,
+                       0);
+            rm.applyUnsafe();
+        }
+        return timestamp;
+    }
+}
diff --git a/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java b/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java
index 56af801..7c3fbc2 100644
--- a/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java
@@ -17,6 +17,7 @@
  */
 package org.apache.cassandra.db.compaction;
 
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.utils.concurrent.Refs;
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertEquals;
@@ -29,7 +30,15 @@
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
-import java.util.concurrent.ExecutionException;
+
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.junit.BeforeClass;
+import org.junit.After;
+import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
@@ -38,23 +47,40 @@
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.Mutation;
-import org.apache.cassandra.dht.BytesToken;
+import org.apache.cassandra.dht.ByteOrderedPartitioner.BytesToken;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.sstable.*;
-import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
+import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.ByteBufferUtil;
-import org.junit.After;
-import org.junit.Test;
 
 import com.google.common.collect.Iterables;
 
-public class AntiCompactionTest extends SchemaLoader
+public class AntiCompactionTest
 {
-    private static final String KEYSPACE1 = "Keyspace1";
+    private static final String KEYSPACE1 = "AntiCompactionTest";
     private static final String CF = "Standard1";
 
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                SimpleStrategy.class,
+                KSMetaData.optsWithRF(1),
+                SchemaLoader.standardCFMD(KEYSPACE1, CF));
+    }
+
+    @After
+    public void truncateCF()
+    {
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore(CF);
+        store.truncateBlocking();
+    }
+
     @Test
     public void antiCompactOne() throws Exception
     {
@@ -64,15 +90,18 @@
         Range<Token> range = new Range<Token>(new BytesToken("0".getBytes()), new BytesToken("4".getBytes()));
         List<Range<Token>> ranges = Arrays.asList(range);
 
-        Refs<SSTableReader> refs = Refs.tryRef(sstables);
-        if (refs == null)
-            throw new IllegalStateException();
-        long repairedAt = 1000;
-        CompactionManager.instance.performAnticompaction(store, ranges, refs, repairedAt);
-
-        assertEquals(2, store.getSSTables().size());
         int repairedKeys = 0;
         int nonRepairedKeys = 0;
+        try (LifecycleTransaction txn = store.getTracker().tryModify(sstables, OperationType.ANTICOMPACTION);
+             Refs<SSTableReader> refs = Refs.ref(sstables))
+        {
+            if (txn == null)
+                throw new IllegalStateException();
+            long repairedAt = 1000;
+            CompactionManager.instance.performAnticompaction(store, ranges, refs, txn, repairedAt);
+        }
+
+        assertEquals(2, store.getSSTables().size());
         for (SSTableReader sstable : store.getSSTables())
         {
             try (ISSTableScanner scanner = sstable.getScanner())
@@ -98,12 +127,13 @@
             assertFalse(sstable.isMarkedCompacted());
             assertEquals(1, sstable.selfRef().globalCount());
         }
-        assertEquals(0, store.getDataTracker().getCompacting().size());
+        assertEquals(0, store.getTracker().getCompacting().size());
         assertEquals(repairedKeys, 4);
         assertEquals(nonRepairedKeys, 6);
     }
+
     @Test
-    public void antiCompactionSizeTest() throws ExecutionException, InterruptedException, IOException
+    public void antiCompactionSizeTest() throws InterruptedException, IOException
     {
         Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
@@ -113,13 +143,16 @@
         long origSize = s.bytesOnDisk();
         Range<Token> range = new Range<Token>(new BytesToken(ByteBufferUtil.bytes(0)), new BytesToken(ByteBufferUtil.bytes(500)));
         Collection<SSTableReader> sstables = cfs.getSSTables();
-        CompactionManager.instance.performAnticompaction(cfs, Arrays.asList(range), Refs.tryRef(sstables), 12345);
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.ANTICOMPACTION);
+             Refs<SSTableReader> refs = Refs.ref(sstables))
+        {
+            CompactionManager.instance.performAnticompaction(cfs, Arrays.asList(range), refs, txn, 12345);
+        }
         long sum = 0;
         for (SSTableReader x : cfs.getSSTables())
             sum += x.bytesOnDisk();
-        assertEquals(sum, cfs.metric.liveDiskSpaceUsed.count());
-        assertEquals(origSize, cfs.metric.liveDiskSpaceUsed.count(), 100000);
-
+        assertEquals(sum, cfs.metric.liveDiskSpaceUsed.getCount());
+        assertEquals(origSize, cfs.metric.liveDiskSpaceUsed.getCount(), 100000);
     }
 
     private SSTableReader writeFile(ColumnFamilyStore cfs, int count)
@@ -130,39 +163,101 @@
         File dir = cfs.directories.getDirectoryForNewSSTables();
         String filename = cfs.getTempSSTablePath(dir);
 
-        SSTableWriter writer = new SSTableWriter(filename,
-                0,
-                0,
-                cfs.metadata,
-                StorageService.getPartitioner(),
-                new MetadataCollector(cfs.metadata.comparator));
+        try (SSTableWriter writer = SSTableWriter.create(filename, 0, 0);)
+        {
+            for (int i = 0; i < count * 5; i++)
+                writer.append(StorageService.getPartitioner().decorateKey(ByteBufferUtil.bytes(i)), cf);
+            return writer.finish(true);
+        }
+    }
 
-        for (int i = 0; i < count * 5; i++)
-            writer.append(StorageService.getPartitioner().decorateKey(ByteBufferUtil.bytes(i)), cf);
-        return writer.closeAndOpenReader();
+    public void generateSStable(ColumnFamilyStore store, String Suffix)
+    {
+    long timestamp = System.currentTimeMillis();
+    for (int i = 0; i < 10; i++)
+        {
+            DecoratedKey key = Util.dk(Integer.toString(i) + "-" + Suffix);
+            Mutation rm = new Mutation(KEYSPACE1, key.getKey());
+            for (int j = 0; j < 10; j++)
+                rm.add("Standard1", Util.cellname(Integer.toString(j)),
+                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
+                        timestamp,
+                        0);
+            rm.apply();
+        }
+        store.forceBlockingFlush();
     }
 
     @Test
-    public void shouldSkipAntiCompactionForNonIntersectingRange() throws InterruptedException, ExecutionException, IOException
+    public void antiCompactTenSTC() throws Exception
     {
-        ColumnFamilyStore store = prepareColumnFamilyStore();
+        antiCompactTen("SizeTieredCompactionStrategy");
+    }
+
+    @Test
+    public void antiCompactTenLC() throws Exception
+    {
+        antiCompactTen("LeveledCompactionStrategy");
+    }
+
+    public void antiCompactTen(String compactionStrategy) throws Exception
+    {
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore(CF);
+        store.setCompactionStrategyClass(compactionStrategy);
+        store.disableAutoCompaction();
+
+        for (int table = 0; table < 10; table++)
+        {
+            generateSStable(store,Integer.toString(table));
+        }
         Collection<SSTableReader> sstables = store.getUnrepairedSSTables();
         assertEquals(store.getSSTables().size(), sstables.size());
-        Range<Token> range = new Range<Token>(new BytesToken("-1".getBytes()), new BytesToken("-10".getBytes()));
+
+        Range<Token> range = new Range<Token>(new BytesToken("0".getBytes()), new BytesToken("4".getBytes()));
         List<Range<Token>> ranges = Arrays.asList(range);
 
-        Refs<SSTableReader> refs = Refs.tryRef(sstables);
-        if (refs == null)
-            throw new IllegalStateException();
-        CompactionManager.instance.performAnticompaction(store, ranges, refs, 1);
-        assertThat(store.getSSTables().size(), is(1));
-        assertThat(Iterables.get(store.getSSTables(), 0).isRepaired(), is(false));
-        assertThat(Iterables.get(store.getSSTables(), 0).selfRef().globalCount(), is(1));
-        assertThat(store.getDataTracker().getCompacting().size(), is(0));
+        long repairedAt = 1000;
+        try (LifecycleTransaction txn = store.getTracker().tryModify(sstables, OperationType.ANTICOMPACTION);
+             Refs<SSTableReader> refs = Refs.ref(sstables))
+        {
+            CompactionManager.instance.performAnticompaction(store, ranges, refs, txn, repairedAt);
+        }
+        /*
+        Anticompaction will be anti-compacting 10 SSTables but will be doing this two at a time
+        so there will be no net change in the number of sstables
+         */
+        assertEquals(10, store.getSSTables().size());
+        int repairedKeys = 0;
+        int nonRepairedKeys = 0;
+        for (SSTableReader sstable : store.getSSTables())
+        {
+            try(ISSTableScanner scanner = sstable.getScanner())
+            {
+                while (scanner.hasNext())
+                {
+                    SSTableIdentityIterator row = (SSTableIdentityIterator) scanner.next();
+                    if (sstable.isRepaired())
+                    {
+                        assertTrue(range.contains(row.getKey().getToken()));
+                        assertEquals(repairedAt, sstable.getSSTableMetadata().repairedAt);
+                        repairedKeys++;
+                    }
+                    else
+                    {
+                        assertFalse(range.contains(row.getKey().getToken()));
+                        assertEquals(ActiveRepairService.UNREPAIRED_SSTABLE, sstable.getSSTableMetadata().repairedAt);
+                        nonRepairedKeys++;
+                    }
+                }
+            }
+        }
+        assertEquals(repairedKeys, 40);
+        assertEquals(nonRepairedKeys, 60);
     }
 
     @Test
-    public void shouldMutateRepairedAt() throws InterruptedException, ExecutionException, IOException
+    public void shouldMutateRepairedAt() throws InterruptedException, IOException
     {
         ColumnFamilyStore store = prepareColumnFamilyStore();
         Collection<SSTableReader> sstables = store.getUnrepairedSSTables();
@@ -170,20 +265,51 @@
         Range<Token> range = new Range<Token>(new BytesToken("0".getBytes()), new BytesToken("9999".getBytes()));
         List<Range<Token>> ranges = Arrays.asList(range);
 
-        CompactionManager.instance.performAnticompaction(store, ranges, Refs.tryRef(sstables), 1);
+        try (LifecycleTransaction txn = store.getTracker().tryModify(sstables, OperationType.ANTICOMPACTION);
+             Refs<SSTableReader> refs = Refs.ref(sstables))
+        {
+            CompactionManager.instance.performAnticompaction(store, ranges, refs, txn, 1);
+        }
 
         assertThat(store.getSSTables().size(), is(1));
         assertThat(Iterables.get(store.getSSTables(), 0).isRepaired(), is(true));
         assertThat(Iterables.get(store.getSSTables(), 0).selfRef().globalCount(), is(1));
-        assertThat(store.getDataTracker().getCompacting().size(), is(0));
+        assertThat(store.getTracker().getCompacting().size(), is(0));
     }
 
 
+    @Test
+    public void shouldSkipAntiCompactionForNonIntersectingRange() throws InterruptedException, IOException
+    {
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore(CF);
+        store.disableAutoCompaction();
+
+        for (int table = 0; table < 10; table++)
+        {
+            generateSStable(store,Integer.toString(table));
+        }
+        Collection<SSTableReader> sstables = store.getUnrepairedSSTables();
+        assertEquals(store.getSSTables().size(), sstables.size());
+        
+        Range<Token> range = new Range<Token>(new BytesToken("-1".getBytes()), new BytesToken("-10".getBytes()));
+        List<Range<Token>> ranges = Arrays.asList(range);
+
+
+        try (LifecycleTransaction txn = store.getTracker().tryModify(sstables, OperationType.ANTICOMPACTION);
+             Refs<SSTableReader> refs = Refs.ref(sstables))
+        {
+            CompactionManager.instance.performAnticompaction(store, ranges, refs, txn, 1);
+        }
+
+        assertThat(store.getSSTables().size(), is(10));
+        assertThat(Iterables.get(store.getSSTables(), 0).isRepaired(), is(false));
+    }
+
     private ColumnFamilyStore prepareColumnFamilyStore()
     {
         Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore(CF);
-        store.truncateBlocking();
         store.disableAutoCompaction();
         long timestamp = System.currentTimeMillis();
         for (int i = 0; i < 10; i++)
@@ -200,9 +326,9 @@
         store.forceBlockingFlush();
         return store;
     }
-    
+
     @After
-    public void truncateCF()
+    public void truncateCfs()
     {
         Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore(CF);
diff --git a/test/unit/org/apache/cassandra/db/compaction/BlacklistingCompactionsTest.java b/test/unit/org/apache/cassandra/db/compaction/BlacklistingCompactionsTest.java
index 572ad36..2b6a62a 100644
--- a/test/unit/org/apache/cassandra/db/compaction/BlacklistingCompactionsTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/BlacklistingCompactionsTest.java
@@ -24,6 +24,7 @@
 import java.io.RandomAccessFile;
 import java.util.*;
 
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -31,9 +32,11 @@
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 import static org.junit.Assert.assertEquals;
@@ -41,9 +44,10 @@
 import static org.junit.Assert.assertTrue;
 import static org.apache.cassandra.Util.cellname;
 
-public class BlacklistingCompactionsTest extends SchemaLoader
+public class BlacklistingCompactionsTest
 {
-    public static final String KEYSPACE = "Keyspace1";
+    private static final String KEYSPACE1 = "BlacklistingCompactionsTest";
+    private static final String CF_STANDARD1 = "Standard1";
 
     @After
     public void leakDetect() throws InterruptedException
@@ -55,6 +59,16 @@
     }
 
     @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1));
+        closeStdErr();
+    }
+
     public static void closeStdErr()
     {
         // These tests generate an error message per CorruptSSTableException since it goes through
@@ -80,7 +94,7 @@
     public void testBlacklisting(String compactionStrategy) throws Exception
     {
         // this test does enough rows to force multiple block indexes to be used
-        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         final ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
 
         final int ROWS_PER_SSTABLE = 10;
@@ -99,11 +113,11 @@
             for (int i = 0; i < ROWS_PER_SSTABLE; i++)
             {
                 DecoratedKey key = Util.dk(String.valueOf(i % 2));
-                Mutation rm = new Mutation(KEYSPACE, key.getKey());
+                Mutation rm = new Mutation(KEYSPACE1, key.getKey());
                 long timestamp = j * ROWS_PER_SSTABLE + i;
                 rm.add("Standard1", cellname(i / 2), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp);
                 maxTimestampExpected = Math.max(timestamp, maxTimestampExpected);
-                rm.apply();
+                rm.applyUnsafe();
                 inserted.add(key);
             }
             cfs.forceBlockingFlush();
diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionAwareWriterTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionAwareWriterTest.java
new file mode 100644
index 0000000..235fd49
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionAwareWriterTest.java
@@ -0,0 +1,244 @@
+/*
+ * 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.db.compaction;
+
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import com.google.common.primitives.Longs;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.db.Cell;
+import org.apache.cassandra.db.ColumnFamily;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.Mutation;
+import org.apache.cassandra.db.compaction.writers.CompactionAwareWriter;
+import org.apache.cassandra.db.compaction.writers.DefaultCompactionWriter;
+import org.apache.cassandra.db.compaction.writers.MajorLeveledCompactionWriter;
+import org.apache.cassandra.db.compaction.writers.MaxSSTableSizeWriter;
+import org.apache.cassandra.db.compaction.writers.SplittingSizeTieredCompactionWriter;
+import org.apache.cassandra.db.filter.QueryFilter;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.locator.SimpleStrategy;
+
+import static org.junit.Assert.assertEquals;
+
+public class CompactionAwareWriterTest
+{
+    private static String KEYSPACE1 = "CompactionAwareWriterTest";
+    private static String CF = "Standard1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF));
+
+    }
+
+    @Before
+    public void clear()
+    {
+        // avoid one test affecting the next one
+        Keyspace ks = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore cfs = ks.getColumnFamilyStore(CF);
+        cfs.clearUnsafe();
+    }
+
+    @Test
+    public void testDefaultCompactionWriter()
+    {
+        Keyspace ks = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore cfs = ks.getColumnFamilyStore(CF);
+        int rowCount = 1000;
+        cfs.disableAutoCompaction();
+        populate(cfs, rowCount);
+        LifecycleTransaction txn = cfs.getTracker().tryModify(cfs.getSSTables(), OperationType.COMPACTION);
+        long beforeSize = txn.originals().iterator().next().onDiskLength();
+        CompactionAwareWriter writer = new DefaultCompactionWriter(cfs, txn, txn.originals(), false, OperationType.COMPACTION);
+        int rows = compact(cfs, txn, writer);
+        assertEquals(1, cfs.getSSTables().size());
+        assertEquals(rowCount, rows);
+        assertEquals(beforeSize, cfs.getSSTables().iterator().next().onDiskLength());
+        validateData(cfs, rowCount);
+        cfs.truncateBlocking();
+    }
+
+    @Test
+    public void testMaxSSTableSizeWriter()
+    {
+        Keyspace ks = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore cfs = ks.getColumnFamilyStore(CF);
+        cfs.disableAutoCompaction();
+        int rowCount = 1000;
+        populate(cfs, rowCount);
+        LifecycleTransaction txn = cfs.getTracker().tryModify(cfs.getSSTables(), OperationType.COMPACTION);
+        long beforeSize = txn.originals().iterator().next().onDiskLength();
+        int sstableSize = (int)beforeSize/10;
+        CompactionAwareWriter writer = new MaxSSTableSizeWriter(cfs, txn, txn.originals(), sstableSize, 0, false, OperationType.COMPACTION);
+        int rows = compact(cfs, txn, writer);
+        assertEquals(10, cfs.getSSTables().size());
+        assertEquals(rowCount, rows);
+        validateData(cfs, rowCount);
+        cfs.truncateBlocking();
+    }
+    @Test
+    public void testSplittingSizeTieredCompactionWriter()
+    {
+        Keyspace ks = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore cfs = ks.getColumnFamilyStore(CF);
+        cfs.disableAutoCompaction();
+        int rowCount = 10000;
+        populate(cfs, rowCount);
+        LifecycleTransaction txn = cfs.getTracker().tryModify(cfs.getSSTables(), OperationType.COMPACTION);
+        long beforeSize = txn.originals().iterator().next().onDiskLength();
+        CompactionAwareWriter writer = new SplittingSizeTieredCompactionWriter(cfs, txn, txn.originals(), OperationType.COMPACTION, 0);
+        int rows = compact(cfs, txn, writer);
+        long expectedSize = beforeSize / 2;
+        List<SSTableReader> sortedSSTables = new ArrayList<>(cfs.getSSTables());
+
+        Collections.sort(sortedSSTables, new Comparator<SSTableReader>()
+                                {
+                                    @Override
+                                    public int compare(SSTableReader o1, SSTableReader o2)
+                                    {
+                                        return Longs.compare(o2.onDiskLength(), o1.onDiskLength());
+                                    }
+                                });
+        for (SSTableReader sstable : sortedSSTables)
+        {
+            // we dont create smaller files than this, everything will be in the last file
+            if (expectedSize > SplittingSizeTieredCompactionWriter.DEFAULT_SMALLEST_SSTABLE_BYTES)
+                assertEquals(expectedSize, sstable.onDiskLength(), expectedSize / 100); // allow 1% diff in estimated vs actual size
+            expectedSize /= 2;
+        }
+        assertEquals(rowCount, rows);
+        validateData(cfs, rowCount);
+        cfs.truncateBlocking();
+    }
+
+    @Test
+    public void testMajorLeveledCompactionWriter()
+    {
+        Keyspace ks = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore cfs = ks.getColumnFamilyStore(CF);
+        cfs.disableAutoCompaction();
+        int rowCount = 20000;
+        int targetSSTableCount = 50;
+        populate(cfs, rowCount);
+        LifecycleTransaction txn = cfs.getTracker().tryModify(cfs.getSSTables(), OperationType.COMPACTION);
+        long beforeSize = txn.originals().iterator().next().onDiskLength();
+        int sstableSize = (int)beforeSize/targetSSTableCount;
+        CompactionAwareWriter writer = new MajorLeveledCompactionWriter(cfs, txn, txn.originals(), sstableSize, false, OperationType.COMPACTION);
+        int rows = compact(cfs, txn, writer);
+        assertEquals(targetSSTableCount, cfs.getSSTables().size());
+        int [] levelCounts = new int[5];
+        assertEquals(rowCount, rows);
+        for (SSTableReader sstable : cfs.getSSTables())
+        {
+            levelCounts[sstable.getSSTableLevel()]++;
+        }
+        assertEquals(0, levelCounts[0]);
+        assertEquals(10, levelCounts[1]);
+        assertEquals(targetSSTableCount - 10, levelCounts[2]); // note that if we want more levels, fix this
+        for (int i = 3; i < levelCounts.length; i++)
+            assertEquals(0, levelCounts[i]);
+        validateData(cfs, rowCount);
+        cfs.truncateBlocking();
+    }
+
+    private int compact(ColumnFamilyStore cfs, LifecycleTransaction txn, CompactionAwareWriter writer)
+    {
+        assert txn.originals().size() == 1;
+        int rowsWritten = 0;
+        try (AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategy().getScanners(txn.originals()))
+        {
+            CompactionController controller = new CompactionController(cfs, txn.originals(), cfs.gcBefore(System.currentTimeMillis()));
+            ISSTableScanner scanner = scanners.scanners.get(0);
+            while(scanner.hasNext())
+            {
+                AbstractCompactedRow row = new LazilyCompactedRow(controller, Arrays.asList(scanner.next()));
+                if (writer.append(row))
+                    rowsWritten++;
+            }
+        }
+        Collection<SSTableReader> newSSTables = writer.finish();
+        return rowsWritten;
+    }
+
+    private void populate(ColumnFamilyStore cfs, int count)
+    {
+        long timestamp = System.currentTimeMillis();
+        byte [] payload = new byte[1000];
+        new Random().nextBytes(payload);
+        ByteBuffer b = ByteBuffer.wrap(payload);
+        for (int i = 0; i < count; i++)
+        {
+            DecoratedKey key = Util.dk(Integer.toString(i));
+            Mutation rm = new Mutation(KEYSPACE1, key.getKey());
+            for (int j = 0; j < 10; j++)
+                rm.add(CF,  Util.cellname(Integer.toString(j)),
+                        b,
+                        timestamp);
+            rm.applyUnsafe();
+        }
+        cfs.forceBlockingFlush();
+        if (cfs.getSSTables().size() > 1)
+        {
+            // we want just one big sstable to avoid doing actual compaction in compact() above
+            try
+            {
+                cfs.forceMajorCompaction();
+            }
+            catch (Throwable t)
+            {
+                throw new RuntimeException(t);
+            }
+        }
+        assert cfs.getSSTables().size() == 1 : cfs.getSSTables();
+    }
+    private void validateData(ColumnFamilyStore cfs, int rowCount)
+    {
+        for (int i = 0; i < rowCount; i++)
+        {
+            ColumnFamily cf = cfs.getTopLevelColumns(QueryFilter.getIdentityFilter(Util.dk(Integer.toString(i)), CF, System.currentTimeMillis()), Integer.MAX_VALUE);
+            Iterator<Cell> iter = cf.iterator();
+            int cellCount = 0;
+            while (iter.hasNext())
+            {
+                Cell c = iter.next();
+                assertEquals(Util.cellname(Integer.toString(cellCount)), c.name());
+                cellCount++;
+            }
+            assertEquals(10, cellCount);
+        }
+    }
+}
diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionControllerTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionControllerTest.java
new file mode 100644
index 0000000..3184159
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionControllerTest.java
@@ -0,0 +1,197 @@
+/*
+ * 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.db.compaction;
+
+import java.nio.ByteBuffer;
+import java.util.Set;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.Sets;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.Mutation;
+import org.apache.cassandra.db.composites.CellName;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+
+import static org.apache.cassandra.Util.cellname;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertNotNull;
+
+public class CompactionControllerTest extends SchemaLoader
+{
+    private static final String KEYSPACE = "CompactionControllerTest";
+    private static final String CF1 = "Standard1";
+    private static final String CF2 = "Standard2";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF1),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF2));
+    }
+
+    @Test
+    public void testMaxPurgeableTimestamp()
+    {
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF1);
+        cfs.truncateBlocking();
+
+        ByteBuffer rowKey = ByteBufferUtil.bytes("k1");
+        DecoratedKey key = DatabaseDescriptor.getPartitioner().decorateKey(rowKey);
+
+        long timestamp1 = FBUtilities.timestampMicros(); // latest timestamp
+        long timestamp2 = timestamp1 - 5;
+        long timestamp3 = timestamp2 - 5; // oldest timestamp
+
+        // add to first memtable
+        applyMutation(CF1, rowKey, timestamp1);
+
+        // check max purgeable timestamp without any sstables
+        try(CompactionController controller = new CompactionController(cfs, null, 0))
+        {
+            assertPurgeBoundary(controller.getPurgeEvaluator(key), timestamp1); //memtable only
+
+            cfs.forceBlockingFlush();
+            assertTrue(controller.getPurgeEvaluator(key).apply(Long.MAX_VALUE)); //no memtables and no sstables
+        }
+
+        Set<SSTableReader> compacting = Sets.newHashSet(cfs.getSSTables()); // first sstable is compacting
+
+        // create another sstable
+        applyMutation(CF1, rowKey, timestamp2);
+        cfs.forceBlockingFlush();
+
+        // check max purgeable timestamp when compacting the first sstable with and without a memtable
+        try (CompactionController controller = new CompactionController(cfs, compacting, 0))
+        {
+            assertPurgeBoundary(controller.getPurgeEvaluator(key), timestamp2);
+
+            applyMutation(CF1, rowKey, timestamp3);
+
+            assertPurgeBoundary(controller.getPurgeEvaluator(key), timestamp3); //second sstable and second memtable
+        }
+
+        // check max purgeable timestamp again without any sstables but with different insertion orders on the memtable
+        cfs.forceBlockingFlush();
+
+        //newest to oldest
+        try (CompactionController controller = new CompactionController(cfs, null, 0))
+        {
+            applyMutation(CF1, rowKey, timestamp1);
+            applyMutation(CF1, rowKey, timestamp2);
+            applyMutation(CF1, rowKey, timestamp3);
+
+            assertPurgeBoundary(controller.getPurgeEvaluator(key), timestamp3); //memtable only
+        }
+
+        cfs.forceBlockingFlush();
+
+        //oldest to newest
+        try (CompactionController controller = new CompactionController(cfs, null, 0))
+        {
+            applyMutation(CF1, rowKey, timestamp3);
+            applyMutation(CF1, rowKey, timestamp2);
+            applyMutation(CF1, rowKey, timestamp1);
+
+            assertPurgeBoundary(controller.getPurgeEvaluator(key), timestamp3);
+        }
+    }
+
+    @Test
+    public void testGetFullyExpiredSSTables()
+    {
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF2);
+        cfs.truncateBlocking();
+
+        ByteBuffer rowKey = ByteBufferUtil.bytes("k1");
+
+        long timestamp1 = FBUtilities.timestampMicros(); // latest timestamp
+        long timestamp2 = timestamp1 - 5;
+        long timestamp3 = timestamp2 - 5; // oldest timestamp
+
+        // create sstable with tombstone that should be expired in no older timestamps
+        applyDeleteMutation(CF2, rowKey, timestamp2);
+        cfs.forceBlockingFlush();
+
+        // first sstable with tombstone is compacting
+        Set<SSTableReader> compacting = Sets.newHashSet(cfs.getSSTables());
+
+        // create another sstable with more recent timestamp
+        applyMutation(CF2, rowKey, timestamp1);
+        cfs.forceBlockingFlush();
+
+        // second sstable is overlapping
+        Set<SSTableReader> overlapping = Sets.difference(Sets.newHashSet(cfs.getSSTables()), compacting);
+
+        // the first sstable should be expired because the overlapping sstable is newer and the gc period is later
+        int gcBefore = (int) (System.currentTimeMillis() / 1000) + 5;
+        Set<SSTableReader> expired = CompactionController.getFullyExpiredSSTables(cfs, compacting, overlapping, gcBefore);
+        assertNotNull(expired);
+        assertEquals(1, expired.size());
+        assertEquals(compacting.iterator().next(), expired.iterator().next());
+
+        // however if we add an older mutation to the memtable then the sstable should not be expired
+        applyMutation(CF2, rowKey, timestamp3);
+        expired = CompactionController.getFullyExpiredSSTables(cfs, compacting, overlapping, gcBefore);
+        assertNotNull(expired);
+        assertEquals(0, expired.size());
+    }
+
+    private void applyMutation(String cf, ByteBuffer rowKey, long timestamp)
+    {
+        CellName colName = cellname("birthdate");
+        ByteBuffer val = ByteBufferUtil.bytes(1L);
+
+        Mutation rm = new Mutation(KEYSPACE, rowKey);
+        rm.add(cf, colName, val, timestamp);
+        rm.applyUnsafe();
+    }
+
+    private void applyDeleteMutation(String cf, ByteBuffer rowKey, long timestamp)
+    {
+        Mutation rm = new Mutation(KEYSPACE, rowKey);
+        rm.delete(cf, timestamp);
+        rm.applyUnsafe();
+    }
+
+    private void assertPurgeBoundary(Predicate<Long> evaluator, long boundary)
+    {
+        assertFalse(evaluator.apply(boundary));
+        assertTrue(evaluator.apply(boundary - 1));
+    }
+}
diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionsCQLTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionsCQLTest.java
index c8fd781..1d77b17 100644
--- a/test/unit/org/apache/cassandra/db/compaction/CompactionsCQLTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionsCQLTest.java
@@ -26,7 +26,7 @@
 import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -64,9 +64,9 @@
     {
         createTable("CREATE TABLE %s (id text PRIMARY KEY) WITH compaction = {'class':'DateTieredCompactionStrategy', 'min_threshold':2};");
         assertTrue(getCurrentColumnFamilyStore().getCompactionStrategy().isEnabled());
-        execute("insert into %s (id) values ('1')");
+        execute("insert into %s (id) values ('1') using timestamp 1000"); // same timestamp = same window = minor compaction triggered
         flush();
-        execute("insert into %s (id) values ('1')");
+        execute("insert into %s (id) values ('1') using timestamp 1000");
         flush();
         waitForMinor(KEYSPACE, currentTable(), 5000, true);
     }
diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java
index c6e9445..4a1f2ca 100644
--- a/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java
@@ -21,20 +21,28 @@
 import java.util.Collection;
 import java.util.concurrent.ExecutionException;
 
+import org.apache.cassandra.cache.CachingOptions;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.*;
 
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.db.filter.QueryFilter;
-import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.Util;
 
 import static org.junit.Assert.assertEquals;
 import static org.apache.cassandra.db.KeyspaceTest.assertColumns;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
 
 import static org.apache.cassandra.cql3.QueryProcessor.executeInternal;
 
@@ -43,13 +51,90 @@
 import org.apache.cassandra.utils.FBUtilities;
 
 
-public class CompactionsPurgeTest extends SchemaLoader
+public class CompactionsPurgeTest
 {
-    public static final String KEYSPACE1 = "Keyspace1";
-    public static final String KEYSPACE2 = "Keyspace2";
+    private static final String KEYSPACE1 = "CompactionsPurgeTest1";
+    private static final String CF_STANDARD1 = "Standard1";
+    private static final String CF_STANDARD2 = "Standard2";
+    private static final String KEYSPACE2 = "CompactionsPurgeTest2";
+    private static final String KEYSPACE_CACHED = "CompactionsPurgeTestCached";
+    private static final String CF_CACHED = "CachedCF";
+    private static final String KEYSPACE_CQL = "cql_keyspace";
+    private static final String CF_CQL = "table1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD2));
+        SchemaLoader.createKeyspace(KEYSPACE2,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE2, CF_STANDARD1));
+        SchemaLoader.createKeyspace(KEYSPACE_CACHED,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE_CACHED, CF_CACHED).caching(CachingOptions.ALL));
+        SchemaLoader.createKeyspace(KEYSPACE_CQL,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    CFMetaData.compile("CREATE TABLE " + CF_CQL + " ("
+                                                     + "k int PRIMARY KEY,"
+                                                     + "v1 text,"
+                                                     + "v2 int"
+                                                     + ")", KEYSPACE_CQL));
+    }
 
     @Test
-    public void testMajorCompactionPurge() throws ExecutionException, InterruptedException
+    public void testMajorCompactionPurge()
+    {
+        CompactionManager.instance.disableAutoCompaction();
+
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        String cfName = "Standard1";
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
+
+        DecoratedKey key = Util.dk("key1");
+        Mutation rm;
+
+        // inserts
+        rm = new Mutation(KEYSPACE1, key.getKey());
+        for (int i = 0; i < 10; i++)
+        {
+            rm.add(cfName, cellname(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
+        }
+        rm.applyUnsafe();
+        cfs.forceBlockingFlush();
+
+        // deletes
+        for (int i = 0; i < 10; i++)
+        {
+            rm = new Mutation(KEYSPACE1, key.getKey());
+            rm.delete(cfName, cellname(String.valueOf(i)), 1);
+            rm.applyUnsafe();
+        }
+        cfs.forceBlockingFlush();
+
+        // resurrect one column
+        rm = new Mutation(KEYSPACE1, key.getKey());
+        rm.add(cfName, cellname(String.valueOf(5)), ByteBufferUtil.EMPTY_BYTE_BUFFER, 2);
+        rm.applyUnsafe();
+        cfs.forceBlockingFlush();
+
+        // major compact and test that all columns but the resurrected one is completely gone
+        FBUtilities.waitOnFutures(CompactionManager.instance.submitMaximal(cfs, Integer.MAX_VALUE, false));
+        cfs.invalidateCachedRow(key);
+        ColumnFamily cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(key, cfName, System.currentTimeMillis()));
+        assertColumns(cf, "5");
+        assertNotNull(cf.getColumn(cellname(String.valueOf(5))));
+    }
+
+    @Test
+    public void testMajorCompactionPurgeTombstonesWithMaxTimestamp()
     {
         CompactionManager.instance.disableAutoCompaction();
 
@@ -73,19 +158,106 @@
         for (int i = 0; i < 10; i++)
         {
             rm = new Mutation(KEYSPACE1, key.getKey());
-            rm.delete(cfName, cellname(String.valueOf(i)), 1);
+            rm.delete(cfName, cellname(String.valueOf(i)), Long.MAX_VALUE);
             rm.apply();
         }
         cfs.forceBlockingFlush();
 
+        // major compact - tombstones should be purged
+        FBUtilities.waitOnFutures(CompactionManager.instance.submitMaximal(cfs, Integer.MAX_VALUE, false));
+
         // resurrect one column
         rm = new Mutation(KEYSPACE1, key.getKey());
         rm.add(cfName, cellname(String.valueOf(5)), ByteBufferUtil.EMPTY_BYTE_BUFFER, 2);
         rm.apply();
         cfs.forceBlockingFlush();
 
-        // major compact and test that all columns but the resurrected one is completely gone
-        FBUtilities.waitOnFutures(CompactionManager.instance.submitMaximal(cfs, Integer.MAX_VALUE));
+        cfs.invalidateCachedRow(key);
+        ColumnFamily cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(key, cfName, System.currentTimeMillis()));
+        assertColumns(cf, "5");
+        assert cf.getColumn(cellname(String.valueOf(5))) != null;
+    }
+
+    @Test
+    public void testMajorCompactionPurgeTopLevelTombstoneWithMaxTimestamp()
+    {
+        CompactionManager.instance.disableAutoCompaction();
+
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        String cfName = "Standard1";
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
+
+        DecoratedKey key = Util.dk("key1");
+        Mutation rm;
+
+        // inserts
+        rm = new Mutation(KEYSPACE1, key.getKey());
+        for (int i = 0; i < 10; i++)
+        {
+            rm.add(cfName, cellname(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
+        }
+        rm.apply();
+        cfs.forceBlockingFlush();
+
+        // delete
+        rm = new Mutation(KEYSPACE1, key.getKey());
+        rm.delete(cfName, Long.MAX_VALUE);
+        rm.apply();
+
+        cfs.forceBlockingFlush();
+
+        // major compact - tombstone should be purged
+        FBUtilities.waitOnFutures(CompactionManager.instance.submitMaximal(cfs, Integer.MAX_VALUE, false));
+
+        // resurrect one column
+        rm = new Mutation(KEYSPACE1, key.getKey());
+        rm.add(cfName, cellname(String.valueOf(5)), ByteBufferUtil.EMPTY_BYTE_BUFFER, 2);
+        rm.apply();
+        cfs.forceBlockingFlush();
+
+        cfs.invalidateCachedRow(key);
+        ColumnFamily cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(key, cfName, System.currentTimeMillis()));
+        assertColumns(cf, "5");
+        assert cf.getColumn(cellname(String.valueOf(5))) != null;
+    }
+
+    @Test
+    public void testMajorCompactionPurgeRangeTombstoneWithMaxTimestamp()
+    {
+        CompactionManager.instance.disableAutoCompaction();
+
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        String cfName = "Standard1";
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
+
+        DecoratedKey key = Util.dk("key1");
+        Mutation rm;
+
+        // inserts
+        rm = new Mutation(KEYSPACE1, key.getKey());
+        for (int i = 0; i < 10; i++)
+        {
+            rm.add(cfName, cellname(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
+        }
+        rm.apply();
+        cfs.forceBlockingFlush();
+
+        // delete
+        rm = new Mutation(KEYSPACE1, key.getKey());
+        rm.deleteRange(cfName, cellname(String.valueOf(0)), cellname(String.valueOf(9)), Long.MAX_VALUE);
+        rm.apply();
+
+        cfs.forceBlockingFlush();
+
+        // major compact - tombstone should be purged
+        FBUtilities.waitOnFutures(CompactionManager.instance.submitMaximal(cfs, Integer.MAX_VALUE, false));
+
+        // resurrect one column
+        rm = new Mutation(KEYSPACE1, key.getKey());
+        rm.add(cfName, cellname(String.valueOf(5)), ByteBufferUtil.EMPTY_BYTE_BUFFER, 2);
+        rm.apply();
+        cfs.forceBlockingFlush();
+
         cfs.invalidateCachedRow(key);
         ColumnFamily cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(key, cfName, System.currentTimeMillis()));
         assertColumns(cf, "5");
@@ -111,7 +283,7 @@
             {
                 rm.add(cfName, cellname(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
             }
-            rm.apply();
+            rm.applyUnsafe();
             cfs.forceBlockingFlush();
 
             // deletes
@@ -119,7 +291,7 @@
             {
                 rm = new Mutation(KEYSPACE2, key.getKey());
                 rm.delete(cfName, cellname(String.valueOf(i)), 1);
-                rm.apply();
+                rm.applyUnsafe();
             }
             cfs.forceBlockingFlush();
         }
@@ -133,14 +305,14 @@
         Collection<SSTableReader> sstablesIncomplete = cfs.getSSTables();
         rm = new Mutation(KEYSPACE2, key1.getKey());
         rm.add(cfName, cellname(String.valueOf(5)), ByteBufferUtil.EMPTY_BYTE_BUFFER, 2);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
         cfs.getCompactionStrategy().getUserDefinedTask(sstablesIncomplete, Integer.MAX_VALUE).execute(null);
 
         // verify that minor compaction does GC when key is provably not
         // present in a non-compacted sstable
         ColumnFamily cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(key2, cfName, System.currentTimeMillis()));
-        assert cf == null;
+        assertNull(cf);
 
         // verify that minor compaction still GC when key is present
         // in a non-compacted sstable but the timestamp ensures we won't miss anything
@@ -166,19 +338,19 @@
         rm = new Mutation(KEYSPACE2, key3.getKey());
         rm.add(cfName, cellname("c1"), ByteBufferUtil.EMPTY_BYTE_BUFFER, 8);
         rm.add(cfName, cellname("c2"), ByteBufferUtil.EMPTY_BYTE_BUFFER, 8);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
         // delete c1
         rm = new Mutation(KEYSPACE2, key3.getKey());
         rm.delete(cfName, cellname("c1"), 10);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
         Collection<SSTableReader> sstablesIncomplete = cfs.getSSTables();
 
         // delete c2 so we have new delete in a diffrent SSTable
         rm = new Mutation(KEYSPACE2, key3.getKey());
         rm.delete(cfName, cellname("c2"), 9);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         // compact the sstables with the c1/c2 data and the c1 tombstone
@@ -209,23 +381,23 @@
         {
             rm.add(cfName, cellname(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
         }
-        rm.apply();
+        rm.applyUnsafe();
 
         // deletes
         for (int i = 0; i < 5; i++)
         {
             rm = new Mutation(KEYSPACE1, key.getKey());
             rm.delete(cfName, cellname(String.valueOf(i)), 1);
-            rm.apply();
+            rm.applyUnsafe();
         }
         cfs.forceBlockingFlush();
-        assert cfs.getSSTables().size() == 1 : cfs.getSSTables(); // inserts & deletes were in the same memtable -> only deletes in sstable
+        assertEquals(String.valueOf(cfs.getSSTables()), 1, cfs.getSSTables().size()); // inserts & deletes were in the same memtable -> only deletes in sstable
 
         // compact and test that the row is completely gone
         Util.compactAll(cfs, Integer.MAX_VALUE).get();
-        assert cfs.getSSTables().isEmpty();
+        assertTrue(cfs.getSSTables().isEmpty());
         ColumnFamily cf = keyspace.getColumnFamilyStore(cfName).getColumnFamily(QueryFilter.getIdentityFilter(key, cfName, System.currentTimeMillis()));
-        assert cf == null : cf;
+        assertNull(String.valueOf(cf), cf);
     }
 
     @Test
@@ -233,8 +405,8 @@
     {
         CompactionManager.instance.disableAutoCompaction();
 
-        String keyspaceName = "RowCacheSpace";
-        String cfName = "CachedCF";
+        String keyspaceName = KEYSPACE_CACHED;
+        String cfName = CF_CACHED;
         Keyspace keyspace = Keyspace.open(keyspaceName);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
 
@@ -247,7 +419,7 @@
         {
             rm.add(cfName, cellname(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
         }
-        rm.apply();
+        rm.applyUnsafe();
 
         // move the key up in row cache
         cfs.getColumnFamily(QueryFilter.getIdentityFilter(key, cfName, System.currentTimeMillis()));
@@ -255,7 +427,7 @@
         // deletes row
         rm = new Mutation(keyspaceName, key.getKey());
         rm.delete(cfName, 1);
-        rm.apply();
+        rm.applyUnsafe();
 
         // flush and major compact
         cfs.forceBlockingFlush();
@@ -267,13 +439,13 @@
         {
             rm.add(cfName, cellname(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
         }
-        rm.apply();
+        rm.applyUnsafe();
 
         // Check that the second insert did went in
         ColumnFamily cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(key, cfName, System.currentTimeMillis()));
         assertEquals(10, cf.getColumnCount());
         for (Cell c : cf)
-            assert c.isLive();
+            assertTrue(c.isLive());
     }
 
     @Test
@@ -281,7 +453,7 @@
     {
         CompactionManager.instance.disableAutoCompaction();
 
-        String keyspaceName = "Keyspace1";
+        String keyspaceName = KEYSPACE1;
         String cfName = "Standard1";
         Keyspace keyspace = Keyspace.open(keyspaceName);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
@@ -293,12 +465,12 @@
         rm = new Mutation(keyspaceName, key.getKey());
         for (int i = 0; i < 10; i++)
             rm.add(cfName, cellname(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, i);
-        rm.apply();
+        rm.applyUnsafe();
 
         // deletes row with timestamp such that not all columns are deleted
         rm = new Mutation(keyspaceName, key.getKey());
         rm.delete(cfName, 4);
-        rm.apply();
+        rm.applyUnsafe();
         ColumnFamily cf = cfs.getColumnFamily(filter);
         assertTrue(cf.isMarkedForDelete());
 
@@ -311,17 +483,17 @@
         rm = new Mutation(keyspaceName, key.getKey());
         for (int i = 0; i < 5; i++)
             rm.add(cfName, cellname(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, i);
-        rm.apply();
+        rm.applyUnsafe();
 
         // Check that the second insert went in
         cf = cfs.getColumnFamily(filter);
         assertEquals(10, cf.getColumnCount());
         for (Cell c : cf)
-            assert c.isLive();
+            assertTrue(c.isLive());
     }
 
     @Test
-    public void testRowTombstoneObservedBeforePurging() throws InterruptedException, ExecutionException
+    public void testRowTombstoneObservedBeforePurging()
     {
         String keyspace = "cql_keyspace";
         String table = "table1";
@@ -346,7 +518,7 @@
         assertEquals(0, result.size());
 
         // compact the two sstables with a gcBefore that does *not* allow the row tombstone to be purged
-        FBUtilities.waitOnFutures(CompactionManager.instance.submitMaximal(cfs, (int) (System.currentTimeMillis() / 1000) - 10000));
+        FBUtilities.waitOnFutures(CompactionManager.instance.submitMaximal(cfs, (int) (System.currentTimeMillis() / 1000) - 10000, false));
 
         // the data should be gone, but the tombstone should still exist
         assertEquals(1, cfs.getSSTables().size());
@@ -366,7 +538,7 @@
         cfs.forceBlockingFlush();
 
         // compact the two sstables with a gcBefore that *does* allow the row tombstone to be purged
-        FBUtilities.waitOnFutures(CompactionManager.instance.submitMaximal(cfs, (int) (System.currentTimeMillis() / 1000) + 10000));
+        FBUtilities.waitOnFutures(CompactionManager.instance.submitMaximal(cfs, (int) (System.currentTimeMillis() / 1000) + 10000, false));
 
         // both the data and the tombstone should be gone this time
         assertEquals(0, cfs.getSSTables().size());
diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
index d2708f0..471f8cf 100644
--- a/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
@@ -18,48 +18,78 @@
 */
 package org.apache.cassandra.db.compaction;
 
-import java.io.*;
+import java.io.File;
+import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.*;
-import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.cassandra.OrderedJUnit4ClassRunner;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.columniterator.IdentityQueryFilter;
+import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
+import org.apache.cassandra.db.filter.QueryFilter;
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.dht.*;
+import org.apache.cassandra.io.sstable.*;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Pair;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
 import com.google.common.base.Function;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Sets;
 
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
-import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.Util;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.columniterator.IdentityQueryFilter;
-import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
-import org.apache.cassandra.db.filter.QueryFilter;
-import org.apache.cassandra.dht.*;
-import org.apache.cassandra.io.sstable.*;
-import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
-import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
-import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.Pair;
-
 import static org.junit.Assert.*;
 
 @RunWith(OrderedJUnit4ClassRunner.class)
-public class CompactionsTest extends SchemaLoader
+public class CompactionsTest
 {
-    private static final String STANDARD1 = "Standard1";
-    public static final String KEYSPACE1 = "Keyspace1";
+    private static final String KEYSPACE1 = "Keyspace1";
+    private static final String CF_STANDARD1 = "CF_STANDARD1";
+    private static final String CF_STANDARD2 = "Standard2";
+    private static final String CF_STANDARD3 = "Standard3";
+    private static final String CF_STANDARD4 = "Standard4";
+    private static final String CF_SUPER1 = "Super1";
+    private static final String CF_SUPER5 = "Super5";
+    private static final String CF_SUPERGC = "SuperDirectGC";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        Map<String, String> compactionOptions = new HashMap<>();
+        compactionOptions.put("tombstone_compaction_interval", "1");
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1).compactionStrategyOptions(compactionOptions),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD2),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD3),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD4),
+                                    SchemaLoader.superCFMD(KEYSPACE1, CF_SUPER1, LongType.instance),
+                                    SchemaLoader.superCFMD(KEYSPACE1, CF_SUPER5, BytesType.instance),
+                                    SchemaLoader.superCFMD(KEYSPACE1, CF_SUPERGC, BytesType.instance).gcGraceSeconds(0));
+    }
 
     public ColumnFamilyStore testSingleSSTableCompaction(String strategyClassName) throws Exception
     {
         Keyspace keyspace = Keyspace.open(KEYSPACE1);
-        ColumnFamilyStore store = keyspace.getColumnFamilyStore(STANDARD1);
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore(CF_STANDARD1);
         store.clearUnsafe();
         store.metadata.gcGraceSeconds(1);
         store.setCompactionStrategyClass(strategyClassName);
@@ -67,7 +97,7 @@
         // disable compaction while flushing
         store.disableAutoCompaction();
 
-        long timestamp = populate(KEYSPACE1, STANDARD1, 0, 9, 3); //ttl=3s
+        long timestamp = populate(KEYSPACE1, CF_STANDARD1, 0, 9, 3); //ttl=3s
 
         store.forceBlockingFlush();
         assertEquals(1, store.getSSTables().size());
@@ -95,7 +125,8 @@
         return store;
     }
 
-    private long populate(String ks, String cf, int startRowKey, int endRowKey, int ttl) {
+    public static long populate(String ks, String cf, int startRowKey, int endRowKey, int ttl)
+    {
         long timestamp = System.currentTimeMillis();
         for (int i = startRowKey; i <= endRowKey; i++)
         {
@@ -106,7 +137,7 @@
                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
                        timestamp,
                        j > 0 ? ttl : 0); // let first column never expire, since deleting all columns does not produce sstable
-            rm.apply();
+            rm.applyUnsafe();
         }
         return timestamp;
     }
@@ -130,7 +161,7 @@
     }
 
     @Test
-    public void testSuperColumnTombstones() throws ExecutionException, InterruptedException
+    public void testSuperColumnTombstones() throws IOException
     {
         Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Super1");
@@ -144,16 +175,16 @@
         rm.add("Super1", Util.cellname(scName, ByteBufferUtil.bytes(0)),
                ByteBufferUtil.EMPTY_BYTE_BUFFER,
                FBUtilities.timestampMicros());
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
         // shadow the subcolumn with a supercolumn tombstone
         rm = new Mutation(KEYSPACE1, key.getKey());
         rm.deleteRange("Super1", SuperColumns.startOf(scName), SuperColumns.endOf(scName), FBUtilities.timestampMicros());
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
-        CompactionManager.instance.performMaximal(cfs);
+        CompactionManager.instance.performMaximal(cfs, false);
         assertEquals(1, cfs.getSSTables().size());
 
         // check that the shadowed column is gone
@@ -162,15 +193,16 @@
         ISSTableScanner scanner = sstable.getScanner(new DataRange(bounds, new IdentityQueryFilter()));
         OnDiskAtomIterator iter = scanner.next();
         assertEquals(key, iter.getKey());
-        assert iter.next() instanceof RangeTombstone;
-        assert !iter.hasNext();
+        assertTrue(iter.next() instanceof RangeTombstone);
+        assertFalse(iter.hasNext());
+        scanner.close();
     }
 
     @Test
     public void testUncheckedTombstoneSizeTieredCompaction() throws Exception
     {
         Keyspace keyspace = Keyspace.open(KEYSPACE1);
-        ColumnFamilyStore store = keyspace.getColumnFamilyStore(STANDARD1);
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore(CF_STANDARD1);
         store.clearUnsafe();
         store.metadata.gcGraceSeconds(1);
         store.metadata.compactionStrategyOptions.put("tombstone_compaction_interval", "1");
@@ -182,11 +214,11 @@
         store.disableAutoCompaction();
 
         //Populate sstable1 with with keys [0..9]
-        populate(KEYSPACE1, STANDARD1, 0, 9, 3); //ttl=3s
+        populate(KEYSPACE1, CF_STANDARD1, 0, 9, 3); //ttl=3s
         store.forceBlockingFlush();
 
         //Populate sstable2 with with keys [10..19] (keys do not overlap with SSTable1)
-        long timestamp2 = populate(KEYSPACE1, STANDARD1, 10, 19, 3); //ttl=3s
+        long timestamp2 = populate(KEYSPACE1, CF_STANDARD1, 10, 19, 3); //ttl=3s
         store.forceBlockingFlush();
 
         assertEquals(2, store.getSSTables().size());
@@ -213,9 +245,9 @@
         long newSize1 = it.next().uncompressedLength();
         long newSize2 = it.next().uncompressedLength();
         assertEquals("candidate sstable should not be tombstone-compacted because its key range overlap with other sstable",
-                      originalSize1, newSize1);
+                     originalSize1, newSize1);
         assertEquals("candidate sstable should not be tombstone-compacted because its key range overlap with other sstable",
-                      originalSize2, newSize2);
+                     originalSize2, newSize2);
 
         // now let's enable the magic property
         store.metadata.compactionStrategyOptions.put("unchecked_tombstone_compaction", "true");
@@ -266,13 +298,13 @@
             DecoratedKey key = Util.dk(String.valueOf(i));
             Mutation rm = new Mutation(KEYSPACE1, key.getKey());
             rm.add("Standard2", Util.cellname(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, i);
-            rm.apply();
+            rm.applyUnsafe();
 
             if (i % 2 == 0)
                 cfs.forceBlockingFlush();
         }
         Collection<SSTableReader> toCompact = cfs.getSSTables();
-        assert toCompact.size() == 2;
+        assertEquals(2, toCompact.size());
 
         // Reinserting the same keys. We will compact only the previous sstable, but we need those new ones
         // to make sure we use EchoedRow, otherwise it won't be used because purge can be done.
@@ -281,14 +313,14 @@
             DecoratedKey key = Util.dk(String.valueOf(i));
             Mutation rm = new Mutation(KEYSPACE1, key.getKey());
             rm.add("Standard2", Util.cellname(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, i);
-            rm.apply();
+            rm.applyUnsafe();
         }
         cfs.forceBlockingFlush();
         SSTableReader tmpSSTable = null;
         for (SSTableReader sstable : cfs.getSSTables())
             if (!toCompact.contains(sstable))
                 tmpSSTable = sstable;
-        assert tmpSSTable != null;
+        assertNotNull(tmpSSTable);
 
         // Force compaction on first sstables. Since each row is in only one sstable, we will be using EchoedRow.
         Util.compact(cfs, toCompact);
@@ -328,16 +360,16 @@
             rm.add(cfname, Util.cellname("col"),
                    ByteBufferUtil.EMPTY_BYTE_BUFFER,
                    System.currentTimeMillis());
-            rm.apply();
+            rm.applyUnsafe();
         }
         cfs.forceBlockingFlush();
         Collection<SSTableReader> sstables = cfs.getSSTables();
 
-        assert sstables.size() == 1;
+        assertEquals(1, sstables.size());
         SSTableReader sstable = sstables.iterator().next();
 
         int prevGeneration = sstable.descriptor.generation;
-        String file = new File(sstable.descriptor.filenameFor(Component.DATA)).getName();
+        String file = new File(sstable.descriptor.filenameFor(Component.DATA)).getAbsolutePath();
         // submit user defined compaction on flushed sstable
         CompactionManager.instance.forceUserDefinedCompaction(file);
         // wait until user defined compaction finishes
@@ -347,12 +379,12 @@
         } while (CompactionManager.instance.getPendingTasks() > 0 || CompactionManager.instance.getActiveCompactions() > 0);
         // CF should have only one sstable with generation number advanced
         sstables = cfs.getSSTables();
-        assert sstables.size() == 1;
-        assert sstables.iterator().next().descriptor.generation == prevGeneration + 1;
+        assertEquals(1, sstables.size());
+        assertEquals( prevGeneration + 1, sstables.iterator().next().descriptor.generation);
     }
 
     @Test
-    public void testRangeTombstones() throws IOException, ExecutionException, InterruptedException
+    public void testRangeTombstones()
     {
         Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard2");
@@ -376,31 +408,24 @@
         cf.addColumn(Util.column("a", "a", 3));
         cf.deletionInfo().add(new RangeTombstone(Util.cellname("0"), Util.cellname("b"), 2, (int) (System.currentTimeMillis()/1000)),cfmeta.comparator);
 
-        SSTableWriter writer = new SSTableWriter(cfs.getTempSSTablePath(dir.getDirectoryForNewSSTables()),
-                                                 0,
-                                                 0,
-                                                 cfs.metadata,
-                                                 StorageService.getPartitioner(),
-                                                 new MetadataCollector(cfs.metadata.comparator));
+        try (SSTableWriter writer = SSTableWriter.create(Descriptor.fromFilename(cfs.getTempSSTablePath(dir.getDirectoryForNewSSTables())), 0, 0, 0);)
+        {
+            writer.append(Util.dk("0"), cf);
+            writer.append(Util.dk("1"), cf);
+            writer.append(Util.dk("3"), cf);
 
+            cfs.addSSTable(writer.finish(true));
+        }
 
-        writer.append(Util.dk("0"), cf);
-        writer.append(Util.dk("1"), cf);
-        writer.append(Util.dk("3"), cf);
+        try (SSTableWriter writer = SSTableWriter.create(Descriptor.fromFilename(cfs.getTempSSTablePath(dir.getDirectoryForNewSSTables())), 0, 0, 0);)
+        {
+            writer.append(Util.dk("0"), cf);
+            writer.append(Util.dk("1"), cf);
+            writer.append(Util.dk("2"), cf);
+            writer.append(Util.dk("3"), cf);
+            cfs.addSSTable(writer.finish(true));
+        }
 
-        cfs.addSSTable(writer.closeAndOpenReader());
-        writer = new SSTableWriter(cfs.getTempSSTablePath(dir.getDirectoryForNewSSTables()),
-                                   0,
-                                   0,
-                                   cfs.metadata,
-                                   StorageService.getPartitioner(),
-                                   new MetadataCollector(cfs.metadata.comparator));
-
-        writer.append(Util.dk("0"), cf);
-        writer.append(Util.dk("1"), cf);
-        writer.append(Util.dk("2"), cf);
-        writer.append(Util.dk("3"), cf);
-        cfs.addSSTable(writer.closeAndOpenReader());
 
         Collection<SSTableReader> toCompact = cfs.getSSTables();
         assert toCompact.size() == 2;
@@ -438,11 +463,11 @@
 
         String cf = "Standard4";
         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(cf);
-        insertData(KEYSPACE1, cf, 0, 1);
+        SchemaLoader.insertData(KEYSPACE1, cf, 0, 1);
         cfs.forceBlockingFlush();
 
         Collection<SSTableReader> sstables = cfs.getSSTables();
-        assert !sstables.isEmpty();
+        assertFalse(sstables.isEmpty());
         Set<Integer> generations = Sets.newHashSet(Iterables.transform(sstables, new Function<SSTableReader, Integer>()
         {
             public Integer apply(SSTableReader sstable)
@@ -453,11 +478,11 @@
         UUID taskId = SystemKeyspace.startCompaction(cfs, sstables);
         Map<Pair<String, String>, Map<Integer, UUID>> compactionLogs = SystemKeyspace.getUnfinishedCompactions();
         Set<Integer> unfinishedCompactions = compactionLogs.get(Pair.create(KEYSPACE1, cf)).keySet();
-        assert unfinishedCompactions.containsAll(generations);
+        assertTrue(unfinishedCompactions.containsAll(generations));
 
         SystemKeyspace.finishCompaction(taskId);
         compactionLogs = SystemKeyspace.getUnfinishedCompactions();
-        assert !compactionLogs.containsKey(Pair.create(KEYSPACE1, cf));
+        assertFalse(compactionLogs.containsKey(Pair.create(KEYSPACE1, cf)));
     }
 
     private void testDontPurgeAccidentaly(String k, String cfname) throws InterruptedException
@@ -474,7 +499,7 @@
         DecoratedKey key = Util.dk(k);
         Mutation rm = new Mutation(KEYSPACE1, key.getKey());
         rm.add(cfname, Util.cellname(ByteBufferUtil.bytes("sc"), ByteBufferUtil.bytes("c")), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
-        rm.apply();
+        rm.applyUnsafe();
 
         cfs.forceBlockingFlush();
 
@@ -486,7 +511,7 @@
         // Remove key
         rm = new Mutation(KEYSPACE1, key.getKey());
         rm.delete(cfname, 2);
-        rm.apply();
+        rm.applyUnsafe();
 
         ColumnFamily cf = cfs.getColumnFamily(filter);
         assertTrue("should be empty: " + cf, cf == null || !cf.hasColumns());
@@ -510,8 +535,8 @@
 
     private static Range<Token> rangeFor(int start, int end)
     {
-        return new Range<Token>(new BytesToken(String.format("%03d", start).getBytes()),
-                                new BytesToken(String.format("%03d", end).getBytes()));
+        return new Range<Token>(new ByteOrderedPartitioner.BytesToken(String.format("%03d", start).getBytes()),
+                                new ByteOrderedPartitioner.BytesToken(String.format("%03d", end).getBytes()));
     }
 
     private static Collection<Range<Token>> makeRanges(int ... keys)
@@ -527,15 +552,16 @@
         long timestamp = System.currentTimeMillis();
         DecoratedKey decoratedKey = Util.dk(String.format("%03d", key));
         Mutation rm = new Mutation(KEYSPACE1, decoratedKey.getKey());
-        rm.add("Standard1", Util.cellname("col"), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp, 1000);
-        rm.apply();
+        rm.add("CF_STANDARD1", Util.cellname("col"), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp, 1000);
+        rm.applyUnsafe();
     }
 
     @Test
+    @Ignore("making ranges based on the keys, not on the tokens")
     public void testNeedsCleanup()
     {
         Keyspace keyspace = Keyspace.open(KEYSPACE1);
-        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore("CF_STANDARD1");
         store.clearUnsafe();
 
         // disable compaction while flushing
diff --git a/test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java b/test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java
index 5afd575..3bc830b 100644
--- a/test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java
@@ -21,6 +21,7 @@
 import java.util.*;
 import java.util.concurrent.TimeUnit;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import com.google.common.collect.Iterables;
@@ -28,13 +29,14 @@
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.io.sstable.SSTableReader;
-import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.Pair;
 
 import static org.apache.cassandra.db.compaction.DateTieredCompactionStrategy.getBuckets;
@@ -46,8 +48,17 @@
 
 public class DateTieredCompactionStrategyTest extends SchemaLoader
 {
-    public static final String KEYSPACE1 = "Keyspace1";
+    public static final String KEYSPACE1 = "DateTieredCompactionStrategyTest";
     private static final String CF_STANDARD1 = "Standard1";
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1));
+    }
 
     @Test
     public void testOptionsValidation() throws ConfigurationException
@@ -311,10 +322,10 @@
         Thread.sleep(2000);
         AbstractCompactionTask t = dtcs.getNextBackgroundTask((int) (System.currentTimeMillis()/1000));
         assertNotNull(t);
-        assertEquals(1, Iterables.size(t.sstables));
-        SSTableReader sstable = t.sstables.iterator().next();
+        assertEquals(1, Iterables.size(t.transaction.originals()));
+        SSTableReader sstable = t.transaction.originals().iterator().next();
         assertEquals(sstable, expiredSSTable);
-        cfs.getDataTracker().unmarkCompacting(cfs.getSSTables());
+        t.transaction.abort();
         cfs.truncateBlocking();
     }
 
@@ -355,8 +366,8 @@
         for (SSTableReader sstable : cfs.getSSTables())
             dtcs.addSSTable(sstable);
         AbstractCompactionTask task = dtcs.getNextBackgroundTask(0);
-        assertEquals(20, task.sstables.size());
-        cfs.getDataTracker().unmarkCompacting(task.sstables);
+        assertEquals(20, task.transaction.originals().size());
+        task.transaction.abort();
         cfs.truncateBlocking();
     }
 }
diff --git a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
index 6ec4c7b..0047678 100644
--- a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
@@ -20,24 +20,38 @@
 import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 import java.util.Random;
 import java.util.UUID;
 import java.util.concurrent.ExecutionException;
 
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import junit.framework.Assert;
 import org.junit.After;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
-import org.apache.cassandra.db.*;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.ISSTableScanner;
-import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.notifications.SSTableAddedNotification;
 import org.apache.cassandra.notifications.SSTableRepairStatusChanged;
 import org.apache.cassandra.repair.RepairJobDesc;
@@ -50,16 +64,34 @@
 import static org.junit.Assert.assertTrue;
 
 @RunWith(OrderedJUnit4ClassRunner.class)
-public class LeveledCompactionStrategyTest extends SchemaLoader
+public class LeveledCompactionStrategyTest
 {
-    private String ksname = "Keyspace1";
-    private String cfname = "StandardLeveled";
-    private Keyspace keyspace = Keyspace.open(ksname);
-    private ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);
+    private static final Logger logger = LoggerFactory.getLogger(LeveledCompactionStrategyTest.class);
+
+    private static final String KEYSPACE1 = "LeveledCompactionStrategyTest";
+    private static final String CF_STANDARDDLEVELED = "StandardLeveled";
+    private Keyspace keyspace;
+    private ColumnFamilyStore cfs;
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        Map<String, String> leveledOptions = new HashMap<>();
+        leveledOptions.put("sstable_size_in_mb", "1");
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARDDLEVELED)
+                                                .compactionStrategyClass(LeveledCompactionStrategy.class)
+                                                .compactionStrategyOptions(leveledOptions));
+        }
 
     @Before
     public void enableCompaction()
     {
+        keyspace = Keyspace.open(KEYSPACE1);
+        cfs = keyspace.getColumnFamilyStore(CF_STANDARDDLEVELED);
         cfs.enableAutoCompaction();
     }
 
@@ -72,6 +104,64 @@
         cfs.truncateBlocking();
     }
 
+    /**
+     * Ensure that the grouping operation preserves the levels of grouped tables
+     */
+    @Test
+    public void testGrouperLevels() throws Exception{
+        ByteBuffer value = ByteBuffer.wrap(new byte[100 * 1024]); // 100 KB value, make it easy to have multiple files
+
+        //Need entropy to prevent compression so size is predictable with compression enabled/disabled
+        new Random().nextBytes(value.array());
+
+        // Enough data to have a level 1 and 2
+        int rows = 40;
+        int columns = 20;
+
+        // Adds enough data to trigger multiple sstable per level
+        for (int r = 0; r < rows; r++)
+        {
+            DecoratedKey key = Util.dk(String.valueOf(r));
+            Mutation rm = new Mutation(KEYSPACE1, key.getKey());
+            for (int c = 0; c < columns; c++)
+            {
+                rm.add(CF_STANDARDDLEVELED, Util.cellname("column" + c), value, 0);
+            }
+            rm.apply();
+            cfs.forceBlockingFlush();
+        }
+
+        waitForLeveling(cfs);
+        WrappingCompactionStrategy strategy = (WrappingCompactionStrategy) cfs.getCompactionStrategy();
+        // Checking we're not completely bad at math
+        int l1Count = strategy.getSSTableCountPerLevel()[1];
+        int l2Count = strategy.getSSTableCountPerLevel()[2];
+        if (l1Count == 0 || l2Count == 0)
+        {
+            logger.error("L1 or L2 has 0 sstables. Expected > 0 on both.");
+            logger.error("L1: " + l1Count);
+            logger.error("L2: " + l2Count);
+            Assert.fail();
+        }
+
+        Collection<Collection<SSTableReader>> groupedSSTables = cfs.getCompactionStrategy().groupSSTablesForAntiCompaction(cfs.getSSTables());
+        for (Collection<SSTableReader> sstableGroup : groupedSSTables)
+        {
+            int groupLevel = -1;
+            Iterator<SSTableReader> it = sstableGroup.iterator();
+            while (it.hasNext())
+            {
+
+                SSTableReader sstable = it.next();
+                int tableLevel = sstable.getSSTableLevel();
+                if (groupLevel == -1)
+                    groupLevel = tableLevel;
+                assert groupLevel == tableLevel;
+            }
+        }
+
+    }
+
     /*
      * This exercises in particular the code of #4142
      */
@@ -83,19 +173,19 @@
         ByteBuffer value = ByteBuffer.wrap(b); // 100 KB value, make it easy to have multiple files
 
         // Enough data to have a level 1 and 2
-        int rows = 20;
-        int columns = 10;
+        int rows = 40;
+        int columns = 20;
 
         // Adds enough data to trigger multiple sstable per level
         for (int r = 0; r < rows; r++)
         {
             DecoratedKey key = Util.dk(String.valueOf(r));
-            Mutation rm = new Mutation(ksname, key.getKey());
+            Mutation rm = new Mutation(KEYSPACE1, key.getKey());
             for (int c = 0; c < columns; c++)
             {
-                rm.add(cfname, Util.cellname("column" + c), value, 0);
+                rm.add(CF_STANDARDDLEVELED, Util.cellname("column" + c), value, 0);
             }
-            rm.apply();
+            rm.applyUnsafe();
             cfs.forceBlockingFlush();
         }
 
@@ -106,10 +196,10 @@
         assertTrue(strategy.getSSTableCountPerLevel()[2] > 0);
 
         Range<Token> range = new Range<>(Util.token(""), Util.token(""));
-        int gcBefore = keyspace.getColumnFamilyStore(cfname).gcBefore(System.currentTimeMillis());
+        int gcBefore = keyspace.getColumnFamilyStore(CF_STANDARDDLEVELED).gcBefore(System.currentTimeMillis());
         UUID parentRepSession = UUID.randomUUID();
-        ActiveRepairService.instance.registerParentRepairSession(parentRepSession, FBUtilities.getBroadcastAddress(), Arrays.asList(cfs), Arrays.asList(range));
-        RepairJobDesc desc = new RepairJobDesc(parentRepSession, UUID.randomUUID(), ksname, cfname, range);
+        ActiveRepairService.instance.registerParentRepairSession(parentRepSession, FBUtilities.getBroadcastAddress(), Arrays.asList(cfs), Arrays.asList(range), false, true);
+        RepairJobDesc desc = new RepairJobDesc(parentRepSession, UUID.randomUUID(), KEYSPACE1, CF_STANDARDDLEVELED, range);
         Validator validator = new Validator(desc, FBUtilities.getBroadcastAddress(), gcBefore);
         CompactionManager.instance.submitValidation(cfs, validator).get();
     }
@@ -117,7 +207,7 @@
     /**
      * wait for leveled compaction to quiesce on the given columnfamily
      */
-    private void waitForLeveling(ColumnFamilyStore cfs) throws InterruptedException
+    public static void waitForLeveling(ColumnFamilyStore cfs) throws InterruptedException
     {
         WrappingCompactionStrategy strategyManager = (WrappingCompactionStrategy)cfs.getCompactionStrategy();
         while (true)
@@ -156,12 +246,12 @@
         for (int r = 0; r < rows; r++)
         {
             DecoratedKey key = Util.dk(String.valueOf(r));
-            Mutation rm = new Mutation(ksname, key.getKey());
+            Mutation rm = new Mutation(KEYSPACE1, key.getKey());
             for (int c = 0; c < columns; c++)
             {
-                rm.add(cfname, Util.cellname("column" + c), value, 0);
+                rm.add(CF_STANDARDDLEVELED, Util.cellname("column" + c), value, 0);
             }
-            rm.apply();
+            rm.applyUnsafe();
             cfs.forceBlockingFlush();
         }
 
@@ -185,35 +275,32 @@
     @Test
     public void testMutateLevel() throws Exception
     {
+        cfs.disableAutoCompaction();
         ByteBuffer value = ByteBuffer.wrap(new byte[100 * 1024]); // 100 KB value, make it easy to have multiple files
 
         // Enough data to have a level 1 and 2
-        int rows = 20;
-        int columns = 10;
+        int rows = 40;
+        int columns = 20;
 
         // Adds enough data to trigger multiple sstable per level
         for (int r = 0; r < rows; r++)
         {
             DecoratedKey key = Util.dk(String.valueOf(r));
-            Mutation rm = new Mutation(ksname, key.getKey());
+            Mutation rm = new Mutation(KEYSPACE1, key.getKey());
             for (int c = 0; c < columns; c++)
             {
-                rm.add(cfname, Util.cellname("column" + c), value, 0);
+                rm.add(CF_STANDARDDLEVELED, Util.cellname("column" + c), value, 0);
             }
-            rm.apply();
+            rm.applyUnsafe();
             cfs.forceBlockingFlush();
         }
-        waitForLeveling(cfs);
         cfs.forceBlockingFlush();
         LeveledCompactionStrategy strategy = (LeveledCompactionStrategy) ((WrappingCompactionStrategy) cfs.getCompactionStrategy()).getWrappedStrategies().get(1);
-        cfs.disableAutoCompaction();
-
-        while(CompactionManager.instance.isCompacting(Arrays.asList(cfs)))
-            Thread.sleep(100);
+        cfs.forceMajorCompaction();
 
         for (SSTableReader s : cfs.getSSTables())
         {
-            assertTrue(s.getSSTableLevel() != 6);
+            assertTrue(s.getSSTableLevel() != 6 && s.getSSTableLevel() > 0);
             strategy.manifest.remove(s);
             s.descriptor.getMetadataSerializer().mutateLevel(s.descriptor, 6);
             s.reloadSSTableMetadata();
@@ -236,19 +323,19 @@
         ByteBuffer value = ByteBuffer.wrap(b); // 100 KB value, make it easy to have multiple files
 
         // Enough data to have a level 1 and 2
-        int rows = 20;
-        int columns = 10;
+        int rows = 40;
+        int columns = 20;
 
         // Adds enough data to trigger multiple sstable per level
         for (int r = 0; r < rows; r++)
         {
             DecoratedKey key = Util.dk(String.valueOf(r));
-            Mutation rm = new Mutation(ksname, key.getKey());
+            Mutation rm = new Mutation(KEYSPACE1, key.getKey());
             for (int c = 0; c < columns; c++)
             {
-                rm.add(cfname, Util.cellname("column" + c), value, 0);
+                rm.add(CF_STANDARDDLEVELED, Util.cellname("column" + c), value, 0);
             }
-            rm.apply();
+            rm.applyUnsafe();
             cfs.forceBlockingFlush();
         }
         waitForLeveling(cfs);
@@ -314,10 +401,10 @@
         for (int r = 0; r < rows; r++)
         {
             DecoratedKey key = Util.dk(String.valueOf(r));
-            Mutation rm = new Mutation(ksname, key.getKey());
+            Mutation rm = new Mutation(KEYSPACE1, key.getKey());
             for (int c = 0; c < columns; c++)
             {
-                rm.add(cfname, Util.cellname("column" + c), value, 0);
+                rm.add(CF_STANDARDDLEVELED, Util.cellname("column" + c), value, 0);
             }
             rm.apply();
             cfs.forceBlockingFlush();
diff --git a/test/unit/org/apache/cassandra/db/compaction/NeverPurgeTest.java b/test/unit/org/apache/cassandra/db/compaction/NeverPurgeTest.java
index 41db92d..362afe7 100644
--- a/test/unit/org/apache/cassandra/db/compaction/NeverPurgeTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/NeverPurgeTest.java
@@ -30,7 +30,7 @@
 import org.apache.cassandra.db.RangeTombstone;
 import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
 import org.apache.cassandra.io.sstable.ISSTableScanner;
-import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
@@ -80,7 +80,7 @@
         execute("DELETE FROM %s WHERE a=3");
         cfs.forceBlockingFlush();
         cfs.enableAutoCompaction();
-        while (cfs.getSSTables().size() > 1)
+        while (cfs.getSSTables().size() > 1 || !cfs.getTracker().getCompacting().isEmpty())
             Thread.sleep(100);
         verifyContainsTombstones(cfs.getSSTables(), 3);
     }
diff --git a/test/unit/org/apache/cassandra/db/compaction/OneCompactionTest.java b/test/unit/org/apache/cassandra/db/compaction/OneCompactionTest.java
index bc3fe35..ec5c280 100644
--- a/test/unit/org/apache/cassandra/db/compaction/OneCompactionTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/OneCompactionTest.java
@@ -18,12 +18,15 @@
 */
 package org.apache.cassandra.db.compaction;
 
+import java.util.HashMap;
+import java.util.Map;
 import java.util.concurrent.ExecutionException;
 import java.util.Set;
 import java.util.HashSet;
 
 import org.apache.cassandra.Util;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import static org.junit.Assert.assertEquals;
@@ -31,40 +34,60 @@
 import org.apache.cassandra.db.*;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 
-public class OneCompactionTest extends SchemaLoader
+public class OneCompactionTest
 {
-    private void testCompaction(String columnFamilyName, int insertsPerTable) throws ExecutionException, InterruptedException
+    public static final String KEYSPACE1 = "OneCompactionTest";
+    private static final String CF_STANDARD1 = "Standard1";
+    private static final String CF_STANDARD2 = "Standard2";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        Map<String, String> leveledOptions = new HashMap<>();
+        leveledOptions.put("sstable_size_in_mb", "1");
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1).compactionStrategyOptions(leveledOptions),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD2));
+    }
+
+    private void testCompaction(String columnFamilyName, int insertsPerTable)
     {
         CompactionManager.instance.disableAutoCompaction();
 
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore(columnFamilyName);
 
         Set<DecoratedKey> inserted = new HashSet<DecoratedKey>();
         for (int j = 0; j < insertsPerTable; j++) {
             DecoratedKey key = Util.dk(String.valueOf(j));
-            Mutation rm = new Mutation("Keyspace1", key.getKey());
+            Mutation rm = new Mutation(KEYSPACE1, key.getKey());
             rm.add(columnFamilyName, Util.cellname("0"), ByteBufferUtil.EMPTY_BYTE_BUFFER, j);
-            rm.apply();
+            rm.applyUnsafe();
             inserted.add(key);
             store.forceBlockingFlush();
             assertEquals(inserted.size(), Util.getRangeSlice(store).size());
         }
-        CompactionManager.instance.performMaximal(store);
+        CompactionManager.instance.performMaximal(store, false);
         assertEquals(1, store.getSSTables().size());
     }
 
     @Test
-    public void testCompaction1() throws ExecutionException, InterruptedException
+    public void testCompaction1()
     {
         testCompaction("Standard1", 1);
     }
 
     @Test
-    public void testCompaction2() throws ExecutionException, InterruptedException
+    public void testCompaction2()
     {
         testCompaction("Standard2", 2);
     }
diff --git a/test/unit/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyTest.java b/test/unit/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyTest.java
index 1591f03..46d54d9 100644
--- a/test/unit/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyTest.java
@@ -20,13 +20,16 @@
 import java.nio.ByteBuffer;
 import java.util.*;
 
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.metrics.RestorableMeter;
 import org.apache.cassandra.utils.Pair;
 
@@ -37,8 +40,23 @@
 
 import static org.junit.Assert.*;
 
-public class SizeTieredCompactionStrategyTest extends SchemaLoader
+public class SizeTieredCompactionStrategyTest
 {
+    public static final String KEYSPACE1 = "SizeTieredCompactionStrategyTest";
+    private static final String CF_STANDARD1 = "Standard1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        Map<String, String> leveledOptions = new HashMap<>();
+        leveledOptions.put("sstable_size_in_mb", "1");
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1)
+                                                .compactionStrategyOptions(leveledOptions));
+    }
 
     @Test
     public void testOptionsValidation() throws ConfigurationException
@@ -125,7 +143,7 @@
     @Test
     public void testPrepBucket() throws Exception
     {
-        String ksname = "Keyspace1";
+        String ksname = KEYSPACE1;
         String cfname = "Standard1";
         Keyspace keyspace = Keyspace.open(ksname);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);
@@ -141,7 +159,7 @@
             DecoratedKey key = Util.dk(String.valueOf(r));
             Mutation rm = new Mutation(ksname, key.getKey());
             rm.add(cfname, Util.cellname("column"), value, 0);
-            rm.apply();
+            rm.applyUnsafe();
             cfs.forceBlockingFlush();
         }
         cfs.forceBlockingFlush();
diff --git a/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java b/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java
index a5b376e..bd1e559 100644
--- a/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java
@@ -20,6 +20,7 @@
  * 
  */
 
+import org.junit.BeforeClass;
 import com.google.common.collect.Multimap;
 import com.google.common.collect.Sets;
 import org.junit.Test;
@@ -28,13 +29,17 @@
 import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.ISSTableScanner;
-import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.tools.SSTableExpiredBlockers;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
+import java.io.IOException;
 import java.util.Collections;
 import java.util.Set;
 
@@ -42,35 +47,48 @@
 import static org.junit.Assert.assertTrue;
 
 @RunWith(OrderedJUnit4ClassRunner.class)
-public class TTLExpiryTest extends SchemaLoader
+public class TTLExpiryTest
 {
+    public static final String KEYSPACE1 = "TTLExpiryTest";
+    private static final String CF_STANDARD1 = "Standard1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1));
+    }
+
     @Test
     public void testAggressiveFullyExpired()
     {
-        ColumnFamilyStore cfs = Keyspace.open("Keyspace1").getColumnFamilyStore("Standard1");
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore("Standard1");
         cfs.disableAutoCompaction();
         cfs.metadata.gcGraceSeconds(0);
 
         DecoratedKey ttlKey = Util.dk("ttl");
-        Mutation rm = new Mutation("Keyspace1", ttlKey.getKey());
+        Mutation rm = new Mutation(KEYSPACE1, ttlKey.getKey());
         rm.add("Standard1", Util.cellname("col1"), ByteBufferUtil.EMPTY_BYTE_BUFFER, 1, 1);
         rm.add("Standard1", Util.cellname("col2"), ByteBufferUtil.EMPTY_BYTE_BUFFER, 3, 1);
         rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
-        rm = new Mutation("Keyspace1", ttlKey.getKey());
+        rm = new Mutation(KEYSPACE1, ttlKey.getKey());
         rm.add("Standard1", Util.cellname("col1"), ByteBufferUtil.EMPTY_BYTE_BUFFER, 2, 1);
         rm.add("Standard1", Util.cellname("col2"), ByteBufferUtil.EMPTY_BYTE_BUFFER, 5, 1);
         rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
-        rm = new Mutation("Keyspace1", ttlKey.getKey());
+        rm = new Mutation(KEYSPACE1, ttlKey.getKey());
         rm.add("Standard1", Util.cellname("col1"), ByteBufferUtil.EMPTY_BYTE_BUFFER, 4, 1);
         rm.add("Standard1", Util.cellname("shadow"), ByteBufferUtil.EMPTY_BYTE_BUFFER, 7, 1);
         rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
-        rm = new Mutation("Keyspace1", ttlKey.getKey());
+        rm = new Mutation(KEYSPACE1, ttlKey.getKey());
         rm.add("Standard1", Util.cellname("shadow"), ByteBufferUtil.EMPTY_BYTE_BUFFER, 6, 3);
         rm.add("Standard1", Util.cellname("col2"), ByteBufferUtil.EMPTY_BYTE_BUFFER, 8, 1);
         rm.applyUnsafe();
@@ -92,11 +110,11 @@
     @Test
     public void testSimpleExpire() throws InterruptedException
     {
-        ColumnFamilyStore cfs = Keyspace.open("Keyspace1").getColumnFamilyStore("Standard1");
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore("Standard1");
         cfs.disableAutoCompaction();
         cfs.metadata.gcGraceSeconds(0);
         long timestamp = System.currentTimeMillis();
-        Mutation rm = new Mutation("Keyspace1", Util.dk("ttl").getKey());
+        Mutation rm = new Mutation(KEYSPACE1, Util.dk("ttl").getKey());
         rm.add("Standard1", Util.cellname("col"),
                ByteBufferUtil.EMPTY_BYTE_BUFFER,
                timestamp,
@@ -106,29 +124,29 @@
                timestamp,
                1);
 
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
-        rm = new Mutation("Keyspace1", Util.dk("ttl").getKey());
+        rm = new Mutation(KEYSPACE1, Util.dk("ttl").getKey());
                 rm.add("Standard1", Util.cellname("col2"),
                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
                        timestamp,
                        1);
-                rm.apply();
+                rm.applyUnsafe();
         cfs.forceBlockingFlush();
-        rm = new Mutation("Keyspace1", Util.dk("ttl").getKey());
+        rm = new Mutation(KEYSPACE1, Util.dk("ttl").getKey());
         rm.add("Standard1", Util.cellname("col3"),
                    ByteBufferUtil.EMPTY_BYTE_BUFFER,
                    timestamp,
                    1);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
-        rm = new Mutation("Keyspace1", Util.dk("ttl").getKey());
+        rm = new Mutation(KEYSPACE1, Util.dk("ttl").getKey());
         rm.add("Standard1", Util.cellname("col311"),
                    ByteBufferUtil.EMPTY_BYTE_BUFFER,
                    timestamp,
                    1);
-        rm.apply();
+        rm.applyUnsafe();
 
         cfs.forceBlockingFlush();
         Thread.sleep(2000); // wait for ttl to expire
@@ -138,13 +156,13 @@
     }
 
     @Test
-    public void testNoExpire() throws InterruptedException
+    public void testNoExpire() throws InterruptedException, IOException
     {
-        ColumnFamilyStore cfs = Keyspace.open("Keyspace1").getColumnFamilyStore("Standard1");
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore("Standard1");
         cfs.disableAutoCompaction();
         cfs.metadata.gcGraceSeconds(0);
         long timestamp = System.currentTimeMillis();
-        Mutation rm = new Mutation("Keyspace1", Util.dk("ttl").getKey());
+        Mutation rm = new Mutation(KEYSPACE1, Util.dk("ttl").getKey());
         rm.add("Standard1", Util.cellname("col"),
                ByteBufferUtil.EMPTY_BYTE_BUFFER,
                timestamp,
@@ -154,29 +172,29 @@
                timestamp,
                1);
 
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
 
-        rm = new Mutation("Keyspace1", Util.dk("ttl").getKey());
+        rm = new Mutation(KEYSPACE1, Util.dk("ttl").getKey());
                 rm.add("Standard1", Util.cellname("col2"),
                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
                        timestamp,
                        1);
-                rm.apply();
+                rm.applyUnsafe();
         cfs.forceBlockingFlush();
-        rm = new Mutation("Keyspace1", Util.dk("ttl").getKey());
+        rm = new Mutation(KEYSPACE1, Util.dk("ttl").getKey());
         rm.add("Standard1", Util.cellname("col3"),
                    ByteBufferUtil.EMPTY_BYTE_BUFFER,
                    timestamp,
                    1);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
         DecoratedKey noTTLKey = Util.dk("nottl");
-        rm = new Mutation("Keyspace1", noTTLKey.getKey());
+        rm = new Mutation(KEYSPACE1, noTTLKey.getKey());
         rm.add("Standard1", Util.cellname("col311"),
                    ByteBufferUtil.EMPTY_BYTE_BUFFER,
                    timestamp);
-        rm.apply();
+        rm.applyUnsafe();
         cfs.forceBlockingFlush();
         Thread.sleep(2000); // wait for ttl to expire
         assertEquals(4, cfs.getSSTables().size());
@@ -190,13 +208,14 @@
             OnDiskAtomIterator iter = scanner.next();
             assertEquals(noTTLKey, iter.getKey());
         }
+
+        scanner.close();
     }
 
     @Test
     public void testCheckForExpiredSSTableBlockers() throws InterruptedException
     {
-        String KEYSPACE1 = "Keyspace1";
-        ColumnFamilyStore cfs = Keyspace.open("Keyspace1").getColumnFamilyStore("Standard1");
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore("Standard1");
         cfs.truncateBlocking();
         cfs.disableAutoCompaction();
         cfs.metadata.gcGraceSeconds(0);
diff --git a/test/unit/org/apache/cassandra/db/context/CounterContextTest.java b/test/unit/org/apache/cassandra/db/context/CounterContextTest.java
index a72d30d..4f587c6 100644
--- a/test/unit/org/apache/cassandra/db/context/CounterContextTest.java
+++ b/test/unit/org/apache/cassandra/db/context/CounterContextTest.java
@@ -22,9 +22,11 @@
 
 import java.nio.ByteBuffer;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ClockAndCount;
 import org.apache.cassandra.db.context.CounterContext.Relationship;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -48,6 +50,12 @@
     private static final int countLength = 8;
     private static final int stepLength = idLength + clockLength + countLength;
 
+    @BeforeClass
+    public static void setupDD()
+    {
+        DatabaseDescriptor.setDaemonInitialized();
+    }
+
     @Test
     public void testAllocate()
     {
diff --git a/test/unit/org/apache/cassandra/db/index/PerRowSecondaryIndexTest.java b/test/unit/org/apache/cassandra/db/index/PerRowSecondaryIndexTest.java
index a332342..19303cf 100644
--- a/test/unit/org/apache/cassandra/db/index/PerRowSecondaryIndexTest.java
+++ b/test/unit/org/apache/cassandra/db/index/PerRowSecondaryIndexTest.java
@@ -26,6 +26,9 @@
 import java.util.List;
 import java.util.Set;
 
+import org.apache.cassandra.service.EmbeddedCassandraService;
+import org.apache.cassandra.thrift.*;
+import org.apache.thrift.TException;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -49,18 +52,9 @@
 import org.apache.cassandra.db.filter.QueryFilter;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.service.EmbeddedCassandraService;
-import org.apache.cassandra.service.MigrationManager;
-import org.apache.cassandra.thrift.CassandraServer;
-import org.apache.cassandra.thrift.Column;
-import org.apache.cassandra.thrift.ColumnOrSuperColumn;
-import org.apache.cassandra.thrift.ColumnParent;
-import org.apache.cassandra.thrift.ColumnPath;
-import org.apache.cassandra.thrift.ConsistencyLevel;
-import org.apache.cassandra.thrift.ThriftSessionManager;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.concurrent.OpOrder;
-import org.apache.thrift.TException;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -77,25 +71,22 @@
     // indexed & stashes it in a static variable for inspection
     // in the test.
 
-    private static final String KEYSPACE1 = "PerRowSecondaryIndex";
+    private static final String KEYSPACE1 = "PerRowSecondaryIndexTest";
     private static final String CF_INDEXED = "Indexed1";
     private static final String INDEXED_COLUMN = "indexed";
 
     private static CassandraServer server;
 
     @BeforeClass
-    public static void defineSchema() throws Exception
+    public static void defineSchema() throws ConfigurationException, IOException, TException
     {
         SchemaLoader.prepareServer();
-        SchemaLoader.startGossiper();
         new EmbeddedCassandraService().start();
         ThriftSessionManager.instance.setCurrentSocket(new InetSocketAddress(9160));
-        for (KSMetaData ksm : SchemaLoader.schemaDefinition(null))
-                MigrationManager.announceNewKeyspace(ksm);
-//        SchemaLoader.createKeyspace(KEYSPACE1,
-//                                    SimpleStrategy.class,
-//                                    KSMetaData.optsWithRF(1),
-//                                    SchemaLoader.perRowIndexedCFMD(KEYSPACE1, CF_INDEXED));
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                SimpleStrategy.class,
+                KSMetaData.optsWithRF(1),
+                SchemaLoader.perRowIndexedCFMD(KEYSPACE1, CF_INDEXED));
         server = new CassandraServer();
         server.set_keyspace(KEYSPACE1);
     }
@@ -111,18 +102,18 @@
     {
         // create a row then test that the configured index instance was able to read the row
         Mutation rm;
-        rm = new Mutation("PerRowSecondaryIndex", ByteBufferUtil.bytes("k1"));
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("k1"));
         rm.add("Indexed1", Util.cellname("indexed"), ByteBufferUtil.bytes("foo"), 1);
-        rm.apply();
+        rm.applyUnsafe();
 
         ColumnFamily indexedRow = PerRowSecondaryIndexTest.TestIndex.LAST_INDEXED_ROW;
         assertNotNull(indexedRow);
         assertEquals(ByteBufferUtil.bytes("foo"), indexedRow.getColumn(Util.cellname("indexed")).value());
 
         // update the row and verify what was indexed
-        rm = new Mutation("PerRowSecondaryIndex", ByteBufferUtil.bytes("k1"));
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("k1"));
         rm.add("Indexed1", Util.cellname("indexed"), ByteBufferUtil.bytes("bar"), 2);
-        rm.apply();
+        rm.applyUnsafe();
 
         indexedRow = PerRowSecondaryIndexTest.TestIndex.LAST_INDEXED_ROW;
         assertNotNull(indexedRow);
@@ -135,9 +126,9 @@
     {
         // issue a column delete and test that the configured index instance was notified to update
         Mutation rm;
-        rm = new Mutation("PerRowSecondaryIndex", ByteBufferUtil.bytes("k2"));
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("k2"));
         rm.delete("Indexed1", Util.cellname("indexed"), 1);
-        rm.apply();
+        rm.applyUnsafe();
 
         ColumnFamily indexedRow = PerRowSecondaryIndexTest.TestIndex.LAST_INDEXED_ROW;
         assertNotNull(indexedRow);
@@ -153,9 +144,9 @@
     {
         // issue a row level delete and test that the configured index instance was notified to update
         Mutation rm;
-        rm = new Mutation("PerRowSecondaryIndex", ByteBufferUtil.bytes("k3"));
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("k3"));
         rm.delete("Indexed1", 1);
-        rm.apply();
+        rm.applyUnsafe();
 
         ColumnFamily indexedRow = PerRowSecondaryIndexTest.TestIndex.LAST_INDEXED_ROW;
         assertNotNull(indexedRow);
@@ -164,23 +155,23 @@
 
         assertTrue(Arrays.equals("k3".getBytes(), PerRowSecondaryIndexTest.TestIndex.LAST_INDEXED_KEY.array()));
     }
-    
+
     @Test
-    public void testInvalidSearch() throws IOException
+    public void testInvalidSearch()
     {
         Mutation rm;
-        rm = new Mutation("PerRowSecondaryIndex", ByteBufferUtil.bytes("k4"));
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("k4"));
         rm.add("Indexed1", Util.cellname("indexed"), ByteBufferUtil.bytes("foo"), 1);
         rm.apply();
-        
+
         // test we can search:
-        UntypedResultSet result = QueryProcessor.executeInternal("SELECT * FROM \"PerRowSecondaryIndex\".\"Indexed1\" WHERE indexed = 'foo'");
+        UntypedResultSet result = QueryProcessor.executeInternal(String.format("SELECT * FROM \"%s\".\"Indexed1\" WHERE indexed = 'foo'", KEYSPACE1));
         assertEquals(1, result.size());
 
         // test we can't search if the searcher doesn't validate the expression:
         try
         {
-            QueryProcessor.executeInternal("SELECT * FROM \"PerRowSecondaryIndex\".\"Indexed1\" WHERE indexed = 'invalid'");
+            QueryProcessor.executeInternal(String.format("SELECT * FROM \"%s\".\"Indexed1\" WHERE indexed = 'invalid'", KEYSPACE1));
             fail("Query should have been invalid!");
         }
         catch (Exception e)
@@ -203,7 +194,7 @@
         }
         catch (Exception e)
         {
-            assertTrue(e.getCause() instanceof InvalidRequestException);
+            assertTrue(e instanceof InvalidRequestException);
         }
 
         // test we can't insert if the index doesn't validate the columns:
@@ -214,7 +205,7 @@
         }
         catch (Exception e)
         {
-            assertTrue(e.getCause() instanceof InvalidRequestException);
+            assertTrue(e instanceof InvalidRequestException);
         }
     }
 
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/HelpersTest.java b/test/unit/org/apache/cassandra/db/lifecycle/HelpersTest.java
new file mode 100644
index 0000000..18bce10
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/lifecycle/HelpersTest.java
@@ -0,0 +1,168 @@
+/*
+* 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.db.lifecycle;
+
+import java.util.Map;
+import java.util.Set;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import junit.framework.Assert;
+import org.apache.cassandra.MockSchema;
+import org.apache.cassandra.Util;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.big.BigTableReader;
+import org.apache.cassandra.utils.concurrent.Refs;
+
+public class HelpersTest
+{
+
+    @BeforeClass
+    public static void setUp()
+    {
+        MockSchema.cleanup();
+    }
+
+    static Set<Integer> a = set(1, 2, 3);
+    static Set<Integer> b = set(4, 5, 6);
+    static Set<Integer> c = set(7, 8, 9);
+    static Set<Integer> abc = set(1, 2, 3, 4, 5, 6, 7, 8, 9);
+
+    // this also tests orIn
+    @Test
+    public void testFilterIn()
+    {
+        check(Helpers.filterIn(abc, a), a);
+        check(Helpers.filterIn(abc, a, c), set(1, 2, 3, 7, 8, 9));
+        check(Helpers.filterIn(a, c), set());
+    }
+
+    // this also tests notIn
+    @Test
+    public void testFilterOut()
+    {
+        check(Helpers.filterOut(abc, a), set(4, 5, 6, 7, 8, 9));
+        check(Helpers.filterOut(abc, b), set(1, 2, 3, 7, 8, 9));
+        check(Helpers.filterOut(a, a), set());
+    }
+
+    @Test
+    public void testConcatUniq()
+    {
+        check(Helpers.concatUniq(a, b, a, c, b, a), abc);
+    }
+
+    @Test
+    public void testIdentityMap()
+    {
+        Integer one = new Integer(1);
+        Integer two = new Integer(2);
+        Integer three = new Integer(3);
+        Map<Integer, Integer> identity = Helpers.identityMap(set(one, two, three));
+        Assert.assertEquals(3, identity.size());
+        Assert.assertSame(one, identity.get(1));
+        Assert.assertSame(two, identity.get(2));
+        Assert.assertSame(three, identity.get(3));
+    }
+
+    @Test
+    public void testReplace()
+    {
+        boolean failure;
+        failure = false;
+        try
+        {
+            Helpers.replace(abc, a, c);
+        }
+        catch (AssertionError e)
+        {
+            failure = true;
+        }
+        Assert.assertTrue(failure);
+
+        failure = false;
+        try
+        {
+            Helpers.replace(a, abc, c);
+        }
+        catch (AssertionError e)
+        {
+            failure = true;
+        }
+        Assert.assertTrue(failure);
+
+        failure = false;
+        try
+        {
+            Map<Integer, Integer> notIdentity = ImmutableMap.of(1, new Integer(1), 2, 2, 3, 3);
+            Helpers.replace(notIdentity, a, b);
+        }
+        catch (AssertionError e)
+        {
+            failure = true;
+        }
+        Assert.assertTrue(failure);
+
+        // check it actually works when correct values provided
+        check(Helpers.replace(a, a, b), b);
+    }
+
+    private static Set<Integer> set(Integer ... contents)
+    {
+        return ImmutableSet.copyOf(contents);
+    }
+
+    private static void check(Iterable<Integer> check, Set<Integer> expected)
+    {
+        Assert.assertEquals(expected, ImmutableSet.copyOf(check));
+    }
+
+    @Test
+    public void testSetupDeletionNotification()
+    {
+        ColumnFamilyStore cfs = MockSchema.newCFS();
+        Iterable<SSTableReader> readers = Lists.newArrayList(MockSchema.sstable(1, cfs), MockSchema.sstable(2, cfs));
+        Throwable accumulate = Helpers.setReplaced(readers, null);
+        Assert.assertNull(accumulate);
+        for (SSTableReader reader : readers)
+            Assert.assertTrue(reader.isReplaced());
+        accumulate = Helpers.setReplaced(readers, null);
+        Assert.assertNotNull(accumulate);
+    }
+
+    @Test
+    public void testMarkObsolete()
+    {
+        ColumnFamilyStore cfs = MockSchema.newCFS();
+        Iterable<SSTableReader> readers = Lists.newArrayList(MockSchema.sstable(1, cfs), MockSchema.sstable(2, cfs));
+        Throwable accumulate = Helpers.markObsolete(null, readers, null);
+        Assert.assertNull(accumulate);
+        for (SSTableReader reader : readers)
+            Assert.assertTrue(reader.isMarkedCompacted());
+        accumulate = Helpers.markObsolete(null, readers, null);
+        Assert.assertNotNull(accumulate);
+    }
+}
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java b/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
new file mode 100644
index 0000000..737392e
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
@@ -0,0 +1,433 @@
+/*
+* 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.db.lifecycle;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import junit.framework.Assert;
+import org.apache.cassandra.MockSchema;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Memtable;
+import org.apache.cassandra.db.commitlog.ReplayPosition;
+import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction.ReaderState;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction.ReaderState.Action;
+import org.apache.cassandra.io.sstable.SSTableDeletingTask;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.metrics.ColumnFamilyMetrics;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.concurrent.AbstractTransactionalTest;
+import org.apache.cassandra.utils.concurrent.Transactional.AbstractTransactional.State;
+
+import static com.google.common.base.Predicates.in;
+import static com.google.common.collect.ImmutableList.copyOf;
+import static com.google.common.collect.ImmutableList.of;
+import static com.google.common.collect.Iterables.all;
+import static com.google.common.collect.Iterables.concat;
+import static com.google.common.collect.Iterables.size;
+import static org.apache.cassandra.db.lifecycle.Helpers.idIn;
+import static org.apache.cassandra.db.lifecycle.Helpers.orIn;
+import static org.apache.cassandra.db.lifecycle.Helpers.select;
+
+public class LifecycleTransactionTest extends AbstractTransactionalTest
+{
+    private boolean incrementalBackups;
+
+    @BeforeClass
+    public static void setUp()
+    {
+        MockSchema.cleanup();
+    }
+
+    @Before
+    public void disableIncrementalBackup()
+    {
+        incrementalBackups = DatabaseDescriptor.isIncrementalBackupsEnabled();
+        DatabaseDescriptor.setIncrementalBackupsEnabled(false);
+    }
+    @After
+    public void restoreIncrementalBackup()
+    {
+        DatabaseDescriptor.setIncrementalBackupsEnabled(incrementalBackups);
+    }
+
+    @Test
+    public void testUpdates() // (including obsoletion)
+    {
+        ColumnFamilyStore cfs = MockSchema.newCFS();
+        Tracker tracker = new Tracker(null, false);
+        SSTableReader[] readers = readersArray(0, 3, cfs);
+        SSTableReader[] readers2 = readersArray(0, 4, cfs);
+        SSTableReader[] readers3 = readersArray(0, 4, cfs);
+        tracker.addInitialSSTables(copyOf(readers));
+        LifecycleTransaction txn = tracker.tryModify(copyOf(readers), OperationType.UNKNOWN);
+
+        txn.update(readers2[0], true);
+        txn.obsolete(readers[1]);
+
+        Assert.assertTrue(txn.isObsolete(readers[1]));
+        Assert.assertFalse(txn.isObsolete(readers[0]));
+
+        testBadUpdate(txn, readers2[0], true);  // same reader && instances
+        testBadUpdate(txn, readers2[1], true);  // staged obsolete; cannot update
+        testBadUpdate(txn, readers3[0], true);  // same reader, diff instances
+        testBadUpdate(txn, readers2[2], false); // incorrectly declared original status
+        testBadUpdate(txn, readers2[3], true); // incorrectly declared original status
+
+        testBadObsolete(txn, readers[1]);  // staged obsolete; cannot obsolete again
+        testBadObsolete(txn, readers2[0]);  // staged update; cannot obsolete
+
+        txn.update(readers2[3], false);
+
+        Assert.assertEquals(3, tracker.getView().compacting.size());
+        txn.checkpoint();
+        Assert.assertTrue(txn.isObsolete(readers[1]));
+        Assert.assertFalse(txn.isObsolete(readers[0]));
+        Assert.assertEquals(4, tracker.getView().compacting.size());
+        Assert.assertEquals(3, tracker.getView().sstables.size());
+        Assert.assertEquals(3, size(txn.current()));
+        Assert.assertTrue(all(of(readers2[0], readers[2], readers2[3]), idIn(tracker.getView().sstablesMap)));
+        Assert.assertTrue(all(txn.current(), idIn(tracker.getView().sstablesMap)));
+
+        testBadObsolete(txn, readers[1]);  // logged obsolete; cannot obsolete again
+        testBadObsolete(txn, readers2[2]);  // never seen instance; cannot obsolete
+        testBadObsolete(txn, readers2[3]);  // non-original; cannot obsolete
+        testBadUpdate(txn, readers3[1], true);  // logged obsolete; cannot update
+        testBadUpdate(txn, readers2[0], true);  // same instance as logged update
+
+        txn.update(readers3[0], true);  // same reader as logged update, different instance
+        txn.checkpoint();
+
+        Assert.assertEquals(4, tracker.getView().compacting.size());
+        Assert.assertEquals(3, tracker.getView().sstables.size());
+        Assert.assertEquals(3, size(txn.current()));
+        Assert.assertTrue(all(of(readers3[0], readers[2], readers2[3]), idIn(tracker.getView().sstablesMap)));
+        Assert.assertTrue(all(txn.current(), idIn(tracker.getView().sstablesMap)));
+
+        testBadObsolete(txn, readers2[0]); // not current version of sstable
+
+        txn.obsoleteOriginals();
+        txn.checkpoint();
+        Assert.assertEquals(1, tracker.getView().sstables.size());
+        txn.obsoleteOriginals(); // should be no-op
+        txn.checkpoint();
+        Assert.assertEquals(1, tracker.getView().sstables.size());
+        Assert.assertEquals(4, tracker.getView().compacting.size());
+    }
+
+    @Test
+    public void testCancellation()
+    {
+        ColumnFamilyStore cfs = MockSchema.newCFS();
+        Tracker tracker = new Tracker(null, false);
+        List<SSTableReader> readers = readers(0, 3, cfs);
+        tracker.addInitialSSTables(readers);
+        LifecycleTransaction txn = tracker.tryModify(readers, OperationType.UNKNOWN);
+
+        SSTableReader cancel = readers.get(0);
+        SSTableReader update = readers(1, 2, cfs).get(0);
+        SSTableReader fresh = readers(3, 4,cfs).get(0);
+        SSTableReader notPresent = readers(4, 5, cfs).get(0);
+
+        txn.cancel(cancel);
+        txn.update(update, true);
+        txn.update(fresh, false);
+
+        testBadCancel(txn, cancel);
+        testBadCancel(txn, update);
+        testBadCancel(txn, fresh);
+        testBadCancel(txn, notPresent);
+        Assert.assertEquals(2, txn.originals().size());
+        Assert.assertEquals(2, tracker.getView().compacting.size());
+        Assert.assertTrue(all(readers.subList(1, 3), idIn(tracker.getView().compacting)));
+
+        txn.checkpoint();
+
+        testBadCancel(txn, cancel);
+        testBadCancel(txn, update);
+        testBadCancel(txn, fresh);
+        testBadCancel(txn, notPresent);
+        Assert.assertEquals(2, txn.originals().size());
+        Assert.assertEquals(3, tracker.getView().compacting.size());
+        Assert.assertEquals(3, size(txn.current()));
+        Assert.assertTrue(all(concat(readers.subList(1, 3), of(fresh)), idIn(tracker.getView().compacting)));
+
+        txn.cancel(readers.get(2));
+        Assert.assertEquals(1, txn.originals().size());
+        Assert.assertEquals(2, tracker.getView().compacting.size());
+        Assert.assertEquals(2, size(txn.current()));
+        Assert.assertTrue(all(of(readers.get(1), fresh), idIn(tracker.getView().compacting)));
+    }
+
+    @Test
+    public void testSplit()
+    {
+        ColumnFamilyStore cfs = MockSchema.newCFS();
+        Tracker tracker = new Tracker(null, false);
+        List<SSTableReader> readers = readers(0, 4, cfs);
+        tracker.addInitialSSTables(readers);
+        LifecycleTransaction txn = tracker.tryModify(readers, OperationType.UNKNOWN);
+        txn.cancel(readers.get(3));
+        LifecycleTransaction txn2 = txn.split(readers.subList(0, 1));
+        Assert.assertEquals(2, txn.originals().size());
+        Assert.assertTrue(all(readers.subList(1, 3), in(txn.originals())));
+        Assert.assertEquals(1, txn2.originals().size());
+        Assert.assertTrue(all(readers.subList(0, 1), in(txn2.originals())));
+        txn.update(readers(1, 2, cfs).get(0), true);
+        boolean failed = false;
+        try
+        {
+            txn.split(readers.subList(2, 3));
+        }
+        catch (Throwable t)
+        {
+            failed = true;
+        }
+        Assert.assertTrue(failed);
+    }
+
+    private static void testBadUpdate(LifecycleTransaction txn, SSTableReader update, boolean original)
+    {
+        boolean failed = false;
+        try
+        {
+            txn.update(update, original);
+        }
+        catch (Throwable t)
+        {
+            failed = true;
+        }
+        Assert.assertTrue(failed);
+    }
+
+    private static void testBadObsolete(LifecycleTransaction txn, SSTableReader update)
+    {
+        boolean failed = false;
+        try
+        {
+            txn.obsolete(update);
+        }
+        catch (Throwable t)
+        {
+            failed = true;
+        }
+        Assert.assertTrue(failed);
+    }
+
+    private static void testBadCancel(LifecycleTransaction txn, SSTableReader cancel)
+    {
+        boolean failed = false;
+        try
+        {
+            txn.cancel(cancel);
+        }
+        catch (Throwable t)
+        {
+            failed = true;
+        }
+        Assert.assertTrue(failed);
+    }
+
+    protected TestableTransaction newTest()
+    {
+        SSTableDeletingTask.waitForDeletions();
+        SSTableReader.resetTidying();
+        return new TxnTest();
+    }
+
+    private static final class TxnTest extends TestableTransaction
+    {
+        final List<SSTableReader> originals;
+        final List<SSTableReader> untouchedOriginals;
+        final List<SSTableReader> loggedUpdate;
+        final List<SSTableReader> loggedObsolete;
+        final List<SSTableReader> stagedObsolete;
+        final List<SSTableReader> loggedNew;
+        final List<SSTableReader> stagedNew;
+        final Tracker tracker;
+        final LifecycleTransaction txn;
+
+        private static Tracker tracker(ColumnFamilyStore cfs, List<SSTableReader> readers)
+        {
+            Tracker tracker = new Tracker(new Memtable(new AtomicReference<>(ReplayPosition.NONE), cfs), false);
+            tracker.addInitialSSTables(readers);
+            return tracker;
+        }
+
+        private TxnTest()
+        {
+            this(MockSchema.newCFS());
+        }
+
+        private TxnTest(ColumnFamilyStore cfs)
+        {
+            this(cfs, readers(0, 8, cfs));
+        }
+
+        private TxnTest(ColumnFamilyStore cfs, List<SSTableReader> readers)
+        {
+            this(tracker(cfs, readers), readers);
+        }
+
+        private TxnTest(Tracker tracker, List<SSTableReader> readers)
+        {
+            this(tracker, readers, tracker.tryModify(readers, OperationType.UNKNOWN));
+        }
+
+        private TxnTest(Tracker tracker, List<SSTableReader> readers, LifecycleTransaction txn)
+        {
+            super(txn);
+            this.tracker = tracker;
+            this.originals = readers;
+            this.txn = txn;
+            update(txn, loggedUpdate = readers(0, 2, tracker.cfstore), true);
+            obsolete(txn, loggedObsolete = readers.subList(2, 4));
+            update(txn, loggedNew = readers(8, 10, tracker.cfstore), false);
+            txn.checkpoint();
+            update(txn, stagedNew = readers(10, 12, tracker.cfstore), false);
+            obsolete(txn, stagedObsolete = copyOf(concat(loggedUpdate, originals.subList(4, 6))));
+            untouchedOriginals = originals.subList(6, 8);
+        }
+
+        private ReaderState state(SSTableReader reader, State state)
+        {
+            SSTableReader original = select(reader, originals);
+            boolean isOriginal = original != null;
+
+            switch (state)
+            {
+                case ABORTED:
+                {
+                    return new ReaderState(Action.NONE, Action.NONE, original, original, isOriginal);
+                }
+
+                case READY_TO_COMMIT:
+                {
+                    ReaderState prev = state(reader, State.IN_PROGRESS);
+                    Action logged;
+                    SSTableReader visible;
+                    if (prev.staged == Action.NONE)
+                    {
+                        logged = prev.logged;
+                        visible = prev.currentlyVisible;
+                    }
+                    else
+                    {
+                        logged = prev.staged;
+                        visible = prev.nextVisible;
+                    }
+                    return new ReaderState(logged, Action.NONE, visible, visible, isOriginal);
+                }
+
+                case IN_PROGRESS:
+                {
+                    Action logged = Action.get(loggedUpdate.contains(reader) || loggedNew.contains(reader), loggedObsolete.contains(reader));
+                    Action staged = Action.get(stagedNew.contains(reader), stagedObsolete.contains(reader));
+                    SSTableReader currentlyVisible = ReaderState.visible(reader, in(loggedObsolete), loggedNew, loggedUpdate, originals);
+                    SSTableReader nextVisible = ReaderState.visible(reader, orIn(stagedObsolete, loggedObsolete), stagedNew, loggedNew, loggedUpdate, originals);
+                    return new ReaderState(logged, staged, currentlyVisible, nextVisible, isOriginal);
+                }
+            }
+            throw new IllegalStateException();
+        }
+
+        private List<Pair<SSTableReader, ReaderState>> states(State state)
+        {
+            List<Pair<SSTableReader, ReaderState>> result = new ArrayList<>();
+            for (SSTableReader reader : concat(originals, loggedNew, stagedNew))
+                result.add(Pair.create(reader, state(reader, state)));
+            return result;
+        }
+
+        protected void doAssert(State state)
+        {
+            for (Pair<SSTableReader, ReaderState> pair : states(state))
+            {
+                SSTableReader reader = pair.left;
+                ReaderState readerState = pair.right;
+
+                Assert.assertEquals(readerState, txn.state(reader));
+                Assert.assertEquals(readerState.currentlyVisible, tracker.getView().sstablesMap.get(reader));
+                if (readerState.currentlyVisible == null && readerState.nextVisible == null && !readerState.original)
+                    Assert.assertTrue(reader.selfRef().globalCount() == 0);
+            }
+        }
+
+        protected void assertInProgress() throws Exception
+        {
+            doAssert(State.IN_PROGRESS);
+        }
+
+        protected void assertPrepared() throws Exception
+        {
+            doAssert(State.READY_TO_COMMIT);
+        }
+
+        protected void assertAborted() throws Exception
+        {
+            doAssert(State.ABORTED);
+            Assert.assertEquals(0, tracker.getView().compacting.size());
+            Assert.assertEquals(8, tracker.getView().sstables.size());
+            for (SSTableReader reader : concat(loggedNew, stagedNew))
+                Assert.assertTrue(reader.selfRef().globalCount() == 0);
+        }
+
+        protected void assertCommitted() throws Exception
+        {
+            doAssert(State.READY_TO_COMMIT);
+            Assert.assertEquals(0, tracker.getView().compacting.size());
+            Assert.assertEquals(6, tracker.getView().sstables.size());
+            for (SSTableReader reader : concat(loggedObsolete, stagedObsolete))
+                Assert.assertTrue(reader.selfRef().globalCount() == 0);
+        }
+    }
+
+    private static SSTableReader[] readersArray(int lb, int ub, ColumnFamilyStore cfs)
+    {
+        return readers(lb, ub, cfs).toArray(new SSTableReader[0]);
+    }
+
+    private static List<SSTableReader> readers(int lb, int ub, ColumnFamilyStore cfs)
+    {
+        List<SSTableReader> readers = new ArrayList<>();
+        for (int i = lb ; i < ub ; i++)
+            readers.add(MockSchema.sstable(i, i, true, cfs));
+        return copyOf(readers);
+    }
+
+    private static void update(LifecycleTransaction txn, Iterable<SSTableReader> readers, boolean originals)
+    {
+        for (SSTableReader reader : readers)
+            txn.update(reader, originals);
+    }
+
+    private static void obsolete(LifecycleTransaction txn, Iterable<SSTableReader> readers)
+    {
+        for (SSTableReader reader : readers)
+            txn.obsolete(reader);
+    }
+}
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java b/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
new file mode 100644
index 0000000..04b4e4a
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
@@ -0,0 +1,367 @@
+/*
+* 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.db.lifecycle;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+
+import javax.annotation.Nullable;
+
+import com.google.common.base.Function;
+import com.google.common.base.Predicate;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import junit.framework.Assert;
+import org.apache.cassandra.MockSchema;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Memtable;
+import org.apache.cassandra.db.commitlog.CommitLog;
+import org.apache.cassandra.db.commitlog.ReplayPosition;
+import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.io.sstable.SSTableDeletingTask;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.notifications.*;
+import org.apache.cassandra.utils.concurrent.OpOrder;
+
+import static com.google.common.collect.ImmutableSet.copyOf;
+import static java.util.Collections.singleton;
+
+public class TrackerTest
+{
+
+    private static final class MockListener implements INotificationConsumer
+    {
+        final boolean throwException;
+        final List<INotification> received = new ArrayList<>();
+        final List<Object> senders = new ArrayList<>();
+
+        private MockListener(boolean throwException)
+        {
+            this.throwException = throwException;
+        }
+
+        public void handleNotification(INotification notification, Object sender)
+        {
+            if (throwException)
+                throw new RuntimeException();
+            received.add(notification);
+            senders.add(sender);
+        }
+    }
+
+    @BeforeClass
+    public static void setUp()
+    {
+        DatabaseDescriptor.setDaemonInitialized();
+        MockSchema.cleanup();
+    }
+
+    @Test
+    public void testTryModify()
+    {
+        ColumnFamilyStore cfs = MockSchema.newCFS();
+        Tracker tracker = new Tracker(null, false);
+        List<SSTableReader> readers = ImmutableList.of(MockSchema.sstable(0, true, cfs), MockSchema.sstable(1, cfs), MockSchema.sstable(2, cfs));
+        tracker.addInitialSSTables(copyOf(readers));
+        Assert.assertNull(tracker.tryModify(ImmutableList.of(MockSchema.sstable(0, cfs)), OperationType.COMPACTION));
+        try (LifecycleTransaction txn = tracker.tryModify(readers.get(0), OperationType.COMPACTION);)
+        {
+            Assert.assertNotNull(txn);
+            Assert.assertNull(tracker.tryModify(readers.get(0), OperationType.COMPACTION));
+            Assert.assertEquals(1, txn.originals().size());
+            Assert.assertTrue(txn.originals().contains(readers.get(0)));
+        }
+        try (LifecycleTransaction txn = tracker.tryModify(Collections.<SSTableReader>emptyList(), OperationType.COMPACTION);)
+        {
+            Assert.assertNotNull(txn);
+            Assert.assertEquals(0, txn.originals().size());
+        }
+        readers.get(0).selfRef().release();
+    }
+
+    @Test
+    public void testApply()
+    {
+        final ColumnFamilyStore cfs = MockSchema.newCFS();
+        final Tracker tracker = new Tracker(null, false);
+        final View resultView = ViewTest.fakeView(0, 0, cfs);
+        final AtomicInteger count = new AtomicInteger();
+        tracker.apply(new Predicate<View>()
+        {
+            public boolean apply(View view)
+            {
+                // confound the CAS by swapping the view, and check we retry
+                if (count.incrementAndGet() < 3)
+                    tracker.view.set(ViewTest.fakeView(0, 0, cfs));
+                return true;
+            }
+        }, new Function<View, View>()
+        {
+            @Nullable
+            public View apply(View view)
+            {
+                return resultView;
+            }
+        });
+        Assert.assertEquals(3, count.get());
+        Assert.assertEquals(resultView, tracker.getView());
+
+        count.set(0);
+        // check that if the predicate returns false, we stop immediately and return null
+        Assert.assertNull(tracker.apply(new Predicate<View>()
+        {
+            public boolean apply(View view)
+            {
+                count.incrementAndGet();
+                return false;
+            }
+        }, null));
+        Assert.assertEquals(1, count.get());
+        Assert.assertEquals(resultView, tracker.getView());
+    }
+
+    @Test
+    public void testAddInitialSSTables()
+    {
+        ColumnFamilyStore cfs = MockSchema.newCFS();
+        Tracker tracker = cfs.getTracker();
+        List<SSTableReader> readers = ImmutableList.of(MockSchema.sstable(0, 17, cfs),
+                                                       MockSchema.sstable(1, 121, cfs),
+                                                       MockSchema.sstable(2, 9, cfs));
+        tracker.addInitialSSTables(copyOf(readers));
+
+        Assert.assertEquals(3, tracker.view.get().sstables.size());
+
+        Assert.assertEquals(17 + 121 + 9, cfs.metric.liveDiskSpaceUsed.getCount());
+    }
+
+    @Test
+    public void testAddSSTables()
+    {
+        boolean backups = DatabaseDescriptor.isIncrementalBackupsEnabled();
+        DatabaseDescriptor.setIncrementalBackupsEnabled(false);
+        ColumnFamilyStore cfs = MockSchema.newCFS();
+        Tracker tracker = cfs.getTracker();
+        MockListener listener = new MockListener(false);
+        tracker.subscribe(listener);
+        List<SSTableReader> readers = ImmutableList.of(MockSchema.sstable(0, 17, cfs),
+                                                       MockSchema.sstable(1, 121, cfs),
+                                                       MockSchema.sstable(2, 9, cfs));
+        tracker.addSSTables(copyOf(readers));
+
+        Assert.assertEquals(3, tracker.view.get().sstables.size());
+
+        Assert.assertEquals(17 + 121 + 9, cfs.metric.liveDiskSpaceUsed.getCount());
+        Assert.assertEquals(3, listener.senders.size());
+        Assert.assertEquals(tracker, listener.senders.get(0));
+        Assert.assertTrue(listener.received.get(0) instanceof SSTableAddedNotification);
+        DatabaseDescriptor.setIncrementalBackupsEnabled(backups);
+    }
+
+    @Test
+    public void testDropSSTables()
+    {
+        testDropSSTables(false);
+        SSTableDeletingTask.waitForDeletions();
+        testDropSSTables(true);
+        SSTableDeletingTask.waitForDeletions();
+    }
+
+    private void testDropSSTables(boolean invalidate)
+    {
+        ColumnFamilyStore cfs = MockSchema.newCFS();
+        Tracker tracker = cfs.getTracker();
+        MockListener listener = new MockListener(false);
+        tracker.subscribe(listener);
+        final List<SSTableReader> readers = ImmutableList.of(MockSchema.sstable(0, 9, true, cfs),
+                                                             MockSchema.sstable(1, 15, true, cfs),
+                                                             MockSchema.sstable(2, 71, true, cfs));
+        tracker.addInitialSSTables(copyOf(readers));
+
+        try
+        {
+            SSTableDeletingTask.pauseDeletions(true);
+            try (LifecycleTransaction txn = tracker.tryModify(readers.get(0), OperationType.COMPACTION))
+            {
+                if (invalidate)
+                    cfs.invalidate(false);
+                else
+                    tracker.dropSSTables();
+                Assert.assertEquals(95, cfs.metric.totalDiskSpaceUsed.getCount());
+                Assert.assertEquals(9, cfs.metric.liveDiskSpaceUsed.getCount());
+                Assert.assertEquals(1, tracker.getView().sstables.size());
+            }
+            if (!invalidate)
+            {
+                Assert.assertEquals(1, tracker.getView().sstables.size());
+                Assert.assertEquals(readers.get(0), Iterables.getFirst(tracker.getView().sstables, null));
+                Assert.assertEquals(1, readers.get(0).selfRef().globalCount());
+                Assert.assertFalse(readers.get(0).isMarkedCompacted());
+                for (SSTableReader reader : readers.subList(1, 3))
+                {
+                    Assert.assertEquals(0, reader.selfRef().globalCount());
+                    Assert.assertTrue(reader.isMarkedCompacted());
+                }
+                Assert.assertNull(tracker.dropSSTables(new Predicate<SSTableReader>() {
+                                                           public boolean apply(SSTableReader reader)
+                                                           {
+                                                               return reader != readers.get(0);
+                                                           }
+                                                       },
+                                                       OperationType.UNKNOWN,
+                                                       null));
+                Assert.assertEquals(1, tracker.getView().sstables.size());
+                Assert.assertEquals(3, listener.received.size());
+                Assert.assertEquals(tracker, listener.senders.get(0));
+                Assert.assertEquals(2, ((SSTableListChangedNotification) listener.received.get(0)).removed.size());
+                Assert.assertEquals(0, ((SSTableListChangedNotification) listener.received.get(0)).added.size());
+                Assert.assertEquals(9, cfs.metric.liveDiskSpaceUsed.getCount());
+                readers.get(0).selfRef().release();
+            }
+            else
+            {
+                Assert.assertEquals(0, tracker.getView().sstables.size());
+                Assert.assertEquals(0, cfs.metric.liveDiskSpaceUsed.getCount());
+                for (SSTableReader reader : readers)
+                    Assert.assertTrue(reader.isMarkedCompacted());
+            }
+        }
+        finally
+        {
+            SSTableDeletingTask.pauseDeletions(false);
+        }
+    }
+
+    @Test
+    public void testMemtableReplacement()
+    {
+        boolean backups = DatabaseDescriptor.isIncrementalBackupsEnabled();
+        DatabaseDescriptor.setIncrementalBackupsEnabled(false);
+        ColumnFamilyStore cfs = MockSchema.newCFS();
+        MockListener listener = new MockListener(false);
+        Tracker tracker = cfs.getTracker();
+        tracker.subscribe(listener);
+
+        Memtable prev1 = tracker.switchMemtable(true, new Memtable(new AtomicReference<>(CommitLog.instance.getContext()), cfs));
+        OpOrder.Group write1 = cfs.keyspace.writeOrder.getCurrent();
+        OpOrder.Barrier barrier1 = cfs.keyspace.writeOrder.newBarrier();
+        prev1.setDiscarding(barrier1, new AtomicReference<>(CommitLog.instance.getContext()));
+        barrier1.issue();
+        Memtable prev2 = tracker.switchMemtable(false, new Memtable(new AtomicReference<>(CommitLog.instance.getContext()), cfs));
+        OpOrder.Group write2 = cfs.keyspace.writeOrder.getCurrent();
+        OpOrder.Barrier barrier2 = cfs.keyspace.writeOrder.newBarrier();
+        prev2.setDiscarding(barrier2, new AtomicReference<>(CommitLog.instance.getContext()));
+        barrier2.issue();
+        Memtable cur = tracker.getView().getCurrentMemtable();
+        OpOrder.Group writecur = cfs.keyspace.writeOrder.getCurrent();
+        Assert.assertEquals(prev1, tracker.getMemtableFor(write1, ReplayPosition.NONE));
+        Assert.assertEquals(prev2, tracker.getMemtableFor(write2, ReplayPosition.NONE));
+        Assert.assertEquals(cur, tracker.getMemtableFor(writecur, ReplayPosition.NONE));
+        Assert.assertEquals(1, listener.received.size());
+        Assert.assertTrue(listener.received.get(0) instanceof MemtableRenewedNotification);
+        listener.received.clear();
+
+        tracker.markFlushing(prev2);
+        Assert.assertEquals(1, tracker.getView().flushingMemtables.size());
+        Assert.assertTrue(tracker.getView().flushingMemtables.contains(prev2));
+
+        tracker.markFlushing(prev1);
+        Assert.assertTrue(tracker.getView().flushingMemtables.contains(prev1));
+        Assert.assertEquals(2, tracker.getView().flushingMemtables.size());
+
+        tracker.replaceFlushed(prev1, null);
+        Assert.assertEquals(1, tracker.getView().flushingMemtables.size());
+        Assert.assertTrue(tracker.getView().flushingMemtables.contains(prev2));
+
+        SSTableReader reader = MockSchema.sstable(0, 10, false, cfs);
+        tracker.replaceFlushed(prev2, reader);
+        Assert.assertEquals(1, tracker.getView().sstables.size());
+        Assert.assertEquals(1, tracker.getView().premature.size());
+        tracker.permitCompactionOfFlushed(reader);
+        Assert.assertEquals(0, tracker.getView().premature.size());
+        Assert.assertEquals(1, listener.received.size());
+        Assert.assertEquals(reader, ((SSTableAddedNotification) listener.received.get(0)).added);
+        listener.received.clear();
+        Assert.assertEquals(10, cfs.metric.liveDiskSpaceUsed.getCount());
+
+        // test invalidated CFS
+        cfs = MockSchema.newCFS();
+        tracker = cfs.getTracker();
+        listener = new MockListener(false);
+        tracker.subscribe(listener);
+        prev1 = tracker.switchMemtable(false, new Memtable(new AtomicReference<>(CommitLog.instance.getContext()), cfs));
+        tracker.markFlushing(prev1);
+        reader = MockSchema.sstable(0, 10, true, cfs);
+        cfs.invalidate(false);
+        tracker.replaceFlushed(prev1, reader);
+        tracker.permitCompactionOfFlushed(reader);
+        Assert.assertEquals(0, tracker.getView().sstables.size());
+        Assert.assertEquals(0, tracker.getView().flushingMemtables.size());
+        Assert.assertEquals(0, cfs.metric.liveDiskSpaceUsed.getCount());
+        Assert.assertEquals(1, ((SSTableListChangedNotification) listener.received.get(0)).removed.size());
+        Assert.assertEquals(reader, (((SSTableDeletingNotification) listener.received.get(1)).deleting));
+        DatabaseDescriptor.setIncrementalBackupsEnabled(backups);
+    }
+
+    @Test
+    public void testNotifications()
+    {
+        ColumnFamilyStore cfs = MockSchema.newCFS();
+        SSTableReader r1 = MockSchema.sstable(0, cfs), r2 = MockSchema.sstable(1, cfs);
+        Tracker tracker = new Tracker(null, false);
+        MockListener listener = new MockListener(false);
+        tracker.subscribe(listener);
+        tracker.notifyAdded(r1);
+        Assert.assertEquals(r1, ((SSTableAddedNotification) listener.received.get(0)).added);
+        listener.received.clear();
+        tracker.notifyDeleting(r1);
+        Assert.assertEquals(r1, ((SSTableDeletingNotification) listener.received.get(0)).deleting);
+        listener.received.clear();
+        Assert.assertNull(tracker.notifySSTablesChanged(singleton(r1), singleton(r2), OperationType.COMPACTION, null));
+        Assert.assertEquals(singleton(r1), ((SSTableListChangedNotification) listener.received.get(0)).removed);
+        Assert.assertEquals(singleton(r2), ((SSTableListChangedNotification) listener.received.get(0)).added);
+        listener.received.clear();
+        tracker.notifySSTableRepairedStatusChanged(singleton(r1));
+        Assert.assertEquals(singleton(r1), ((SSTableRepairStatusChanged) listener.received.get(0)).sstable);
+        listener.received.clear();
+        Memtable memtable = MockSchema.memtable(cfs);
+        tracker.notifyRenewed(memtable);
+        Assert.assertEquals(memtable, ((MemtableRenewedNotification) listener.received.get(0)).renewed);
+        listener.received.clear();
+        tracker.unsubscribe(listener);
+        MockListener failListener = new MockListener(true);
+        tracker.subscribe(failListener);
+        tracker.subscribe(listener);
+        Assert.assertNotNull(tracker.notifyAdded(r1, null));
+        Assert.assertEquals(r1, ((SSTableAddedNotification) listener.received.get(0)).added);
+        listener.received.clear();
+        Assert.assertNotNull(tracker.notifySSTablesChanged(singleton(r1), singleton(r2), OperationType.COMPACTION, null));
+        Assert.assertEquals(singleton(r1), ((SSTableListChangedNotification) listener.received.get(0)).removed);
+        Assert.assertEquals(singleton(r2), ((SSTableListChangedNotification) listener.received.get(0)).added);
+        listener.received.clear();
+    }
+
+}
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/ViewTest.java b/test/unit/org/apache/cassandra/db/lifecycle/ViewTest.java
new file mode 100644
index 0000000..5706598
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/lifecycle/ViewTest.java
@@ -0,0 +1,213 @@
+/*
+* 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.db.lifecycle;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import com.google.common.base.Function;
+import com.google.common.base.Predicates;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import junit.framework.Assert;
+import org.apache.cassandra.MockSchema;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Memtable;
+import org.apache.cassandra.db.RowPosition;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+
+import static com.google.common.collect.ImmutableSet.copyOf;
+import static com.google.common.collect.ImmutableSet.of;
+import static com.google.common.collect.Iterables.concat;
+import static org.apache.cassandra.db.lifecycle.Helpers.emptySet;
+
+public class ViewTest
+{
+    @BeforeClass
+    public static void setUp()
+    {
+        MockSchema.cleanup();
+    }
+
+    @Test
+    public void testSSTablesInBounds()
+    {
+        ColumnFamilyStore cfs = MockSchema.newCFS();
+        View initialView = fakeView(0, 5, cfs);
+        for (int i = 0 ; i < 5 ; i++)
+        {
+            for (int j = i ; j < 5 ; j++)
+            {
+                RowPosition min = MockSchema.readerBounds(i);
+                RowPosition max = MockSchema.readerBounds(j);
+                for (boolean minInc : new boolean[] { true })//, false} )
+                {
+                    for (boolean maxInc : new boolean[] { true })//, false} )
+                    {
+                        if (i == j && !(minInc && maxInc))
+                            continue;
+                        AbstractBounds<RowPosition> bounds = AbstractBounds.bounds(min, minInc, max, maxInc);
+                        List<SSTableReader> r = initialView.sstablesInBounds(bounds.left, bounds.right);
+                        Assert.assertEquals(String.format("%d(%s) %d(%s)", i, minInc, j, maxInc), j - i + (minInc ? 0 : -1) + (maxInc ? 1 : 0), r.size());
+                    }
+                }
+            }
+        }
+    }
+
+    @Test
+    public void testCompaction()
+    {
+        ColumnFamilyStore cfs = MockSchema.newCFS();
+        View initialView = fakeView(0, 5, cfs);
+        View cur = initialView;
+        List<SSTableReader> readers = ImmutableList.copyOf(initialView.sstables);
+        Assert.assertTrue(View.permitCompacting(readers).apply(cur));
+        // check we permit compacting duplicates in the predicate, so we don't spin infinitely if there is a screw up
+        Assert.assertTrue(View.permitCompacting(ImmutableList.copyOf(concat(readers, readers))).apply(cur));
+        // check we fail in the application in the presence of duplicates
+        testFailure(View.updateCompacting(emptySet(), concat(readers.subList(0, 1), readers.subList(0, 1))), cur);
+
+        // do lots of trivial checks that the compacting set and related methods behave properly for a simple update
+        cur = View.updateCompacting(emptySet(), readers.subList(0, 2)).apply(cur);
+        Assert.assertTrue(View.permitCompacting(readers.subList(2, 5)).apply(cur));
+        Assert.assertFalse(View.permitCompacting(readers.subList(0, 2)).apply(cur));
+        Assert.assertFalse(View.permitCompacting(readers.subList(0, 1)).apply(cur));
+        Assert.assertFalse(View.permitCompacting(readers.subList(1, 2)).apply(cur));
+        Assert.assertTrue(readers.subList(2, 5).containsAll(copyOf(cur.getUncompacting(readers))));
+        Assert.assertEquals(3, copyOf(cur.getUncompacting(readers)).size());
+        Assert.assertTrue(cur.nonCompactingSStables().containsAll(readers.subList(2, 5)));
+        Assert.assertEquals(3, cur.nonCompactingSStables().size());
+
+        // check marking already compacting readers fails with an exception
+        testFailure(View.updateCompacting(emptySet(), readers.subList(0, 1)), cur);
+        testFailure(View.updateCompacting(emptySet(), readers.subList(1, 2)), cur);
+        testFailure(View.updateCompacting(copyOf(readers.subList(0, 1)), readers.subList(1, 2)), cur);
+
+        // make equivalents of readers.subList(0, 3) that are different instances
+        SSTableReader r0 = MockSchema.sstable(0, cfs), r1 = MockSchema.sstable(1, cfs), r2 = MockSchema.sstable(2, cfs);
+        // attempt to mark compacting a version not in the live set
+        testFailure(View.updateCompacting(emptySet(), of(r2)), cur);
+        // update one compacting, one non-compacting, of the liveset to another instance of the same readers;
+        // confirm liveset changes but compacting does not
+        cur = View.updateLiveSet(copyOf(readers.subList(1, 3)), of(r1, r2)).apply(cur);
+        Assert.assertSame(readers.get(0), cur.sstablesMap.get(r0));
+        Assert.assertSame(r1, cur.sstablesMap.get(r1));
+        Assert.assertSame(r2, cur.sstablesMap.get(r2));
+        testFailure(View.updateCompacting(emptySet(), readers.subList(2, 3)), cur);
+        Assert.assertSame(readers.get(1), Iterables.getFirst(Iterables.filter(cur.compacting, Predicates.equalTo(r1)), null));
+
+        // unmark compacting, and check our methods are all correctly updated
+        cur = View.updateCompacting(copyOf(readers.subList(0, 1)), emptySet()).apply(cur);
+        Assert.assertTrue(View.permitCompacting(concat(readers.subList(0, 1), of(r2), readers.subList(3, 5))).apply(cur));
+        Assert.assertFalse(View.permitCompacting(readers.subList(1, 2)).apply(cur));
+        testFailure(View.updateCompacting(emptySet(), readers.subList(1, 2)), cur);
+        testFailure(View.updateCompacting(copyOf(readers.subList(0, 2)), emptySet()), cur);
+        Assert.assertTrue(copyOf(concat(readers.subList(0, 1), readers.subList(2, 5))).containsAll(copyOf(cur.getUncompacting(readers))));
+        Assert.assertEquals(4, copyOf(cur.getUncompacting(readers)).size());
+        Assert.assertTrue(cur.nonCompactingSStables().containsAll(readers.subList(2, 5)));
+        Assert.assertTrue(cur.nonCompactingSStables().containsAll(readers.subList(0, 1)));
+        Assert.assertEquals(4, cur.nonCompactingSStables().size());
+    }
+
+    private static void testFailure(Function<View, ?> function, View view)
+    {
+        boolean failed = true;
+        try
+        {
+            function.apply(view);
+            failed = false;
+        }
+        catch (Throwable t)
+        {
+        }
+        Assert.assertTrue(failed);
+    }
+
+    @Test
+    public void testFlushing()
+    {
+        ColumnFamilyStore cfs = MockSchema.newCFS();
+        View initialView = fakeView(1, 0, cfs);
+        View cur = initialView;
+        Memtable memtable1 = initialView.getCurrentMemtable();
+        Memtable memtable2 = MockSchema.memtable(cfs);
+
+        cur = View.switchMemtable(memtable2).apply(cur);
+        Assert.assertEquals(2, cur.liveMemtables.size());
+        Assert.assertEquals(memtable1, cur.liveMemtables.get(0));
+        Assert.assertEquals(memtable2, cur.getCurrentMemtable());
+
+        Memtable memtable3 = MockSchema.memtable(cfs);
+        cur = View.switchMemtable(memtable3).apply(cur);
+        Assert.assertEquals(3, cur.liveMemtables.size());
+        Assert.assertEquals(0, cur.flushingMemtables.size());
+        Assert.assertEquals(memtable1, cur.liveMemtables.get(0));
+        Assert.assertEquals(memtable2, cur.liveMemtables.get(1));
+        Assert.assertEquals(memtable3, cur.getCurrentMemtable());
+
+        testFailure(View.replaceFlushed(memtable2, null), cur);
+
+        cur = View.markFlushing(memtable2).apply(cur);
+        Assert.assertTrue(cur.flushingMemtables.contains(memtable2));
+        Assert.assertEquals(2, cur.liveMemtables.size());
+        Assert.assertEquals(1, cur.flushingMemtables.size());
+        Assert.assertEquals(memtable2, cur.flushingMemtables.get(0));
+        Assert.assertEquals(memtable1, cur.liveMemtables.get(0));
+        Assert.assertEquals(memtable3, cur.getCurrentMemtable());
+
+        cur = View.markFlushing(memtable1).apply(cur);
+        Assert.assertEquals(1, cur.liveMemtables.size());
+        Assert.assertEquals(2, cur.flushingMemtables.size());
+        Assert.assertEquals(memtable1, cur.flushingMemtables.get(0));
+        Assert.assertEquals(memtable2, cur.flushingMemtables.get(1));
+        Assert.assertEquals(memtable3, cur.getCurrentMemtable());
+
+        cur = View.replaceFlushed(memtable2, null).apply(cur);
+        Assert.assertEquals(1, cur.liveMemtables.size());
+        Assert.assertEquals(1, cur.flushingMemtables.size());
+        Assert.assertEquals(memtable1, cur.flushingMemtables.get(0));
+        Assert.assertEquals(memtable3, cur.getCurrentMemtable());
+
+        SSTableReader sstable = MockSchema.sstable(1, cfs);
+        cur = View.replaceFlushed(memtable1, sstable).apply(cur);
+        Assert.assertEquals(0, cur.flushingMemtables.size());
+        Assert.assertEquals(1, cur.liveMemtables.size());
+        Assert.assertEquals(memtable3, cur.getCurrentMemtable());
+        Assert.assertEquals(1, cur.sstables.size());
+        Assert.assertEquals(sstable, cur.sstablesMap.get(sstable));
+    }
+
+    static View fakeView(int memtableCount, int sstableCount, ColumnFamilyStore cfs)
+    {
+        List<Memtable> memtables = new ArrayList<>();
+        List<SSTableReader> sstables = new ArrayList<>();
+        for (int i = 0 ; i < memtableCount ; i++)
+            memtables.add(MockSchema.memtable(cfs));
+        for (int i = 0 ; i < sstableCount ; i++)
+            sstables.add(MockSchema.sstable(i, cfs));
+        return new View(ImmutableList.copyOf(memtables), Collections.<Memtable>emptyList(), Helpers.identityMap(sstables),
+                        Collections.<SSTableReader>emptySet(), Collections.<SSTableReader>emptySet(), SSTableIntervalTree.build(sstables));
+    }
+}
diff --git a/test/unit/org/apache/cassandra/db/marshal/CompositeTypeTest.java b/test/unit/org/apache/cassandra/db/marshal/CompositeTypeTest.java
index f606780..25c1c7d 100644
--- a/test/unit/org/apache/cassandra/db/marshal/CompositeTypeTest.java
+++ b/test/unit/org/apache/cassandra/db/marshal/CompositeTypeTest.java
@@ -19,28 +19,30 @@
 package org.apache.cassandra.db.marshal;
 
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.UUID;
+import java.util.*;
 
-import org.apache.cassandra.serializers.MarshalException;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import static org.junit.Assert.fail;
 import static org.junit.Assert.assertEquals;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.composites.CellNames;
 import org.apache.cassandra.db.filter.QueryFilter;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.utils.*;
 
-public class CompositeTypeTest extends SchemaLoader
+public class CompositeTypeTest
 {
-    private static final String cfName = "StandardComposite";
+    private static final String KEYSPACE1 = "CompositeTypeTest";
+    private static final String CF_STANDARDCOMPOSITE = "StandardComposite";
     private static final CompositeType comparator;
     static
     {
@@ -59,6 +61,17 @@
             uuids[i] = UUIDGen.getTimeUUID();
     }
 
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        AbstractType<?> composite = CompositeType.getInstance(Arrays.asList(new AbstractType<?>[]{BytesType.instance, TimeUUIDType.instance, IntegerType.instance}));
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARDCOMPOSITE, composite));
+    }
+
     @Test
     public void testEndOfComponent()
     {
@@ -164,8 +177,8 @@
     @Test
     public void testFullRound() throws Exception
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_STANDARDCOMPOSITE);
 
         ByteBuffer cname1 = createCompositeKey("test1", null, -1, false);
         ByteBuffer cname2 = createCompositeKey("test1", uuids[0], 24, false);
@@ -174,15 +187,15 @@
         ByteBuffer cname5 = createCompositeKey("test2", uuids[1], 42, false);
 
         ByteBuffer key = ByteBufferUtil.bytes("k");
-        Mutation rm = new Mutation("Keyspace1", key);
+        Mutation rm = new Mutation(KEYSPACE1, key);
         addColumn(rm, cname5);
         addColumn(rm, cname1);
         addColumn(rm, cname4);
         addColumn(rm, cname2);
         addColumn(rm, cname3);
-        rm.apply();
+        rm.applyUnsafe();
 
-        ColumnFamily cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(Util.dk("k"), cfName, System.currentTimeMillis()));
+        ColumnFamily cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(Util.dk("k"), CF_STANDARDCOMPOSITE, System.currentTimeMillis()));
 
         Iterator<Cell> iter = cf.getSortedColumns().iterator();
 
@@ -201,16 +214,14 @@
             TypeParser.parse("CompositeType");
             fail("Shouldn't work");
         }
-        catch (ConfigurationException e) {}
-        catch (SyntaxException e) {}
+        catch (ConfigurationException | SyntaxException e) {}
 
         try
         {
             TypeParser.parse("CompositeType()");
             fail("Shouldn't work");
         }
-        catch (ConfigurationException e) {}
-        catch (SyntaxException e) {}
+        catch (ConfigurationException | SyntaxException e) {}
     }
 
     @Test
@@ -258,7 +269,7 @@
 
     private void addColumn(Mutation rm, ByteBuffer cname)
     {
-        rm.add(cfName, CellNames.simpleDense(cname), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
+        rm.add(CF_STANDARDCOMPOSITE, CellNames.simpleDense(cname), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
     }
 
     private ByteBuffer createCompositeKey(String s, UUID uuid, int i, boolean lastIsOne)
diff --git a/test/unit/org/apache/cassandra/db/marshal/DynamicCompositeTypeTest.java b/test/unit/org/apache/cassandra/db/marshal/DynamicCompositeTypeTest.java
index e9c47a9..1a6ddc9 100644
--- a/test/unit/org/apache/cassandra/db/marshal/DynamicCompositeTypeTest.java
+++ b/test/unit/org/apache/cassandra/db/marshal/DynamicCompositeTypeTest.java
@@ -24,25 +24,31 @@
 import java.util.Map;
 import java.util.UUID;
 
-import org.apache.cassandra.serializers.MarshalException;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import static org.junit.Assert.fail;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.db.filter.QueryFilter;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.utils.*;
 
-public class DynamicCompositeTypeTest extends SchemaLoader
+public class DynamicCompositeTypeTest
 {
-    private static final String cfName = "StandardDynamicComposite";
+    private static final String KEYSPACE1 = "DynamicCompositeType";
+    private static final String CF_STANDARDDYNCOMPOSITE = "StandardDynamicComposite";
+    private static Map<Byte, AbstractType<?>> aliases = new HashMap<>();
 
     private static final DynamicCompositeType comparator;
     static
     {
-        Map<Byte, AbstractType<?>> aliases = new HashMap<Byte, AbstractType<?>>();
         aliases.put((byte)'b', BytesType.instance);
         aliases.put((byte)'B', ReversedType.getInstance(BytesType.instance));
         aliases.put((byte)'t', TimeUUIDType.instance);
@@ -58,6 +64,17 @@
             uuids[i] = UUIDGen.getTimeUUID();
     }
 
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        AbstractType<?> dynamicComposite = DynamicCompositeType.getInstance(aliases);
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARDDYNCOMPOSITE, dynamicComposite));
+    }
+
     @Test
     public void testEndOfComponent()
     {
@@ -165,8 +182,8 @@
     @Test
     public void testFullRound() throws Exception
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_STANDARDDYNCOMPOSITE);
 
         ByteBuffer cname1 = createDynamicCompositeKey("test1", null, -1, false);
         ByteBuffer cname2 = createDynamicCompositeKey("test1", uuids[0], 24, false);
@@ -175,15 +192,15 @@
         ByteBuffer cname5 = createDynamicCompositeKey("test2", uuids[1], 42, false);
 
         ByteBuffer key = ByteBufferUtil.bytes("k");
-        Mutation rm = new Mutation("Keyspace1", key);
+        Mutation rm = new Mutation(KEYSPACE1, key);
         addColumn(rm, cname5);
         addColumn(rm, cname1);
         addColumn(rm, cname4);
         addColumn(rm, cname2);
         addColumn(rm, cname3);
-        rm.apply();
+        rm.applyUnsafe();
 
-        ColumnFamily cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(Util.dk("k"), cfName, System.currentTimeMillis()));
+        ColumnFamily cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(Util.dk("k"), CF_STANDARDDYNCOMPOSITE, System.currentTimeMillis()));
 
         Iterator<Cell> iter = cf.getSortedColumns().iterator();
 
@@ -197,8 +214,8 @@
     @Test
     public void testFullRoundReversed() throws Exception
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_STANDARDDYNCOMPOSITE);
 
         ByteBuffer cname1 = createDynamicCompositeKey("test1", null, -1, false, true);
         ByteBuffer cname2 = createDynamicCompositeKey("test1", uuids[0], 24, false, true);
@@ -207,7 +224,7 @@
         ByteBuffer cname5 = createDynamicCompositeKey("test2", uuids[1], 42, false, true);
 
         ByteBuffer key = ByteBufferUtil.bytes("kr");
-        Mutation rm = new Mutation("Keyspace1", key);
+        Mutation rm = new Mutation(KEYSPACE1, key);
         addColumn(rm, cname5);
         addColumn(rm, cname1);
         addColumn(rm, cname4);
@@ -215,7 +232,7 @@
         addColumn(rm, cname3);
         rm.apply();
 
-        ColumnFamily cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(Util.dk("kr"), cfName, System.currentTimeMillis()));
+        ColumnFamily cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(Util.dk("kr"), CF_STANDARDDYNCOMPOSITE, System.currentTimeMillis()));
 
         Iterator<Cell> iter = cf.getSortedColumns().iterator();
 
@@ -294,7 +311,7 @@
 
     private void addColumn(Mutation rm, ByteBuffer cname)
     {
-        rm.add(cfName, CellNames.simpleDense(cname), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
+        rm.add(CF_STANDARDDYNCOMPOSITE, CellNames.simpleDense(cname), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
     }
 
     private ByteBuffer createDynamicCompositeKey(String s, UUID uuid, int i, boolean lastIsOne)
diff --git a/test/unit/org/apache/cassandra/db/marshal/SimpleDateTypeTest.java b/test/unit/org/apache/cassandra/db/marshal/SimpleDateTypeTest.java
new file mode 100644
index 0000000..5c9ed4e
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/marshal/SimpleDateTypeTest.java
@@ -0,0 +1,153 @@
+/**
+ * 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.db.marshal;
+
+import java.nio.ByteBuffer;
+
+import org.junit.Test;
+import org.apache.cassandra.serializers.SimpleDateSerializer;
+
+public class SimpleDateTypeTest
+{
+    @Test public void TestComparison()
+    {
+        ByteBuffer d1 = SimpleDateType.instance.fromString("1970-01-05");
+        ByteBuffer d2 = SimpleDateSerializer.instance.serialize(makeUnsigned(4));
+        assert SimpleDateType.instance.compare(d1, d2) == 0 : "Failed == comparison";
+            String.format("Failed == comparison with %s and %s",
+                SimpleDateSerializer.instance.deserialize(d1),
+                SimpleDateSerializer.instance.deserialize(d2));
+
+        d1 = SimpleDateType.instance.fromString("1970-01-05");
+        d2 = SimpleDateSerializer.instance.serialize(makeUnsigned(10));
+        assert SimpleDateType.instance.compare(d1, d2) < 0 :
+            String.format("Failed comparison of %s and %s, expected <",
+                SimpleDateSerializer.instance.deserialize(d1),
+                SimpleDateSerializer.instance.deserialize(d2));
+
+        d1 = SimpleDateType.instance.fromString("1970-01-05"); // 4
+        d2 = SimpleDateSerializer.instance.serialize(makeUnsigned(-10));
+        assert SimpleDateType.instance.compare(d1, d2) > 0 :
+            String.format("Failed comparison of %s and %s, expected > 0",
+                SimpleDateSerializer.instance.deserialize(d1),
+                SimpleDateSerializer.instance.deserialize(d2));
+
+        d1 = SimpleDateType.instance.fromString("1");
+        d2 = SimpleDateType.instance.fromString("1000");
+        assert SimpleDateType.instance.compare(d1, d2) < 0 :
+                String.format("Failed < comparison with string inputs %s and %s",
+                        SimpleDateSerializer.instance.deserialize(d1),
+                        SimpleDateSerializer.instance.deserialize(d2));
+
+        Integer intLimit = Integer.MAX_VALUE;
+        d1 = SimpleDateType.instance.fromString("0");
+        d2 = SimpleDateType.instance.fromString(intLimit.toString());
+        assert SimpleDateType.instance.compare(d1, d2) < 0 :
+                String.format("Failed < comparison with string inputs at integer bounds %s and %s",
+                        SimpleDateSerializer.instance.deserialize(d1),
+                        SimpleDateSerializer.instance.deserialize(d2));
+
+        Long overLimit = (long)(Integer.MAX_VALUE);
+        d1 = SimpleDateType.instance.fromString("0");
+        d2 = SimpleDateType.instance.fromString(overLimit.toString());
+        assert SimpleDateType.instance.compare(d1, d2) < 0 :
+                String.format("Failed < comparison with string inputs at integer bounds %s and %s",
+                        SimpleDateSerializer.instance.deserialize(d1),
+                        SimpleDateSerializer.instance.deserialize(d2));
+
+        Long i1 = 0L;
+        Long i2 = (long)Math.pow(2,32) - 1;
+        d1 = SimpleDateType.instance.fromString(i1.toString());
+        d2 = SimpleDateType.instance.fromString(i2.toString());
+        assert SimpleDateType.instance.compare(d1, d2) < 0 :
+            String.format("Failed limits comparison with %s and %s",
+                SimpleDateSerializer.instance.deserialize(d1),
+                SimpleDateSerializer.instance.deserialize(d2));
+
+        d1 = SimpleDateType.instance.fromString("256");
+        d2 = SimpleDateType.instance.fromString("512");
+        assert SimpleDateType.instance.compare(d1, d2) < 0 :
+            String.format("Failed comparison with %s and %s",
+                SimpleDateSerializer.instance.deserialize(d1),
+                SimpleDateSerializer.instance.deserialize(d2));
+
+        d1 = SimpleDateSerializer.instance.serialize(makeUnsigned(0));
+        d2 = SimpleDateSerializer.instance.serialize(makeUnsigned(Integer.MAX_VALUE));
+        assert SimpleDateType.instance.compare(d1, d2) < 0 :
+            String.format("Failed neg/pos comparison with %s and %s",
+                SimpleDateSerializer.instance.deserialize(d1),
+                SimpleDateSerializer.instance.deserialize(d2));
+
+        d1 = SimpleDateType.instance.fromString("-10000-10-10");
+        d2 = SimpleDateType.instance.fromString("10000-10-10");
+        assert SimpleDateType.instance.compare(d1, d2) < 0 :
+            String.format("Failed neg/pos string comparison with %s and %s",
+                SimpleDateSerializer.instance.deserialize(d1),
+                SimpleDateSerializer.instance.deserialize(d2));
+
+        d1 = SimpleDateType.instance.fromString("1969-12-31");
+        d2 = SimpleDateType.instance.fromString("1970-1-1");
+        assert SimpleDateType.instance.compare(d1, d2) < 0 :
+            String.format("Failed pre/post epoch comparison with %s and %s",
+                SimpleDateSerializer.instance.deserialize(d1),
+                SimpleDateSerializer.instance.deserialize(d2));
+
+        d1 = SimpleDateType.instance.fromString("1970-1-1");
+        d2 = SimpleDateType.instance.fromString("1970-1-1");
+        assert SimpleDateType.instance.compare(d1, d2) == 0 :
+            String.format("Failed == date from string comparison with %s and %s",
+                SimpleDateSerializer.instance.deserialize(d1),
+                SimpleDateSerializer.instance.deserialize(d2));
+
+        d1 = SimpleDateType.instance.fromString("1970-1-1");
+        d2 = SimpleDateType.instance.fromString("1970-1-2");
+        assert SimpleDateType.instance.compare(d1, d2) < 0 :
+            String.format("Failed post epoch string comparison with %s and %s",
+                SimpleDateSerializer.instance.deserialize(d1),
+                SimpleDateSerializer.instance.deserialize(d2));
+
+        for (int i = 0; i < 32; ++i)
+        {
+            int offset = (int)Math.pow(2,i);
+            d1 = SimpleDateSerializer.instance.serialize(makeUnsigned(0 - offset));
+            d2 = SimpleDateSerializer.instance.serialize(makeUnsigned(offset));
+            assert SimpleDateType.instance.compare(d1, d2) < 0 :
+                String.format("Failed < comparison of %s and %s",
+                    SimpleDateSerializer.instance.deserialize(d1),
+                    SimpleDateSerializer.instance.deserialize(d2));
+        }
+
+        for (int i = 0; i < 32; ++i)
+        {
+            int offset = (int)Math.pow(2,i);
+            d1 = SimpleDateSerializer.instance.serialize(makeUnsigned(offset));
+            d2 = SimpleDateSerializer.instance.serialize(makeUnsigned(0 - offset));
+            assert SimpleDateType.instance.compare(d1, d2) > 0 :
+                String.format("Failed > comparison of %s and %s",
+                    SimpleDateSerializer.instance.deserialize(d1),
+                    SimpleDateSerializer.instance.deserialize(d2));
+        }
+    }
+
+    private Integer makeUnsigned(int input)
+    {
+        return input - Integer.MIN_VALUE;
+    }
+}
diff --git a/test/unit/org/apache/cassandra/db/marshal/TimeTypeTest.java b/test/unit/org/apache/cassandra/db/marshal/TimeTypeTest.java
new file mode 100644
index 0000000..699c805
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/marshal/TimeTypeTest.java
@@ -0,0 +1,61 @@
+/**
+ * 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.db.marshal;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.TimeUnit;
+
+import org.junit.Test;
+import org.apache.cassandra.serializers.TimeSerializer;
+
+public class TimeTypeTest
+{
+    @Test public void TestComparison()
+    {
+        Long t1 = TimeSerializer.timeStringToLong("01:00:00.123456789");
+        Long t2 = new Long((1L * 60L * 60L * 1000L * 1000L * 1000L) + 123456789);
+        ByteBuffer b1 = TimeSerializer.instance.serialize(t1);
+        ByteBuffer b2 = TimeSerializer.instance.serialize(t2);
+        assert TimeType.instance.compare(b1, b2) == 0 : "Failed == comparison";
+
+        b2 = TimeSerializer.instance.serialize(123456789L);
+        assert TimeType.instance.compare(b1, b2) > 0 : "Failed > comparison";
+
+        t2 = new Long(2L * 60L * 60L * 1000L * 1000L * 1000L + 123456789);
+        b2 = TimeSerializer.instance.serialize(t2);
+        assert TimeType.instance.compare(b1, b2) < 0 : "Failed < comparison";
+
+        b1 = TimeSerializer.instance.serialize(0L);
+        b2 = TimeSerializer.instance.serialize(0L);
+        assert TimeType.instance.compare(b1, b2) == 0 : "Failed == comparison on 0";
+
+        b1 = TimeSerializer.instance.serialize(0L);
+        b2 = TimeSerializer.instance.serialize(10000000L);
+        assert TimeType.instance.compare(b1, b2) < 0 : "Failed < comparison on 0";
+
+        b1 = TimeSerializer.instance.serialize(0L);
+        b2 = TimeSerializer.instance.serialize(TimeUnit.DAYS.toNanos(1));
+        assert TimeType.instance.compare(b1, b2) < 0 : "Failed < comparison against max range.";
+
+        b1 = TimeSerializer.instance.serialize(TimeUnit.DAYS.toNanos(1));
+        b2 = TimeSerializer.instance.serialize(0L);
+        assert TimeType.instance.compare(b1, b2) > 0 : "Failed > comparison against max range.";
+    }
+}
diff --git a/test/unit/org/apache/cassandra/db/marshal/TimeUUIDTypeTest.java b/test/unit/org/apache/cassandra/db/marshal/TimeUUIDTypeTest.java
index 4e22df8..0054163 100644
--- a/test/unit/org/apache/cassandra/db/marshal/TimeUUIDTypeTest.java
+++ b/test/unit/org/apache/cassandra/db/marshal/TimeUUIDTypeTest.java
@@ -22,7 +22,9 @@
 import java.util.Arrays;
 import java.util.Random;
 import java.util.UUID;
+import java.util.concurrent.ThreadLocalRandom;
 
+import junit.framework.Assert;
 import org.apache.cassandra.serializers.MarshalException;
 import org.junit.Test;
 import static org.junit.Assert.assertEquals;
@@ -79,23 +81,74 @@
     @Test
     public void testTimestampComparison()
     {
-        Random rng = new Random();
-        ByteBuffer[] uuids = new ByteBuffer[100];
-        for (int i = 0; i < uuids.length; i++)
+        compareAll(UUIDTypeTest.random(1000, (byte) 0x10));
+        for (ByteBuffer[] permutations : UUIDTypeTest.permutations(100, (byte) 0x10))
+            compareAll(permutations);
+    }
+
+    private void compareAll(ByteBuffer[] uuids)
+    {
+        for (int i = 0 ; i < uuids.length ; i++)
         {
-            uuids[i] = ByteBuffer.allocate(16);
-            rng.nextBytes(uuids[i].array());
-            // set version to 1
-            uuids[i].array()[6] &= 0x0F;
-            uuids[i].array()[6] |= 0x10;
+            for (int j = i + 1 ; j < uuids.length ; j++)
+            {
+                ByteBuffer bi = uuids[i];
+                ByteBuffer bj = uuids[j];
+                long i0 = UUIDGen.getUUID(bi).timestamp();
+                long i1 = UUIDGen.getUUID(bj).timestamp();
+                int c = timeUUIDType.compare(bi, bj);
+                if (i0 == i1) Assert.assertTrue(isComparisonEquivalent(bi.compareTo(bj), c));
+                else Assert.assertTrue(isComparisonEquivalent(Long.compare(i0, i1), c));
+                Assert.assertTrue(isComparisonEquivalent(compareV1(bi, bj), c));
+            }
         }
-        Arrays.sort(uuids, timeUUIDType);
-        for (int i = 1; i < uuids.length; i++)
-        {
-            long i0 = UUIDGen.getUUID(uuids[i - 1]).timestamp();
-            long i1 = UUIDGen.getUUID(uuids[i]).timestamp();
-            assert i0 <= i1;
-        }
+    }
+
+    private static int compareV1(ByteBuffer o1, ByteBuffer o2)
+    {
+        if (!o1.hasRemaining() || !o2.hasRemaining())
+            return o1.hasRemaining() ? 1 : o2.hasRemaining() ? -1 : 0;
+
+        int res = compareTimestampBytes(o1, o2);
+        if (res != 0)
+            return res;
+        return o1.compareTo(o2);
+    }
+
+    private static int compareTimestampBytes(ByteBuffer o1, ByteBuffer o2)
+    {
+        int o1Pos = o1.position();
+        int o2Pos = o2.position();
+
+        int d = (o1.get(o1Pos + 6) & 0xF) - (o2.get(o2Pos + 6) & 0xF);
+        if (d != 0) return d;
+
+        d = (o1.get(o1Pos + 7) & 0xFF) - (o2.get(o2Pos + 7) & 0xFF);
+        if (d != 0) return d;
+
+        d = (o1.get(o1Pos + 4) & 0xFF) - (o2.get(o2Pos + 4) & 0xFF);
+        if (d != 0) return d;
+
+        d = (o1.get(o1Pos + 5) & 0xFF) - (o2.get(o2Pos + 5) & 0xFF);
+        if (d != 0) return d;
+
+        d = (o1.get(o1Pos) & 0xFF) - (o2.get(o2Pos) & 0xFF);
+        if (d != 0) return d;
+
+        d = (o1.get(o1Pos + 1) & 0xFF) - (o2.get(o2Pos + 1) & 0xFF);
+        if (d != 0) return d;
+
+        d = (o1.get(o1Pos + 2) & 0xFF) - (o2.get(o2Pos + 2) & 0xFF);
+        if (d != 0) return d;
+
+        return (o1.get(o1Pos + 3) & 0xFF) - (o2.get(o2Pos + 3) & 0xFF);
+    }
+
+    private static boolean isComparisonEquivalent(int c1, int c2)
+    {
+        c1 = c1 < -1 ? -1 : c1 > 1 ? 1 : c1;
+        c2 = c2 < -1 ? -1 : c2 > 1 ? 1 : c2;
+        return c1 == c2;
     }
 
     @Test
diff --git a/test/unit/org/apache/cassandra/db/marshal/TypeCompareTest.java b/test/unit/org/apache/cassandra/db/marshal/TypeCompareTest.java
index 04b030e..fae04a2 100644
--- a/test/unit/org/apache/cassandra/db/marshal/TypeCompareTest.java
+++ b/test/unit/org/apache/cassandra/db/marshal/TypeCompareTest.java
@@ -93,6 +93,48 @@
     }
 
     @Test
+    public void testByte()
+    {
+        Random rng = new Random();
+        ByteBuffer[] data = new ByteBuffer[Byte.MAX_VALUE];
+        for (int i = 0; i < data.length; i++)
+        {
+            data[i] = ByteBuffer.allocate(1);
+            rng.nextBytes(data[i].array());
+        }
+
+        Arrays.sort(data, ByteType.instance);
+
+        for (int i = 1; i < data.length; i++)
+        {
+            byte b0 = data[i - 1].get(data[i - 1].position());
+            byte b1 = data[i].get(data[i].position());
+            assert b0 <= b1;
+        }
+    }
+
+    @Test
+    public void testShort()
+    {
+        Random rng = new Random();
+        ByteBuffer[] data = new ByteBuffer[1000];
+        for (int i = 0; i < data.length; i++)
+        {
+            data[i] = ByteBuffer.allocate(2);
+            rng.nextBytes(data[i].array());
+        }
+
+        Arrays.sort(data, ShortType.instance);
+
+        for (int i = 1; i < data.length; i++)
+        {
+            short s0 = data[i - 1].getShort(data[i - 1].position());
+            short s1 = data[i].getShort(data[i].position());
+            assert s0 <= s1;
+        }
+    }
+
+    @Test
     public void testInt()
     {
         Random rng = new Random();
diff --git a/test/unit/org/apache/cassandra/db/marshal/TypeParserTest.java b/test/unit/org/apache/cassandra/db/marshal/TypeParserTest.java
index ee3052c..6581fc7 100644
--- a/test/unit/org/apache/cassandra/db/marshal/TypeParserTest.java
+++ b/test/unit/org/apache/cassandra/db/marshal/TypeParserTest.java
@@ -40,6 +40,12 @@
         type = TypeParser.parse("    ");
         assert type == BytesType.instance;
 
+        type = TypeParser.parse("ByteType");
+        assert type == ByteType.instance;
+
+        type = TypeParser.parse("ShortType");
+        assert type == ShortType.instance;
+
         type = TypeParser.parse("LongType");
         assert type == LongType.instance;
 
@@ -69,15 +75,13 @@
             TypeParser.parse("y");
             fail("Should not pass");
         }
-        catch (ConfigurationException e) {}
-        catch (SyntaxException e) {}
+        catch (ConfigurationException | SyntaxException e) {}
 
         try
         {
             TypeParser.parse("LongType(reversed@)");
             fail("Should not pass");
         }
-        catch (ConfigurationException e) {}
-        catch (SyntaxException e) {}
+        catch (ConfigurationException | SyntaxException e) {}
     }
 }
diff --git a/test/unit/org/apache/cassandra/db/marshal/TypeValidationTest.java b/test/unit/org/apache/cassandra/db/marshal/TypeValidationTest.java
index ed5e2bf..5ebeb64 100644
--- a/test/unit/org/apache/cassandra/db/marshal/TypeValidationTest.java
+++ b/test/unit/org/apache/cassandra/db/marshal/TypeValidationTest.java
@@ -65,6 +65,32 @@
     }
 
     @Test
+    public void testValidShort()
+    {
+        ShortType.instance.validate(Util.getBytes((short) 5));
+        ShortType.instance.validate(Util.getBytes(Short.MAX_VALUE));
+    }
+
+    @Test(expected = MarshalException.class)
+    public void testInvalidShort()
+    {
+        ShortType.instance.validate(Util.getBytes(2057022603));
+    }
+
+    @Test
+    public void testValidByte()
+    {
+        ByteType.instance.validate(Util.getBytes((byte) 5));
+        ByteType.instance.validate(Util.getBytes(Byte.MAX_VALUE));
+    }
+
+    @Test(expected = MarshalException.class)
+    public void testInvalidByte()
+    {
+        ByteType.instance.validate(Util.getBytes(2057022603));
+    }
+
+    @Test
     public void testValidUtf8() throws UnsupportedEncodingException
     {
         assert Character.MAX_CODE_POINT == 0x0010ffff;
diff --git a/test/unit/org/apache/cassandra/db/marshal/UUIDTypeTest.java b/test/unit/org/apache/cassandra/db/marshal/UUIDTypeTest.java
index 1ecacf3..335860c 100644
--- a/test/unit/org/apache/cassandra/db/marshal/UUIDTypeTest.java
+++ b/test/unit/org/apache/cassandra/db/marshal/UUIDTypeTest.java
@@ -24,15 +24,17 @@
 import static org.junit.Assert.assertEquals;
 
 import java.nio.ByteBuffer;
-import java.util.Arrays;
+import java.util.Iterator;
 import java.util.Random;
 import java.util.UUID;
 
+import org.junit.Test;
+
+import junit.framework.Assert;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.UUIDGen;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.junit.Test;
 
 public class UUIDTypeTest
 {
@@ -42,7 +44,7 @@
     UUIDType uuidType = new UUIDType();
 
     @Test
-    public void testCompare()
+    public void testRandomCompare()
     {
 
         UUID t1 = UUIDGen.getTimeUUID();
@@ -201,24 +203,255 @@
     }
 
     @Test
-    public void testTimestampComparison()
+    public void testPermutations()
     {
-        Random rng = new Random();
-        ByteBuffer[] uuids = new ByteBuffer[100];
-        for (int i = 0; i < uuids.length; i++)
+        compareAll(random(1000, (byte) 0x00, (byte) 0x10, (byte) 0x20));
+        for (ByteBuffer[] permutations : permutations(10,  (byte) 0x00, (byte) 0x10, (byte) 0x20))
+            compareAll(permutations);
+    }
+
+    private void compareAll(ByteBuffer[] uuids)
+    {
+        for (int i = 0 ; i < uuids.length ; i++)
         {
-            uuids[i] = ByteBuffer.allocate(16);
-            rng.nextBytes(uuids[i].array());
-            // set version to 1
-            uuids[i].array()[6] &= 0x0F;
-            uuids[i].array()[6] |= 0x10;
+            for (int j = i + 1 ; j < uuids.length ; j++)
+            {
+                ByteBuffer bi = uuids[i];
+                ByteBuffer bj = uuids[j];
+                UUID ui = UUIDGen.getUUID(bi);
+                UUID uj = UUIDGen.getUUID(bj);
+                int c = uuidType.compare(bi, bj);
+                if (ui.version() != uj.version())
+                {
+                    Assert.assertTrue(isComparisonEquivalent(ui.version() - uj.version(), c));
+                }
+                else if (ui.version() == 1)
+                {
+                    long i0 = ui.timestamp();
+                    long i1 = uj.timestamp();
+                    if (i0 == i1) Assert.assertTrue(isComparisonEquivalent(ByteBufferUtil.compareUnsigned(bi, bj), c));
+                    else Assert.assertTrue(isComparisonEquivalent(Long.compare(i0, i1), c));
+                }
+                else
+                {
+                    Assert.assertTrue(isComparisonEquivalent(ByteBufferUtil.compareUnsigned(bi, bj), c));
+                }
+                Assert.assertTrue(isComparisonEquivalent(compareV1(bi, bj), c));
+            }
         }
-        Arrays.sort(uuids, uuidType);
-        for (int i = 1; i < uuids.length; i++)
+    }
+
+    private static boolean isComparisonEquivalent(int c1, int c2)
+    {
+        c1 = c1 < -1 ? -1 : c1 > 1 ? 1 : c1;
+        c2 = c2 < -1 ? -1 : c2 > 1 ? 1 : c2;
+        return c1 == c2;
+    }
+
+    // produce randomCount random byte strings, and permute every possible byte within each
+    // for all provided types, using permute()
+    static Iterable<ByteBuffer[]> permutations(final int randomCount, final byte ... types)
+    {
+        final Random random = new Random();
+        long seed = random.nextLong();
+        random.setSeed(seed);
+        System.out.println("UUIDTypeTest.permutations.seed=" + seed);
+        return new Iterable<ByteBuffer[]>()
         {
-            long i0 = UUIDGen.getUUID(uuids[i - 1]).timestamp();
-            long i1 = UUIDGen.getUUID(uuids[i]).timestamp();
-            assert i0 <= i1;
+            public Iterator<ByteBuffer[]> iterator()
+            {
+                return new Iterator<ByteBuffer[]>()
+                {
+                    byte[] bytes = new byte[16];
+                    int c = -1, i = 16;
+                    public boolean hasNext()
+                    {
+                        return i < 16 || c < randomCount - 1;
+                    }
+
+                    public ByteBuffer[] next()
+                    {
+                        if (i == 16)
+                        {
+                            random.nextBytes(bytes);
+                            i = 0;
+                            c++;
+                        }
+                        return permute(bytes, i++, types);
+                    }
+                    public void remove()
+                    {
+                    }
+                };
+            }
+        };
+    }
+
+    // for each of the given UUID types provided, produce every possible
+    // permutation of the provided byte[] for the given index
+    static ByteBuffer[] permute(byte[] src, int byteIndex, byte ... types)
+    {
+        assert src.length == 16;
+        assert byteIndex < 16;
+        byte[] bytes = src.clone();
+        ByteBuffer[] permute;
+        if (byteIndex == 6)
+        {
+            permute = new ByteBuffer[16 * types.length];
+            for (int i = 0 ; i < types.length ; i++)
+            {
+                for (int j = 0 ; j < 16 ; j++)
+                {
+                    int k = i * 16 + j;
+                    bytes[6] = (byte)(types[i] | j);
+                    permute[k] = ByteBuffer.wrap(bytes.clone());
+                }
+            }
         }
+        else
+        {
+            permute = new ByteBuffer[256 * types.length];
+            for (int i = 0 ; i < types.length ; i++)
+            {
+                bytes[6] = types[i];
+                for (int j = 0 ; j < 256 ; j++)
+                {
+                    int k = i * 256 + j;
+                    bytes[byteIndex] = (byte) ((bytes[byteIndex] & 0x0F) | i);
+                    permute[k] = ByteBuffer.wrap(bytes.clone());
+                }
+            }
+        }
+        return permute;
+    }
+
+    static ByteBuffer[] random(int count, byte ... types)
+    {
+        Random random = new Random();
+        long seed = random.nextLong();
+        random.setSeed(seed);
+        System.out.println("UUIDTypeTest.random.seed=" + seed);
+        ByteBuffer[] uuids = new ByteBuffer[count * types.length];
+        for (int i = 0 ; i < types.length ; i++)
+        {
+            for (int j = 0; j < count; j++)
+            {
+                int k = (i * count) + j;
+                uuids[k] = ByteBuffer.allocate(16);
+                random.nextBytes(uuids[k].array());
+                // set version to 1
+                uuids[k].array()[6] &= 0x0F;
+                uuids[k].array()[6] |= types[i];
+            }
+        }
+        return uuids;
+    }
+
+    private static int compareV1(ByteBuffer b1, ByteBuffer b2)
+    {
+
+        // Compare for length
+
+        if ((b1 == null) || (b1.remaining() < 16))
+        {
+            return ((b2 == null) || (b2.remaining() < 16)) ? 0 : -1;
+        }
+        if ((b2 == null) || (b2.remaining() < 16))
+        {
+            return 1;
+        }
+
+        int s1 = b1.position();
+        int s2 = b2.position();
+
+        // Compare versions
+
+        int v1 = (b1.get(s1 + 6) >> 4) & 0x0f;
+        int v2 = (b2.get(s2 + 6) >> 4) & 0x0f;
+
+        if (v1 != v2)
+        {
+            return v1 - v2;
+        }
+
+        // Compare timestamps for version 1
+
+        if (v1 == 1)
+        {
+            // if both time-based, compare as timestamps
+            int c = compareTimestampBytes(b1, b2);
+            if (c != 0)
+            {
+                return c;
+            }
+        }
+
+        // Compare the two byte arrays starting from the first
+        // byte in the sequence until an inequality is
+        // found. This should provide equivalent results
+        // to the comparison performed by the RFC 4122
+        // Appendix A - Sample Implementation.
+        // Note: java.util.UUID.compareTo is not a lexical
+        // comparison
+        for (int i = 0; i < 16; i++)
+        {
+            int c = ((b1.get(s1 + i)) & 0xFF) - ((b2.get(s2 + i)) & 0xFF);
+            if (c != 0)
+            {
+                return c;
+            }
+        }
+
+        return 0;
+    }
+
+    private static int compareTimestampBytes(ByteBuffer o1, ByteBuffer o2)
+    {
+        int o1Pos = o1.position();
+        int o2Pos = o2.position();
+
+        int d = (o1.get(o1Pos + 6) & 0xF) - (o2.get(o2Pos + 6) & 0xF);
+        if (d != 0)
+        {
+            return d;
+        }
+
+        d = (o1.get(o1Pos + 7) & 0xFF) - (o2.get(o2Pos + 7) & 0xFF);
+        if (d != 0)
+        {
+            return d;
+        }
+
+        d = (o1.get(o1Pos + 4) & 0xFF) - (o2.get(o2Pos + 4) & 0xFF);
+        if (d != 0)
+        {
+            return d;
+        }
+
+        d = (o1.get(o1Pos + 5) & 0xFF) - (o2.get(o2Pos + 5) & 0xFF);
+        if (d != 0)
+        {
+            return d;
+        }
+
+        d = (o1.get(o1Pos) & 0xFF) - (o2.get(o2Pos) & 0xFF);
+        if (d != 0)
+        {
+            return d;
+        }
+
+        d = (o1.get(o1Pos + 1) & 0xFF) - (o2.get(o2Pos + 1) & 0xFF);
+        if (d != 0)
+        {
+            return d;
+        }
+
+        d = (o1.get(o1Pos + 2) & 0xFF) - (o2.get(o2Pos + 2) & 0xFF);
+        if (d != 0)
+        {
+            return d;
+        }
+
+        return (o1.get(o1Pos + 3) & 0xFF) - (o2.get(o2Pos + 3) & 0xFF);
     }
 }
diff --git a/test/unit/org/apache/cassandra/dht/BootStrapperTest.java b/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
index 471bc4b..ababd99 100644
--- a/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
+++ b/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
@@ -25,13 +25,16 @@
 import java.util.Set;
 import java.util.Map;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 
 import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.gms.IFailureDetectionEventListener;
 import org.apache.cassandra.gms.IFailureDetector;
 import org.apache.cassandra.locator.TokenMetadata;
@@ -40,8 +43,16 @@
 import static org.junit.Assert.*;
 
 @RunWith(OrderedJUnit4ClassRunner.class)
-public class BootStrapperTest extends SchemaLoader
+public class BootStrapperTest
 {
+    @BeforeClass
+    public static void setup() throws ConfigurationException
+    {
+        SchemaLoader.startGossiper();
+        SchemaLoader.prepareServer();
+        SchemaLoader.schemaDefinition("BootStrapperTest");
+    }
+
     @Test
     public void testSourceTargetComputation() throws UnknownHostException
     {
@@ -65,7 +76,7 @@
 
         TokenMetadata tmd = ss.getTokenMetadata();
         assertEquals(numOldNodes, tmd.sortedTokens().size());
-        RangeStreamer s = new RangeStreamer(tmd, myEndpoint, "Bootstrap");
+        RangeStreamer s = new RangeStreamer(tmd, null, myEndpoint, "Bootstrap", true, DatabaseDescriptor.getEndpointSnitch(), new StreamStateStore());
         IFailureDetector mockFailureDetector = new IFailureDetector()
         {
             public boolean isAlive(InetAddress ep)
@@ -86,7 +97,7 @@
         Collection<Map.Entry<InetAddress, Collection<Range<Token>>>> toFetch = s.toFetch().get(keyspaceName);
 
         // Check we get get RF new ranges in total
-        Set<Range<Token>> ranges = new HashSet<Range<Token>>();
+        Set<Range<Token>> ranges = new HashSet<>();
         for (Map.Entry<InetAddress, Collection<Range<Token>>> e : toFetch)
             ranges.addAll(e.getValue());
 
diff --git a/test/unit/org/apache/cassandra/dht/BoundsTest.java b/test/unit/org/apache/cassandra/dht/BoundsTest.java
index 527b498..2ac06d9 100644
--- a/test/unit/org/apache/cassandra/dht/BoundsTest.java
+++ b/test/unit/org/apache/cassandra/dht/BoundsTest.java
@@ -32,7 +32,7 @@
 
     private Bounds<Token> bounds(long left, long right)
     {
-        return new Bounds<Token>(new LongToken(left), new LongToken(right));
+        return new Bounds<Token>(new Murmur3Partitioner.LongToken(left), new Murmur3Partitioner.LongToken(right));
     }
 
     @Test
diff --git a/test/unit/org/apache/cassandra/dht/ByteOrderedPartitionerTest.java b/test/unit/org/apache/cassandra/dht/ByteOrderedPartitionerTest.java
index cfe5f52..e70e086 100644
--- a/test/unit/org/apache/cassandra/dht/ByteOrderedPartitionerTest.java
+++ b/test/unit/org/apache/cassandra/dht/ByteOrderedPartitionerTest.java
@@ -22,6 +22,6 @@
 {
     public void initPartitioner()
     {
-        partitioner = new ByteOrderedPartitioner();
+        partitioner = ByteOrderedPartitioner.instance;
     }
 }
diff --git a/test/unit/org/apache/cassandra/dht/KeyCollisionTest.java b/test/unit/org/apache/cassandra/dht/KeyCollisionTest.java
index e143f30..e8a5ee2 100644
--- a/test/unit/org/apache/cassandra/dht/KeyCollisionTest.java
+++ b/test/unit/org/apache/cassandra/dht/KeyCollisionTest.java
@@ -22,6 +22,8 @@
 import java.nio.ByteBuffer;
 import java.util.*;
 
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
@@ -30,6 +32,8 @@
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.IntegerType;
 import org.apache.cassandra.config.*;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.*;
 
@@ -43,19 +47,26 @@
  * length partitioner that takes the length of the key as token, making
  * collision easy and predictable.
  */
-public class KeyCollisionTest extends SchemaLoader
+public class KeyCollisionTest
 {
-    IPartitioner oldPartitioner;
-    private static final String KEYSPACE = "Keyspace1";
+    static IPartitioner oldPartitioner;
+    private static final String KEYSPACE1 = "KeyCollisionTest1";
     private static final String CF = "Standard1";
 
-    protected void setUp()
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
     {
         oldPartitioner = DatabaseDescriptor.getPartitioner();
-        DatabaseDescriptor.setPartitioner(new LengthPartitioner());
+        DatabaseDescriptor.setPartitioner(LengthPartitioner.instance);
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF));
     }
 
-    protected void tearDown()
+    @AfterClass
+    public static void tearDown()
     {
         DatabaseDescriptor.setPartitioner(oldPartitioner);
     }
@@ -63,18 +74,18 @@
     @Test
     public void testGetSliceWithCollision() throws Exception
     {
-        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
         cfs.clearUnsafe();
 
-        insert("k1", "k2", "k3");       // token = 2
+        insert("k1", "k2", "kq");       // token = 2, kq ordered after row below lexicographically
         insert("key1", "key2", "key3"); // token = 4
         insert("longKey1", "longKey2"); // token = 8
 
         List<Row> rows = cfs.getRangeSlice(new Bounds<RowPosition>(dk("k2"), dk("key2")), null, new IdentityQueryFilter(), 10000);
         assert rows.size() == 4 : "Expecting 4 keys, got " + rows.size();
         assert rows.get(0).key.getKey().equals(ByteBufferUtil.bytes("k2"));
-        assert rows.get(1).key.getKey().equals(ByteBufferUtil.bytes("k3"));
+        assert rows.get(1).key.getKey().equals(ByteBufferUtil.bytes("kq"));
         assert rows.get(2).key.getKey().equals(ByteBufferUtil.bytes("key1"));
         assert rows.get(3).key.getKey().equals(ByteBufferUtil.bytes("key2"));
     }
@@ -88,16 +99,45 @@
     private void insert(String key)
     {
         Mutation rm;
-        rm = new Mutation(KEYSPACE, ByteBufferUtil.bytes(key));
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes(key));
         rm.add(CF, Util.cellname("column"), ByteBufferUtil.bytes("asdf"), 0);
-        rm.apply();
+        rm.applyUnsafe();
     }
 
-    public static class LengthPartitioner extends AbstractPartitioner
+    static class BigIntegerToken extends ComparableObjectToken<BigInteger>
+    {
+        private static final long serialVersionUID = 1L;
+
+        public BigIntegerToken(BigInteger token)
+        {
+            super(token);
+        }
+
+        // convenience method for testing
+        public BigIntegerToken(String token) {
+            this(new BigInteger(token));
+        }
+
+        @Override
+        public IPartitioner getPartitioner()
+        {
+            return LengthPartitioner.instance;
+        }
+
+        @Override
+        public long getHeapSize()
+        {
+            return 0;
+        }
+    }
+
+    public static class LengthPartitioner implements IPartitioner
     {
         public static final BigInteger ZERO = new BigInteger("0");
         public static final BigIntegerToken MINIMUM = new BigIntegerToken("-1");
 
+        public static LengthPartitioner instance = new LengthPartitioner();
+
         public DecoratedKey decorateKey(ByteBuffer key)
         {
             return new BufferDecoratedKey(getToken(key), key);
@@ -166,12 +206,6 @@
             return new BigIntegerToken(BigInteger.valueOf(key.remaining()));
         }
 
-        @Override
-        public long getHeapSizeOf(Token token)
-        {
-            return 0;
-        }
-
         public Map<Token, Float> describeOwnership(List<Token> sortedTokens)
         {
             // allTokens will contain the count and be returned, sorted_ranges is shorthand for token<->token math.
diff --git a/test/unit/org/apache/cassandra/dht/Murmur3PartitionerTest.java b/test/unit/org/apache/cassandra/dht/Murmur3PartitionerTest.java
index 71c7e89..9f330d3 100644
--- a/test/unit/org/apache/cassandra/dht/Murmur3PartitionerTest.java
+++ b/test/unit/org/apache/cassandra/dht/Murmur3PartitionerTest.java
@@ -24,7 +24,7 @@
 {
     public void initPartitioner()
     {
-        partitioner = new Murmur3Partitioner();
+        partitioner = Murmur3Partitioner.instance;
     }
 
     @Override
diff --git a/test/unit/org/apache/cassandra/dht/OrderPreservingPartitionerTest.java b/test/unit/org/apache/cassandra/dht/OrderPreservingPartitionerTest.java
index aa43265..0449258 100644
--- a/test/unit/org/apache/cassandra/dht/OrderPreservingPartitionerTest.java
+++ b/test/unit/org/apache/cassandra/dht/OrderPreservingPartitionerTest.java
@@ -18,6 +18,8 @@
 */
 package org.apache.cassandra.dht;
 
+import java.io.IOException;
+
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -26,7 +28,7 @@
 public class OrderPreservingPartitionerTest extends PartitionerTestCase
 {
     @BeforeClass
-    public static void cleanStatesFromPreviousTest()
+    public static void cleanStatesFromPreviousTest() throws IOException
     {
         // Since OrderPreservingPartitioner#describeOwnership tries to read SSTables,
         // we need to clear data dir to clear garbage from previous test before running tests.
@@ -35,7 +37,7 @@
 
     public void initPartitioner()
     {
-        partitioner = new OrderPreservingPartitioner();
+        partitioner = OrderPreservingPartitioner.instance;
     }
 
     @Test
diff --git a/test/unit/org/apache/cassandra/dht/PartitionerTestCase.java b/test/unit/org/apache/cassandra/dht/PartitionerTestCase.java
index db0349d..8080a0c 100644
--- a/test/unit/org/apache/cassandra/dht/PartitionerTestCase.java
+++ b/test/unit/org/apache/cassandra/dht/PartitionerTestCase.java
@@ -21,6 +21,7 @@
 import java.nio.ByteBuffer;
 import java.util.*;
 
+import org.apache.cassandra.service.StorageService;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -122,6 +123,10 @@
     @Test
     public void testDescribeOwnership()
     {
+        // This call initializes StorageService, needed to populate the keyspaces.
+        // TODO: This points to potential problems in the initialization sequence. Should be solved by CASSANDRA-7837.
+        StorageService.getPartitioner();
+
         try
         {
             testDescribeOwnershipWith(0);
diff --git a/test/unit/org/apache/cassandra/dht/RandomPartitionerTest.java b/test/unit/org/apache/cassandra/dht/RandomPartitionerTest.java
index 09a1d65..6b22617 100644
--- a/test/unit/org/apache/cassandra/dht/RandomPartitionerTest.java
+++ b/test/unit/org/apache/cassandra/dht/RandomPartitionerTest.java
@@ -24,6 +24,6 @@
 {
     public void initPartitioner()
     {
-        partitioner = new RandomPartitioner();
+        partitioner = RandomPartitioner.instance;
     }
 }
diff --git a/test/unit/org/apache/cassandra/dht/RangeTest.java b/test/unit/org/apache/cassandra/dht/RangeTest.java
index 2083f53..9fb49cf 100644
--- a/test/unit/org/apache/cassandra/dht/RangeTest.java
+++ b/test/unit/org/apache/cassandra/dht/RangeTest.java
@@ -19,7 +19,13 @@
 package org.apache.cassandra.dht;
 
 import java.nio.ByteBuffer;
-import java.util.*;
+import java.util.Collection;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
 
 import com.google.common.base.Joiner;
 
@@ -28,8 +34,10 @@
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.junit.Test;
-
 import org.apache.cassandra.db.RowPosition;
+import org.apache.cassandra.dht.RandomPartitioner.BigIntegerToken;
+import org.apache.cassandra.dht.ByteOrderedPartitioner.BytesToken;
+
 import static org.apache.cassandra.Util.range;
 import static org.junit.Assert.*;
 
@@ -39,7 +47,7 @@
     @Test
     public void testContains()
     {
-        Range left = new Range(new BigIntegerToken("0"), new BigIntegerToken("100"));
+        Range<Token> left = new Range<Token>(new BigIntegerToken("0"), new BigIntegerToken("100"));
         assert !left.contains(new BigIntegerToken("0"));
         assert left.contains(new BigIntegerToken("10"));
         assert left.contains(new BigIntegerToken("100"));
@@ -49,13 +57,13 @@
     @Test
     public void testContainsWrapping()
     {
-        Range range = new Range(new BigIntegerToken("0"), new BigIntegerToken("0"));
+        Range<Token> range = new Range<Token>(new BigIntegerToken("0"), new BigIntegerToken("0"));
         assert range.contains(new BigIntegerToken("0"));
         assert range.contains(new BigIntegerToken("10"));
         assert range.contains(new BigIntegerToken("100"));
         assert range.contains(new BigIntegerToken("101"));
 
-        range = new Range(new BigIntegerToken("100"), new BigIntegerToken("0"));
+        range = new Range<Token>(new BigIntegerToken("100"), new BigIntegerToken("0"));
         assert range.contains(new BigIntegerToken("0"));
         assert !range.contains(new BigIntegerToken("1"));
         assert !range.contains(new BigIntegerToken("100"));
@@ -65,10 +73,10 @@
     @Test
     public void testContainsRange()
     {
-        Range one = new Range(new BigIntegerToken("2"), new BigIntegerToken("10"));
-        Range two = new Range(new BigIntegerToken("2"), new BigIntegerToken("5"));
-        Range thr = new Range(new BigIntegerToken("5"), new BigIntegerToken("10"));
-        Range fou = new Range(new BigIntegerToken("10"), new BigIntegerToken("12"));
+        Range<Token> one = new Range<Token>(new BigIntegerToken("2"), new BigIntegerToken("10"));
+        Range<Token> two = new Range<Token>(new BigIntegerToken("2"), new BigIntegerToken("5"));
+        Range<Token> thr = new Range<Token>(new BigIntegerToken("5"), new BigIntegerToken("10"));
+        Range<Token> fou = new Range<Token>(new BigIntegerToken("10"), new BigIntegerToken("12"));
 
         assert one.contains(two);
         assert one.contains(thr);
@@ -90,11 +98,11 @@
     @Test
     public void testContainsRangeWrapping()
     {
-        Range one = new Range(new BigIntegerToken("10"), new BigIntegerToken("2"));
-        Range two = new Range(new BigIntegerToken("5"), new BigIntegerToken("3"));
-        Range thr = new Range(new BigIntegerToken("10"), new BigIntegerToken("12"));
-        Range fou = new Range(new BigIntegerToken("2"), new BigIntegerToken("6"));
-        Range fiv = new Range(new BigIntegerToken("0"), new BigIntegerToken("0"));
+        Range<Token> one = new Range<Token>(new BigIntegerToken("10"), new BigIntegerToken("2"));
+        Range<Token> two = new Range<Token>(new BigIntegerToken("5"), new BigIntegerToken("3"));
+        Range<Token> thr = new Range<Token>(new BigIntegerToken("10"), new BigIntegerToken("12"));
+        Range<Token> fou = new Range<Token>(new BigIntegerToken("2"), new BigIntegerToken("6"));
+        Range<Token> fiv = new Range<Token>(new BigIntegerToken("0"), new BigIntegerToken("0"));
 
         assert !one.contains(two);
         assert one.contains(thr);
@@ -121,12 +129,12 @@
     @Test
     public void testContainsRangeOneWrapping()
     {
-        Range wrap1 = new Range(new BigIntegerToken("0"), new BigIntegerToken("0"));
-        Range wrap2 = new Range(new BigIntegerToken("10"), new BigIntegerToken("2"));
+        Range<Token> wrap1 = new Range<Token>(new BigIntegerToken("0"), new BigIntegerToken("0"));
+        Range<Token> wrap2 = new Range<Token>(new BigIntegerToken("10"), new BigIntegerToken("2"));
 
-        Range nowrap1 = new Range(new BigIntegerToken("0"), new BigIntegerToken("2"));
-        Range nowrap2 = new Range(new BigIntegerToken("2"), new BigIntegerToken("10"));
-        Range nowrap3 = new Range(new BigIntegerToken("10"), new BigIntegerToken("100"));
+        Range<Token> nowrap1 = new Range<Token>(new BigIntegerToken("0"), new BigIntegerToken("2"));
+        Range<Token> nowrap2 = new Range<Token>(new BigIntegerToken("2"), new BigIntegerToken("10"));
+        Range<Token> nowrap3 = new Range<Token>(new BigIntegerToken("10"), new BigIntegerToken("100"));
 
         assert wrap1.contains(nowrap1);
         assert wrap1.contains(nowrap2);
@@ -140,10 +148,10 @@
     @Test
     public void testIntersects()
     {
-        Range all = new Range(new BigIntegerToken("0"), new BigIntegerToken("0")); // technically, this is a wrapping range
-        Range one = new Range(new BigIntegerToken("2"), new BigIntegerToken("10"));
-        Range two = new Range(new BigIntegerToken("0"), new BigIntegerToken("8"));
-        Range not = new Range(new BigIntegerToken("10"), new BigIntegerToken("12"));
+        Range<Token> all = new Range<Token>(new BigIntegerToken("0"), new BigIntegerToken("0")); // technically, this is a wrapping range
+        Range<Token> one = new Range<Token>(new BigIntegerToken("2"), new BigIntegerToken("10"));
+        Range<Token> two = new Range<Token>(new BigIntegerToken("0"), new BigIntegerToken("8"));
+        Range<Token> not = new Range<Token>(new BigIntegerToken("10"), new BigIntegerToken("12"));
 
         assert all.intersects(one);
         assert all.intersects(two);
@@ -161,12 +169,12 @@
     @Test
     public void testIntersectsWrapping()
     {
-        Range onewrap = new Range(new BigIntegerToken("10"), new BigIntegerToken("2"));
-        Range onecomplement = new Range(onewrap.right, onewrap.left);
-        Range onestartswith = new Range(onewrap.left, new BigIntegerToken("12"));
-        Range oneendswith = new Range(new BigIntegerToken("1"), onewrap.right);
-        Range twowrap = new Range(new BigIntegerToken("5"), new BigIntegerToken("3"));
-        Range not = new Range(new BigIntegerToken("2"), new BigIntegerToken("6"));
+        Range<Token> onewrap = new Range<Token>(new BigIntegerToken("10"), new BigIntegerToken("2"));
+        Range<Token> onecomplement = new Range<Token>(onewrap.right, onewrap.left);
+        Range<Token> onestartswith = new Range<Token>(onewrap.left, new BigIntegerToken("12"));
+        Range<Token> oneendswith = new Range<Token>(new BigIntegerToken("1"), onewrap.right);
+        Range<Token> twowrap = new Range<Token>(new BigIntegerToken("5"), new BigIntegerToken("3"));
+        Range<Token> not = new Range<Token>(new BigIntegerToken("2"), new BigIntegerToken("6"));
 
         assert !onewrap.intersects(onecomplement);
         assert onewrap.intersects(onestartswith);
@@ -182,20 +190,21 @@
         assert not.intersects(twowrap);
     }
 
-    static <T extends RingPosition<T>> void assertIntersection(Range one, Range two, Range<T> ... ranges)
+    @SafeVarargs
+    static <T extends RingPosition<T>> void assertIntersection(Range<T> one, Range<T> two, Range<T> ... ranges)
     {
         Set<Range<T>> correct = Range.rangeSet(ranges);
-        Set<Range> result1 = one.intersectionWith(two);
+        Set<Range<T>> result1 = one.intersectionWith(two);
         assert result1.equals(correct) : String.format("%s != %s",
-                StringUtils.join(result1, ","),
-                StringUtils.join(correct, ","));
-        Set<Range> result2 = two.intersectionWith(one);
+                                                       StringUtils.join(result1, ","),
+                                                       StringUtils.join(correct, ","));
+        Set<Range<T>> result2 = two.intersectionWith(one);
         assert result2.equals(correct) : String.format("%s != %s",
-                StringUtils.join(result2, ","),
-                StringUtils.join(correct, ","));
+                                                       StringUtils.join(result2, ","),
+                                                       StringUtils.join(correct, ","));
     }
 
-    private void assertNoIntersection(Range wraps1, Range nowrap3)
+    private void assertNoIntersection(Range<Token> wraps1, Range<Token> nowrap3)
     {
         assertIntersection(wraps1, nowrap3);
     }
@@ -203,11 +212,11 @@
     @Test
     public void testIntersectionWithAll()
     {
-        Range all0 = new Range(new BigIntegerToken("0"), new BigIntegerToken("0"));
-        Range all10 = new Range(new BigIntegerToken("10"), new BigIntegerToken("10"));
-        Range all100 = new Range(new BigIntegerToken("100"), new BigIntegerToken("100"));
-        Range all1000 = new Range(new BigIntegerToken("1000"), new BigIntegerToken("1000"));
-        Range wraps = new Range(new BigIntegerToken("100"), new BigIntegerToken("10"));
+        Range<Token> all0 = new Range<Token>(new BigIntegerToken("0"), new BigIntegerToken("0"));
+        Range<Token> all10 = new Range<Token>(new BigIntegerToken("10"), new BigIntegerToken("10"));
+        Range<Token> all100 = new Range<Token>(new BigIntegerToken("100"), new BigIntegerToken("100"));
+        Range<Token> all1000 = new Range<Token>(new BigIntegerToken("1000"), new BigIntegerToken("1000"));
+        Range<Token> wraps = new Range<Token>(new BigIntegerToken("100"), new BigIntegerToken("10"));
 
         assertIntersection(all0, wraps, wraps);
         assertIntersection(all10, wraps, wraps);
@@ -218,12 +227,12 @@
     @Test
     public void testIntersectionContains()
     {
-        Range wraps1 = new Range(new BigIntegerToken("100"), new BigIntegerToken("10"));
-        Range wraps2 = new Range(new BigIntegerToken("90"), new BigIntegerToken("20"));
-        Range wraps3 = new Range(new BigIntegerToken("90"), new BigIntegerToken("0"));
-        Range nowrap1 = new Range(new BigIntegerToken("100"), new BigIntegerToken("110"));
-        Range nowrap2 = new Range(new BigIntegerToken("0"), new BigIntegerToken("10"));
-        Range nowrap3 = new Range(new BigIntegerToken("0"), new BigIntegerToken("9"));
+        Range<Token> wraps1 = new Range<Token>(new BigIntegerToken("100"), new BigIntegerToken("10"));
+        Range<Token> wraps2 = new Range<Token>(new BigIntegerToken("90"), new BigIntegerToken("20"));
+        Range<Token> wraps3 = new Range<Token>(new BigIntegerToken("90"), new BigIntegerToken("0"));
+        Range<Token> nowrap1 = new Range<Token>(new BigIntegerToken("100"), new BigIntegerToken("110"));
+        Range<Token> nowrap2 = new Range<Token>(new BigIntegerToken("0"), new BigIntegerToken("10"));
+        Range<Token> nowrap3 = new Range<Token>(new BigIntegerToken("0"), new BigIntegerToken("9"));
 
         assertIntersection(wraps1, wraps2, wraps1);
         assertIntersection(wraps3, wraps2, wraps3);
@@ -241,11 +250,11 @@
     @Test
     public void testNoIntersection()
     {
-        Range wraps1 = new Range(new BigIntegerToken("100"), new BigIntegerToken("10"));
-        Range wraps2 = new Range(new BigIntegerToken("100"), new BigIntegerToken("0"));
-        Range nowrap1 = new Range(new BigIntegerToken("0"), new BigIntegerToken("100"));
-        Range nowrap2 = new Range(new BigIntegerToken("100"), new BigIntegerToken("200"));
-        Range nowrap3 = new Range(new BigIntegerToken("10"), new BigIntegerToken("100"));
+        Range<Token> wraps1 = new Range<Token>(new BigIntegerToken("100"), new BigIntegerToken("10"));
+        Range<Token> wraps2 = new Range<Token>(new BigIntegerToken("100"), new BigIntegerToken("0"));
+        Range<Token> nowrap1 = new Range<Token>(new BigIntegerToken("0"), new BigIntegerToken("100"));
+        Range<Token> nowrap2 = new Range<Token>(new BigIntegerToken("100"), new BigIntegerToken("200"));
+        Range<Token> nowrap3 = new Range<Token>(new BigIntegerToken("10"), new BigIntegerToken("100"));
 
         assertNoIntersection(wraps1, nowrap3);
         assertNoIntersection(wraps2, nowrap1);
@@ -255,51 +264,51 @@
     @Test
     public void testIntersectionOneWraps()
     {
-        Range wraps1 = new Range(new BigIntegerToken("100"), new BigIntegerToken("10"));
-        Range wraps2 = new Range(new BigIntegerToken("100"), new BigIntegerToken("0"));
-        Range nowrap1 = new Range(new BigIntegerToken("0"), new BigIntegerToken("200"));
-        Range nowrap2 = new Range(new BigIntegerToken("0"), new BigIntegerToken("100"));
+        Range<Token> wraps1 = new Range<Token>(new BigIntegerToken("100"), new BigIntegerToken("10"));
+        Range<Token> wraps2 = new Range<Token>(new BigIntegerToken("100"), new BigIntegerToken("0"));
+        Range<Token> nowrap1 = new Range<Token>(new BigIntegerToken("0"), new BigIntegerToken("200"));
+        Range<Token> nowrap2 = new Range<Token>(new BigIntegerToken("0"), new BigIntegerToken("100"));
 
         assertIntersection(wraps1,
-                nowrap1,
-                new Range(new BigIntegerToken("0"), new BigIntegerToken("10")),
-                new Range(new BigIntegerToken("100"), new BigIntegerToken("200")));
+                           nowrap1,
+                           new Range<Token>(new BigIntegerToken("0"), new BigIntegerToken("10")),
+                           new Range<Token>(new BigIntegerToken("100"), new BigIntegerToken("200")));
         assertIntersection(wraps2,
-                nowrap1,
-                new Range(new BigIntegerToken("100"), new BigIntegerToken("200")));
+                           nowrap1,
+                           new Range<Token>(new BigIntegerToken("100"), new BigIntegerToken("200")));
         assertIntersection(wraps1,
-                nowrap2,
-                new Range(new BigIntegerToken("0"), new BigIntegerToken("10")));
+                           nowrap2,
+                           new Range<Token>(new BigIntegerToken("0"), new BigIntegerToken("10")));
     }
 
     @Test
     public void testIntersectionTwoWraps()
     {
-        Range wraps1 = new Range(new BigIntegerToken("100"), new BigIntegerToken("20"));
-        Range wraps2 = new Range(new BigIntegerToken("120"), new BigIntegerToken("90"));
-        Range wraps3 = new Range(new BigIntegerToken("120"), new BigIntegerToken("110"));
-        Range wraps4 = new Range(new BigIntegerToken("10"), new BigIntegerToken("0"));
-        Range wraps5 = new Range(new BigIntegerToken("10"), new BigIntegerToken("1"));
-        Range wraps6 = new Range(new BigIntegerToken("30"), new BigIntegerToken("10"));
+        Range<Token> wraps1 = new Range<Token>(new BigIntegerToken("100"), new BigIntegerToken("20"));
+        Range<Token> wraps2 = new Range<Token>(new BigIntegerToken("120"), new BigIntegerToken("90"));
+        Range<Token> wraps3 = new Range<Token>(new BigIntegerToken("120"), new BigIntegerToken("110"));
+        Range<Token> wraps4 = new Range<Token>(new BigIntegerToken("10"), new BigIntegerToken("0"));
+        Range<Token> wraps5 = new Range<Token>(new BigIntegerToken("10"), new BigIntegerToken("1"));
+        Range<Token> wraps6 = new Range<Token>(new BigIntegerToken("30"), new BigIntegerToken("10"));
 
         assertIntersection(wraps1,
                            wraps2,
-                           new Range(new BigIntegerToken("120"), new BigIntegerToken("20")));
+                           new Range<Token>(new BigIntegerToken("120"), new BigIntegerToken("20")));
         assertIntersection(wraps1,
                            wraps3,
-                           new Range(new BigIntegerToken("120"), new BigIntegerToken("20")),
-                           new Range(new BigIntegerToken("100"), new BigIntegerToken("110")));
+                           new Range<Token>(new BigIntegerToken("120"), new BigIntegerToken("20")),
+                           new Range<Token>(new BigIntegerToken("100"), new BigIntegerToken("110")));
         assertIntersection(wraps1,
                            wraps4,
-                           new Range(new BigIntegerToken("10"), new BigIntegerToken("20")),
-                           new Range(new BigIntegerToken("100"), new BigIntegerToken("0")));
+                           new Range<Token>(new BigIntegerToken("10"), new BigIntegerToken("20")),
+                           new Range<Token>(new BigIntegerToken("100"), new BigIntegerToken("0")));
         assertIntersection(wraps1,
                            wraps5,
-                           new Range(new BigIntegerToken("10"), new BigIntegerToken("20")),
-                           new Range(new BigIntegerToken("100"), new BigIntegerToken("1")));
+                           new Range<Token>(new BigIntegerToken("10"), new BigIntegerToken("20")),
+                           new Range<Token>(new BigIntegerToken("100"), new BigIntegerToken("1")));
         assertIntersection(wraps1,
                            wraps6,
-                           new Range(new BigIntegerToken("100"), new BigIntegerToken("10")));
+                           new Range<Token>(new BigIntegerToken("100"), new BigIntegerToken("10")));
     }
 
     @Test
@@ -322,14 +331,9 @@
         assert t1.compareTo(t4) == 0;
     }
 
-    private Range makeRange(String token1, String token2)
-    {
-        return new Range(new BigIntegerToken(token1), new BigIntegerToken(token2));
-    }
-
     private Range<Token> makeRange(long token1, long token2)
     {
-        return   new Range<Token>(new LongToken(token1), new LongToken(token2));
+        return new Range<Token>(new Murmur3Partitioner.LongToken(token1), new Murmur3Partitioner.LongToken(token2));
     }
 
     private void assertRanges(Set<Range<Token>> result, Long ... tokens)
@@ -349,7 +353,7 @@
     @Test
     public void testSubtractAll()
     {
-        Range<Token> range = new Range<Token>(new LongToken(1L), new LongToken(100L));
+        Range<Token> range = new Range<Token>(new Murmur3Partitioner.LongToken(1L), new Murmur3Partitioner.LongToken(100L));
 
         Collection<Range<Token>> collection = new HashSet<>();
         collection.add(makeRange(1L, 10L));
@@ -365,7 +369,7 @@
     @Test
     public void testSubtractAllWithWrapAround()
     {
-        Range<Token> range = new Range<Token>(new LongToken(100L), new LongToken(10L));
+        Range<Token> range = new Range<Token>(new Murmur3Partitioner.LongToken(100L), new Murmur3Partitioner.LongToken(10L));
 
         Collection<Range<Token>> collection = new HashSet<>();
         collection.add(makeRange(20L, 30L));
@@ -379,21 +383,26 @@
         collection.add(makeRange(1000L, 0));
         assertRanges(range.subtractAll(collection), 100L, 200L, 500L, 1000L);
     }
-
-    private Set<Range> makeRanges(String[][] tokenPairs)
+    
+    private Range<Token> makeRange(String token1, String token2)
     {
-        Set<Range> ranges = new HashSet<Range>();
+        return new Range<Token>(new BigIntegerToken(token1), new BigIntegerToken(token2));
+    }
+
+    private Set<Range<Token>> makeRanges(String[][] tokenPairs)
+    {
+        Set<Range<Token>> ranges = new HashSet<Range<Token>>();
         for (int i = 0; i < tokenPairs.length; ++i)
             ranges.add(makeRange(tokenPairs[i][0], tokenPairs[i][1]));
         return ranges;
     }
 
-    private void checkDifference(Range oldRange, String[][] newTokens, String[][] expected)
+    private void checkDifference(Range<Token> oldRange, String[][] newTokens, String[][] expected)
     {
-        Set<Range> ranges = makeRanges(newTokens);
-        for (Range newRange : ranges)
+        Set<Range<Token>> ranges = makeRanges(newTokens);
+        for (Range<Token> newRange : ranges)
         {
-            Set<Range> diff = oldRange.differenceToFetch(newRange);
+            Set<Range<Token>> diff = oldRange.differenceToFetch(newRange);
             assert diff.equals(makeRanges(expected)) : "\n" +
                                                        "Old range: " + oldRange.toString() + "\n" +
                                                        "New range: " + newRange.toString() + "\n" +
@@ -404,7 +413,7 @@
     @Test
     public void testDifferenceToFetchNoWrap()
     {
-        Range oldRange = makeRange("10", "40");
+        Range<Token> oldRange = makeRange("10", "40");
 
         // New range is entirely contained
         String[][] newTokens1 = { { "20", "30" }, { "10", "20" }, { "10", "40" }, { "20", "40" } };
@@ -430,7 +439,7 @@
     @Test
     public void testDifferenceToFetchBothWrap()
     {
-        Range oldRange = makeRange("1010", "40");
+        Range<Token> oldRange = makeRange("1010", "40");
 
         // New range is entirely contained
         String[][] newTokens1 = { { "1020", "30" }, { "1010", "20" }, { "1010", "40" }, { "1020", "40" } };
@@ -456,7 +465,7 @@
     @Test
     public void testDifferenceToFetchOldWraps()
     {
-        Range oldRange = makeRange("1010", "40");
+        Range<Token> oldRange = makeRange("1010", "40");
 
         // New range is entirely contained
         String[][] newTokens1 = { { "0", "30" }, { "0", "40" }, { "10", "40" } };
@@ -487,7 +496,7 @@
     @Test
     public void testDifferenceToFetchNewWraps()
     {
-        Range oldRange = makeRange("0", "40");
+        Range<Token> oldRange = makeRange("0", "40");
 
         // Only the LHS of the new range is needed
         String[][] newTokens1 = { { "1010", "0" }, { "1010", "10" }, { "1010", "40" } };
@@ -650,6 +659,21 @@
     }
     private static Token t(long t)
     {
-        return new LongToken(t);
+        return new Murmur3Partitioner.LongToken(t);
+    }
+
+    @Test
+    public void testCompareTo_SameObject_WrapAround()
+    {
+        Range<Token> range = r(10, -10);
+        assertEquals(0, range.compareTo(range));
+    }
+
+    @Test
+    public void testCompareTo_BothWrapAround()
+    {
+        Range<Token> r0 = r(10, -10);
+        Range<Token> r1 = r(20, -5);
+        assertNotSame(r0.compareTo(r1), r1.compareTo(r0));
     }
 }
diff --git a/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java b/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java
new file mode 100644
index 0000000..bdb654a
--- /dev/null
+++ b/test/unit/org/apache/cassandra/dht/StreamStateStoreTest.java
@@ -0,0 +1,83 @@
+/*
+ * 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.dht;
+
+import java.net.InetAddress;
+import java.util.Collections;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.streaming.DefaultConnectionFactory;
+import org.apache.cassandra.streaming.StreamEvent;
+import org.apache.cassandra.streaming.StreamSession;
+import org.apache.cassandra.utils.FBUtilities;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class StreamStateStoreTest
+{
+    @BeforeClass
+    public static void initDD()
+    {
+        DatabaseDescriptor.setDaemonInitialized();
+    }
+
+    @Test
+    public void testUpdateAndQueryAvailableRanges()
+    {
+        // let range (0, 100] of keyspace1 be bootstrapped.
+        IPartitioner p = new Murmur3Partitioner();
+        Token.TokenFactory factory = p.getTokenFactory();
+        Range<Token> range = new Range<>(factory.fromString("0"), factory.fromString("100"));
+
+        InetAddress local = FBUtilities.getBroadcastAddress();
+        StreamSession session = new StreamSession(local, local, new DefaultConnectionFactory(), 0, true, false);
+        session.addStreamRequest("keyspace1", Collections.singleton(range), Collections.singleton("cf"), 0);
+
+        StreamStateStore store = new StreamStateStore();
+        // session complete event that is not completed makes data not available for keyspace/ranges
+        store.handleStreamEvent(new StreamEvent.SessionCompleteEvent(session));
+        assertFalse(store.isDataAvailable("keyspace1", factory.fromString("50")));
+
+        // successfully completed session adds available keyspace/ranges
+        session.state(StreamSession.State.COMPLETE);
+        store.handleStreamEvent(new StreamEvent.SessionCompleteEvent(session));
+        // check if token in range (0, 100] appears available.
+        assertTrue(store.isDataAvailable("keyspace1", factory.fromString("50")));
+        // check if token out of range returns false
+        assertFalse(store.isDataAvailable("keyspace1", factory.fromString("0")));
+        assertFalse(store.isDataAvailable("keyspace1", factory.fromString("101")));
+        // check if different keyspace returns false
+        assertFalse(store.isDataAvailable("keyspace2", factory.fromString("50")));
+
+        // add different range within the same keyspace
+        Range<Token> range2 = new Range<>(factory.fromString("100"), factory.fromString("200"));
+        session = new StreamSession(local, local, new DefaultConnectionFactory(), 0, true, false);
+        session.addStreamRequest("keyspace1", Collections.singleton(range2), Collections.singleton("cf"), 0);
+        session.state(StreamSession.State.COMPLETE);
+        store.handleStreamEvent(new StreamEvent.SessionCompleteEvent(session));
+
+        // newly added range should be available
+        assertTrue(store.isDataAvailable("keyspace1", factory.fromString("101")));
+        // as well as the old one
+        assertTrue(store.isDataAvailable("keyspace1", factory.fromString("50")));
+    }
+}
\ No newline at end of file
diff --git a/test/unit/org/apache/cassandra/gms/FailureDetectorTest.java b/test/unit/org/apache/cassandra/gms/FailureDetectorTest.java
index 9325922..af099b0 100644
--- a/test/unit/org/apache/cassandra/gms/FailureDetectorTest.java
+++ b/test/unit/org/apache/cassandra/gms/FailureDetectorTest.java
@@ -45,6 +45,8 @@
     {
         // slow unit tests can cause problems with FailureDetector's GC pause handling
         System.setProperty("cassandra.max_local_pause_in_ms", "20000");
+
+        DatabaseDescriptor.setDaemonInitialized();
     }
 
     @Test
diff --git a/test/unit/org/apache/cassandra/gms/GossiperTest.java b/test/unit/org/apache/cassandra/gms/GossiperTest.java
index ad07165..eb01305 100644
--- a/test/unit/org/apache/cassandra/gms/GossiperTest.java
+++ b/test/unit/org/apache/cassandra/gms/GossiperTest.java
@@ -33,6 +33,7 @@
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.RandomPartitioner;
 import org.apache.cassandra.dht.Token;
@@ -44,6 +45,10 @@
 
 public class GossiperTest
 {
+    static
+    {
+        DatabaseDescriptor.setDaemonInitialized();
+    }
     static final IPartitioner partitioner = new RandomPartitioner();
     StorageService ss = StorageService.instance;
     TokenMetadata tmd = StorageService.instance.getTokenMetadata();
diff --git a/test/unit/org/apache/cassandra/gms/PendingRangeCalculatorServiceTest.java b/test/unit/org/apache/cassandra/gms/PendingRangeCalculatorServiceTest.java
new file mode 100644
index 0000000..507948c
--- /dev/null
+++ b/test/unit/org/apache/cassandra/gms/PendingRangeCalculatorServiceTest.java
@@ -0,0 +1,133 @@
+/*
+ * 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.gms;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.service.StorageService;
+import org.jboss.byteman.contrib.bmunit.BMRule;
+import org.jboss.byteman.contrib.bmunit.BMUnitRunner;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+
+/**
+ * Test for "Gossip blocks on startup when another node is bootstrapping" (CASSANDRA-12281).
+ */
+@RunWith(BMUnitRunner.class)
+public class PendingRangeCalculatorServiceTest
+{
+    static ReentrantLock calculationLock = new ReentrantLock();
+
+    @BeforeClass
+    public static void setUp() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        StorageService.instance.initServer();
+    }
+
+    @Test
+    @BMRule(name = "Block pending range calculation",
+            targetClass = "TokenMetadata",
+            targetMethod = "calculatePendingRanges",
+            targetLocation = "AT INVOKE org.apache.cassandra.locator.AbstractReplicationStrategy.getAddressRanges",
+            action = "org.apache.cassandra.gms.PendingRangeCalculatorServiceTest.calculationLock.lock()")
+    public void testDelayedResponse() throws UnknownHostException, InterruptedException
+    {
+        final InetAddress otherNodeAddr = InetAddress.getByName("127.0.0.2");
+        final UUID otherHostId = UUID.randomUUID();
+
+        // introduce node for first major state change
+        Gossiper.instance.applyStateLocally(getStates(otherNodeAddr, otherHostId, 1, false));
+
+        // acquire lock to block pending range calculation via byteman
+        calculationLock.lock();
+        try
+        {
+            // spawn thread that will trigger handling of a bootstrap state change which in turn will trigger
+            // the pending range calculation that will be blocked by our lock
+            Thread t1 = new Thread()
+            {
+                public void run()
+                {
+                    Gossiper.instance.applyStateLocally(getStates(otherNodeAddr, otherHostId, 2, true));
+                }
+            };
+            t1.start();
+
+            // busy-spin until t1 is blocked by lock
+            while (!calculationLock.hasQueuedThreads()) ;
+
+            // trigger further state changes in case we don't want the blocked thread from the
+            // expensive range calculation to block us here as well
+            Thread t2 = new Thread()
+            {
+                public void run()
+                {
+                    Gossiper.instance.applyStateLocally(getStates(otherNodeAddr, otherHostId, 3, false));
+                    Gossiper.instance.applyStateLocally(getStates(otherNodeAddr, otherHostId, 4, false));
+                    Gossiper.instance.applyStateLocally(getStates(otherNodeAddr, otherHostId, 5, false));
+                }
+            };
+            t2.start();
+            t2.join(2000);
+            assertFalse("Thread still blocked by pending range calculation", t2.isAlive());
+            assertEquals(5, Gossiper.instance.getEndpointStateForEndpoint(otherNodeAddr).getHeartBeatState().getHeartBeatVersion());
+        }
+        finally
+        {
+            calculationLock.unlock();
+        }
+    }
+
+    private Map<InetAddress, EndpointState> getStates(InetAddress otherNodeAddr, UUID hostId, int ver, boolean bootstrapping)
+    {
+        HeartBeatState hb = new HeartBeatState(1, ver);
+        EndpointState state = new EndpointState(hb);
+        Collection<Token> tokens = new ArrayList<>();
+
+        tokens.add(new ByteOrderedPartitioner.BytesToken(new byte[]{1,2,3}));
+        state.addApplicationState(ApplicationState.TOKENS, StorageService.instance.valueFactory.tokens(tokens));
+        state.addApplicationState(ApplicationState.STATUS, bootstrapping ?
+                StorageService.instance.valueFactory.bootstrapping(tokens) : StorageService.instance.valueFactory.normal(tokens));
+        state.addApplicationState(ApplicationState.HOST_ID, StorageService.instance.valueFactory.hostId(hostId));
+        state.addApplicationState(ApplicationState.NET_VERSION, StorageService.instance.valueFactory.networkVersion());
+
+        Map<InetAddress, EndpointState> states = new HashMap<>();
+        states.put(otherNodeAddr, state);
+        return states;
+    }
+}
diff --git a/test/unit/org/apache/cassandra/gms/SerializationsTest.java b/test/unit/org/apache/cassandra/gms/SerializationsTest.java
index 6317a98..080ae53 100644
--- a/test/unit/org/apache/cassandra/gms/SerializationsTest.java
+++ b/test/unit/org/apache/cassandra/gms/SerializationsTest.java
@@ -20,13 +20,12 @@
 
 import org.apache.cassandra.AbstractSerializationsTester;
 import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.io.util.DataOutputStreamAndChannel;
+import org.apache.cassandra.io.util.DataOutputStreamPlus;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.FBUtilities;
 import org.junit.Test;
 
 import java.io.DataInputStream;
-import java.io.DataOutputStream;
 import java.io.IOException;
 import java.net.InetAddress;
 import java.util.ArrayList;
@@ -39,7 +38,7 @@
 {
     private void testEndpointStateWrite() throws IOException
     {
-        DataOutputStreamAndChannel out = getOutput("gms.EndpointState.bin");
+        DataOutputStreamPlus out = getOutput("gms.EndpointState.bin");
         HeartBeatState.serializer.serialize(Statics.HeartbeatSt, out, getVersion());
         EndpointState.serializer.serialize(Statics.EndpointSt, out, getVersion());
         VersionedValue.serializer.serialize(Statics.vv0, out, getVersion());
@@ -76,7 +75,7 @@
         GossipDigestAck2 ack2 = new GossipDigestAck2(states);
         GossipDigestSyn syn = new GossipDigestSyn("Not a real cluster name", StorageService.getPartitioner().getClass().getCanonicalName(), Statics.Digests);
 
-        DataOutputStreamAndChannel out = getOutput("gms.Gossip.bin");
+        DataOutputStreamPlus out = getOutput("gms.Gossip.bin");
         for (GossipDigest gd : Statics.Digests)
             GossipDigest.serializer.serialize(gd, out, getVersion());
         GossipDigestAck.serializer.serialize(ack, out, getVersion());
diff --git a/test/unit/org/apache/cassandra/io/BloomFilterTrackerTest.java b/test/unit/org/apache/cassandra/io/BloomFilterTrackerTest.java
index 4943a3a..afcf2a5 100644
--- a/test/unit/org/apache/cassandra/io/BloomFilterTrackerTest.java
+++ b/test/unit/org/apache/cassandra/io/BloomFilterTrackerTest.java
@@ -23,12 +23,11 @@
 
 import org.junit.Test;
 
-import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.io.sstable.BloomFilterTracker;
 
 import static org.junit.Assert.assertEquals;
 
-public class BloomFilterTrackerTest extends SchemaLoader
+public class BloomFilterTrackerTest
 {
     @Test
     public void testAddingFalsePositives()
diff --git a/test/unit/org/apache/cassandra/io/ChecksummedRandomAccessReaderTest.java b/test/unit/org/apache/cassandra/io/ChecksummedRandomAccessReaderTest.java
new file mode 100644
index 0000000..c1e43c9
--- /dev/null
+++ b/test/unit/org/apache/cassandra/io/ChecksummedRandomAccessReaderTest.java
@@ -0,0 +1,127 @@
+/*
+ * 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.io;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.util.Arrays;
+import java.util.concurrent.ThreadLocalRandom;
+
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+import org.apache.cassandra.io.util.ChecksummedRandomAccessReader;
+import org.apache.cassandra.io.util.ChecksummedSequentialWriter;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.io.util.SequentialWriter;
+
+public class ChecksummedRandomAccessReaderTest
+{
+    @Test
+    public void readFully() throws IOException
+    {
+        final File data = File.createTempFile("testReadFully", "data");
+        final File crc = File.createTempFile("testReadFully", "crc");
+
+        final byte[] expected = new byte[70 * 1024];   // bit more than crc chunk size, so we can test rebuffering.
+        ThreadLocalRandom.current().nextBytes(expected);
+
+        SequentialWriter writer = ChecksummedSequentialWriter.open(data, crc);
+        writer.write(expected);
+        writer.finish();
+
+        assert data.exists();
+
+        RandomAccessReader reader = ChecksummedRandomAccessReader.open(data, crc);
+        byte[] b = new byte[expected.length];
+        reader.readFully(b);
+
+        assertArrayEquals(expected, b);
+
+        assertTrue(reader.isEOF());
+
+        reader.close();
+    }
+
+    @Test
+    public void seek() throws IOException
+    {
+        final File data = File.createTempFile("testSeek", "data");
+        final File crc = File.createTempFile("testSeek", "crc");
+
+        final byte[] dataBytes = new byte[70 * 1024];   // bit more than crc chunk size
+        ThreadLocalRandom.current().nextBytes(dataBytes);
+
+        SequentialWriter writer = ChecksummedSequentialWriter.open(data, crc);
+        writer.write(dataBytes);
+        writer.finish();
+
+        assert data.exists();
+
+        RandomAccessReader reader = ChecksummedRandomAccessReader.open(data, crc);
+
+        final int seekPosition = 66000;
+        reader.seek(seekPosition);
+
+        byte[] b = new byte[dataBytes.length - seekPosition];
+        reader.readFully(b);
+
+        byte[] expected = Arrays.copyOfRange(dataBytes, seekPosition, dataBytes.length);
+
+        assertArrayEquals(expected, b);
+
+        assertTrue(reader.isEOF());
+
+        reader.close();
+    }
+
+    @Test(expected = ChecksummedRandomAccessReader.CorruptFileException.class)
+    public void corruptionDetection() throws IOException
+    {
+        final File data = File.createTempFile("corruptionDetection", "data");
+        final File crc = File.createTempFile("corruptionDetection", "crc");
+
+        final byte[] expected = new byte[5 * 1024];
+        Arrays.fill(expected, (byte) 0);
+
+        SequentialWriter writer = ChecksummedSequentialWriter.open(data, crc);
+        writer.write(expected);
+        writer.finish();
+
+        assert data.exists();
+
+        // simulate corruption of file
+        try (RandomAccessFile dataFile = new RandomAccessFile(data, "rw"))
+        {
+            dataFile.seek(1024);
+            dataFile.write((byte) 5);
+        }
+
+        RandomAccessReader reader = ChecksummedRandomAccessReader.open(data, crc);
+        byte[] b = new byte[expected.length];
+        reader.readFully(b);
+
+        assertArrayEquals(expected, b);
+
+        assertTrue(reader.isEOF());
+
+        reader.close();
+    }
+}
diff --git a/test/unit/org/apache/cassandra/io/RandomAccessReaderTest.java b/test/unit/org/apache/cassandra/io/RandomAccessReaderTest.java
new file mode 100644
index 0000000..8c6cc90
--- /dev/null
+++ b/test/unit/org/apache/cassandra/io/RandomAccessReaderTest.java
@@ -0,0 +1,251 @@
+/*
+ *
+ * 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.io;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.Charset;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+import org.apache.cassandra.io.util.ChannelProxy;
+import org.apache.cassandra.io.util.FileMark;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.io.util.SequentialWriter;
+
+public class RandomAccessReaderTest
+{
+    @Test
+    public void testReadFully() throws IOException
+    {
+        final File f = File.createTempFile("testReadFully", "1");
+        final String expected = "The quick brown fox jumps over the lazy dog";
+
+        SequentialWriter writer = SequentialWriter.open(f);
+        writer.write(expected.getBytes());
+        writer.finish();
+
+        assert f.exists();
+
+        ChannelProxy channel = new ChannelProxy(f);
+        RandomAccessReader reader = RandomAccessReader.open(channel);
+        assertEquals(f.getAbsolutePath(), reader.getPath());
+        assertEquals(expected.length(), reader.length());
+
+        byte[] b = new byte[expected.length()];
+        reader.readFully(b);
+        assertEquals(expected, new String(b));
+
+        assertTrue(reader.isEOF());
+        assertEquals(0, reader.bytesRemaining());
+
+        reader.close();
+        channel.close();
+    }
+
+    @Test
+    public void testReadBytes() throws IOException
+    {
+        File f = File.createTempFile("testReadBytes", "1");
+        final String expected = "The quick brown fox jumps over the lazy dog";
+
+        SequentialWriter writer = SequentialWriter.open(f);
+        writer.write(expected.getBytes());
+        writer.finish();
+
+        assert f.exists();
+
+        ChannelProxy channel = new ChannelProxy(f);
+        RandomAccessReader reader = RandomAccessReader.open(channel);
+        assertEquals(f.getAbsolutePath(), reader.getPath());
+        assertEquals(expected.length(), reader.length());
+
+        ByteBuffer b = reader.readBytes(expected.length());
+        assertEquals(expected, new String(b.array(), Charset.forName("UTF-8")));
+
+        assertTrue(reader.isEOF());
+        assertEquals(0, reader.bytesRemaining());
+
+        reader.close();
+        channel.close();
+    }
+
+    @Test
+    public void testReset() throws IOException
+    {
+        File f = File.createTempFile("testMark", "1");
+        final String expected = "The quick brown fox jumps over the lazy dog";
+        final int numIterations = 10;
+
+        SequentialWriter writer = SequentialWriter.open(f);
+        for (int i = 0; i < numIterations; i++)
+            writer.write(expected.getBytes());
+        writer.finish();
+
+        assert f.exists();
+
+        ChannelProxy channel = new ChannelProxy(f);
+        RandomAccessReader reader = RandomAccessReader.open(channel);
+        assertEquals(expected.length() * numIterations, reader.length());
+
+        ByteBuffer b = reader.readBytes(expected.length());
+        assertEquals(expected, new String(b.array(), Charset.forName("UTF-8")));
+
+        assertFalse(reader.isEOF());
+        assertEquals((numIterations - 1) * expected.length(), reader.bytesRemaining());
+
+        FileMark mark = reader.mark();
+        assertEquals(0, reader.bytesPastMark());
+        assertEquals(0, reader.bytesPastMark(mark));
+
+        for (int i = 0; i < (numIterations - 1); i++)
+        {
+            b = reader.readBytes(expected.length());
+            assertEquals(expected, new String(b.array(), Charset.forName("UTF-8")));
+        }
+        assertTrue(reader.isEOF());
+        assertEquals(expected.length() * (numIterations -1), reader.bytesPastMark());
+        assertEquals(expected.length() * (numIterations - 1), reader.bytesPastMark(mark));
+
+        reader.reset(mark);
+        assertEquals(0, reader.bytesPastMark());
+        assertEquals(0, reader.bytesPastMark(mark));
+        assertFalse(reader.isEOF());
+        for (int i = 0; i < (numIterations - 1); i++)
+        {
+            b = reader.readBytes(expected.length());
+            assertEquals(expected, new String(b.array(), Charset.forName("UTF-8")));
+        }
+
+        reader.reset();
+        assertEquals(0, reader.bytesPastMark());
+        assertEquals(0, reader.bytesPastMark(mark));
+        assertFalse(reader.isEOF());
+        for (int i = 0; i < (numIterations - 1); i++)
+        {
+            b = reader.readBytes(expected.length());
+            assertEquals(expected, new String(b.array(), Charset.forName("UTF-8")));
+        }
+
+        assertTrue(reader.isEOF());
+        reader.close();
+        channel.close();
+    }
+
+    @Test
+    public void testSeekSingleThread() throws IOException, InterruptedException
+    {
+        testSeek(1);
+    }
+
+    @Test
+    public void testSeekMultipleThreads() throws IOException, InterruptedException
+    {
+        testSeek(10);
+    }
+
+    private void testSeek(int numThreads) throws IOException, InterruptedException
+    {
+        final File f = File.createTempFile("testMark", "1");
+        final String[] expected = new String[10];
+        int len = 0;
+        for (int i = 0; i < expected.length; i++)
+        {
+            expected[i] = UUID.randomUUID().toString();
+            len += expected[i].length();
+        }
+        final int totalLength = len;
+
+        SequentialWriter writer = SequentialWriter.open(f);
+        for (int i = 0; i < expected.length; i++)
+            writer.write(expected[i].getBytes());
+        writer.finish();
+
+        assert f.exists();
+
+        final ChannelProxy channel = new ChannelProxy(f);
+
+        final Runnable worker = new Runnable() {
+
+            @Override
+            public void run()
+            {
+                try
+                {
+                    RandomAccessReader reader = RandomAccessReader.open(channel);
+                    assertEquals(totalLength, reader.length());
+
+                    ByteBuffer b = reader.readBytes(expected[0].length());
+                    assertEquals(expected[0], new String(b.array(), Charset.forName("UTF-8")));
+
+                    assertFalse(reader.isEOF());
+                    assertEquals(totalLength - expected[0].length(), reader.bytesRemaining());
+
+                    long filePointer = reader.getFilePointer();
+
+                    for (int i = 1; i < expected.length; i++)
+                    {
+                        b = reader.readBytes(expected[i].length());
+                        assertEquals(expected[i], new String(b.array(), Charset.forName("UTF-8")));
+                    }
+                    assertTrue(reader.isEOF());
+
+                    reader.seek(filePointer);
+                    assertFalse(reader.isEOF());
+                    for (int i = 1; i < expected.length; i++)
+                    {
+                        b = reader.readBytes(expected[i].length());
+                        assertEquals(expected[i], new String(b.array(), Charset.forName("UTF-8")));
+                    }
+
+                    assertTrue(reader.isEOF());
+                    reader.close();
+                }
+                catch (Exception ex)
+                {
+                    ex.printStackTrace();
+                    fail(ex.getMessage());
+                }
+            }
+        };
+
+        if(numThreads == 1)
+        {
+            worker.run();
+            return;
+        }
+
+        ExecutorService executor = Executors.newFixedThreadPool(numThreads);
+        for (int i = 0; i < numThreads; i++)
+            executor.submit(worker);
+
+        executor.shutdown();
+        executor.awaitTermination(1, TimeUnit.MINUTES);
+
+        channel.close();
+    }
+}
diff --git a/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java b/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
index 900abd8..3bef89e 100644
--- a/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
+++ b/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
@@ -24,6 +24,7 @@
 import java.util.Collections;
 import java.util.Random;
 
+import com.google.common.util.concurrent.RateLimiter;
 import org.junit.Test;
 
 import org.apache.cassandra.db.composites.SimpleDenseCellNameType;
@@ -31,12 +32,18 @@
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.CorruptSSTableException;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
+import org.apache.cassandra.io.util.ChannelProxy;
+import org.apache.cassandra.io.util.CompressedPoolingSegmentedFile;
+import org.apache.cassandra.io.util.FileDataInput;
 import org.apache.cassandra.io.util.FileMark;
 import org.apache.cassandra.io.util.RandomAccessReader;
 import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.utils.SyncUtil;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
 
 public class CompressedRandomAccessReaderTest
 {
@@ -55,11 +62,13 @@
         testResetAndTruncate(File.createTempFile("compressed", "1"), true, 10);
         testResetAndTruncate(File.createTempFile("compressed", "2"), true, CompressionParameters.DEFAULT_CHUNK_LENGTH);
     }
+
     @Test
     public void test6791() throws IOException, ConfigurationException
     {
         File f = File.createTempFile("compressed6791_", "3");
         String filename = f.getAbsolutePath();
+        ChannelProxy channel = new ChannelProxy(f);
         try
         {
 
@@ -78,9 +87,9 @@
 
             for (int i = 0; i < 20; i++)
                 writer.write("x".getBytes());
-            writer.close();
+            writer.finish();
 
-            CompressedRandomAccessReader reader = CompressedRandomAccessReader.open(filename, new CompressionMetadata(filename + ".metadata", f.length(), true));
+            CompressedRandomAccessReader reader = CompressedRandomAccessReader.open(channel, new CompressionMetadata(filename + ".metadata", f.length()));
             String res = reader.readLine();
             assertEquals(res, "xxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxx");
             assertEquals(40, res.length());
@@ -88,6 +97,8 @@
         finally
         {
             // cleanup
+            channel.close();
+
             if (f.exists())
                 f.delete();
             File metadata = new File(filename+ ".metadata");
@@ -99,13 +110,13 @@
     private void testResetAndTruncate(File f, boolean compressed, int junkSize) throws IOException
     {
         final String filename = f.getAbsolutePath();
-
+        ChannelProxy channel = new ChannelProxy(f);
         try
         {
-            MetadataCollector sstableMetadataCollector = new MetadataCollector(new SimpleDenseCellNameType(BytesType.instance)).replayPosition(null);
+            MetadataCollector sstableMetadataCollector = new MetadataCollector(new SimpleDenseCellNameType(BytesType.instance));
             SequentialWriter writer = compressed
                 ? new CompressedSequentialWriter(f, filename + ".metadata", new CompressionParameters(SnappyCompressor.instance), sstableMetadataCollector)
-                : new SequentialWriter(f, CompressionParameters.DEFAULT_CHUNK_LENGTH);
+                : SequentialWriter.open(f);
 
             writer.write("The quick ".getBytes());
             FileMark mark = writer.mark();
@@ -119,11 +130,11 @@
 
             writer.resetAndTruncate(mark);
             writer.write("brown fox jumps over the lazy dog".getBytes());
-            writer.close();
+            writer.finish();
 
             assert f.exists();
             RandomAccessReader reader = compressed
-                                      ? CompressedRandomAccessReader.open(filename, new CompressionMetadata(filename + ".metadata", f.length(), true))
+                                      ? CompressedRandomAccessReader.open(channel, new CompressionMetadata(filename + ".metadata", f.length()))
                                       : RandomAccessReader.open(f);
             String expected = "The quick brown fox jumps over the lazy dog";
             assertEquals(expected.length(), reader.length());
@@ -134,6 +145,8 @@
         finally
         {
             // cleanup
+            channel.close();
+
             if (f.exists())
                 f.delete();
             File metadata = new File(filename + ".metadata");
@@ -153,21 +166,22 @@
         File metadata = new File(file.getPath() + ".meta");
         metadata.deleteOnExit();
 
-        MetadataCollector sstableMetadataCollector = new MetadataCollector(new SimpleDenseCellNameType(BytesType.instance)).replayPosition(null);
-        SequentialWriter writer = new CompressedSequentialWriter(file, metadata.getPath(), new CompressionParameters(SnappyCompressor.instance), sstableMetadataCollector);
+        MetadataCollector sstableMetadataCollector = new MetadataCollector(new SimpleDenseCellNameType(BytesType.instance));
+        try (SequentialWriter writer = new CompressedSequentialWriter(file, metadata.getPath(), new CompressionParameters(SnappyCompressor.instance), sstableMetadataCollector))
+        {
+            writer.write(CONTENT.getBytes());
+            writer.finish();
+        }
 
-        writer.write(CONTENT.getBytes());
-        writer.close();
+        ChannelProxy channel = new ChannelProxy(file);
 
         // open compression metadata and get chunk information
-        CompressionMetadata meta = new CompressionMetadata(metadata.getPath(), file.length(), true);
+        CompressionMetadata meta = new CompressionMetadata(metadata.getPath(), file.length());
         CompressionMetadata.Chunk chunk = meta.chunkFor(0);
 
-        RandomAccessReader reader = CompressedRandomAccessReader.open(file.getPath(), meta);
+        RandomAccessReader reader = CompressedRandomAccessReader.open(channel, meta);
         // read and verify compressed data
         assertEquals(CONTENT, reader.readLine());
-        // close reader
-        reader.close();
 
         Random random = new Random();
         RandomAccessFile checksumModifier = null;
@@ -188,9 +202,9 @@
             for (int i = 0; i < checksum.length; i++)
             {
                 checksumModifier.write(random.nextInt());
-                checksumModifier.getFD().sync(); // making sure that change was synced with disk
+                SyncUtil.sync(checksumModifier); // making sure that change was synced with disk
 
-                final RandomAccessReader r = CompressedRandomAccessReader.open(file.getPath(), meta);
+                final RandomAccessReader r = CompressedRandomAccessReader.open(channel, meta);
 
                 Throwable exception = null;
                 try
@@ -211,7 +225,7 @@
             // lets write original checksum and check if we can read data
             updateChecksum(checksumModifier, chunk.length, checksum);
 
-            reader = CompressedRandomAccessReader.open(file.getPath(), meta);
+            reader = CompressedRandomAccessReader.open(channel, meta);
             // read and verify compressed data
             assertEquals(CONTENT, reader.readLine());
             // close reader
@@ -219,15 +233,71 @@
         }
         finally
         {
+            channel.close();
+
             if (checksumModifier != null)
                 checksumModifier.close();
         }
     }
 
+    @Test
+    public void testThrottledReadersAreNotCached() throws IOException
+    {
+        String CONTENT = "Lorem ipsum dolor sit amet, consectetur adipiscing elit. Etiam vitae.";
+
+        File file = new File("testThrottledReadersAreNotCached");
+        file.deleteOnExit();
+
+        File metadata = new File(file.getPath() + ".meta");
+        metadata.deleteOnExit();
+
+        MetadataCollector sstableMetadataCollector = new MetadataCollector(new SimpleDenseCellNameType(BytesType.instance));
+        try (SequentialWriter writer = new CompressedSequentialWriter(file, metadata.getPath(), new CompressionParameters(SnappyCompressor.instance), sstableMetadataCollector))
+        {
+            writer.write(CONTENT.getBytes());
+            writer.finish();
+        }
+
+        CompressionMetadata meta = new CompressionMetadata(metadata.getPath(), file.length());
+
+        try(ChannelProxy channel = new ChannelProxy(file);
+            CompressedPoolingSegmentedFile segmentedFile = new CompressedPoolingSegmentedFile(channel, meta))
+        {
+            //The cache bucket is only initialized by a call to FileCacheService.instance.get() so first
+            // we must create a reader using the interface for accessing segments
+            FileDataInput reader = segmentedFile.getSegment(0);
+            assertNotNull(reader);
+            reader.close();
+
+            //Now we create a throttled reader, this should not be added to the cache
+            RateLimiter limiter = RateLimiter.create(1024);
+            reader = segmentedFile.createThrottledReader(limiter);
+            assertNotNull(reader);
+            assertTrue(reader instanceof CompressedThrottledReader);
+            reader.close();
+
+            //We retrieve 2 readers, neither should be a throttled reader
+            FileDataInput[] readers =
+            {
+                segmentedFile.getSegment(0),
+                segmentedFile.getSegment(0)
+            };
+
+            for (FileDataInput r : readers)
+            {
+                assertNotNull(r);
+                assertFalse(r instanceof CompressedThrottledReader);
+            }
+
+            for (FileDataInput r : readers)
+                r.close();
+        }
+    }
+
     private void updateChecksum(RandomAccessFile file, long checksumOffset, byte[] checksum) throws IOException
     {
         file.seek(checksumOffset);
         file.write(checksum);
-        file.getFD().sync();
+        SyncUtil.sync(file);
     }
 }
diff --git a/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java b/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java
index fe3ba46..43c44fd 100644
--- a/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java
+++ b/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java
@@ -17,25 +17,35 @@
  */
 package org.apache.cassandra.io.compress;
 
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
 import java.io.File;
 import java.io.IOException;
-import java.util.Arrays;
-import java.util.Random;
+import java.nio.ByteBuffer;
+import java.util.*;
 
+import static org.apache.commons.io.FileUtils.readFileToByteArray;
 import static org.junit.Assert.assertEquals;
+
+import org.junit.After;
 import org.junit.Test;
 
+import junit.framework.Assert;
+import org.apache.cassandra.db.composites.CellNames;
 import org.apache.cassandra.db.composites.SimpleDenseCellNameType;
 import org.apache.cassandra.db.marshal.BytesType;
-import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
+import org.apache.cassandra.io.util.ChannelProxy;
 import org.apache.cassandra.io.util.FileMark;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.io.util.SequentialWriterTest;
 
-public class CompressedSequentialWriterTest
+public class CompressedSequentialWriterTest extends SequentialWriterTest
 {
     private ICompressor compressor;
 
-    private void runTests(String testName) throws IOException, ConfigurationException
+    private void runTests(String testName) throws IOException
     {
         // Test small < 1 chunk data set
         testWrite(File.createTempFile(testName + "_small", "1"), 25);
@@ -48,57 +58,62 @@
     }
 
     @Test
-    public void testLZ4Writer() throws IOException, ConfigurationException
+    public void testLZ4Writer() throws IOException
     {
         compressor = LZ4Compressor.instance;
         runTests("LZ4");
     }
 
     @Test
-    public void testDeflateWriter() throws IOException, ConfigurationException
+    public void testDeflateWriter() throws IOException
     {
         compressor = DeflateCompressor.instance;
         runTests("Deflate");
     }
 
     @Test
-    public void testSnappyWriter() throws IOException, ConfigurationException
+    public void testSnappyWriter() throws IOException
     {
         compressor = SnappyCompressor.instance;
         runTests("Snappy");
     }
 
-    private void testWrite(File f, int bytesToTest) throws IOException, ConfigurationException
+    private void testWrite(File f, int bytesToTest) throws IOException
     {
+        final String filename = f.getAbsolutePath();
+        final ChannelProxy channel = new ChannelProxy(f);
+
         try
         {
-            final String filename = f.getAbsolutePath();
-
-            MetadataCollector sstableMetadataCollector = new MetadataCollector(new SimpleDenseCellNameType(BytesType.instance)).replayPosition(null);
-            CompressedSequentialWriter writer = new CompressedSequentialWriter(f, filename + ".metadata", new CompressionParameters(compressor), sstableMetadataCollector);
-
+            MetadataCollector sstableMetadataCollector = new MetadataCollector(new SimpleDenseCellNameType(BytesType.instance));
             byte[] dataPre = new byte[bytesToTest];
             byte[] rawPost = new byte[bytesToTest];
-            Random r = new Random();
-
-            // Test both write with byte[] and ByteBuffer
-            r.nextBytes(dataPre);
-            r.nextBytes(rawPost);
-
-            writer.write(dataPre);
-            FileMark mark = writer.mark();
-
-            // Write enough garbage to transition chunk
-            for (int i = 0; i < CompressionParameters.DEFAULT_CHUNK_LENGTH; i++)
+            try (CompressedSequentialWriter writer = new CompressedSequentialWriter(f, filename + ".metadata", new CompressionParameters(compressor), sstableMetadataCollector);)
             {
-                writer.write((byte)i);
+                Random r = new Random();
+
+                // Test both write with byte[] and ByteBuffer
+                r.nextBytes(dataPre);
+                r.nextBytes(rawPost);
+                ByteBuffer dataPost = makeBB(bytesToTest);
+                dataPost.put(rawPost);
+                dataPost.flip();
+
+                writer.write(dataPre);
+                FileMark mark = writer.mark();
+
+                // Write enough garbage to transition chunk
+                for (int i = 0; i < CompressionParameters.DEFAULT_CHUNK_LENGTH; i++)
+                {
+                    writer.write((byte)i);
+                }
+                writer.resetAndTruncate(mark);
+                writer.write(dataPost);
+                writer.finish();
             }
-            writer.resetAndTruncate(mark);
-            writer.write(rawPost);
-            writer.close();
 
             assert f.exists();
-            CompressedRandomAccessReader reader = CompressedRandomAccessReader.open(filename, new CompressionMetadata(filename + ".metadata", f.length(), true));
+            RandomAccessReader reader = CompressedRandomAccessReader.open(channel, new CompressionMetadata(filename + ".metadata", f.length()));
             assertEquals(dataPre.length + rawPost.length, reader.length());
             byte[] result = new byte[(int)reader.length()];
 
@@ -115,6 +130,8 @@
         finally
         {
             // cleanup
+            channel.close();
+
             if (f.exists())
                 f.delete();
             File metadata = new File(f + ".metadata");
@@ -122,4 +139,90 @@
                 metadata.delete();
         }
     }
+
+    private ByteBuffer makeBB(int size)
+    {
+        return compressor.preferredBufferType().allocate(size);
+    }
+
+    private final List<TestableCSW> writers = new ArrayList<>();
+
+    @After
+    public void cleanup()
+    {
+        for (TestableCSW sw : writers)
+            sw.cleanup();
+        writers.clear();
+    }
+
+    protected TestableTransaction newTest() throws IOException
+    {
+        TestableCSW sw = new TestableCSW();
+        writers.add(sw);
+        return sw;
+    }
+
+    private static class TestableCSW extends TestableSW
+    {
+        final File offsetsFile;
+
+        private TestableCSW() throws IOException
+        {
+            this(tempFile("compressedsequentialwriter"),
+                 tempFile("compressedsequentialwriter.offsets"));
+        }
+
+        private TestableCSW(File file, File offsetsFile) throws IOException
+        {
+            this(file, offsetsFile, new CompressedSequentialWriter(file, offsetsFile.getPath(), new CompressionParameters(LZ4Compressor.instance, BUFFER_SIZE, new HashMap<String, String>()), new MetadataCollector(CellNames.fromAbstractType(UTF8Type.instance, false))));
+        }
+
+        private TestableCSW(File file, File offsetsFile, CompressedSequentialWriter sw) throws IOException
+        {
+            super(file, sw);
+            this.offsetsFile = offsetsFile;
+        }
+
+        protected void assertInProgress() throws Exception
+        {
+            Assert.assertTrue(file.exists());
+            Assert.assertFalse(offsetsFile.exists());
+            byte[] compressed = readFileToByteArray(file);
+            byte[] uncompressed = new byte[partialContents.length];
+            LZ4Compressor.instance.uncompress(compressed, 0, compressed.length - 4, uncompressed, 0);
+            Assert.assertTrue(Arrays.equals(partialContents, uncompressed));
+        }
+
+        protected void assertPrepared() throws Exception
+        {
+            Assert.assertTrue(file.exists());
+            Assert.assertTrue(offsetsFile.exists());
+            DataInputStream offsets = new DataInputStream(new ByteArrayInputStream(readFileToByteArray(offsetsFile)));
+            Assert.assertTrue(offsets.readUTF().endsWith("LZ4Compressor"));
+            Assert.assertEquals(0, offsets.readInt());
+            Assert.assertEquals(BUFFER_SIZE, offsets.readInt());
+            Assert.assertEquals(fullContents.length, offsets.readLong());
+            Assert.assertEquals(2, offsets.readInt());
+            Assert.assertEquals(0, offsets.readLong());
+            int offset = (int) offsets.readLong();
+            byte[] compressed = readFileToByteArray(file);
+            byte[] uncompressed = new byte[fullContents.length];
+            LZ4Compressor.instance.uncompress(compressed, 0, offset - 4, uncompressed, 0);
+            LZ4Compressor.instance.uncompress(compressed, offset, compressed.length - (4 + offset), uncompressed, partialContents.length);
+            Assert.assertTrue(Arrays.equals(fullContents, uncompressed));
+        }
+
+        protected void assertAborted() throws Exception
+        {
+            super.assertAborted();
+            Assert.assertFalse(offsetsFile.exists());
+        }
+
+        void cleanup()
+        {
+            file.delete();
+            offsetsFile.delete();
+        }
+    }
+
 }
diff --git a/test/unit/org/apache/cassandra/io/compress/CompressorTest.java b/test/unit/org/apache/cassandra/io/compress/CompressorTest.java
new file mode 100644
index 0000000..1d285ea4
--- /dev/null
+++ b/test/unit/org/apache/cassandra/io/compress/CompressorTest.java
@@ -0,0 +1,252 @@
+/*
+ * 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.io.compress;
+
+import java.io.*;
+import java.nio.ByteBuffer;
+import java.nio.MappedByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.StandardOpenOption;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Random;
+
+import com.google.common.io.Files;
+import static org.junit.Assert.*;
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class CompressorTest
+{
+    ICompressor compressor;
+
+    ICompressor[] compressors = new ICompressor[] {
+            LZ4Compressor.create(Collections.<String, String>emptyMap()),
+            DeflateCompressor.create(Collections.<String, String>emptyMap()),
+            SnappyCompressor.create(Collections.<String, String>emptyMap())
+    };
+
+    @Test
+    public void testAllCompressors() throws IOException
+    {
+        for (ICompressor compressor : compressors)
+        {
+            this.compressor = compressor;
+
+            testEmptyArray();
+            testLongArray();
+            testShortArray();
+            testMappedFile();
+        }
+    }
+
+    public void testArrayUncompress(byte[] data, int off, int len) throws IOException
+    {
+        final int inOffset = 2;
+        ByteBuffer src = makeBB(len + inOffset);
+        src.position(inOffset);
+        src.put(data, off, len);
+        src.flip().position(inOffset);
+
+        final int outOffset = 3;
+        final ByteBuffer compressed = makeBB(outOffset + compressor.initialCompressedBufferLength(len));
+        fillBBWithRandom(compressed);
+        compressed.position(outOffset);
+
+        compressor.compress(src, compressed);
+        compressed.flip().position(outOffset);
+
+        final int restoreOffset = 5;
+        final byte[] restored = new byte[restoreOffset + len];
+        new Random().nextBytes(restored);
+
+        // need byte[] representation which direct buffers don't have
+        byte[] compressedBytes = new byte[compressed.capacity()];
+        ByteBufferUtil.arrayCopy(compressed, outOffset, compressedBytes, outOffset, compressed.limit() - outOffset);
+
+        final int decompressedLength = compressor.uncompress(compressedBytes, outOffset, compressed.remaining(), restored, restoreOffset);
+
+        assertEquals(decompressedLength, len);
+        assertArrayEquals(Arrays.copyOfRange(data, off, off + len),
+                Arrays.copyOfRange(restored, restoreOffset, restoreOffset + decompressedLength));
+    }
+
+    public void testArrayUncompress(byte[] data) throws IOException
+    {
+        testArrayUncompress(data, 0, data.length);
+    }
+
+    public void testEmptyArray() throws IOException
+    {
+        testArrayUncompress(new byte[0]);
+    }
+
+    public void testShortArray() throws UnsupportedEncodingException, IOException
+    {
+        testArrayUncompress("Cassandra".getBytes("UTF-8"), 1, 7);
+    }
+
+    public void testLongArray() throws UnsupportedEncodingException, IOException
+    {
+        byte[] data = new byte[1 << 20];
+        testArrayUncompress(data, 13, 1 << 19);
+        new Random(0).nextBytes(data);
+        testArrayUncompress(data, 13, 1 << 19);
+    }
+
+    public void testMappedFile() throws IOException
+    {
+        byte[] data = new byte[1 << 20];
+        new Random().nextBytes(data);
+        ByteBuffer src = makeBB(data.length);
+        src.put(data);
+        src.flip();
+
+        // create a temp file
+        File temp = File.createTempFile("tempfile", ".tmp");
+        temp.deleteOnExit();
+
+        // Prepend some random bytes to the output and compress
+        final int outOffset = 3;
+        byte[] garbage = new byte[outOffset + compressor.initialCompressedBufferLength(data.length)];
+        new Random().nextBytes(garbage);
+        ByteBuffer dest = makeBB(outOffset + compressor.initialCompressedBufferLength(data.length));
+        dest.put(garbage);
+        dest.clear();
+        dest.position(outOffset);
+
+        compressor.compress(src, dest);
+        int compressedLength = dest.position() - outOffset;
+
+        FileChannel channel = FileChannel.open(temp.toPath(), StandardOpenOption.READ, StandardOpenOption.WRITE);
+        dest.clear();
+        channel.write(dest);
+
+        MappedByteBuffer mappedData = Files.map(temp);
+        ByteBuffer result = makeBB(data.length + 100);
+        mappedData.position(outOffset).limit(outOffset + compressedLength);
+
+        compressor.uncompress(mappedData, result);
+        channel.close();
+        result.flip();
+
+        Assert.assertEquals(data.length, result.limit());
+        for (int i = 0; i < result.limit(); i++)
+        {
+            Assert.assertEquals("Decompression mismatch at byte "+i, data[i], result.get());
+        }
+    }
+
+    @Test
+    public void testLZ4ByteBuffers() throws IOException
+    {
+        compressor = LZ4Compressor.create(Collections.<String, String>emptyMap());
+        testByteBuffers();
+    }
+
+    @Test
+    public void testDeflateByteBuffers() throws IOException
+    {
+        compressor = DeflateCompressor.create(Collections.<String, String>emptyMap());
+        testByteBuffers();
+    }
+
+    @Test
+    public void testSnappyByteBuffers() throws IOException
+    {
+        compressor = SnappyCompressor.create(Collections.<String, String>emptyMap());
+        testByteBuffers();
+    }
+
+    private void testByteBuffers() throws IOException
+    {
+        assert compressor.supports(BufferType.OFF_HEAP);
+        assert compressor.supports(compressor.preferredBufferType());
+
+        for (BufferType in: BufferType.values())
+            if (compressor.supports(in))
+                for (BufferType comp: BufferType.values())
+                    if (compressor.supports(comp))
+                        for (BufferType out: BufferType.values())
+                            if (compressor.supports(out))
+                                testByteBuffers(in, comp, out);
+    }
+
+    private void testByteBuffers(BufferType typeIn, BufferType typeComp, BufferType typeOut) throws IOException
+    {
+        try
+        {
+            int n = RandomAccessReader.DEFAULT_BUFFER_SIZE;
+            byte[] srcData = new byte[n];
+            new Random().nextBytes(srcData);
+
+            final int inOffset = 2;
+            ByteBuffer src = typeIn.allocate(inOffset + n + inOffset);
+            src.position(inOffset);
+            src.put(srcData, 0, n);
+            src.flip().position(inOffset);
+
+            int outOffset = 5;
+            ByteBuffer compressed = typeComp.allocate(outOffset + compressor.initialCompressedBufferLength(srcData.length) + outOffset);
+            byte[] garbage = new byte[compressed.capacity()];
+            new Random().nextBytes(garbage);
+            compressed.put(garbage);
+            compressed.position(outOffset).limit(compressed.capacity() - outOffset);
+
+            compressor.compress(src, compressed);
+            assertEquals(inOffset + n, src.position());
+            assertEquals(inOffset + n, src.limit());
+            assertEquals(compressed.capacity() - outOffset, compressed.limit());
+            compressed.flip().position(outOffset);
+            int len = compressed.remaining();
+
+            ByteBuffer result = typeOut.allocate(inOffset + n + inOffset);
+            result.position(inOffset).limit(result.capacity() - inOffset);
+            compressor.uncompress(compressed, result);
+            assertEquals(outOffset + len, compressed.position());
+            assertEquals(outOffset + len, compressed.limit());
+            assertEquals(result.capacity() - inOffset, result.limit());
+
+            int decompressed = result.position() - inOffset;
+            assert decompressed == n : "Failed uncompressed size";
+            for (int i = 0; i < n; ++i)
+                assert srcData[i] == result.get(inOffset + i) : "Failed comparison on index: " + i;
+        }
+        catch (Throwable e)
+        {
+            throw new AssertionError("Failed testing compressor " + compressor.getClass().getSimpleName() + " with buffer types in:" + typeIn + " compressed:" + typeComp + " out:" + typeOut, e);
+        }
+    }
+
+    private ByteBuffer makeBB(int size)
+    {
+        return compressor.preferredBufferType().allocate(size);
+    }
+
+    private void fillBBWithRandom(ByteBuffer dest)
+    {
+        byte[] random = new byte[dest.capacity()];
+        new Random().nextBytes(random);
+        dest.clear();
+        dest.put(random);
+    }
+
+}
diff --git a/test/unit/org/apache/cassandra/io/compress/LZ4CompressorTest.java b/test/unit/org/apache/cassandra/io/compress/LZ4CompressorTest.java
deleted file mode 100644
index 56ffdf1..0000000
--- a/test/unit/org/apache/cassandra/io/compress/LZ4CompressorTest.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * 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.io.compress;
-
-import java.io.IOException;
-import java.io.UnsupportedEncodingException;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Random;
-
-import org.apache.cassandra.io.compress.ICompressor.WrappedArray;
-
-import org.junit.Before;
-import org.junit.Test;
-
-import static org.junit.Assert.*;
-
-public class LZ4CompressorTest
-{
-
-    LZ4Compressor compressor;
-
-    @Before
-    public void setUp()
-    {
-        compressor = LZ4Compressor.create(Collections.<String, String>emptyMap());
-    }
-
-    public void test(byte[] data, int off, int len) throws IOException
-    {
-        final int outOffset = 3;
-        final WrappedArray out = new WrappedArray(new byte[outOffset + compressor.initialCompressedBufferLength(len)]);
-        new Random().nextBytes(out.buffer);
-        final int compressedLength = compressor.compress(data, off, len, out, outOffset);
-        final int restoredOffset = 5;
-        final byte[] restored = new byte[restoredOffset + len];
-        new Random().nextBytes(restored);
-        final int decompressedLength = compressor.uncompress(out.buffer, outOffset, compressedLength, restored, restoredOffset);
-        assertEquals(decompressedLength, len);
-        assertArrayEquals(Arrays.copyOfRange(data, off, off + len),
-                          Arrays.copyOfRange(restored, restoredOffset, restoredOffset + decompressedLength));
-    }
-
-    public void test(byte[] data) throws IOException
-    {
-        test(data, 0, data.length);
-    }
-
-    @Test
-    public void testEmptyArray() throws IOException
-    {
-        test(new byte[0]);
-    }
-
-    @Test
-    public void testShortArray() throws UnsupportedEncodingException, IOException
-    {
-        test("Cassandra".getBytes("UTF-8"), 1, 7);
-    }
-
-    @Test
-    public void testLongArray() throws UnsupportedEncodingException, IOException
-    {
-        byte[] data = new byte[1 << 20];
-        test(data, 13, 1 << 19);
-        new Random(0).nextBytes(data);
-        test(data, 13, 1 << 19);
-    }
-}
diff --git a/test/unit/org/apache/cassandra/io/sstable/BigTableWriterTest.java b/test/unit/org/apache/cassandra/io/sstable/BigTableWriterTest.java
new file mode 100644
index 0000000..dfb55a1
--- /dev/null
+++ b/test/unit/org/apache/cassandra/io/sstable/BigTableWriterTest.java
@@ -0,0 +1,130 @@
+/*
+* 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.io.sstable;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.junit.After;
+import org.junit.BeforeClass;
+
+import junit.framework.Assert;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.db.ArrayBackedSortedColumns;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.concurrent.AbstractTransactionalTest;
+
+public class BigTableWriterTest extends AbstractTransactionalTest
+{
+    public static final String KEYSPACE1 = "BigTableWriterTest";
+    public static final String CF_STANDARD = "Standard1";
+
+    private static ColumnFamilyStore cfs;
+
+    @BeforeClass
+    public static void defineSchema() throws Exception
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD));
+        cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD);
+    }
+
+    protected TestableTransaction newTest() throws IOException
+    {
+        return new TestableBTW();
+    }
+
+    private static class TestableBTW extends TestableTransaction
+    {
+        final File file;
+        final Descriptor descriptor;
+        final SSTableWriter writer;
+
+        private TestableBTW() throws IOException
+        {
+            this(cfs.getTempSSTablePath(cfs.directories.getDirectoryForNewSSTables()));
+        }
+
+        private TestableBTW(String file) throws IOException
+        {
+            this(file, SSTableWriter.create(file, 0, 0));
+        }
+
+        private TestableBTW(String file, SSTableWriter sw) throws IOException
+        {
+            super(sw);
+            this.file = new File(file);
+            this.descriptor = Descriptor.fromFilename(file);
+            this.writer = sw;
+            ArrayBackedSortedColumns cf = ArrayBackedSortedColumns.factory.create(cfs.metadata);
+            for (int i = 0; i < 10; i++)
+                cf.addColumn(Util.cellname(i), SSTableRewriterTest.random(0, 1000), 1);
+            for (int i = 0; i < 100; i++)
+                writer.append(StorageService.getPartitioner().decorateKey(ByteBufferUtil.bytes(i)), cf);
+        }
+
+        protected void assertInProgress() throws Exception
+        {
+            assertExists(Descriptor.Type.TEMP, Component.DATA, Component.PRIMARY_INDEX);
+            assertNotExists(Descriptor.Type.TEMP, Component.FILTER, Component.SUMMARY);
+            assertNotExists(Descriptor.Type.FINAL, Component.DATA, Component.PRIMARY_INDEX, Component.FILTER, Component.SUMMARY);
+            Assert.assertTrue(file.length() > 0);
+        }
+
+        protected void assertPrepared() throws Exception
+        {
+            assertNotExists(Descriptor.Type.TEMP, Component.DATA, Component.PRIMARY_INDEX, Component.FILTER, Component.SUMMARY);
+            assertExists(Descriptor.Type.FINAL, Component.DATA, Component.PRIMARY_INDEX, Component.FILTER, Component.SUMMARY);
+        }
+
+        protected void assertAborted() throws Exception
+        {
+            assertNotExists(Descriptor.Type.TEMP, Component.DATA, Component.PRIMARY_INDEX, Component.FILTER, Component.SUMMARY);
+            assertNotExists(Descriptor.Type.FINAL, Component.DATA, Component.PRIMARY_INDEX, Component.FILTER, Component.SUMMARY);
+            Assert.assertFalse(file.exists());
+        }
+
+        protected void assertCommitted() throws Exception
+        {
+            assertPrepared();
+        }
+
+        private void assertExists(Descriptor.Type type, Component ... components)
+        {
+            for (Component component : components)
+                Assert.assertTrue(new File(descriptor.asType(type).filenameFor(component)).exists());
+        }
+        private void assertNotExists(Descriptor.Type type, Component ... components)
+        {
+            for (Component component : components)
+                Assert.assertFalse(type.toString() + " " + component.toString(), new File(descriptor.asType(type).filenameFor(component)).exists());
+        }
+    }
+
+}
diff --git a/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterClientTest.java b/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterClientTest.java
index d10c9fb..ad2d876 100644
--- a/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterClientTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterClientTest.java
@@ -86,17 +86,6 @@
         writer.close();
         writer2.close();
 
-        assertContainsDataFiles(this.testDirectory, "client_test-table1", "client_test-table2");
-    }
-
-    /**
-     * Checks that the specified directory contains the files with the specified prefixes.
-     *
-     * @param directory the directory containing the data files
-     * @param prefixes the file prefixes
-     */
-    private static void assertContainsDataFiles(File directory, String... prefixes)
-    {
         FilenameFilter filter = new FilenameFilter()
         {
             @Override
@@ -106,11 +95,8 @@
             }
         };
 
-        File[] dataFiles = directory.listFiles(filter);
-        Arrays.sort(dataFiles);
+        File[] dataFiles = this.testDirectory.listFiles(filter);
+        assertEquals(2, dataFiles.length);
 
-        assertEquals(dataFiles.length, prefixes.length);
-        for (int i = 0; i < dataFiles.length; i++)
-            assertTrue(dataFiles[i].toString().contains(prefixes[i]));
     }
 }
diff --git a/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java b/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
index 9c8a2c2..5e2fffe 100644
--- a/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/CQLSSTableWriterTest.java
@@ -34,6 +34,7 @@
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
@@ -52,13 +53,14 @@
     @BeforeClass
     public static void setup() throws Exception
     {
+        DatabaseDescriptor.setDaemonInitialized();
         SchemaLoader.cleanupAndLeaveDirs();
         Keyspace.setInitialized();
         StorageService.instance.initServer();
     }
 
     @AfterClass
-    public static void tearDown()
+    public static void tearDown() throws Exception
     {
         Config.setClientMode(false);
     }
@@ -82,7 +84,7 @@
         CQLSSTableWriter writer = CQLSSTableWriter.builder()
                                                   .inDirectory(dataDir)
                                                   .forTable(schema)
-                                                  .withPartitioner(StorageService.instance.getPartitioner())
+                                                  .withPartitioner(StorageService.getPartitioner())
                                                   .using(insert).build();
 
         writer.addRow(0, "test1", 24);
@@ -93,16 +95,19 @@
 
         SSTableLoader loader = new SSTableLoader(dataDir, new SSTableLoader.Client()
         {
+            private String keyspace;
+
             public void init(String keyspace)
             {
+                this.keyspace = keyspace;
                 for (Range<Token> range : StorageService.instance.getLocalRanges("cql_keyspace"))
                     addRangeForEndpoint(range, FBUtilities.getBroadcastAddress());
                 setPartitioner(StorageService.getPartitioner());
             }
 
-            public CFMetaData getCFMetaData(String keyspace, String cfName)
+            public CFMetaData getTableMetadata(String tableName)
             {
-                return Schema.instance.getCFMetaData(keyspace, cfName);
+                return Schema.instance.getCFMetaData(keyspace, tableName);
             }
         }, new OutputHandler.SystemOutput(false, false));
 
@@ -163,7 +168,12 @@
         // Check that the write respect the buffer size even if we only insert rows withing the same partition (#7360)
         // To do that simply, we use a writer with a buffer of 1MB, and write 2 rows in the same partition with a value
         // > 1MB and validate that this created more than 1 sstable.
+        String KS = "ks";
+        String TABLE = "test";
+
         File tempdir = Files.createTempDir();
+        File dataDir = new File(tempdir.getAbsolutePath() + File.separator + KS + File.separator + TABLE);
+        assert dataDir.mkdirs();
         String schema = "CREATE TABLE ks.test ("
                       + "  k int,"
                       + "  c int,"
@@ -172,9 +182,9 @@
                       + ")";
         String insert = "INSERT INTO ks.test (k, c, v) VALUES (?, ?, ?)";
         CQLSSTableWriter writer = CQLSSTableWriter.builder()
-                                                  .inDirectory(tempdir)
+                                                  .inDirectory(dataDir)
                                                   .forTable(schema)
-                                                  .withPartitioner(StorageService.instance.getPartitioner())
+                                                  .withPartitioner(StorageService.getPartitioner())
                                                   .using(insert)
                                                   .withBufferSizeInMB(1)
                                                   .build();
@@ -192,7 +202,7 @@
                 return name.endsWith("-Data.db");
             }
         };
-        assert tempdir.list(filterDataFiles).length > 1 : Arrays.toString(tempdir.list(filterDataFiles));
+        assert dataDir.list(filterDataFiles).length > 1 : Arrays.toString(dataDir.list(filterDataFiles));
     }
 
 
@@ -270,8 +280,8 @@
     @Test
     public void testConcurrentWriters() throws Exception
     {
-        String KS = "cql_keyspace2";
-        String TABLE = "table2";
+        final String KS = "cql_keyspace2";
+        final String TABLE = "table2";
 
         File tempdir = Files.createTempDir();
         File dataDir = new File(tempdir.getAbsolutePath() + File.separator + KS + File.separator + TABLE);
@@ -297,16 +307,19 @@
 
         SSTableLoader loader = new SSTableLoader(dataDir, new SSTableLoader.Client()
         {
+            private String keyspace;
+
             public void init(String keyspace)
             {
-                for (Range<Token> range : StorageService.instance.getLocalRanges("cql_keyspace2"))
+                this.keyspace = keyspace;
+                for (Range<Token> range : StorageService.instance.getLocalRanges(KS))
                     addRangeForEndpoint(range, FBUtilities.getBroadcastAddress());
                 setPartitioner(StorageService.getPartitioner());
             }
 
-            public CFMetaData getCFMetaData(String keyspace, String cfName)
+            public CFMetaData getTableMetadata(String tableName)
             {
-                return Schema.instance.getCFMetaData(keyspace, cfName);
+                return Schema.instance.getCFMetaData(keyspace, tableName);
             }
         }, new OutputHandler.SystemOutput(false, false));
 
diff --git a/test/unit/org/apache/cassandra/io/sstable/DescriptorTest.java b/test/unit/org/apache/cassandra/io/sstable/DescriptorTest.java
new file mode 100644
index 0000000..6354fc2
--- /dev/null
+++ b/test/unit/org/apache/cassandra/io/sstable/DescriptorTest.java
@@ -0,0 +1,187 @@
+/*
+ * 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.io.sstable;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.UUID;
+
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.commons.lang3.StringUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.cassandra.db.Directories;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+
+import static org.junit.Assert.*;
+
+public class DescriptorTest
+{
+    private final String ksname = "ks";
+    private final String cfname = "cf";
+    private final String cfId = ByteBufferUtil.bytesToHex(ByteBufferUtil.bytes(UUID.randomUUID()));
+    private final File tempDataDir;
+
+    public DescriptorTest() throws IOException
+    {
+        // create CF directories, one without CFID and one with it
+        tempDataDir = File.createTempFile("DescriptorTest", null).getParentFile();
+    }
+
+    @Test
+    public void testFromFilename() throws Exception
+    {
+        File cfIdDir = new File(tempDataDir.getAbsolutePath() + File.separator + ksname + File.separator + cfname + '-' + cfId);
+        testFromFilenameFor(cfIdDir);
+    }
+
+    @Test
+    public void testFromFilenameInBackup() throws Exception
+    {
+        File backupDir = new File(StringUtils.join(new String[]{tempDataDir.getAbsolutePath(), ksname, cfname + '-' + cfId, Directories.BACKUPS_SUBDIR}, File.separator));
+        testFromFilenameFor(backupDir);
+    }
+
+    @Test
+    public void testFromFilenameInSnapshot() throws Exception
+    {
+        File snapshotDir = new File(StringUtils.join(new String[]{tempDataDir.getAbsolutePath(), ksname, cfname + '-' + cfId, Directories.SNAPSHOT_SUBDIR, "snapshot_name"}, File.separator));
+        testFromFilenameFor(snapshotDir);
+    }
+
+    @Test
+    public void testFromFilenameInLegacyDirectory() throws Exception
+    {
+        File cfDir = new File(tempDataDir.getAbsolutePath() + File.separator + ksname + File.separator + cfname);
+        testFromFilenameFor(cfDir);
+    }
+
+    private void testFromFilenameFor(File dir)
+    {
+        // normal
+        checkFromFilename(new Descriptor(dir, ksname, cfname, 1, Descriptor.Type.FINAL), false);
+        // skip component (for streaming lock file)
+        checkFromFilename(new Descriptor(dir, ksname, cfname, 2, Descriptor.Type.FINAL), true);
+        // tmp
+        checkFromFilename(new Descriptor(dir, ksname, cfname, 3, Descriptor.Type.TEMP), false);
+        // secondary index
+        String idxName = "myidx";
+        File idxDir = new File(dir.getAbsolutePath() + File.separator + Directories.SECONDARY_INDEX_NAME_SEPARATOR + idxName);
+        checkFromFilename(new Descriptor(idxDir, ksname, cfname + Directories.SECONDARY_INDEX_NAME_SEPARATOR + idxName,
+                                         4, Descriptor.Type.FINAL), false);
+        // secondary index tmp
+        checkFromFilename(new Descriptor(idxDir, ksname, cfname + Directories.SECONDARY_INDEX_NAME_SEPARATOR + idxName,
+                                         5, Descriptor.Type.TEMP), false);
+
+        // legacy version
+        checkFromFilename(new Descriptor("ja", dir, ksname, cfname, 1, Descriptor.Type.FINAL,
+                                         SSTableFormat.Type.LEGACY), false);
+        // legacy tmp
+        checkFromFilename(new Descriptor("ja", dir, ksname, cfname, 2, Descriptor.Type.TEMP, SSTableFormat.Type.LEGACY),
+                          false);
+        // legacy secondary index
+        checkFromFilename(new Descriptor("ja", dir, ksname,
+                                         cfname + Directories.SECONDARY_INDEX_NAME_SEPARATOR + idxName, 3,
+                                         Descriptor.Type.FINAL, SSTableFormat.Type.LEGACY), false);
+    }
+
+    private void checkFromFilename(Descriptor original, boolean skipComponent)
+    {
+        File file = new File(skipComponent ? original.baseFilename() : original.filenameFor(Component.DATA));
+
+        Pair<Descriptor, String> pair = Descriptor.fromFilename(file.getParentFile(), file.getName(), skipComponent);
+        Descriptor desc = pair.left;
+
+        assertEquals(original.directory, desc.directory);
+        assertEquals(original.ksname, desc.ksname);
+        assertEquals(original.cfname, desc.cfname);
+        assertEquals(original.version, desc.version);
+        assertEquals(original.generation, desc.generation);
+        assertEquals(original.type, desc.type);
+
+        if (skipComponent)
+        {
+            assertNull(pair.right);
+        }
+        else
+        {
+            assertEquals(Component.DATA.name(), pair.right);
+        }
+    }
+
+    @Test
+    public void testEquality()
+    {
+        // Descriptor should be equal when parent directory points to the same directory
+        File dir = new File(".");
+        Descriptor desc1 = new Descriptor(dir, "ks", "cf", 1, Descriptor.Type.FINAL);
+        Descriptor desc2 = new Descriptor(dir.getAbsoluteFile(), "ks", "cf", 1, Descriptor.Type.FINAL);
+        assertEquals(desc1, desc2);
+        assertEquals(desc1.hashCode(), desc2.hashCode());
+    }
+
+    @Test
+    public void validateNames()
+    {
+
+        String[] names = {
+             // old formats
+             "system-schema_keyspaces-jb-1-Data.db",
+             "system-schema_keyspaces-tmp-jb-1-Data.db",
+             "system-schema_keyspaces-ka-1-big-Data.db",
+             "system-schema_keyspaces-tmp-ka-1-big-Data.db",
+             // 2ndary index
+             "keyspace1-standard1.idx1-ka-1-big-Data.db",
+             // new formats
+             "la-1-big-Data.db",
+             "tmp-la-1-big-Data.db",
+             // 2ndary index
+             ".idx1" + File.separator + "la-1-big-Data.db",
+        };
+
+        for (String name : names)
+        {
+            assertNotNull(Descriptor.fromFilename(name));
+        }
+    }
+
+    @Test
+    public void badNames()
+    {
+        String names[] = {
+                "system-schema_keyspaces-k234a-1-CompressionInfo.db",  "system-schema_keyspaces-ka-aa-Summary.db",
+                "system-schema_keyspaces-XXX-ka-1-Data.db",             "system-schema_keyspaces-k",
+                "system-schema_keyspace-ka-1-AAA-Data.db",  "system-schema-keyspace-ka-1-AAA-Data.db"
+        };
+
+        for (String name : names)
+        {
+            try
+            {
+                Descriptor d = Descriptor.fromFilename(name);
+                Assert.fail(name);
+            } catch (Throwable e) {
+                //good
+            }
+        }
+    }
+
+
+}
diff --git a/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java b/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java
index 63928e2..a1c0e77 100644
--- a/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java
@@ -21,41 +21,48 @@
 import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.concurrent.*;
-import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
 
 import com.google.common.base.Joiner;
 import com.google.common.collect.Sets;
 import org.junit.After;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import junit.framework.Assert;
 import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.cache.CachingOptions;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.compaction.CompactionInfo;
 import org.apache.cassandra.db.compaction.CompactionInterruptedException;
 import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.db.filter.QueryFilter;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.metrics.CompactionMetrics;
 import org.apache.cassandra.metrics.RestorableMeter;
 
+import static com.google.common.collect.ImmutableMap.of;
+import static java.util.Arrays.asList;
 import static org.apache.cassandra.io.sstable.Downsampling.BASE_SAMPLING_LEVEL;
-import static org.apache.cassandra.io.sstable.IndexSummaryManager.DOWNSAMPLE_THESHOLD;
-import static org.apache.cassandra.io.sstable.IndexSummaryManager.UPSAMPLE_THRESHOLD;
-import static org.apache.cassandra.io.sstable.IndexSummaryManager.redistributeSummaries;
+import static org.apache.cassandra.io.sstable.IndexSummaryRedistribution.DOWNSAMPLE_THESHOLD;
+import static org.apache.cassandra.io.sstable.IndexSummaryRedistribution.UPSAMPLE_THRESHOLD;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
 @RunWith(OrderedJUnit4ClassRunner.class)
-public class IndexSummaryManagerTest extends SchemaLoader
+public class IndexSummaryManagerTest
 {
     private static final Logger logger = LoggerFactory.getLogger(IndexSummaryManagerTest.class);
 
@@ -63,11 +70,33 @@
     int originalMaxIndexInterval;
     long originalCapacity;
 
+    private static final String KEYSPACE1 = "IndexSummaryManagerTest";
+    // index interval of 8, no key caching
+    private static final String CF_STANDARDLOWiINTERVAL = "StandardLowIndexInterval";
+    private static final String CF_STANDARDRACE = "StandardRace";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARDLOWiINTERVAL)
+                                                .minIndexInterval(8)
+                                                .maxIndexInterval(256)
+                                                .caching(CachingOptions.NONE),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARDRACE)
+                                                .minIndexInterval(8)
+                                                .maxIndexInterval(256)
+                                                .caching(CachingOptions.NONE));
+    }
+
     @Before
     public void beforeTest()
     {
-        String ksname = "Keyspace1";
-        String cfname = "StandardLowIndexInterval"; // index interval of 8, no key caching
+        String ksname = KEYSPACE1;
+        String cfname = CF_STANDARDLOWiINTERVAL; // index interval of 8, no key caching
         Keyspace keyspace = Keyspace.open(ksname);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);
         originalMinIndexInterval = cfs.metadata.getMinIndexInterval();
@@ -78,13 +107,13 @@
     @After
     public void afterTest()
     {
-        for (CompactionInfo.Holder holder: CompactionMetrics.getCompactions())
+        for (CompactionInfo.Holder holder : CompactionMetrics.getCompactions())
         {
             holder.stop();
         }
 
-        String ksname = "Keyspace1";
-        String cfname = "StandardLowIndexInterval"; // index interval of 8, no key caching
+        String ksname = KEYSPACE1;
+        String cfname = CF_STANDARDLOWiINTERVAL; // index interval of 8, no key caching
         Keyspace keyspace = Keyspace.open(ksname);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);
         cfs.metadata.minIndexInterval(originalMinIndexInterval);
@@ -100,12 +129,15 @@
         return total;
     }
 
-    private static List<SSTableReader> resetSummaries(List<SSTableReader> sstables, long originalOffHeapSize) throws IOException
+    private static List<SSTableReader> resetSummaries(ColumnFamilyStore cfs, List<SSTableReader> sstables, long originalOffHeapSize) throws IOException
     {
         for (SSTableReader sstable : sstables)
             sstable.overrideReadMeter(new RestorableMeter(100.0, 100.0));
 
-        sstables = redistributeSummaries(Collections.EMPTY_LIST, sstables, originalOffHeapSize * sstables.size());
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+        {
+            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), originalOffHeapSize * sstables.size());
+        }
         for (SSTableReader sstable : sstables)
             assertEquals(BASE_SAMPLING_LEVEL, sstable.getIndexSummarySamplingLevel());
 
@@ -159,11 +191,8 @@
             try
             {
                 future.get();
-            } catch (InterruptedException e)
-            {
-                throw new RuntimeException(e);
             }
-            catch (ExecutionException e)
+            catch (InterruptedException | ExecutionException e)
             {
                 throw new RuntimeException(e);
             }
@@ -175,8 +204,8 @@
     @Test
     public void testChangeMinIndexInterval() throws IOException
     {
-        String ksname = "Keyspace1";
-        String cfname = "StandardLowIndexInterval"; // index interval of 8, no key caching
+        String ksname = KEYSPACE1;
+        String cfname = CF_STANDARDLOWiINTERVAL; // index interval of 8, no key caching
         Keyspace keyspace = Keyspace.open(ksname);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);
         int numSSTables = 1;
@@ -213,7 +242,11 @@
         cfs.metadata.minIndexInterval(originalMinIndexInterval / 2);
         SSTableReader sstable = cfs.getSSTables().iterator().next();
         long summarySpace = sstable.getIndexSummaryOffHeapSize();
-        IndexSummaryManager.redistributeSummaries(Collections.EMPTY_LIST, Arrays.asList(sstable), summarySpace);
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(asList(sstable), OperationType.UNKNOWN))
+        {
+            redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), summarySpace);
+        }
+
         sstable = cfs.getSSTables().iterator().next();
         assertEquals(originalMinIndexInterval, sstable.getEffectiveIndexInterval(), 0.001);
         assertEquals(numRows / originalMinIndexInterval, sstable.getIndexSummarySize());
@@ -221,7 +254,10 @@
         // keep the min_index_interval the same, but now give the summary enough space to grow by 50%
         double previousInterval = sstable.getEffectiveIndexInterval();
         int previousSize = sstable.getIndexSummarySize();
-        IndexSummaryManager.redistributeSummaries(Collections.EMPTY_LIST, Arrays.asList(sstable), (long) Math.ceil(summarySpace * 1.5));
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(asList(sstable), OperationType.UNKNOWN))
+        {
+            redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (long) Math.ceil(summarySpace * 1.5));
+        }
         sstable = cfs.getSSTables().iterator().next();
         assertEquals(previousSize * 1.5, (double) sstable.getIndexSummarySize(), 1);
         assertEquals(previousInterval * (1.0 / 1.5), sstable.getEffectiveIndexInterval(), 0.001);
@@ -229,7 +265,10 @@
         // return min_index_interval to it's original value (double it), but only give the summary enough space
         // to have an effective index interval of twice the new min
         cfs.metadata.minIndexInterval(originalMinIndexInterval);
-        IndexSummaryManager.redistributeSummaries(Collections.EMPTY_LIST, Arrays.asList(sstable), (long) Math.ceil(summarySpace / 2.0));
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(asList(sstable), OperationType.UNKNOWN))
+        {
+            redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (long) Math.ceil(summarySpace / 2.0));
+        }
         sstable = cfs.getSSTables().iterator().next();
         assertEquals(originalMinIndexInterval * 2, sstable.getEffectiveIndexInterval(), 0.001);
         assertEquals(numRows / (originalMinIndexInterval * 2), sstable.getIndexSummarySize());
@@ -239,7 +278,10 @@
         // result in an effective interval above the new max)
         cfs.metadata.minIndexInterval(originalMinIndexInterval * 4);
         cfs.metadata.maxIndexInterval(originalMinIndexInterval * 4);
-        IndexSummaryManager.redistributeSummaries(Collections.EMPTY_LIST, Arrays.asList(sstable), 10);
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(asList(sstable), OperationType.UNKNOWN))
+        {
+            redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), 10);
+        }
         sstable = cfs.getSSTables().iterator().next();
         assertEquals(cfs.metadata.getMinIndexInterval(), sstable.getEffectiveIndexInterval(), 0.001);
     }
@@ -247,8 +289,8 @@
     @Test
     public void testChangeMaxIndexInterval() throws IOException
     {
-        String ksname = "Keyspace1";
-        String cfname = "StandardLowIndexInterval"; // index interval of 8, no key caching
+        String ksname = KEYSPACE1;
+        String cfname = CF_STANDARDLOWiINTERVAL; // index interval of 8, no key caching
         Keyspace keyspace = Keyspace.open(ksname);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);
         int numSSTables = 1;
@@ -259,14 +301,20 @@
         for (SSTableReader sstable : sstables)
             sstable.overrideReadMeter(new RestorableMeter(100.0, 100.0));
 
-        IndexSummaryManager.redistributeSummaries(Collections.EMPTY_LIST, sstables, 1);
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+        {
+            redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), 10);
+        }
         sstables = new ArrayList<>(cfs.getSSTables());
         for (SSTableReader sstable : sstables)
             assertEquals(cfs.metadata.getMaxIndexInterval(), sstable.getEffectiveIndexInterval(), 0.01);
 
         // halve the max_index_interval
         cfs.metadata.maxIndexInterval(cfs.metadata.getMaxIndexInterval() / 2);
-        IndexSummaryManager.redistributeSummaries(Collections.EMPTY_LIST, sstables, 1);
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+        {
+            redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), 1);
+        }
         sstables = new ArrayList<>(cfs.getSSTables());
         for (SSTableReader sstable : sstables)
         {
@@ -276,7 +324,10 @@
 
         // return max_index_interval to its original value
         cfs.metadata.maxIndexInterval(cfs.metadata.getMaxIndexInterval() * 2);
-        IndexSummaryManager.redistributeSummaries(Collections.EMPTY_LIST, sstables, 1);
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+        {
+            redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), 1);
+        }
         for (SSTableReader sstable : cfs.getSSTables())
         {
             assertEquals(cfs.metadata.getMaxIndexInterval(), sstable.getEffectiveIndexInterval(), 0.01);
@@ -287,8 +338,8 @@
     @Test(timeout = 10000)
     public void testRedistributeSummaries() throws IOException
     {
-        String ksname = "Keyspace1";
-        String cfname = "StandardLowIndexInterval"; // index interval of 8, no key caching
+        String ksname = KEYSPACE1;
+        String cfname = CF_STANDARDLOWiINTERVAL; // index interval of 8, no key caching
         Keyspace keyspace = Keyspace.open(ksname);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);
         int numSSTables = 4;
@@ -304,7 +355,10 @@
         long singleSummaryOffHeapSpace = sstables.get(0).getIndexSummaryOffHeapSize();
 
         // there should be enough space to not downsample anything
-        sstables = redistributeSummaries(Collections.EMPTY_LIST, sstables, (singleSummaryOffHeapSpace * numSSTables));
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+        {
+            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (singleSummaryOffHeapSpace * numSSTables));
+        }
         for (SSTableReader sstable : sstables)
             assertEquals(BASE_SAMPLING_LEVEL, sstable.getIndexSummarySamplingLevel());
         assertEquals(singleSummaryOffHeapSpace * numSSTables, totalOffHeapSize(sstables));
@@ -312,26 +366,38 @@
 
         // everything should get cut in half
         assert sstables.size() == 4;
-        sstables = redistributeSummaries(Collections.EMPTY_LIST, sstables, (singleSummaryOffHeapSpace * (numSSTables / 2)));
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+        {
+            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (singleSummaryOffHeapSpace * (numSSTables / 2)));
+        }
         for (SSTableReader sstable : sstables)
             assertEquals(BASE_SAMPLING_LEVEL / 2, sstable.getIndexSummarySamplingLevel());
         validateData(cfs, numRows);
 
         // everything should get cut to a quarter
-        sstables = redistributeSummaries(Collections.EMPTY_LIST, sstables, (singleSummaryOffHeapSpace * (numSSTables / 4)));
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+        {
+            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (singleSummaryOffHeapSpace * (numSSTables / 4)));
+        }
         for (SSTableReader sstable : sstables)
             assertEquals(BASE_SAMPLING_LEVEL / 4, sstable.getIndexSummarySamplingLevel());
         validateData(cfs, numRows);
 
         // upsample back up to half
-        sstables = redistributeSummaries(Collections.EMPTY_LIST, sstables,(singleSummaryOffHeapSpace * (numSSTables / 2) + 4));
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+        {
+            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (singleSummaryOffHeapSpace * (numSSTables / 2) + 4));
+        }
         assert sstables.size() == 4;
         for (SSTableReader sstable : sstables)
             assertEquals(BASE_SAMPLING_LEVEL / 2, sstable.getIndexSummarySamplingLevel());
         validateData(cfs, numRows);
 
         // upsample back up to the original index summary
-        sstables = redistributeSummaries(Collections.EMPTY_LIST, sstables, (singleSummaryOffHeapSpace * numSSTables));
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+        {
+            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (singleSummaryOffHeapSpace * numSSTables));
+        }
         for (SSTableReader sstable : sstables)
             assertEquals(BASE_SAMPLING_LEVEL, sstable.getIndexSummarySamplingLevel());
         validateData(cfs, numRows);
@@ -340,7 +406,10 @@
         // so the two cold sstables should get downsampled to be half of their original size
         sstables.get(0).overrideReadMeter(new RestorableMeter(50.0, 50.0));
         sstables.get(1).overrideReadMeter(new RestorableMeter(50.0, 50.0));
-        sstables = redistributeSummaries(Collections.EMPTY_LIST, sstables, (singleSummaryOffHeapSpace * 3));
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+        {
+            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (singleSummaryOffHeapSpace * 3));
+        }
         Collections.sort(sstables, hotnessComparator);
         assertEquals(BASE_SAMPLING_LEVEL / 2, sstables.get(0).getIndexSummarySamplingLevel());
         assertEquals(BASE_SAMPLING_LEVEL / 2, sstables.get(1).getIndexSummarySamplingLevel());
@@ -353,7 +422,10 @@
         double higherRate = 50.0 * (UPSAMPLE_THRESHOLD - (UPSAMPLE_THRESHOLD * 0.10));
         sstables.get(0).overrideReadMeter(new RestorableMeter(lowerRate, lowerRate));
         sstables.get(1).overrideReadMeter(new RestorableMeter(higherRate, higherRate));
-        sstables = redistributeSummaries(Collections.EMPTY_LIST, sstables, (singleSummaryOffHeapSpace * 3));
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+        {
+            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (singleSummaryOffHeapSpace * 3));
+        }
         Collections.sort(sstables, hotnessComparator);
         assertEquals(BASE_SAMPLING_LEVEL / 2, sstables.get(0).getIndexSummarySamplingLevel());
         assertEquals(BASE_SAMPLING_LEVEL / 2, sstables.get(1).getIndexSummarySamplingLevel());
@@ -362,13 +434,16 @@
         validateData(cfs, numRows);
 
         // reset, and then this time, leave enough space for one of the cold sstables to not get downsampled
-        sstables = resetSummaries(sstables, singleSummaryOffHeapSpace);
+        sstables = resetSummaries(cfs, sstables, singleSummaryOffHeapSpace);
         sstables.get(0).overrideReadMeter(new RestorableMeter(1.0, 1.0));
         sstables.get(1).overrideReadMeter(new RestorableMeter(2.0, 2.0));
         sstables.get(2).overrideReadMeter(new RestorableMeter(1000.0, 1000.0));
         sstables.get(3).overrideReadMeter(new RestorableMeter(1000.0, 1000.0));
 
-        sstables = redistributeSummaries(Collections.EMPTY_LIST, sstables, (singleSummaryOffHeapSpace * 3) + 50);
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+        {
+            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (singleSummaryOffHeapSpace * 3) + 50);
+        }
         Collections.sort(sstables, hotnessComparator);
 
         if (sstables.get(0).getIndexSummarySamplingLevel() == minSamplingLevel)
@@ -389,7 +464,10 @@
         sstables.get(1).overrideReadMeter(new RestorableMeter(0.0, 0.0));
         sstables.get(2).overrideReadMeter(new RestorableMeter(92, 92));
         sstables.get(3).overrideReadMeter(new RestorableMeter(128.0, 128.0));
-        sstables = redistributeSummaries(Collections.EMPTY_LIST, sstables, (long) (singleSummaryOffHeapSpace + (singleSummaryOffHeapSpace * (92.0 / BASE_SAMPLING_LEVEL))));
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+        {
+            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (long) (singleSummaryOffHeapSpace + (singleSummaryOffHeapSpace * (92.0 / BASE_SAMPLING_LEVEL))));
+        }
         Collections.sort(sstables, hotnessComparator);
         assertEquals(1, sstables.get(0).getIndexSummarySize());  // at the min sampling level
         assertEquals(1, sstables.get(0).getIndexSummarySize());  // at the min sampling level
@@ -399,7 +477,10 @@
         validateData(cfs, numRows);
 
         // Don't leave enough space for even the minimal index summaries
-        sstables = redistributeSummaries(Collections.EMPTY_LIST, sstables, 10);
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+        {
+            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), 10);
+        }
         for (SSTableReader sstable : sstables)
             assertEquals(1, sstable.getIndexSummarySize());  // at the min sampling level
         validateData(cfs, numRows);
@@ -408,8 +489,8 @@
     @Test
     public void testRebuildAtSamplingLevel() throws IOException
     {
-        String ksname = "Keyspace1";
-        String cfname = "StandardLowIndexInterval";
+        String ksname = KEYSPACE1;
+        String cfname = CF_STANDARDLOWiINTERVAL;
         Keyspace keyspace = Keyspace.open(ksname);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);
         cfs.truncateBlocking();
@@ -423,7 +504,7 @@
             DecoratedKey key = Util.dk(String.valueOf(row));
             Mutation rm = new Mutation(ksname, key.getKey());
             rm.add(cfname, Util.cellname("column"), value, 0);
-            rm.apply();
+            rm.applyUnsafe();
         }
         cfs.forceBlockingFlush();
 
@@ -432,19 +513,19 @@
         SSTableReader original = sstables.get(0);
 
         SSTableReader sstable = original;
-        for (int samplingLevel = 1; samplingLevel < BASE_SAMPLING_LEVEL; samplingLevel++)
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(asList(sstable), OperationType.UNKNOWN))
         {
-            SSTableReader prev = sstable;
-            sstable = sstable.cloneWithNewSummarySamplingLevel(cfs, samplingLevel);
-            assertEquals(samplingLevel, sstable.getIndexSummarySamplingLevel());
-            int expectedSize = (numRows * samplingLevel) / (sstable.metadata.getMinIndexInterval() * BASE_SAMPLING_LEVEL);
-            assertEquals(expectedSize, sstable.getIndexSummarySize(), 1);
-            if (prev != original)
-                prev.selfRef().release();
+            for (int samplingLevel = 1; samplingLevel < BASE_SAMPLING_LEVEL; samplingLevel++)
+            {
+                sstable = sstable.cloneWithNewSummarySamplingLevel(cfs, samplingLevel);
+                assertEquals(samplingLevel, sstable.getIndexSummarySamplingLevel());
+                int expectedSize = (numRows * samplingLevel) / (sstable.metadata.getMinIndexInterval() * BASE_SAMPLING_LEVEL);
+                assertEquals(expectedSize, sstable.getIndexSummarySize(), 1);
+                txn.update(sstable, true);
+                txn.checkpoint();
+            }
+            txn.finish();
         }
-
-        // don't leave replaced SSTRs around to break other tests
-        cfs.getDataTracker().replaceWithNewInstances(Collections.singleton(original), Collections.singleton(sstable));
     }
 
     @Test
@@ -468,8 +549,8 @@
         manager.setMemoryPoolCapacityInMB(10);
         assertEquals(10, manager.getMemoryPoolCapacityInMB());
 
-        String ksname = "Keyspace1";
-        String cfname = "StandardLowIndexInterval"; // index interval of 8, no key caching
+        String ksname = KEYSPACE1;
+        String cfname = CF_STANDARDLOWiINTERVAL; // index interval of 8, no key caching
         Keyspace keyspace = Keyspace.open(ksname);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);
         cfs.truncateBlocking();
@@ -486,7 +567,7 @@
                 DecoratedKey key = Util.dk(String.valueOf(row));
                 Mutation rm = new Mutation(ksname, key.getKey());
                 rm.add(cfname, Util.cellname("column"), value, 0);
-                rm.apply();
+                rm.applyUnsafe();
             }
             cfs.forceBlockingFlush();
         }
@@ -494,7 +575,7 @@
         assertTrue(manager.getAverageIndexInterval() >= cfs.metadata.getMinIndexInterval());
         Map<String, Integer> intervals = manager.getIndexIntervals();
         for (Map.Entry<String, Integer> entry : intervals.entrySet())
-            if (entry.getKey().contains("StandardLowIndexInterval"))
+            if (entry.getKey().contains(CF_STANDARDLOWiINTERVAL))
                 assertEquals(cfs.metadata.getMinIndexInterval(), entry.getValue(), 0.001);
 
         manager.setMemoryPoolCapacityInMB(0);
@@ -503,7 +584,7 @@
         intervals = manager.getIndexIntervals();
         for (Map.Entry<String, Integer> entry : intervals.entrySet())
         {
-            if (entry.getKey().contains("StandardLowIndexInterval"))
+            if (entry.getKey().contains(CF_STANDARDLOWiINTERVAL))
                 assertTrue(entry.getValue() >= cfs.metadata.getMinIndexInterval());
         }
     }
@@ -511,11 +592,12 @@
     @Test
     public void testCancelIndex() throws Exception
     {
-        String ksname = "Keyspace1";
-        String cfname = "StandardLowIndexInterval"; // index interval of 8, no key caching
+        String ksname = KEYSPACE1;
+        String cfname = CF_STANDARDLOWiINTERVAL; // index interval of 8, no key caching
         Keyspace keyspace = Keyspace.open(ksname);
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);
+        final ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);
         final int numSSTables = 4;
+        final int numTries = 4;
         int numRows = 256;
         createSSTables(ksname, cfname, numSSTables, numRows);
 
@@ -527,13 +609,23 @@
 
         // everything should get cut in half
         final AtomicReference<CompactionInterruptedException> exception = new AtomicReference<>();
+        // barrier to control when redistribution runs
+        final CountDownLatch barrier = new CountDownLatch(1);
+
         Thread t = new Thread(new Runnable()
         {
             public void run()
             {
                 try
                 {
-                    redistributeSummaries(Collections.<SSTableReader>emptyList(), sstables, (singleSummaryOffHeapSpace * (numSSTables / 2)));
+                    // Don't leave enough space for even the minimal index summaries
+                    try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+                    {
+                        IndexSummaryManager.redistributeSummaries(new ObservableRedistribution(Collections.EMPTY_LIST,
+                                                                                               of(cfs.metadata.cfId, txn),
+                                                                                               singleSummaryOffHeapSpace,
+                                                                                               barrier));
+                    }
                 }
                 catch (CompactionInterruptedException ex)
                 {
@@ -546,8 +638,13 @@
         });
         t.start();
         while (CompactionManager.instance.getActiveCompactions() == 0 && t.isAlive())
-            Thread.sleep(1);
+            Thread.yield();
+        // to ensure that the stop condition check in IndexSummaryRedistribution::redistributeSummaries
+        // is made *after* the halt request is made to the CompactionManager, don't allow the redistribution
+        // to proceed until stopCompaction has been called.
         CompactionManager.instance.stopCompaction("INDEX_SUMMARY");
+        // allows the redistribution to proceed
+        barrier.countDown();
         t.join();
 
         assertNotNull("Expected compaction interrupted exception", exception.get());
@@ -562,4 +659,41 @@
 
         validateData(cfs, numRows);
     }
+
+    private static List<SSTableReader> redistributeSummaries(List<SSTableReader> compacting,
+                                                             Map<UUID, LifecycleTransaction> transactions,
+                                                             long memoryPoolBytes)
+    throws IOException
+    {
+        return IndexSummaryManager.redistributeSummaries(new IndexSummaryRedistribution(compacting,
+                                                                                        transactions,
+                                                                                        memoryPoolBytes));
+    }
+
+    private static class ObservableRedistribution extends IndexSummaryRedistribution
+    {
+        CountDownLatch barrier;
+
+        ObservableRedistribution(List<SSTableReader> compacting,
+                                 Map<UUID, LifecycleTransaction> transactions,
+                                 long memoryPoolBytes,
+                                 CountDownLatch barrier)
+        {
+            super(compacting, transactions, memoryPoolBytes);
+            this.barrier = barrier;
+        }
+
+        public List<SSTableReader> redistributeSummaries() throws IOException
+        {
+            try
+            {
+                barrier.await();
+            }
+            catch (InterruptedException e)
+            {
+                throw new RuntimeException("Interrupted waiting on test barrier");
+            }
+            return super.redistributeSummaries();
+        }
+    }
 }
diff --git a/test/unit/org/apache/cassandra/io/sstable/IndexSummaryTest.java b/test/unit/org/apache/cassandra/io/sstable/IndexSummaryTest.java
index c9322b0..7442a22 100644
--- a/test/unit/org/apache/cassandra/io/sstable/IndexSummaryTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/IndexSummaryTest.java
@@ -30,6 +30,7 @@
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.RandomPartitioner;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -89,6 +90,7 @@
         assertEquals(dis.readUTF(), "JUNK");
         is.close();
         FileUtils.closeQuietly(dis);
+        random.right.close();
     }
 
     @Test
@@ -133,6 +135,10 @@
             IndexSummary summary = builder.build(DatabaseDescriptor.getPartitioner());
             return Pair.create(list, summary);
         }
+        catch (IOException e)
+        {
+            throw new RuntimeException(e);
+        }
     }
 
     @Test
diff --git a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java b/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
index 51d695f..f4b9617 100644
--- a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
@@ -26,21 +26,27 @@
 import java.util.List;
 import java.util.Set;
 
+import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.Version;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.ColumnFamily;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.DeletionInfo;
 import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.columniterator.SSTableNamesIterator;
 import org.apache.cassandra.db.composites.CellNameType;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.streaming.StreamPlan;
 import org.apache.cassandra.streaming.StreamSession;
@@ -50,7 +56,7 @@
 /**
  * Tests backwards compatibility for SSTables
  */
-public class LegacySSTableTest extends SchemaLoader
+public class LegacySSTableTest
 {
     public static final String LEGACY_SSTABLE_PROP = "legacy-sstable-root";
     public static final String KSNAME = "Keyspace1";
@@ -60,6 +66,16 @@
     public static File LEGACY_SSTABLE_ROOT;
 
     @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KSNAME,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KSNAME, CFNAME));
+        beforeClass();
+    }
+
     public static void beforeClass()
     {
         Keyspace.setInitialized();
@@ -79,7 +95,7 @@
     protected Descriptor getDescriptor(String ver)
     {
         File directory = new File(LEGACY_SSTABLE_ROOT + File.separator + ver + File.separator + KSNAME);
-        return new Descriptor(ver, directory, KSNAME, CFNAME, 0, Descriptor.Type.FINAL);
+        return new Descriptor(ver, directory, KSNAME, CFNAME, 0, Descriptor.Type.FINAL, SSTableFormat.Type.LEGACY);
     }
 
     /**
@@ -106,7 +122,7 @@
         StorageService.instance.initServer();
 
         for (File version : LEGACY_SSTABLE_ROOT.listFiles())
-            if (Descriptor.Version.validate(version.getName()) && new Descriptor.Version(version.getName()).isCompatible())
+            if (Version.validate(version.getName()) && SSTableFormat.Type.LEGACY.info.getVersion(version.getName()).isCompatible())
                 testStreaming(version.getName());
     }
 
@@ -131,7 +147,7 @@
         for (String keystring : TEST_DATA)
         {
             ByteBuffer key = ByteBufferUtil.bytes(keystring);
-            SSTableNamesIterator iter = new SSTableNamesIterator(sstable, Util.dk(key), FBUtilities.singleton(Util.cellname(key), type));
+            OnDiskAtomIterator iter = sstable.iterator(Util.dk(key), FBUtilities.singleton(Util.cellname(key), type));
             ColumnFamily cf = iter.getColumnFamily();
 
             // check not deleted (CASSANDRA-6527)
@@ -148,7 +164,7 @@
 
         for (File version : LEGACY_SSTABLE_ROOT.listFiles())
         {
-            if (Descriptor.Version.validate(version.getName()) && new Descriptor.Version(version.getName()).isCompatible())
+            if (Version.validate(version.getName()) && SSTableFormat.Type.LEGACY.info.getVersion(version.getName()).isCompatible())
             {
                 notSkipped = true;
                 testVersion(version.getName());
@@ -169,7 +185,7 @@
                 ByteBuffer key = ByteBufferUtil.bytes(keystring);
                 // confirm that the bloom filter does not reject any keys/names
                 DecoratedKey dk = reader.partitioner.decorateKey(key);
-                SSTableNamesIterator iter = new SSTableNamesIterator(reader, dk, FBUtilities.singleton(Util.cellname(key), type));
+                OnDiskAtomIterator iter = reader.iterator(dk, FBUtilities.singleton(Util.cellname(key), type));
                 assert iter.next().name().toByteBuffer().equals(key);
             }
 
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java
index 39beb94..4a51fbd 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java
@@ -27,12 +27,14 @@
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.Row;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
@@ -40,12 +42,24 @@
 
 import static org.junit.Assert.assertEquals;
 
-public class SSTableLoaderTest extends SchemaLoader
+public class SSTableLoaderTest
 {
+    public static final String KEYSPACE1 = "SSTableLoaderTest";
+    public static final String CF_STANDARD = "Standard1";
+
     @BeforeClass
+    public static void defineSchema() throws Exception
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD));
+        setup();
+    }
+
     public static void setup() throws Exception
     {
-        Keyspace.setInitialized();
         StorageService.instance.initServer();
     }
 
@@ -53,36 +67,41 @@
     public void testLoadingSSTable() throws Exception
     {
         File tempdir = Files.createTempDir();
-        File dataDir = new File(tempdir.getAbsolutePath() + File.separator + "Keyspace1" + File.separator + "Standard1");
+        File dataDir = new File(tempdir.getAbsolutePath() + File.separator + KEYSPACE1 + File.separator + CF_STANDARD);
         assert dataDir.mkdirs();
-        CFMetaData cfmeta = Schema.instance.getCFMetaData("Keyspace1", "Standard1");
-        SSTableSimpleUnsortedWriter writer = new SSTableSimpleUnsortedWriter(dataDir,
+        CFMetaData cfmeta = Schema.instance.getCFMetaData(KEYSPACE1, CF_STANDARD);
+        DecoratedKey key = Util.dk("key1");
+        
+        try (SSTableSimpleUnsortedWriter writer = new SSTableSimpleUnsortedWriter(dataDir,
                                                                              cfmeta,
                                                                              StorageService.getPartitioner(),
-                                                                             1);
-        DecoratedKey key = Util.dk("key1");
-        writer.newRow(key.getKey());
-        writer.addColumn(ByteBufferUtil.bytes("col1"), ByteBufferUtil.bytes(100), 1);
-        writer.close();
+                                                                             1))
+        {
+            writer.newRow(key.getKey());
+            writer.addColumn(ByteBufferUtil.bytes("col1"), ByteBufferUtil.bytes(100), 1);
+        }
 
         SSTableLoader loader = new SSTableLoader(dataDir, new SSTableLoader.Client()
         {
+            private String keyspace;
+
             public void init(String keyspace)
             {
-                for (Range<Token> range : StorageService.instance.getLocalRanges("Keyspace1"))
+                this.keyspace = keyspace;
+                for (Range<Token> range : StorageService.instance.getLocalRanges(KEYSPACE1))
                     addRangeForEndpoint(range, FBUtilities.getBroadcastAddress());
                 setPartitioner(StorageService.getPartitioner());
             }
 
-            public CFMetaData getCFMetaData(String keyspace, String cfName)
+            public CFMetaData getTableMetadata(String tableName)
             {
-                return Schema.instance.getCFMetaData(keyspace, cfName);
+                return Schema.instance.getCFMetaData(keyspace, tableName);
             }
         }, new OutputHandler.SystemOutput(false, false));
 
         loader.stream().get();
 
-        List<Row> rows = Util.getRangeSlice(Keyspace.open("Keyspace1").getColumnFamilyStore("Standard1"));
+        List<Row> rows = Util.getRangeSlice(Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD));
         assertEquals(1, rows.size());
         assertEquals(key, rows.get(0).key);
         assertEquals(ByteBufferUtil.bytes(100), rows.get(0).cf.getColumn(Util.cellname("col1")).value());
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataTest.java
index 72307c5..755225e 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataTest.java
@@ -23,15 +23,23 @@
 import java.nio.ByteBuffer;
 import java.nio.charset.CharacterCodingException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 import java.util.concurrent.ExecutionException;
+
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.db.context.CounterContext;
+import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.CounterId;
 
@@ -41,31 +49,53 @@
 
 import static org.apache.cassandra.Util.cellname;
 
-public class SSTableMetadataTest extends SchemaLoader
+public class SSTableMetadataTest
 {
+    public static final String KEYSPACE1 = "SSTableMetadataTest";
+    public static final String CF_STANDARD = "Standard1";
+    public static final String CF_STANDARD2 = "Standard2";
+    public static final String CF_STANDARD3 = "Standard3";
+    public static final String CF_STANDARDCOMPOSITE2 = "StandardComposite2";
+    public static final String CF_COUNTER1 = "Counter1";
+
+    @BeforeClass
+    public static void defineSchema() throws Exception
+    {
+        AbstractType<?> compositeMaxMin = CompositeType.getInstance(Arrays.asList(new AbstractType<?>[]{BytesType.instance, IntegerType.instance}));
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD2),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD3),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARDCOMPOSITE2, compositeMaxMin),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_COUNTER1).defaultValidator(CounterColumnType.instance));
+    }
+
     @Test
     public void testTrackMaxDeletionTime()
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
         long timestamp = System.currentTimeMillis();
         for(int i = 0; i < 10; i++)
         {
             DecoratedKey key = Util.dk(Integer.toString(i));
-            Mutation rm = new Mutation("Keyspace1", key.getKey());
+            Mutation rm = new Mutation(KEYSPACE1, key.getKey());
             for (int j = 0; j < 10; j++)
                 rm.add("Standard1", cellname(Integer.toString(j)),
                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
                        timestamp,
                        10 + j);
-            rm.apply();
+            rm.applyUnsafe();
         }
-        Mutation rm = new Mutation("Keyspace1", Util.dk("longttl").getKey());
+        Mutation rm = new Mutation(KEYSPACE1, Util.dk("longttl").getKey());
         rm.add("Standard1", cellname("col"),
                ByteBufferUtil.EMPTY_BYTE_BUFFER,
                timestamp,
                10000);
-        rm.apply();
+        rm.applyUnsafe();
         store.forceBlockingFlush();
         assertEquals(1, store.getSSTables().size());
         int ttltimestamp = (int)(System.currentTimeMillis()/1000);
@@ -76,12 +106,12 @@
             assertEquals(ttltimestamp + 10000, firstDelTime, 10);
 
         }
-        rm = new Mutation("Keyspace1", Util.dk("longttl2").getKey());
+        rm = new Mutation(KEYSPACE1, Util.dk("longttl2").getKey());
         rm.add("Standard1", cellname("col"),
                ByteBufferUtil.EMPTY_BYTE_BUFFER,
                timestamp,
                20000);
-        rm.apply();
+        rm.applyUnsafe();
         ttltimestamp = (int) (System.currentTimeMillis()/1000);
         store.forceBlockingFlush();
         assertEquals(2, store.getSSTables().size());
@@ -119,11 +149,11 @@
     @Test
     public void testWithDeletes() throws ExecutionException, InterruptedException
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard2");
         long timestamp = System.currentTimeMillis();
         DecoratedKey key = Util.dk("deletetest");
-        Mutation rm = new Mutation("Keyspace1", key.getKey());
+        Mutation rm = new Mutation(KEYSPACE1, key.getKey());
         for (int i = 0; i<5; i++)
             rm.add("Standard2", cellname("deletecolumn" + i),
                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
@@ -133,7 +163,7 @@
                    ByteBufferUtil.EMPTY_BYTE_BUFFER,
                    timestamp,
                    1000);
-        rm.apply();
+        rm.applyUnsafe();
         store.forceBlockingFlush();
         assertEquals(1,store.getSSTables().size());
         int ttltimestamp = (int) (System.currentTimeMillis()/1000);
@@ -143,9 +173,9 @@
             firstMaxDelTime = sstable.getSSTableMetadata().maxLocalDeletionTime;
             assertEquals(ttltimestamp + 1000, firstMaxDelTime, 10);
         }
-        rm = new Mutation("Keyspace1", key.getKey());
+        rm = new Mutation(KEYSPACE1, key.getKey());
         rm.delete("Standard2", cellname("todelete"), timestamp + 1);
-        rm.apply();
+        rm.applyUnsafe();
         store.forceBlockingFlush();
         assertEquals(2,store.getSSTables().size());
         boolean foundDelete = false;
@@ -169,18 +199,18 @@
     @Test
     public void trackMaxMinColNames() throws CharacterCodingException, ExecutionException, InterruptedException
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard3");
         store.getCompactionStrategy();
         for (int j = 0; j < 8; j++)
         {
             DecoratedKey key = Util.dk("row"+j);
-            Mutation rm = new Mutation("Keyspace1", key.getKey());
+            Mutation rm = new Mutation(KEYSPACE1, key.getKey());
             for (int i = 100; i<150; i++)
             {
                 rm.add("Standard3", cellname(j + "col" + i), ByteBufferUtil.EMPTY_BYTE_BUFFER, System.currentTimeMillis());
             }
-            rm.apply();
+            rm.applyUnsafe();
         }
         store.forceBlockingFlush();
         assertEquals(1, store.getSSTables().size());
@@ -190,12 +220,12 @@
             assertEquals(ByteBufferUtil.string(sstable.getSSTableMetadata().maxColumnNames.get(0)), "7col149");
         }
         DecoratedKey key = Util.dk("row2");
-        Mutation rm = new Mutation("Keyspace1", key.getKey());
+        Mutation rm = new Mutation(KEYSPACE1, key.getKey());
         for (int i = 101; i<299; i++)
         {
             rm.add("Standard3", cellname(9 + "col" + i), ByteBufferUtil.EMPTY_BYTE_BUFFER, System.currentTimeMillis());
         }
-        rm.apply();
+        rm.applyUnsafe();
 
         store.forceBlockingFlush();
         store.forceMajorCompaction();
@@ -221,7 +251,7 @@
         ---------------------
         meaning max columns are b9 and 9, min is a0 and 0
          */
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
 
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("StandardComposite2");
 
@@ -230,20 +260,20 @@
         ByteBuffer key = ByteBufferUtil.bytes("k");
         for (int i = 0; i < 10; i++)
         {
-            Mutation rm = new Mutation("Keyspace1", key);
+            Mutation rm = new Mutation(KEYSPACE1, key);
             CellName colName = type.makeCellName(ByteBufferUtil.bytes("a"+(9-i)), ByteBufferUtil.bytes(i));
             rm.add("StandardComposite2", colName, ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
-            rm.apply();
+            rm.applyUnsafe();
         }
         cfs.forceBlockingFlush();
 
         key = ByteBufferUtil.bytes("k2");
         for (int i = 0; i < 10; i++)
         {
-            Mutation rm = new Mutation("Keyspace1", key);
+            Mutation rm = new Mutation(KEYSPACE1, key);
             CellName colName = type.makeCellName(ByteBufferUtil.bytes("b"+(9-i)), ByteBufferUtil.bytes(i));
             rm.add("StandardComposite2", colName, ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
-            rm.apply();
+            rm.applyUnsafe();
         }
         cfs.forceBlockingFlush();
         cfs.forceMajorCompaction();
@@ -260,7 +290,7 @@
     @Test
     public void testLegacyCounterShardTracking()
     {
-        ColumnFamilyStore cfs = Keyspace.open("Keyspace1").getColumnFamilyStore("Counter1");
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore("Counter1");
 
         // A cell with all shards
         CounterContext.ContextState state = CounterContext.ContextState.allocate(1, 1, 1);
@@ -269,7 +299,7 @@
         state.writeRemote(CounterId.fromInt(3), 1L, 1L);
         ColumnFamily cells = ArrayBackedSortedColumns.factory.create(cfs.metadata);
         cells.addColumn(new BufferCounterCell(cellname("col"), state.context, 1L, Long.MIN_VALUE));
-        new Mutation(Util.dk("k").getKey(), cells).apply();
+        new Mutation(Util.dk("k").getKey(), cells).applyUnsafe();
         cfs.forceBlockingFlush();
         assertTrue(cfs.getSSTables().iterator().next().getSSTableMetadata().hasLegacyCounterShards);
         cfs.truncateBlocking();
@@ -280,7 +310,7 @@
         state.writeRemote(CounterId.fromInt(3), 1L, 1L);
         cells = ArrayBackedSortedColumns.factory.create(cfs.metadata);
         cells.addColumn(new BufferCounterCell(cellname("col"), state.context, 1L, Long.MIN_VALUE));
-        new Mutation(Util.dk("k").getKey(), cells).apply();
+        new Mutation(Util.dk("k").getKey(), cells).applyUnsafe();
         cfs.forceBlockingFlush();
         assertTrue(cfs.getSSTables().iterator().next().getSSTableMetadata().hasLegacyCounterShards);
         cfs.truncateBlocking();
@@ -291,7 +321,7 @@
         state.writeLocal(CounterId.fromInt(2), 1L, 1L);
         cells = ArrayBackedSortedColumns.factory.create(cfs.metadata);
         cells.addColumn(new BufferCounterCell(cellname("col"), state.context, 1L, Long.MIN_VALUE));
-        new Mutation(Util.dk("k").getKey(), cells).apply();
+        new Mutation(Util.dk("k").getKey(), cells).applyUnsafe();
         cfs.forceBlockingFlush();
         assertTrue(cfs.getSSTables().iterator().next().getSSTableMetadata().hasLegacyCounterShards);
         cfs.truncateBlocking();
@@ -301,7 +331,7 @@
         state.writeGlobal(CounterId.fromInt(1), 1L, 1L);
         cells = ArrayBackedSortedColumns.factory.create(cfs.metadata);
         cells.addColumn(new BufferCounterCell(cellname("col"), state.context, 1L, Long.MIN_VALUE));
-        new Mutation(Util.dk("k").getKey(), cells).apply();
+        new Mutation(Util.dk("k").getKey(), cells).applyUnsafe();
         cfs.forceBlockingFlush();
         assertFalse(cfs.getSSTables().iterator().next().getSSTableMetadata().hasLegacyCounterShards);
         cfs.truncateBlocking();
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
deleted file mode 100644
index a037599..0000000
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
+++ /dev/null
@@ -1,510 +0,0 @@
-package org.apache.cassandra.io.sstable;
-/*
- *
- * 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.
- *
- */
-
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.ThreadPoolExecutor;
-
-import com.google.common.collect.Sets;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
-import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.Util;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.cql3.Operator;
-import org.apache.cassandra.db.BufferDecoratedKey;
-import org.apache.cassandra.db.ColumnFamily;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.DecoratedKey;
-import org.apache.cassandra.db.IndexExpression;
-import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.Mutation;
-import org.apache.cassandra.db.Row;
-import org.apache.cassandra.db.RowPosition;
-import org.apache.cassandra.db.columniterator.IdentityQueryFilter;
-import org.apache.cassandra.db.compaction.CompactionManager;
-import org.apache.cassandra.db.composites.Composites;
-import org.apache.cassandra.dht.LocalPartitioner;
-import org.apache.cassandra.dht.LocalToken;
-import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.io.util.FileDataInput;
-import org.apache.cassandra.io.util.MmappedSegmentedFile;
-import org.apache.cassandra.io.util.SegmentedFile;
-import org.apache.cassandra.service.CacheService;
-import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.Pair;
-import static org.apache.cassandra.Util.cellname;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-@RunWith(OrderedJUnit4ClassRunner.class)
-public class SSTableReaderTest extends SchemaLoader
-{
-    private static final Logger logger = LoggerFactory.getLogger(SSTableReaderTest.class);
-
-    static Token t(int i)
-    {
-        return StorageService.getPartitioner().getToken(ByteBufferUtil.bytes(String.valueOf(i)));
-    }
-
-    @Test
-    public void testGetPositionsForRanges() throws ExecutionException, InterruptedException
-    {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard2");
-
-        // insert data and compact to a single sstable
-        CompactionManager.instance.disableAutoCompaction();
-        for (int j = 0; j < 10; j++)
-        {
-            ByteBuffer key = ByteBufferUtil.bytes(String.valueOf(j));
-            Mutation rm = new Mutation("Keyspace1", key);
-            rm.add("Standard2", cellname("0"), ByteBufferUtil.EMPTY_BYTE_BUFFER, j);
-            rm.apply();
-        }
-        store.forceBlockingFlush();
-        CompactionManager.instance.performMaximal(store);
-
-        List<Range<Token>> ranges = new ArrayList<Range<Token>>();
-        // 1 key
-        ranges.add(new Range<Token>(t(0), t(1)));
-        // 2 keys
-        ranges.add(new Range<Token>(t(2), t(4)));
-        // wrapping range from key to end
-        ranges.add(new Range<Token>(t(6), StorageService.getPartitioner().getMinimumToken()));
-        // empty range (should be ignored)
-        ranges.add(new Range<Token>(t(9), t(91)));
-
-        // confirm that positions increase continuously
-        SSTableReader sstable = store.getSSTables().iterator().next();
-        long previous = -1;
-        for (Pair<Long,Long> section : sstable.getPositionsForRanges(ranges))
-        {
-            assert previous <= section.left : previous + " ! < " + section.left;
-            assert section.left < section.right : section.left + " ! < " + section.right;
-            previous = section.right;
-        }
-    }
-
-    @Test
-    public void testSpannedIndexPositions() throws IOException, ExecutionException, InterruptedException
-    {
-        MmappedSegmentedFile.MAX_SEGMENT_SIZE = 40; // each index entry is ~11 bytes, so this will generate lots of segments
-
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
-
-        // insert a bunch of data and compact to a single sstable
-        CompactionManager.instance.disableAutoCompaction();
-        for (int j = 0; j < 100; j += 2)
-        {
-            ByteBuffer key = ByteBufferUtil.bytes(String.valueOf(j));
-            Mutation rm = new Mutation("Keyspace1", key);
-            rm.add("Standard1", cellname("0"), ByteBufferUtil.EMPTY_BYTE_BUFFER, j);
-            rm.apply();
-        }
-        store.forceBlockingFlush();
-        CompactionManager.instance.performMaximal(store);
-
-        // check that all our keys are found correctly
-        SSTableReader sstable = store.getSSTables().iterator().next();
-        for (int j = 0; j < 100; j += 2)
-        {
-            DecoratedKey dk = Util.dk(String.valueOf(j));
-            FileDataInput file = sstable.getFileDataInput(sstable.getPosition(dk, SSTableReader.Operator.EQ).position);
-            DecoratedKey keyInDisk = sstable.partitioner.decorateKey(ByteBufferUtil.readWithShortLength(file));
-            assert keyInDisk.equals(dk) : String.format("%s != %s in %s", keyInDisk, dk, file.getPath());
-        }
-
-        // check no false positives
-        for (int j = 1; j < 110; j += 2)
-        {
-            DecoratedKey dk = Util.dk(String.valueOf(j));
-            assert sstable.getPosition(dk, SSTableReader.Operator.EQ) == null;
-        }
-    }
-
-    @Test
-    public void testPersistentStatistics()
-    {
-
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
-
-        for (int j = 0; j < 100; j += 2)
-        {
-            ByteBuffer key = ByteBufferUtil.bytes(String.valueOf(j));
-            Mutation rm = new Mutation("Keyspace1", key);
-            rm.add("Standard1", cellname("0"), ByteBufferUtil.EMPTY_BYTE_BUFFER, j);
-            rm.apply();
-        }
-        store.forceBlockingFlush();
-
-        clearAndLoad(store);
-        assert store.getMaxRowSize() != 0;
-    }
-
-    private void clearAndLoad(ColumnFamilyStore cfs)
-    {
-        cfs.clearUnsafe();
-        cfs.loadNewSSTables();
-    }
-
-    @Test
-    public void testReadRateTracking()
-    {
-        // try to make sure CASSANDRA-8239 never happens again
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
-
-        for (int j = 0; j < 10; j++)
-        {
-            ByteBuffer key = ByteBufferUtil.bytes(String.valueOf(j));
-            Mutation rm = new Mutation("Keyspace1", key);
-            rm.add("Standard1", cellname("0"), ByteBufferUtil.EMPTY_BYTE_BUFFER, j);
-            rm.apply();
-        }
-        store.forceBlockingFlush();
-
-        SSTableReader sstable = store.getSSTables().iterator().next();
-        assertEquals(0, sstable.getReadMeter().count());
-
-        DecoratedKey key = sstable.partitioner.decorateKey(ByteBufferUtil.bytes("4"));
-        store.getColumnFamily(key, Composites.EMPTY, Composites.EMPTY, false, 100, 100);
-        assertEquals(1, sstable.getReadMeter().count());
-        store.getColumnFamily(key, cellname("0"), cellname("0"), false, 100, 100);
-        assertEquals(2, sstable.getReadMeter().count());
-        store.getColumnFamily(Util.namesQueryFilter(store, key, cellname("0")));
-        assertEquals(3, sstable.getReadMeter().count());
-    }
-
-    @Test
-    public void testGetPositionsForRangesWithKeyCache() throws ExecutionException, InterruptedException
-    {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard2");
-        CacheService.instance.keyCache.setCapacity(100);
-
-        // insert data and compact to a single sstable
-        CompactionManager.instance.disableAutoCompaction();
-        for (int j = 0; j < 10; j++)
-        {
-            ByteBuffer key = ByteBufferUtil.bytes(String.valueOf(j));
-            Mutation rm = new Mutation("Keyspace1", key);
-            rm.add("Standard2", cellname("0"), ByteBufferUtil.EMPTY_BYTE_BUFFER, j);
-            rm.apply();
-        }
-        store.forceBlockingFlush();
-        CompactionManager.instance.performMaximal(store);
-
-        SSTableReader sstable = store.getSSTables().iterator().next();
-        long p2 = sstable.getPosition(k(2), SSTableReader.Operator.EQ).position;
-        long p3 = sstable.getPosition(k(3), SSTableReader.Operator.EQ).position;
-        long p6 = sstable.getPosition(k(6), SSTableReader.Operator.EQ).position;
-        long p7 = sstable.getPosition(k(7), SSTableReader.Operator.EQ).position;
-
-        Pair<Long, Long> p = sstable.getPositionsForRanges(makeRanges(t(2), t(6))).iterator().next();
-
-        // range are start exclusive so we should start at 3
-        assert p.left == p3;
-
-        // to capture 6 we have to stop at the start of 7
-        assert p.right == p7;
-    }
-
-    @Test
-    public void testPersistentStatisticsWithSecondaryIndex()
-    {
-        // Create secondary index and flush to disk
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Indexed1");
-        ByteBuffer key = ByteBufferUtil.bytes(String.valueOf("k1"));
-        Mutation rm = new Mutation("Keyspace1", key);
-        rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(1L), System.currentTimeMillis());
-        rm.apply();
-        store.forceBlockingFlush();
-
-        // check if opening and querying works
-        assertIndexQueryWorks(store);
-    }
-    public void testGetPositionsKeyCacheStats() throws IOException, ExecutionException, InterruptedException
-    {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard2");
-        CacheService.instance.keyCache.setCapacity(1000);
-
-        // insert data and compact to a single sstable
-        CompactionManager.instance.disableAutoCompaction();
-        for (int j = 0; j < 10; j++)
-        {
-            ByteBuffer key = ByteBufferUtil.bytes(String.valueOf(j));
-            Mutation rm = new Mutation("Keyspace1", key);
-            rm.add("Standard2", cellname("0"), ByteBufferUtil.EMPTY_BYTE_BUFFER, j);
-            rm.apply();
-        }
-        store.forceBlockingFlush();
-        CompactionManager.instance.performMaximal(store);
-
-        SSTableReader sstable = store.getSSTables().iterator().next();
-        sstable.getPosition(k(2), SSTableReader.Operator.EQ);
-        assertEquals(0, sstable.getKeyCacheHit());
-        assertEquals(1, sstable.getBloomFilterTruePositiveCount());
-        sstable.getPosition(k(2), SSTableReader.Operator.EQ);
-        assertEquals(1, sstable.getKeyCacheHit());
-        assertEquals(2, sstable.getBloomFilterTruePositiveCount());
-        sstable.getPosition(k(15), SSTableReader.Operator.EQ);
-        assertEquals(1, sstable.getKeyCacheHit());
-        assertEquals(2, sstable.getBloomFilterTruePositiveCount());
-
-    }
-
-
-    @Test
-    public void testOpeningSSTable() throws Exception
-    {
-        String ks = "Keyspace1";
-        String cf = "Standard1";
-
-        // clear and create just one sstable for this test
-        Keyspace keyspace = Keyspace.open(ks);
-        ColumnFamilyStore store = keyspace.getColumnFamilyStore(cf);
-        store.clearUnsafe();
-        store.disableAutoCompaction();
-
-        DecoratedKey firstKey = null, lastKey = null;
-        long timestamp = System.currentTimeMillis();
-        for (int i = 0; i < store.metadata.getMinIndexInterval(); i++)
-        {
-            DecoratedKey key = Util.dk(String.valueOf(i));
-            if (firstKey == null)
-                firstKey = key;
-            if (lastKey == null)
-                lastKey = key;
-            if (store.metadata.getKeyValidator().compare(lastKey.getKey(), key.getKey()) < 0)
-                lastKey = key;
-            Mutation rm = new Mutation(ks, key.getKey());
-            rm.add(cf, cellname("col"),
-                   ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp);
-            rm.apply();
-        }
-        store.forceBlockingFlush();
-
-        SSTableReader sstable = store.getSSTables().iterator().next();
-        Descriptor desc = sstable.descriptor;
-
-        // test to see if sstable can be opened as expected
-        SSTableReader target = SSTableReader.open(desc);
-        Assert.assertEquals(target.getIndexSummarySize(), 1);
-        Assert.assertArrayEquals(ByteBufferUtil.getArray(firstKey.getKey()), target.getIndexSummaryKey(0));
-        assert target.first.equals(firstKey);
-        assert target.last.equals(lastKey);
-        target.selfRef().release();
-    }
-
-    @Test
-    public void testLoadingSummaryUsesCorrectPartitioner() throws Exception
-    {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Indexed1");
-        ByteBuffer key = ByteBufferUtil.bytes(String.valueOf("k1"));
-        Mutation rm = new Mutation("Keyspace1", key);
-        rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(1L), System.currentTimeMillis());
-        rm.apply();
-        store.forceBlockingFlush();
-
-        ColumnFamilyStore indexCfs = store.indexManager.getIndexForColumn(ByteBufferUtil.bytes("birthdate")).getIndexCfs();
-        assert indexCfs.partitioner instanceof LocalPartitioner;
-        SSTableReader sstable = indexCfs.getSSTables().iterator().next();
-        assert sstable.first.getToken() instanceof LocalToken;
-
-        SegmentedFile.Builder ibuilder = SegmentedFile.getBuilder(DatabaseDescriptor.getIndexAccessMode());
-        SegmentedFile.Builder dbuilder = sstable.compression
-                                          ? SegmentedFile.getCompressedBuilder()
-                                          : SegmentedFile.getBuilder(DatabaseDescriptor.getDiskAccessMode());
-        sstable.saveSummary(ibuilder, dbuilder);
-
-        SSTableReader reopened = SSTableReader.open(sstable.descriptor);
-        assert reopened.first.getToken() instanceof LocalToken;
-        reopened.selfRef().release();
-    }
-
-    /** see CASSANDRA-5407 */
-    @Test
-    public void testGetScannerForNoIntersectingRanges()
-    {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
-        ByteBuffer key = ByteBufferUtil.bytes(String.valueOf("k1"));
-        Mutation rm = new Mutation("Keyspace1", key);
-        rm.add("Standard1", cellname("xyz"), ByteBufferUtil.bytes("abc"), 0);
-        rm.apply();
-        store.forceBlockingFlush();
-        boolean foundScanner = false;
-        for (SSTableReader s : store.getSSTables())
-        {
-            ISSTableScanner scanner = s.getScanner(new Range<Token>(t(0), t(1), s.partitioner), null);
-            scanner.next(); // throws exception pre 5407
-            foundScanner = true;
-        }
-        assertTrue(foundScanner);
-    }
-
-    @Test
-    public void testGetPositionsForRangesFromTableOpenedForBulkLoading() throws IOException, ExecutionException, InterruptedException
-    {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard2");
-
-        // insert data and compact to a single sstable. The
-        // number of keys inserted is greater than index_interval
-        // to ensure multiple segments in the index file
-        CompactionManager.instance.disableAutoCompaction();
-        for (int j = 0; j < 130; j++)
-        {
-            ByteBuffer key = ByteBufferUtil.bytes(String.valueOf(j));
-            Mutation rm = new Mutation("Keyspace1", key);
-            rm.add("Standard2", cellname("0"), ByteBufferUtil.EMPTY_BYTE_BUFFER, j);
-            rm.apply();
-        }
-        store.forceBlockingFlush();
-        CompactionManager.instance.performMaximal(store);
-
-        // construct a range which is present in the sstable, but whose
-        // keys are not found in the first segment of the index.
-        List<Range<Token>> ranges = new ArrayList<Range<Token>>();
-        ranges.add(new Range<Token>(t(98), t(99)));
-
-        SSTableReader sstable = store.getSSTables().iterator().next();
-        List<Pair<Long,Long>> sections = sstable.getPositionsForRanges(ranges);
-        assert sections.size() == 1 : "Expected to find range in sstable" ;
-
-        // re-open the same sstable as it would be during bulk loading
-        Set<Component> components = Sets.newHashSet(Component.DATA, Component.PRIMARY_INDEX);
-        if (sstable.components.contains(Component.COMPRESSION_INFO))
-            components.add(Component.COMPRESSION_INFO);
-        SSTableReader bulkLoaded = SSTableReader.openForBatch(sstable.descriptor, components, store.metadata, sstable.partitioner);
-        sections = bulkLoaded.getPositionsForRanges(ranges);
-        assert sections.size() == 1 : "Expected to find range in sstable opened for bulk loading";
-        bulkLoaded.selfRef().release();
-    }
-
-    @Test
-    public void testIndexSummaryReplacement() throws IOException, ExecutionException, InterruptedException
-    {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        final ColumnFamilyStore store = keyspace.getColumnFamilyStore("StandardLowIndexInterval"); // index interval of 8, no key caching
-        CompactionManager.instance.disableAutoCompaction();
-
-        final int NUM_ROWS = 512;
-        for (int j = 0; j < NUM_ROWS; j++)
-        {
-            ByteBuffer key = ByteBufferUtil.bytes(String.format("%3d", j));
-            Mutation rm = new Mutation("Keyspace1", key);
-            rm.add("StandardLowIndexInterval", Util.cellname("0"), ByteBufferUtil.bytes(String.format("%3d", j)), j);
-            rm.apply();
-        }
-        store.forceBlockingFlush();
-        CompactionManager.instance.performMaximal(store);
-
-        Collection<SSTableReader> sstables = store.getSSTables();
-        assert sstables.size() == 1;
-        final SSTableReader sstable = sstables.iterator().next();
-
-        ThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(5);
-        List<Future> futures = new ArrayList<>(NUM_ROWS * 2);
-        for (int i = 0; i < NUM_ROWS; i++)
-        {
-            final ByteBuffer key = ByteBufferUtil.bytes(String.format("%3d", i));
-            final int index = i;
-
-            futures.add(executor.submit(new Runnable()
-            {
-                public void run()
-                {
-                    ColumnFamily result = store.getColumnFamily(sstable.partitioner.decorateKey(key), Composites.EMPTY, Composites.EMPTY, false, 100, 100);
-                    assertFalse(result.isEmpty());
-                    assertEquals(0, ByteBufferUtil.compare(String.format("%3d", index).getBytes(), result.getColumn(Util.cellname("0")).value()));
-                }
-            }));
-
-            futures.add(executor.submit(new Runnable()
-            {
-                public void run()
-                {
-                    Iterable<DecoratedKey> results = store.keySamples(
-                            new Range<>(sstable.partitioner.getMinimumToken(), sstable.partitioner.getToken(key)));
-                    assertTrue(results.iterator().hasNext());
-                }
-            }));
-        }
-
-        SSTableReader replacement = sstable.cloneWithNewSummarySamplingLevel(store, 1);
-        store.getDataTracker().replaceWithNewInstances(Arrays.asList(sstable), Arrays.asList(replacement));
-        for (Future future : futures)
-            future.get();
-
-        assertEquals(sstable.estimatedKeys(), replacement.estimatedKeys(), 1);
-    }
-
-    private void assertIndexQueryWorks(ColumnFamilyStore indexedCFS)
-    {
-        assert "Indexed1".equals(indexedCFS.name);
-
-        // make sure all sstables including 2ary indexes load from disk
-        for (ColumnFamilyStore cfs : indexedCFS.concatWithIndexes())
-            clearAndLoad(cfs);
-
-        // query using index to see if sstable for secondary index opens
-        IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), Operator.EQ, ByteBufferUtil.bytes(1L));
-        List<IndexExpression> clause = Arrays.asList(expr);
-        Range<RowPosition> range = Util.range("", "");
-        List<Row> rows = indexedCFS.search(range, clause, new IdentityQueryFilter(), 100);
-        assert rows.size() == 1;
-    }
-
-    private List<Range<Token>> makeRanges(Token left, Token right)
-    {
-        return Arrays.asList(new Range<>(left, right));
-    }
-
-    private DecoratedKey k(int i)
-    {
-        return new BufferDecoratedKey(t(i), ByteBufferUtil.bytes(String.valueOf(i)));
-    }
-}
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
index 1fb28f5..f50953a 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
@@ -23,24 +23,39 @@
 import java.util.*;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import com.google.common.collect.Iterables;
 import com.google.common.collect.Sets;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import com.google.common.util.concurrent.Uninterruptibles;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.Config;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.compaction.AbstractCompactedRow;
 import org.apache.cassandra.db.compaction.AbstractCompactionStrategy;
 import org.apache.cassandra.db.compaction.CompactionController;
+import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.compaction.LazilyCompactedRow;
 import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.db.compaction.SSTableSplitter;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 import org.apache.cassandra.metrics.StorageMetrics;
 import org.apache.cassandra.notifications.INotification;
 import org.apache.cassandra.notifications.INotificationConsumer;
@@ -48,16 +63,54 @@
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.streaming.StreamSession;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Pair;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.*;
 
 public class SSTableRewriterTest extends SchemaLoader
 {
-    private static final String KEYSPACE = "Keyspace1";
+    private static final String KEYSPACE = "SSTableRewriterTest";
     private static final String CF = "Standard1";
+
+    private static Config.DiskAccessMode standardMode;
+    private static Config.DiskAccessMode indexMode;
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        if (FBUtilities.isWindows())
+        {
+            standardMode = DatabaseDescriptor.getDiskAccessMode();
+            indexMode = DatabaseDescriptor.getIndexAccessMode();
+
+            DatabaseDescriptor.setDiskAccessMode(Config.DiskAccessMode.standard);
+            DatabaseDescriptor.setIndexAccessMode(Config.DiskAccessMode.standard);
+        }
+
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE,
+                SimpleStrategy.class,
+                KSMetaData.optsWithRF(1),
+                SchemaLoader.standardCFMD(KEYSPACE, CF));
+    }
+
+    @AfterClass
+    public static void revertDiskAccess()
+    {
+        DatabaseDescriptor.setDiskAccessMode(standardMode);
+        DatabaseDescriptor.setIndexAccessMode(indexMode);
+    }
+
+    @After
+    public void truncateCF()
+    {
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore(CF);
+        store.truncateBlocking();
+        SSTableDeletingTask.waitForDeletions();
+    }
+
     @Test
     public void basicTest() throws InterruptedException
     {
@@ -75,8 +128,10 @@
         cfs.forceBlockingFlush();
         Set<SSTableReader> sstables = new HashSet<>(cfs.getSSTables());
         assertEquals(1, sstables.size());
-        SSTableRewriter writer = new SSTableRewriter(cfs, sstables, 1000, false);
-        try (AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategy().getScanners(sstables);)
+        assertEquals(sstables.iterator().next().bytesOnDisk(), cfs.metric.liveDiskSpaceUsed.getCount());
+        try (AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategy().getScanners(sstables);
+             LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN);
+             SSTableRewriter writer = new SSTableRewriter(cfs, txn, 1000, false);)
         {
             ISSTableScanner scanner = scanners.scanners.get(0);
             CompactionController controller = new CompactionController(cfs, sstables, cfs.gcBefore(System.currentTimeMillis()));
@@ -86,14 +141,13 @@
                 AbstractCompactedRow row = new LazilyCompactedRow(controller, Arrays.asList(scanner.next()));
                 writer.append(row);
             }
+            writer.finish();
         }
-        Collection<SSTableReader> newsstables = writer.finish();
-        cfs.getDataTracker().markCompactedSSTablesReplaced(sstables, newsstables, OperationType.COMPACTION);
-        Thread.sleep(100);
+        SSTableDeletingTask.waitForDeletions();
         validateCFS(cfs);
         int filecounts = assertFileCounts(sstables.iterator().next().descriptor.directory.list(), 0, 0);
         assertEquals(1, filecounts);
-
+        truncate(cfs);
     }
     @Test
     public void basicTest2() throws InterruptedException
@@ -106,8 +160,10 @@
         cfs.addSSTable(s);
         Set<SSTableReader> sstables = new HashSet<>(cfs.getSSTables());
         assertEquals(1, sstables.size());
-        SSTableRewriter writer = new SSTableRewriter(cfs, sstables, 1000, false, 10000000);
-        try (AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategy().getScanners(sstables);)
+
+        try (AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategy().getScanners(sstables);
+             LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN);
+             SSTableRewriter writer = new SSTableRewriter(cfs, txn, 1000, false, 10000000);)
         {
             ISSTableScanner scanner = scanners.scanners.get(0);
             CompactionController controller = new CompactionController(cfs, sstables, cfs.gcBefore(System.currentTimeMillis()));
@@ -117,10 +173,9 @@
                 AbstractCompactedRow row = new LazilyCompactedRow(controller, Arrays.asList(scanner.next()));
                 writer.append(row);
             }
+            writer.finish();
         }
-        Collection<SSTableReader> newsstables = writer.finish();
-        cfs.getDataTracker().markCompactedSSTablesReplaced(sstables, newsstables, OperationType.COMPACTION);
-        Thread.sleep(100);
+        SSTableDeletingTask.waitForDeletions();
         validateCFS(cfs);
         int filecounts = assertFileCounts(sstables.iterator().next().descriptor.directory.list(), 0, 0);
         assertEquals(1, filecounts);
@@ -137,9 +192,11 @@
         cfs.addSSTable(s);
         Set<SSTableReader> sstables = new HashSet<>(cfs.getSSTables());
         assertEquals(1, sstables.size());
-        SSTableRewriter writer = new SSTableRewriter(cfs, sstables, 1000, false, 10000000);
+
         boolean checked = false;
-        try (AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategy().getScanners(sstables);)
+        try (AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategy().getScanners(sstables);
+             LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN);
+             SSTableRewriter writer = new SSTableRewriter(cfs, txn, 1000, false, 10000000))
         {
             ISSTableScanner scanner = scanners.scanners.get(0);
             CompactionController controller = new CompactionController(cfs, sstables, cfs.gcBefore(System.currentTimeMillis()));
@@ -155,7 +212,7 @@
                     {
                         if (sstable.openReason == SSTableReader.OpenReason.EARLY)
                         {
-                            SSTableReader c = sstables.iterator().next();
+                            SSTableReader c = txn.current(sstables.iterator().next());
                             Collection<Range<Token>> r = Arrays.asList(new Range<>(cfs.partitioner.getMinimumToken(), cfs.partitioner.getMinimumToken()));
                             List<Pair<Long, Long>> tmplinkPositions = sstable.getPositionsForRanges(r);
                             List<Pair<Long, Long>> compactingPositions = c.getPositionsForRanges(r);
@@ -169,11 +226,10 @@
                     }
                 }
             }
+            assertTrue(checked);
+            writer.finish();
         }
-        assertTrue(checked);
-        Collection<SSTableReader> newsstables = writer.finish();
-        cfs.getDataTracker().markCompactedSSTablesReplaced(sstables, newsstables, OperationType.COMPACTION);
-        Thread.sleep(100);
+        SSTableDeletingTask.waitForDeletions();
         validateCFS(cfs);
         int filecounts = assertFileCounts(sstables.iterator().next().descriptor.directory.list(), 0, 0);
         assertEquals(1, filecounts);
@@ -188,37 +244,38 @@
         truncate(cfs);
 
         ArrayBackedSortedColumns cf = ArrayBackedSortedColumns.factory.create(cfs.metadata);
-        for (int i = 0; i < 1000; i++)
-            cf.addColumn(Util.column(String.valueOf(i), "a", 1));
+        for (int i = 0; i < 100; i++)
+            cf.addColumn(Util.cellname(i), ByteBuffer.allocate(1000), 1);
         File dir = cfs.directories.getDirectoryForNewSSTables();
-        SSTableWriter writer = getWriter(cfs, dir);
-        try
+
+        try (SSTableWriter writer = getWriter(cfs, dir);)
         {
-            for (int i = 0; i < 1000; i++)
+            for (int i = 0; i < 10000; i++)
                 writer.append(StorageService.getPartitioner().decorateKey(random(i, 10)), cf);
-            SSTableReader s = writer.openEarly(1000);
+            SSTableReader s = writer.setMaxDataAge(1000).openEarly();
+            assert s != null;
             assertFileCounts(dir.list(), 2, 2);
-            for (int i = 1000; i < 2000; i++)
+            for (int i = 10000; i < 20000; i++)
                 writer.append(StorageService.getPartitioner().decorateKey(random(i, 10)), cf);
-            SSTableReader s2 = writer.openEarly(1000);
+            SSTableReader s2 = writer.setMaxDataAge(1000).openEarly();
             assertTrue(s.last.compareTo(s2.last) < 0);
             assertFileCounts(dir.list(), 2, 2);
-            s.markObsolete(cfs.getDataTracker());
+            s.markObsolete(cfs.getTracker());
             s.selfRef().release();
             s2.selfRef().release();
-            Thread.sleep(1000);
-            assertFileCounts(dir.list(), 0, 2);
+            // These checks don't work on Windows because the writer has the channel still
+            // open till .abort() is called (via the builder)
+            if (!FBUtilities.isWindows())
+            {
+                SSTableDeletingTask.waitForDeletions();
+                assertFileCounts(dir.list(), 0, 2);
+            }
             writer.abort();
-            Thread.sleep(1000);
+            SSTableDeletingTask.waitForDeletions();
             int datafiles = assertFileCounts(dir.list(), 0, 0);
             assertEquals(datafiles, 0);
             validateCFS(cfs);
         }
-        catch (Throwable t)
-        {
-            writer.abort();
-            throw t;
-        }
     }
 
     @Test
@@ -230,15 +287,18 @@
 
         SSTableReader s = writeFile(cfs, 1000);
         cfs.addSSTable(s);
-        long startStorageMetricsLoad = StorageMetrics.load.count();
+        long startStorageMetricsLoad = StorageMetrics.load.getCount();
+        long sBytesOnDisk = s.bytesOnDisk();
         Set<SSTableReader> compacting = Sets.newHashSet(s);
-        SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false, 10000000);
-        rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
 
+        List<SSTableReader> sstables;
         int files = 1;
         try (ISSTableScanner scanner = s.getScanner();
-             CompactionController controller = new CompactionController(cfs, compacting, 0))
+             CompactionController controller = new CompactionController(cfs, compacting, 0);
+             LifecycleTransaction txn = cfs.getTracker().tryModify(compacting, OperationType.UNKNOWN);
+             SSTableRewriter rewriter = new SSTableRewriter(cfs, txn, 1000, false, 10000000))
         {
+            rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
             while(scanner.hasNext())
             {
                 rewriter.append(new LazilyCompactedRow(controller, Arrays.asList(scanner.next())));
@@ -247,24 +307,24 @@
                     rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
                     files++;
                     assertEquals(cfs.getSSTables().size(), files); // we have one original file plus the ones we have switched out.
-                    assertEquals(s.bytesOnDisk(), cfs.metric.liveDiskSpaceUsed.count());
-                    assertEquals(s.bytesOnDisk(), cfs.metric.totalDiskSpaceUsed.count());
+                    assertEquals(s.bytesOnDisk(), cfs.metric.liveDiskSpaceUsed.getCount());
+                    assertEquals(s.bytesOnDisk(), cfs.metric.totalDiskSpaceUsed.getCount());
 
                 }
             }
+            sstables = rewriter.finish();
         }
-        List<SSTableReader> sstables = rewriter.finish();
-        cfs.getDataTracker().markCompactedSSTablesReplaced(compacting, sstables, OperationType.COMPACTION);
         long sum = 0;
         for (SSTableReader x : cfs.getSSTables())
             sum += x.bytesOnDisk();
-        assertEquals(sum, cfs.metric.liveDiskSpaceUsed.count());
-        assertEquals(startStorageMetricsLoad - s.bytesOnDisk() + sum, StorageMetrics.load.count());
+        assertEquals(sum, cfs.metric.liveDiskSpaceUsed.getCount());
+        assertEquals(startStorageMetricsLoad - sBytesOnDisk + sum, StorageMetrics.load.getCount());
         assertEquals(files, sstables.size());
         assertEquals(files, cfs.getSSTables().size());
-        Thread.sleep(1000);
+        SSTableDeletingTask.waitForDeletions();
+
         // tmplink and tmp files should be gone:
-        assertEquals(sum, cfs.metric.totalDiskSpaceUsed.count());
+        assertEquals(sum, cfs.metric.totalDiskSpaceUsed.getCount());
         assertFileCounts(s.descriptor.directory.list(), 0, 0);
         validateCFS(cfs);
     }
@@ -280,13 +340,15 @@
         cfs.addSSTable(s);
 
         Set<SSTableReader> compacting = Sets.newHashSet(s);
-        SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false, 10000000);
-        rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
 
+        List<SSTableReader> sstables;
         int files = 1;
         try (ISSTableScanner scanner = s.getScanner();
-             CompactionController controller = new CompactionController(cfs, compacting, 0))
+             CompactionController controller = new CompactionController(cfs, compacting, 0);
+             LifecycleTransaction txn = cfs.getTracker().tryModify(compacting, OperationType.UNKNOWN);
+             SSTableRewriter rewriter = new SSTableRewriter(cfs, txn, 1000, false, 10000000))
         {
+            rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
             while(scanner.hasNext())
             {
                 rewriter.append(new LazilyCompactedRow(controller, Arrays.asList(scanner.next())));
@@ -297,23 +359,15 @@
                     assertEquals(cfs.getSSTables().size(), files); // we have one original file plus the ones we have switched out.
                 }
             }
+            sstables = rewriter.finish();
+        }
 
-            List<SSTableReader> sstables = rewriter.finish();
-            assertEquals(files, sstables.size());
-            assertEquals(files, cfs.getSSTables().size());
-            assertEquals(1, cfs.getDataTracker().getView().shadowed.size());
-            cfs.getDataTracker().markCompactedSSTablesReplaced(compacting, sstables, OperationType.COMPACTION);
-            assertEquals(files, cfs.getSSTables().size());
-            assertEquals(0, cfs.getDataTracker().getView().shadowed.size());
-            Thread.sleep(1000);
-            assertFileCounts(s.descriptor.directory.list(), 0, 0);
-            validateCFS(cfs);
-        }
-        catch (Throwable t)
-        {
-            rewriter.abort();
-            throw t;
-        }
+        assertEquals(files, sstables.size());
+        assertEquals(files, cfs.getSSTables().size());
+        SSTableDeletingTask.waitForDeletions();
+
+        assertFileCounts(s.descriptor.directory.list(), 0, 0);
+        validateCFS(cfs);
     }
 
 
@@ -407,24 +461,21 @@
 
         DecoratedKey origFirst = s.first;
         DecoratedKey origLast = s.last;
-        long startSize = cfs.metric.liveDiskSpaceUsed.count();
+        long startSize = cfs.metric.liveDiskSpaceUsed.getCount();
         Set<SSTableReader> compacting = Sets.newHashSet(s);
-        SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false, 10000000);
-        rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
 
         try (ISSTableScanner scanner = s.getScanner();
-             CompactionController controller = new CompactionController(cfs, compacting, 0))
+             CompactionController controller = new CompactionController(cfs, compacting, 0);
+             LifecycleTransaction txn = cfs.getTracker().tryModify(compacting, OperationType.UNKNOWN);
+             SSTableRewriter rewriter = new SSTableRewriter(cfs, txn, 1000, false, 10000000);)
         {
+            rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
             test.run(scanner, controller, s, cfs, rewriter);
         }
-        catch (Throwable t)
-        {
-            rewriter.abort();
-            throw t;
-        }
 
-        Thread.sleep(1000);
-        assertEquals(startSize, cfs.metric.liveDiskSpaceUsed.count());
+        SSTableDeletingTask.waitForDeletions();
+
+        assertEquals(startSize, cfs.metric.liveDiskSpaceUsed.getCount());
         assertEquals(1, cfs.getSSTables().size());
         assertFileCounts(s.descriptor.directory.list(), 0, 0);
         assertEquals(cfs.getSSTables().iterator().next().first, origFirst);
@@ -443,13 +494,14 @@
         cfs.addSSTable(s);
 
         Set<SSTableReader> compacting = Sets.newHashSet(s);
-        SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false, 10000000);
-        rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
 
         int files = 1;
         try (ISSTableScanner scanner = s.getScanner();
-             CompactionController controller = new CompactionController(cfs, compacting, 0))
+             CompactionController controller = new CompactionController(cfs, compacting, 0);
+             LifecycleTransaction txn = cfs.getTracker().tryModify(compacting, OperationType.UNKNOWN);
+             SSTableRewriter rewriter = new SSTableRewriter(cfs, txn, 1000, false, 10000000))
         {
+            rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
             while(scanner.hasNext())
             {
                 rewriter.append(new LazilyCompactedRow(controller, Arrays.asList(scanner.next())));
@@ -462,22 +514,17 @@
                 if (files == 3)
                 {
                     //testing to finish when we have nothing written in the new file
-                    List<SSTableReader> sstables = rewriter.finish();
-                    cfs.getDataTracker().markCompactedSSTablesReplaced(compacting, sstables, OperationType.COMPACTION);
+                    rewriter.finish();
                     break;
                 }
             }
+        }
 
-            Thread.sleep(1000);
-            assertEquals(files - 1, cfs.getSSTables().size()); // we never wrote anything to the last file
-            assertFileCounts(s.descriptor.directory.list(), 0, 0);
-            validateCFS(cfs);
-        }
-        catch (Throwable t)
-        {
-            rewriter.abort();
-            throw t;
-        }
+        SSTableDeletingTask.waitForDeletions();
+
+        assertEquals(files - 1, cfs.getSSTables().size()); // we never wrote anything to the last file
+        assertFileCounts(s.descriptor.directory.list(), 0, 0);
+        validateCFS(cfs);
     }
 
     @Test
@@ -491,13 +538,15 @@
         SSTableReader s = writeFile(cfs, 1000);
         cfs.addSSTable(s);
         Set<SSTableReader> compacting = Sets.newHashSet(s);
-        SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false, 10000000);
-        rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
 
+        List<SSTableReader> sstables;
         int files = 1;
         try (ISSTableScanner scanner = s.getScanner();
-             CompactionController controller = new CompactionController(cfs, compacting, 0))
+             CompactionController controller = new CompactionController(cfs, compacting, 0);
+             LifecycleTransaction txn = cfs.getTracker().tryModify(compacting, OperationType.UNKNOWN);
+             SSTableRewriter rewriter = new SSTableRewriter(cfs, txn, 1000, false, 10000000))
         {
+            rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
             while(scanner.hasNext())
             {
                 rewriter.append(new LazilyCompactedRow(controller, Arrays.asList(scanner.next())));
@@ -509,17 +558,12 @@
                 }
             }
 
-            List<SSTableReader> sstables = rewriter.finish();
-            cfs.getDataTracker().markCompactedSSTablesReplaced(compacting, sstables, OperationType.COMPACTION);
-            Thread.sleep(1000);
-            assertFileCounts(s.descriptor.directory.list(), 0, 0);
-            truncate(cfs);
+            sstables = rewriter.finish();
         }
-        catch (Throwable t)
-        {
-            rewriter.abort();
-            throw t;
-        }
+
+        SSTableDeletingTask.waitForDeletions();
+        assertFileCounts(s.descriptor.directory.list(), 0, 0);
+        validateCFS(cfs);
     }
 
     @Test
@@ -533,13 +577,15 @@
         SSTableReader s = writeFile(cfs, 400);
         cfs.addSSTable(s);
         Set<SSTableReader> compacting = Sets.newHashSet(s);
-        SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false, 1000000);
-        rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
 
+        List<SSTableReader> sstables;
         int files = 1;
         try (ISSTableScanner scanner = s.getScanner();
-             CompactionController controller = new CompactionController(cfs, compacting, 0))
+             CompactionController controller = new CompactionController(cfs, compacting, 0);
+             LifecycleTransaction txn = cfs.getTracker().tryModify(compacting, OperationType.UNKNOWN);
+             SSTableRewriter rewriter = new SSTableRewriter(cfs, txn, 1000, false, 1000000);)
         {
+            rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
             while(scanner.hasNext())
             {
                 rewriter.append(new LazilyCompactedRow(controller, Arrays.asList(scanner.next())));
@@ -551,20 +597,17 @@
                 }
             }
 
-            List<SSTableReader> sstables = rewriter.finish();
-            cfs.getDataTracker().markCompactedSSTablesReplaced(compacting, sstables, OperationType.COMPACTION);
-            assertEquals(files, sstables.size());
-            assertEquals(files, cfs.getSSTables().size());
-            Thread.sleep(1000);
-            assertFileCounts(s.descriptor.directory.list(), 0, 0);
-            validateCFS(cfs);
+            sstables = rewriter.finish();
         }
-        catch (Throwable t)
-        {
-            rewriter.abort();
-            throw t;
-        }
+
+        assertEquals(files, sstables.size());
+        assertEquals(files, cfs.getSSTables().size());
+        SSTableDeletingTask.waitForDeletions();
+        assertFileCounts(s.descriptor.directory.list(), 0, 0);
+
+        validateCFS(cfs);
     }
+
     @Test
     public void testSSTableSplit() throws InterruptedException
     {
@@ -573,16 +616,21 @@
         truncate(cfs);
         cfs.disableAutoCompaction();
         SSTableReader s = writeFile(cfs, 1000);
-        cfs.getDataTracker().markCompacting(Arrays.asList(s), true, false);
-        SSTableSplitter splitter = new SSTableSplitter(cfs, s, 10);
-        splitter.split();
-        Thread.sleep(1000);
-        assertFileCounts(s.descriptor.directory.list(), 0, 0);
-        s.selfRef().release();
-        for (File f : s.descriptor.directory.listFiles())
+        try (LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.UNKNOWN, s))
         {
-            // we need to clear out the data dir, otherwise tests running after this breaks
-            f.delete();
+            SSTableSplitter splitter = new SSTableSplitter(cfs, txn, 10);
+            splitter.split();
+
+            assertFileCounts(s.descriptor.directory.list(), 0, 0);
+
+            s.selfRef().release();
+            SSTableDeletingTask.waitForDeletions();
+
+            for (File f : s.descriptor.directory.listFiles())
+            {
+                // we need to clear out the data dir, otherwise tests running after this breaks
+                FileUtils.deleteRecursive(f);
+            }
         }
         truncate(cfs);
     }
@@ -619,13 +667,14 @@
         if (!offline)
             cfs.addSSTable(s);
         Set<SSTableReader> compacting = Sets.newHashSet(s);
-        cfs.getDataTracker().markCompacting(compacting);
-        SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, offline, 10000000);
-        SSTableWriter w = getWriter(cfs, s.descriptor.directory);
-        rewriter.switchWriter(w);
         try (ISSTableScanner scanner = compacting.iterator().next().getScanner();
-             CompactionController controller = new CompactionController(cfs, compacting, 0))
+             CompactionController controller = new CompactionController(cfs, compacting, 0);
+             LifecycleTransaction txn = offline ? LifecycleTransaction.offline(OperationType.UNKNOWN, compacting)
+                                       : cfs.getTracker().tryModify(compacting, OperationType.UNKNOWN);
+             SSTableRewriter rewriter = new SSTableRewriter(cfs, txn, 1000, offline, 10000000);
+        )
         {
+            rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
             while (scanner.hasNext())
             {
                 rewriter.append(new LazilyCompactedRow(controller, Arrays.asList(scanner.next())));
@@ -636,7 +685,8 @@
             }
             try
             {
-                rewriter.finishAndThrow(earlyException);
+                rewriter.throwDuringPrepare(earlyException);
+                rewriter.prepareToCommit();
             }
             catch (Throwable t)
             {
@@ -645,9 +695,12 @@
         }
         finally
         {
-            cfs.getDataTracker().unmarkCompacting(compacting);
+            if (offline)
+                s.selfRef().release();
         }
-        Thread.sleep(1000);
+
+        SSTableDeletingTask.waitForDeletions();
+
         int filecount = assertFileCounts(s.descriptor.directory.list(), 0, 0);
         assertEquals(filecount, 1);
         if (!offline)
@@ -664,12 +717,11 @@
         filecount = assertFileCounts(s.descriptor.directory.list(), 0, 0);
         if (offline)
         {
-            s.selfRef().release();
-            // the file is not added to the CFS, therefor not truncated away above
+            // the file is not added to the CFS, therefore not truncated away above
             assertEquals(1, filecount);
             for (File f : s.descriptor.directory.listFiles())
             {
-                f.delete();
+                FileUtils.deleteRecursive(f);
             }
             filecount = assertFileCounts(s.descriptor.directory.list(), 0, 0);
         }
@@ -700,15 +752,15 @@
         SSTableReader s = cfs.getSSTables().iterator().next();
         Set<SSTableReader> compacting = new HashSet<>();
         compacting.add(s);
-        cfs.getDataTracker().markCompacting(compacting);
 
-        SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false, 1);
-        SSTableWriter w = getWriter(cfs, s.descriptor.directory);
-        rewriter.switchWriter(w);
         int keyCount = 0;
         try (ISSTableScanner scanner = compacting.iterator().next().getScanner();
-             CompactionController controller = new CompactionController(cfs, compacting, 0))
+             CompactionController controller = new CompactionController(cfs, compacting, 0);
+             LifecycleTransaction txn = cfs.getTracker().tryModify(compacting, OperationType.UNKNOWN);
+             SSTableRewriter rewriter = new SSTableRewriter(cfs, txn, 1000, false, 1);
+        )
         {
+            rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
             while (scanner.hasNext())
             {
                 rewriter.append(new LazilyCompactedRow(controller, Arrays.asList(scanner.next())));
@@ -719,24 +771,16 @@
                 keyCount++;
                 validateKeys(keyspace);
             }
-            try
-            {
-                cfs.getDataTracker().markCompactedSSTablesReplaced(compacting, rewriter.finish(), OperationType.COMPACTION);
-                cfs.getDataTracker().unmarkCompacting(compacting);
-            }
-            catch (Throwable t)
-            {
-                rewriter.abort();
-            }
+            rewriter.finish();
         }
         validateKeys(keyspace);
-        Thread.sleep(1000);
+        SSTableDeletingTask.waitForDeletions();
         validateCFS(cfs);
         truncate(cfs);
     }
 
     @Test
-    public void testCanonicalView() throws IOException
+    public void testCanonicalView() throws Exception
     {
         Keyspace keyspace = Keyspace.open(KEYSPACE);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
@@ -744,14 +788,15 @@
 
         SSTableReader s = writeFile(cfs, 1000);
         cfs.addSSTable(s);
-        Set<SSTableReader> sstables = Sets.newHashSet(cfs.markAllCompacting());
+        Set<SSTableReader> sstables = Sets.newHashSet(s);
         assertEquals(1, sstables.size());
-        SSTableRewriter writer = new SSTableRewriter(cfs, sstables, 1000, false, 10000000);
         boolean checked = false;
-        try (AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategy().getScanners(sstables))
+        try (ISSTableScanner scanner = sstables.iterator().next().getScanner();
+             CompactionController controller = new CompactionController(cfs, sstables, 0);
+             LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN);
+             SSTableRewriter writer = new SSTableRewriter(cfs, txn, 1000, false, 10000000);
+        )
         {
-            ISSTableScanner scanner = scanners.scanners.get(0);
-            CompactionController controller = new CompactionController(cfs, sstables, cfs.gcBefore(System.currentTimeMillis()));
             writer.switchWriter(getWriter(cfs, sstables.iterator().next().descriptor.directory));
             while (scanner.hasNext())
             {
@@ -769,9 +814,8 @@
                 }
             }
         }
-        writer.abort();
-        cfs.getDataTracker().unmarkCompacting(sstables);
-        truncate(cfs);
+        truncateCF();
+        validateCFS(cfs);
     }
 
     @Test
@@ -796,45 +840,39 @@
             section.ref.release();
         final AtomicInteger checkCount = new AtomicInteger();
         // needed since we get notified when compaction is done as well - we can't get sections for ranges for obsoleted sstables
-        INotificationConsumer consumer = new INotificationConsumer()
+        final AtomicBoolean done = new AtomicBoolean(false);
+        final AtomicBoolean failed = new AtomicBoolean(false);
+        Runnable r = new Runnable()
+        {
+            public void run()
+            {
+                while (!done.get())
                 {
-                    public void handleNotification(INotification notification, Object sender)
-                    {
-                        if (notification instanceof SSTableListChangedNotification)
-                        {
-                            Collection<SSTableReader> added = ((SSTableListChangedNotification) notification).added;
-                            Collection<SSTableReader> removed = ((SSTableListChangedNotification) notification).removed;
-                            // note that we need to check if added.equals(removed) because once the compaction is done the old sstable will have
-                            // selfRef().globalCount() == 0 and we cant get the SectionsForRanges then. During incremental opening we always add and remove the same
-                            // sstable (note that the sstables are x.equal(y) but not x == y since the new one will be a new instance with a moved starting point
-                            // In this case we must avoid trying to call getSSTableSectionsForRanges since we are in the notification
-                            // method and trying to reference an sstable with globalcount == 0 puts it into a loop, and this blocks the tracker from removing the
-                            // unreferenced sstable.
-                            if (added.isEmpty() || !added.iterator().next().getColumnFamilyName().equals(cfs.getColumnFamilyName()) || !added.equals(removed))
-                                return;
-
-                            // at no point must the rewrite process hide
-                            // sections returned by getSSTableSectionsForRanges
-                            Set<Range<Token>> range = Collections.singleton(new Range<Token>(firstToken, firstToken));
-                            List<StreamSession.SSTableStreamingSections> sections = StreamSession.getSSTableSectionsForRanges(range, Collections.singleton(cfs), 0L, false);
-                            assertEquals(1, sections.size());
-                            for (StreamSession.SSTableStreamingSections section : sections)
-                                section.ref.release();
-                            checkCount.incrementAndGet();
-                        }
-                    }
-                };
-        cfs.getDataTracker().subscribe(consumer);
+                    Set<Range<Token>> range = Collections.singleton(new Range<Token>(firstToken, firstToken));
+                    List<StreamSession.SSTableStreamingSections> sections = StreamSession.getSSTableSectionsForRanges(range, Collections.singleton(cfs), 0L, false);
+                    if (sections.size() != 1)
+                        failed.set(true);
+                    for (StreamSession.SSTableStreamingSections section : sections)
+                        section.ref.release();
+                    checkCount.incrementAndGet();
+                    Uninterruptibles.sleepUninterruptibly(5, TimeUnit.MILLISECONDS);
+                }
+            }
+        };
+        Thread t = new Thread(r);
         try
         {
+            t.start();
             cfs.forceMajorCompaction();
             // reset
         }
         finally
         {
             DatabaseDescriptor.setSSTablePreempiveOpenIntervalInMB(50);
-            cfs.getDataTracker().unsubscribe(consumer);
+            done.set(true);
+            t.join(20);
         }
+        assertFalse(failed.get());
         assertTrue(checkCount.get() >= 2);
         truncate(cfs);
     }
@@ -853,11 +891,12 @@
 
         SSTableReader s = writeFile(cfs, 1000);
         cfs.addSSTable(s);
-        Set<SSTableReader> sstables = Sets.newHashSet(cfs.markAllCompacting());
+        Set<SSTableReader> sstables = Sets.newHashSet(s);
         assertEquals(1, sstables.size());
-        SSTableRewriter writer = new SSTableRewriter(cfs, sstables, 1000, false, false);
-        SSTableRewriter writer2 = new SSTableRewriter(cfs, sstables, 1000, false, false);
-        try (AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategy().getScanners(sstables))
+        try (AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategy().getScanners(sstables);
+             LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN);
+             SSTableRewriter writer = new SSTableRewriter(cfs, txn, 1000, false, false);
+             SSTableRewriter writer2 = new SSTableRewriter(cfs, txn, 1000, false, false))
         {
             ISSTableScanner scanner = scanners.scanners.get(0);
             CompactionController controller = new CompactionController(cfs, sstables, cfs.gcBefore(System.currentTimeMillis()));
@@ -879,10 +918,8 @@
                 assertTrue(cf != null);
             }
         }
-        writer.abort();
-        writer2.abort();
-        cfs.getDataTracker().unmarkCompacting(sstables);
-        truncate(cfs);
+        truncateCF();
+        validateCFS(cfs);
     }
 
 
@@ -896,59 +933,74 @@
         }
     }
 
-    private SSTableReader writeFile(ColumnFamilyStore cfs, int count)
+    public static void truncate(ColumnFamilyStore cfs)
     {
-        ArrayBackedSortedColumns cf = ArrayBackedSortedColumns.factory.create(cfs.metadata);
-        for (int i = 0; i < count / 100; i++)
-            cf.addColumn(Util.cellname(i), random(0, 1000), 1);
-        File dir = cfs.directories.getDirectoryForNewSSTables();
-        String filename = cfs.getTempSSTablePath(dir);
-
-        SSTableWriter writer = new SSTableWriter(filename,
-                0,
-                0,
-                cfs.metadata,
-                StorageService.getPartitioner(),
-                new MetadataCollector(cfs.metadata.comparator));
-
-        for (int i = 0; i < count * 5; i++)
-            writer.append(StorageService.getPartitioner().decorateKey(ByteBufferUtil.bytes(i)), cf);
-        return writer.closeAndOpenReader();
+        cfs.truncateBlocking();
+        SSTableDeletingTask.waitForDeletions();
+        Uninterruptibles.sleepUninterruptibly(10L,TimeUnit.MILLISECONDS);
+        assertEquals(0, cfs.metric.liveDiskSpaceUsed.getCount());
+        assertEquals(0, cfs.metric.totalDiskSpaceUsed.getCount());
+        validateCFS(cfs);
     }
 
-    private void validateCFS(ColumnFamilyStore cfs)
+    public static SSTableReader writeFile(ColumnFamilyStore cfs, int count)
+    {
+        return Iterables.getFirst(writeFiles(cfs, 1, count * 5, count / 100, 1000), null);
+    }
+
+    public static Set<SSTableReader> writeFiles(ColumnFamilyStore cfs, int fileCount, int partitionCount, int cellCount, int cellSize)
+    {
+        int i = 0;
+        Set<SSTableReader> result = new LinkedHashSet<>();
+        for (int f = 0 ; f < fileCount ; f++)
+        {
+            File dir = cfs.directories.getDirectoryForNewSSTables();
+            String filename = cfs.getTempSSTablePath(dir);
+
+            SSTableWriter writer = SSTableWriter.create(filename, 0, 0);
+            int end = f == fileCount - 1 ? partitionCount : ((f + 1) * partitionCount) / fileCount;
+            for ( ; i < end ; i++)
+            {
+                ArrayBackedSortedColumns cf = ArrayBackedSortedColumns.factory.create(cfs.metadata);
+                for (int j = 0; j < cellCount ; j++)
+                    cf.addColumn(Util.cellname(j), random(0, cellSize), 1);
+                writer.append(StorageService.getPartitioner().decorateKey(ByteBufferUtil.bytes(i)), cf);
+            }
+            result.add(writer.finish(true));
+        }
+        return result;
+    }
+
+    public static void validateCFS(ColumnFamilyStore cfs)
     {
         Set<Integer> liveDescriptors = new HashSet<>();
+        long spaceUsed = 0;
         for (SSTableReader sstable : cfs.getSSTables())
         {
             assertFalse(sstable.isMarkedCompacted());
             assertEquals(1, sstable.selfRef().globalCount());
             liveDescriptors.add(sstable.descriptor.generation);
+            spaceUsed += sstable.bytesOnDisk();
         }
         for (File dir : cfs.directories.getCFDirectories())
         {
-            for (String f : dir.list())
+            for (File f : dir.listFiles())
             {
-                if (f.contains("Data"))
+                if (f.getName().contains("Data"))
                 {
-                    Descriptor d = Descriptor.fromFilename(f);
+                    Descriptor d = Descriptor.fromFilename(f.getAbsolutePath());
                     assertTrue(d.toString(), liveDescriptors.contains(d.generation));
                 }
             }
         }
-        assertTrue(cfs.getDataTracker().getCompacting().isEmpty());
-        assertTrue("" + cfs.getTotalDiskSpaceUsed(), cfs.getTotalDiskSpaceUsed() >= 0);
+        assertEquals(spaceUsed, cfs.metric.liveDiskSpaceUsed.getCount());
+        assertEquals(spaceUsed, cfs.metric.totalDiskSpaceUsed.getCount());
+        assertTrue(cfs.getTracker().getCompacting().isEmpty());
+        if (cfs.getSSTables().size() > 0)
+            assertFalse(CompactionManager.instance.submitMaximal(cfs, cfs.gcBefore(System.currentTimeMillis()/1000), false).isEmpty());
     }
 
-    private void truncate(ColumnFamilyStore cfs)
-    {
-        cfs.truncateBlocking();
-        SSTableDeletingTask.waitForDeletions();
-        validateCFS(cfs);
-        assertTrue(cfs.getTotalDiskSpaceUsed() == 0);
-    }
-
-    private int assertFileCounts(String [] files, int expectedtmplinkCount, int expectedtmpCount)
+    public static int assertFileCounts(String [] files, int expectedtmplinkCount, int expectedtmpCount)
     {
         int tmplinkcount = 0;
         int tmpcount = 0;
@@ -957,9 +1009,9 @@
         {
             if (f.endsWith("-CRC.db"))
                 continue;
-            if (f.contains("-tmplink-"))
+            if (f.contains("tmplink-"))
                 tmplinkcount++;
-            else if (f.contains("-tmp-"))
+            else if (f.contains("tmp-"))
                 tmpcount++;
             else if (f.contains("Data"))
                 datacount++;
@@ -969,18 +1021,13 @@
         return datacount;
     }
 
-    private SSTableWriter getWriter(ColumnFamilyStore cfs, File directory)
+    public static SSTableWriter getWriter(ColumnFamilyStore cfs, File directory)
     {
         String filename = cfs.getTempSSTablePath(directory);
-        return new SSTableWriter(filename,
-                                 0,
-                                 0,
-                                 cfs.metadata,
-                                 StorageService.getPartitioner(),
-                                 new MetadataCollector(cfs.metadata.comparator));
+        return SSTableWriter.create(filename, 0, 0);
     }
 
-    private ByteBuffer random(int i, int size)
+    public static ByteBuffer random(int i, int size)
     {
         byte[] bytes = new byte[size + 4];
         ThreadLocalRandom.current().nextBytes(bytes);
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java
index 91a820c..f8b808d 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java
@@ -18,26 +18,41 @@
 */
 package org.apache.cassandra.io.sstable;
 
+import java.io.IOException;
 import java.util.*;
 
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.junit.BeforeClass;
 import com.google.common.collect.Iterables;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.columniterator.IdentityQueryFilter;
 import org.apache.cassandra.dht.*;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 import static org.apache.cassandra.dht.AbstractBounds.isEmpty;
 import static org.junit.Assert.*;
 
-public class SSTableScannerTest extends SchemaLoader
+public class SSTableScannerTest
 {
-    public static final String KEYSPACE = "Keyspace1";
+    public static final String KEYSPACE = "SSTableScannerTest";
     public static final String TABLE = "Standard1";
 
+    @BeforeClass
+    public static void defineSchema() throws Exception
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE, TABLE));
+    }
+
     private static String toKey(int key)
     {
         return String.format("%03d", key);
@@ -100,7 +115,7 @@
 
     private static Token token(int key)
     {
-        return key == Integer.MIN_VALUE ? ByteOrderedPartitioner.MINIMUM : new BytesToken(toKey(key).getBytes());
+        return key == Integer.MIN_VALUE ? ByteOrderedPartitioner.MINIMUM : new ByteOrderedPartitioner.BytesToken(toKey(key).getBytes());
     }
 
     private static RowPosition min(int key)
@@ -120,8 +135,8 @@
 
     private static Range<Token> rangeFor(int start, int end)
     {
-        return new Range<Token>(new BytesToken(toKey(start).getBytes()),
-                                end == Integer.MIN_VALUE ? ByteOrderedPartitioner.MINIMUM : new BytesToken(toKey(end).getBytes()));
+        return new Range<Token>(new ByteOrderedPartitioner.BytesToken(toKey(start).getBytes()),
+                                end == Integer.MIN_VALUE ? ByteOrderedPartitioner.MINIMUM : new ByteOrderedPartitioner.BytesToken(toKey(end).getBytes()));
     }
 
     private static Collection<Range<Token>> makeRanges(int ... keys)
@@ -138,7 +153,7 @@
         DecoratedKey decoratedKey = Util.dk(toKey(key));
         Mutation rm = new Mutation(KEYSPACE, decoratedKey.getKey());
         rm.add(TABLE, Util.cellname("col"), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp, 1000);
-        rm.apply();
+        rm.applyUnsafe();
     }
 
     private static void assertScanMatches(SSTableReader sstable, int scanStart, int scanEnd, int ... boundaries)
@@ -146,11 +161,17 @@
         assert boundaries.length % 2 == 0;
         for (DataRange range : dataRanges(scanStart, scanEnd))
         {
-            ISSTableScanner scanner = sstable.getScanner(range);
-            for (int b = 0 ; b < boundaries.length ; b += 2)
-                for (int i = boundaries[b] ; i <= boundaries[b + 1] ; i++)
-                    assertEquals(toKey(i), new String(scanner.next().getKey().getKey().array()));
-            assertFalse(scanner.hasNext());
+            try(ISSTableScanner scanner = sstable.getScanner(range))
+            {
+                for (int b = 0; b < boundaries.length; b += 2)
+                    for (int i = boundaries[b]; i <= boundaries[b + 1]; i++)
+                        assertEquals(toKey(i), new String(scanner.next().getKey().getKey().array()));
+                assertFalse(scanner.hasNext());
+            }
+            catch (Exception e)
+            {
+                throw new RuntimeException(e);
+            }
         }
     }
 
@@ -160,7 +181,7 @@
     }
 
     @Test
-    public void testSingleDataRange()
+    public void testSingleDataRange() throws IOException
     {
         Keyspace keyspace = Keyspace.open(KEYSPACE);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore(TABLE);
@@ -181,6 +202,8 @@
         for (int i = 2; i < 10; i++)
             assertEquals(toKey(i), new String(scanner.next().getKey().getKey().array()));
 
+        scanner.close();
+
         // a simple read of a chunk in the middle
         assertScanMatches(sstable, 3, 6, 3, 6);
 
@@ -243,7 +266,7 @@
         assertScanMatches(sstable, 1, 0, 2, 9);
     }
 
-    private static void assertScanContainsRanges(ISSTableScanner scanner, int ... rangePairs)
+    private static void assertScanContainsRanges(ISSTableScanner scanner, int ... rangePairs) throws IOException
     {
         assert rangePairs.length % 2 == 0;
 
@@ -259,10 +282,11 @@
             }
         }
         assertFalse(scanner.hasNext());
+        scanner.close();
     }
 
     @Test
-    public void testMultipleRanges()
+    public void testMultipleRanges() throws IOException
     {
         Keyspace keyspace = Keyspace.open(KEYSPACE);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore(TABLE);
@@ -394,7 +418,7 @@
     }
 
     @Test
-    public void testSingleKeyMultipleRanges()
+    public void testSingleKeyMultipleRanges() throws IOException
     {
         Keyspace keyspace = Keyspace.open(KEYSPACE);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore(TABLE);
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableSimpleWriterTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableSimpleWriterTest.java
index 15980a4..499caf7 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableSimpleWriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableSimpleWriterTest.java
@@ -20,27 +20,43 @@
 
 import java.io.File;
 
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.dht.IPartitioner;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.marshal.IntegerType;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.service.StorageService;
 import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
 import static org.apache.cassandra.utils.ByteBufferUtil.toInt;
 
-public class SSTableSimpleWriterTest extends SchemaLoader
+public class SSTableSimpleWriterTest
 {
+    public static final String KEYSPACE = "SSTableSimpleWriterTest";
+    public static final String CF_STANDARDINT = "StandardInteger1";
+
+    @BeforeClass
+    public static void defineSchema() throws Exception
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF_STANDARDINT));
+    }
+
     @Test
     public void testSSTableSimpleUnsortedWriter() throws Exception
     {
         final int INC = 5;
         final int NBCOL = 10;
 
-        String keyspaceName = "Keyspace1";
+        String keyspaceName = KEYSPACE;
         String cfname = "StandardInteger1";
 
         Keyspace t = Keyspace.open(keyspaceName); // make sure we create the directory
@@ -48,42 +64,42 @@
         assert dir.exists();
 
         IPartitioner partitioner = StorageService.getPartitioner();
-        SSTableSimpleUnsortedWriter writer = new SSTableSimpleUnsortedWriter(dir, partitioner, keyspaceName, cfname, IntegerType.instance, null, 16);
-
-        int k = 0;
-
-        // Adding a few rows first
-        for (; k < 10; ++k)
+        try (SSTableSimpleUnsortedWriter writer = new SSTableSimpleUnsortedWriter(dir, partitioner, keyspaceName, cfname, IntegerType.instance, null, 16))
         {
-            writer.newRow(bytes("Key" + k));
-            writer.addColumn(bytes(1), bytes("v"), 0);
-            writer.addColumn(bytes(2), bytes("v"), 0);
-            writer.addColumn(bytes(3), bytes("v"), 0);
-        }
 
-
-        // Testing multiple opening of the same row
-        // We'll write column 0, 5, 10, .., on the first row, then 1, 6, 11, ... on the second one, etc.
-        for (int i = 0; i < INC; ++i)
-        {
-            writer.newRow(bytes("Key" + k));
-            for (int j = 0; j < NBCOL; ++j)
+            int k = 0;
+    
+            // Adding a few rows first
+            for (; k < 10; ++k)
             {
-                writer.addColumn(bytes(i + INC * j), bytes("v"), 1);
+                writer.newRow(bytes("Key" + k));
+                writer.addColumn(bytes(1), bytes("v"), 0);
+                writer.addColumn(bytes(2), bytes("v"), 0);
+                writer.addColumn(bytes(3), bytes("v"), 0);
+            }
+    
+    
+            // Testing multiple opening of the same row
+            // We'll write column 0, 5, 10, .., on the first row, then 1, 6, 11, ... on the second one, etc.
+            for (int i = 0; i < INC; ++i)
+            {
+                writer.newRow(bytes("Key" + k));
+                for (int j = 0; j < NBCOL; ++j)
+                {
+                    writer.addColumn(bytes(i + INC * j), bytes("v"), 1);
+                }
+            }
+            k++;
+    
+            // Adding a few more rows
+            for (; k < 20; ++k)
+            {
+                writer.newRow(bytes("Key" + k));
+                writer.addColumn(bytes(1), bytes("v"), 0);
+                writer.addColumn(bytes(2), bytes("v"), 0);
+                writer.addColumn(bytes(3), bytes("v"), 0);
             }
         }
-        k++;
-
-        // Adding a few more rows
-        for (; k < 20; ++k)
-        {
-            writer.newRow(bytes("Key" + k));
-            writer.addColumn(bytes(1), bytes("v"), 0);
-            writer.addColumn(bytes(2), bytes("v"), 0);
-            writer.addColumn(bytes(3), bytes("v"), 0);
-        }
-
-        writer.close();
 
         // Now add that newly created files to the column family
         ColumnFamilyStore cfs = t.getColumnFamilyStore(cfname);
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java b/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
index b9a3821..a116b84 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
@@ -25,6 +25,8 @@
 
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
 import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
@@ -37,9 +39,21 @@
     public static String KEYSPACENAME = "Keyspace1";
     public static String CFNAME = "Standard1";
 
+    public SSTableUtils(String ksname, String cfname)
+    {
+        KEYSPACENAME = ksname;
+        CFNAME = cfname;
+    }
+
+    /**/
     public static ColumnFamily createCF(long mfda, int ldt, Cell... cols)
     {
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACENAME, CFNAME);
+        return createCF(KEYSPACENAME, CFNAME, mfda, ldt, cols);
+    }
+
+    public static ColumnFamily createCF(String ksname, String cfname, long mfda, int ldt, Cell... cols)
+    {
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(ksname, cfname);
         cf.delete(new DeletionInfo(mfda, ldt));
         for (Cell col : cols)
             cf.addColumn(col);
@@ -57,28 +71,30 @@
         if(!tempdir.delete() || !tempdir.mkdir())
             throw new IOException("Temporary directory creation failed.");
         tempdir.deleteOnExit();
-        File keyspaceDir = new File(tempdir, keyspaceName);
-        keyspaceDir.mkdir();
-        keyspaceDir.deleteOnExit();
-        File datafile = new File(new Descriptor(keyspaceDir, keyspaceName, cfname, generation, Descriptor.Type.FINAL).filenameFor("Data.db"));
+        File cfDir = new File(tempdir, keyspaceName + File.separator + cfname);
+        cfDir.mkdirs();
+        cfDir.deleteOnExit();
+        File datafile = new File(new Descriptor(cfDir, keyspaceName, cfname, generation, Descriptor.Type.FINAL).filenameFor("Data.db"));
         if (!datafile.createNewFile())
             throw new IOException("unable to create file " + datafile);
         datafile.deleteOnExit();
         return datafile;
     }
 
-    public static void assertContentEquals(SSTableReader lhs, SSTableReader rhs)
+    public static void assertContentEquals(SSTableReader lhs, SSTableReader rhs) throws Exception
     {
-        ISSTableScanner slhs = lhs.getScanner();
-        ISSTableScanner srhs = rhs.getScanner();
-        while (slhs.hasNext())
+        try (ISSTableScanner slhs = lhs.getScanner();
+             ISSTableScanner srhs = rhs.getScanner())
         {
-            OnDiskAtomIterator ilhs = slhs.next();
-            assert srhs.hasNext() : "LHS contained more rows than RHS";
-            OnDiskAtomIterator irhs = srhs.next();
-            assertContentEquals(ilhs, irhs);
+            while (slhs.hasNext())
+            {
+                OnDiskAtomIterator ilhs = slhs.next();
+                assert srhs.hasNext() : "LHS contained more rows than RHS";
+                OnDiskAtomIterator irhs = srhs.next();
+                assertContentEquals(ilhs, irhs);
+            }
+            assert !srhs.hasNext() : "RHS contained more rows than LHS";
         }
-        assert !srhs.hasNext() : "RHS contained more rows than LHS";
     }
 
     public static void assertContentEquals(OnDiskAtomIterator lhs, OnDiskAtomIterator rhs)
@@ -199,9 +215,9 @@
         public SSTableReader write(int expectedSize, Appender appender) throws IOException
         {
             File datafile = (dest == null) ? tempSSTableFile(ksname, cfname, generation) : new File(dest.filenameFor(Component.DATA));
-            SSTableWriter writer = new SSTableWriter(datafile.getAbsolutePath(), expectedSize, ActiveRepairService.UNREPAIRED_SSTABLE);
+            SSTableWriter writer = SSTableWriter.create(Descriptor.fromFilename(datafile.getAbsolutePath()), expectedSize, ActiveRepairService.UNREPAIRED_SSTABLE, 0);
             while (appender.append(writer)) { /* pass */ }
-            SSTableReader reader = writer.closeAndOpenReader();
+            SSTableReader reader = writer.finish(true);
             // mark all components for removal
             if (cleanup)
                 for (Component component : reader.components)
diff --git a/test/unit/org/apache/cassandra/io/sstable/format/SSTableReaderTest.java b/test/unit/org/apache/cassandra/io/sstable/format/SSTableReaderTest.java
new file mode 100644
index 0000000..6d07f1c
--- /dev/null
+++ b/test/unit/org/apache/cassandra/io/sstable/format/SSTableReaderTest.java
@@ -0,0 +1,648 @@
+package org.apache.cassandra.io.sstable.format;
+/*
+ *
+ * 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.
+ *
+ */
+
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.ThreadPoolExecutor;
+
+import com.google.common.collect.Sets;
+import org.apache.cassandra.cache.CachingOptions;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import org.apache.cassandra.OrderedJUnit4ClassRunner;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.Util;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.Operator;
+import org.apache.cassandra.db.BufferDecoratedKey;
+import org.apache.cassandra.db.ColumnFamily;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.IndexExpression;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.Mutation;
+import org.apache.cassandra.db.Row;
+import org.apache.cassandra.db.RowPosition;
+import org.apache.cassandra.db.columniterator.IdentityQueryFilter;
+import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.db.composites.Composites;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.dht.LocalPartitioner.LocalToken;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.MmappedSegmentedFile;
+import org.apache.cassandra.io.util.SegmentedFile;
+import org.apache.cassandra.service.CacheService;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import static org.apache.cassandra.Util.cellname;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(OrderedJUnit4ClassRunner.class)
+public class SSTableReaderTest
+{
+    public static final String KEYSPACE1 = "SSTableReaderTest";
+    public static final String CF_STANDARD = "Standard1";
+    public static final String CF_STANDARD2 = "Standard2";
+    public static final String CF_INDEXED = "Indexed1";
+    public static final String CF_STANDARDLOWINDEXINTERVAL = "StandardLowIndexInterval";
+
+    static Token t(int i)
+    {
+        return StorageService.getPartitioner().getToken(ByteBufferUtil.bytes(String.valueOf(i)));
+    }
+
+    @BeforeClass
+    public static void defineSchema() throws Exception
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD2),
+                                    SchemaLoader.indexCFMD(KEYSPACE1, CF_INDEXED, true),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARDLOWINDEXINTERVAL)
+                                                .minIndexInterval(8)
+                                                .maxIndexInterval(256)
+                                                .caching(CachingOptions.NONE));
+    }
+
+    @Test
+    public void testGetPositionsForRanges()
+    {
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard2");
+
+        // insert data and compact to a single sstable
+        CompactionManager.instance.disableAutoCompaction();
+        for (int j = 0; j < 10; j++)
+        {
+            ByteBuffer key = ByteBufferUtil.bytes(String.valueOf(j));
+            Mutation rm = new Mutation(KEYSPACE1, key);
+            rm.add("Standard2", cellname("0"), ByteBufferUtil.EMPTY_BYTE_BUFFER, j);
+            rm.applyUnsafe();
+        }
+        store.forceBlockingFlush();
+        CompactionManager.instance.performMaximal(store, false);
+
+        List<Range<Token>> ranges = new ArrayList<Range<Token>>();
+        // 1 key
+        ranges.add(new Range<Token>(t(0), t(1)));
+        // 2 keys
+        ranges.add(new Range<Token>(t(2), t(4)));
+        // wrapping range from key to end
+        ranges.add(new Range<Token>(t(6), StorageService.getPartitioner().getMinimumToken()));
+        // empty range (should be ignored)
+        ranges.add(new Range<Token>(t(9), t(91)));
+
+        // confirm that positions increase continuously
+        SSTableReader sstable = store.getSSTables().iterator().next();
+        long previous = -1;
+        for (Pair<Long,Long> section : sstable.getPositionsForRanges(ranges))
+        {
+            assert previous <= section.left : previous + " ! < " + section.left;
+            assert section.left < section.right : section.left + " ! < " + section.right;
+            previous = section.right;
+        }
+    }
+
+    @Test
+    public void testSpannedIndexPositions() throws IOException
+    {
+        long originalMaxSegmentSize = MmappedSegmentedFile.MAX_SEGMENT_SIZE;
+        MmappedSegmentedFile.MAX_SEGMENT_SIZE = 40; // each index entry is ~11 bytes, so this will generate lots of segments
+
+        try
+        {
+            Keyspace keyspace = Keyspace.open(KEYSPACE1);
+            ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
+
+            // insert a bunch of data and compact to a single sstable
+            CompactionManager.instance.disableAutoCompaction();
+            for (int j = 0; j < 100; j += 2)
+            {
+                ByteBuffer key = ByteBufferUtil.bytes(String.valueOf(j));
+                Mutation rm = new Mutation(KEYSPACE1, key);
+                rm.add("Standard1", cellname("0"), ByteBufferUtil.EMPTY_BYTE_BUFFER, j);
+                rm.applyUnsafe();
+            }
+            store.forceBlockingFlush();
+            CompactionManager.instance.performMaximal(store, false);
+
+            // check that all our keys are found correctly
+            SSTableReader sstable = store.getSSTables().iterator().next();
+            for (int j = 0; j < 100; j += 2)
+            {
+                DecoratedKey dk = Util.dk(String.valueOf(j));
+                FileDataInput file = sstable.getFileDataInput(sstable.getPosition(dk, SSTableReader.Operator.EQ).position);
+                DecoratedKey keyInDisk = sstable.partitioner.decorateKey(ByteBufferUtil.readWithShortLength(file));
+                assert keyInDisk.equals(dk) : String.format("%s != %s in %s", keyInDisk, dk, file.getPath());
+            }
+
+            // check no false positives
+            for (int j = 1; j < 110; j += 2)
+            {
+                DecoratedKey dk = Util.dk(String.valueOf(j));
+                assert sstable.getPosition(dk, SSTableReader.Operator.EQ) == null;
+            }
+        }
+        finally
+        {
+            MmappedSegmentedFile.MAX_SEGMENT_SIZE = originalMaxSegmentSize;
+        }
+    }
+
+    @Test
+    public void testPersistentStatistics()
+    {
+
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
+
+        for (int j = 0; j < 100; j += 2)
+        {
+            ByteBuffer key = ByteBufferUtil.bytes(String.valueOf(j));
+            Mutation rm = new Mutation(KEYSPACE1, key);
+            rm.add("Standard1", cellname("0"), ByteBufferUtil.EMPTY_BYTE_BUFFER, j);
+            rm.applyUnsafe();
+        }
+        store.forceBlockingFlush();
+
+        clearAndLoad(store);
+        assert store.metric.maxRowSize.getValue() != 0;
+    }
+
+    private void clearAndLoad(ColumnFamilyStore cfs)
+    {
+        cfs.clearUnsafe();
+        cfs.loadNewSSTables();
+    }
+
+    @Test
+    public void testReadRateTracking()
+    {
+        // try to make sure CASSANDRA-8239 never happens again
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
+
+        for (int j = 0; j < 10; j++)
+        {
+            ByteBuffer key = ByteBufferUtil.bytes(String.valueOf(j));
+            Mutation rm = new Mutation(KEYSPACE1, key);
+            rm.add("Standard1", cellname("0"), ByteBufferUtil.EMPTY_BYTE_BUFFER, j);
+            rm.apply();
+        }
+        store.forceBlockingFlush();
+
+        SSTableReader sstable = store.getSSTables().iterator().next();
+        assertEquals(0, sstable.getReadMeter().count());
+
+        DecoratedKey key = sstable.partitioner.decorateKey(ByteBufferUtil.bytes("4"));
+        store.getColumnFamily(key, Composites.EMPTY, Composites.EMPTY, false, 100, 100);
+        assertEquals(1, sstable.getReadMeter().count());
+        store.getColumnFamily(key, cellname("0"), cellname("0"), false, 100, 100);
+        assertEquals(2, sstable.getReadMeter().count());
+        store.getColumnFamily(Util.namesQueryFilter(store, key, cellname("0")));
+        assertEquals(3, sstable.getReadMeter().count());
+    }
+
+    @Test
+    public void testGetPositionsForRangesWithKeyCache()
+    {
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard2");
+        CacheService.instance.keyCache.setCapacity(100);
+
+        // insert data and compact to a single sstable
+        CompactionManager.instance.disableAutoCompaction();
+        for (int j = 0; j < 10; j++)
+        {
+            ByteBuffer key = ByteBufferUtil.bytes(String.valueOf(j));
+            Mutation rm = new Mutation(KEYSPACE1, key);
+            rm.add("Standard2", cellname("0"), ByteBufferUtil.EMPTY_BYTE_BUFFER, j);
+            rm.applyUnsafe();
+        }
+        store.forceBlockingFlush();
+        CompactionManager.instance.performMaximal(store, false);
+
+        SSTableReader sstable = store.getSSTables().iterator().next();
+        long p2 = sstable.getPosition(k(2), SSTableReader.Operator.EQ).position;
+        long p3 = sstable.getPosition(k(3), SSTableReader.Operator.EQ).position;
+        long p6 = sstable.getPosition(k(6), SSTableReader.Operator.EQ).position;
+        long p7 = sstable.getPosition(k(7), SSTableReader.Operator.EQ).position;
+
+        Pair<Long, Long> p = sstable.getPositionsForRanges(makeRanges(t(2), t(6))).get(0);
+
+        // range are start exclusive so we should start at 3
+        assert p.left == p3;
+
+        // to capture 6 we have to stop at the start of 7
+        assert p.right == p7;
+    }
+
+    @Test
+    public void testPersistentStatisticsWithSecondaryIndex()
+    {
+        // Create secondary index and flush to disk
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Indexed1");
+        ByteBuffer key = ByteBufferUtil.bytes(String.valueOf("k1"));
+        Mutation rm = new Mutation(KEYSPACE1, key);
+        rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(1L), System.currentTimeMillis());
+        rm.applyUnsafe();
+        store.forceBlockingFlush();
+
+        // check if opening and querying works
+        assertIndexQueryWorks(store);
+    }
+    public void testGetPositionsKeyCacheStats()
+    {
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard2");
+        CacheService.instance.keyCache.setCapacity(1000);
+
+        // insert data and compact to a single sstable
+        CompactionManager.instance.disableAutoCompaction();
+        for (int j = 0; j < 10; j++)
+        {
+            ByteBuffer key = ByteBufferUtil.bytes(String.valueOf(j));
+            Mutation rm = new Mutation("Keyspace1", key);
+            rm.add("Standard2", cellname("0"), ByteBufferUtil.EMPTY_BYTE_BUFFER, j);
+            rm.apply();
+        }
+        store.forceBlockingFlush();
+        CompactionManager.instance.performMaximal(store, false);
+
+        SSTableReader sstable = store.getSSTables().iterator().next();
+        sstable.getPosition(k(2), SSTableReader.Operator.EQ);
+        assertEquals(0, sstable.getKeyCacheHit());
+        assertEquals(1, sstable.getBloomFilterTruePositiveCount());
+        sstable.getPosition(k(2), SSTableReader.Operator.EQ);
+        assertEquals(1, sstable.getKeyCacheHit());
+        assertEquals(2, sstable.getBloomFilterTruePositiveCount());
+        sstable.getPosition(k(15), SSTableReader.Operator.EQ);
+        assertEquals(1, sstable.getKeyCacheHit());
+        assertEquals(2, sstable.getBloomFilterTruePositiveCount());
+
+    }
+
+
+    @Test
+    public void testOpeningSSTable() throws Exception
+    {
+        String ks = KEYSPACE1;
+        String cf = "Standard1";
+
+        // clear and create just one sstable for this test
+        Keyspace keyspace = Keyspace.open(ks);
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore(cf);
+        store.clearUnsafe();
+        store.disableAutoCompaction();
+
+        DecoratedKey firstKey = null, lastKey = null;
+        long timestamp = System.currentTimeMillis();
+        for (int i = 0; i < store.metadata.getMinIndexInterval(); i++)
+        {
+            DecoratedKey key = Util.dk(String.valueOf(i));
+            if (firstKey == null)
+                firstKey = key;
+            if (lastKey == null)
+                lastKey = key;
+            if (store.metadata.getKeyValidator().compare(lastKey.getKey(), key.getKey()) < 0)
+                lastKey = key;
+            Mutation rm = new Mutation(ks, key.getKey());
+            rm.add(cf, cellname("col"),
+                   ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp);
+            rm.applyUnsafe();
+        }
+        store.forceBlockingFlush();
+
+        SSTableReader sstable = store.getSSTables().iterator().next();
+        Descriptor desc = sstable.descriptor;
+
+        // test to see if sstable can be opened as expected
+        SSTableReader target = SSTableReader.open(desc);
+        Assert.assertEquals(target.getIndexSummarySize(), 1);
+        Assert.assertArrayEquals(ByteBufferUtil.getArray(firstKey.getKey()), target.getIndexSummaryKey(0));
+        assert target.first.equals(firstKey);
+        assert target.last.equals(lastKey);
+        target.selfRef().release();
+    }
+
+    @Test
+    public void testLoadingSummaryUsesCorrectPartitioner() throws Exception
+    {
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Indexed1");
+        ByteBuffer key = ByteBufferUtil.bytes(String.valueOf("k1"));
+        Mutation rm = new Mutation(KEYSPACE1, key);
+        rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(1L), System.currentTimeMillis());
+        rm.applyUnsafe();
+        store.forceBlockingFlush();
+
+        ColumnFamilyStore indexCfs = store.indexManager.getIndexForColumn(ByteBufferUtil.bytes("birthdate")).getIndexCfs();
+        assert indexCfs.partitioner instanceof LocalPartitioner;
+        SSTableReader sstable = indexCfs.getSSTables().iterator().next();
+        assert sstable.first.getToken() instanceof LocalToken;
+
+        try(SegmentedFile.Builder ibuilder = SegmentedFile.getBuilder(DatabaseDescriptor.getIndexAccessMode(), false);
+            SegmentedFile.Builder dbuilder = SegmentedFile.getBuilder(DatabaseDescriptor.getDiskAccessMode(), sstable.compression))
+        {
+            sstable.saveSummary(ibuilder, dbuilder);
+        }
+        SSTableReader reopened = SSTableReader.open(sstable.descriptor);
+        assert reopened.first.getToken() instanceof LocalToken;
+        reopened.selfRef().release();
+    }
+
+    /** see CASSANDRA-5407 */
+    @Test
+    public void testGetScannerForNoIntersectingRanges() throws Exception
+    {
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
+        ByteBuffer key = ByteBufferUtil.bytes(String.valueOf("k1"));
+        Mutation rm = new Mutation(KEYSPACE1, key);
+        rm.add("Standard1", cellname("xyz"), ByteBufferUtil.bytes("abc"), 0);
+        rm.applyUnsafe();
+        store.forceBlockingFlush();
+        boolean foundScanner = false;
+        for (SSTableReader s : store.getSSTables())
+        {
+            try (ISSTableScanner scanner = s.getScanner(new Range<Token>(t(0), t(1)), null))
+            {
+                scanner.next(); // throws exception pre 5407
+                foundScanner = true;
+            }
+        }
+        assertTrue(foundScanner);
+    }
+
+    @Test
+    public void testGetPositionsForRangesFromTableOpenedForBulkLoading() throws IOException
+    {
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard2");
+
+        // insert data and compact to a single sstable. The
+        // number of keys inserted is greater than index_interval
+        // to ensure multiple segments in the index file
+        CompactionManager.instance.disableAutoCompaction();
+        for (int j = 0; j < 130; j++)
+        {
+            ByteBuffer key = ByteBufferUtil.bytes(String.valueOf(j));
+            Mutation rm = new Mutation(KEYSPACE1, key);
+            rm.add("Standard2", cellname("0"), ByteBufferUtil.EMPTY_BYTE_BUFFER, j);
+            rm.applyUnsafe();
+        }
+        store.forceBlockingFlush();
+        CompactionManager.instance.performMaximal(store, false);
+
+        // construct a range which is present in the sstable, but whose
+        // keys are not found in the first segment of the index.
+        List<Range<Token>> ranges = new ArrayList<Range<Token>>();
+        ranges.add(new Range<Token>(t(98), t(99)));
+
+        SSTableReader sstable = store.getSSTables().iterator().next();
+        List<Pair<Long,Long>> sections = sstable.getPositionsForRanges(ranges);
+        assert sections.size() == 1 : "Expected to find range in sstable" ;
+
+        // re-open the same sstable as it would be during bulk loading
+        Set<Component> components = Sets.newHashSet(Component.DATA, Component.PRIMARY_INDEX);
+        if (sstable.compression)
+            components.add(Component.COMPRESSION_INFO);
+        SSTableReader bulkLoaded = SSTableReader.openForBatch(sstable.descriptor, components, store.metadata, sstable.partitioner);
+        sections = bulkLoaded.getPositionsForRanges(ranges);
+        assert sections.size() == 1 : "Expected to find range in sstable opened for bulk loading";
+        bulkLoaded.selfRef().release();
+    }
+
+    @Test
+    public void testIndexSummaryReplacement() throws IOException, ExecutionException, InterruptedException
+    {
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        final ColumnFamilyStore store = keyspace.getColumnFamilyStore("StandardLowIndexInterval"); // index interval of 8, no key caching
+        CompactionManager.instance.disableAutoCompaction();
+
+        final int NUM_ROWS = 512;
+        for (int j = 0; j < NUM_ROWS; j++)
+        {
+            ByteBuffer key = ByteBufferUtil.bytes(String.format("%3d", j));
+            Mutation rm = new Mutation(KEYSPACE1, key);
+            rm.add("StandardLowIndexInterval", Util.cellname("0"), ByteBufferUtil.bytes(String.format("%3d", j)), j);
+            rm.applyUnsafe();
+        }
+        store.forceBlockingFlush();
+        CompactionManager.instance.performMaximal(store, false);
+
+        Collection<SSTableReader> sstables = store.getSSTables();
+        assert sstables.size() == 1;
+        final SSTableReader sstable = sstables.iterator().next();
+
+        ThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(5);
+        List<Future> futures = new ArrayList<>(NUM_ROWS * 2);
+        for (int i = 0; i < NUM_ROWS; i++)
+        {
+            final ByteBuffer key = ByteBufferUtil.bytes(String.format("%3d", i));
+            final int index = i;
+
+            futures.add(executor.submit(new Runnable()
+            {
+                public void run()
+                {
+                    ColumnFamily result = store.getColumnFamily(sstable.partitioner.decorateKey(key), Composites.EMPTY, Composites.EMPTY, false, 100, 100);
+                    assertFalse(result.isEmpty());
+                    assertEquals(0, ByteBufferUtil.compare(String.format("%3d", index).getBytes(), result.getColumn(Util.cellname("0")).value()));
+                }
+            }));
+
+            futures.add(executor.submit(new Runnable()
+            {
+                public void run()
+                {
+                    Iterable<DecoratedKey> results = store.keySamples(
+                            new Range<>(sstable.partitioner.getMinimumToken(), sstable.partitioner.getToken(key)));
+                    assertTrue(results.iterator().hasNext());
+                }
+            }));
+        }
+
+        SSTableReader replacement;
+        try (LifecycleTransaction txn = store.getTracker().tryModify(Arrays.asList(sstable), OperationType.UNKNOWN))
+        {
+            replacement = sstable.cloneWithNewSummarySamplingLevel(store, 1);
+            txn.update(replacement, true);
+            txn.finish();
+        }
+        for (Future future : futures)
+            future.get();
+
+        assertEquals(sstable.estimatedKeys(), replacement.estimatedKeys(), 1);
+    }
+
+    @Test
+    public void testIndexSummaryUpsampleAndReload() throws Exception
+    {
+        long originalMaxSegmentSize = MmappedSegmentedFile.MAX_SEGMENT_SIZE;
+        MmappedSegmentedFile.MAX_SEGMENT_SIZE = 40; // each index entry is ~11 bytes, so this will generate lots of segments
+
+        try
+        {
+            testIndexSummaryUpsampleAndReload0();
+        }
+        finally
+        {
+            MmappedSegmentedFile.MAX_SEGMENT_SIZE = originalMaxSegmentSize;
+        }
+    }
+
+    private void testIndexSummaryUpsampleAndReload0() throws Exception
+    {
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        final ColumnFamilyStore store = keyspace.getColumnFamilyStore("StandardLowIndexInterval"); // index interval of 8, no key caching
+        CompactionManager.instance.disableAutoCompaction();
+
+        final int NUM_ROWS = 512;
+        for (int j = 0; j < NUM_ROWS; j++)
+        {
+            ByteBuffer key = ByteBufferUtil.bytes(String.format("%3d", j));
+            Mutation rm = new Mutation(KEYSPACE1, key);
+            rm.add("StandardLowIndexInterval", Util.cellname("0"), ByteBufferUtil.bytes(String.format("%3d", j)), j);
+            rm.applyUnsafe();
+        }
+        store.forceBlockingFlush();
+        CompactionManager.instance.performMaximal(store, false);
+
+        Collection<SSTableReader> sstables = store.getSSTables();
+        assert sstables.size() == 1;
+        final SSTableReader sstable = sstables.iterator().next();
+
+        try (LifecycleTransaction txn = store.getTracker().tryModify(Arrays.asList(sstable), OperationType.UNKNOWN))
+        {
+            SSTableReader replacement = sstable.cloneWithNewSummarySamplingLevel(store, sstable.getIndexSummarySamplingLevel() + 1);
+            txn.update(replacement, true);
+            txn.finish();
+        }
+        SSTableReader reopen = SSTableReader.open(sstable.descriptor);
+        assert reopen.getIndexSummarySamplingLevel() == sstable.getIndexSummarySamplingLevel() + 1;
+    }
+
+    @Test
+    public void testIndexSummaryDownsampleAndReload() throws Exception
+    {
+        long originalMaxSegmentSize = MmappedSegmentedFile.MAX_SEGMENT_SIZE;
+        MmappedSegmentedFile.MAX_SEGMENT_SIZE = 40; // each index entry is ~11 bytes, so this will generate lots of segments
+
+        try
+        {
+            testIndexSummaryDownsampleAndReload0();
+        }
+        finally
+        {
+            MmappedSegmentedFile.MAX_SEGMENT_SIZE = originalMaxSegmentSize;
+        }
+    }
+
+    private void testIndexSummaryDownsampleAndReload0() throws Exception
+    {
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        final ColumnFamilyStore store = keyspace.getColumnFamilyStore("StandardLowIndexInterval"); // index interval of 8, no key caching
+        CompactionManager.instance.disableAutoCompaction();
+
+        final int NUM_ROWS = 512;
+        for (int j = 0; j < NUM_ROWS; j++)
+        {
+            ByteBuffer key = ByteBufferUtil.bytes(String.format("%3d", j));
+            Mutation rm = new Mutation(KEYSPACE1, key);
+            rm.add("StandardLowIndexInterval", Util.cellname("0"), ByteBufferUtil.bytes(String.format("%3d", j)), j);
+            rm.applyUnsafe();
+        }
+        store.forceBlockingFlush();
+        CompactionManager.instance.performMaximal(store, false);
+
+        Collection<SSTableReader> sstables = store.getSSTables();
+        assert sstables.size() == 1;
+        final SSTableReader sstable = sstables.iterator().next();
+
+        try (LifecycleTransaction txn = store.getTracker().tryModify(Arrays.asList(sstable), OperationType.UNKNOWN))
+        {
+            SSTableReader replacement = sstable.cloneWithNewSummarySamplingLevel(store, sstable.getIndexSummarySamplingLevel() / 2);
+            txn.update(replacement, true);
+            txn.finish();
+        }
+        SSTableReader reopen = SSTableReader.open(sstable.descriptor);
+        assert Arrays.equals(sstable.ifile.copyReadableBounds(), reopen.ifile.copyReadableBounds());
+        assert Arrays.equals(sstable.dfile.copyReadableBounds(), reopen.dfile.copyReadableBounds());
+    }
+
+
+    private void assertIndexQueryWorks(ColumnFamilyStore indexedCFS)
+    {
+        assert "Indexed1".equals(indexedCFS.name);
+
+        // make sure all sstables including 2ary indexes load from disk
+        for (ColumnFamilyStore cfs : indexedCFS.concatWithIndexes())
+            clearAndLoad(cfs);
+
+        // query using index to see if sstable for secondary index opens
+        IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), Operator.EQ, ByteBufferUtil.bytes(1L));
+        List<IndexExpression> clause = Arrays.asList(expr);
+        Range<RowPosition> range = Util.range("", "");
+        List<Row> rows = indexedCFS.search(range, clause, new IdentityQueryFilter(), 100);
+        assert rows.size() == 1;
+    }
+
+    private List<Range<Token>> makeRanges(Token left, Token right)
+    {
+        return Arrays.asList(new Range<>(left, right));
+    }
+
+    private DecoratedKey k(int i)
+    {
+        return new BufferDecoratedKey(t(i), ByteBufferUtil.bytes(String.valueOf(i)));
+    }
+}
diff --git a/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java b/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
index 7751a51..19fa7c4 100644
--- a/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
@@ -18,6 +18,7 @@
 package org.apache.cassandra.io.sstable.metadata;
 
 import java.io.File;
+import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.util.EnumSet;
@@ -25,15 +26,20 @@
 import java.util.Set;
 
 import com.google.common.collect.Sets;
+
 import org.junit.Test;
 
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.apache.cassandra.db.composites.SimpleDenseCellNameType;
 import org.apache.cassandra.db.marshal.BytesType;
 import org.apache.cassandra.dht.RandomPartitioner;
 import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.util.DataOutputStreamAndChannel;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.io.sstable.format.big.BigFormat;
+import org.apache.cassandra.io.util.DataOutputStreamPlus;
+import org.apache.cassandra.io.util.BufferedDataOutputStreamPlus;
 import org.apache.cassandra.io.util.RandomAccessReader;
 import org.apache.cassandra.utils.EstimatedHistogram;
 
@@ -44,18 +50,51 @@
     @Test
     public void testSerialization() throws IOException
     {
+        Map<MetadataType, MetadataComponent> originalMetadata = constructMetadata();
+
+        MetadataSerializer serializer = new MetadataSerializer();
+        File statsFile = serialize(originalMetadata, serializer, BigFormat.latestVersion);
+
+        Descriptor desc = new Descriptor( statsFile.getParentFile(), "", "", 0, Descriptor.Type.FINAL);
+        try (RandomAccessReader in = RandomAccessReader.open(statsFile))
+        {
+            Map<MetadataType, MetadataComponent> deserialized = serializer.deserialize(desc, in, EnumSet.allOf(MetadataType.class));
+
+            for (MetadataType type : MetadataType.values())
+            {
+                assertEquals(originalMetadata.get(type), deserialized.get(type));
+            }
+        }
+    }
+
+    public File serialize(Map<MetadataType, MetadataComponent> metadata, MetadataSerializer serializer, Version version)
+            throws IOException, FileNotFoundException
+    {
+        // Serialize to tmp file
+        File statsFile = File.createTempFile(Component.STATS.name, null);
+        try (DataOutputStreamPlus out = new BufferedDataOutputStreamPlus(new FileOutputStream(statsFile)))
+        {
+            serializer.serialize(metadata, version, out);
+        }
+        return statsFile;
+    }
+
+    public Map<MetadataType, MetadataComponent> constructMetadata()
+    {
         EstimatedHistogram rowSizes = new EstimatedHistogram(new long[] { 1L, 2L },
                                                              new long[] { 3L, 4L, 5L });
         EstimatedHistogram columnCounts = new EstimatedHistogram(new long[] { 6L, 7L },
                                                                  new long[] { 8L, 9L, 10L });
-        ReplayPosition rp = new ReplayPosition(11L, 12);
+        ReplayPosition start = new ReplayPosition(11L, 12);
+        ReplayPosition end = new ReplayPosition(15L, 9);
         long minTimestamp = 2162517136L;
         long maxTimestamp = 4162517136L;
 
         MetadataCollector collector = new MetadataCollector(new SimpleDenseCellNameType(BytesType.instance))
                                                       .estimatedRowSize(rowSizes)
                                                       .estimatedColumnCount(columnCounts)
-                                                      .replayPosition(rp);
+                                                      .commitLogLowerBound(start)
+                                                      .commitLogUpperBound(end);
         collector.updateMinTimestamp(minTimestamp);
         collector.updateMaxTimestamp(maxTimestamp);
 
@@ -66,23 +105,35 @@
         String partitioner = RandomPartitioner.class.getCanonicalName();
         double bfFpChance = 0.1;
         Map<MetadataType, MetadataComponent> originalMetadata = collector.finalizeMetadata(partitioner, bfFpChance, 0);
+        return originalMetadata;
+    }
+
+    @Test
+    public void testLaReadsLb() throws IOException
+    {
+        Map<MetadataType, MetadataComponent> originalMetadata = constructMetadata();
 
         MetadataSerializer serializer = new MetadataSerializer();
-        // Serialize to tmp file
-        File statsFile = File.createTempFile(Component.STATS.name, null);
-        try (DataOutputStreamAndChannel out = new DataOutputStreamAndChannel(new FileOutputStream(statsFile)))
-        {
-            serializer.serialize(originalMetadata, out);
-        }
+        // Write metadata in two minor formats.
+        File statsFileLb = serialize(originalMetadata, serializer, BigFormat.instance.getVersion("lb"));
+        File statsFileLa = serialize(originalMetadata, serializer, BigFormat.instance.getVersion("la"));
 
-        Descriptor desc = new Descriptor(Descriptor.Version.CURRENT, statsFile.getParentFile(), "", "", 0, Descriptor.Type.FINAL);
-        try (RandomAccessReader in = RandomAccessReader.open(statsFile))
+        // Reading both as earlier version should yield identical results.
+        Descriptor desc = new Descriptor("la", statsFileLb.getParentFile(), "", "", 0, Descriptor.Type.FINAL, DatabaseDescriptor.getSSTableFormat());
+        try (RandomAccessReader inLb = RandomAccessReader.open(statsFileLb);
+             RandomAccessReader inLa = RandomAccessReader.open(statsFileLa))
         {
-            Map<MetadataType, MetadataComponent> deserialized = serializer.deserialize(desc, in, EnumSet.allOf(MetadataType.class));
+            Map<MetadataType, MetadataComponent> deserializedLb = serializer.deserialize(desc, inLb, EnumSet.allOf(MetadataType.class));
+            Map<MetadataType, MetadataComponent> deserializedLa = serializer.deserialize(desc, inLa, EnumSet.allOf(MetadataType.class));
 
             for (MetadataType type : MetadataType.values())
             {
-                assertEquals(originalMetadata.get(type), deserialized.get(type));
+                assertEquals(deserializedLa.get(type), deserializedLb.get(type));
+                if (!originalMetadata.get(type).equals(deserializedLb.get(type)))
+                {
+                    // Currently only STATS can be different. Change if no longer the case
+                    assertEquals(MetadataType.STATS, type);
+                }
             }
         }
     }
diff --git a/test/unit/org/apache/cassandra/io/util/BufferedDataOutputStreamTest.java b/test/unit/org/apache/cassandra/io/util/BufferedDataOutputStreamTest.java
new file mode 100644
index 0000000..0c58e41
--- /dev/null
+++ b/test/unit/org/apache/cassandra/io/util/BufferedDataOutputStreamTest.java
@@ -0,0 +1,509 @@
+/*
+ *
+ * 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.io.util;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.UTFDataFormatException;
+import java.lang.reflect.Field;
+import java.nio.BufferOverflowException;
+import java.nio.ByteBuffer;
+import java.nio.channels.WritableByteChannel;
+import java.util.Random;
+
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+public class BufferedDataOutputStreamTest
+{
+
+    @Test(expected = BufferOverflowException.class)
+    public void testDataOutputBufferFixedByes() throws Exception
+    {
+        try (DataOutputBufferFixed dob = new DataOutputBufferFixed())
+        {
+            try
+            {
+                for (int ii = 0; ii < 128; ii++)
+                    dob.write(0);
+            }
+            catch (BufferOverflowException e)
+            {
+                fail("Should not throw BufferOverflowException yet");
+            }
+            dob.write(0);
+        }
+    }
+
+    @Test(expected = BufferOverflowException.class)
+    public void testDataOutputBufferFixedByteBuffer() throws Exception
+    {
+        try (DataOutputBufferFixed dob = new DataOutputBufferFixed())
+        {
+            try
+            {
+                dob.write(ByteBuffer.allocateDirect(128));
+            }
+            catch (BufferOverflowException e)
+            {
+                fail("Should not throw BufferOverflowException yet");
+            }
+            dob.write(ByteBuffer.allocateDirect(1));
+        }
+    }
+
+    WritableByteChannel adapter = new WritableByteChannel()
+    {
+
+        @Override
+        public boolean isOpen()  {return true;}
+
+        @Override
+        public void close() throws IOException {}
+
+        @Override
+        public int write(ByteBuffer src) throws IOException
+        {
+            int retval = src.remaining();
+            while (src.hasRemaining())
+                generated.write(src.get());
+            return retval;
+        }
+
+    };
+
+    BufferedDataOutputStreamPlus fakeStream = new BufferedDataOutputStreamPlus(adapter, 8);
+
+    @SuppressWarnings("resource")
+    @Test(expected = NullPointerException.class)
+    public void testNullChannel()
+    {
+        new BufferedDataOutputStreamPlus((WritableByteChannel)null, 8);
+    }
+
+    @SuppressWarnings("resource")
+    @Test(expected = IllegalArgumentException.class)
+    public void testTooSmallBuffer()
+    {
+        new BufferedDataOutputStreamPlus(adapter, 7);
+    }
+
+    @Test(expected = NullPointerException.class)
+    public void testNullBuffer() throws Exception
+    {
+        byte type[] = null;
+        fakeStream.write(type, 0, 1);
+    }
+
+    @Test(expected = IndexOutOfBoundsException.class)
+    public void testNegativeOffset() throws Exception
+    {
+        byte type[] = new byte[10];
+        fakeStream.write(type, -1, 1);
+    }
+
+    @Test(expected = IndexOutOfBoundsException.class)
+    public void testNegativeLength() throws Exception
+    {
+        byte type[] = new byte[10];
+        fakeStream.write(type, 0, -1);
+    }
+
+    @Test(expected = IndexOutOfBoundsException.class)
+    public void testTooBigLength() throws Exception
+    {
+        byte type[] = new byte[10];
+        fakeStream.write(type, 0, 11);
+    }
+
+    @Test(expected = IndexOutOfBoundsException.class)
+    public void testTooBigLengthWithOffset() throws Exception
+    {
+        byte type[] = new byte[10];
+        fakeStream.write(type, 8, 3);
+    }
+
+    @Test(expected = IndexOutOfBoundsException.class)
+    public void testTooBigOffset() throws Exception
+    {
+        byte type[] = new byte[10];
+        fakeStream.write(type, 11, 1);
+    }
+
+    static final Random r;
+
+    static Field baos_bytes;
+    static {
+        long seed = System.nanoTime();
+        //seed = 210187780999648L;
+        System.out.println("Seed " + seed);
+        r = new Random(seed);
+        try
+        {
+            baos_bytes = ByteArrayOutputStream.class.getDeclaredField("buf");
+            baos_bytes.setAccessible(true);
+        }
+        catch (Throwable t)
+        {
+            throw new RuntimeException(t);
+        }
+    }
+
+    private ByteArrayOutputStream generated;
+    private BufferedDataOutputStreamPlus ndosp;
+
+    private ByteArrayOutputStream canonical;
+    private DataOutputStreamPlus dosp;
+
+    void setUp()
+    {
+
+        generated = new ByteArrayOutputStream();
+        canonical = new ByteArrayOutputStream();
+        dosp = new WrappedDataOutputStreamPlus(canonical);
+        ndosp = new BufferedDataOutputStreamPlus(adapter, 4096);
+    }
+
+    @Test
+    public void testFuzz() throws Exception
+    {
+        for (int ii = 0; ii < 30; ii++)
+            fuzzOnce();
+    }
+
+    String simple = "foobar42";
+    public static final String twoByte = "\u0180";
+    public static final String threeByte = "\u34A8";
+    public static final String fourByte = "\uD841\uDF79";
+
+    @SuppressWarnings("unused")
+    private void fuzzOnce() throws Exception
+    {
+        setUp();
+        int iteration = 0;
+        int bytesChecked = 0;
+        int action = 0;
+        while (generated.size() < 1024 * 1024 * 8)
+        {
+            action = r.nextInt(19);
+
+            //System.out.println("Action " + action + " iteration " + iteration);
+            iteration++;
+
+            switch (action)
+            {
+            case 0:
+            {
+                generated.flush();
+                dosp.flush();
+                break;
+            }
+            case 1:
+            {
+                int val = r.nextInt();
+                dosp.write(val);
+                ndosp.write(val);
+                break;
+            }
+            case 2:
+            {
+                byte randomBytes[] = new byte[r.nextInt(4096 * 2 + 1)];
+                r.nextBytes(randomBytes);
+                dosp.write(randomBytes);
+                ndosp.write(randomBytes);
+                break;
+            }
+            case 3:
+            {
+                byte randomBytes[] = new byte[r.nextInt(4096 * 2 + 1)];
+                r.nextBytes(randomBytes);
+                int offset = randomBytes.length == 0 ? 0 : r.nextInt(randomBytes.length);
+                int length = randomBytes.length == 0 ? 0 : r.nextInt(randomBytes.length - offset);
+                dosp.write(randomBytes, offset, length);
+                ndosp.write(randomBytes, offset, length);
+                break;
+            }
+            case 4:
+            {
+                boolean val = r.nextInt(2) == 0;
+                dosp.writeBoolean(val);
+                ndosp.writeBoolean(val);
+                break;
+            }
+            case 5:
+            {
+                int val = r.nextInt();
+                dosp.writeByte(val);
+                ndosp.writeByte(val);
+                break;
+            }
+            case 6:
+            {
+                int val = r.nextInt();
+                dosp.writeShort(val);
+                ndosp.writeShort(val);
+                break;
+            }
+            case 7:
+            {
+                int val = r.nextInt();
+                dosp.writeChar(val);
+                ndosp.writeChar(val);
+                break;
+            }
+            case 8:
+            {
+                int val = r.nextInt();
+                dosp.writeInt(val);
+                ndosp.writeInt(val);
+                break;
+            }
+            case 9:
+            {
+                int val = r.nextInt();
+                dosp.writeLong(val);
+                ndosp.writeLong(val);
+                break;
+            }
+            case 10:
+            {
+                float val = r.nextFloat();
+                dosp.writeFloat(val);
+                ndosp.writeFloat(val);
+                break;
+            }
+            case 11:
+            {
+                double val = r.nextDouble();
+                dosp.writeDouble(val);
+                ndosp.writeDouble(val);
+                break;
+            }
+            case 12:
+            {
+                dosp.writeBytes(simple);
+                ndosp.writeBytes(simple);
+                break;
+            }
+            case 13:
+            {
+                dosp.writeChars(twoByte);
+                ndosp.writeChars(twoByte);
+                break;
+            }
+            case 14:
+            {
+                StringBuilder sb = new StringBuilder();
+                int length = r.nextInt(500);
+                //Some times do big strings
+                if (r.nextDouble() > .95)
+                    length += 4000;
+                sb.append(simple + twoByte + threeByte + fourByte);
+                for (int ii = 0; ii < length; ii++)
+                {
+                    sb.append((char)(r.nextInt() & 0xffff));
+                }
+                String str = sb.toString();
+                writeUTFLegacy(str, dosp);
+                ndosp.writeUTF(str);
+                break;
+            }
+            case 15:
+            {
+                StringBuilder sb = new StringBuilder();
+                int length = r.nextInt(500);
+                sb.append("the very model of a modern major general familiar with all things animal vegetable and mineral");
+                for (int ii = 0; ii < length; ii++)
+                {
+                    sb.append(' ');
+                }
+                String str = sb.toString();
+                writeUTFLegacy(str, dosp);
+                ndosp.writeUTF(str);
+                break;
+            }
+            case 16:
+            {
+                ByteBuffer buf = ByteBuffer.allocate(r.nextInt(1024 * 8 + 1));
+                r.nextBytes(buf.array());
+                buf.position(buf.capacity() == 0 ? 0 : r.nextInt(buf.capacity()));
+                buf.limit(buf.position() + (buf.capacity() - buf.position() == 0 ? 0 : r.nextInt(buf.capacity() - buf.position())));
+                ByteBuffer dup = buf.duplicate();
+                ndosp.write(buf.duplicate());
+                assertEquals(dup.position(), buf.position());
+                assertEquals(dup.limit(), buf.limit());
+                dosp.write(buf.duplicate());
+                break;
+            }
+            case 17:
+            {
+                ByteBuffer buf = ByteBuffer.allocateDirect(r.nextInt(1024 * 8 + 1));
+                while (buf.hasRemaining())
+                    buf.put((byte)r.nextInt());
+                buf.position(buf.capacity() == 0 ? 0 : r.nextInt(buf.capacity()));
+                buf.limit(buf.position() + (buf.capacity() - buf.position() == 0 ? 0 : r.nextInt(buf.capacity() - buf.position())));
+                ByteBuffer dup = buf.duplicate();
+                ndosp.write(buf.duplicate());
+                assertEquals(dup.position(), buf.position());
+                assertEquals(dup.limit(), buf.limit());
+                dosp.write(buf.duplicate());
+                break;
+            }
+            case 18:
+            {
+                try (Memory buf = Memory.allocate(r.nextInt(1024 * 8 - 1) + 1);)
+                {
+                    for (int ii = 0; ii < buf.size(); ii++)
+                        buf.setByte(ii, (byte)r.nextInt());
+                    long offset = buf.size() == 0 ? 0 : r.nextInt((int)buf.size());
+                    long length = (buf.size() - offset == 0 ? 0 : r.nextInt((int)(buf.size() - offset)));
+                    ndosp.write(buf, offset, length);
+                    dosp.write(buf, offset, length);
+                }
+                break;
+            }
+            default:
+                fail("Shouldn't reach here");
+            }
+            //bytesChecked = assertSameOutput(bytesChecked, action, iteration);
+        }
+
+        assertSameOutput(0, -1, iteration);
+    }
+
+    public static void writeUTFLegacy(String str, OutputStream out) throws IOException
+    {
+        int utfCount = 0, length = str.length();
+        for (int i = 0; i < length; i++)
+        {
+            int charValue = str.charAt(i);
+            if (charValue > 0 && charValue <= 127)
+            {
+                utfCount++;
+            }
+            else if (charValue <= 2047)
+            {
+                utfCount += 2;
+            }
+            else
+            {
+                utfCount += 3;
+            }
+        }
+        if (utfCount > 65535)
+        {
+            throw new UTFDataFormatException(); //$NON-NLS-1$
+        }
+        byte utfBytes[] = new byte[utfCount + 2];
+        int utfIndex = 2;
+        for (int i = 0; i < length; i++)
+        {
+            int charValue = str.charAt(i);
+            if (charValue > 0 && charValue <= 127)
+            {
+                utfBytes[utfIndex++] = (byte) charValue;
+            }
+            else if (charValue <= 2047)
+            {
+                utfBytes[utfIndex++] = (byte) (0xc0 | (0x1f & (charValue >> 6)));
+                utfBytes[utfIndex++] = (byte) (0x80 | (0x3f & charValue));
+            }
+            else
+            {
+                utfBytes[utfIndex++] = (byte) (0xe0 | (0x0f & (charValue >> 12)));
+                utfBytes[utfIndex++] = (byte) (0x80 | (0x3f & (charValue >> 6)));
+                utfBytes[utfIndex++] = (byte) (0x80 | (0x3f & charValue));
+            }
+        }
+        utfBytes[0] = (byte) (utfCount >> 8);
+        utfBytes[1] = (byte) utfCount;
+        out.write(utfBytes);
+    }
+
+    private int assertSameOutput(int bytesChecked, int lastAction, int iteration) throws Exception
+    {
+        ndosp.flush();
+        dosp.flush();
+
+        byte generatedBytes[] = (byte[])baos_bytes.get(generated);
+        byte canonicalBytes[] = (byte[])baos_bytes.get(canonical);
+
+        int count = generated.size();
+        if (count != canonical.size())
+            System.out.println("Failed at " + bytesChecked + " last action " + lastAction + " iteration " + iteration);
+        assertEquals(count, canonical.size());
+        for (;bytesChecked < count; bytesChecked++)
+        {
+            byte generatedByte = generatedBytes[bytesChecked];
+            byte canonicalByte = canonicalBytes[bytesChecked];
+            if (generatedByte != canonicalByte)
+                System.out.println("Failed at " + bytesChecked + " last action " + lastAction + " iteration " + iteration);
+            assertEquals(generatedByte, canonicalByte);
+        }
+        return count;
+    }
+
+    @Test
+    public void testWriteUTF() throws Exception
+    {
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        DataOutput dataOut = new DataOutputStream(baos);
+
+        StringBuilder sb = new StringBuilder(65535);
+        for (int ii = 0; ii < 1 << 16; ii++)
+        {
+            String s = sb.toString();
+            UnbufferedDataOutputStreamPlus.writeUTF(s, dataOut);
+            DataInput dataIn = new DataInputStream(new ByteArrayInputStream(baos.toByteArray()));
+            assertEquals(s, dataIn.readUTF());
+            baos.reset();
+            sb.append("a");
+        }
+    }
+
+    @Test
+    public void testWriteUTFBigChar() throws Exception
+    {
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        DataOutput dataOut = new DataOutputStream(baos);
+
+        StringBuilder sb = new StringBuilder(65535);
+        for (int ii = 0; ii < 1 << 15; ii++)
+        {
+            String s = sb.toString();
+            UnbufferedDataOutputStreamPlus.writeUTF(s, dataOut);
+            DataInput dataIn = new DataInputStream(new ByteArrayInputStream(baos.toByteArray()));
+            assertEquals(s, dataIn.readUTF());
+            baos.reset();
+            if (ii == (1 << 15) - 1)
+                sb.append("a");
+            else
+                sb.append(twoByte);
+        }
+    }
+}
diff --git a/test/unit/org/apache/cassandra/io/util/BufferedRandomAccessFileTest.java b/test/unit/org/apache/cassandra/io/util/BufferedRandomAccessFileTest.java
index 7dbbdc2..0c1583d 100644
--- a/test/unit/org/apache/cassandra/io/util/BufferedRandomAccessFileTest.java
+++ b/test/unit/org/apache/cassandra/io/util/BufferedRandomAccessFileTest.java
@@ -21,6 +21,7 @@
 
 import org.apache.cassandra.service.FileCacheService;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.SyncUtil;
 
 import java.io.File;
 import java.io.FileOutputStream;
@@ -33,11 +34,9 @@
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
 
 import static org.apache.cassandra.Util.expectEOF;
 import static org.apache.cassandra.Util.expectException;
-
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertEquals;
 
@@ -129,7 +128,7 @@
             assert data[i] == 0;
         }
 
-        w.close();
+        w.finish();
         r.close();
     }
 
@@ -154,7 +153,7 @@
         assert negone == -1 : "We read past the end of the file, should have gotten EOF -1. Instead, " + negone;
 
         r.close();
-        w.close();
+        w.finish();
     }
 
     @Test
@@ -179,7 +178,7 @@
         w.write(biggerThenBuffer);
         assertEquals(biggerThenBuffer.length + lessThenBuffer.length, w.length());
 
-        w.close();
+        w.finish();
 
         // will use cachedlength
         RandomAccessReader r = RandomAccessReader.open(tmpFile);
@@ -224,7 +223,7 @@
             }
         });
 
-        w.close();
+        w.finish();
         r.close();
     }
 
@@ -234,7 +233,7 @@
         SequentialWriter w = createTempFile("brafSeek");
         byte[] data = generateByteArray(RandomAccessReader.DEFAULT_BUFFER_SIZE + 20);
         w.write(data);
-        w.close();
+        w.finish();
 
         final RandomAccessReader file = RandomAccessReader.open(w);
 
@@ -273,7 +272,7 @@
     {
         SequentialWriter w = createTempFile("brafSkipBytes");
         w.write(generateByteArray(RandomAccessReader.DEFAULT_BUFFER_SIZE * 2));
-        w.close();
+        w.finish();
 
         RandomAccessReader file = RandomAccessReader.open(w);
 
@@ -321,7 +320,7 @@
         r.read(new byte[4]);
         assertEquals(r.getFilePointer(), 20);
 
-        w.close();
+        w.finish();
         r.close();
     }
 
@@ -330,7 +329,7 @@
     {
         SequentialWriter file = createTempFile("brafGetPath");
         assert file.getPath().contains("brafGetPath");
-        file.close();
+        file.finish();
     }
 
     @Test
@@ -344,7 +343,8 @@
             for (final int offset : Arrays.asList(0, 8))
             {
                 File file1 = writeTemporaryFile(new byte[16]);
-                try (final RandomAccessReader file = RandomAccessReader.open(file1, bufferSize, null))
+                try (final ChannelProxy channel = new ChannelProxy(file1);
+                     final RandomAccessReader file = RandomAccessReader.open(channel, bufferSize, null))
                 {
                     expectEOF(new Callable<Object>()
                     {
@@ -361,7 +361,8 @@
             for (final int n : Arrays.asList(1, 2, 4, 8))
             {
                 File file1 = writeTemporaryFile(new byte[16]);
-                try (final RandomAccessReader file = RandomAccessReader.open(file1, bufferSize, null))
+                try (final ChannelProxy channel = new ChannelProxy(file1);
+                     final RandomAccessReader file = RandomAccessReader.open(channel, bufferSize, null))
                 {
                     expectEOF(new Callable<Object>()
                     {
@@ -379,7 +380,10 @@
     @Test
     public void testNotEOF() throws IOException
     {
-        assertEquals(1, RandomAccessReader.open(writeTemporaryFile(new byte[1])).read(new byte[2]));
+        try (final RandomAccessReader reader = RandomAccessReader.open(writeTemporaryFile(new byte[1])))
+        {
+            assertEquals(1, reader.read(new byte[2]));
+        }
     }
 
     @Test
@@ -407,7 +411,7 @@
         r.skipBytes(10);
         assertEquals(r.bytesRemaining(), r.length() - 10);
 
-        w.close();
+        w.finish();
         r.close();
     }
 
@@ -421,11 +425,11 @@
         try (final RandomAccessReader r = RandomAccessReader.open(new File(tmpFile.getPath())))
         {
             assert tmpFile.getPath().equals(r.getPath());
-    
+
             // Create a mark and move the rw there.
             final FileMark mark = r.mark();
             r.reset(mark);
-    
+
             // Expect this call to succeed.
             r.bytesPastMark(mark);
         }
@@ -439,7 +443,7 @@
         byte[] data = generateByteArray(RandomAccessReader.DEFAULT_BUFFER_SIZE + 20);
 
         w.write(data);
-        w.close(); // will flush
+        w.finish();
 
         final RandomAccessReader r = RandomAccessReader.open(new File(w.getPath()));
 
@@ -465,7 +469,7 @@
         try (RandomAccessReader copy = RandomAccessReader.open(new File(r.getPath())))
         {
             ByteBuffer contents = copy.readBytes((int) copy.length());
-    
+
             assertEquals(contents.limit(), data.length);
             assertEquals(ByteBufferUtil.compare(contents, data), 0);
         }
@@ -477,7 +481,7 @@
         SequentialWriter w = createTempFile("brafTestMark");
         w.write(new byte[30]);
 
-        w.close();
+        w.finish();
 
         RandomAccessReader file = RandomAccessReader.open(w);
 
@@ -513,12 +517,12 @@
         {
             w.write(new byte[30]);
             w.flush();
-    
+
             try (RandomAccessReader r = RandomAccessReader.open(w))
             {
                 r.seek(10);
                 r.mark();
-        
+
                 r.seek(0);
                 r.bytesPastMark();
             }
@@ -531,7 +535,6 @@
         //see https://issues.apache.org/jira/browse/CASSANDRA-7756
 
         final FileCacheService.CacheKey cacheKey = new FileCacheService.CacheKey();
-
         final int THREAD_COUNT = 40;
         ExecutorService executorService = Executors.newFixedThreadPool(THREAD_COUNT);
 
@@ -539,10 +542,10 @@
         SequentialWriter w2 = createTempFile("fscache2");
 
         w1.write(new byte[30]);
-        w1.close();
+        w1.finish();
 
         w2.write(new byte[30]);
-        w2.close();
+        w2.finish();
 
         for (int i = 0; i < 20; i++)
         {
@@ -619,34 +622,6 @@
             }
         }, IllegalArgumentException.class);
 
-        // Any write() call should fail
-        expectException(new Callable<Object>()
-        {
-            public Object call() throws IOException
-            {
-                copy.write(1);
-                return null;
-            }
-        }, UnsupportedOperationException.class);
-
-        expectException(new Callable<Object>()
-        {
-            public Object call() throws IOException
-            {
-                copy.write(new byte[1]);
-                return null;
-            }
-        }, UnsupportedOperationException.class);
-
-        expectException(new Callable<Object>()
-        {
-            public Object call() throws IOException
-            {
-                copy.write(new byte[3], 0, 2);
-                return null;
-            }
-        }, UnsupportedOperationException.class);
-
         copy.seek(0);
         copy.skipBytes(5);
 
@@ -677,7 +652,7 @@
 
         assertEquals(new String(content), "cccccccccc");
 
-        file.close();
+        file.finish();
         copy.close();
     }
 
@@ -691,16 +666,6 @@
         }
     }
 
-    @Test (expected=IOException.class)
-    public void testSetLengthDuringReadMode() throws IOException
-    {
-        File tmpFile = File.createTempFile("set_length_during_read_mode", "bin");
-        try (RandomAccessReader file = RandomAccessReader.open(tmpFile))
-        {
-            file.setLength(4L);
-        }
-    }
-
     private SequentialWriter createTempFile(String name) throws IOException
     {
         File tempFile = File.createTempFile(name, null);
@@ -715,7 +680,7 @@
         f.deleteOnExit();
         FileOutputStream fout = new FileOutputStream(f);
         fout.write(data);
-        fout.getFD().sync();
+        SyncUtil.sync(fout);
         fout.close();
         return f;
     }
diff --git a/test/unit/org/apache/cassandra/io/util/ByteBufferDataInputTest.java b/test/unit/org/apache/cassandra/io/util/ByteBufferDataInputTest.java
new file mode 100644
index 0000000..af2d1dc
--- /dev/null
+++ b/test/unit/org/apache/cassandra/io/util/ByteBufferDataInputTest.java
@@ -0,0 +1,67 @@
+/*
+* 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.io.util;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.MappedByteBuffer;
+
+import org.junit.Test;
+
+import junit.framework.Assert;
+
+public class ByteBufferDataInputTest
+{
+
+    @Test
+    public void testPositionAndSeek() throws IOException
+    {
+        ByteBufferDataInput bbdi = new ByteBufferDataInput(ByteBuffer.allocate(100), "", 15, 1);
+        Assert.assertEquals(99, bbdi.bytesRemaining());
+        Assert.assertEquals(16, bbdi.getPosition());
+        Assert.assertEquals(16, bbdi.getFilePointer());
+//        Assert.assertTrue(bbdi.markSupported());
+        FileMark mark = bbdi.mark();
+        bbdi.seek(115);
+        Assert.assertEquals(115, bbdi.getFilePointer());
+        Assert.assertEquals(115, bbdi.getPosition());
+        Assert.assertEquals(99, bbdi.bytesPastMark(mark));
+        Assert.assertTrue(bbdi.isEOF());
+        bbdi.seek(15);
+        Assert.assertEquals(15, bbdi.getFilePointer());
+        Assert.assertEquals(15, bbdi.getPosition());
+        try
+        {
+            bbdi.seek(14);
+            Assert.assertTrue(false);
+        }
+        catch (IOException t)
+        {
+        }
+        try
+        {
+            bbdi.seek(116);
+            Assert.assertTrue(false);
+        }
+        catch (IOException t)
+        {
+        }
+    }
+
+}
diff --git a/test/unit/org/apache/cassandra/io/util/ChecksummedSequentialWriterTest.java b/test/unit/org/apache/cassandra/io/util/ChecksummedSequentialWriterTest.java
new file mode 100644
index 0000000..9731a8d
--- /dev/null
+++ b/test/unit/org/apache/cassandra/io/util/ChecksummedSequentialWriterTest.java
@@ -0,0 +1,92 @@
+/*
+* 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.io.util;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.junit.After;
+
+import junit.framework.Assert;
+
+public class ChecksummedSequentialWriterTest extends SequentialWriterTest
+{
+
+    private final List<TestableCSW> writers = new ArrayList<>();
+
+    @After
+    public void cleanup()
+    {
+        for (TestableSW sw : writers)
+            sw.file.delete();
+        writers.clear();
+    }
+
+    protected TestableTransaction newTest() throws IOException
+    {
+        TestableCSW sw = new TestableCSW();
+        writers.add(sw);
+        return sw;
+    }
+
+    private static class TestableCSW extends TestableSW
+    {
+        final File crcFile;
+
+        private TestableCSW() throws IOException
+        {
+            this(tempFile("compressedsequentialwriter"),
+                 tempFile("compressedsequentialwriter.checksum"));
+        }
+
+        private TestableCSW(File file, File crcFile) throws IOException
+        {
+            this(file, crcFile, new ChecksummedSequentialWriter(file, BUFFER_SIZE, crcFile));
+        }
+
+        private TestableCSW(File file, File crcFile, SequentialWriter sw) throws IOException
+        {
+            super(file, sw);
+            this.crcFile = crcFile;
+        }
+
+        protected void assertInProgress() throws Exception
+        {
+            super.assertInProgress();
+            Assert.assertTrue(crcFile.exists());
+            Assert.assertEquals(0, crcFile.length());
+        }
+
+        protected void assertPrepared() throws Exception
+        {
+            super.assertPrepared();
+            Assert.assertTrue(crcFile.exists());
+            Assert.assertFalse(0 == crcFile.length());
+        }
+
+        protected void assertAborted() throws Exception
+        {
+            super.assertAborted();
+            Assert.assertFalse(crcFile.exists());
+        }
+    }
+
+}
diff --git a/test/unit/org/apache/cassandra/io/util/DataOutputTest.java b/test/unit/org/apache/cassandra/io/util/DataOutputTest.java
index 2063e9a..1fb5597 100644
--- a/test/unit/org/apache/cassandra/io/util/DataOutputTest.java
+++ b/test/unit/org/apache/cassandra/io/util/DataOutputTest.java
@@ -31,38 +31,52 @@
 import java.io.RandomAccessFile;
 import java.nio.ByteBuffer;
 import java.nio.channels.Channels;
+import java.util.ArrayDeque;
+import java.util.Deque;
 import java.util.Random;
+import java.util.concurrent.Callable;
 import java.util.concurrent.ThreadLocalRandom;
 
 import org.junit.Assert;
 import org.junit.Test;
 
+import org.apache.cassandra.io.compress.BufferType;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 public class DataOutputTest
 {
-
     @Test
-    public void testDataOutputStreamPlus() throws IOException
+    public void testWrappedDataOutputStreamPlus() throws IOException
     {
         ByteArrayOutputStream bos = new ByteArrayOutputStream();
-        DataOutputStreamPlus write = new DataOutputStreamPlus(bos);
+        DataOutputStreamPlus write = new WrappedDataOutputStreamPlus(bos);
         DataInput canon = testWrite(write);
         DataInput test = new DataInputStream(new ByteArrayInputStream(bos.toByteArray()));
         testRead(test, canon);
     }
 
     @Test
-    public void testDataOutputChannelAndChannel() throws IOException
+    public void testWrappedDataOutputChannelAndChannel() throws IOException
     {
         ByteArrayOutputStream bos = new ByteArrayOutputStream();
-        DataOutputStreamPlus write = new DataOutputStreamAndChannel(Channels.newChannel(bos));
+        DataOutputStreamPlus write = new WrappedDataOutputStreamPlus(bos);
         DataInput canon = testWrite(write);
         DataInput test = new DataInputStream(new ByteArrayInputStream(bos.toByteArray()));
         testRead(test, canon);
     }
 
     @Test
+    public void testBufferedDataOutputStreamPlusAndChannel() throws IOException
+    {
+        ByteArrayOutputStream bos = new ByteArrayOutputStream();
+        DataOutputStreamPlus write = new BufferedDataOutputStreamPlus(Channels.newChannel(bos));
+        DataInput canon = testWrite(write);
+        write.close();
+        DataInput test = new DataInputStream(new ByteArrayInputStream(bos.toByteArray()));
+        testRead(test, canon);
+    }
+
+    @Test
     public void testDataOutputBuffer() throws IOException
     {
         DataOutputBuffer write = new DataOutputBuffer();
@@ -72,10 +86,22 @@
     }
 
     @Test
+    public void testDataOutputBufferZeroReallocate() throws IOException
+    {
+        try (DataOutputBufferSpy write = new DataOutputBufferSpy())
+        {
+            for (int ii = 0; ii < 1000000; ii++)
+            {
+                write.superReallocate(0);
+            }
+        }
+    }
+
+    @Test
     public void testDataOutputDirectByteBuffer() throws IOException
     {
         ByteBuffer buf = wrap(new byte[345], true);
-        DataOutputByteBuffer write = new DataOutputByteBuffer(buf.duplicate());
+        BufferedDataOutputStreamPlus write = new BufferedDataOutputStreamPlus(null, buf.duplicate());
         DataInput canon = testWrite(write);
         DataInput test = new DataInputStream(new ByteArrayInputStream(ByteBufferUtil.getArray(buf)));
         testRead(test, canon);
@@ -85,21 +111,229 @@
     public void testDataOutputHeapByteBuffer() throws IOException
     {
         ByteBuffer buf = wrap(new byte[345], false);
-        DataOutputByteBuffer write = new DataOutputByteBuffer(buf.duplicate());
+        BufferedDataOutputStreamPlus write = new BufferedDataOutputStreamPlus(null, buf.duplicate());
         DataInput canon = testWrite(write);
         DataInput test = new DataInputStream(new ByteArrayInputStream(ByteBufferUtil.getArray(buf)));
         testRead(test, canon);
     }
 
+    private static class DataOutputBufferSpy extends DataOutputBuffer
+    {
+        Deque<Long> sizes = new ArrayDeque<>();
+
+        DataOutputBufferSpy()
+        {
+            sizes.offer(128L);
+        }
+
+        void publicFlush() throws IOException
+        {
+            //Going to allow it to double instead of specifying a count
+            doFlush(1);
+        }
+
+        void superReallocate(int count) throws IOException
+        {
+            super.reallocate(count);
+        }
+
+        @Override
+        protected void reallocate(long count)
+        {
+            if (count <= 0)
+                return;
+            Long lastSize = sizes.peekLast();
+            long newSize = calculateNewSize(count);
+            sizes.offer(newSize);
+            if (newSize > DataOutputBuffer.MAX_ARRAY_SIZE)
+                throw new RuntimeException();
+            if (newSize < 0)
+                throw new AssertionError();
+            if (lastSize != null && newSize <= lastSize)
+                throw new AssertionError();
+        }
+
+        @Override
+        protected long capacity()
+        {
+            return sizes.peekLast().intValue();
+        }
+    }
+
+    //Check for overflow at the max size, without actually allocating all the memory
+    @Test
+    public void testDataOutputBufferMaxSizeFake() throws IOException
+    {
+        try (DataOutputBufferSpy write = new DataOutputBufferSpy())
+        {
+            boolean threw = false;
+            try
+            {
+                while (true)
+                    write.publicFlush();
+            }
+            catch (RuntimeException e) {
+                if (e.getClass() == RuntimeException.class)
+                    threw = true;
+            }
+            Assert.assertTrue(threw);
+            Assert.assertTrue(write.sizes.peekLast() >= DataOutputBuffer.MAX_ARRAY_SIZE);
+        }
+    }
+
+    @Test
+    public void testDataOutputBufferMaxSize() throws IOException
+    {
+        //Need a lot of heap to run this test for real.
+        //Tested everything else as much as possible since we can't do it all the time
+        if (Runtime.getRuntime().maxMemory() < 5033164800L)
+            return;
+
+        try (DataOutputBuffer write = new DataOutputBuffer())
+        {
+            //Doesn't throw up to DataOuptutBuffer.MAX_ARRAY_SIZE which is the array size limit in Java
+            for (int ii = 0; ii < DataOutputBuffer.MAX_ARRAY_SIZE / 8; ii++)
+                write.writeLong(0);
+            write.write(new byte[7]);
+
+            //Should fail due to validation
+            checkThrowsRuntimeException(validateReallocationCallable( write, DataOutputBuffer.MAX_ARRAY_SIZE + 1));
+            //Check that it does throw
+            checkThrowsRuntimeException(new Callable<Object>()
+            {
+                public Object call() throws Exception
+                {
+                    write.write(42);
+                    return null;
+                }
+            });
+        }
+    }
+
+    //Can't test it for real without tons of heap so test as much validation as possible
+    @Test
+    public void testDataOutputBufferBigReallocation() throws Exception
+    {
+        //Check saturating cast behavior
+        Assert.assertEquals(DataOutputBuffer.MAX_ARRAY_SIZE, DataOutputBuffer.saturatedArraySizeCast(DataOutputBuffer.MAX_ARRAY_SIZE + 1L));
+        Assert.assertEquals(DataOutputBuffer.MAX_ARRAY_SIZE, DataOutputBuffer.saturatedArraySizeCast(DataOutputBuffer.MAX_ARRAY_SIZE));
+        Assert.assertEquals(DataOutputBuffer.MAX_ARRAY_SIZE - 1, DataOutputBuffer.saturatedArraySizeCast(DataOutputBuffer.MAX_ARRAY_SIZE - 1));
+        Assert.assertEquals(0, DataOutputBuffer.saturatedArraySizeCast(0));
+        Assert.assertEquals(1, DataOutputBuffer.saturatedArraySizeCast(1));
+        checkThrowsIAE(saturatedArraySizeCastCallable(-1));
+
+        //Check checked cast behavior
+        checkThrowsIAE(checkedArraySizeCastCallable(DataOutputBuffer.MAX_ARRAY_SIZE + 1L));
+        Assert.assertEquals(DataOutputBuffer.MAX_ARRAY_SIZE, DataOutputBuffer.checkedArraySizeCast(DataOutputBuffer.MAX_ARRAY_SIZE));
+        Assert.assertEquals(DataOutputBuffer.MAX_ARRAY_SIZE - 1, DataOutputBuffer.checkedArraySizeCast(DataOutputBuffer.MAX_ARRAY_SIZE - 1));
+        Assert.assertEquals(0, DataOutputBuffer.checkedArraySizeCast(0));
+        Assert.assertEquals(1, DataOutputBuffer.checkedArraySizeCast(1));
+        checkThrowsIAE(checkedArraySizeCastCallable(-1));
+
+
+        try (DataOutputBuffer write = new DataOutputBuffer())
+        {
+            //Checked validation performed by DOB
+            Assert.assertEquals(DataOutputBuffer.MAX_ARRAY_SIZE, write.validateReallocation(DataOutputBuffer.MAX_ARRAY_SIZE + 1L));
+            Assert.assertEquals(DataOutputBuffer.MAX_ARRAY_SIZE, write.validateReallocation(DataOutputBuffer.MAX_ARRAY_SIZE));
+            Assert.assertEquals(DataOutputBuffer.MAX_ARRAY_SIZE - 1, write.validateReallocation(DataOutputBuffer.MAX_ARRAY_SIZE - 1));
+            checkThrowsRuntimeException(validateReallocationCallable( write, 0));
+            checkThrowsRuntimeException(validateReallocationCallable( write, 1));
+            checkThrowsIAE(validateReallocationCallable( write, -1));
+        }
+    }
+
+    Callable<Object> saturatedArraySizeCastCallable(final long value)
+    {
+        return new Callable<Object>()
+        {
+            @Override
+            public Object call() throws Exception
+            {
+                return DataOutputBuffer.saturatedArraySizeCast(value);
+            }
+        };
+    }
+
+    Callable<Object> checkedArraySizeCastCallable(final long value)
+    {
+        return new Callable<Object>()
+        {
+            @Override
+            public Object call() throws Exception
+            {
+                return DataOutputBuffer.checkedArraySizeCast(value);
+            }
+        };
+    }
+
+    Callable<Object> validateReallocationCallable(final DataOutputBuffer write, final long value)
+    {
+        return new Callable<Object>()
+        {
+            @Override
+            public Object call() throws Exception
+            {
+                return write.validateReallocation(value);
+            }
+        };
+    }
+
+    private static void checkThrowsIAE(Callable<Object> c)
+    {
+        checkThrowsException(c, IllegalArgumentException.class);
+    }
+
+    private static void checkThrowsRuntimeException(Callable<Object> c)
+    {
+        checkThrowsException(c, RuntimeException.class);
+    }
+
+    private static void checkThrowsException(Callable<Object> c, Class<?> exceptionClass)
+    {
+        boolean threw = false;
+        try
+        {
+            c.call();
+        }
+        catch (Throwable t)
+        {
+            if (t.getClass() == exceptionClass)
+                threw = true;
+        }
+        Assert.assertTrue(threw);
+    }
+
     @Test
     public void testSafeMemoryWriter() throws IOException
     {
-        SafeMemoryWriter write = new SafeMemoryWriter(10);
-        DataInput canon = testWrite(write);
-        byte[] bytes = new byte[345];
-        write.currentBuffer().getBytes(0, bytes, 0, 345);
-        DataInput test = new DataInputStream(new ByteArrayInputStream(bytes));
-        testRead(test, canon);
+        try (SafeMemoryWriter write = new SafeMemoryWriter(10))
+        {
+            DataInput canon = testWrite(write);
+            byte[] bytes = new byte[345];
+            write.currentBuffer().getBytes(0, bytes, 0, 345);
+            DataInput test = new DataInputStream(new ByteArrayInputStream(bytes));
+            testRead(test, canon);
+        }
+    }
+
+    @Test
+    public void testWrappedFileOutputStream() throws IOException
+    {
+        File file = FileUtils.createTempFile("dataoutput", "test");
+        try
+        {
+            DataOutputStreamPlus write = new WrappedDataOutputStreamPlus(new FileOutputStream(file));
+            DataInput canon = testWrite(write);
+            write.close();
+            DataInputStream test = new DataInputStream(new FileInputStream(file));
+            testRead(test, canon);
+            test.close();
+        }
+        finally
+        {
+            Assert.assertTrue(file.delete());
+        }
     }
 
     @Test
@@ -108,7 +342,7 @@
         File file = FileUtils.createTempFile("dataoutput", "test");
         try
         {
-            DataOutputStreamAndChannel write = new DataOutputStreamAndChannel(new FileOutputStream(file));
+            DataOutputStreamPlus write = new BufferedDataOutputStreamPlus(new FileOutputStream(file));
             DataInput canon = testWrite(write);
             write.close();
             DataInputStream test = new DataInputStream(new FileInputStream(file));
@@ -127,8 +361,9 @@
         File file = FileUtils.createTempFile("dataoutput", "test");
         try
         {
+            @SuppressWarnings("resource")
             final RandomAccessFile raf = new RandomAccessFile(file, "rw");
-            DataOutputStreamAndChannel write = new DataOutputStreamAndChannel(Channels.newOutputStream(raf.getChannel()), raf.getChannel());
+            DataOutputStreamPlus write = new BufferedDataOutputStreamPlus(raf.getChannel());
             DataInput canon = testWrite(write);
             write.close();
             DataInputStream test = new DataInputStream(new FileInputStream(file));
@@ -145,8 +380,8 @@
     public void testSequentialWriter() throws IOException
     {
         File file = FileUtils.createTempFile("dataoutput", "test");
-        final SequentialWriter writer = new SequentialWriter(file, 32);
-        DataOutputStreamAndChannel write = new DataOutputStreamAndChannel(writer, writer);
+        final SequentialWriter writer = new SequentialWriter(file, 32, BufferType.ON_HEAP);
+        DataOutputStreamPlus write = new WrappedDataOutputStreamPlus(writer.finishOnClose());
         DataInput canon = testWrite(write);
         write.flush();
         write.close();
@@ -162,32 +397,33 @@
         final DataOutput canon = new DataOutputStream(bos);
         Random rnd = ThreadLocalRandom.current();
 
-        byte[] bytes = new byte[50];
+        int size = 50;
+        byte[] bytes = new byte[size];
         rnd.nextBytes(bytes);
         ByteBufferUtil.writeWithLength(bytes, test);
         ByteBufferUtil.writeWithLength(bytes, canon);
 
-        bytes = new byte[50];
+        bytes = new byte[size];
         rnd.nextBytes(bytes);
         ByteBufferUtil.writeWithLength(wrap(bytes, false), test);
         ByteBufferUtil.writeWithLength(bytes, canon);
 
-        bytes = new byte[50];
+        bytes = new byte[size];
         rnd.nextBytes(bytes);
         ByteBufferUtil.writeWithLength(wrap(bytes, true), test);
         ByteBufferUtil.writeWithLength(bytes, canon);
 
-        bytes = new byte[50];
+        bytes = new byte[size];
         rnd.nextBytes(bytes);
         ByteBufferUtil.writeWithShortLength(bytes, test);
         ByteBufferUtil.writeWithShortLength(bytes, canon);
 
-        bytes = new byte[50];
+        bytes = new byte[size];
         rnd.nextBytes(bytes);
         ByteBufferUtil.writeWithShortLength(wrap(bytes, false), test);
         ByteBufferUtil.writeWithShortLength(bytes, canon);
 
-        bytes = new byte[50];
+        bytes = new byte[size];
         rnd.nextBytes(bytes);
         ByteBufferUtil.writeWithShortLength(wrap(bytes, true), test);
         ByteBufferUtil.writeWithShortLength(bytes, canon);
@@ -247,8 +483,9 @@
             test.readInt();
             assert false;
         }
-        catch (EOFException exc)
+        catch (EOFException ignore)
         {
+            // it worked
         }
     }
 
diff --git a/test/unit/org/apache/cassandra/io/util/FileUtilsTest.java b/test/unit/org/apache/cassandra/io/util/FileUtilsTest.java
new file mode 100644
index 0000000..7110504
--- /dev/null
+++ b/test/unit/org/apache/cassandra/io/util/FileUtilsTest.java
@@ -0,0 +1,55 @@
+/**
+ * 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.io.util;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.nio.file.Files;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class FileUtilsTest
+{
+
+    @Test
+    public void testTruncate() throws IOException
+    {
+        File file = FileUtils.createTempFile("testTruncate", "1");
+        final String expected = "The quick brown fox jumps over the lazy dog";
+
+        Files.write(file.toPath(), expected.getBytes());
+        assertTrue(file.exists());
+
+        byte[] b = Files.readAllBytes(file.toPath());
+        assertEquals(expected, new String(b, Charset.forName("UTF-8")));
+
+        FileUtils.truncate(file.getAbsolutePath(), 10);
+        b = Files.readAllBytes(file.toPath());
+        assertEquals("The quick ", new String(b, Charset.forName("UTF-8")));
+
+        FileUtils.truncate(file.getAbsolutePath(), 0);
+        b = Files.readAllBytes(file.toPath());
+        assertEquals(0, b.length);
+    }
+
+}
diff --git a/test/unit/org/apache/cassandra/io/util/MappedFileDataInputTest.java b/test/unit/org/apache/cassandra/io/util/MappedFileDataInputTest.java
deleted file mode 100644
index 2bc388d..0000000
--- a/test/unit/org/apache/cassandra/io/util/MappedFileDataInputTest.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
-* 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.io.util;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.MappedByteBuffer;
-
-import org.junit.Test;
-
-import junit.framework.Assert;
-
-public class MappedFileDataInputTest
-{
-
-    @Test
-    public void testPositionAndSeek() throws IOException
-    {
-        MappedFileDataInput bbdi = new MappedFileDataInput((MappedByteBuffer) ByteBuffer.allocateDirect(100), "", 15, 1);
-        Assert.assertEquals(99, bbdi.bytesRemaining());
-//        Assert.assertEquals(16, bbdi.getPosition());
-        Assert.assertEquals(16, bbdi.getFilePointer());
-//        Assert.assertTrue(bbdi.markSupported());
-        FileMark mark = bbdi.mark();
-        bbdi.seek(115);
-        Assert.assertEquals(115, bbdi.getFilePointer());
-//        Assert.assertEquals(115, bbdi.getPosition());
-        Assert.assertEquals(99, bbdi.bytesPastMark(mark));
-        Assert.assertTrue(bbdi.isEOF());
-        bbdi.seek(15);
-        Assert.assertEquals(15, bbdi.getFilePointer());
-//        Assert.assertEquals(15, bbdi.getPosition());
-        try
-        {
-            bbdi.seek(14);
-            Assert.assertTrue(false);
-        }
-        catch (IOException t)
-        {
-        }
-        try
-        {
-            bbdi.seek(116);
-            Assert.assertTrue(false);
-        }
-        catch (IOException t)
-        {
-        }
-    }
-
-}
diff --git a/test/unit/org/apache/cassandra/io/util/MemoryTest.java b/test/unit/org/apache/cassandra/io/util/MemoryTest.java
new file mode 100644
index 0000000..9be69ac
--- /dev/null
+++ b/test/unit/org/apache/cassandra/io/util/MemoryTest.java
@@ -0,0 +1,76 @@
+/*
+* 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.io.util;
+
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.concurrent.ThreadLocalRandom;
+
+import org.junit.Test;
+
+import junit.framework.Assert;
+import org.apache.cassandra.utils.memory.MemoryUtil;
+
+public class MemoryTest
+{
+
+    @Test
+    public void testByteBuffers()
+    {
+        byte[] bytes = new byte[1000];
+        ThreadLocalRandom.current().nextBytes(bytes);
+        final Memory memory = Memory.allocate(bytes.length);
+        memory.setBytes(0, bytes, 0, bytes.length);
+        ByteBuffer canon = ByteBuffer.wrap(bytes).order(ByteOrder.nativeOrder());
+        test(canon, memory);
+        memory.setBytes(0, new byte[1000], 0, 1000);
+        memory.setBytes(0, canon.duplicate());
+        test(canon, memory);
+        memory.close();
+    }
+
+    private static void test(ByteBuffer canon, Memory memory)
+    {
+        ByteBuffer hollow = MemoryUtil.getHollowDirectByteBuffer();
+        test(canon, hollow, memory, 0, 1000);
+        test(canon, hollow, memory, 33, 100);
+        test(canon, hollow, memory, 77, 77);
+        test(canon, hollow, memory, 903, 96);
+    }
+
+    private static void test(ByteBuffer canon, ByteBuffer hollow, Memory memory, int offset, int length)
+    {
+        canon = canon.duplicate();
+        canon.position(offset).limit(offset + length);
+        canon = canon.slice().order(ByteOrder.nativeOrder());
+        test(canon, memory.asByteBuffer(offset, length));
+        memory.setByteBuffer(hollow, offset, length);
+        test(canon, hollow);
+    }
+
+    private static void test(ByteBuffer canon, ByteBuffer test)
+    {
+        Assert.assertEquals(canon, test);
+        for (int i = 0 ; i <= canon.limit() - 4 ; i += 4)
+            Assert.assertEquals(canon.getInt(i), test.getInt(i));
+        for (int i = 0 ; i <= canon.limit() - 8 ; i += 8)
+            Assert.assertEquals(canon.getLong(i), test.getLong(i));
+    }
+
+}
diff --git a/test/unit/org/apache/cassandra/io/util/NIODataInputStreamTest.java b/test/unit/org/apache/cassandra/io/util/NIODataInputStreamTest.java
new file mode 100644
index 0000000..953d882
--- /dev/null
+++ b/test/unit/org/apache/cassandra/io/util/NIODataInputStreamTest.java
@@ -0,0 +1,684 @@
+/*
+ *
+ * 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.io.util;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.EOFException;
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.channels.ReadableByteChannel;
+import java.util.ArrayDeque;
+import java.util.Queue;
+import java.util.Random;
+
+import org.apache.cassandra.io.util.NIODataInputStream;
+import org.junit.Test;
+
+import com.google.common.base.Charsets;
+
+import static org.junit.Assert.*;
+
+public class NIODataInputStreamTest
+{
+
+    Random r;
+    ByteBuffer corpus = ByteBuffer.allocate(1024 * 1024 * 8);
+
+    void init()
+    {
+        long seed = System.nanoTime();
+        //seed = 365238103404423L;
+        System.out.println("Seed " + seed);
+        r = new Random(seed);
+        r.nextBytes(corpus.array());
+    }
+
+    class FakeChannel implements ReadableByteChannel
+    {
+
+        @Override
+        public boolean isOpen() { return true; }
+
+        @Override
+        public void close() throws IOException {}
+
+        @Override
+        public int read(ByteBuffer dst) throws IOException { return 0; }
+
+    }
+
+    class DummyChannel implements ReadableByteChannel
+    {
+
+        boolean isOpen = true;
+        Queue<ByteBuffer> slices = new ArrayDeque<ByteBuffer>();
+
+        DummyChannel()
+        {
+            slices.clear();
+            corpus.clear();
+
+            while (corpus.hasRemaining())
+            {
+                int sliceSize = Math.min(corpus.remaining(), r.nextInt(8193));
+                corpus.limit(corpus.position() + sliceSize);
+                slices.offer(corpus.slice());
+                corpus.position(corpus.limit());
+                corpus.limit(corpus.capacity());
+            }
+            corpus.clear();
+        }
+
+        @Override
+        public boolean isOpen()
+        {
+            return isOpen();
+        }
+
+        @Override
+        public void close() throws IOException
+        {
+            isOpen = false;
+        }
+
+        @Override
+        public int read(ByteBuffer dst) throws IOException
+        {
+            if (!isOpen) throw new IOException("closed");
+            if (slices.isEmpty()) return -1;
+
+            if (!slices.peek().hasRemaining())
+            {
+                if (r.nextInt(2) == 1)
+                {
+                    return 0;
+                }
+                else
+                {
+                    slices.poll();
+                    if (slices.isEmpty()) return -1;
+                }
+            }
+
+            ByteBuffer slice = slices.peek();
+            int oldLimit = slice.limit();
+
+            int copied = 0;
+            if (slice.remaining() > dst.remaining())
+            {
+                slice.limit(slice.position() + dst.remaining());
+                copied = dst.remaining();
+            }
+            else
+            {
+                copied = slice.remaining();
+            }
+
+            dst.put(slice);
+            slice.limit(oldLimit);
+
+
+            return copied;
+        }
+
+    }
+
+    NIODataInputStream fakeStream = new NIODataInputStream(new FakeChannel(), 8);
+
+    @Test(expected = IOException.class)
+    public void testResetThrows() throws Exception
+    {
+        fakeStream.reset();
+    }
+
+    @Test(expected = NullPointerException.class)
+    public void testNullReadBuffer() throws Exception
+    {
+        fakeStream.read(null, 0, 1);
+    }
+
+    @Test(expected = IndexOutOfBoundsException.class)
+    public void testNegativeOffsetReadBuffer() throws Exception
+    {
+        fakeStream.read(new byte[1], -1, 1);
+    }
+
+    @Test(expected = IndexOutOfBoundsException.class)
+    public void testNegativeLengthReadBuffer() throws Exception
+    {
+        fakeStream.read(new byte[1], 0, -1);
+    }
+
+    @Test(expected = IndexOutOfBoundsException.class)
+    public void testLengthToBigReadBuffer() throws Exception
+    {
+        fakeStream.read(new byte[1], 0, 2);
+    }
+
+    @Test(expected = IndexOutOfBoundsException.class)
+    public void testLengthToWithOffsetBigReadBuffer() throws Exception
+    {
+        fakeStream.read(new byte[1], 1, 1);
+    }
+
+    @Test(expected = UnsupportedOperationException.class)
+    public void testReadLine() throws Exception
+    {
+        fakeStream.readLine();
+    }
+
+    @Test
+    public void testMarkSupported() throws Exception
+    {
+        assertFalse(fakeStream.markSupported());
+    }
+
+    @SuppressWarnings("resource")
+    @Test(expected = IllegalArgumentException.class)
+    public void testTooSmallBufferSize() throws Exception
+    {
+        new NIODataInputStream(new FakeChannel(), 4);
+    }
+
+    @SuppressWarnings("resource")
+    @Test(expected = NullPointerException.class)
+    public void testNullRBC() throws Exception
+    {
+        new NIODataInputStream(null, 8);
+    }
+
+    @SuppressWarnings("resource")
+    @Test
+    public void testAvailable() throws Exception
+    {
+        init();
+        DummyChannel dc = new DummyChannel();
+        dc.slices.clear();
+        dc.slices.offer(ByteBuffer.allocate(8190));
+        NIODataInputStream is = new NIODataInputStream(dc, 4096);
+        assertEquals(0, is.available());
+        is.read();
+        assertEquals(4095, is.available());
+        is.read(new byte[4095]);
+        assertEquals(0, is.available());
+        is.read(new byte[10]);
+        assertEquals(8190 - 10 - 4096, is.available());
+
+        File f = File.createTempFile("foo", "bar");
+        RandomAccessFile fos = new RandomAccessFile(f, "rw");
+        fos.write(new byte[10]);
+        fos.seek(0);
+
+        is = new NIODataInputStream(fos.getChannel(), 8);
+
+        int remaining = 10;
+        assertEquals(10, is.available());
+
+        while (remaining > 0)
+        {
+            is.read();
+            remaining--;
+            assertEquals(remaining, is.available());
+        }
+        assertEquals(0, is.available());
+    }
+
+    @SuppressWarnings("resource")
+    @Test
+    public void testReadUTF() throws Exception
+    {
+        final ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        DataOutputStream daos = new DataOutputStream(baos);
+
+        String simple = "foobar42";
+
+        assertEquals(2, BufferedDataOutputStreamTest.twoByte.getBytes(Charsets.UTF_8).length);
+        assertEquals(3, BufferedDataOutputStreamTest.threeByte.getBytes(Charsets.UTF_8).length);
+        assertEquals(4, BufferedDataOutputStreamTest.fourByte.getBytes(Charsets.UTF_8).length);
+
+        daos.writeUTF(simple);
+        daos.writeUTF(BufferedDataOutputStreamTest.twoByte);
+        daos.writeUTF(BufferedDataOutputStreamTest.threeByte);
+        daos.writeUTF(BufferedDataOutputStreamTest.fourByte);
+
+        NIODataInputStream is = new NIODataInputStream(new ReadableByteChannel()
+        {
+
+            @Override
+            public boolean isOpen() {return false;}
+
+            @Override
+            public void close() throws IOException {}
+
+            @Override
+            public int read(ByteBuffer dst) throws IOException
+            {
+                dst.put(baos.toByteArray());
+                return baos.toByteArray().length;
+            }
+
+        }, 4096);
+
+        assertEquals(simple, is.readUTF());
+        assertEquals(BufferedDataOutputStreamTest.twoByte, is.readUTF());
+        assertEquals(BufferedDataOutputStreamTest.threeByte, is.readUTF());
+        assertEquals(BufferedDataOutputStreamTest.fourByte, is.readUTF());
+    }
+
+    @Test
+    public void testFuzz() throws Exception
+    {
+        for (int ii = 0; ii < 80; ii++)
+            fuzzOnce();
+    }
+
+    void validateAgainstCorpus(byte bytes[], int offset, int length, int position) throws Exception
+    {
+        assertEquals(corpus.position(), position);
+        int startPosition = corpus.position();
+        for (int ii = 0; ii < length; ii++)
+        {
+            byte expected = corpus.get();
+            byte actual = bytes[ii + offset];
+            if (expected != actual)
+                fail("Mismatch compared to ByteBuffer");
+            byte canonical = dis.readByte();
+            if (canonical != actual)
+                fail("Mismatch compared to DataInputStream");
+        }
+        assertEquals(length, corpus.position() - startPosition);
+    }
+
+    DataInputStream dis;
+
+    @SuppressWarnings({ "resource", "unused" })
+    void fuzzOnce() throws Exception
+    {
+        init();
+        int read = 0;
+        int totalRead = 0;
+
+        DummyChannel dc = new DummyChannel();
+        NIODataInputStream is = new NIODataInputStream( dc, 1024 * 4);
+        dis = new DataInputStream(new ByteArrayInputStream(corpus.array()));
+
+        int iteration = 0;
+        while (totalRead < corpus.capacity())
+        {
+            assertEquals(corpus.position(), totalRead);
+            int action = r.nextInt(16);
+
+//            System.out.println("Action " + action + " iteration " + iteration + " remaining " + corpus.remaining());
+//            if (iteration == 434756) {
+//                System.out.println("Here we go");
+//            }
+            iteration++;
+
+            switch (action) {
+            case 0:
+            {
+                byte bytes[] = new byte[111];
+
+                int expectedBytes = corpus.capacity() - totalRead;
+                boolean expectEOF = expectedBytes < 111;
+                boolean threwEOF = false;
+                try
+                {
+                    is.readFully(bytes);
+                }
+                catch (EOFException e)
+                {
+                    threwEOF = true;
+                }
+
+                assertEquals(expectEOF, threwEOF);
+
+                if (expectEOF)
+                    return;
+
+                validateAgainstCorpus(bytes, 0, 111, totalRead);
+
+                totalRead += 111;
+                break;
+            }
+            case 1:
+            {
+                byte bytes[] = new byte[r.nextInt(1024 * 8 + 1)];
+
+                int offset = bytes.length == 0 ? 0 : r.nextInt(bytes.length);
+                int length = bytes.length == 0 ? 0 : r.nextInt(bytes.length - offset);
+                int expectedBytes = corpus.capacity() - totalRead;
+                boolean expectEOF = expectedBytes < length;
+                boolean threwEOF = false;
+                try {
+                    is.readFully(bytes, offset, length);
+                }
+                catch (EOFException e)
+                {
+                    threwEOF = true;
+                }
+
+                assertEquals(expectEOF, threwEOF);
+
+                if (expectEOF)
+                    return;
+
+                validateAgainstCorpus(bytes, offset, length, totalRead);
+
+                totalRead += length;
+                break;
+            }
+            case 2:
+            {
+                byte bytes[] = new byte[r.nextInt(1024 * 8 + 1)];
+
+                int offset = bytes.length == 0 ? 0 : r.nextInt(bytes.length);
+                int length = bytes.length == 0 ? 0 : r.nextInt(bytes.length - offset);
+                int expectedBytes = corpus.capacity() - totalRead;
+                boolean expectEOF = expectedBytes == 0;
+                read = is.read(bytes, offset, length);
+
+                assertTrue((expectEOF && read <= 0) || (!expectEOF && read >= 0));
+
+                if (expectEOF)
+                    return;
+
+                validateAgainstCorpus(bytes, offset, read, totalRead);
+
+                totalRead += read;
+                break;
+            }
+            case 3:
+            {
+                byte bytes[] = new byte[111];
+
+                int expectedBytes = corpus.capacity() - totalRead;
+                boolean expectEOF = expectedBytes == 0;
+                read = is.read(bytes);
+
+                assertTrue((expectEOF && read <= 0) || (!expectEOF && read >= 0));
+
+                if (expectEOF)
+                    return;
+
+                validateAgainstCorpus(bytes, 0, read, totalRead);
+
+                totalRead += read;
+                break;
+            }
+            case 4:
+            {
+                boolean expected = corpus.get() != 0;
+                boolean canonical = dis.readBoolean();
+                boolean actual = is.readBoolean();
+                assertTrue(expected == canonical && canonical == actual);
+                totalRead++;
+                break;
+            }
+            case 5:
+            {
+                byte expected = corpus.get();
+                byte canonical = dis.readByte();
+                byte actual = is.readByte();
+                assertTrue(expected == canonical && canonical == actual);
+                totalRead++;
+                break;
+            }
+            case 6:
+            {
+                int expected = corpus.get() & 0xFF;
+                int canonical = dis.read();
+                int actual = is.read();
+                assertTrue(expected == canonical && canonical == actual);
+                totalRead++;
+                break;
+            }
+            case 7:
+            {
+                int expected = corpus.get() & 0xFF;
+                int canonical = dis.readUnsignedByte();
+                int actual = is.readUnsignedByte();
+                assertTrue(expected == canonical && canonical == actual);
+                totalRead++;
+                break;
+            }
+            case 8:
+            {
+                if (corpus.remaining() < 2)
+                {
+                    boolean threw = false;
+                    try
+                    {
+                        is.readShort();
+                    }
+                    catch (EOFException e)
+                    {
+                        try { dis.readShort(); } catch (EOFException e2) {}
+                        threw = true;
+                    }
+                    assertTrue(threw);
+                    assertTrue(corpus.remaining() - totalRead < 2);
+                    totalRead = corpus.capacity();
+                    break;
+                }
+                short expected = corpus.getShort();
+                short canonical = dis.readShort();
+                short actual = is.readShort();
+                assertTrue(expected == canonical && canonical == actual);
+                totalRead += 2;
+                break;
+            }
+            case 9:
+            {
+                if (corpus.remaining() < 2)
+                {
+                    boolean threw = false;
+                    try
+                    {
+                        is.readUnsignedShort();
+                    }
+                    catch (EOFException e)
+                    {
+                        try { dis.readUnsignedShort(); } catch (EOFException e2) {}
+                        threw = true;
+                    }
+                    assertTrue(threw);
+                    assertTrue(corpus.remaining() - totalRead < 2);
+                    totalRead = corpus.capacity();
+                    break;
+                }
+                int ch1 = corpus.get() & 0xFF;
+                int ch2 = corpus.get() & 0xFF;
+                int expected = (ch1 << 8) + (ch2 << 0);
+                int canonical = dis.readUnsignedShort();
+                int actual = is.readUnsignedShort();
+                assertTrue(expected == canonical && canonical == actual);
+                totalRead += 2;
+                break;
+            }
+            case 10:
+            {
+                if (corpus.remaining() < 2)
+                {
+                    boolean threw = false;
+                    try
+                    {
+                        is.readChar();
+                    }
+                    catch (EOFException e)
+                    {
+                        try { dis.readChar(); } catch (EOFException e2) {}
+                        threw = true;
+                    }
+                    assertTrue(threw);
+                    assertTrue(corpus.remaining() - totalRead < 2);
+                    totalRead = corpus.capacity();
+                    break;
+                }
+                char expected = corpus.getChar();
+                char canonical = dis.readChar();
+                char actual = is.readChar();
+                assertTrue(expected == canonical && canonical == actual);
+                totalRead += 2;
+                break;
+            }
+            case 11:
+            {
+                if (corpus.remaining() < 4)
+                {
+                    boolean threw = false;
+                    try
+                    {
+                        is.readInt();
+                    }
+                    catch (EOFException e)
+                    {
+                        try { dis.readInt(); } catch (EOFException e2) {}
+                        threw = true;
+                    }
+                    assertTrue(threw);
+                    assertTrue(corpus.remaining() - totalRead < 4);
+                    totalRead = corpus.capacity();
+                    break;
+                }
+                int expected = corpus.getInt();
+                int canonical = dis.readInt();
+                int actual = is.readInt();
+                assertTrue(expected == canonical && canonical == actual);
+                totalRead += 4;
+                break;
+            }
+            case 12:
+            {
+                if (corpus.remaining() < 4)
+                {
+                    boolean threw = false;
+                    try
+                    {
+                        is.readFloat();
+                    }
+                    catch (EOFException e)
+                    {
+                        try { dis.readFloat(); } catch (EOFException e2) {}
+                        threw = true;
+                    }
+                    assertTrue(threw);
+                    assertTrue(corpus.remaining() - totalRead < 4);
+                    totalRead = corpus.capacity();
+                    break;
+                }
+                float expected = corpus.getFloat();
+                float canonical = dis.readFloat();
+                float actual = is.readFloat();
+                totalRead += 4;
+
+                if (Float.isNaN(expected)) {
+                    assertTrue(Float.isNaN(canonical) && Float.isNaN(actual));
+                } else {
+                    assertTrue(expected == canonical && canonical == actual);
+                }
+                break;
+            }
+            case 13:
+            {
+                if (corpus.remaining() < 8)
+                {
+                    boolean threw = false;
+                    try
+                    {
+                        is.readLong();
+                    }
+                    catch (EOFException e)
+                    {
+                        try { dis.readLong(); } catch (EOFException e2) {}
+                        threw = true;
+                    }
+                    assertTrue(threw);
+                    assertTrue(corpus.remaining() - totalRead < 8);
+                    totalRead = corpus.capacity();
+                    break;
+                }
+                long expected = corpus.getLong();
+                long canonical = dis.readLong();
+                long actual = is.readLong();
+
+                assertTrue(expected == canonical && canonical == actual);
+                totalRead += 8;
+                break;
+            }
+            case 14:
+            {
+                if (corpus.remaining() < 8)
+                {
+                    boolean threw = false;
+                    try
+                    {
+                        is.readDouble();
+                    }
+                    catch (EOFException e)
+                    {
+                        try { dis.readDouble(); } catch (EOFException e2) {}
+                        threw = true;
+                    }
+                    assertTrue(threw);
+                    assertTrue(corpus.remaining() - totalRead < 8);
+                    totalRead = corpus.capacity();
+                    break;
+                }
+                double expected = corpus.getDouble();
+                double canonical = dis.readDouble();
+                double actual = is.readDouble();
+                totalRead += 8;
+
+                if (Double.isNaN(expected)) {
+                    assertTrue(Double.isNaN(canonical) && Double.isNaN(actual));
+                } else {
+                    assertTrue(expected == canonical && canonical == actual);
+                }
+                break;
+            }
+            case 15:
+            {
+                int skipBytes = r.nextInt(1024);
+                int actuallySkipped =  Math.min(skipBytes, corpus.remaining());
+
+                totalRead += actuallySkipped;
+                corpus.position(corpus.position() + actuallySkipped);
+                int canonical = dis.skipBytes(actuallySkipped);
+                int actual = is.skipBytes(actuallySkipped);
+                assertEquals(actuallySkipped, canonical);
+                assertEquals(canonical, actual);
+                break;
+            }
+            default:
+                fail("Should never reach here");
+            }
+        }
+
+        assertEquals(totalRead, corpus.capacity());
+        assertEquals(-1, dis.read());
+    }
+}
diff --git a/test/unit/org/apache/cassandra/io/util/SequentialWriterTest.java b/test/unit/org/apache/cassandra/io/util/SequentialWriterTest.java
new file mode 100644
index 0000000..fd38427
--- /dev/null
+++ b/test/unit/org/apache/cassandra/io/util/SequentialWriterTest.java
@@ -0,0 +1,143 @@
+/*
+* 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.io.util;
+
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.ThreadLocalRandom;
+
+import com.google.common.io.Files;
+import org.junit.After;
+import org.junit.Test;
+
+import junit.framework.Assert;
+
+import org.apache.cassandra.io.compress.BufferType;
+import org.apache.cassandra.utils.concurrent.AbstractTransactionalTest;
+
+import static org.apache.commons.io.FileUtils.*;
+
+public class SequentialWriterTest extends AbstractTransactionalTest
+{
+
+    private final List<TestableSW> writers = new ArrayList<>();
+
+    @After
+    public void cleanup()
+    {
+        for (TestableSW sw : writers)
+            sw.file.delete();
+        writers.clear();
+    }
+
+    protected TestableTransaction newTest() throws IOException
+    {
+        TestableSW sw = new TestableSW();
+        writers.add(sw);
+        return sw;
+    }
+
+    protected static class TestableSW extends TestableTransaction
+    {
+        protected static final int BUFFER_SIZE = 8 << 10;
+        protected final File file;
+        protected final SequentialWriter writer;
+        protected final byte[] fullContents, partialContents;
+
+        protected TestableSW() throws IOException
+        {
+            this(tempFile("sequentialwriter"));
+        }
+
+        protected TestableSW(File file) throws IOException
+        {
+            this(file, new SequentialWriter(file, 8 << 10, BufferType.OFF_HEAP));
+        }
+
+        protected TestableSW(File file, SequentialWriter sw) throws IOException
+        {
+            super(sw);
+            this.file = file;
+            this.writer = sw;
+            fullContents = new byte[BUFFER_SIZE + BUFFER_SIZE / 2];
+            ThreadLocalRandom.current().nextBytes(fullContents);
+            partialContents = Arrays.copyOf(fullContents, BUFFER_SIZE);
+            sw.write(fullContents);
+        }
+
+        protected void assertInProgress() throws Exception
+        {
+            Assert.assertTrue(file.exists());
+            byte[] bytes = readFileToByteArray(file);
+            Assert.assertTrue(Arrays.equals(partialContents, bytes));
+        }
+
+        protected void assertPrepared() throws Exception
+        {
+            Assert.assertTrue(file.exists());
+            byte[] bytes = readFileToByteArray(file);
+            Assert.assertTrue(Arrays.equals(fullContents, bytes));
+        }
+
+        protected void assertAborted() throws Exception
+        {
+            Assert.assertFalse(writer.isOpen());
+            Assert.assertFalse(file.exists());
+        }
+
+        protected void assertCommitted() throws Exception
+        {
+            assertPrepared();
+            Assert.assertFalse(writer.isOpen());
+        }
+
+        protected static File tempFile(String prefix)
+        {
+            File file = FileUtils.createTempFile(prefix, "test");
+            file.delete();
+            return file;
+        }
+    }
+
+    /**
+     * Tests that the output stream exposed by SequentialWriter behaves as expected
+     */
+    @Test
+    public void outputStream()
+    {
+        File tempFile = new File(Files.createTempDir(), "test.txt");
+        Assert.assertFalse("temp file shouldn't exist yet", tempFile.exists());
+
+        try (DataOutputStream os = new DataOutputStream(SequentialWriter.open(tempFile).finishOnClose()))
+        {
+            os.writeUTF("123");
+        }
+        catch (IOException e)
+        {
+            Assert.fail();
+        }
+
+        Assert.assertTrue("temp file should exist", tempFile.exists());
+    }
+
+}
diff --git a/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java b/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
index 90e63e0..bc5be46 100644
--- a/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
+++ b/test/unit/org/apache/cassandra/locator/CloudstackSnitchTest.java
@@ -27,6 +27,7 @@
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.gms.ApplicationState;
 import org.apache.cassandra.gms.Gossiper;
@@ -43,6 +44,7 @@
     @BeforeClass
     public static void setup() throws Exception
     {
+        DatabaseDescriptor.setDaemonInitialized();
         SchemaLoader.mkdirs();
         SchemaLoader.cleanup();
         Keyspace.setInitialized();
diff --git a/test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java b/test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java
index c1928d8..d27edbf 100644
--- a/test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java
+++ b/test/unit/org/apache/cassandra/locator/DynamicEndpointSnitchTest.java
@@ -33,7 +33,6 @@
 
 public class DynamicEndpointSnitchTest
 {
-
     private static void setScores(DynamicEndpointSnitch dsnitch,  int rounds, List<InetAddress> hosts, Integer... scores) throws InterruptedException
     {
         for (int round = 0; round < rounds; round++)
@@ -48,13 +47,14 @@
     public void testSnitch() throws InterruptedException, IOException, ConfigurationException
     {
         // do this because SS needs to be initialized before DES can work properly.
-        StorageService.instance.initClient(0);
+        StorageService.instance.unsafeInitialize();
         SimpleSnitch ss = new SimpleSnitch();
         DynamicEndpointSnitch dsnitch = new DynamicEndpointSnitch(ss, String.valueOf(ss.hashCode()));
         InetAddress self = FBUtilities.getBroadcastAddress();
         InetAddress host1 = InetAddress.getByName("127.0.0.2");
         InetAddress host2 = InetAddress.getByName("127.0.0.3");
         InetAddress host3 = InetAddress.getByName("127.0.0.4");
+        InetAddress host4 = InetAddress.getByName("127.0.0.5");
         List<InetAddress> hosts = Arrays.asList(host1, host2, host3);
 
         // first, make all hosts equal
@@ -88,5 +88,12 @@
         setScores(dsnitch, 20, hosts, 10, 70, 20);
         order = Arrays.asList(host1, host3, host2);
         assertEquals(order, dsnitch.getSortedListByProximity(self, Arrays.asList(host1, host2, host3)));
+
+        order = Arrays.asList(host4, host1, host3, host2);
+        assertEquals(order, dsnitch.getSortedListByProximity(self, Arrays.asList(host1, host2, host3, host4)));
+
+        setScores(dsnitch, 20, hosts, 10, 10, 10);
+        order = Arrays.asList(host1, host2, host3, host4);
+        assertEquals(order, dsnitch.getSortedListByProximity(self, Arrays.asList(host1, host2, host3, host4)));
     }
 }
diff --git a/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java b/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java
index 56bbb77..32383d9 100644
--- a/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java
+++ b/test/unit/org/apache/cassandra/locator/EC2SnitchTest.java
@@ -33,6 +33,7 @@
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.gms.ApplicationState;
@@ -51,6 +52,7 @@
     @BeforeClass
     public static void setup() throws Exception
     {
+        DatabaseDescriptor.setDaemonInitialized();
         SchemaLoader.mkdirs();
         SchemaLoader.cleanup();
         Keyspace.setInitialized();
diff --git a/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java b/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java
index 1521454..f2450f4 100644
--- a/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java
+++ b/test/unit/org/apache/cassandra/locator/GoogleCloudSnitchTest.java
@@ -31,6 +31,7 @@
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.gms.ApplicationState;
@@ -47,6 +48,7 @@
     @BeforeClass
     public static void setup() throws Exception
     {
+        DatabaseDescriptor.setDaemonInitialized();
         SchemaLoader.mkdirs();
         SchemaLoader.cleanup();
         Keyspace.setInitialized();
diff --git a/test/unit/org/apache/cassandra/locator/GossipingPropertyFileSnitchTest.java b/test/unit/org/apache/cassandra/locator/GossipingPropertyFileSnitchTest.java
index 80d4559..20608ba 100644
--- a/test/unit/org/apache/cassandra/locator/GossipingPropertyFileSnitchTest.java
+++ b/test/unit/org/apache/cassandra/locator/GossipingPropertyFileSnitchTest.java
@@ -17,22 +17,33 @@
  */
 package org.apache.cassandra.locator;
 
+import java.net.InetAddress;
+
+import com.google.common.net.InetAddresses;
 import org.junit.Test;
 
 import org.apache.cassandra.utils.FBUtilities;
 
+import static org.junit.Assert.*;
+
 /**
  * Unit tests for {@link GossipingPropertyFileSnitch}.
  */
 public class GossipingPropertyFileSnitchTest
 {
+    public static void checkEndpoint(final AbstractNetworkTopologySnitch snitch,
+                                     final String endpointString, final String expectedDatacenter,
+                                     final String expectedRack)
+    {
+        final InetAddress endpoint = InetAddresses.forString(endpointString);
+        assertEquals(expectedDatacenter, snitch.getDatacenter(endpoint));
+        assertEquals(expectedRack, snitch.getRack(endpoint));
+    }
+
     @Test
     public void testLoadConfig() throws Exception
     {
         final GossipingPropertyFileSnitch snitch = new GossipingPropertyFileSnitch();
-        YamlFileNetworkTopologySnitchTest.checkEndpoint(snitch,
-                                                        FBUtilities.getBroadcastAddress().getHostAddress(),
-                                                        "DC1",
-                                                        "RAC1");
+        checkEndpoint(snitch, FBUtilities.getBroadcastAddress().getHostAddress(), "DC1", "RAC1");
     }
 }
diff --git a/test/unit/org/apache/cassandra/locator/NetworkTopologyStrategyTest.java b/test/unit/org/apache/cassandra/locator/NetworkTopologyStrategyTest.java
index d4d5a85..a3ac416 100644
--- a/test/unit/org/apache/cassandra/locator/NetworkTopologyStrategyTest.java
+++ b/test/unit/org/apache/cassandra/locator/NetworkTopologyStrategyTest.java
@@ -37,7 +37,7 @@
 
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.dht.StringToken;
+import org.apache.cassandra.dht.OrderPreservingPartitioner.StringToken;
 import org.apache.cassandra.dht.Token;
 
 import com.google.common.collect.HashMultimap;
@@ -121,7 +121,7 @@
                     byte[] ipBytes = new byte[]{10, (byte)dc, (byte)rack, (byte)ep};
                     InetAddress address = InetAddress.getByAddress(ipBytes);
                     StringToken token = new StringToken(String.format("%02x%02x%02x", ep, rack, dc));
-                    logger.debug("adding node " + address + " at " + token);
+                    logger.debug("adding node {} at {}", address, token);
                     tokens.put(address, token);
                 }
             }
@@ -137,7 +137,7 @@
 
             Assert.assertEquals(totalRF, endpoints.size());
             Assert.assertEquals(totalRF, epSet.size());
-            logger.debug(testToken + ": " + endpoints.toString());
+            logger.debug("{}: {}", testToken, endpoints);
         }
     }
 
diff --git a/test/unit/org/apache/cassandra/locator/OldNetworkTopologyStrategyTest.java b/test/unit/org/apache/cassandra/locator/OldNetworkTopologyStrategyTest.java
index 14cb54d..eceb847 100644
--- a/test/unit/org/apache/cassandra/locator/OldNetworkTopologyStrategyTest.java
+++ b/test/unit/org/apache/cassandra/locator/OldNetworkTopologyStrategyTest.java
@@ -31,19 +31,18 @@
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.KSMetaData;
-import org.apache.cassandra.dht.BigIntegerToken;
+import org.apache.cassandra.dht.RandomPartitioner.BigIntegerToken;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.Pair;
+
 import org.junit.Before;
 import org.junit.Test;
 
-public class OldNetworkTopologyStrategyTest extends SchemaLoader
+public class OldNetworkTopologyStrategyTest
 {
-    private List<Token> endpointTokens;
     private List<Token> keyTokens;
     private TokenMetadata tmd;
     private Map<String, ArrayList<InetAddress>> expectedResults;
@@ -51,7 +50,6 @@
     @Before
     public void init()
     {
-        endpointTokens = new ArrayList<Token>();
         keyTokens = new ArrayList<Token>();
         tmd = new TokenMetadata();
         expectedResults = new HashMap<String, ArrayList<InetAddress>>();
@@ -145,7 +143,6 @@
     private void addEndpoint(String endpointTokenID, String keyTokenID, String endpointAddress) throws UnknownHostException
     {
         BigIntegerToken endpointToken = new BigIntegerToken(endpointTokenID);
-        endpointTokens.add(endpointToken);
 
         BigIntegerToken keyToken = new BigIntegerToken(keyTokenID);
         keyTokens.add(keyToken);
@@ -206,6 +203,7 @@
 
     }
 
+    @SuppressWarnings("unchecked")
     @Test
     public void testMoveMiddleOfRing() throws UnknownHostException
     {
@@ -219,25 +217,26 @@
         Pair<Set<Range<Token>>, Set<Range<Token>>> ranges = calculateStreamAndFetchRanges(tokens, tokensAfterMove, movingNodeIdx);
 
         // sort the results, so they can be compared
-        Range[] toStream = ranges.left.toArray(new Range[0]);
-        Range[] toFetch = ranges.right.toArray(new Range[0]);
+        Range<Token>[] toStream = ranges.left.toArray(new Range[0]);
+        Range<Token>[] toFetch = ranges.right.toArray(new Range[0]);
         Arrays.sort(toStream);
         Arrays.sort(toFetch);
 
         // build expected ranges
-        Range[] toStreamExpected = new Range[2];
-        toStreamExpected[0] = new Range(getToken(movingNodeIdx - 2, tokens), getToken(movingNodeIdx - 1, tokens));
-        toStreamExpected[1] = new Range(getToken(movingNodeIdx - 1, tokens), getToken(movingNodeIdx, tokens));
+        Range<Token>[] toStreamExpected = new Range[2];
+        toStreamExpected[0] = new Range<Token>(getToken(movingNodeIdx - 2, tokens), getToken(movingNodeIdx - 1, tokens));
+        toStreamExpected[1] = new Range<Token>(getToken(movingNodeIdx - 1, tokens), getToken(movingNodeIdx, tokens));
         Arrays.sort(toStreamExpected);
-        Range[] toFetchExpected = new Range[2];
-        toFetchExpected[0] = new Range(getToken(movingNodeIdxAfterMove - 1, tokens), getToken(movingNodeIdxAfterMove, tokens));
-        toFetchExpected[1] = new Range(getToken(movingNodeIdxAfterMove, tokensAfterMove), getToken(movingNodeIdx, tokensAfterMove));
+        Range<Token>[] toFetchExpected = new Range[2];
+        toFetchExpected[0] = new Range<Token>(getToken(movingNodeIdxAfterMove - 1, tokens), getToken(movingNodeIdxAfterMove, tokens));
+        toFetchExpected[1] = new Range<Token>(getToken(movingNodeIdxAfterMove, tokensAfterMove), getToken(movingNodeIdx, tokensAfterMove));
         Arrays.sort(toFetchExpected);
 
         assertEquals(Arrays.equals(toStream, toStreamExpected), true);
         assertEquals(Arrays.equals(toFetch, toFetchExpected), true);
     }
 
+    @SuppressWarnings("unchecked")
     @Test
     public void testMoveAfterNextNeighbors() throws UnknownHostException
     {
@@ -252,24 +251,25 @@
 
 
         // sort the results, so they can be compared
-        Range[] toStream = ranges.left.toArray(new Range[0]);
-        Range[] toFetch = ranges.right.toArray(new Range[0]);
+        Range<Token>[] toStream = ranges.left.toArray(new Range[0]);
+        Range<Token>[] toFetch = ranges.right.toArray(new Range[0]);
         Arrays.sort(toStream);
         Arrays.sort(toFetch);
 
         // build expected ranges
-        Range[] toStreamExpected = new Range[1];
-        toStreamExpected[0] = new Range(getToken(movingNodeIdx - 2, tokens), getToken(movingNodeIdx - 1, tokens));
+        Range<Token>[] toStreamExpected = new Range[1];
+        toStreamExpected[0] = new Range<Token>(getToken(movingNodeIdx - 2, tokens), getToken(movingNodeIdx - 1, tokens));
         Arrays.sort(toStreamExpected);
-        Range[] toFetchExpected = new Range[2];
-        toFetchExpected[0] = new Range(getToken(movingNodeIdxAfterMove - 1, tokens), getToken(movingNodeIdxAfterMove, tokens));
-        toFetchExpected[1] = new Range(getToken(movingNodeIdxAfterMove, tokensAfterMove), getToken(movingNodeIdx, tokensAfterMove));
+        Range<Token>[] toFetchExpected = new Range[2];
+        toFetchExpected[0] = new Range<Token>(getToken(movingNodeIdxAfterMove - 1, tokens), getToken(movingNodeIdxAfterMove, tokens));
+        toFetchExpected[1] = new Range<Token>(getToken(movingNodeIdxAfterMove, tokensAfterMove), getToken(movingNodeIdx, tokensAfterMove));
         Arrays.sort(toFetchExpected);
 
         assertEquals(Arrays.equals(toStream, toStreamExpected), true);
         assertEquals(Arrays.equals(toFetch, toFetchExpected), true);
     }
 
+    @SuppressWarnings("unchecked")
     @Test
     public void testMoveBeforePreviousNeighbor() throws UnknownHostException
     {
@@ -282,17 +282,17 @@
         BigIntegerToken[] tokensAfterMove = initTokensAfterMove(tokens, movingNodeIdx, newToken);
         Pair<Set<Range<Token>>, Set<Range<Token>>> ranges = calculateStreamAndFetchRanges(tokens, tokensAfterMove, movingNodeIdx);
 
-        Range[] toStream = ranges.left.toArray(new Range[0]);
-        Range[] toFetch = ranges.right.toArray(new Range[0]);
+        Range<Token>[] toStream = ranges.left.toArray(new Range[0]);
+        Range<Token>[] toFetch = ranges.right.toArray(new Range[0]);
         Arrays.sort(toStream);
         Arrays.sort(toFetch);
 
-        Range[] toStreamExpected = new Range[2];
-        toStreamExpected[0] = new Range(getToken(movingNodeIdx, tokensAfterMove), getToken(movingNodeIdx - 1, tokensAfterMove));
-        toStreamExpected[1] = new Range(getToken(movingNodeIdx - 1, tokens), getToken(movingNodeIdx, tokens));
+        Range<Token>[] toStreamExpected = new Range[2];
+        toStreamExpected[0] = new Range<Token>(getToken(movingNodeIdx, tokensAfterMove), getToken(movingNodeIdx - 1, tokensAfterMove));
+        toStreamExpected[1] = new Range<Token>(getToken(movingNodeIdx - 1, tokens), getToken(movingNodeIdx, tokens));
         Arrays.sort(toStreamExpected);
-        Range[] toFetchExpected = new Range[1];
-        toFetchExpected[0] = new Range(getToken(movingNodeIdxAfterMove - 1, tokens), getToken(movingNodeIdxAfterMove, tokens));
+        Range<Token>[] toFetchExpected = new Range[1];
+        toFetchExpected[0] = new Range<Token>(getToken(movingNodeIdxAfterMove - 1, tokens), getToken(movingNodeIdxAfterMove, tokens));
         Arrays.sort(toFetchExpected);
 
         System.out.println("toStream : " + Arrays.toString(toStream));
diff --git a/test/unit/org/apache/cassandra/locator/PendingRangeMapsTest.java b/test/unit/org/apache/cassandra/locator/PendingRangeMapsTest.java
new file mode 100644
index 0000000..7121550
--- /dev/null
+++ b/test/unit/org/apache/cassandra/locator/PendingRangeMapsTest.java
@@ -0,0 +1,98 @@
+/*
+ *
+ * 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.locator;
+
+import org.apache.cassandra.dht.RandomPartitioner.BigIntegerToken;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.junit.Test;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Collection;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class PendingRangeMapsTest {
+
+    private Range<Token> genRange(String left, String right)
+    {
+        return new Range<Token>(new BigIntegerToken(left), new BigIntegerToken(right));
+    }
+
+    @Test
+    public void testPendingEndpoints() throws UnknownHostException
+    {
+        PendingRangeMaps pendingRangeMaps = new PendingRangeMaps();
+
+        pendingRangeMaps.addPendingRange(genRange("5", "15"), InetAddress.getByName("127.0.0.1"));
+        pendingRangeMaps.addPendingRange(genRange("15", "25"), InetAddress.getByName("127.0.0.2"));
+        pendingRangeMaps.addPendingRange(genRange("25", "35"), InetAddress.getByName("127.0.0.3"));
+        pendingRangeMaps.addPendingRange(genRange("35", "45"), InetAddress.getByName("127.0.0.4"));
+        pendingRangeMaps.addPendingRange(genRange("45", "55"), InetAddress.getByName("127.0.0.5"));
+        pendingRangeMaps.addPendingRange(genRange("45", "65"), InetAddress.getByName("127.0.0.6"));
+
+        assertEquals(0, pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("0")).size());
+        assertEquals(0, pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("5")).size());
+        assertEquals(1, pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("10")).size());
+        assertEquals(1, pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("15")).size());
+        assertEquals(1, pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("20")).size());
+        assertEquals(1, pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("25")).size());
+        assertEquals(1, pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("35")).size());
+        assertEquals(1, pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("45")).size());
+        assertEquals(2, pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("55")).size());
+        assertEquals(1, pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("65")).size());
+
+        Collection<InetAddress> endpoints = pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("15"));
+        assertTrue(endpoints.contains(InetAddress.getByName("127.0.0.1")));
+    }
+
+    @Test
+    public void testWrapAroundRanges() throws UnknownHostException
+    {
+        PendingRangeMaps pendingRangeMaps = new PendingRangeMaps();
+
+        pendingRangeMaps.addPendingRange(genRange("5", "15"), InetAddress.getByName("127.0.0.1"));
+        pendingRangeMaps.addPendingRange(genRange("15", "25"), InetAddress.getByName("127.0.0.2"));
+        pendingRangeMaps.addPendingRange(genRange("25", "35"), InetAddress.getByName("127.0.0.3"));
+        pendingRangeMaps.addPendingRange(genRange("35", "45"), InetAddress.getByName("127.0.0.4"));
+        pendingRangeMaps.addPendingRange(genRange("45", "55"), InetAddress.getByName("127.0.0.5"));
+        pendingRangeMaps.addPendingRange(genRange("45", "65"), InetAddress.getByName("127.0.0.6"));
+        pendingRangeMaps.addPendingRange(genRange("65", "7"), InetAddress.getByName("127.0.0.7"));
+
+        assertEquals(1, pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("0")).size());
+        assertEquals(1, pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("5")).size());
+        assertEquals(2, pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("7")).size());
+        assertEquals(1, pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("10")).size());
+        assertEquals(1, pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("15")).size());
+        assertEquals(1, pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("20")).size());
+        assertEquals(1, pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("25")).size());
+        assertEquals(1, pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("35")).size());
+        assertEquals(1, pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("45")).size());
+        assertEquals(2, pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("55")).size());
+        assertEquals(1, pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("65")).size());
+
+        Collection<InetAddress> endpoints = pendingRangeMaps.pendingEndpointsFor(new BigIntegerToken("6"));
+        assertTrue(endpoints.contains(InetAddress.getByName("127.0.0.1")));
+        assertTrue(endpoints.contains(InetAddress.getByName("127.0.0.7")));
+    }
+}
diff --git a/test/unit/org/apache/cassandra/locator/PropertyFileSnitchTest.java b/test/unit/org/apache/cassandra/locator/PropertyFileSnitchTest.java
index 24b8c77..e9a307b 100644
--- a/test/unit/org/apache/cassandra/locator/PropertyFileSnitchTest.java
+++ b/test/unit/org/apache/cassandra/locator/PropertyFileSnitchTest.java
@@ -33,6 +33,8 @@
 import java.util.Set;
 import java.util.UUID;
 
+import com.google.common.net.InetAddresses;
+
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.RandomPartitioner;
 import org.apache.cassandra.dht.Token;
@@ -46,6 +48,8 @@
 import org.junit.Before;
 import org.junit.Test;
 
+import static org.junit.Assert.*;
+
 /**
  * Unit tests for {@link PropertyFileSnitch}.
  */
@@ -146,6 +150,15 @@
         StorageService.instance.getTokenMetadata().updateNormalTokens(tokenMap.get(host), host);
     }
 
+    private static void checkEndpoint(final AbstractNetworkTopologySnitch snitch,
+                                      final String endpointString, final String expectedDatacenter,
+                                      final String expectedRack)
+    {
+        final InetAddress endpoint = InetAddresses.forString(endpointString);
+        assertEquals(expectedDatacenter, snitch.getDatacenter(endpoint));
+        assertEquals(expectedRack, snitch.getRack(endpoint));
+    }
+
     /**
      * Test that changing rack for a host in the configuration file is only effective if the host is not live.
      * The original configuration file contains: 127.0.0.1=DC1:RAC1
@@ -155,7 +168,7 @@
     {
         final InetAddress host = InetAddress.getByName("127.0.0.1");
         final PropertyFileSnitch snitch = new PropertyFileSnitch(/*refreshPeriodInSeconds*/1);
-        YamlFileNetworkTopologySnitchTest.checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC1");
+        checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC1");
 
         try
         {
@@ -165,13 +178,13 @@
             replaceConfigFile(Collections.singletonMap(host.getHostAddress(), "DC1:RAC2"));
 
             Thread.sleep(1500);
-            YamlFileNetworkTopologySnitchTest.checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC1");
+            checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC1");
 
             setNodeShutdown(host);
             replaceConfigFile(Collections.singletonMap(host.getHostAddress(), "DC1:RAC2"));
 
             Thread.sleep(1500);
-            YamlFileNetworkTopologySnitchTest.checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC2");
+            checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC2");
         }
         finally
         {
@@ -189,7 +202,7 @@
     {
         final InetAddress host = InetAddress.getByName("127.0.0.1");
         final PropertyFileSnitch snitch = new PropertyFileSnitch(/*refreshPeriodInSeconds*/1);
-        YamlFileNetworkTopologySnitchTest.checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC1");
+        checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC1");
 
         try
         {
@@ -199,13 +212,13 @@
             replaceConfigFile(Collections.singletonMap(host.getHostAddress(), "DC2:RAC1"));
 
             Thread.sleep(1500);
-            YamlFileNetworkTopologySnitchTest.checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC1");
+            checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC1");
 
             setNodeShutdown(host);
             replaceConfigFile(Collections.singletonMap(host.getHostAddress(), "DC2:RAC1"));
 
             Thread.sleep(1500);
-            YamlFileNetworkTopologySnitchTest.checkEndpoint(snitch, host.getHostAddress(), "DC2", "RAC1");
+            checkEndpoint(snitch, host.getHostAddress(), "DC2", "RAC1");
         }
         finally
         {
@@ -224,7 +237,7 @@
     {
         final InetAddress host = InetAddress.getByName("127.0.0.9");
         final PropertyFileSnitch snitch = new PropertyFileSnitch(/*refreshPeriodInSeconds*/1);
-        YamlFileNetworkTopologySnitchTest.checkEndpoint(snitch, host.getHostAddress(), "DC1", "r1"); // default
+        checkEndpoint(snitch, host.getHostAddress(), "DC1", "r1"); // default
 
         try
         {
@@ -234,13 +247,13 @@
             replaceConfigFile(Collections.singletonMap(host.getHostAddress(), "DC2:RAC2")); // add this line if not yet there
 
             Thread.sleep(1500);
-            YamlFileNetworkTopologySnitchTest.checkEndpoint(snitch, host.getHostAddress(), "DC1", "r1"); // unchanged
+            checkEndpoint(snitch, host.getHostAddress(), "DC1", "r1"); // unchanged
 
             setNodeShutdown(host);
             replaceConfigFile(Collections.singletonMap(host.getHostAddress(), "DC2:RAC2")); // add this line if not yet there
 
             Thread.sleep(1500);
-            YamlFileNetworkTopologySnitchTest.checkEndpoint(snitch, host.getHostAddress(), "DC2", "RAC2"); // changed
+            checkEndpoint(snitch, host.getHostAddress(), "DC2", "RAC2"); // changed
         }
         finally
         {
@@ -259,7 +272,7 @@
     {
         final InetAddress host = InetAddress.getByName("127.0.0.2");
         final PropertyFileSnitch snitch = new PropertyFileSnitch(/*refreshPeriodInSeconds*/1);
-        YamlFileNetworkTopologySnitchTest.checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC2");
+        checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC2");
 
         try
         {
@@ -269,13 +282,13 @@
             replaceConfigFile(Collections.singletonMap(host.getHostAddress(), "")); // removes line if found
 
             Thread.sleep(1500);
-            YamlFileNetworkTopologySnitchTest.checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC2"); // unchanged
+            checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC2"); // unchanged
 
             setNodeShutdown(host);
             replaceConfigFile(Collections.singletonMap(host.getHostAddress(), "")); // removes line if found
 
             Thread.sleep(1500);
-            YamlFileNetworkTopologySnitchTest.checkEndpoint(snitch, host.getHostAddress(), "DC1", "r1"); // default
+            checkEndpoint(snitch, host.getHostAddress(), "DC1", "r1"); // default
         }
         finally
         {
@@ -294,7 +307,7 @@
     {
         final InetAddress host = InetAddress.getByName("127.0.0.9");
         final PropertyFileSnitch snitch = new PropertyFileSnitch(/*refreshPeriodInSeconds*/1);
-        YamlFileNetworkTopologySnitchTest.checkEndpoint(snitch, host.getHostAddress(), "DC1", "r1"); // default
+        checkEndpoint(snitch, host.getHostAddress(), "DC1", "r1"); // default
 
         try
         {
@@ -304,13 +317,13 @@
             replaceConfigFile(Collections.singletonMap("default", "DC2:r2")); // change default
 
             Thread.sleep(1500);
-            YamlFileNetworkTopologySnitchTest.checkEndpoint(snitch, host.getHostAddress(), "DC1", "r1"); // unchanged
+            checkEndpoint(snitch, host.getHostAddress(), "DC1", "r1"); // unchanged
 
             setNodeShutdown(host);
             replaceConfigFile(Collections.singletonMap("default", "DC2:r2")); // change default again (refresh file update)
 
             Thread.sleep(1500);
-            YamlFileNetworkTopologySnitchTest.checkEndpoint(snitch, host.getHostAddress(), "DC2", "r2"); // default updated
+            checkEndpoint(snitch, host.getHostAddress(), "DC2", "r2"); // default updated
         }
         finally
         {
diff --git a/test/unit/org/apache/cassandra/locator/ReplicationStrategyEndpointCacheTest.java b/test/unit/org/apache/cassandra/locator/ReplicationStrategyEndpointCacheTest.java
index 91f6111..093de9b 100644
--- a/test/unit/org/apache/cassandra/locator/ReplicationStrategyEndpointCacheTest.java
+++ b/test/unit/org/apache/cassandra/locator/ReplicationStrategyEndpointCacheTest.java
@@ -22,28 +22,39 @@
 import java.net.InetAddress;
 import java.util.*;
 
-import org.apache.cassandra.db.Keyspace;
-
 import org.apache.commons.lang3.StringUtils;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.dht.BigIntegerToken;
+import org.apache.cassandra.dht.RandomPartitioner.BigIntegerToken;
 import org.apache.cassandra.dht.Token;
 
-public class ReplicationStrategyEndpointCacheTest extends SchemaLoader
+public class ReplicationStrategyEndpointCacheTest
 {
     private TokenMetadata tmd;
     private Token searchToken;
     private AbstractReplicationStrategy strategy;
+    public static final String KEYSPACE = "ReplicationStrategyEndpointCacheTest";
+
+    @BeforeClass
+    public static void defineSchema() throws Exception
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(5));
+    }
 
     public void setup(Class stratClass, Map<String, String> strategyOptions) throws Exception
     {
         tmd = new TokenMetadata();
         searchToken = new BigIntegerToken(String.valueOf(15));
 
-        strategy = getStrategyWithNewTokenMetadata(Keyspace.open("Keyspace3").getReplicationStrategy(), tmd);
+        strategy = getStrategyWithNewTokenMetadata(Keyspace.open(KEYSPACE).getReplicationStrategy(), tmd);
 
         tmd.updateNormalToken(new BigIntegerToken(String.valueOf(10)), InetAddress.getByName("127.0.0.1"));
         tmd.updateNormalToken(new BigIntegerToken(String.valueOf(20)), InetAddress.getByName("127.0.0.2"));
diff --git a/test/unit/org/apache/cassandra/locator/SimpleStrategyTest.java b/test/unit/org/apache/cassandra/locator/SimpleStrategyTest.java
index d4978ae..61255f3 100644
--- a/test/unit/org/apache/cassandra/locator/SimpleStrategyTest.java
+++ b/test/unit/org/apache/cassandra/locator/SimpleStrategyTest.java
@@ -26,25 +26,39 @@
 import java.util.HashSet;
 import java.util.List;
 
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.service.PendingRangeCalculatorService;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.dht.*;
+import org.apache.cassandra.dht.RandomPartitioner.BigIntegerToken;
+import org.apache.cassandra.dht.OrderPreservingPartitioner.StringToken;
+import org.apache.cassandra.service.PendingRangeCalculatorService;
 import org.apache.cassandra.service.StorageServiceAccessor;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 import static org.junit.Assert.*;
 
-public class SimpleStrategyTest extends SchemaLoader
+public class SimpleStrategyTest
 {
+    public static final String KEYSPACE1 = "SimpleStrategyTest";
+
+    @BeforeClass
+    public static void defineSchema() throws Exception
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1));
+    }
+
     @Test
     public void tryValidKeyspace()
     {
-        assert Keyspace.open("Keyspace1").getReplicationStrategy() != null;
+        assert Keyspace.open(KEYSPACE1).getReplicationStrategy() != null;
     }
 
     @Test
@@ -62,7 +76,7 @@
     @Test
     public void testStringEndpoints() throws UnknownHostException
     {
-        IPartitioner partitioner = new OrderPreservingPartitioner();
+        IPartitioner partitioner = OrderPreservingPartitioner.instance;
 
         List<Token> endpointTokens = new ArrayList<Token>();
         List<Token> keyTokens = new ArrayList<Token>();
diff --git a/test/unit/org/apache/cassandra/locator/YamlFileNetworkTopologySnitchTest.java b/test/unit/org/apache/cassandra/locator/YamlFileNetworkTopologySnitchTest.java
deleted file mode 100644
index 9507f25..0000000
--- a/test/unit/org/apache/cassandra/locator/YamlFileNetworkTopologySnitchTest.java
+++ /dev/null
@@ -1,339 +0,0 @@
-/*
- * 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.locator;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-
-import org.apache.cassandra.dht.IPartitioner;
-import org.apache.cassandra.dht.RandomPartitioner;
-import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.gms.ApplicationState;
-import org.apache.cassandra.gms.Gossiper;
-import org.apache.cassandra.gms.VersionedValue;
-import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.utils.FBUtilities;
-
-import static org.apache.cassandra.locator.YamlFileNetworkTopologySnitch.*;
-
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import com.google.common.net.InetAddresses;
-
-/**
- * Unit tests for {@link YamlFileNetworkTopologySnitch}.
- */
-public class YamlFileNetworkTopologySnitchTest
-{
-    private String confFile;
-
-    private VersionedValue.VersionedValueFactory valueFactory;
-    private Map<InetAddress, Set<Token>> tokenMap;
-
-    @Before
-    public void setup() throws ConfigurationException, IOException
-    {
-        confFile = YamlFileNetworkTopologySnitch.DEFAULT_TOPOLOGY_CONFIG_FILENAME;
-
-        InetAddress[] hosts = {
-                              InetAddress.getByName("127.0.0.1"), // this exists in the config file
-                              InetAddress.getByName("127.0.0.2"), // this exists in the config file
-                              InetAddress.getByName("127.0.0.9"), // this does not exist in the config file
-        };
-
-        IPartitioner partitioner = new RandomPartitioner();
-        valueFactory = new VersionedValue.VersionedValueFactory(partitioner);
-        tokenMap = new HashMap<>();
-
-        for (InetAddress host : hosts)
-        {
-            Set<Token> tokens = Collections.singleton(partitioner.getRandomToken());
-            Gossiper.instance.initializeNodeUnsafe(host, UUID.randomUUID(), 1);
-            Gossiper.instance.injectApplicationState(host, ApplicationState.TOKENS, valueFactory.tokens(tokens));
-
-            setNodeShutdown(host);
-            tokenMap.put(host, tokens);
-        }
-    }
-
-    /**
-     * A basic test case.
-     *
-     * @throws Exception on failure
-     */
-    @Test
-    public void testBasic() throws Exception
-    {
-        final YamlFileNetworkTopologySnitch snitch = new YamlFileNetworkTopologySnitch(confFile);
-        checkEndpoint(snitch, FBUtilities.getBroadcastAddress().getHostAddress(), "DC1", "RAC1");
-        checkEndpoint(snitch, "192.168.1.100", "DC1", "RAC1");
-        checkEndpoint(snitch, "10.0.0.12", "DC1", "RAC2");
-        checkEndpoint(snitch, "127.0.0.3", "DC1", "RAC3");
-        checkEndpoint(snitch, "10.20.114.10", "DC2", "RAC1");
-        checkEndpoint(snitch, "127.0.0.8", "DC3", "RAC8");
-        checkEndpoint(snitch, "6.6.6.6", "DC1", "r1");
-    }
-
-    /**
-     * Asserts that a snitch's determination of data center and rack for an endpoint match what we expect.
-     *
-     * @param snitch
-     *            snitch
-     * @param endpointString
-     *            endpoint address as a string
-     * @param expectedDatacenter
-     *            expected data center
-     * @param expectedRack
-     *            expected rack
-     */
-    public static void checkEndpoint(final AbstractNetworkTopologySnitch snitch,
-                                     final String endpointString, final String expectedDatacenter,
-                                     final String expectedRack)
-    {
-        final InetAddress endpoint = InetAddresses.forString(endpointString);
-        Assert.assertEquals(expectedDatacenter, snitch.getDatacenter(endpoint));
-        Assert.assertEquals(expectedRack, snitch.getRack(endpoint));
-    }
-
-
-    private static TopologyConfig moveNode(TopologyConfig topologyConfig,
-                                           String broadcastAddress, String dcLocalAddress,
-                                           String oldDC, String newDC,
-                                           String oldRack, String newRack)
-    {
-
-        for (Datacenter dc : topologyConfig.topology)
-        {
-            if (oldDC != null && oldRack != null)
-            {
-                if (dc.dc_name.equals(oldDC))
-                {
-                    for (Rack rack : dc.racks)
-                    {
-                        if (rack.rack_name.equals(oldRack))
-                        {
-                            for (Node node : rack.nodes)
-                            {
-                                if (node.broadcast_address.equals(broadcastAddress))
-                                {
-                                    rack.nodes.remove(node);
-                                    break;
-                                }
-                            }
-                        }
-                    }
-                }
-            }
-
-            if (newDC != null && newRack != null)
-            {
-                if (dc.dc_name.equals(newDC))
-                {
-                    for (Rack rack : dc.racks)
-                    {
-                        if (rack.rack_name.equals(newRack))
-                        {
-                            Node node = new Node();
-                            node.broadcast_address = broadcastAddress;
-                            node.dc_local_address = dcLocalAddress;
-                            rack.nodes.add(node);
-                        }
-                    }
-                }
-            }
-        }
-
-        return topologyConfig;
-    }
-
-    private void setNodeShutdown(InetAddress host)
-    {
-        StorageService.instance.getTokenMetadata().removeEndpoint(host);
-        Gossiper.instance.injectApplicationState(host, ApplicationState.STATUS, valueFactory.shutdown(true));
-        Gossiper.instance.markDead(host, Gossiper.instance.getEndpointStateForEndpoint(host));
-    }
-
-    private void setNodeLive(InetAddress host)
-    {
-        Gossiper.instance.injectApplicationState(host, ApplicationState.STATUS, valueFactory.normal(tokenMap.get(host)));
-        Gossiper.instance.realMarkAlive(host, Gossiper.instance.getEndpointStateForEndpoint(host));
-        StorageService.instance.getTokenMetadata().updateNormalTokens(tokenMap.get(host), host);
-    }
-
-    /**
-     * Test that changing rack for a host in the configuration file is only effective if the host is not live.
-     * The original configuration file contains DC1, RAC1 for broadcast address 127.0.0.1 and dc_local_address 9.0.0.1.
-     */
-    @Test
-    public void testChangeHostRack() throws Exception
-    {
-        final InetAddress host = InetAddress.getByName("127.0.0.1");
-        final YamlFileNetworkTopologySnitch snitch = new YamlFileNetworkTopologySnitch(confFile);
-        checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC1");
-
-        try
-        {
-            final TopologyConfig topologyConfig = snitch.readConfig();
-            moveNode(topologyConfig, host.getHostAddress(), "9.0.0.1", "DC1", "DC1", "RAC1", "RAC2");
-
-            setNodeLive(host);
-            snitch.loadTopologyConfiguration(true, topologyConfig);
-            checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC1");
-
-            setNodeShutdown(host);
-            snitch.loadTopologyConfiguration(true, topologyConfig);
-            checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC2");
-        }
-        finally
-        {
-            setNodeShutdown(host);
-        }
-    }
-
-    /**
-     * Test that changing dc for a host in the configuration file is only effective if the host is not live.
-     * The original configuration file contains DC1, RAC1 for broadcast address 127.0.0.1 and dc_local_address 9.0.0.1.
-     */
-    @Test
-    public void testChangeHostDc() throws Exception
-    {
-        final InetAddress host = InetAddress.getByName("127.0.0.1");
-        final YamlFileNetworkTopologySnitch snitch = new YamlFileNetworkTopologySnitch(confFile);
-        checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC1");
-
-        try
-        {
-            final TopologyConfig topologyConfig = snitch.readConfig();
-            moveNode(topologyConfig, host.getHostAddress(), "9.0.0.1", "DC1", "DC2", "RAC1", "RAC1");
-
-            setNodeLive(host);
-            snitch.loadTopologyConfiguration(true, topologyConfig);
-            checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC1");
-
-            setNodeShutdown(host);
-            snitch.loadTopologyConfiguration(true, topologyConfig);
-            checkEndpoint(snitch, host.getHostAddress(), "DC2", "RAC1");
-        }
-        finally
-        {
-            setNodeShutdown(host);
-        }
-    }
-
-    /**
-     * Test that adding a host to the configuration file changes the host dc and rack only if the host
-     * is not live. The original configuration file does not contain 127.0.0.9 and so it should use
-     * the default data center DC1 and rack r1.
-     */
-    @Test
-    public void testAddHost() throws Exception
-    {
-        final InetAddress host = InetAddress.getByName("127.0.0.9");
-        final YamlFileNetworkTopologySnitch snitch = new YamlFileNetworkTopologySnitch(confFile);
-        checkEndpoint(snitch, host.getHostAddress(), "DC1", "r1"); // default
-
-        try
-        {
-            final TopologyConfig topologyConfig = snitch.readConfig();
-            moveNode(topologyConfig, host.getHostAddress(), "9.0.0.9", null, "DC2", null, "RAC2");
-
-            setNodeLive(host);
-            snitch.loadTopologyConfiguration(true, topologyConfig);
-            checkEndpoint(snitch, host.getHostAddress(), "DC1", "r1"); // unchanged
-
-            setNodeShutdown(host);
-            snitch.loadTopologyConfiguration(true, topologyConfig);
-            checkEndpoint(snitch, host.getHostAddress(), "DC2", "RAC2"); // changed
-        }
-        finally
-        {
-            setNodeShutdown(host);
-        }
-    }
-
-    /**
-     * Test that removing a host from the configuration file changes the host rack only if the host
-     * is not live. The original configuration file contains 127.0.0.2 in DC1, RAC2 and default DC1, r1 so removing
-     * this host should result in a different rack if the host is not live.
-     */
-    @Test
-    public void testRemoveHost() throws Exception
-    {
-        final InetAddress host = InetAddress.getByName("127.0.0.2");
-        final YamlFileNetworkTopologySnitch snitch = new YamlFileNetworkTopologySnitch(confFile);
-        checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC2");
-
-        try
-        {
-            final TopologyConfig topologyConfig = snitch.readConfig();
-            moveNode(topologyConfig, host.getHostAddress(), "9.0.0.2", "DC1", null, "RAC2", null);
-
-            setNodeLive(host);
-            snitch.loadTopologyConfiguration(true, topologyConfig);
-            checkEndpoint(snitch, host.getHostAddress(), "DC1", "RAC2"); // unchanged
-
-            setNodeShutdown(host);
-            snitch.loadTopologyConfiguration(true, topologyConfig);
-            checkEndpoint(snitch, host.getHostAddress(), "DC1", "r1"); // default
-        }
-        finally
-        {
-            setNodeShutdown(host);
-        }
-    }
-
-    /**
-     * Test that we can change the default only if this does not result in any live node changing dc or rack.
-     * The configuration file contains default DC1 and r1 and we change it to DC2 and r2. Let's use host 127.0.0.9
-     * since it is not in the configuration file.
-     */
-    @Test
-    public void testChangeDefault() throws Exception
-    {
-        final InetAddress host = InetAddress.getByName("127.0.0.9");
-        final YamlFileNetworkTopologySnitch snitch = new YamlFileNetworkTopologySnitch(confFile);
-        checkEndpoint(snitch, host.getHostAddress(), "DC1", "r1"); // default
-
-        try
-        {
-            final TopologyConfig topologyConfig = snitch.readConfig();
-            topologyConfig.default_dc_name = "DC2";
-            topologyConfig.default_rack_name = "r2";
-
-            setNodeLive(host);
-            snitch.loadTopologyConfiguration(true, topologyConfig);
-            checkEndpoint(snitch, host.getHostAddress(), "DC1", "r1"); // unchanged
-
-            setNodeShutdown(host);
-            snitch.loadTopologyConfiguration(true, topologyConfig);
-            checkEndpoint(snitch, host.getHostAddress(), "DC2", "r2"); // default updated
-        }
-        finally
-        {
-            setNodeShutdown(host);
-        }
-    }
-}
diff --git a/test/unit/org/apache/cassandra/metrics/CQLMetricsTest.java b/test/unit/org/apache/cassandra/metrics/CQLMetricsTest.java
index 88104a6..a357d24 100644
--- a/test/unit/org/apache/cassandra/metrics/CQLMetricsTest.java
+++ b/test/unit/org/apache/cassandra/metrics/CQLMetricsTest.java
@@ -64,9 +64,9 @@
     @Test
     public void testPreparedStatementsCount()
     {
-        assertEquals(0, (int) QueryProcessor.metrics.preparedStatementsCount.value());
+        assertEquals(0, (int) QueryProcessor.metrics.preparedStatementsCount.getValue());
         metricsStatement = session.prepare("INSERT INTO junit.metricstest (id, val) VALUES (?, ?)");
-        assertEquals(1, (int) QueryProcessor.metrics.preparedStatementsCount.value());
+        assertEquals(1, (int) QueryProcessor.metrics.preparedStatementsCount.getValue());
     }
 
     @Test
@@ -74,14 +74,14 @@
     {
         clearMetrics();
 
-        assertEquals(0, QueryProcessor.metrics.preparedStatementsExecuted.count());
-        assertEquals(0, QueryProcessor.metrics.regularStatementsExecuted.count());
+        assertEquals(0, QueryProcessor.metrics.preparedStatementsExecuted.getCount());
+        assertEquals(0, QueryProcessor.metrics.regularStatementsExecuted.getCount());
 
         for (int i = 0; i < 10; i++)
             session.execute(String.format("INSERT INTO junit.metricstest (id, val) VALUES (%d, '%s')", i, "val" + i));
 
-        assertEquals(0, QueryProcessor.metrics.preparedStatementsExecuted.count());
-        assertEquals(10, QueryProcessor.metrics.regularStatementsExecuted.count());
+        assertEquals(0, QueryProcessor.metrics.preparedStatementsExecuted.getCount());
+        assertEquals(10, QueryProcessor.metrics.regularStatementsExecuted.getCount());
     }
 
     @Test
@@ -89,14 +89,14 @@
     {
         clearMetrics();
 
-        assertEquals(0, QueryProcessor.metrics.preparedStatementsExecuted.count());
-        assertEquals(0, QueryProcessor.metrics.regularStatementsExecuted.count());
+        assertEquals(0, QueryProcessor.metrics.preparedStatementsExecuted.getCount());
+        assertEquals(0, QueryProcessor.metrics.regularStatementsExecuted.getCount());
 
         for (int i = 0; i < 10; i++)
             session.execute(metricsStatement.bind(i, "val" + i));
 
-        assertEquals(10, QueryProcessor.metrics.preparedStatementsExecuted.count());
-        assertEquals(0, QueryProcessor.metrics.regularStatementsExecuted.count());
+        assertEquals(10, QueryProcessor.metrics.preparedStatementsExecuted.getCount());
+        assertEquals(0, QueryProcessor.metrics.regularStatementsExecuted.getCount());
     }
 
     @Test
@@ -104,22 +104,22 @@
     {
         clearMetrics();
 
-        assertEquals(Double.NaN, QueryProcessor.metrics.preparedStatementsRatio.value());
+        assertEquals(Double.NaN, QueryProcessor.metrics.preparedStatementsRatio.getValue());
 
         for (int i = 0; i < 10; i++)
             session.execute(metricsStatement.bind(i, "val" + i));
-        assertEquals(1.0, QueryProcessor.metrics.preparedStatementsRatio.value());
+        assertEquals(1.0, QueryProcessor.metrics.preparedStatementsRatio.getValue());
 
         for (int i = 0; i < 10; i++)
             session.execute(String.format("INSERT INTO junit.metricstest (id, val) VALUES (%d, '%s')", i, "val" + i));
-        assertEquals(0.5, QueryProcessor.metrics.preparedStatementsRatio.value());
+        assertEquals(0.5, QueryProcessor.metrics.preparedStatementsRatio.getValue());
     }
 
     private void clearMetrics()
     {
-        QueryProcessor.metrics.preparedStatementsExecuted.clear();
-        QueryProcessor.metrics.regularStatementsExecuted.clear();
-        QueryProcessor.metrics.preparedStatementsEvicted.clear();
+        QueryProcessor.metrics.preparedStatementsExecuted.dec(QueryProcessor.metrics.preparedStatementsExecuted.getCount());
+        QueryProcessor.metrics.regularStatementsExecuted.dec(QueryProcessor.metrics.regularStatementsExecuted.getCount());
+        QueryProcessor.metrics.preparedStatementsEvicted.dec(QueryProcessor.metrics.preparedStatementsEvicted.getCount());
     }
 }
 
diff --git a/test/unit/org/apache/cassandra/metrics/DecayingEstimatedHistogramReservoirTest.java b/test/unit/org/apache/cassandra/metrics/DecayingEstimatedHistogramReservoirTest.java
new file mode 100644
index 0000000..ef1fed3
--- /dev/null
+++ b/test/unit/org/apache/cassandra/metrics/DecayingEstimatedHistogramReservoirTest.java
@@ -0,0 +1,412 @@
+/*
+ * 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.metrics;
+
+import org.junit.Test;
+
+import com.codahale.metrics.Clock;
+import com.codahale.metrics.Snapshot;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+
+public class DecayingEstimatedHistogramReservoirTest
+{
+    private static final double DOUBLE_ASSERT_DELTA = 0;
+
+    @Test
+    public void testSimple()
+    {
+        {
+            // 0 and 1 map to the same, first bucket
+            DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir();
+            histogram.update(0);
+            assertEquals(1, histogram.getSnapshot().getValues()[0]);
+            histogram.update(1);
+            assertEquals(2, histogram.getSnapshot().getValues()[0]);
+        }
+        {
+            // 0 and 1 map to different buckets
+            DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(true, DecayingEstimatedHistogramReservoir.DEFAULT_BUCKET_COUNT);
+            histogram.update(0);
+            assertEquals(1, histogram.getSnapshot().getValues()[0]);
+            histogram.update(1);
+            Snapshot snapshot = histogram.getSnapshot();
+            assertEquals(1, snapshot.getValues()[0]);
+            assertEquals(1, snapshot.getValues()[1]);
+        }
+    }
+
+    @Test
+    public void testOverflow()
+    {
+        DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(DecayingEstimatedHistogramReservoir.DEFAULT_ZERO_CONSIDERATION, 1);
+        histogram.update(100);
+        assert histogram.isOverflowed();
+        assertEquals(Long.MAX_VALUE, histogram.getSnapshot().getMax());
+    }
+
+    @Test
+    public void testMinMax()
+    {
+        DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir();
+        histogram.update(16);
+        Snapshot snapshot = histogram.getSnapshot();
+        assertEquals(15, snapshot.getMin());
+        assertEquals(17, snapshot.getMax());
+    }
+
+    @Test
+    public void testMean()
+    {
+        {
+            TestClock clock = new TestClock();
+
+            DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(DecayingEstimatedHistogramReservoir.DEFAULT_ZERO_CONSIDERATION, DecayingEstimatedHistogramReservoir.DEFAULT_BUCKET_COUNT, clock);
+            for (int i = 0; i < 40; i++)
+                histogram.update(0);
+            for (int i = 0; i < 20; i++)
+                histogram.update(1);
+            for (int i = 0; i < 10; i++)
+                histogram.update(2);
+            assertEquals(1.14D, histogram.getSnapshot().getMean(), 0.1D);
+        }
+        {
+            TestClock clock = new TestClock();
+
+            DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(true, DecayingEstimatedHistogramReservoir.DEFAULT_BUCKET_COUNT, clock);
+            for (int i = 0; i < 40; i++)
+                histogram.update(0);
+            for (int i = 0; i < 20; i++)
+                histogram.update(1);
+            for (int i = 0; i < 10; i++)
+                histogram.update(2);
+            assertEquals(0.57D, histogram.getSnapshot().getMean(), 0.1D);
+        }
+    }
+
+    @Test
+    public void testStdDev()
+    {
+        {
+            TestClock clock = new TestClock();
+
+            DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(DecayingEstimatedHistogramReservoir.DEFAULT_ZERO_CONSIDERATION, DecayingEstimatedHistogramReservoir.DEFAULT_BUCKET_COUNT, clock);
+            for (int i = 0; i < 20; i++)
+                histogram.update(10);
+            for (int i = 0; i < 40; i++)
+                histogram.update(20);
+            for (int i = 0; i < 20; i++)
+                histogram.update(30);
+
+            Snapshot snapshot = histogram.getSnapshot();
+            assertEquals(20.0D, snapshot.getMean(), 2.0D);
+            assertEquals(7.07D, snapshot.getStdDev(), 2.0D);
+        }
+    }
+
+    @Test
+    public void testFindingCorrectBuckets()
+    {
+        TestClock clock = new TestClock();
+
+        DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(DecayingEstimatedHistogramReservoir.DEFAULT_ZERO_CONSIDERATION, 90, clock);
+        histogram.update(23282687);
+        assertFalse(histogram.isOverflowed());
+        assertEquals(1, histogram.getSnapshot().getValues()[89]);
+
+        histogram.update(9);
+        assertEquals(1, histogram.getSnapshot().getValues()[8]);
+
+        histogram.update(21);
+        histogram.update(22);
+        Snapshot snapshot = histogram.getSnapshot();
+        assertEquals(2, snapshot.getValues()[13]);
+        assertEquals(6277304.5D, snapshot.getMean(), DOUBLE_ASSERT_DELTA);
+    }
+
+    @Test
+    public void testPercentile()
+    {
+        {
+            TestClock clock = new TestClock();
+
+            DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(DecayingEstimatedHistogramReservoir.DEFAULT_ZERO_CONSIDERATION, DecayingEstimatedHistogramReservoir.DEFAULT_BUCKET_COUNT, clock);
+            // percentile of empty histogram is 0
+            assertEquals(0D, histogram.getSnapshot().getValue(0.99), DOUBLE_ASSERT_DELTA);
+
+            histogram.update(1);
+            // percentile of a histogram with one element should be that element
+            assertEquals(1D, histogram.getSnapshot().getValue(0.99), DOUBLE_ASSERT_DELTA);
+
+            histogram.update(10);
+            assertEquals(10D, histogram.getSnapshot().getValue(0.99), DOUBLE_ASSERT_DELTA);
+        }
+
+        {
+            TestClock clock = new TestClock();
+
+            DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(DecayingEstimatedHistogramReservoir.DEFAULT_ZERO_CONSIDERATION, DecayingEstimatedHistogramReservoir.DEFAULT_BUCKET_COUNT, clock);
+
+            histogram.update(1);
+            histogram.update(2);
+            histogram.update(3);
+            histogram.update(4);
+            histogram.update(5);
+
+            Snapshot snapshot = histogram.getSnapshot();
+            assertEquals(0, snapshot.getValue(0.00), DOUBLE_ASSERT_DELTA);
+            assertEquals(3, snapshot.getValue(0.50), DOUBLE_ASSERT_DELTA);
+            assertEquals(3, snapshot.getValue(0.60), DOUBLE_ASSERT_DELTA);
+            assertEquals(5, snapshot.getValue(1.00), DOUBLE_ASSERT_DELTA);
+        }
+
+        {
+            TestClock clock = new TestClock();
+
+            DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(DecayingEstimatedHistogramReservoir.DEFAULT_ZERO_CONSIDERATION, DecayingEstimatedHistogramReservoir.DEFAULT_BUCKET_COUNT, clock);
+
+            for (int i = 11; i <= 20; i++)
+                histogram.update(i);
+
+            // Right now the histogram looks like:
+            //    10   12   14   17   20
+            //     0    2    2    3    3
+            // %:  0   20   40   70  100
+            Snapshot snapshot = histogram.getSnapshot();
+            assertEquals(12, snapshot.getValue(0.01), DOUBLE_ASSERT_DELTA);
+            assertEquals(14, snapshot.getValue(0.30), DOUBLE_ASSERT_DELTA);
+            assertEquals(17, snapshot.getValue(0.50), DOUBLE_ASSERT_DELTA);
+            assertEquals(17, snapshot.getValue(0.60), DOUBLE_ASSERT_DELTA);
+            assertEquals(20, snapshot.getValue(0.80), DOUBLE_ASSERT_DELTA);
+        }
+        {
+            TestClock clock = new TestClock();
+
+            DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(true, DecayingEstimatedHistogramReservoir.DEFAULT_BUCKET_COUNT, clock);
+            histogram.update(0);
+            histogram.update(0);
+            histogram.update(1);
+
+            Snapshot snapshot = histogram.getSnapshot();
+            assertEquals(0, snapshot.getValue(0.5), DOUBLE_ASSERT_DELTA);
+            assertEquals(1, snapshot.getValue(0.99), DOUBLE_ASSERT_DELTA);
+        }
+    }
+
+
+    @Test
+    public void testDecayingPercentile()
+    {
+        {
+            TestClock clock = new TestClock();
+
+            DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(DecayingEstimatedHistogramReservoir.DEFAULT_ZERO_CONSIDERATION, DecayingEstimatedHistogramReservoir.DEFAULT_BUCKET_COUNT, clock);
+            // percentile of empty histogram is 0
+            assertEquals(0, histogram.getSnapshot().getValue(1.0), DOUBLE_ASSERT_DELTA);
+
+            for (int v = 1; v <= 100; v++)
+            {
+                for (int i = 0; i < 10_000; i++)
+                {
+                    histogram.update(v);
+                }
+            }
+
+            Snapshot snapshot = histogram.getSnapshot();
+            assertEstimatedQuantile(05, snapshot.getValue(0.05));
+            assertEstimatedQuantile(20, snapshot.getValue(0.20));
+            assertEstimatedQuantile(40, snapshot.getValue(0.40));
+            assertEstimatedQuantile(99, snapshot.getValue(0.99));
+
+            clock.addSeconds(DecayingEstimatedHistogramReservoir.HALF_TIME_IN_S);
+            snapshot = histogram.getSnapshot();
+            assertEstimatedQuantile(05, snapshot.getValue(0.05));
+            assertEstimatedQuantile(20, snapshot.getValue(0.20));
+            assertEstimatedQuantile(40, snapshot.getValue(0.40));
+            assertEstimatedQuantile(99, snapshot.getValue(0.99));
+
+            for (int v = 1; v <= 50; v++)
+            {
+                for (int i = 0; i < 10_000; i++)
+                {
+                    histogram.update(v);
+                }
+            }
+
+            snapshot = histogram.getSnapshot();
+            assertEstimatedQuantile(04, snapshot.getValue(0.05));
+            assertEstimatedQuantile(14, snapshot.getValue(0.20));
+            assertEstimatedQuantile(27, snapshot.getValue(0.40));
+            assertEstimatedQuantile(98, snapshot.getValue(0.99));
+
+            clock.addSeconds(DecayingEstimatedHistogramReservoir.HALF_TIME_IN_S);
+            snapshot = histogram.getSnapshot();
+            assertEstimatedQuantile(04, snapshot.getValue(0.05));
+            assertEstimatedQuantile(14, snapshot.getValue(0.20));
+            assertEstimatedQuantile(27, snapshot.getValue(0.40));
+            assertEstimatedQuantile(98, snapshot.getValue(0.99));
+
+            for (int v = 1; v <= 50; v++)
+            {
+                for (int i = 0; i < 10_000; i++)
+                {
+                    histogram.update(v);
+                }
+            }
+
+            snapshot = histogram.getSnapshot();
+            assertEstimatedQuantile(03, snapshot.getValue(0.05));
+            assertEstimatedQuantile(12, snapshot.getValue(0.20));
+            assertEstimatedQuantile(23, snapshot.getValue(0.40));
+            assertEstimatedQuantile(96, snapshot.getValue(0.99));
+
+            clock.addSeconds(DecayingEstimatedHistogramReservoir.HALF_TIME_IN_S);
+            snapshot = histogram.getSnapshot();
+            assertEstimatedQuantile(03, snapshot.getValue(0.05));
+            assertEstimatedQuantile(12, snapshot.getValue(0.20));
+            assertEstimatedQuantile(23, snapshot.getValue(0.40));
+            assertEstimatedQuantile(96, snapshot.getValue(0.99));
+
+            for (int v = 11; v <= 20; v++)
+            {
+                for (int i = 0; i < 5_000; i++)
+                {
+                    histogram.update(v);
+                }
+            }
+
+            snapshot = histogram.getSnapshot();
+            assertEstimatedQuantile(04, snapshot.getValue(0.05));
+            assertEstimatedQuantile(12, snapshot.getValue(0.20));
+            assertEstimatedQuantile(20, snapshot.getValue(0.40));
+            assertEstimatedQuantile(95, snapshot.getValue(0.99));
+
+            clock.addSeconds(DecayingEstimatedHistogramReservoir.HALF_TIME_IN_S);
+            snapshot = histogram.getSnapshot();
+            assertEstimatedQuantile(04, snapshot.getValue(0.05));
+            assertEstimatedQuantile(12, snapshot.getValue(0.20));
+            assertEstimatedQuantile(20, snapshot.getValue(0.40));
+            assertEstimatedQuantile(95, snapshot.getValue(0.99));
+
+        }
+
+        {
+            TestClock clock = new TestClock();
+
+            DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(DecayingEstimatedHistogramReservoir.DEFAULT_ZERO_CONSIDERATION, DecayingEstimatedHistogramReservoir.DEFAULT_BUCKET_COUNT, clock);
+            // percentile of empty histogram is 0
+            assertEquals(0, histogram.getSnapshot().getValue(0.99), DOUBLE_ASSERT_DELTA);
+
+            for (int m = 0; m < 40; m++)
+            {
+                for (int i = 0; i < 1_000_000; i++)
+                {
+                    histogram.update(2);
+                }
+                // percentile of a histogram with one element should be that element
+                clock.addSeconds(DecayingEstimatedHistogramReservoir.HALF_TIME_IN_S);
+                assertEquals(2, histogram.getSnapshot().getValue(0.99), DOUBLE_ASSERT_DELTA);
+            }
+
+            clock.addSeconds(DecayingEstimatedHistogramReservoir.HALF_TIME_IN_S * 100);
+            assertEquals(0, histogram.getSnapshot().getValue(0.99), DOUBLE_ASSERT_DELTA);
+        }
+
+        {
+            TestClock clock = new TestClock();
+
+            DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(DecayingEstimatedHistogramReservoir.DEFAULT_ZERO_CONSIDERATION, DecayingEstimatedHistogramReservoir.DEFAULT_BUCKET_COUNT, clock);
+
+            histogram.update(20);
+            histogram.update(21);
+            histogram.update(22);
+            Snapshot snapshot = histogram.getSnapshot();
+            assertEquals(1, snapshot.getValues()[12]);
+            assertEquals(2, snapshot.getValues()[13]);
+
+            clock.addSeconds(DecayingEstimatedHistogramReservoir.HALF_TIME_IN_S);
+
+            histogram.update(20);
+            histogram.update(21);
+            histogram.update(22);
+            snapshot = histogram.getSnapshot();
+            assertEquals(2, snapshot.getValues()[12]);
+            assertEquals(4, snapshot.getValues()[13]);
+        }
+    }
+
+    @Test
+    public void testDecayingMean()
+    {
+        {
+            TestClock clock = new TestClock();
+
+            DecayingEstimatedHistogramReservoir histogram = new DecayingEstimatedHistogramReservoir(DecayingEstimatedHistogramReservoir.DEFAULT_ZERO_CONSIDERATION, DecayingEstimatedHistogramReservoir.DEFAULT_BUCKET_COUNT, clock);
+
+            clock.addMillis(DecayingEstimatedHistogramReservoir.LANDMARK_RESET_INTERVAL_IN_MS - 1_000L);
+
+            while (clock.getTime() < DecayingEstimatedHistogramReservoir.LANDMARK_RESET_INTERVAL_IN_MS + 1_000L)
+            {
+                clock.addMillis(900);
+                for (int i = 0; i < 1_000_000; i++)
+                {
+                    histogram.update(1000);
+                    histogram.update(2000);
+                    histogram.update(3000);
+                    histogram.update(4000);
+                    histogram.update(5000);
+                }
+                assertEquals(3000D, histogram.getSnapshot().getMean(), 500D);
+            }
+        }
+    }
+
+    private void assertEstimatedQuantile(long expectedValue, double actualValue)
+    {
+        assertTrue("Expected at least [" + expectedValue + "] but actual is [" + actualValue + "]", actualValue >= expectedValue);
+        assertTrue("Expected less than [" + Math.round(expectedValue * 1.2) + "] but actual is [" + actualValue + "]", actualValue < Math.round(expectedValue * 1.2));
+    }
+
+    public class TestClock extends Clock {
+        private long tick = 0;
+
+        public void addMillis(long millis)
+        {
+            tick += millis * 1_000_000L;
+        }
+
+        public void addSeconds(long seconds)
+        {
+            tick += seconds * 1_000_000_000L;
+        }
+
+        public long getTick()
+        {
+            return tick;
+        }
+
+        public long getTime()
+        {
+            return tick / 1_000_000L;
+        };
+    }
+}
diff --git a/test/unit/org/apache/cassandra/metrics/LatencyMetricsTest.java b/test/unit/org/apache/cassandra/metrics/LatencyMetricsTest.java
index e5f0804..ae4b733 100644
--- a/test/unit/org/apache/cassandra/metrics/LatencyMetricsTest.java
+++ b/test/unit/org/apache/cassandra/metrics/LatencyMetricsTest.java
@@ -43,7 +43,7 @@
 
         for (int i = 0; i < 10000; i++)
         {
-            Double recent = l.getRecentLatency();
+            Double recent = l.latency.getOneMinuteRate();
             assertFalse(recent.equals(Double.POSITIVE_INFINITY));
         }
     }
diff --git a/test/unit/org/apache/cassandra/net/MessagingServiceTest.java b/test/unit/org/apache/cassandra/net/MessagingServiceTest.java
index 766eb39..8631f03 100644
--- a/test/unit/org/apache/cassandra/net/MessagingServiceTest.java
+++ b/test/unit/org/apache/cassandra/net/MessagingServiceTest.java
@@ -42,7 +42,6 @@
         assertEquals(1, logs.size());
         assertEquals("READ messages were dropped in last 5000 ms: 2500 for internal timeout and 2500 for cross node timeout", logs.get(0));
         assertEquals(5000, (int)messagingService.getDroppedMessages().get(verb.toString()));
-        assertEquals(5000, (int) messagingService.getRecentlyDroppedMessages().get(verb.toString()));
 
         logs = messagingService.getDroppedMessagesLogs();
         assertEquals(0, logs.size());
@@ -53,7 +52,6 @@
         logs = messagingService.getDroppedMessagesLogs();
         assertEquals("READ messages were dropped in last 5000 ms: 1250 for internal timeout and 1250 for cross node timeout", logs.get(0));
         assertEquals(7500, (int)messagingService.getDroppedMessages().get(verb.toString()));
-        assertEquals(2500, (int) messagingService.getRecentlyDroppedMessages().get(verb.toString()));
     }
 
 }
diff --git a/test/unit/org/apache/cassandra/repair/DifferencerTest.java b/test/unit/org/apache/cassandra/repair/DifferencerTest.java
deleted file mode 100644
index 3229c58..0000000
--- a/test/unit/org/apache/cassandra/repair/DifferencerTest.java
+++ /dev/null
@@ -1,150 +0,0 @@
-/*
- * 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.repair;
-
-import java.net.InetAddress;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.Set;
-import java.util.UUID;
-
-import org.junit.After;
-import org.junit.Test;
-
-import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.dht.IPartitioner;
-import org.apache.cassandra.dht.Murmur3Partitioner;
-import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.net.MessageIn;
-import org.apache.cassandra.net.MessageOut;
-import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.service.ActiveRepairService;
-import org.apache.cassandra.sink.IMessageSink;
-import org.apache.cassandra.sink.SinkManager;
-import org.apache.cassandra.repair.messages.RepairMessage;
-import org.apache.cassandra.repair.messages.SyncComplete;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.MerkleTree;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-public class DifferencerTest extends SchemaLoader
-{
-    private static final IPartitioner partirioner = new Murmur3Partitioner();
-
-    @After
-    public void tearDown()
-    {
-        SinkManager.clear();
-    }
-
-    /**
-     * When there is no difference between two, Differencer should respond SYNC_COMPLETE
-     */
-    @Test
-    public void testNoDifference() throws Throwable
-    {
-        final InetAddress ep1 = InetAddress.getByName("127.0.0.1");
-        final InetAddress ep2 = InetAddress.getByName("127.0.0.1");
-
-        SinkManager.add(new IMessageSink()
-        {
-            @SuppressWarnings("unchecked")
-            public MessageOut handleMessage(MessageOut message, int id, InetAddress to)
-            {
-                if (message.verb == MessagingService.Verb.REPAIR_MESSAGE)
-                {
-                    RepairMessage m = (RepairMessage) message.payload;
-                    assertEquals(RepairMessage.Type.SYNC_COMPLETE, m.messageType);
-                    // we should see SYNC_COMPLETE
-                    assertEquals(new NodePair(ep1, ep2), ((SyncComplete)m).nodes);
-                }
-                return null;
-            }
-
-            public MessageIn handleMessage(MessageIn message, int id, InetAddress to)
-            {
-                return null;
-            }
-        });
-        Range<Token> range = new Range<>(partirioner.getMinimumToken(), partirioner.getRandomToken());
-        RepairJobDesc desc = new RepairJobDesc(UUID.randomUUID(), UUID.randomUUID(), "Keyspace1", "Standard1", range);
-
-        MerkleTree tree1 = createInitialTree(desc);
-        MerkleTree tree2 = createInitialTree(desc);
-
-        // difference the trees
-        // note: we reuse the same endpoint which is bogus in theory but fine here
-        TreeResponse r1 = new TreeResponse(ep1, tree1);
-        TreeResponse r2 = new TreeResponse(ep2, tree2);
-        Differencer diff = new Differencer(desc, r1, r2);
-        diff.run();
-
-        assertTrue(diff.differences.isEmpty());
-    }
-
-    @Test
-    public void testDifference() throws Throwable
-    {
-        Range<Token> range = new Range<>(partirioner.getMinimumToken(), partirioner.getRandomToken());
-        UUID parentRepairSession = UUID.randomUUID();
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
-
-        ActiveRepairService.instance.registerParentRepairSession(parentRepairSession, FBUtilities.getBroadcastAddress(), Arrays.asList(cfs), Arrays.asList(range));
-
-        RepairJobDesc desc = new RepairJobDesc(parentRepairSession, UUID.randomUUID(), "Keyspace1", "Standard1", range);
-
-        MerkleTree tree1 = createInitialTree(desc);
-        MerkleTree tree2 = createInitialTree(desc);
-
-        // change a range in one of the trees
-        Token token = partirioner.midpoint(range.left, range.right);
-        tree1.invalidate(token);
-        MerkleTree.TreeRange changed = tree1.get(token);
-        changed.hash("non-empty hash!".getBytes());
-
-        Set<Range<Token>> interesting = new HashSet<>();
-        interesting.add(changed);
-
-        // difference the trees
-        // note: we reuse the same endpoint which is bogus in theory but fine here
-        TreeResponse r1 = new TreeResponse(InetAddress.getByName("127.0.0.1"), tree1);
-        TreeResponse r2 = new TreeResponse(InetAddress.getByName("127.0.0.2"), tree2);
-        Differencer diff = new Differencer(desc, r1, r2);
-        diff.run();
-
-        // ensure that the changed range was recorded
-        assertEquals("Wrong differing ranges", interesting, new HashSet<>(diff.differences));
-    }
-
-    private MerkleTree createInitialTree(RepairJobDesc desc)
-    {
-        MerkleTree tree = new MerkleTree(partirioner, desc.range, MerkleTree.RECOMMENDED_DEPTH, (int)Math.pow(2, 15));
-        tree.init();
-        for (MerkleTree.TreeRange r : tree.invalids())
-        {
-            r.ensureHashInitialised();
-        }
-        return tree;
-    }
-}
diff --git a/test/unit/org/apache/cassandra/repair/LocalSyncTaskTest.java b/test/unit/org/apache/cassandra/repair/LocalSyncTaskTest.java
new file mode 100644
index 0000000..892ced1
--- /dev/null
+++ b/test/unit/org/apache/cassandra/repair/LocalSyncTaskTest.java
@@ -0,0 +1,130 @@
+/*
+ * 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.repair;
+
+import java.net.InetAddress;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.UUID;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.service.ActiveRepairService;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.MerkleTree;
+
+import static org.junit.Assert.assertEquals;
+
+public class LocalSyncTaskTest extends SchemaLoader
+{
+    private static final IPartitioner partirioner = Murmur3Partitioner.instance;
+    public static final String KEYSPACE1 = "DifferencerTest";
+    public static final String CF_STANDARD = "Standard1";
+
+    @BeforeClass
+    public static void defineSchema() throws Exception
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD));
+    }
+
+    /**
+     * When there is no difference between two, LocalSyncTask should return stats with 0 difference.
+     */
+    @Test
+    public void testNoDifference() throws Throwable
+    {
+        final InetAddress ep1 = InetAddress.getByName("127.0.0.1");
+        final InetAddress ep2 = InetAddress.getByName("127.0.0.1");
+
+        Range<Token> range = new Range<>(partirioner.getMinimumToken(), partirioner.getRandomToken());
+        RepairJobDesc desc = new RepairJobDesc(UUID.randomUUID(), UUID.randomUUID(), KEYSPACE1, "Standard1", range);
+
+        MerkleTree tree1 = createInitialTree(desc);
+        MerkleTree tree2 = createInitialTree(desc);
+
+        // difference the trees
+        // note: we reuse the same endpoint which is bogus in theory but fine here
+        TreeResponse r1 = new TreeResponse(ep1, tree1);
+        TreeResponse r2 = new TreeResponse(ep2, tree2);
+        LocalSyncTask task = new LocalSyncTask(desc, r1, r2, ActiveRepairService.UNREPAIRED_SSTABLE);
+        task.run();
+
+        assertEquals(0, task.get().numberOfDifferences);
+    }
+
+    @Test
+    public void testDifference() throws Throwable
+    {
+        Range<Token> range = new Range<>(partirioner.getMinimumToken(), partirioner.getRandomToken());
+        UUID parentRepairSession = UUID.randomUUID();
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
+
+        ActiveRepairService.instance.registerParentRepairSession(parentRepairSession, FBUtilities.getBroadcastAddress(), Arrays.asList(cfs), Arrays.asList(range), false, false);
+
+        RepairJobDesc desc = new RepairJobDesc(parentRepairSession, UUID.randomUUID(), KEYSPACE1, "Standard1", range);
+
+        MerkleTree tree1 = createInitialTree(desc);
+        MerkleTree tree2 = createInitialTree(desc);
+
+        // change a range in one of the trees
+        Token token = partirioner.midpoint(range.left, range.right);
+        tree1.invalidate(token);
+        MerkleTree.TreeRange changed = tree1.get(token);
+        changed.hash("non-empty hash!".getBytes());
+
+        Set<Range<Token>> interesting = new HashSet<>();
+        interesting.add(changed);
+
+        // difference the trees
+        // note: we reuse the same endpoint which is bogus in theory but fine here
+        TreeResponse r1 = new TreeResponse(InetAddress.getByName("127.0.0.1"), tree1);
+        TreeResponse r2 = new TreeResponse(InetAddress.getByName("127.0.0.2"), tree2);
+        LocalSyncTask task = new LocalSyncTask(desc, r1, r2, ActiveRepairService.UNREPAIRED_SSTABLE);
+        task.run();
+
+        // ensure that the changed range was recorded
+        assertEquals("Wrong differing ranges", interesting.size(), task.getCurrentStat().numberOfDifferences);
+    }
+
+    private MerkleTree createInitialTree(RepairJobDesc desc)
+    {
+        MerkleTree tree = new MerkleTree(partirioner, desc.range, MerkleTree.RECOMMENDED_DEPTH, (int)Math.pow(2, 15));
+        tree.init();
+        for (MerkleTree.TreeRange r : tree.invalids())
+        {
+            r.ensureHashInitialised();
+        }
+        return tree;
+    }
+}
diff --git a/test/unit/org/apache/cassandra/repair/RepairSessionTest.java b/test/unit/org/apache/cassandra/repair/RepairSessionTest.java
new file mode 100644
index 0000000..8ea2bfa
--- /dev/null
+++ b/test/unit/org/apache/cassandra/repair/RepairSessionTest.java
@@ -0,0 +1,71 @@
+/*
+ * 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.repair;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+
+import com.google.common.collect.Sets;
+import org.junit.Test;
+
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.gms.Gossiper;
+import org.apache.cassandra.service.ActiveRepairService;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.UUIDGen;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+public class RepairSessionTest
+{
+    @Test
+    public void testConviction() throws Exception
+    {
+        InetAddress remote = InetAddress.getByName("127.0.0.2");
+        Gossiper.instance.initializeNodeUnsafe(remote, UUID.randomUUID(), 1);
+
+        // Set up RepairSession
+        UUID parentSessionId = UUIDGen.getTimeUUID();
+        UUID sessionId = UUID.randomUUID();
+        IPartitioner p = Murmur3Partitioner.instance;
+        Range<Token> repairRange = new Range<>(p.getToken(ByteBufferUtil.bytes(0)), p.getToken(ByteBufferUtil.bytes(100)));
+        Set<InetAddress> endpoints = Sets.newHashSet(remote);
+        RepairSession session = new RepairSession(parentSessionId, sessionId, repairRange, "Keyspace1", RepairParallelism.SEQUENTIAL, endpoints, ActiveRepairService.UNREPAIRED_SSTABLE, "Standard1");
+
+        // perform convict
+        session.convict(remote, Double.MAX_VALUE);
+
+        // RepairSession should throw ExecutorException with the cause of IOException when getting its value
+        try
+        {
+            session.get();
+            fail();
+        }
+        catch (ExecutionException ex)
+        {
+            assertEquals(IOException.class, ex.getCause().getClass());
+        }
+    }
+}
diff --git a/test/unit/org/apache/cassandra/repair/RequestCoordinatorTest.java b/test/unit/org/apache/cassandra/repair/RequestCoordinatorTest.java
deleted file mode 100644
index ad6eabe..0000000
--- a/test/unit/org/apache/cassandra/repair/RequestCoordinatorTest.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/*
- * 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.repair;
-
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.locator.AbstractEndpointSnitch;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-
-import static org.junit.Assert.assertEquals;
-
-public class RequestCoordinatorTest implements IRequestProcessor<InetAddress>
-{
-    private InetAddress[] endpoints;
-    private List<InetAddress> activeRequests;
-    private static Random random = new Random(0);
-
-    @Before
-    public void setup() throws UnknownHostException
-    {
-        endpoints = new InetAddress[12];
-        for (int i = 0; i < 12; i++)
-            endpoints[i] = InetAddress.getByName("127.0.0." + (i + 1));
-        activeRequests = new ArrayList<>();
-        DatabaseDescriptor.setEndpointSnitch(new AbstractEndpointSnitch()
-        {
-            @Override
-            public String getRack(InetAddress endpoint)
-            {
-                return "rack1";
-            }
-
-            @Override
-            public String getDatacenter(InetAddress endpoint)
-            {
-                // 127.0.0.1, 127.0.0.2, 127.0.0.3 -> datacenter1
-                // 127.0.0.4, 127.0.0.5, 127.0.0.6 -> datacenter2 etc
-                int no = endpoint.getAddress()[3] - 1;
-                return "datacenter" + (no / 3 + 1);
-            }
-
-            @Override
-            public int compareEndpoints(InetAddress target, InetAddress a1, InetAddress a2)
-            {
-                return 0;
-            }
-        });
-    }
-
-    @Override
-    public void process(InetAddress request)
-    {
-        activeRequests.add(request);
-    }
-
-    @Test
-    public void testSequentialRequestCoordinator()
-    {
-        SequentialRequestCoordinator<InetAddress> coordinator = new SequentialRequestCoordinator<>(this);
-        for (InetAddress endpoint : endpoints)
-            coordinator.add(endpoint);
-        coordinator.start();
-        int max = finishRequests(coordinator);
-        assertEquals(1, max);
-    }
-
-    @Test
-    public void testParallelRequestCoordinator()
-    {
-        ParallelRequestCoordinator<InetAddress> coordinator = new ParallelRequestCoordinator<>(this);
-        for (InetAddress endpoint : endpoints)
-            coordinator.add(endpoint);
-        coordinator.start();
-        int max = finishRequests(coordinator);
-        assertEquals(endpoints.length, max);
-    }
-
-    @Test
-    public void testDatacenterAwareRequestCoordinator()
-    {
-        DatacenterAwareRequestCoordinator coordinator = new DatacenterAwareRequestCoordinator(this);
-        for (InetAddress endpoint : endpoints)
-            coordinator.add(endpoint);
-        coordinator.start();
-        int max = finishRequests(coordinator);
-        assertEquals(4, max);
-    }
-
-    private int finishRequests(IRequestCoordinator<InetAddress> requestCoordinator)
-    {
-        int max = 0;
-        while (activeRequests.size() > 0)
-        {
-            max = Math.max(max, activeRequests.size());
-            // Finish a request
-            int ix = random.nextInt(activeRequests.size());
-            InetAddress finished = activeRequests.get(ix);
-            activeRequests.remove(ix);
-            requestCoordinator.completed(finished);
-        }
-        return max;
-    }
-}
diff --git a/test/unit/org/apache/cassandra/repair/ValidatorTest.java b/test/unit/org/apache/cassandra/repair/ValidatorTest.java
index 4d65cdb..61ab3da 100644
--- a/test/unit/org/apache/cassandra/repair/ValidatorTest.java
+++ b/test/unit/org/apache/cassandra/repair/ValidatorTest.java
@@ -20,12 +20,23 @@
 import java.io.IOException;
 import java.net.InetAddress;
 import java.security.MessageDigest;
+import java.util.Collections;
 import java.util.UUID;
+import java.util.concurrent.TimeUnit;
 
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.SettableFuture;
+
+import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.db.compaction.CompactionsTest;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.SequentialWriter;
 import org.junit.After;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.BufferDecoratedKey;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.DecoratedKey;
@@ -36,31 +47,45 @@
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.sstable.ColumnStats;
-import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.sink.IMessageSink;
-import org.apache.cassandra.sink.SinkManager;
+import org.apache.cassandra.net.IMessageSink;
 import org.apache.cassandra.repair.messages.RepairMessage;
 import org.apache.cassandra.repair.messages.ValidationComplete;
+import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.MerkleTree;
+import org.apache.cassandra.utils.UUIDGen;
 import org.apache.cassandra.utils.concurrent.SimpleCondition;
 
 import static org.junit.Assert.*;
 
-public class ValidatorTest extends SchemaLoader
+public class ValidatorTest
 {
-    private final String keyspace = "Keyspace1";
-    private final String columnFamily = "Standard1";
+    private static final long TEST_TIMEOUT = 60; //seconds
+
+    private static final String keyspace = "ValidatorTest";
+    private static final String columnFamily = "Standard1";
     private final IPartitioner partitioner = StorageService.getPartitioner();
 
+    @BeforeClass
+    public static void defineSchema() throws Exception
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(keyspace,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(keyspace, columnFamily));
+    }
+
     @After
     public void tearDown()
     {
-        SinkManager.clear();
+        MessagingService.instance().clearMessageSinks();
     }
 
     @Test
@@ -69,35 +94,7 @@
         Range<Token> range = new Range<>(partitioner.getMinimumToken(), partitioner.getRandomToken());
         final RepairJobDesc desc = new RepairJobDesc(UUID.randomUUID(), UUID.randomUUID(), keyspace, columnFamily, range);
 
-        final SimpleCondition lock = new SimpleCondition();
-        SinkManager.add(new IMessageSink()
-        {
-            @SuppressWarnings("unchecked")
-            public MessageOut handleMessage(MessageOut message, int id, InetAddress to)
-            {
-                try
-                {
-                    if (message.verb == MessagingService.Verb.REPAIR_MESSAGE)
-                    {
-                        RepairMessage m = (RepairMessage) message.payload;
-                        assertEquals(RepairMessage.Type.VALIDATION_COMPLETE, m.messageType);
-                        assertEquals(desc, m.desc);
-                        assertTrue(((ValidationComplete)m).success);
-                        assertNotNull(((ValidationComplete)m).tree);
-                    }
-                }
-                finally
-                {
-                    lock.signalAll();
-                }
-                return null;
-            }
-
-            public MessageIn handleMessage(MessageIn message, int id, InetAddress to)
-            {
-                return null;
-            }
-        });
+        final ListenableFuture<MessageOut> outgoingMessageSink = registerOutgoingMessageSink();
 
         InetAddress remote = InetAddress.getByName("127.0.0.2");
 
@@ -119,8 +116,13 @@
         Token min = tree.partitioner().getMinimumToken();
         assertNotNull(tree.hash(new Range<>(min, min)));
 
-        if (!lock.isSignaled())
-            lock.await();
+        MessageOut message = outgoingMessageSink.get(TEST_TIMEOUT, TimeUnit.SECONDS);
+        assertEquals(MessagingService.Verb.REPAIR_MESSAGE, message.verb);
+        RepairMessage m = (RepairMessage) message.payload;
+        assertEquals(RepairMessage.Type.VALIDATION_COMPLETE, m.messageType);
+        assertEquals(desc, m.desc);
+        assertTrue(((ValidationComplete) m).success);
+        assertNotNull(((ValidationComplete) m).tree);
     }
 
     private static class CompactedRowStub extends AbstractCompactedRow
@@ -130,7 +132,7 @@
             super(key);
         }
 
-        public RowIndexEntry write(long currentPosition, DataOutputPlus out) throws IOException
+        public RowIndexEntry write(long currentPosition, SequentialWriter out) throws IOException
         {
             throw new UnsupportedOperationException();
         }
@@ -151,42 +153,99 @@
         Range<Token> range = new Range<>(partitioner.getMinimumToken(), partitioner.getRandomToken());
         final RepairJobDesc desc = new RepairJobDesc(UUID.randomUUID(), UUID.randomUUID(), keyspace, columnFamily, range);
 
-        final SimpleCondition lock = new SimpleCondition();
-        SinkManager.add(new IMessageSink()
-        {
-            @SuppressWarnings("unchecked")
-            public MessageOut handleMessage(MessageOut message, int id, InetAddress to)
-            {
-                try
-                {
-                    if (message.verb == MessagingService.Verb.REPAIR_MESSAGE)
-                    {
-                        RepairMessage m = (RepairMessage) message.payload;
-                        assertEquals(RepairMessage.Type.VALIDATION_COMPLETE, m.messageType);
-                        assertEquals(desc, m.desc);
-                        assertFalse(((ValidationComplete) m).success);
-                        assertNull(((ValidationComplete)m).tree);
-                    }
-                }
-                finally
-                {
-                    lock.signalAll();
-                }
-                return null;
-            }
-
-            public MessageIn handleMessage(MessageIn message, int id, InetAddress to)
-            {
-                return null;
-            }
-        });
+        final ListenableFuture<MessageOut> outgoingMessageSink = registerOutgoingMessageSink();
 
         InetAddress remote = InetAddress.getByName("127.0.0.2");
 
         Validator validator = new Validator(desc, remote, 0);
         validator.fail();
 
-        if (!lock.isSignaled())
-            lock.await();
+        MessageOut message = outgoingMessageSink.get(TEST_TIMEOUT, TimeUnit.SECONDS);
+        assertEquals(MessagingService.Verb.REPAIR_MESSAGE, message.verb);
+        RepairMessage m = (RepairMessage) message.payload;
+        assertEquals(RepairMessage.Type.VALIDATION_COMPLETE, m.messageType);
+        assertEquals(desc, m.desc);
+        assertFalse(((ValidationComplete) m).success);
+        assertNull(((ValidationComplete) m).tree);
+    }
+
+    @Test
+    public void simpleValidationTest128() throws Exception
+    {
+        simpleValidationTest(128);
+    }
+
+    @Test
+    public void simpleValidationTest1500() throws Exception
+    {
+        simpleValidationTest(1500);
+    }
+
+    /**
+     * Test for CASSANDRA-5263
+     * 1. Create N rows
+     * 2. Run validation compaction
+     * 3. Expect merkle tree with size 2^(log2(n))
+     */
+    public void simpleValidationTest(int n) throws Exception
+    {
+        Keyspace ks = Keyspace.open(keyspace);
+        ColumnFamilyStore cfs = ks.getColumnFamilyStore(columnFamily);
+        cfs.clearUnsafe();
+
+        // disable compaction while flushing
+        cfs.disableAutoCompaction();
+
+        CompactionsTest.populate(keyspace, columnFamily, 0, n, 0); //ttl=3s
+
+        cfs.forceBlockingFlush();
+        assertEquals(1, cfs.getSSTables().size());
+
+        // wait enough to force single compaction
+        TimeUnit.SECONDS.sleep(5);
+
+        SSTableReader sstable = cfs.getSSTables().iterator().next();
+        UUID repairSessionId = UUIDGen.getTimeUUID();
+        final RepairJobDesc desc = new RepairJobDesc(repairSessionId, UUIDGen.getTimeUUID(), cfs.keyspace.getName(),
+                                               cfs.getColumnFamilyName(), new Range<Token>(sstable.first.getToken(),
+                                                                                             sstable.last.getToken()));
+
+        ActiveRepairService.instance.registerParentRepairSession(repairSessionId, FBUtilities.getBroadcastAddress(),
+                                                                 Collections.singletonList(cfs), Collections.singleton(desc.range),
+                                                                 false, false);
+
+        final ListenableFuture<MessageOut> outgoingMessageSink = registerOutgoingMessageSink();
+        Validator validator = new Validator(desc, FBUtilities.getBroadcastAddress(), 0, true);
+        CompactionManager.instance.submitValidation(cfs, validator);
+
+        MessageOut message = outgoingMessageSink.get(TEST_TIMEOUT, TimeUnit.SECONDS);
+        assertEquals(MessagingService.Verb.REPAIR_MESSAGE, message.verb);
+        RepairMessage m = (RepairMessage) message.payload;
+        assertEquals(RepairMessage.Type.VALIDATION_COMPLETE, m.messageType);
+        assertEquals(desc, m.desc);
+        assertTrue(((ValidationComplete) m).success);
+        MerkleTree tree = ((ValidationComplete) m).tree;
+
+        assertEquals(Math.pow(2, Math.ceil(Math.log(n) / Math.log(2))), tree.size(), 0.0);
+        assertEquals(tree.rowCount(), n);
+    }
+
+    private ListenableFuture<MessageOut> registerOutgoingMessageSink()
+    {
+        final SettableFuture<MessageOut> future = SettableFuture.create();
+        MessagingService.instance().addMessageSink(new IMessageSink()
+        {
+            public boolean allowOutgoingMessage(MessageOut message, int id, InetAddress to)
+            {
+                future.set(message);
+                return false;
+            }
+
+            public boolean allowIncomingMessage(MessageIn message, int id)
+            {
+                return false;
+            }
+        });
+        return future;
     }
 }
diff --git a/test/unit/org/apache/cassandra/repair/messages/RepairOptionTest.java b/test/unit/org/apache/cassandra/repair/messages/RepairOptionTest.java
new file mode 100644
index 0000000..a0eea4e
--- /dev/null
+++ b/test/unit/org/apache/cassandra/repair/messages/RepairOptionTest.java
@@ -0,0 +1,129 @@
+/*
+ * 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.repair.messages;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.junit.Test;
+
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.repair.RepairParallelism;
+import org.apache.cassandra.utils.FBUtilities;
+
+import static org.junit.Assert.*;
+
+public class RepairOptionTest
+{
+    @Test
+    public void testParseOptions()
+    {
+        IPartitioner partitioner = Murmur3Partitioner.instance;
+        Token.TokenFactory tokenFactory = partitioner.getTokenFactory();
+
+        // parse with empty options
+        RepairOption option = RepairOption.parse(new HashMap<String, String>(), partitioner);
+
+        if (FBUtilities.isWindows() && (DatabaseDescriptor.getDiskAccessMode() != Config.DiskAccessMode.standard || DatabaseDescriptor.getIndexAccessMode() != Config.DiskAccessMode.standard))
+            assertTrue(option.getParallelism() == RepairParallelism.PARALLEL);
+        else
+            assertTrue(option.getParallelism() == RepairParallelism.SEQUENTIAL);
+
+        assertFalse(option.isPrimaryRange());
+        assertFalse(option.isIncremental());
+
+        // parse everything
+        Map<String, String> options = new HashMap<>();
+        options.put(RepairOption.PARALLELISM_KEY, "parallel");
+        options.put(RepairOption.PRIMARY_RANGE_KEY, "false");
+        options.put(RepairOption.INCREMENTAL_KEY, "false");
+        options.put(RepairOption.RANGES_KEY, "0:10,11:20,21:30");
+        options.put(RepairOption.COLUMNFAMILIES_KEY, "cf1,cf2,cf3");
+        options.put(RepairOption.DATACENTERS_KEY, "dc1,dc2,dc3");
+        options.put(RepairOption.HOSTS_KEY, "127.0.0.1,127.0.0.2,127.0.0.3");
+
+        option = RepairOption.parse(options, partitioner);
+        assertTrue(option.getParallelism() == RepairParallelism.PARALLEL);
+        assertFalse(option.isPrimaryRange());
+        assertFalse(option.isIncremental());
+
+        Set<Range<Token>> expectedRanges = new HashSet<>(3);
+        expectedRanges.add(new Range<>(tokenFactory.fromString("0"), tokenFactory.fromString("10")));
+        expectedRanges.add(new Range<>(tokenFactory.fromString("11"), tokenFactory.fromString("20")));
+        expectedRanges.add(new Range<>(tokenFactory.fromString("21"), tokenFactory.fromString("30")));
+        assertEquals(expectedRanges, option.getRanges());
+
+        Set<String> expectedCFs = new HashSet<>(3);
+        expectedCFs.add("cf1");
+        expectedCFs.add("cf2");
+        expectedCFs.add("cf3");
+        assertEquals(expectedCFs, option.getColumnFamilies());
+
+        Set<String> expectedDCs = new HashSet<>(3);
+        expectedDCs.add("dc1");
+        expectedDCs.add("dc2");
+        expectedDCs.add("dc3");
+        assertEquals(expectedDCs, option.getDataCenters());
+
+        Set<String> expectedHosts = new HashSet<>(3);
+        expectedHosts.add("127.0.0.1");
+        expectedHosts.add("127.0.0.2");
+        expectedHosts.add("127.0.0.3");
+        assertEquals(expectedHosts, option.getHosts());
+    }
+
+    @Test
+    public void testPrWithLocalParseOptions()
+    {
+        DatabaseDescriptor.forceStaticInitialization();
+
+        Map<String, String> options = new HashMap<>();
+        options.put(RepairOption.PARALLELISM_KEY, "parallel");
+        options.put(RepairOption.PRIMARY_RANGE_KEY, "true");
+        options.put(RepairOption.INCREMENTAL_KEY, "false");
+        options.put(RepairOption.COLUMNFAMILIES_KEY, "cf1,cf2,cf3");
+        options.put(RepairOption.DATACENTERS_KEY, "datacenter1");
+
+        RepairOption option = RepairOption.parse(options, Murmur3Partitioner.instance);
+        assertTrue(option.isPrimaryRange());
+
+        Set<String> expectedDCs = new HashSet<>(3);
+        expectedDCs.add("datacenter1");
+        assertEquals(expectedDCs, option.getDataCenters());
+    }
+
+    @Test
+    public void testIncrementalRepairWithSubrangesIsNotGlobal() throws Exception
+    {
+        RepairOption ro = RepairOption.parse(ImmutableMap.of(RepairOption.INCREMENTAL_KEY, "true", RepairOption.RANGES_KEY, "42:42"),
+                           Murmur3Partitioner.instance);
+        assertFalse(ro.isGlobal());
+        ro = RepairOption.parse(ImmutableMap.of(RepairOption.INCREMENTAL_KEY, "true", RepairOption.RANGES_KEY, ""),
+                Murmur3Partitioner.instance);
+        assertTrue(ro.isGlobal());
+    }
+}
diff --git a/test/unit/org/apache/cassandra/schema/CreateTableValidationTest.java b/test/unit/org/apache/cassandra/schema/CreateTableValidationTest.java
new file mode 100644
index 0000000..9708552
--- /dev/null
+++ b/test/unit/org/apache/cassandra/schema/CreateTableValidationTest.java
@@ -0,0 +1,51 @@
+/*
+ * 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.schema;
+
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.junit.Test;
+
+import static org.junit.Assert.fail;
+
+public class CreateTableValidationTest extends CQLTester
+{
+    private static final String KEYSPACE1 = "CreateTableValidationTest";
+
+    @Test
+    public void testInvalidBloomFilterFPRatio() throws Throwable
+    {
+        try
+        {
+            createTableMayThrow("CREATE TABLE %s (a int PRIMARY KEY, b int) WITH bloom_filter_fp_chance = 0.0000001");
+            fail("Expected an fp chance of 0.0000001 to be rejected");
+        }
+        catch (ConfigurationException exc) { }
+
+        try
+        {
+            createTableMayThrow("CREATE TABLE %s (a int PRIMARY KEY, b int) WITH bloom_filter_fp_chance = 1.1");
+            fail("Expected an fp chance of 1.1 to be rejected");
+        }
+        catch (ConfigurationException exc) { }
+
+        // sanity check
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b int) WITH bloom_filter_fp_chance = 0.1");
+    }
+}
diff --git a/test/unit/org/apache/cassandra/schema/DefsTest.java b/test/unit/org/apache/cassandra/schema/DefsTest.java
new file mode 100644
index 0000000..ecf6709
--- /dev/null
+++ b/test/unit/org/apache/cassandra/schema/DefsTest.java
@@ -0,0 +1,578 @@
+/**
+ * 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.schema;
+
+import java.io.File;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.OrderedJUnit4ClassRunner;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.Util;
+import org.apache.cassandra.config.*;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.composites.*;
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.db.marshal.TimeUUIDType;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.SSTableDeletingTask;
+import org.apache.cassandra.locator.OldNetworkTopologyStrategy;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.schema.LegacySchemaTables;
+import org.apache.cassandra.service.MigrationManager;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+
+import static org.apache.cassandra.Util.cellname;
+
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import com.google.common.base.Supplier;
+
+@RunWith(OrderedJUnit4ClassRunner.class)
+public class DefsTest
+{
+    private static final String KEYSPACE1 = "Keyspace1";
+    private static final String KEYSPACE3 = "Keyspace3";
+    private static final String KEYSPACE6 = "Keyspace6";
+    private static final String EMPTYKEYSPACE = "DefsTestEmptyKeyspace";
+    private static final String CF_STANDARD1 = "Standard1";
+    private static final String CF_STANDARD2 = "Standard2";
+    private static final String CF_INDEXED = "Indexed1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.startGossiper();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD2));
+        SchemaLoader.createKeyspace(KEYSPACE3, true, false,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(5),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1),
+                                    SchemaLoader.indexCFMD(KEYSPACE3, CF_INDEXED, true));
+        SchemaLoader.createKeyspace(KEYSPACE6,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.indexCFMD(KEYSPACE6, CF_INDEXED, true));
+    }
+
+    @Test
+    public void testCFMetaDataApply() throws ConfigurationException
+    {
+        CFMetaData cfm = new CFMetaData(KEYSPACE1,
+                                        "TestApplyCFM_CF",
+                                        ColumnFamilyType.Standard,
+                                        new SimpleDenseCellNameType(BytesType.instance));
+
+        for (int i = 0; i < 5; i++)
+        {
+            ByteBuffer name = ByteBuffer.wrap(new byte[] { (byte)i });
+            cfm.addColumnDefinition(ColumnDefinition.regularDef(cfm, name, BytesType.instance, null).setIndex(Integer.toString(i), IndexType.KEYS, null));
+        }
+
+        cfm.comment("No comment")
+           .readRepairChance(0.5)
+           .gcGraceSeconds(100000)
+           .minCompactionThreshold(500)
+           .maxCompactionThreshold(500);
+
+        // we'll be adding this one later. make sure it's not already there.
+        Assert.assertNull(cfm.getColumnDefinition(ByteBuffer.wrap(new byte[] { 5 })));
+
+        CFMetaData cfNew = cfm.copy();
+
+        // add one.
+        ColumnDefinition addIndexDef = ColumnDefinition.regularDef(cfm, ByteBuffer.wrap(new byte[] { 5 }), BytesType.instance, null)
+                                                       .setIndex("5", IndexType.KEYS, null);
+        cfNew.addColumnDefinition(addIndexDef);
+
+        // remove one.
+        ColumnDefinition removeIndexDef = ColumnDefinition.regularDef(cfm, ByteBuffer.wrap(new byte[] { 0 }), BytesType.instance, null)
+                                                          .setIndex("0", IndexType.KEYS, null);
+        Assert.assertTrue(cfNew.removeColumnDefinition(removeIndexDef));
+
+        cfm.apply(cfNew);
+
+        for (int i = 1; i < cfm.allColumns().size(); i++)
+            Assert.assertNotNull(cfm.getColumnDefinition(ByteBuffer.wrap(new byte[] { 1 })));
+        Assert.assertNull(cfm.getColumnDefinition(ByteBuffer.wrap(new byte[] { 0 })));
+        Assert.assertNotNull(cfm.getColumnDefinition(ByteBuffer.wrap(new byte[] { 5 })));
+    }
+
+    @Test
+    public void testInvalidNames()
+    {
+        String[] valid = {"1", "a", "_1", "b_", "__", "1_a"};
+        for (String s : valid)
+            Assert.assertTrue(CFMetaData.isNameValid(s));
+
+        String[] invalid = {"b@t", "dash-y", "", " ", "dot.s", ".hidden"};
+        for (String s : invalid)
+            Assert.assertFalse(CFMetaData.isNameValid(s));
+    }
+
+    @Ignore
+    @Test
+    public void saveAndRestore()
+    {
+        /*
+        // verify dump and reload.
+        UUID first = UUIDGen.makeType1UUIDFromHost(FBUtilities.getBroadcastAddress());
+        DefsTables.dumpToStorage(first);
+        List<KSMetaData> defs = new ArrayList<KSMetaData>(DefsTables.loadFromStorage(first));
+
+        Assert.assertTrue(defs.size() > 0);
+        Assert.assertEquals(defs.size(), Schema.instance.getNonSystemKeyspaces().size());
+        for (KSMetaData loaded : defs)
+        {
+            KSMetaData defined = Schema.instance.getKeyspaceDefinition(loaded.name);
+            Assert.assertTrue(String.format("%s != %s", loaded, defined), defined.equals(loaded));
+        }
+        */
+    }
+
+    @Test
+    public void addNewCfToBogusKeyspace()
+    {
+        CFMetaData newCf = addTestCF("MadeUpKeyspace", "NewCF", "new cf");
+        try
+        {
+            MigrationManager.announceNewColumnFamily(newCf);
+            throw new AssertionError("You shouldn't be able to do anything to a keyspace that doesn't exist.");
+        }
+        catch (ConfigurationException expected)
+        {
+        }
+    }
+
+    @Test
+    public void addNewCfWithNullComment() throws ConfigurationException
+    {
+        final String ks = KEYSPACE1;
+        final String cf = "BrandNewCfWithNull";
+        KSMetaData original = Schema.instance.getKSMetaData(ks);
+
+        CFMetaData newCf = addTestCF(original.name, cf, null);
+
+        Assert.assertFalse(Schema.instance.getKSMetaData(ks).cfMetaData().containsKey(newCf.cfName));
+        MigrationManager.announceNewColumnFamily(newCf);
+
+        Assert.assertTrue(Schema.instance.getKSMetaData(ks).cfMetaData().containsKey(newCf.cfName));
+        Assert.assertEquals(newCf, Schema.instance.getKSMetaData(ks).cfMetaData().get(newCf.cfName));
+    }
+
+    @Test
+    public void addNewCF() throws ConfigurationException
+    {
+        final String ks = KEYSPACE1;
+        final String cf = "BrandNewCf";
+        KSMetaData original = Schema.instance.getKSMetaData(ks);
+
+        CFMetaData newCf = addTestCF(original.name, cf, "A New Table");
+
+        Assert.assertFalse(Schema.instance.getKSMetaData(ks).cfMetaData().containsKey(newCf.cfName));
+        MigrationManager.announceNewColumnFamily(newCf);
+
+        Assert.assertTrue(Schema.instance.getKSMetaData(ks).cfMetaData().containsKey(newCf.cfName));
+        Assert.assertEquals(newCf, Schema.instance.getKSMetaData(ks).cfMetaData().get(newCf.cfName));
+
+        // now read and write to it.
+        CellName col0 = cellname("col0");
+        DecoratedKey dk = Util.dk("key0");
+        Mutation rm = new Mutation(ks, dk.getKey());
+        rm.add(cf, col0, ByteBufferUtil.bytes("value0"), 1L);
+        rm.applyUnsafe();
+        ColumnFamilyStore store = Keyspace.open(ks).getColumnFamilyStore(cf);
+        Assert.assertNotNull(store);
+        store.forceBlockingFlush();
+
+        ColumnFamily cfam = store.getColumnFamily(Util.namesQueryFilter(store, dk, col0));
+        Assert.assertNotNull(cfam.getColumn(col0));
+        Cell col = cfam.getColumn(col0);
+        Assert.assertEquals(ByteBufferUtil.bytes("value0"), col.value());
+    }
+
+    @Test
+    public void dropCf() throws ConfigurationException
+    {
+        DecoratedKey dk = Util.dk("dropCf");
+        // sanity
+        final KSMetaData ks = Schema.instance.getKSMetaData(KEYSPACE1);
+        Assert.assertNotNull(ks);
+        final CFMetaData cfm = ks.cfMetaData().get("Standard1");
+        Assert.assertNotNull(cfm);
+
+        // write some data, force a flush, then verify that files exist on disk.
+        Mutation rm = new Mutation(ks.name, dk.getKey());
+        for (int i = 0; i < 100; i++)
+            rm.add(cfm.cfName, cellname("col" + i), ByteBufferUtil.bytes("anyvalue"), 1L);
+        rm.applyUnsafe();
+        final ColumnFamilyStore store = Keyspace.open(cfm.ksName).getColumnFamilyStore(cfm.cfName);
+        Assert.assertNotNull(store);
+        store.forceBlockingFlush();
+        Assert.assertTrue(store.directories.sstableLister().list().size() > 0);
+
+        MigrationManager.announceColumnFamilyDrop(ks.name, cfm.cfName);
+
+        Assert.assertFalse(Schema.instance.getKSMetaData(ks.name).cfMetaData().containsKey(cfm.cfName));
+
+        // any write should fail.
+        rm = new Mutation(ks.name, dk.getKey());
+        boolean success = true;
+        try
+        {
+            rm.add("Standard1", cellname("col0"), ByteBufferUtil.bytes("value0"), 1L);
+            rm.applyUnsafe();
+        }
+        catch (Throwable th)
+        {
+            success = false;
+        }
+        Assert.assertFalse("This mutation should have failed since the CF no longer exists.", success);
+
+        // verify that the files are gone.
+        Supplier<Object> lambda = new Supplier<Object>() {
+            @Override
+            public Boolean get() {
+                for (File file : store.directories.sstableLister().listFiles())
+                {
+                    if (file.getPath().endsWith("Data.db") && !new File(file.getPath().replace("Data.db", "Compacted")).exists())
+                        return false;
+                }
+                return true;
+            }
+        };
+        Util.spinAssertEquals(true, lambda, 30);
+
+    }
+
+    @Test
+    public void addNewKS() throws ConfigurationException
+    {
+        DecoratedKey dk = Util.dk("key0");
+        CFMetaData newCf = addTestCF("NewKeyspace1", "AddedStandard1", "A new cf for a new ks");
+
+        KSMetaData newKs = KSMetaData.testMetadata(newCf.ksName, SimpleStrategy.class, KSMetaData.optsWithRF(5), newCf);
+
+        MigrationManager.announceNewKeyspace(newKs);
+
+        Assert.assertNotNull(Schema.instance.getKSMetaData(newCf.ksName));
+        Assert.assertEquals(Schema.instance.getKSMetaData(newCf.ksName), newKs);
+
+        // test reads and writes.
+        CellName col0 = cellname("col0");
+        Mutation rm = new Mutation(newCf.ksName, dk.getKey());
+        rm.add(newCf.cfName, col0, ByteBufferUtil.bytes("value0"), 1L);
+        rm.applyUnsafe();
+        ColumnFamilyStore store = Keyspace.open(newCf.ksName).getColumnFamilyStore(newCf.cfName);
+        Assert.assertNotNull(store);
+        store.forceBlockingFlush();
+
+        ColumnFamily cfam = store.getColumnFamily(Util.namesQueryFilter(store, dk, col0));
+        Assert.assertNotNull(cfam.getColumn(col0));
+        Cell col = cfam.getColumn(col0);
+        Assert.assertEquals(ByteBufferUtil.bytes("value0"), col.value());
+    }
+
+    @Test
+    public void dropKS() throws ConfigurationException
+    {
+        DecoratedKey dk = Util.dk("dropKs");
+        // sanity
+        final KSMetaData ks = Schema.instance.getKSMetaData(KEYSPACE1);
+        Assert.assertNotNull(ks);
+        final CFMetaData cfm = ks.cfMetaData().get("Standard2");
+        Assert.assertNotNull(cfm);
+
+        // write some data, force a flush, then verify that files exist on disk.
+        Mutation rm = new Mutation(ks.name, dk.getKey());
+        for (int i = 0; i < 100; i++)
+            rm.add(cfm.cfName, cellname("col" + i), ByteBufferUtil.bytes("anyvalue"), 1L);
+        rm.applyUnsafe();
+        ColumnFamilyStore store = Keyspace.open(cfm.ksName).getColumnFamilyStore(cfm.cfName);
+        Assert.assertNotNull(store);
+        store.forceBlockingFlush();
+        Assert.assertTrue(store.directories.sstableLister().list().size() > 0);
+
+        MigrationManager.announceKeyspaceDrop(ks.name);
+
+        Assert.assertNull(Schema.instance.getKSMetaData(ks.name));
+
+        // write should fail.
+        rm = new Mutation(ks.name, dk.getKey());
+        boolean success = true;
+        try
+        {
+            rm.add("Standard1", cellname("col0"), ByteBufferUtil.bytes("value0"), 1L);
+            rm.applyUnsafe();
+        }
+        catch (Throwable th)
+        {
+            success = false;
+        }
+        Assert.assertFalse("This mutation should have failed since the CF no longer exists.", success);
+
+        // reads should fail too.
+        boolean threw = false;
+        try
+        {
+            Keyspace.open(ks.name);
+        }
+        catch (Throwable th)
+        {
+            threw = true;
+        }
+        Assert.assertTrue(threw);
+    }
+
+    @Test
+    public void dropKSUnflushed() throws ConfigurationException
+    {
+        DecoratedKey dk = Util.dk("dropKs");
+        // sanity
+        final KSMetaData ks = Schema.instance.getKSMetaData(KEYSPACE3);
+        Assert.assertNotNull(ks);
+        final CFMetaData cfm = ks.cfMetaData().get("Standard1");
+        Assert.assertNotNull(cfm);
+
+        // write some data
+        Mutation rm = new Mutation(ks.name, dk.getKey());
+        for (int i = 0; i < 100; i++)
+            rm.add(cfm.cfName, cellname("col" + i), ByteBufferUtil.bytes("anyvalue"), 1L);
+        rm.applyUnsafe();
+
+        MigrationManager.announceKeyspaceDrop(ks.name);
+
+        Assert.assertNull(Schema.instance.getKSMetaData(ks.name));
+    }
+
+    @Test
+    public void createEmptyKsAddNewCf() throws ConfigurationException
+    {
+        Assert.assertNull(Schema.instance.getKSMetaData(EMPTYKEYSPACE));
+
+        KSMetaData newKs = KSMetaData.testMetadata(EMPTYKEYSPACE, SimpleStrategy.class, KSMetaData.optsWithRF(5));
+
+        MigrationManager.announceNewKeyspace(newKs);
+        Assert.assertNotNull(Schema.instance.getKSMetaData(EMPTYKEYSPACE));
+
+        CFMetaData newCf = addTestCF(EMPTYKEYSPACE, "AddedLater", "A new CF to add to an empty KS");
+
+        //should not exist until apply
+        Assert.assertFalse(Schema.instance.getKSMetaData(newKs.name).cfMetaData().containsKey(newCf.cfName));
+
+        //add the new CF to the empty space
+        MigrationManager.announceNewColumnFamily(newCf);
+
+        Assert.assertTrue(Schema.instance.getKSMetaData(newKs.name).cfMetaData().containsKey(newCf.cfName));
+        Assert.assertEquals(Schema.instance.getKSMetaData(newKs.name).cfMetaData().get(newCf.cfName), newCf);
+
+        // now read and write to it.
+        CellName col0 = cellname("col0");
+        DecoratedKey dk = Util.dk("key0");
+        Mutation rm = new Mutation(newKs.name, dk.getKey());
+        rm.add(newCf.cfName, col0, ByteBufferUtil.bytes("value0"), 1L);
+        rm.applyUnsafe();
+        ColumnFamilyStore store = Keyspace.open(newKs.name).getColumnFamilyStore(newCf.cfName);
+        Assert.assertNotNull(store);
+        store.forceBlockingFlush();
+
+        ColumnFamily cfam = store.getColumnFamily(Util.namesQueryFilter(store, dk, col0));
+        Assert.assertNotNull(cfam.getColumn(col0));
+        Cell col = cfam.getColumn(col0);
+        Assert.assertEquals(ByteBufferUtil.bytes("value0"), col.value());
+    }
+
+    @Test
+    public void testUpdateKeyspace() throws ConfigurationException
+    {
+        // create a keyspace to serve as existing.
+        CFMetaData cf = addTestCF("UpdatedKeyspace", "AddedStandard1", "A new cf for a new ks");
+        KSMetaData oldKs = KSMetaData.testMetadata(cf.ksName, SimpleStrategy.class, KSMetaData.optsWithRF(5), cf);
+
+        MigrationManager.announceNewKeyspace(oldKs);
+
+        Assert.assertNotNull(Schema.instance.getKSMetaData(cf.ksName));
+        Assert.assertEquals(Schema.instance.getKSMetaData(cf.ksName), oldKs);
+
+        // names should match.
+        KSMetaData newBadKs2 = KSMetaData.testMetadata(cf.ksName + "trash", SimpleStrategy.class, KSMetaData.optsWithRF(4));
+        try
+        {
+            MigrationManager.announceKeyspaceUpdate(newBadKs2);
+            throw new AssertionError("Should not have been able to update a KS with an invalid KS name.");
+        }
+        catch (ConfigurationException ex)
+        {
+            // expected.
+        }
+
+        KSMetaData newKs = KSMetaData.testMetadata(cf.ksName, OldNetworkTopologyStrategy.class, KSMetaData.optsWithRF(1));
+        MigrationManager.announceKeyspaceUpdate(newKs);
+
+        KSMetaData newFetchedKs = Schema.instance.getKSMetaData(newKs.name);
+        Assert.assertEquals(newFetchedKs.strategyClass, newKs.strategyClass);
+        Assert.assertFalse(newFetchedKs.strategyClass.equals(oldKs.strategyClass));
+    }
+
+    @Test
+    public void testUpdateColumnFamilyNoIndexes() throws ConfigurationException
+    {
+        // create a keyspace with a cf to update.
+        CFMetaData cf = addTestCF("UpdatedCfKs", "Standard1added", "A new cf that will be updated");
+        KSMetaData ksm = KSMetaData.testMetadata(cf.ksName, SimpleStrategy.class, KSMetaData.optsWithRF(1), cf);
+        MigrationManager.announceNewKeyspace(ksm);
+
+        Assert.assertNotNull(Schema.instance.getKSMetaData(cf.ksName));
+        Assert.assertEquals(Schema.instance.getKSMetaData(cf.ksName), ksm);
+        Assert.assertNotNull(Schema.instance.getCFMetaData(cf.ksName, cf.cfName));
+
+        // updating certain fields should fail.
+        CFMetaData newCfm = cf.copy();
+        newCfm.defaultValidator(BytesType.instance);
+        newCfm.minCompactionThreshold(5);
+        newCfm.maxCompactionThreshold(31);
+
+        // test valid operations.
+        newCfm.comment("Modified comment");
+        MigrationManager.announceColumnFamilyUpdate(newCfm); // doesn't get set back here.
+
+        newCfm.readRepairChance(0.23);
+        MigrationManager.announceColumnFamilyUpdate(newCfm);
+
+        newCfm.gcGraceSeconds(12);
+        MigrationManager.announceColumnFamilyUpdate(newCfm);
+
+        newCfm.defaultValidator(UTF8Type.instance);
+        MigrationManager.announceColumnFamilyUpdate(newCfm);
+
+        newCfm.minCompactionThreshold(3);
+        MigrationManager.announceColumnFamilyUpdate(newCfm);
+
+        newCfm.maxCompactionThreshold(33);
+        MigrationManager.announceColumnFamilyUpdate(newCfm);
+
+        // can't test changing the reconciler because there is only one impl.
+
+        // check the cumulative affect.
+        Assert.assertEquals(Schema.instance.getCFMetaData(cf.ksName, cf.cfName).getComment(), newCfm.getComment());
+        Assert.assertEquals(Schema.instance.getCFMetaData(cf.ksName, cf.cfName).getReadRepairChance(), newCfm.getReadRepairChance(), 0.0001);
+        Assert.assertEquals(Schema.instance.getCFMetaData(cf.ksName, cf.cfName).getGcGraceSeconds(), newCfm.getGcGraceSeconds());
+        Assert.assertEquals(UTF8Type.instance, Schema.instance.getCFMetaData(cf.ksName, cf.cfName).getDefaultValidator());
+
+        // Change cfId
+        newCfm = new CFMetaData(cf.ksName, cf.cfName, cf.cfType, cf.comparator);
+        CFMetaData.copyOpts(newCfm, cf);
+        try
+        {
+            cf.apply(newCfm);
+            throw new AssertionError("Should have blown up when you used a different id.");
+        }
+        catch (ConfigurationException expected) {}
+
+        // Change cfName
+        newCfm = new CFMetaData(cf.ksName, cf.cfName + "_renamed", cf.cfType, cf.comparator);
+        CFMetaData.copyOpts(newCfm, cf);
+        try
+        {
+            cf.apply(newCfm);
+            throw new AssertionError("Should have blown up when you used a different name.");
+        }
+        catch (ConfigurationException expected) {}
+
+        // Change ksName
+        newCfm = new CFMetaData(cf.ksName + "_renamed", cf.cfName, cf.cfType, cf.comparator);
+        CFMetaData.copyOpts(newCfm, cf);
+        try
+        {
+            cf.apply(newCfm);
+            throw new AssertionError("Should have blown up when you used a different keyspace.");
+        }
+        catch (ConfigurationException expected) {}
+
+        // Change cf type
+        newCfm = new CFMetaData(cf.ksName, cf.cfName, ColumnFamilyType.Super, cf.comparator);
+        CFMetaData.copyOpts(newCfm, cf);
+        try
+        {
+            cf.apply(newCfm);
+            throw new AssertionError("Should have blwon up when you used a different cf type.");
+        }
+        catch (ConfigurationException expected) {}
+
+        // Change comparator
+        newCfm = new CFMetaData(cf.ksName, cf.cfName, cf.cfType, new SimpleDenseCellNameType(TimeUUIDType.instance));
+        CFMetaData.copyOpts(newCfm, cf);
+        try
+        {
+            cf.apply(newCfm);
+            throw new AssertionError("Should have blown up when you used a different comparator.");
+        }
+        catch (ConfigurationException expected) {}
+    }
+
+    @Test
+    public void testDropIndex() throws ConfigurationException
+    {
+        // persist keyspace definition in the system keyspace
+        LegacySchemaTables.makeCreateKeyspaceMutation(Schema.instance.getKSMetaData(KEYSPACE6), FBUtilities.timestampMicros()).applyUnsafe();
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE6).getColumnFamilyStore("Indexed1");
+
+        // insert some data.  save the sstable descriptor so we can make sure it's marked for delete after the drop
+        Mutation rm = new Mutation(KEYSPACE6, ByteBufferUtil.bytes("k1"));
+        rm.add("Indexed1", cellname("notbirthdate"), ByteBufferUtil.bytes(1L), 0);
+        rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(1L), 0);
+        rm.applyUnsafe();
+        cfs.forceBlockingFlush();
+        ColumnFamilyStore indexedCfs = cfs.indexManager.getIndexForColumn(ByteBufferUtil.bytes("birthdate")).getIndexCfs();
+        Descriptor desc = indexedCfs.getSSTables().iterator().next().descriptor;
+
+        // drop the index
+        CFMetaData meta = cfs.metadata.copy();
+        ColumnDefinition cdOld = meta.regularColumns().iterator().next();
+        ColumnDefinition cdNew = ColumnDefinition.regularDef(meta, cdOld.name.bytes, cdOld.type, null);
+        meta.addOrReplaceColumnDefinition(cdNew);
+        MigrationManager.announceColumnFamilyUpdate(meta);
+
+        // check
+        Assert.assertTrue(cfs.indexManager.getIndexes().isEmpty());
+        SSTableDeletingTask.waitForDeletions();
+        Assert.assertFalse(new File(desc.filenameFor(Component.DATA)).exists());
+    }
+
+    private CFMetaData addTestCF(String ks, String cf, String comment)
+    {
+        CFMetaData newCFMD = new CFMetaData(ks, cf, ColumnFamilyType.Standard, new SimpleDenseCellNameType(UTF8Type.instance));
+        newCFMD.comment(comment)
+               .readRepairChance(0.0);
+
+        return newCFMD;
+    }
+}
diff --git a/test/unit/org/apache/cassandra/security/SSLFactoryTest.java b/test/unit/org/apache/cassandra/security/SSLFactoryTest.java
new file mode 100644
index 0000000..b3510bd
--- /dev/null
+++ b/test/unit/org/apache/cassandra/security/SSLFactoryTest.java
@@ -0,0 +1,75 @@
+/*
+* 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.security;
+
+import static org.junit.Assert.assertArrayEquals;
+
+import java.io.IOException;
+import java.net.InetAddress;
+
+import javax.net.ssl.SSLServerSocket;
+
+import com.google.common.base.Predicates;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.config.EncryptionOptions;
+import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions;
+import org.junit.Test;
+
+public class SSLFactoryTest
+{
+
+    @Test
+    public void testFilterCipherSuites()
+    {
+        String[] supported = new String[] {"x", "b", "c", "f"};
+        String[] desired = new String[] { "k", "a", "b", "c" };
+        assertArrayEquals(new String[] { "b", "c" }, SSLFactory.filterCipherSuites(supported, desired));
+
+        desired = new String[] { "c", "b", "x" };
+        assertArrayEquals(desired, SSLFactory.filterCipherSuites(supported, desired));
+    }
+
+    @Test
+    public void testServerSocketCiphers() throws IOException
+    {
+        ServerEncryptionOptions options = new EncryptionOptions.ServerEncryptionOptions();
+        options.keystore = "test/conf/keystore.jks";
+        options.keystore_password = "cassandra";
+        options.truststore = options.keystore;
+        options.truststore_password = options.keystore_password;
+        options.cipher_suites = new String[] {
+            "TLS_RSA_WITH_AES_128_CBC_SHA", "TLS_RSA_WITH_AES_256_CBC_SHA",
+            "TLS_DHE_RSA_WITH_AES_128_CBC_SHA", "TLS_DHE_RSA_WITH_AES_256_CBC_SHA",
+            "TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA", "TLS_ECDHE_RSA_WITH_AES_256_CBC_SHA"
+        };
+
+        // enabled ciphers must be a subset of configured ciphers with identical order
+        try (SSLServerSocket socket = SSLFactory.getServerSocket(options, InetAddress.getLocalHost(), 55123))
+        {
+            String[] enabled = socket.getEnabledCipherSuites();
+            String[] wanted = Iterables.toArray(Iterables.filter(Lists.newArrayList(options.cipher_suites),
+                                                                 Predicates.in(Lists.newArrayList(enabled))),
+                                                String.class);
+            assertArrayEquals(wanted, enabled);
+        }
+    }
+
+}
diff --git a/test/unit/org/apache/cassandra/serializers/SimpleDateSerializerTest.java b/test/unit/org/apache/cassandra/serializers/SimpleDateSerializerTest.java
new file mode 100644
index 0000000..4c0751f
--- /dev/null
+++ b/test/unit/org/apache/cassandra/serializers/SimpleDateSerializerTest.java
@@ -0,0 +1,155 @@
+/**
+ * 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.serializers;
+
+import org.apache.cassandra.db.marshal.SimpleDateType;
+import org.apache.cassandra.utils.Pair;
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+import java.sql.Timestamp;
+import java.text.SimpleDateFormat;
+import java.util.*;
+
+public class SimpleDateSerializerTest
+{
+    private static final long millisPerDay = 1000 * 60 * 60 * 24;
+
+    private String dates[] = new String[]
+    {
+            "1970-01-01",
+            "1970-01-02",
+            "1969-12-31",
+            "-0001-01-02",
+            "-5877521-01-02",
+            "2014-01-01",
+            "5881580-01-10",
+            "1920-12-01",
+            "1582-10-19"
+    };
+
+    private static GregorianCalendar testCalendar = new GregorianCalendar();
+    private static SimpleDateFormat dateFormatUTC = new SimpleDateFormat("yyyy-MM-dd");
+
+    {
+        testCalendar.setGregorianChange(new Date(Long.MIN_VALUE));
+        testCalendar.setTimeZone(TimeZone.getTimeZone("UTC"));
+        dateFormatUTC.setCalendar(testCalendar);
+        dateFormatUTC.setLenient(false);
+    }
+
+    @Test
+    public void testDateStringToTimestamp()
+    {
+        List<String> unparsedDates = new ArrayList<>();
+        List<String> badParseResults = new ArrayList<>();
+        for (String date : dates)
+        {
+            try
+            {
+                Integer days = SimpleDateSerializer.dateStringToDays(date);
+                ByteBuffer value = SimpleDateSerializer.instance.serialize(days);
+                Integer deserialized = SimpleDateSerializer.instance.deserialize(value);
+
+                String toStringValue = SimpleDateSerializer.instance.toString(deserialized);
+                if (!date.equals(toStringValue)) {
+                    badParseResults.add(String.format("Failed to parse date correctly.  Expected %s, got %s\n", date, toStringValue));
+                }
+            }
+            catch (MarshalException e)
+            {
+                System.err.println("Got an exception: " + e);
+                unparsedDates.add(date);
+            }
+        }
+        assert unparsedDates.isEmpty() : "Unable to parse: " + unparsedDates;
+        assert badParseResults.isEmpty() : "Incorrect parse results: " + badParseResults;
+    }
+
+    @Test
+    public void testDaysStringToInt()
+    {
+        Integer value = SimpleDateSerializer.dateStringToDays("12345");
+        assert value.compareTo(12345) == 0 : String.format("Failed to parse integer based date.  Expected %s, got %s",
+                12345,
+                value);
+    }
+
+    @Test
+    public void testProlepticRange()
+    {
+        for (int i = 1; i < 31; ++i)
+        {
+            String date = "1582-10-";
+            if (i < 10) date += "0";
+            date += i;
+
+            Integer days = SimpleDateSerializer.dateStringToDays(date);
+
+            ByteBuffer value = SimpleDateType.instance.fromString(days.toString());
+            Integer deserialized = SimpleDateSerializer.instance.deserialize(value);
+
+            // Serialized values are unsigned int, unwrap bits w/overflow
+            deserialized -= Integer.MIN_VALUE;
+
+            Timestamp ts = new Timestamp(deserialized * millisPerDay);
+            testCalendar.setTime(ts);
+
+            Date newDate = testCalendar.getTime();
+            assert (dateFormatUTC.format(newDate)).equals(date) :
+                    String.format("Expected [%s], got [%s]", date, dateFormatUTC.format(newDate).toString());
+        }
+    }
+
+    @Test (expected=MarshalException.class)
+    public void testOutOfBoundsLow()
+    {
+        Integer days = SimpleDateSerializer.dateStringToDays(new Date(Integer.MIN_VALUE * millisPerDay - millisPerDay).toString());
+    }
+
+    @Test (expected=MarshalException.class)
+    public void testOutOfBoundsHigh()
+    {
+        Integer days = SimpleDateSerializer.dateStringToDays(new Date(Integer.MAX_VALUE * millisPerDay + millisPerDay).toString());
+    }
+
+    @Test (expected=MarshalException.class)
+    public void testBadInput()
+    {
+        Integer days = SimpleDateSerializer.dateStringToDays("12A-01-01");
+    }
+
+    @Test (expected=MarshalException.class)
+    public void testBadMonth()
+    {
+        Integer days = SimpleDateSerializer.dateStringToDays("1000-13-01");
+    }
+
+    @Test (expected=MarshalException.class)
+    public void testBadDay()
+    {
+        Integer days = SimpleDateSerializer.dateStringToDays("1000-12-32");
+    }
+
+    @Test (expected=MarshalException.class)
+    public void testBadDayToMonth()
+    {
+        Integer days = SimpleDateSerializer.dateStringToDays("1000-09-31");
+    }
+}
diff --git a/test/unit/org/apache/cassandra/serializers/TimeSerializerTest.java b/test/unit/org/apache/cassandra/serializers/TimeSerializerTest.java
new file mode 100644
index 0000000..f8af48c
--- /dev/null
+++ b/test/unit/org/apache/cassandra/serializers/TimeSerializerTest.java
@@ -0,0 +1,242 @@
+/**
+ * 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.serializers;
+
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.TimeUnit;
+
+public class TimeSerializerTest
+{
+    @Test
+    public void testSerializerFromString()
+    {
+        // nano
+        long expected = 5;
+        Long time = TimeSerializer.timeStringToLong("00:00:00.000000005");
+        assert time == expected : String.format("Failed nano conversion.  Expected %s, got %s", expected, time);
+
+        // usec
+        expected = TimeUnit.MICROSECONDS.toNanos(123);
+        time = TimeSerializer.timeStringToLong("00:00:00.000123000");
+        assert time == expected : String.format("Failed usec conversion.  Expected %s, got %s", expected, time);
+
+        // milli
+        expected = TimeUnit.MILLISECONDS.toNanos(123);
+        time = TimeSerializer.timeStringToLong("00:00:00.123000");
+        assert time == expected : String.format("Failed milli conversion.  Expected %s, got %s", expected, time);
+
+        // sec
+        expected = TimeUnit.SECONDS.toNanos(15);
+        time = TimeSerializer.timeStringToLong("00:00:15.000");
+        assert time == expected : String.format("Failed sec conversion.  Expected %s, got %s", expected, time);
+
+        // min
+        expected = TimeUnit.MINUTES.toNanos(13);
+        time = TimeSerializer.timeStringToLong("00:13:00.000");
+        assert time == expected : String.format("Failed min conversion.  Expected %s, got %s", expected, time);
+
+        // hour
+        expected = TimeUnit.HOURS.toNanos(2);
+        time = TimeSerializer.timeStringToLong("02:0:00.000");
+        assert time == expected : String.format("Failed min conversion.  Expected %s, got %s", expected, time);
+
+        // complex
+        expected = buildExpected(4, 31, 12, 123, 456, 789);
+        time = TimeSerializer.timeStringToLong("4:31:12.123456789");
+        assert time == expected : String.format("Failed complex conversion.  Expected %s, got %s", expected, time);
+
+        // upper bound
+        expected = buildExpected(23, 59, 59, 999, 999, 999);
+        time = TimeSerializer.timeStringToLong("23:59:59.999999999");
+        assert time == expected : String.format("Failed upper bounds conversion.  Expected %s, got %s", expected, time);
+
+        // Test partial nano
+        expected = buildExpected(12, 13, 14, 123, 654, 120);
+        time = TimeSerializer.timeStringToLong("12:13:14.12365412");
+        assert time == expected : String.format("Failed partial nano timestring.  Expected %s, got %s", expected, time);
+
+        // Test raw long value
+        expected = 10;
+        time = TimeSerializer.timeStringToLong("10");
+        assert time == expected : String.format("Failed long conversion.  Expected %s, got %s", expected, time);
+
+        // Test 0 long
+        expected = 0;
+        time = TimeSerializer.timeStringToLong("0");
+        assert time == expected : String.format("Failed long conversion.  Expected %s, got %s", expected, time);
+    }
+
+    private long buildExpected(int hour, int minute, int second, int milli, int micro, int nano)
+    {
+        return  TimeUnit.HOURS.toNanos(hour) +
+                TimeUnit.MINUTES.toNanos(minute) +
+                TimeUnit.SECONDS.toNanos(second) +
+                TimeUnit.MILLISECONDS.toNanos(milli) +
+                TimeUnit.MICROSECONDS.toNanos(micro) +
+                nano;
+    }
+
+    @Test
+    public void testSerializerToString()
+    {
+        String source = "00:00:00.000000011";
+        Long time = TimeSerializer.timeStringToLong(source);
+        assert(source.equals(TimeSerializer.instance.toString(time)));
+
+        source = "00:00:00.000012311";
+        time = TimeSerializer.timeStringToLong(source);
+        assert(source.equals(TimeSerializer.instance.toString(time)));
+
+        source = "00:00:00.123000000";
+        time = TimeSerializer.timeStringToLong(source);
+        assert(source.equals(TimeSerializer.instance.toString(time)));
+
+        source = "00:00:12.123450000";
+        time = TimeSerializer.timeStringToLong(source);
+        assert(source.equals(TimeSerializer.instance.toString(time)));
+
+        source = "00:34:12.123450000";
+        time = TimeSerializer.timeStringToLong(source);
+        assert(source.equals(TimeSerializer.instance.toString(time)));
+
+        source = "15:00:12.123450000";
+        time = TimeSerializer.timeStringToLong(source);
+        assert(source.equals(TimeSerializer.instance.toString(time)));
+
+        // boundaries
+        source = "00:00:00.000000000";
+        time = TimeSerializer.timeStringToLong(source);
+        assert(source.equals(TimeSerializer.instance.toString(time)));
+
+        source = "23:59:59.999999999";
+        time = TimeSerializer.timeStringToLong(source);
+        assert(source.equals(TimeSerializer.instance.toString(time)));
+
+        // truncated
+        source = "01:14:18.12";
+        time = TimeSerializer.timeStringToLong(source);
+        String result = TimeSerializer.instance.toString(time);
+        assert(result.equals("01:14:18.120000000"));
+
+        source = "01:14:18.1201";
+        time = TimeSerializer.timeStringToLong(source);
+        result = TimeSerializer.instance.toString(time);
+        assert(result.equals("01:14:18.120100000"));
+
+        source = "01:14:18.1201098";
+        time = TimeSerializer.timeStringToLong(source);
+        result = TimeSerializer.instance.toString(time);
+        assert(result.equals("01:14:18.120109800"));
+    }
+
+    @Test public void testSerialization()
+    {
+        String source = "01:01:01.123123123";
+        Long nt = TimeSerializer.timeStringToLong(source);
+
+        ByteBuffer buf = TimeSerializer.instance.serialize(nt);
+        TimeSerializer.instance.validate(buf);
+
+        Long result = TimeSerializer.instance.deserialize(buf);
+        String strResult = TimeSerializer.instance.toString(result);
+
+        assert(strResult.equals(source));
+    }
+
+    @Test (expected=MarshalException.class)
+    public void testBadHourLow()
+    {
+        Long time = TimeSerializer.timeStringToLong("-1:0:0.123456789");
+    }
+
+    @Test (expected=MarshalException.class)
+    public void testBadHourHigh()
+    {
+        Long time = TimeSerializer.timeStringToLong("24:0:0.123456789");
+    }
+
+    @Test (expected=MarshalException.class)
+    public void testBadMinuteLow()
+    {
+        Long time = TimeSerializer.timeStringToLong("23:-1:0.123456789");
+    }
+
+    @Test (expected=MarshalException.class)
+    public void testBadMinuteHigh()
+    {
+        Long time = TimeSerializer.timeStringToLong("23:60:0.123456789");
+    }
+
+    @Test (expected=MarshalException.class)
+    public void testEmpty()
+    {
+        Long time = TimeSerializer.timeStringToLong("");
+    }
+
+    @Test (expected=MarshalException.class)
+    public void testBadSecondLow()
+    {
+        Long time = TimeSerializer.timeStringToLong("23:59:-1.123456789");
+    }
+
+    @Test (expected=MarshalException.class)
+    public void testBadSecondHigh()
+    {
+        Long time = TimeSerializer.timeStringToLong("23:59:60.123456789");
+    }
+
+    @Test (expected=MarshalException.class)
+    public void testBadSecondHighNoMilli()
+    {
+        Long time = TimeSerializer.timeStringToLong("23:59:60");
+    }
+
+    @Test (expected=MarshalException.class)
+    public void testBadNanoLow()
+    {
+        Long time = TimeSerializer.timeStringToLong("23:59:59.-123456789");
+    }
+
+    @Test (expected=MarshalException.class)
+    public void testBadNanoHigh()
+    {
+        Long time = TimeSerializer.timeStringToLong("23:59:59.1234567899");
+    }
+
+    @Test (expected=MarshalException.class)
+    public void testBadNanoCharacter()
+    {
+        Long time = TimeSerializer.timeStringToLong("23:59:59.12345A789");
+    }
+
+    @Test (expected=MarshalException.class)
+    public void testNegativeLongTime()
+    {
+        Long time = TimeSerializer.timeStringToLong("-10");
+    }
+
+    @Test (expected=MarshalException.class)
+    public void testRawLongOverflow()
+    {
+        Long input = TimeUnit.DAYS.toNanos(1) + 1;
+        Long time = TimeSerializer.timeStringToLong(input.toString());
+    }
+}
diff --git a/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java b/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java
index cf64322..03a25c6 100644
--- a/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java
+++ b/test/unit/org/apache/cassandra/service/ActiveRepairServiceTest.java
@@ -1,41 +1,49 @@
 /*
- * 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.
- */
-
+* 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.service;
 
+import java.net.InetAddress;
+import java.util.*;
+import java.util.concurrent.ExecutionException;
 
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Set;
-import java.util.UUID;
-
+import com.google.common.base.Predicate;
 import com.google.common.collect.Sets;
-
+import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.Mutation;
-import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.locator.AbstractReplicationStrategy;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.locator.TokenMetadata;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.concurrent.Refs;
@@ -44,11 +52,182 @@
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-public class ActiveRepairServiceTest extends SchemaLoader
+public class ActiveRepairServiceTest
 {
+    public static final String KEYSPACE5 = "Keyspace5";
+    public static final String CF_STANDARD1 = "Standard1";
+    public static final String CF_COUNTER = "Counter1";
 
-    private static final String KEYSPACE1 = "Keyspace1";
-    private static final String CF = "Standard1";
+    public String cfname;
+    public ColumnFamilyStore store;
+    public InetAddress LOCAL, REMOTE;
+
+    private boolean initialized;
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE5,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(2),
+                                    SchemaLoader.standardCFMD(KEYSPACE5, CF_COUNTER),
+                                    SchemaLoader.standardCFMD(KEYSPACE5, CF_STANDARD1));
+    }
+
+    @Before
+    public void prepare() throws Exception
+    {
+        if (!initialized)
+        {
+            SchemaLoader.startGossiper();
+            initialized = true;
+
+            LOCAL = FBUtilities.getBroadcastAddress();
+            // generate a fake endpoint for which we can spoof receiving/sending trees
+            REMOTE = InetAddress.getByName("127.0.0.2");
+        }
+
+        TokenMetadata tmd = StorageService.instance.getTokenMetadata();
+        tmd.clearUnsafe();
+        StorageService.instance.setTokens(Collections.singleton(StorageService.getPartitioner().getRandomToken()));
+        tmd.updateNormalToken(StorageService.getPartitioner().getMinimumToken(), REMOTE);
+        assert tmd.isMember(REMOTE);
+    }
+
+    @Test
+    public void testGetNeighborsPlusOne() throws Throwable
+    {
+        // generate rf+1 nodes, and ensure that all nodes are returned
+        Set<InetAddress> expected = addTokens(1 + Keyspace.open(KEYSPACE5).getReplicationStrategy().getReplicationFactor());
+        expected.remove(FBUtilities.getBroadcastAddress());
+        Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(KEYSPACE5);
+        Set<InetAddress> neighbors = new HashSet<>();
+        for (Range<Token> range : ranges)
+        {
+            neighbors.addAll(ActiveRepairService.getNeighbors(KEYSPACE5, ranges, range, null, null));
+        }
+        assertEquals(expected, neighbors);
+    }
+
+    @Test
+    public void testGetNeighborsTimesTwo() throws Throwable
+    {
+        TokenMetadata tmd = StorageService.instance.getTokenMetadata();
+
+        // generate rf*2 nodes, and ensure that only neighbors specified by the ARS are returned
+        addTokens(2 * Keyspace.open(KEYSPACE5).getReplicationStrategy().getReplicationFactor());
+        AbstractReplicationStrategy ars = Keyspace.open(KEYSPACE5).getReplicationStrategy();
+        Set<InetAddress> expected = new HashSet<>();
+        for (Range<Token> replicaRange : ars.getAddressRanges().get(FBUtilities.getBroadcastAddress()))
+        {
+            expected.addAll(ars.getRangeAddresses(tmd.cloneOnlyTokenMap()).get(replicaRange));
+        }
+        expected.remove(FBUtilities.getBroadcastAddress());
+        Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(KEYSPACE5);
+        Set<InetAddress> neighbors = new HashSet<>();
+        for (Range<Token> range : ranges)
+        {
+            neighbors.addAll(ActiveRepairService.getNeighbors(KEYSPACE5, ranges, range, null, null));
+        }
+        assertEquals(expected, neighbors);
+    }
+
+    @Test
+    public void testGetNeighborsPlusOneInLocalDC() throws Throwable
+    {
+        TokenMetadata tmd = StorageService.instance.getTokenMetadata();
+
+        // generate rf+1 nodes, and ensure that all nodes are returned
+        Set<InetAddress> expected = addTokens(1 + Keyspace.open(KEYSPACE5).getReplicationStrategy().getReplicationFactor());
+        expected.remove(FBUtilities.getBroadcastAddress());
+        // remove remote endpoints
+        TokenMetadata.Topology topology = tmd.cloneOnlyTokenMap().getTopology();
+        HashSet<InetAddress> localEndpoints = Sets.newHashSet(topology.getDatacenterEndpoints().get(DatabaseDescriptor.getLocalDataCenter()));
+        expected = Sets.intersection(expected, localEndpoints);
+
+        Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(KEYSPACE5);
+        Set<InetAddress> neighbors = new HashSet<>();
+        for (Range<Token> range : ranges)
+        {
+            neighbors.addAll(ActiveRepairService.getNeighbors(KEYSPACE5, ranges, range, Arrays.asList(DatabaseDescriptor.getLocalDataCenter()), null));
+        }
+        assertEquals(expected, neighbors);
+    }
+
+    @Test
+    public void testGetNeighborsTimesTwoInLocalDC() throws Throwable
+    {
+        TokenMetadata tmd = StorageService.instance.getTokenMetadata();
+
+        // generate rf*2 nodes, and ensure that only neighbors specified by the ARS are returned
+        addTokens(2 * Keyspace.open(KEYSPACE5).getReplicationStrategy().getReplicationFactor());
+        AbstractReplicationStrategy ars = Keyspace.open(KEYSPACE5).getReplicationStrategy();
+        Set<InetAddress> expected = new HashSet<>();
+        for (Range<Token> replicaRange : ars.getAddressRanges().get(FBUtilities.getBroadcastAddress()))
+        {
+            expected.addAll(ars.getRangeAddresses(tmd.cloneOnlyTokenMap()).get(replicaRange));
+        }
+        expected.remove(FBUtilities.getBroadcastAddress());
+        // remove remote endpoints
+        TokenMetadata.Topology topology = tmd.cloneOnlyTokenMap().getTopology();
+        HashSet<InetAddress> localEndpoints = Sets.newHashSet(topology.getDatacenterEndpoints().get(DatabaseDescriptor.getLocalDataCenter()));
+        expected = Sets.intersection(expected, localEndpoints);
+
+        Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(KEYSPACE5);
+        Set<InetAddress> neighbors = new HashSet<>();
+        for (Range<Token> range : ranges)
+        {
+            neighbors.addAll(ActiveRepairService.getNeighbors(KEYSPACE5, ranges, range, Arrays.asList(DatabaseDescriptor.getLocalDataCenter()), null));
+        }
+        assertEquals(expected, neighbors);
+    }
+
+    @Test
+    public void testGetNeighborsTimesTwoInSpecifiedHosts() throws Throwable
+    {
+        TokenMetadata tmd = StorageService.instance.getTokenMetadata();
+
+        // generate rf*2 nodes, and ensure that only neighbors specified by the hosts are returned
+        addTokens(2 * Keyspace.open(KEYSPACE5).getReplicationStrategy().getReplicationFactor());
+        AbstractReplicationStrategy ars = Keyspace.open(KEYSPACE5).getReplicationStrategy();
+        List<InetAddress> expected = new ArrayList<>();
+        for (Range<Token> replicaRange : ars.getAddressRanges().get(FBUtilities.getBroadcastAddress()))
+        {
+            expected.addAll(ars.getRangeAddresses(tmd.cloneOnlyTokenMap()).get(replicaRange));
+        }
+
+        expected.remove(FBUtilities.getBroadcastAddress());
+        Collection<String> hosts = Arrays.asList(FBUtilities.getBroadcastAddress().getCanonicalHostName(),expected.get(0).getCanonicalHostName());
+        Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(KEYSPACE5);
+
+        assertEquals(expected.get(0), ActiveRepairService.getNeighbors(KEYSPACE5, ranges,
+                                                                       ranges.iterator().next(),
+                                                                       null, hosts).iterator().next());
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void testGetNeighborsSpecifiedHostsWithNoLocalHost() throws Throwable
+    {
+        addTokens(2 * Keyspace.open(KEYSPACE5).getReplicationStrategy().getReplicationFactor());
+        //Dont give local endpoint
+        Collection<String> hosts = Arrays.asList("127.0.0.3");
+        Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(KEYSPACE5);
+        ActiveRepairService.getNeighbors(KEYSPACE5, ranges, ranges.iterator().next(), null, hosts);
+    }
+
+    Set<InetAddress> addTokens(int max) throws Throwable
+    {
+        TokenMetadata tmd = StorageService.instance.getTokenMetadata();
+        Set<InetAddress> endpoints = new HashSet<>();
+        for (int i = 1; i <= max; i++)
+        {
+            InetAddress endpoint = InetAddress.getByName("127.0.0." + i);
+            tmd.updateNormalToken(StorageService.getPartitioner().getRandomToken(), endpoint);
+            endpoints.add(endpoint);
+        }
+        return endpoints;
+    }
 
     @Test
     public void testGetActiveRepairedSSTableRefs()
@@ -57,11 +236,12 @@
         Set<SSTableReader> original = store.getUnrepairedSSTables();
 
         UUID prsId = UUID.randomUUID();
-        ActiveRepairService.instance.registerParentRepairSession(prsId, FBUtilities.getBroadcastAddress(), Collections.singletonList(store), null);
+        ActiveRepairService.instance.registerParentRepairSession(prsId, FBUtilities.getBroadcastAddress(), Collections.singletonList(store), null, true, false);
         ActiveRepairService.ParentRepairSession prs = ActiveRepairService.instance.getParentRepairSession(prsId);
         prs.markSSTablesRepairing(store.metadata.cfId, prsId);
+
         //retrieve all sstable references from parent repair sessions
-        Refs<SSTableReader> refs = prs.getActiveRepairedSSTableRefsForAntiCompaction(store.metadata.cfId);
+        Refs<SSTableReader> refs = prs.getActiveRepairedSSTableRefsForAntiCompaction(store.metadata.cfId, prsId);
         Set<SSTableReader> retrieved = Sets.newHashSet(refs.iterator());
         assertEquals(original, retrieved);
         refs.release();
@@ -69,12 +249,18 @@
         //remove 1 sstable from data data tracker
         Set<SSTableReader> newLiveSet = new HashSet<>(original);
         Iterator<SSTableReader> it = newLiveSet.iterator();
-        SSTableReader removed = it.next();
+        final SSTableReader removed = it.next();
         it.remove();
-        store.getDataTracker().replaceWithNewInstances(Collections.singleton(removed), Collections.EMPTY_SET);
+        store.getTracker().dropSSTables(new Predicate<SSTableReader>()
+        {
+            public boolean apply(SSTableReader reader)
+            {
+                return removed.equals(reader);
+            }
+        }, OperationType.COMPACTION, null);
 
         //retrieve sstable references from parent repair session again - removed sstable must not be present
-        refs = prs.getActiveRepairedSSTableRefsForAntiCompaction(store.metadata.cfId);
+        refs = prs.getActiveRepairedSSTableRefsForAntiCompaction(store.metadata.cfId, prsId);
         retrieved = Sets.newHashSet(refs.iterator());
         assertEquals(newLiveSet, retrieved);
         assertFalse(retrieved.contains(removed));
@@ -87,10 +273,10 @@
         ColumnFamilyStore store = prepareColumnFamilyStore();
         Set<SSTableReader> original = store.getUnrepairedSSTables();
         UUID prsId = UUID.randomUUID();
-        ActiveRepairService.instance.registerParentRepairSession(prsId, FBUtilities.getBroadcastAddress(), Collections.singletonList(store), null);
+        ActiveRepairService.instance.registerParentRepairSession(prsId, FBUtilities.getBroadcastAddress(), Collections.singletonList(store), null, true, true);
         ActiveRepairService.ParentRepairSession prs = ActiveRepairService.instance.getParentRepairSession(prsId);
         prs.markSSTablesRepairing(store.metadata.cfId, prsId);
-        try (Refs<SSTableReader> refs = prs.getActiveRepairedSSTableRefsForAntiCompaction(store.metadata.cfId))
+        try (Refs<SSTableReader> refs = prs.getActiveRepairedSSTableRefsForAntiCompaction(store.metadata.cfId, prsId))
         {
             Set<SSTableReader> retrieved = Sets.newHashSet(refs.iterator());
             assertEquals(original, retrieved);
@@ -100,7 +286,7 @@
         try
         {
             UUID newPrsId = UUID.randomUUID();
-            ActiveRepairService.instance.registerParentRepairSession(newPrsId, FBUtilities.getBroadcastAddress(), Collections.singletonList(store), null);
+            ActiveRepairService.instance.registerParentRepairSession(newPrsId, FBUtilities.getBroadcastAddress(), Collections.singletonList(store), null, true, true);
             ActiveRepairService.instance.getParentRepairSession(newPrsId).markSSTablesRepairing(store.metadata.cfId, newPrsId);
         }
         catch (Throwable t)
@@ -109,17 +295,69 @@
         }
         assertTrue(exception);
 
-        try (Refs<SSTableReader> refs = prs.getActiveRepairedSSTableRefsForAntiCompaction(store.metadata.cfId))
+        try (Refs<SSTableReader> refs = prs.getActiveRepairedSSTableRefsForAntiCompaction(store.metadata.cfId, prsId))
         {
             Set<SSTableReader> retrieved = Sets.newHashSet(refs.iterator());
             assertEquals(original, retrieved);
         }
     }
 
+    @Test
+    public void testSnapshotAddSSTables() throws ExecutionException, InterruptedException
+    {
+        ColumnFamilyStore store = prepareColumnFamilyStore();
+        UUID prsId = UUID.randomUUID();
+        Set<SSTableReader> original = store.getUnrepairedSSTables();
+        ActiveRepairService.instance.registerParentRepairSession(prsId, FBUtilities.getBroadcastAddress(), Collections.singletonList(store), Collections.singleton(new Range<>(store.partitioner.getMinimumToken(), store.partitioner.getMinimumToken())), true, true);
+        ActiveRepairService.instance.getParentRepairSession(prsId).maybeSnapshot(store.metadata.cfId, prsId);
+
+        UUID prsId2 = UUID.randomUUID();
+        ActiveRepairService.instance.registerParentRepairSession(prsId2, FBUtilities.getBroadcastAddress(), Collections.singletonList(store), Collections.singleton(new Range<>(store.partitioner.getMinimumToken(), store.partitioner.getMinimumToken())), true, true);
+        createSSTables(store, 2);
+        ActiveRepairService.instance.getParentRepairSession(prsId).maybeSnapshot(store.metadata.cfId, prsId);
+        try (Refs<SSTableReader> refs = ActiveRepairService.instance.getParentRepairSession(prsId).getActiveRepairedSSTableRefsForAntiCompaction(store.metadata.cfId, prsId))
+        {
+            assertEquals(original, Sets.newHashSet(refs.iterator()));
+        }
+        store.forceMajorCompaction();
+        // after a major compaction the original sstables will be gone and we will have no sstables to anticompact:
+        try (Refs<SSTableReader> refs = ActiveRepairService.instance.getParentRepairSession(prsId).getActiveRepairedSSTableRefsForAntiCompaction(store.metadata.cfId, prsId))
+        {
+            assertEquals(0, refs.size());
+        }
+    }
+
+    @Test
+    public void testSnapshotMultipleRepairs()
+    {
+        ColumnFamilyStore store = prepareColumnFamilyStore();
+        Set<SSTableReader> original = store.getUnrepairedSSTables();
+        UUID prsId = UUID.randomUUID();
+        ActiveRepairService.instance.registerParentRepairSession(prsId, FBUtilities.getBroadcastAddress(), Collections.singletonList(store), Collections.singleton(new Range<>(store.partitioner.getMinimumToken(), store.partitioner.getMinimumToken())), true, true);
+        ActiveRepairService.instance.getParentRepairSession(prsId).maybeSnapshot(store.metadata.cfId, prsId);
+
+        UUID prsId2 = UUID.randomUUID();
+        ActiveRepairService.instance.registerParentRepairSession(prsId2, FBUtilities.getBroadcastAddress(), Collections.singletonList(store), Collections.singleton(new Range<>(store.partitioner.getMinimumToken(), store.partitioner.getMinimumToken())), true, true);
+        boolean exception = false;
+        try
+        {
+            ActiveRepairService.instance.getParentRepairSession(prsId2).maybeSnapshot(store.metadata.cfId, prsId2);
+        }
+        catch (Throwable t)
+        {
+            exception = true;
+        }
+        assertTrue(exception);
+        try (Refs<SSTableReader> refs = ActiveRepairService.instance.getParentRepairSession(prsId).getActiveRepairedSSTableRefsForAntiCompaction(store.metadata.cfId, prsId))
+        {
+            assertEquals(original, Sets.newHashSet(refs.iterator()));
+        }
+    }
+
     private ColumnFamilyStore prepareColumnFamilyStore()
     {
-        Keyspace keyspace = Keyspace.open(KEYSPACE1);
-        ColumnFamilyStore store = keyspace.getColumnFamilyStore(CF);
+        Keyspace keyspace = Keyspace.open(KEYSPACE5);
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore(CF_STANDARD1);
         store.truncateBlocking();
         store.disableAutoCompaction();
         createSSTables(store, 10);
@@ -132,7 +370,7 @@
         for (int i = 0; i < count; i++)
         {
             DecoratedKey key = Util.dk(Integer.toString(i));
-            Mutation rm = new Mutation(KEYSPACE1, key.getKey());
+            Mutation rm = new Mutation(KEYSPACE5, key.getKey());
             for (int j = 0; j < 10; j++)
                 rm.add("Standard1", Util.cellname(Integer.toString(j)),
                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
diff --git a/test/unit/org/apache/cassandra/service/AntiEntropyServiceCounterTest.java b/test/unit/org/apache/cassandra/service/AntiEntropyServiceCounterTest.java
deleted file mode 100644
index 4a82183..0000000
--- a/test/unit/org/apache/cassandra/service/AntiEntropyServiceCounterTest.java
+++ /dev/null
@@ -1,47 +0,0 @@
-package org.apache.cassandra.service;
-/*
- *
- * 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.
- *
- */
-
-import java.util.List;
-import java.util.LinkedList;
-
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.composites.CellNames;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.db.ConsistencyLevel;
-
-public class AntiEntropyServiceCounterTest extends AntiEntropyServiceTestAbstract
-{
-    public void init()
-    {
-        keyspaceName = "Keyspace5";
-        cfname    = "Counter1";
-    }
-
-    public List<IMutation> getWriteData()
-    {
-        List<IMutation> rms = new LinkedList<IMutation>();
-        Mutation rm = new Mutation(keyspaceName, ByteBufferUtil.bytes("key1"));
-        rm.addCounter(cfname, CellNames.simpleDense(ByteBufferUtil.bytes("Column1")), 42);
-        rms.add(new CounterMutation(rm, ConsistencyLevel.ONE));
-        return rms;
-    }
-}
diff --git a/test/unit/org/apache/cassandra/service/AntiEntropyServiceStandardTest.java b/test/unit/org/apache/cassandra/service/AntiEntropyServiceStandardTest.java
deleted file mode 100644
index f4025cc..0000000
--- a/test/unit/org/apache/cassandra/service/AntiEntropyServiceStandardTest.java
+++ /dev/null
@@ -1,47 +0,0 @@
-package org.apache.cassandra.service;
-/*
- *
- * 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.
- *
- */
-
-import java.util.List;
-import java.util.LinkedList;
-
-import org.apache.cassandra.Util;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-public class AntiEntropyServiceStandardTest extends AntiEntropyServiceTestAbstract
-{
-    public void init()
-    {
-        keyspaceName = "Keyspace5";
-        cfname    = "Standard1";
-    }
-
-    public List<IMutation> getWriteData()
-    {
-        List<IMutation> rms = new LinkedList<IMutation>();
-        Mutation rm;
-        rm = new Mutation(keyspaceName, ByteBufferUtil.bytes("key1"));
-        rm.add(cfname, Util.cellname("Column1"), ByteBufferUtil.bytes("asdfasdf"), 0);
-        rms.add(rm);
-        return rms;
-    }
-}
diff --git a/test/unit/org/apache/cassandra/service/AntiEntropyServiceTestAbstract.java b/test/unit/org/apache/cassandra/service/AntiEntropyServiceTestAbstract.java
deleted file mode 100644
index 21eb492..0000000
--- a/test/unit/org/apache/cassandra/service/AntiEntropyServiceTestAbstract.java
+++ /dev/null
@@ -1,264 +0,0 @@
-/*
-* 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.service;
-
-import java.net.InetAddress;
-import java.util.*;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.TimeUnit;
-
-import com.google.common.collect.Sets;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.concurrent.Stage;
-import org.apache.cassandra.concurrent.StageManager;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.IMutation;
-import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.gms.Gossiper;
-import org.apache.cassandra.locator.AbstractReplicationStrategy;
-import org.apache.cassandra.locator.TokenMetadata;
-import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.repair.RepairJobDesc;
-import org.apache.cassandra.utils.FBUtilities;
-
-import static org.junit.Assert.assertEquals;
-
-public abstract class AntiEntropyServiceTestAbstract extends SchemaLoader
-{
-    // keyspace and column family to test against
-    public ActiveRepairService aes;
-
-    public String keyspaceName;
-    public String cfname;
-    public RepairJobDesc desc;
-    public ColumnFamilyStore store;
-    public InetAddress LOCAL, REMOTE;
-
-    public Range<Token> local_range;
-
-    private boolean initialized;
-
-    public abstract void init();
-
-    public abstract List<IMutation> getWriteData();
-
-    @Before
-    public void prepare() throws Exception
-    {
-        if (!initialized)
-        {
-            initialized = true;
-
-            init();
-
-            LOCAL = FBUtilities.getBroadcastAddress();
-            // generate a fake endpoint for which we can spoof receiving/sending trees
-            REMOTE = InetAddress.getByName("127.0.0.2");
-            store = null;
-            for (ColumnFamilyStore cfs : Keyspace.open(keyspaceName).getColumnFamilyStores())
-            {
-                if (cfs.name.equals(cfname))
-                {
-                    store = cfs;
-                    break;
-                }
-            }
-            assert store != null : "CF not found: " + cfname;
-        }
-
-        aes = ActiveRepairService.instance;
-        TokenMetadata tmd = StorageService.instance.getTokenMetadata();
-        tmd.clearUnsafe();
-        StorageService.instance.setTokens(Collections.singleton(StorageService.getPartitioner().getRandomToken()));
-        tmd.updateNormalToken(StorageService.getPartitioner().getMinimumToken(), REMOTE);
-        assert tmd.isMember(REMOTE);
-
-        MessagingService.instance().setVersion(REMOTE, MessagingService.current_version);
-        Gossiper.instance.initializeNodeUnsafe(REMOTE, UUID.randomUUID(), 1);
-
-        local_range = StorageService.instance.getPrimaryRangesForEndpoint(keyspaceName, LOCAL).iterator().next();
-
-        desc = new RepairJobDesc(UUID.randomUUID(), UUID.randomUUID(), keyspaceName, cfname, local_range);
-        // Set a fake session corresponding to this fake request
-        ActiveRepairService.instance.submitArtificialRepairSession(desc);
-    }
-
-    @After
-    public void teardown() throws Exception
-    {
-        flushAES();
-    }
-
-    @Test
-    public void testGetNeighborsPlusOne() throws Throwable
-    {
-        // generate rf+1 nodes, and ensure that all nodes are returned
-        Set<InetAddress> expected = addTokens(1 + Keyspace.open(keyspaceName).getReplicationStrategy().getReplicationFactor());
-        expected.remove(FBUtilities.getBroadcastAddress());
-        Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(keyspaceName);
-        Set<InetAddress> neighbors = new HashSet<InetAddress>();
-        for (Range<Token> range : ranges)
-        {
-            neighbors.addAll(ActiveRepairService.getNeighbors(keyspaceName, ranges, range, null, null));
-        }
-        assertEquals(expected, neighbors);
-    }
-
-    @Test
-    public void testGetNeighborsTimesTwo() throws Throwable
-    {
-        TokenMetadata tmd = StorageService.instance.getTokenMetadata();
-
-        // generate rf*2 nodes, and ensure that only neighbors specified by the ARS are returned
-        addTokens(2 * Keyspace.open(keyspaceName).getReplicationStrategy().getReplicationFactor());
-        AbstractReplicationStrategy ars = Keyspace.open(keyspaceName).getReplicationStrategy();
-        Set<InetAddress> expected = new HashSet<InetAddress>();
-        for (Range<Token> replicaRange : ars.getAddressRanges().get(FBUtilities.getBroadcastAddress()))
-        {
-            expected.addAll(ars.getRangeAddresses(tmd.cloneOnlyTokenMap()).get(replicaRange));
-        }
-        expected.remove(FBUtilities.getBroadcastAddress());
-        Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(keyspaceName);
-        Set<InetAddress> neighbors = new HashSet<InetAddress>();
-        for (Range<Token> range : ranges)
-        {
-            neighbors.addAll(ActiveRepairService.getNeighbors(keyspaceName, ranges, range, null, null));
-        }
-        assertEquals(expected, neighbors);
-    }
-
-    @Test
-    public void testGetNeighborsPlusOneInLocalDC() throws Throwable
-    {
-        TokenMetadata tmd = StorageService.instance.getTokenMetadata();
-        
-        // generate rf+1 nodes, and ensure that all nodes are returned
-        Set<InetAddress> expected = addTokens(1 + Keyspace.open(keyspaceName).getReplicationStrategy().getReplicationFactor());
-        expected.remove(FBUtilities.getBroadcastAddress());
-        // remove remote endpoints
-        TokenMetadata.Topology topology = tmd.cloneOnlyTokenMap().getTopology();
-        HashSet<InetAddress> localEndpoints = Sets.newHashSet(topology.getDatacenterEndpoints().get(DatabaseDescriptor.getLocalDataCenter()));
-        expected = Sets.intersection(expected, localEndpoints);
-
-        Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(keyspaceName);
-        Set<InetAddress> neighbors = new HashSet<InetAddress>();
-        for (Range<Token> range : ranges)
-        {
-            neighbors.addAll(ActiveRepairService.getNeighbors(keyspaceName, ranges, range, Arrays.asList(DatabaseDescriptor.getLocalDataCenter()), null));
-        }
-        assertEquals(expected, neighbors);
-    }
-
-    @Test
-    public void testGetNeighborsTimesTwoInLocalDC() throws Throwable
-    {
-        TokenMetadata tmd = StorageService.instance.getTokenMetadata();
-
-        // generate rf*2 nodes, and ensure that only neighbors specified by the ARS are returned
-        addTokens(2 * Keyspace.open(keyspaceName).getReplicationStrategy().getReplicationFactor());
-        AbstractReplicationStrategy ars = Keyspace.open(keyspaceName).getReplicationStrategy();
-        Set<InetAddress> expected = new HashSet<InetAddress>();
-        for (Range<Token> replicaRange : ars.getAddressRanges().get(FBUtilities.getBroadcastAddress()))
-        {
-            expected.addAll(ars.getRangeAddresses(tmd.cloneOnlyTokenMap()).get(replicaRange));
-        }
-        expected.remove(FBUtilities.getBroadcastAddress());
-        // remove remote endpoints
-        TokenMetadata.Topology topology = tmd.cloneOnlyTokenMap().getTopology();
-        HashSet<InetAddress> localEndpoints = Sets.newHashSet(topology.getDatacenterEndpoints().get(DatabaseDescriptor.getLocalDataCenter()));
-        expected = Sets.intersection(expected, localEndpoints);
-        
-        Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(keyspaceName);
-        Set<InetAddress> neighbors = new HashSet<InetAddress>();
-        for (Range<Token> range : ranges)
-        {
-            neighbors.addAll(ActiveRepairService.getNeighbors(keyspaceName, ranges, range, Arrays.asList(DatabaseDescriptor.getLocalDataCenter()), null));
-        }
-        assertEquals(expected, neighbors);
-    }
-
-    @Test
-    public void testGetNeighborsTimesTwoInSpecifiedHosts() throws Throwable
-    {
-        TokenMetadata tmd = StorageService.instance.getTokenMetadata();
-
-        // generate rf*2 nodes, and ensure that only neighbors specified by the hosts are returned
-        addTokens(2 * Keyspace.open(keyspaceName).getReplicationStrategy().getReplicationFactor());
-        AbstractReplicationStrategy ars = Keyspace.open(keyspaceName).getReplicationStrategy();
-        List<InetAddress> expected = new ArrayList<>();
-        for (Range<Token> replicaRange : ars.getAddressRanges().get(FBUtilities.getBroadcastAddress()))
-        {
-            expected.addAll(ars.getRangeAddresses(tmd.cloneOnlyTokenMap()).get(replicaRange));
-        }
-
-        expected.remove(FBUtilities.getBroadcastAddress());
-        Collection<String> hosts = Arrays.asList(FBUtilities.getBroadcastAddress().getCanonicalHostName(),expected.get(0).getCanonicalHostName());
-
-        Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(keyspaceName);
-        assertEquals(expected.get(0), ActiveRepairService.getNeighbors(keyspaceName, ranges, ranges.iterator().next(), null, hosts).iterator().next());
-    }
-
-    @Test(expected = IllegalArgumentException.class)
-    public void testGetNeighborsSpecifiedHostsWithNoLocalHost() throws Throwable
-    {
-        addTokens(2 * Keyspace.open(keyspaceName).getReplicationStrategy().getReplicationFactor());
-        //Dont give local endpoint
-        Collection<String> hosts = Arrays.asList("127.0.0.3");
-        Collection<Range<Token>> ranges = StorageService.instance.getLocalRanges(keyspaceName);
-        ActiveRepairService.getNeighbors(keyspaceName, ranges, ranges.iterator().next(), null, hosts);
-    }
-
-    Set<InetAddress> addTokens(int max) throws Throwable
-    {
-        TokenMetadata tmd = StorageService.instance.getTokenMetadata();
-        Set<InetAddress> endpoints = new HashSet<InetAddress>();
-        for (int i = 1; i <= max; i++)
-        {
-            InetAddress endpoint = InetAddress.getByName("127.0.0." + i);
-            tmd.updateNormalToken(StorageService.getPartitioner().getRandomToken(), endpoint);
-            endpoints.add(endpoint);
-        }
-        return endpoints;
-    }
-
-    void flushAES() throws Exception
-    {
-        final ExecutorService stage = StageManager.getStage(Stage.ANTI_ENTROPY);
-        final Callable noop = new Callable<Object>()
-        {
-            public Boolean call()
-            {
-                return true;
-            }
-        };
-
-        // send two tasks through the stage: one to follow existing tasks and a second to follow tasks created by
-        // those existing tasks: tasks won't recursively create more tasks
-        stage.submit(noop).get(5000, TimeUnit.MILLISECONDS);
-        stage.submit(noop).get(5000, TimeUnit.MILLISECONDS);
-    }
-}
diff --git a/test/unit/org/apache/cassandra/service/ClientWarningsTest.java b/test/unit/org/apache/cassandra/service/ClientWarningsTest.java
new file mode 100644
index 0000000..d22a8f6
--- /dev/null
+++ b/test/unit/org/apache/cassandra/service/ClientWarningsTest.java
@@ -0,0 +1,158 @@
+/*
+ * 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.service;
+
+import org.apache.commons.lang3.StringUtils;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.transport.Message;
+import org.apache.cassandra.transport.Server;
+import org.apache.cassandra.transport.SimpleClient;
+import org.apache.cassandra.transport.messages.QueryMessage;
+
+import static junit.framework.Assert.assertEquals;
+import static junit.framework.Assert.assertNull;
+
+public class ClientWarningsTest extends CQLTester
+{
+    @BeforeClass
+    public static void setUp()
+    {
+        DatabaseDescriptor.setPartitioner(ByteOrderedPartitioner.instance);
+
+        requireNetwork();
+        DatabaseDescriptor.setBatchSizeWarnThresholdInKB(1);
+    }
+
+    @Test
+    public void testUnloggedBatchWithProtoV4() throws Exception
+    {
+        createTable("CREATE TABLE %s (pk int PRIMARY KEY, v text)");
+
+        try (SimpleClient client = new SimpleClient(nativeAddr.getHostAddress(), nativePort, Server.VERSION_4))
+        {
+            client.connect(false);
+
+            QueryMessage query = new QueryMessage(createBatchStatement2(1), QueryOptions.DEFAULT);
+            Message.Response resp = client.execute(query);
+            assertNull(resp.getWarnings());
+
+            query = new QueryMessage(createBatchStatement2(DatabaseDescriptor.getBatchSizeWarnThreshold()), QueryOptions.DEFAULT);
+            resp = client.execute(query);
+            assertEquals(1, resp.getWarnings().size());
+        }
+    }
+
+    @Test
+    public void testLargeBatchWithProtoV4() throws Exception
+    {
+        createTable("CREATE TABLE %s (pk int PRIMARY KEY, v text)");
+
+        try (SimpleClient client = new SimpleClient(nativeAddr.getHostAddress(), nativePort, Server.VERSION_4))
+        {
+            client.connect(false);
+
+            QueryMessage query = new QueryMessage(createBatchStatement(DatabaseDescriptor.getBatchSizeWarnThreshold()), QueryOptions.DEFAULT);
+            Message.Response resp = client.execute(query);
+            assertEquals(1, resp.getWarnings().size());
+        }
+    }
+
+    @Test
+    public void testTombstoneWarning() throws Exception
+    {
+        final int iterations = 10000;
+        createTable("CREATE TABLE %s (pk int, ck int, v int, PRIMARY KEY (pk, ck))");
+
+        try (SimpleClient client = new SimpleClient(nativeAddr.getHostAddress(), nativePort, Server.VERSION_4))
+        {
+            client.connect(false);
+
+            for (int i = 0; i < iterations; i++)
+            {
+                QueryMessage query = new QueryMessage(String.format("INSERT INTO %s.%s (pk, ck, v) VALUES (1, %s, 1)",
+                                                                    KEYSPACE,
+                                                                    currentTable(),
+                                                                    i), QueryOptions.DEFAULT);
+                client.execute(query);
+            }
+            ColumnFamilyStore store = Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable());
+            store.forceBlockingFlush();
+
+            for (int i = 0; i < iterations; i++)
+            {
+                QueryMessage query = new QueryMessage(String.format("DELETE v FROM %s.%s WHERE pk = 1 AND ck = %s",
+                                                                    KEYSPACE,
+                                                                    currentTable(),
+                                                                    i), QueryOptions.DEFAULT);
+                client.execute(query);
+            }
+            store.forceBlockingFlush();
+
+            {
+                QueryMessage query = new QueryMessage(String.format("SELECT * FROM %s.%s WHERE pk = 1",
+                                                                    KEYSPACE,
+                                                                    currentTable()), QueryOptions.DEFAULT);
+                Message.Response resp = client.execute(query);
+                assertEquals(1, resp.getWarnings().size());
+            }
+        }
+    }
+
+    @Test
+    public void testLargeBatchWithProtoV2() throws Exception
+    {
+        createTable("CREATE TABLE %s (pk int PRIMARY KEY, v text)");
+
+        try (SimpleClient client = new SimpleClient(nativeAddr.getHostAddress(), nativePort, Server.VERSION_2))
+        {
+            client.connect(false);
+
+            QueryMessage query = new QueryMessage(createBatchStatement(DatabaseDescriptor.getBatchSizeWarnThreshold()), QueryOptions.DEFAULT);
+            Message.Response resp = client.execute(query);
+            assertNull(resp.getWarnings());
+        }
+    }
+
+    private String createBatchStatement(int minSize)
+    {
+        return String.format("BEGIN UNLOGGED BATCH INSERT INTO %s.%s (pk, v) VALUES (1, '%s') APPLY BATCH;",
+                             KEYSPACE,
+                             currentTable(),
+                             StringUtils.repeat('1', minSize));
+    }
+
+    private String createBatchStatement2(int minSize)
+    {
+        return String.format("BEGIN UNLOGGED BATCH INSERT INTO %s.%s (pk, v) VALUES (1, '%s'); INSERT INTO %s.%s (pk, v) VALUES (2, '%s'); APPLY BATCH;",
+                             KEYSPACE,
+                             currentTable(),
+                             StringUtils.repeat('1', minSize),
+                             KEYSPACE,
+                             currentTable(),
+                             StringUtils.repeat('1', minSize));
+    }
+
+}
diff --git a/test/unit/org/apache/cassandra/service/EmbeddedCassandraServiceTest.java b/test/unit/org/apache/cassandra/service/EmbeddedCassandraServiceTest.java
index 24b5a74..ed0efee 100644
--- a/test/unit/org/apache/cassandra/service/EmbeddedCassandraServiceTest.java
+++ b/test/unit/org/apache/cassandra/service/EmbeddedCassandraServiceTest.java
@@ -27,7 +27,8 @@
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.thrift.*;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.thrift.TException;
@@ -47,10 +48,23 @@
  * Tests connect to localhost:9160 when the embedded server is running.
  *
  */
-public class EmbeddedCassandraServiceTest extends SchemaLoader
+public class EmbeddedCassandraServiceTest
 {
 
     private static EmbeddedCassandraService cassandra;
+    private static final String KEYSPACE1 = "EmbeddedCassandraServiceTest";
+    private static final String CF_STANDARD = "Standard1";
+
+    @BeforeClass
+    public static void defineSchema() throws Exception
+    {
+        SchemaLoader.prepareServer();
+        setup();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD));
+    }
 
     /**
      * Set embedded cassandra up and spawn it in a new thread.
@@ -59,10 +73,9 @@
      * @throws IOException
      * @throws InterruptedException
      */
-    @BeforeClass
     public static void setup() throws TTransportException, IOException, InterruptedException
     {
-        Schema.instance.clear(); // Schema are now written on disk and will be reloaded
+        // unique ks / cfs mean no need to clear the schema
         cassandra = new EmbeddedCassandraService();
         cassandra.start();
     }
@@ -72,7 +85,7 @@
     throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, CharacterCodingException
     {
         Cassandra.Client client = getClient();
-        client.set_keyspace("Keyspace1");
+        client.set_keyspace(KEYSPACE1);
 
         ByteBuffer key_user_id = ByteBufferUtil.bytes("1");
 
diff --git a/test/unit/org/apache/cassandra/service/InitClientTest.java b/test/unit/org/apache/cassandra/service/InitClientTest.java
deleted file mode 100644
index 4ce0678..0000000
--- a/test/unit/org/apache/cassandra/service/InitClientTest.java
+++ /dev/null
@@ -1,33 +0,0 @@
-package org.apache.cassandra.service;
-
-import org.junit.Test;
-
-import org.apache.cassandra.exceptions.ConfigurationException;
-
-/**
- * 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.
- */
-
-
-public class InitClientTest // extends CleanupHelper
-{
-    @Test
-    public void testInitClientStartup() throws ConfigurationException
-    {
-        StorageService.instance.initClient(0);
-    }
-}
diff --git a/test/unit/org/apache/cassandra/service/LeaveAndBootstrapTest.java b/test/unit/org/apache/cassandra/service/LeaveAndBootstrapTest.java
index eef8c86..4a09b7a 100644
--- a/test/unit/org/apache/cassandra/service/LeaveAndBootstrapTest.java
+++ b/test/unit/org/apache/cassandra/service/LeaveAndBootstrapTest.java
@@ -23,9 +23,6 @@
 import java.net.UnknownHostException;
 import java.util.*;
 
-import org.apache.cassandra.db.SystemKeyspace;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.config.Schema;
 import org.junit.Test;
 
 import static org.junit.Assert.*;
@@ -36,7 +33,10 @@
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.dht.*;
+import org.apache.cassandra.dht.RandomPartitioner.BigIntegerToken;
 import org.apache.cassandra.gms.ApplicationState;
 import org.apache.cassandra.gms.Gossiper;
 import org.apache.cassandra.gms.VersionedValue;
@@ -46,21 +46,25 @@
 
 public class LeaveAndBootstrapTest
 {
-    private static final IPartitioner partitioner = new RandomPartitioner();
+    private static final IPartitioner partitioner = RandomPartitioner.instance;
     private static IPartitioner oldPartitioner;
+    private static final String KEYSPACE1 = "LeaveAndBootstrapTestKeyspace1";
+    private static final String KEYSPACE2 = "LeaveAndBootstrapTestKeyspace2";
+    private static final String KEYSPACE3 = "LeaveAndBootstrapTestKeyspace3";
+    private static final String KEYSPACE4 = "LeaveAndBootstrapTestKeyspace4";
 
     @BeforeClass
-    public static void setup() throws ConfigurationException
+    public static void defineSchema() throws Exception
     {
         oldPartitioner = StorageService.instance.setPartitionerUnsafe(partitioner);
         SchemaLoader.loadSchema();
+        SchemaLoader.schemaDefinition("LeaveAndBootstrapTest");
     }
 
     @AfterClass
     public static void tearDown()
     {
         StorageService.instance.setPartitionerUnsafe(oldPartitioner);
-        SchemaLoader.stopGossiper();
     }
 
     /**
@@ -76,7 +80,7 @@
 
         TokenMetadata tmd = ss.getTokenMetadata();
         tmd.clearUnsafe();
-        IPartitioner partitioner = new RandomPartitioner();
+        IPartitioner partitioner = RandomPartitioner.instance;
         VersionedValue.VersionedValueFactory valueFactory = new VersionedValue.VersionedValueFactory(partitioner);
 
         ArrayList<Token> endpointTokens = new ArrayList<Token>();
@@ -147,7 +151,7 @@
         final int RING_SIZE = 10;
         TokenMetadata tmd = ss.getTokenMetadata();
         tmd.clearUnsafe();
-        IPartitioner partitioner = new RandomPartitioner();
+        IPartitioner partitioner = RandomPartitioner.instance;
         VersionedValue.VersionedValueFactory valueFactory = new VersionedValue.VersionedValueFactory(partitioner);
 
         ArrayList<Token> endpointTokens = new ArrayList<Token>();
@@ -185,55 +189,55 @@
         Map<String, AbstractReplicationStrategy> keyspaceStrategyMap = new HashMap<String, AbstractReplicationStrategy>();
         for (int i=1; i<=4; i++)
         {
-            keyspaceStrategyMap.put("Keyspace" + i, getStrategy("Keyspace" + i, tmd));
+            keyspaceStrategyMap.put("LeaveAndBootstrapTestKeyspace" + i, getStrategy("LeaveAndBootstrapTestKeyspace" + i, tmd));
         }
 
         // pre-calculate the results.
         Map<String, Multimap<Token, InetAddress>> expectedEndpoints = new HashMap<String, Multimap<Token, InetAddress>>();
-        expectedEndpoints.put("Keyspace1", HashMultimap.<Token, InetAddress>create());
-        expectedEndpoints.get("Keyspace1").putAll(new BigIntegerToken("5"), makeAddrs("127.0.0.2"));
-        expectedEndpoints.get("Keyspace1").putAll(new BigIntegerToken("15"), makeAddrs("127.0.0.3"));
-        expectedEndpoints.get("Keyspace1").putAll(new BigIntegerToken("25"), makeAddrs("127.0.0.4"));
-        expectedEndpoints.get("Keyspace1").putAll(new BigIntegerToken("35"), makeAddrs("127.0.0.5"));
-        expectedEndpoints.get("Keyspace1").putAll(new BigIntegerToken("45"), makeAddrs("127.0.0.6"));
-        expectedEndpoints.get("Keyspace1").putAll(new BigIntegerToken("55"), makeAddrs("127.0.0.7", "127.0.0.8", "127.0.1.1"));
-        expectedEndpoints.get("Keyspace1").putAll(new BigIntegerToken("65"), makeAddrs("127.0.0.8"));
-        expectedEndpoints.get("Keyspace1").putAll(new BigIntegerToken("75"), makeAddrs("127.0.0.9", "127.0.1.2", "127.0.0.1"));
-        expectedEndpoints.get("Keyspace1").putAll(new BigIntegerToken("85"), makeAddrs("127.0.0.10", "127.0.0.1"));
-        expectedEndpoints.get("Keyspace1").putAll(new BigIntegerToken("95"), makeAddrs("127.0.0.1"));
-        expectedEndpoints.put("Keyspace2", HashMultimap.<Token, InetAddress>create());
-        expectedEndpoints.get("Keyspace2").putAll(new BigIntegerToken("5"), makeAddrs("127.0.0.2"));
-        expectedEndpoints.get("Keyspace2").putAll(new BigIntegerToken("15"), makeAddrs("127.0.0.3"));
-        expectedEndpoints.get("Keyspace2").putAll(new BigIntegerToken("25"), makeAddrs("127.0.0.4"));
-        expectedEndpoints.get("Keyspace2").putAll(new BigIntegerToken("35"), makeAddrs("127.0.0.5"));
-        expectedEndpoints.get("Keyspace2").putAll(new BigIntegerToken("45"), makeAddrs("127.0.0.6"));
-        expectedEndpoints.get("Keyspace2").putAll(new BigIntegerToken("55"), makeAddrs("127.0.0.7", "127.0.0.8", "127.0.1.1"));
-        expectedEndpoints.get("Keyspace2").putAll(new BigIntegerToken("65"), makeAddrs("127.0.0.8"));
-        expectedEndpoints.get("Keyspace2").putAll(new BigIntegerToken("75"), makeAddrs("127.0.0.9", "127.0.1.2", "127.0.0.1"));
-        expectedEndpoints.get("Keyspace2").putAll(new BigIntegerToken("85"), makeAddrs("127.0.0.10", "127.0.0.1"));
-        expectedEndpoints.get("Keyspace2").putAll(new BigIntegerToken("95"), makeAddrs("127.0.0.1"));
-        expectedEndpoints.put("Keyspace3", HashMultimap.<Token, InetAddress>create());
-        expectedEndpoints.get("Keyspace3").putAll(new BigIntegerToken("5"), makeAddrs("127.0.0.2", "127.0.0.3", "127.0.0.4", "127.0.0.5", "127.0.0.6"));
-        expectedEndpoints.get("Keyspace3").putAll(new BigIntegerToken("15"), makeAddrs("127.0.0.3", "127.0.0.4", "127.0.0.5", "127.0.0.6", "127.0.0.7", "127.0.1.1", "127.0.0.8"));
-        expectedEndpoints.get("Keyspace3").putAll(new BigIntegerToken("25"), makeAddrs("127.0.0.4", "127.0.0.5", "127.0.0.6", "127.0.0.7", "127.0.0.8", "127.0.1.2", "127.0.0.1", "127.0.1.1"));
-        expectedEndpoints.get("Keyspace3").putAll(new BigIntegerToken("35"), makeAddrs("127.0.0.5", "127.0.0.6", "127.0.0.7", "127.0.0.8", "127.0.0.9", "127.0.1.2", "127.0.0.1", "127.0.0.2", "127.0.1.1"));
-        expectedEndpoints.get("Keyspace3").putAll(new BigIntegerToken("45"), makeAddrs("127.0.0.6", "127.0.0.7", "127.0.0.8", "127.0.0.9", "127.0.0.10", "127.0.1.2", "127.0.0.1", "127.0.0.2", "127.0.1.1", "127.0.0.3"));
-        expectedEndpoints.get("Keyspace3").putAll(new BigIntegerToken("55"), makeAddrs("127.0.0.7", "127.0.0.8", "127.0.0.9", "127.0.0.10", "127.0.0.1", "127.0.0.2", "127.0.0.3", "127.0.0.4", "127.0.1.1", "127.0.1.2"));
-        expectedEndpoints.get("Keyspace3").putAll(new BigIntegerToken("65"), makeAddrs("127.0.0.8", "127.0.0.9", "127.0.0.10", "127.0.0.1", "127.0.0.2", "127.0.1.2", "127.0.0.3", "127.0.0.4"));
-        expectedEndpoints.get("Keyspace3").putAll(new BigIntegerToken("75"), makeAddrs("127.0.0.9", "127.0.0.10", "127.0.0.1", "127.0.0.2", "127.0.0.3", "127.0.1.2", "127.0.0.4", "127.0.0.5"));
-        expectedEndpoints.get("Keyspace3").putAll(new BigIntegerToken("85"), makeAddrs("127.0.0.10", "127.0.0.1", "127.0.0.2", "127.0.0.3", "127.0.0.4", "127.0.0.5"));
-        expectedEndpoints.get("Keyspace3").putAll(new BigIntegerToken("95"), makeAddrs("127.0.0.1", "127.0.0.2", "127.0.0.3", "127.0.0.4", "127.0.0.5"));
-        expectedEndpoints.put("Keyspace4", HashMultimap.<Token, InetAddress>create());
-        expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("5"), makeAddrs("127.0.0.2", "127.0.0.3", "127.0.0.4"));
-        expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("15"), makeAddrs("127.0.0.3", "127.0.0.4", "127.0.0.5"));
-        expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("25"), makeAddrs("127.0.0.4", "127.0.0.5", "127.0.0.6"));
-        expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("35"), makeAddrs("127.0.0.5", "127.0.0.6", "127.0.0.7", "127.0.1.1", "127.0.0.8"));
-        expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("45"), makeAddrs("127.0.0.6", "127.0.0.7", "127.0.0.8", "127.0.1.2", "127.0.0.1", "127.0.1.1"));
-        expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("55"), makeAddrs("127.0.0.7", "127.0.0.8", "127.0.0.9", "127.0.0.1", "127.0.0.2", "127.0.1.1", "127.0.1.2"));
-        expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("65"), makeAddrs("127.0.0.8", "127.0.0.9", "127.0.0.10", "127.0.1.2", "127.0.0.1", "127.0.0.2"));
-        expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("75"), makeAddrs("127.0.0.9", "127.0.0.10", "127.0.0.1", "127.0.1.2", "127.0.0.2", "127.0.0.3"));
-        expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("85"), makeAddrs("127.0.0.10", "127.0.0.1", "127.0.0.2", "127.0.0.3"));
-        expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("95"), makeAddrs("127.0.0.1", "127.0.0.2", "127.0.0.3"));
+        expectedEndpoints.put(KEYSPACE1, HashMultimap.<Token, InetAddress>create());
+        expectedEndpoints.get(KEYSPACE1).putAll(new BigIntegerToken("5"), makeAddrs("127.0.0.2"));
+        expectedEndpoints.get(KEYSPACE1).putAll(new BigIntegerToken("15"), makeAddrs("127.0.0.3"));
+        expectedEndpoints.get(KEYSPACE1).putAll(new BigIntegerToken("25"), makeAddrs("127.0.0.4"));
+        expectedEndpoints.get(KEYSPACE1).putAll(new BigIntegerToken("35"), makeAddrs("127.0.0.5"));
+        expectedEndpoints.get(KEYSPACE1).putAll(new BigIntegerToken("45"), makeAddrs("127.0.0.6"));
+        expectedEndpoints.get(KEYSPACE1).putAll(new BigIntegerToken("55"), makeAddrs("127.0.0.7", "127.0.0.8", "127.0.1.1"));
+        expectedEndpoints.get(KEYSPACE1).putAll(new BigIntegerToken("65"), makeAddrs("127.0.0.8"));
+        expectedEndpoints.get(KEYSPACE1).putAll(new BigIntegerToken("75"), makeAddrs("127.0.0.9", "127.0.1.2", "127.0.0.1"));
+        expectedEndpoints.get(KEYSPACE1).putAll(new BigIntegerToken("85"), makeAddrs("127.0.0.10", "127.0.0.1"));
+        expectedEndpoints.get(KEYSPACE1).putAll(new BigIntegerToken("95"), makeAddrs("127.0.0.1"));
+        expectedEndpoints.put(KEYSPACE2, HashMultimap.<Token, InetAddress>create());
+        expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("5"), makeAddrs("127.0.0.2"));
+        expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("15"), makeAddrs("127.0.0.3"));
+        expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("25"), makeAddrs("127.0.0.4"));
+        expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("35"), makeAddrs("127.0.0.5"));
+        expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("45"), makeAddrs("127.0.0.6"));
+        expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("55"), makeAddrs("127.0.0.7", "127.0.0.8", "127.0.1.1"));
+        expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("65"), makeAddrs("127.0.0.8"));
+        expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("75"), makeAddrs("127.0.0.9", "127.0.1.2", "127.0.0.1"));
+        expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("85"), makeAddrs("127.0.0.10", "127.0.0.1"));
+        expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("95"), makeAddrs("127.0.0.1"));
+        expectedEndpoints.put(KEYSPACE3, HashMultimap.<Token, InetAddress>create());
+        expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("5"), makeAddrs("127.0.0.2", "127.0.0.3", "127.0.0.4", "127.0.0.5", "127.0.0.6"));
+        expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("15"), makeAddrs("127.0.0.3", "127.0.0.4", "127.0.0.5", "127.0.0.6", "127.0.0.7", "127.0.1.1", "127.0.0.8"));
+        expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("25"), makeAddrs("127.0.0.4", "127.0.0.5", "127.0.0.6", "127.0.0.7", "127.0.0.8", "127.0.1.2", "127.0.0.1", "127.0.1.1"));
+        expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("35"), makeAddrs("127.0.0.5", "127.0.0.6", "127.0.0.7", "127.0.0.8", "127.0.0.9", "127.0.1.2", "127.0.0.1", "127.0.0.2", "127.0.1.1"));
+        expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("45"), makeAddrs("127.0.0.6", "127.0.0.7", "127.0.0.8", "127.0.0.9", "127.0.0.10", "127.0.1.2", "127.0.0.1", "127.0.0.2", "127.0.1.1", "127.0.0.3"));
+        expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("55"), makeAddrs("127.0.0.7", "127.0.0.8", "127.0.0.9", "127.0.0.10", "127.0.0.1", "127.0.0.2", "127.0.0.3", "127.0.0.4", "127.0.1.1", "127.0.1.2"));
+        expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("65"), makeAddrs("127.0.0.8", "127.0.0.9", "127.0.0.10", "127.0.0.1", "127.0.0.2", "127.0.1.2", "127.0.0.3", "127.0.0.4"));
+        expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("75"), makeAddrs("127.0.0.9", "127.0.0.10", "127.0.0.1", "127.0.0.2", "127.0.0.3", "127.0.1.2", "127.0.0.4", "127.0.0.5"));
+        expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("85"), makeAddrs("127.0.0.10", "127.0.0.1", "127.0.0.2", "127.0.0.3", "127.0.0.4", "127.0.0.5"));
+        expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("95"), makeAddrs("127.0.0.1", "127.0.0.2", "127.0.0.3", "127.0.0.4", "127.0.0.5"));
+        expectedEndpoints.put(KEYSPACE4, HashMultimap.<Token, InetAddress>create());
+        expectedEndpoints.get(KEYSPACE4).putAll(new BigIntegerToken("5"), makeAddrs("127.0.0.2", "127.0.0.3", "127.0.0.4"));
+        expectedEndpoints.get(KEYSPACE4).putAll(new BigIntegerToken("15"), makeAddrs("127.0.0.3", "127.0.0.4", "127.0.0.5"));
+        expectedEndpoints.get(KEYSPACE4).putAll(new BigIntegerToken("25"), makeAddrs("127.0.0.4", "127.0.0.5", "127.0.0.6"));
+        expectedEndpoints.get(KEYSPACE4).putAll(new BigIntegerToken("35"), makeAddrs("127.0.0.5", "127.0.0.6", "127.0.0.7", "127.0.1.1", "127.0.0.8"));
+        expectedEndpoints.get(KEYSPACE4).putAll(new BigIntegerToken("45"), makeAddrs("127.0.0.6", "127.0.0.7", "127.0.0.8", "127.0.1.2", "127.0.0.1", "127.0.1.1"));
+        expectedEndpoints.get(KEYSPACE4).putAll(new BigIntegerToken("55"), makeAddrs("127.0.0.7", "127.0.0.8", "127.0.0.9", "127.0.0.1", "127.0.0.2", "127.0.1.1", "127.0.1.2"));
+        expectedEndpoints.get(KEYSPACE4).putAll(new BigIntegerToken("65"), makeAddrs("127.0.0.8", "127.0.0.9", "127.0.0.10", "127.0.1.2", "127.0.0.1", "127.0.0.2"));
+        expectedEndpoints.get(KEYSPACE4).putAll(new BigIntegerToken("75"), makeAddrs("127.0.0.9", "127.0.0.10", "127.0.0.1", "127.0.1.2", "127.0.0.2", "127.0.0.3"));
+        expectedEndpoints.get(KEYSPACE4).putAll(new BigIntegerToken("85"), makeAddrs("127.0.0.10", "127.0.0.1", "127.0.0.2", "127.0.0.3"));
+        expectedEndpoints.get(KEYSPACE4).putAll(new BigIntegerToken("95"), makeAddrs("127.0.0.1", "127.0.0.2", "127.0.0.3"));
 
         PendingRangeCalculatorService.instance.blockUntilFinished();
 
@@ -338,24 +342,24 @@
         ss.onChange(boot1, ApplicationState.STATUS, valueFactory.normal(Collections.singleton(keyTokens.get(5))));
 
         // adjust precalcuated results.  this changes what the epected endpoints are.
-        expectedEndpoints.get("Keyspace1").get(new BigIntegerToken("55")).removeAll(makeAddrs("127.0.0.7", "127.0.0.8"));
-        expectedEndpoints.get("Keyspace1").get(new BigIntegerToken("85")).removeAll(makeAddrs("127.0.0.10"));
-        expectedEndpoints.get("Keyspace2").get(new BigIntegerToken("55")).removeAll(makeAddrs("127.0.0.7", "127.0.0.8"));
-        expectedEndpoints.get("Keyspace2").get(new BigIntegerToken("85")).removeAll(makeAddrs("127.0.0.10"));
-        expectedEndpoints.get("Keyspace3").get(new BigIntegerToken("15")).removeAll(makeAddrs("127.0.0.7", "127.0.0.8"));
-        expectedEndpoints.get("Keyspace3").get(new BigIntegerToken("25")).removeAll(makeAddrs("127.0.0.7", "127.0.1.2", "127.0.0.1"));
-        expectedEndpoints.get("Keyspace3").get(new BigIntegerToken("35")).removeAll(makeAddrs("127.0.0.7", "127.0.0.2"));
-        expectedEndpoints.get("Keyspace3").get(new BigIntegerToken("45")).removeAll(makeAddrs("127.0.0.7", "127.0.0.10", "127.0.0.3"));
-        expectedEndpoints.get("Keyspace3").get(new BigIntegerToken("55")).removeAll(makeAddrs("127.0.0.7", "127.0.0.10", "127.0.0.4"));
-        expectedEndpoints.get("Keyspace3").get(new BigIntegerToken("65")).removeAll(makeAddrs("127.0.0.10"));
-        expectedEndpoints.get("Keyspace3").get(new BigIntegerToken("75")).removeAll(makeAddrs("127.0.0.10"));
-        expectedEndpoints.get("Keyspace3").get(new BigIntegerToken("85")).removeAll(makeAddrs("127.0.0.10"));
-        expectedEndpoints.get("Keyspace4").get(new BigIntegerToken("35")).removeAll(makeAddrs("127.0.0.7", "127.0.0.8"));
-        expectedEndpoints.get("Keyspace4").get(new BigIntegerToken("45")).removeAll(makeAddrs("127.0.0.7", "127.0.1.2", "127.0.0.1"));
-        expectedEndpoints.get("Keyspace4").get(new BigIntegerToken("55")).removeAll(makeAddrs("127.0.0.2", "127.0.0.7"));
-        expectedEndpoints.get("Keyspace4").get(new BigIntegerToken("65")).removeAll(makeAddrs("127.0.0.10"));
-        expectedEndpoints.get("Keyspace4").get(new BigIntegerToken("75")).removeAll(makeAddrs("127.0.0.10"));
-        expectedEndpoints.get("Keyspace4").get(new BigIntegerToken("85")).removeAll(makeAddrs("127.0.0.10"));
+        expectedEndpoints.get(KEYSPACE1).get(new BigIntegerToken("55")).removeAll(makeAddrs("127.0.0.7", "127.0.0.8"));
+        expectedEndpoints.get(KEYSPACE1).get(new BigIntegerToken("85")).removeAll(makeAddrs("127.0.0.10"));
+        expectedEndpoints.get(KEYSPACE2).get(new BigIntegerToken("55")).removeAll(makeAddrs("127.0.0.7", "127.0.0.8"));
+        expectedEndpoints.get(KEYSPACE2).get(new BigIntegerToken("85")).removeAll(makeAddrs("127.0.0.10"));
+        expectedEndpoints.get(KEYSPACE3).get(new BigIntegerToken("15")).removeAll(makeAddrs("127.0.0.7", "127.0.0.8"));
+        expectedEndpoints.get(KEYSPACE3).get(new BigIntegerToken("25")).removeAll(makeAddrs("127.0.0.7", "127.0.1.2", "127.0.0.1"));
+        expectedEndpoints.get(KEYSPACE3).get(new BigIntegerToken("35")).removeAll(makeAddrs("127.0.0.7", "127.0.0.2"));
+        expectedEndpoints.get(KEYSPACE3).get(new BigIntegerToken("45")).removeAll(makeAddrs("127.0.0.7", "127.0.0.10", "127.0.0.3"));
+        expectedEndpoints.get(KEYSPACE3).get(new BigIntegerToken("55")).removeAll(makeAddrs("127.0.0.7", "127.0.0.10", "127.0.0.4"));
+        expectedEndpoints.get(KEYSPACE3).get(new BigIntegerToken("65")).removeAll(makeAddrs("127.0.0.10"));
+        expectedEndpoints.get(KEYSPACE3).get(new BigIntegerToken("75")).removeAll(makeAddrs("127.0.0.10"));
+        expectedEndpoints.get(KEYSPACE3).get(new BigIntegerToken("85")).removeAll(makeAddrs("127.0.0.10"));
+        expectedEndpoints.get(KEYSPACE4).get(new BigIntegerToken("35")).removeAll(makeAddrs("127.0.0.7", "127.0.0.8"));
+        expectedEndpoints.get(KEYSPACE4).get(new BigIntegerToken("45")).removeAll(makeAddrs("127.0.0.7", "127.0.1.2", "127.0.0.1"));
+        expectedEndpoints.get(KEYSPACE4).get(new BigIntegerToken("55")).removeAll(makeAddrs("127.0.0.2", "127.0.0.7"));
+        expectedEndpoints.get(KEYSPACE4).get(new BigIntegerToken("65")).removeAll(makeAddrs("127.0.0.10"));
+        expectedEndpoints.get(KEYSPACE4).get(new BigIntegerToken("75")).removeAll(makeAddrs("127.0.0.10"));
+        expectedEndpoints.get(KEYSPACE4).get(new BigIntegerToken("85")).removeAll(makeAddrs("127.0.0.10"));
 
         PendingRangeCalculatorService.instance.blockUntilFinished();
 
@@ -447,7 +451,7 @@
         StorageService ss = StorageService.instance;
         TokenMetadata tmd = ss.getTokenMetadata();
         tmd.clearUnsafe();
-        IPartitioner partitioner = new RandomPartitioner();
+        IPartitioner partitioner = RandomPartitioner.instance;
         VersionedValue.VersionedValueFactory valueFactory = new VersionedValue.VersionedValueFactory(partitioner);
 
         ArrayList<Token> endpointTokens = new ArrayList<Token>();
@@ -524,7 +528,7 @@
         StorageService ss = StorageService.instance;
         TokenMetadata tmd = ss.getTokenMetadata();
         tmd.clearUnsafe();
-        IPartitioner partitioner = new RandomPartitioner();
+        IPartitioner partitioner = RandomPartitioner.instance;
         VersionedValue.VersionedValueFactory valueFactory = new VersionedValue.VersionedValueFactory(partitioner);
 
         ArrayList<Token> endpointTokens = new ArrayList<Token>();
@@ -566,7 +570,7 @@
         StorageService ss = StorageService.instance;
         TokenMetadata tmd = ss.getTokenMetadata();
         tmd.clearUnsafe();
-        IPartitioner partitioner = new RandomPartitioner();
+        IPartitioner partitioner = RandomPartitioner.instance;
         VersionedValue.VersionedValueFactory valueFactory = new VersionedValue.VersionedValueFactory(partitioner);
 
         ArrayList<Token> endpointTokens = new ArrayList<Token>();
@@ -616,7 +620,7 @@
         StorageService ss = StorageService.instance;
         TokenMetadata tmd = ss.getTokenMetadata();
         tmd.clearUnsafe();
-        IPartitioner partitioner = new RandomPartitioner();
+        IPartitioner partitioner = RandomPartitioner.instance;
         VersionedValue.VersionedValueFactory valueFactory = new VersionedValue.VersionedValueFactory(partitioner);
 
         ArrayList<Token> endpointTokens = new ArrayList<Token>();
diff --git a/test/unit/org/apache/cassandra/service/MoveTest.java b/test/unit/org/apache/cassandra/service/MoveTest.java
index 49e3391..bd4317d 100644
--- a/test/unit/org/apache/cassandra/service/MoveTest.java
+++ b/test/unit/org/apache/cassandra/service/MoveTest.java
@@ -29,12 +29,13 @@
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.ColumnFamilyType;
 import org.apache.cassandra.db.marshal.BytesType;
 import org.apache.cassandra.gms.Gossiper;
 import org.apache.cassandra.locator.AbstractNetworkTopologySnitch;
 import org.apache.cassandra.locator.NetworkTopologyStrategy;
+import org.apache.cassandra.locator.PendingRangeMaps;
 import org.junit.AfterClass;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -44,6 +45,7 @@
 import org.apache.cassandra.Util;
 import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.dht.*;
+import org.apache.cassandra.dht.RandomPartitioner.BigIntegerToken;
 import org.apache.cassandra.gms.ApplicationState;
 import org.apache.cassandra.gms.VersionedValue;
 import org.apache.cassandra.locator.AbstractReplicationStrategy;
@@ -52,8 +54,19 @@
 
 public class MoveTest
 {
-    private static final IPartitioner partitioner = new RandomPartitioner();
+    private static final IPartitioner partitioner = RandomPartitioner.instance;
     private static IPartitioner oldPartitioner;
+    //Simple Strategy Keyspaces
+    private static final String Simple_RF1_KeyspaceName = "MoveTestKeyspace1";
+    private static final String Simple_RF2_KeyspaceName = "MoveTestKeyspace5";
+    private static final String Simple_RF3_KeyspaceName = "MoveTestKeyspace4";
+    private static final String KEYSPACE2 = "MoveTestKeyspace2";
+    private static final String KEYSPACE3 = "MoveTestKeyspace3";
+
+    //Network Strategy Keyspace with RF DC1=1 and DC2=1 and so on.
+    private static final String Network_11_KeyspaceName = "MoveTestNetwork11";
+    private static final String Network_22_KeyspaceName = "MoveTestNetwork22";
+    private static final String Network_33_KeyspaceName = "MoveTestNetwork33";
 
     /*
      * NOTE: the tests above uses RandomPartitioner, which is not the default
@@ -66,6 +79,7 @@
     {
         oldPartitioner = StorageService.instance.setPartitionerUnsafe(partitioner);
         SchemaLoader.loadSchema();
+        SchemaLoader.schemaDefinition("MoveTest");
         addNetworkTopologyKeyspace(Network_11_KeyspaceName, 1, 1);
         addNetworkTopologyKeyspace(Network_22_KeyspaceName, 2, 2);
         addNetworkTopologyKeyspace(Network_33_KeyspaceName, 3, 3);
@@ -75,17 +89,14 @@
     public static void tearDown()
     {
         StorageService.instance.setPartitionerUnsafe(oldPartitioner);
-        SchemaLoader.stopGossiper();
     }
 
-    //Simple Strategy Keyspaces with RF1, 2 and 3
-    private static final String Simple_RF1_KeyspaceName = "Keyspace6";
-    private static final String Simple_RF2_KeyspaceName = "Keyspace5";
-    private static final String Simple_RF3_KeyspaceName = "Keyspace4";
-    //Network Strategy Keyspace with RF DC1=1 and DC2=1 and so on.
-    private static final String Network_11_KeyspaceName = "Network11";
-    private static final String Network_22_KeyspaceName = "Network22";
-    private static final String Network_33_KeyspaceName = "Network33";
+    @Before
+    public void clearTokenMetadata()
+    {
+        PendingRangeCalculatorService.instance.blockUntilFinished();
+        StorageService.instance.getTokenMetadata().clearUnsafe();
+    }
 
     private static void addNetworkTopologyKeyspace(String keyspaceName, Integer... replicas) throws ConfigurationException
     {
@@ -123,7 +134,7 @@
 
         Class<? extends AbstractReplicationStrategy> strategy = NetworkTopologyStrategy.class;
         KSMetaData keyspace = KSMetaData.testMetadata(keyspaceName, strategy, configOptions(replicas),
-                CFMetaData.sparseCFMetaData(keyspaceName, "CF1", BytesType.instance));
+                CFMetaData.denseCFMetaData(keyspaceName, "CF1", BytesType.instance));
         MigrationManager.announceNewKeyspace(keyspace);
     }
 
@@ -199,6 +210,7 @@
             endpointTokens.add(new BigIntegerToken(String.valueOf(10 * i)));
             endpointTokens.add(new BigIntegerToken(String.valueOf((10 * i) + 1)));
         }
+
         Util.createInitialRing(ss, partitioner, endpointTokens, keyTokens, hosts, hostIds, RING_SIZE);
         PendingRangeCalculatorService.instance.blockUntilFinished();
 
@@ -492,14 +504,19 @@
         assert keyspaceFound;
     }
 
-    private void assertMaps(Map<Range<Token>,  Collection<InetAddress>> expected, Map<Range<Token>,  Collection<InetAddress>> actual)
+    private void assertMaps(Map<Range<Token>, Collection<InetAddress>> expected, PendingRangeMaps actual)
     {
-        assertEquals(expected.size(), actual.size());
-        for(Map.Entry<Range<Token>,  Collection<InetAddress>> expectedEntry : expected.entrySet())
+        int sizeOfActual = 0;
+        Iterator<Map.Entry<Range<Token>, List<InetAddress>>> iterator = actual.iterator();
+        while(iterator.hasNext())
         {
-            assertNotNull(actual.get(expectedEntry.getKey()));
-            assertEquals(new ArrayList<>(expectedEntry.getValue()), new ArrayList<>(actual.get(expectedEntry.getKey())));
+            Map.Entry<Range<Token>, List<InetAddress>> actualEntry = iterator.next();
+            assertNotNull(expected.get(actualEntry.getKey()));
+            assertEquals(new HashSet<>(expected.get(actualEntry.getKey())), new HashSet<>(actualEntry.getValue()));
+            sizeOfActual++;
         }
+
+        assertEquals(expected.size(), sizeOfActual);
     }
 
     /*
@@ -514,7 +531,6 @@
         final int MOVING_NODE = 3; // index of the moving node
 
         TokenMetadata tmd = ss.getTokenMetadata();
-        tmd.clearUnsafe();
         VersionedValue.VersionedValueFactory valueFactory = new VersionedValue.VersionedValueFactory(partitioner);
 
         ArrayList<Token> endpointTokens = new ArrayList<Token>();
@@ -572,8 +588,7 @@
                 	numMoved++;
                 }
             }
-            // This assertion isn't reliable in 2.0 and 2.1.  See CASSANDRA-7390
-            // assertEquals("mismatched number of moved token", numMoved, 1);
+            assertEquals("mismatched number of moved token", 1, numMoved);
         }
 
         // moving endpoint back to the normal state
@@ -589,8 +604,7 @@
         StorageService ss = StorageService.instance;
         final int RING_SIZE = 10;
         TokenMetadata tmd = ss.getTokenMetadata();
-        tmd.clearUnsafe();
-        IPartitioner partitioner = new RandomPartitioner();
+        IPartitioner partitioner = RandomPartitioner.instance;
         VersionedValue.VersionedValueFactory valueFactory = new VersionedValue.VersionedValueFactory(partitioner);
 
         ArrayList<Token> endpointTokens = new ArrayList<Token>();
@@ -627,6 +641,8 @@
         ss.onChange(boot1,
                     ApplicationState.STATUS,
                     valueFactory.bootstrapping(Collections.<Token>singleton(keyTokens.get(5))));
+        PendingRangeCalculatorService.instance.blockUntilFinished();
+
         InetAddress boot2 = InetAddress.getByName("127.0.1.2");
         Gossiper.instance.initializeNodeUnsafe(boot2, UUID.randomUUID(), 1);
         Gossiper.instance.injectApplicationState(boot2, ApplicationState.TOKENS, valueFactory.tokens(Collections.singleton(keyTokens.get(7))));
@@ -639,7 +655,7 @@
         Map<String, AbstractReplicationStrategy> keyspaceStrategyMap = new HashMap<String, AbstractReplicationStrategy>();
         for (int i = 1; i <= 4; i++)
         {
-            keyspaceStrategyMap.put("Keyspace" + i, getStrategy("Keyspace" + i, tmd));
+            keyspaceStrategyMap.put("MoveTestKeyspace" + i, getStrategy("MoveTestKeyspace" + i, tmd));
         }
 
        /**
@@ -658,7 +674,7 @@
         *  }
         */
 
-        Multimap<InetAddress, Range<Token>> keyspace1ranges = keyspaceStrategyMap.get("Keyspace1").getAddressRanges();
+        Multimap<InetAddress, Range<Token>> keyspace1ranges = keyspaceStrategyMap.get(Simple_RF1_KeyspaceName).getAddressRanges();
         Collection<Range<Token>> ranges1 = keyspace1ranges.get(InetAddress.getByName("127.0.0.1"));
         assertEquals(1, collectionSize(ranges1));
         assertEquals(generateRange(97, 0), ranges1.iterator().next());
@@ -707,7 +723,7 @@
         * }
         */
 
-        Multimap<InetAddress, Range<Token>> keyspace3ranges = keyspaceStrategyMap.get("Keyspace3").getAddressRanges();
+        Multimap<InetAddress, Range<Token>> keyspace3ranges = keyspaceStrategyMap.get(KEYSPACE3).getAddressRanges();
         ranges1 = keyspace3ranges.get(InetAddress.getByName("127.0.0.1"));
         assertEquals(collectionSize(ranges1), 5);
         assertTrue(ranges1.equals(generateRanges(97, 0, 70, 87, 50, 67, 87, 97, 67, 70)));
@@ -755,7 +771,7 @@
          *      /127.0.0.10=[(70,87], (87,97], (67,70]]
          *  }
          */
-        Multimap<InetAddress, Range<Token>> keyspace4ranges = keyspaceStrategyMap.get("Keyspace4").getAddressRanges();
+        Multimap<InetAddress, Range<Token>> keyspace4ranges = keyspaceStrategyMap.get(Simple_RF3_KeyspaceName).getAddressRanges();
         ranges1 = keyspace4ranges.get(InetAddress.getByName("127.0.0.1"));
         assertEquals(collectionSize(ranges1), 3);
         assertTrue(ranges1.equals(generateRanges(97, 0, 70, 87, 87, 97)));
@@ -789,50 +805,50 @@
 
         // pre-calculate the results.
         Map<String, Multimap<Token, InetAddress>> expectedEndpoints = new HashMap<String, Multimap<Token, InetAddress>>();
-        expectedEndpoints.put("Keyspace1", HashMultimap.<Token, InetAddress>create());
-        expectedEndpoints.get("Keyspace1").putAll(new BigIntegerToken("5"), makeAddrs("127.0.0.2"));
-        expectedEndpoints.get("Keyspace1").putAll(new BigIntegerToken("15"), makeAddrs("127.0.0.3"));
-        expectedEndpoints.get("Keyspace1").putAll(new BigIntegerToken("25"), makeAddrs("127.0.0.4"));
-        expectedEndpoints.get("Keyspace1").putAll(new BigIntegerToken("35"), makeAddrs("127.0.0.5"));
-        expectedEndpoints.get("Keyspace1").putAll(new BigIntegerToken("45"), makeAddrs("127.0.0.6"));
-        expectedEndpoints.get("Keyspace1").putAll(new BigIntegerToken("55"), makeAddrs("127.0.0.7", "127.0.1.1"));
-        expectedEndpoints.get("Keyspace1").putAll(new BigIntegerToken("65"), makeAddrs("127.0.0.7"));
-        expectedEndpoints.get("Keyspace1").putAll(new BigIntegerToken("75"), makeAddrs("127.0.0.9", "127.0.1.2"));
-        expectedEndpoints.get("Keyspace1").putAll(new BigIntegerToken("85"), makeAddrs("127.0.0.9"));
-        expectedEndpoints.get("Keyspace1").putAll(new BigIntegerToken("95"), makeAddrs("127.0.0.10"));
-        expectedEndpoints.put("Keyspace2", HashMultimap.<Token, InetAddress>create());
-        expectedEndpoints.get("Keyspace2").putAll(new BigIntegerToken("5"), makeAddrs("127.0.0.2"));
-        expectedEndpoints.get("Keyspace2").putAll(new BigIntegerToken("15"), makeAddrs("127.0.0.3"));
-        expectedEndpoints.get("Keyspace2").putAll(new BigIntegerToken("25"), makeAddrs("127.0.0.4"));
-        expectedEndpoints.get("Keyspace2").putAll(new BigIntegerToken("35"), makeAddrs("127.0.0.5"));
-        expectedEndpoints.get("Keyspace2").putAll(new BigIntegerToken("45"), makeAddrs("127.0.0.6"));
-        expectedEndpoints.get("Keyspace2").putAll(new BigIntegerToken("55"), makeAddrs("127.0.0.7", "127.0.1.1"));
-        expectedEndpoints.get("Keyspace2").putAll(new BigIntegerToken("65"), makeAddrs("127.0.0.7"));
-        expectedEndpoints.get("Keyspace2").putAll(new BigIntegerToken("75"), makeAddrs("127.0.0.9", "127.0.1.2"));
-        expectedEndpoints.get("Keyspace2").putAll(new BigIntegerToken("85"), makeAddrs("127.0.0.9"));
-        expectedEndpoints.get("Keyspace2").putAll(new BigIntegerToken("95"), makeAddrs("127.0.0.10"));
-        expectedEndpoints.put("Keyspace3", HashMultimap.<Token, InetAddress>create());
-        expectedEndpoints.get("Keyspace3").putAll(new BigIntegerToken("5"), makeAddrs("127.0.0.2", "127.0.0.3", "127.0.0.4", "127.0.0.5", "127.0.0.6"));
-        expectedEndpoints.get("Keyspace3").putAll(new BigIntegerToken("15"), makeAddrs("127.0.0.3", "127.0.0.4", "127.0.0.5", "127.0.0.6", "127.0.0.7", "127.0.1.1"));
-        expectedEndpoints.get("Keyspace3").putAll(new BigIntegerToken("25"), makeAddrs("127.0.0.4", "127.0.0.5", "127.0.0.6", "127.0.0.7", "127.0.0.8", "127.0.1.1"));
-        expectedEndpoints.get("Keyspace3").putAll(new BigIntegerToken("35"), makeAddrs("127.0.0.5", "127.0.0.6", "127.0.0.7", "127.0.0.8", "127.0.0.9", "127.0.1.1", "127.0.1.2"));
-        expectedEndpoints.get("Keyspace3").putAll(new BigIntegerToken("45"), makeAddrs("127.0.0.6", "127.0.0.7", "127.0.0.8", "127.0.0.9", "127.0.0.10", "127.0.1.1", "127.0.1.2"));
-        expectedEndpoints.get("Keyspace3").putAll(new BigIntegerToken("55"), makeAddrs("127.0.0.7", "127.0.0.8", "127.0.0.9", "127.0.0.10", "127.0.0.1", "127.0.1.1", "127.0.1.2"));
-        expectedEndpoints.get("Keyspace3").putAll(new BigIntegerToken("65"), makeAddrs("127.0.0.7", "127.0.0.8", "127.0.0.9", "127.0.0.10", "127.0.0.1", "127.0.1.2"));
-        expectedEndpoints.get("Keyspace3").putAll(new BigIntegerToken("75"), makeAddrs("127.0.0.9", "127.0.0.10", "127.0.0.1", "127.0.0.2", "127.0.0.3", "127.0.1.2"));
-        expectedEndpoints.get("Keyspace3").putAll(new BigIntegerToken("85"), makeAddrs("127.0.0.9", "127.0.0.10", "127.0.0.1", "127.0.0.2", "127.0.0.3"));
-        expectedEndpoints.get("Keyspace3").putAll(new BigIntegerToken("95"), makeAddrs("127.0.0.10", "127.0.0.1", "127.0.0.2", "127.0.0.3", "127.0.0.4"));
-        expectedEndpoints.put("Keyspace4", HashMultimap.<Token, InetAddress>create());
-        expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("5"), makeAddrs("127.0.0.2", "127.0.0.3", "127.0.0.4"));
-        expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("15"), makeAddrs("127.0.0.3", "127.0.0.4", "127.0.0.5"));
-        expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("25"), makeAddrs("127.0.0.4", "127.0.0.5", "127.0.0.6"));
-        expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("35"), makeAddrs("127.0.0.5", "127.0.0.6", "127.0.0.7", "127.0.1.1"));
-        expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("45"), makeAddrs("127.0.0.6", "127.0.0.7", "127.0.0.8", "127.0.1.1"));
-        expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("55"), makeAddrs("127.0.0.7", "127.0.0.8", "127.0.0.9", "127.0.1.1", "127.0.1.2"));
-        expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("65"), makeAddrs("127.0.0.7", "127.0.0.8", "127.0.0.9", "127.0.1.2"));
-        expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("75"), makeAddrs("127.0.0.9", "127.0.0.10", "127.0.0.1", "127.0.1.2"));
-        expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("85"), makeAddrs("127.0.0.9", "127.0.0.10", "127.0.0.1"));
-        expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("95"), makeAddrs("127.0.0.10", "127.0.0.1", "127.0.0.2"));
+        expectedEndpoints.put(Simple_RF1_KeyspaceName, HashMultimap.<Token, InetAddress>create());
+        expectedEndpoints.get(Simple_RF1_KeyspaceName).putAll(new BigIntegerToken("5"), makeAddrs("127.0.0.2"));
+        expectedEndpoints.get(Simple_RF1_KeyspaceName).putAll(new BigIntegerToken("15"), makeAddrs("127.0.0.3"));
+        expectedEndpoints.get(Simple_RF1_KeyspaceName).putAll(new BigIntegerToken("25"), makeAddrs("127.0.0.4"));
+        expectedEndpoints.get(Simple_RF1_KeyspaceName).putAll(new BigIntegerToken("35"), makeAddrs("127.0.0.5"));
+        expectedEndpoints.get(Simple_RF1_KeyspaceName).putAll(new BigIntegerToken("45"), makeAddrs("127.0.0.6"));
+        expectedEndpoints.get(Simple_RF1_KeyspaceName).putAll(new BigIntegerToken("55"), makeAddrs("127.0.0.7", "127.0.1.1"));
+        expectedEndpoints.get(Simple_RF1_KeyspaceName).putAll(new BigIntegerToken("65"), makeAddrs("127.0.0.7"));
+        expectedEndpoints.get(Simple_RF1_KeyspaceName).putAll(new BigIntegerToken("75"), makeAddrs("127.0.0.9", "127.0.1.2"));
+        expectedEndpoints.get(Simple_RF1_KeyspaceName).putAll(new BigIntegerToken("85"), makeAddrs("127.0.0.9"));
+        expectedEndpoints.get(Simple_RF1_KeyspaceName).putAll(new BigIntegerToken("95"), makeAddrs("127.0.0.10"));
+        expectedEndpoints.put(KEYSPACE2, HashMultimap.<Token, InetAddress>create());
+        expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("5"), makeAddrs("127.0.0.2"));
+        expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("15"), makeAddrs("127.0.0.3"));
+        expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("25"), makeAddrs("127.0.0.4"));
+        expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("35"), makeAddrs("127.0.0.5"));
+        expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("45"), makeAddrs("127.0.0.6"));
+        expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("55"), makeAddrs("127.0.0.7", "127.0.1.1"));
+        expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("65"), makeAddrs("127.0.0.7"));
+        expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("75"), makeAddrs("127.0.0.9", "127.0.1.2"));
+        expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("85"), makeAddrs("127.0.0.9"));
+        expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("95"), makeAddrs("127.0.0.10"));
+        expectedEndpoints.put(KEYSPACE3, HashMultimap.<Token, InetAddress>create());
+        expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("5"), makeAddrs("127.0.0.2", "127.0.0.3", "127.0.0.4", "127.0.0.5", "127.0.0.6"));
+        expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("15"), makeAddrs("127.0.0.3", "127.0.0.4", "127.0.0.5", "127.0.0.6", "127.0.0.7", "127.0.1.1"));
+        expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("25"), makeAddrs("127.0.0.4", "127.0.0.5", "127.0.0.6", "127.0.0.7", "127.0.0.8", "127.0.1.1"));
+        expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("35"), makeAddrs("127.0.0.5", "127.0.0.6", "127.0.0.7", "127.0.0.8", "127.0.0.9", "127.0.1.1", "127.0.1.2"));
+        expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("45"), makeAddrs("127.0.0.6", "127.0.0.7", "127.0.0.8", "127.0.0.9", "127.0.0.10", "127.0.1.1", "127.0.1.2"));
+        expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("55"), makeAddrs("127.0.0.7", "127.0.0.8", "127.0.0.9", "127.0.0.10", "127.0.0.1", "127.0.1.1", "127.0.1.2"));
+        expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("65"), makeAddrs("127.0.0.7", "127.0.0.8", "127.0.0.9", "127.0.0.10", "127.0.0.1", "127.0.1.2"));
+        expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("75"), makeAddrs("127.0.0.9", "127.0.0.10", "127.0.0.1", "127.0.0.2", "127.0.0.3", "127.0.1.2"));
+        expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("85"), makeAddrs("127.0.0.9", "127.0.0.10", "127.0.0.1", "127.0.0.2", "127.0.0.3"));
+        expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("95"), makeAddrs("127.0.0.10", "127.0.0.1", "127.0.0.2", "127.0.0.3", "127.0.0.4"));
+        expectedEndpoints.put(Simple_RF3_KeyspaceName, HashMultimap.<Token, InetAddress>create());
+        expectedEndpoints.get(Simple_RF3_KeyspaceName).putAll(new BigIntegerToken("5"), makeAddrs("127.0.0.2", "127.0.0.3", "127.0.0.4"));
+        expectedEndpoints.get(Simple_RF3_KeyspaceName).putAll(new BigIntegerToken("15"), makeAddrs("127.0.0.3", "127.0.0.4", "127.0.0.5"));
+        expectedEndpoints.get(Simple_RF3_KeyspaceName).putAll(new BigIntegerToken("25"), makeAddrs("127.0.0.4", "127.0.0.5", "127.0.0.6"));
+        expectedEndpoints.get(Simple_RF3_KeyspaceName).putAll(new BigIntegerToken("35"), makeAddrs("127.0.0.5", "127.0.0.6", "127.0.0.7", "127.0.1.1"));
+        expectedEndpoints.get(Simple_RF3_KeyspaceName).putAll(new BigIntegerToken("45"), makeAddrs("127.0.0.6", "127.0.0.7", "127.0.0.8", "127.0.1.1"));
+        expectedEndpoints.get(Simple_RF3_KeyspaceName).putAll(new BigIntegerToken("55"), makeAddrs("127.0.0.7", "127.0.0.8", "127.0.0.9", "127.0.1.1", "127.0.1.2"));
+        expectedEndpoints.get(Simple_RF3_KeyspaceName).putAll(new BigIntegerToken("65"), makeAddrs("127.0.0.7", "127.0.0.8", "127.0.0.9", "127.0.1.2"));
+        expectedEndpoints.get(Simple_RF3_KeyspaceName).putAll(new BigIntegerToken("75"), makeAddrs("127.0.0.9", "127.0.0.10", "127.0.0.1", "127.0.1.2"));
+        expectedEndpoints.get(Simple_RF3_KeyspaceName).putAll(new BigIntegerToken("85"), makeAddrs("127.0.0.9", "127.0.0.10", "127.0.0.1"));
+        expectedEndpoints.get(Simple_RF3_KeyspaceName).putAll(new BigIntegerToken("95"), makeAddrs("127.0.0.10", "127.0.0.1", "127.0.0.2"));
 
         for (Map.Entry<String, AbstractReplicationStrategy> keyspaceStrategy : keyspaceStrategyMap.entrySet())
         {
@@ -930,8 +946,7 @@
     {
         StorageService ss = StorageService.instance;
         TokenMetadata tmd = ss.getTokenMetadata();
-        tmd.clearUnsafe();
-        IPartitioner partitioner = new RandomPartitioner();
+        IPartitioner partitioner = RandomPartitioner.instance;
         VersionedValue.VersionedValueFactory valueFactory = new VersionedValue.VersionedValueFactory(partitioner);
 
         ArrayList<Token> endpointTokens = new ArrayList<Token>();
diff --git a/test/unit/org/apache/cassandra/service/PaxosStateTest.java b/test/unit/org/apache/cassandra/service/PaxosStateTest.java
index d41d89f..7f4bc49 100644
--- a/test/unit/org/apache/cassandra/service/PaxosStateTest.java
+++ b/test/unit/org/apache/cassandra/service/PaxosStateTest.java
@@ -45,6 +45,7 @@
     public static void setUpClass() throws Throwable
     {
         SchemaLoader.loadSchema();
+        SchemaLoader.schemaDefinition("PaxosStateTest");
     }
 
     @AfterClass
@@ -56,7 +57,7 @@
     @Test
     public void testCommittingAfterTruncation() throws Exception
     {
-        ColumnFamilyStore cfs = Keyspace.open("Keyspace1").getColumnFamilyStore("Standard1");
+        ColumnFamilyStore cfs = Keyspace.open("PaxosStateTestKeyspace1").getColumnFamilyStore("Standard1");
         DecoratedKey key = Util.dk("key" + System.nanoTime());
         CellName name = Util.cellname("col");
         ByteBuffer value = ByteBufferUtil.bytes(0);
diff --git a/test/unit/org/apache/cassandra/service/QueryPagerTest.java b/test/unit/org/apache/cassandra/service/QueryPagerTest.java
index 1939c4a..33a7585 100644
--- a/test/unit/org/apache/cassandra/service/QueryPagerTest.java
+++ b/test/unit/org/apache/cassandra/service/QueryPagerTest.java
@@ -29,11 +29,15 @@
 import org.apache.cassandra.Util;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.OrderedJUnit4ClassRunner;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.db.filter.*;
 import org.apache.cassandra.db.marshal.CompositeType;
 import org.apache.cassandra.dht.*;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.service.pager.*;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
@@ -43,10 +47,39 @@
 import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
 
 @RunWith(OrderedJUnit4ClassRunner.class)
-public class QueryPagerTest extends SchemaLoader
+public class QueryPagerTest
 {
-    private static final String KS = "Keyspace1";
-    private static final String CF = "Standard1";
+    public static final String KEYSPACE1 = "QueryPagerTest";
+    public static final String CF_STANDARD = "Standard1";
+    public static final String KEYSPACE_CQL = "cql_keyspace";
+    public static final String CF_CQL = "table2";
+    public static final String CF_CQL_WITH_STATIC = "with_static";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD));
+        SchemaLoader.createKeyspace(KEYSPACE_CQL,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    CFMetaData.compile("CREATE TABLE " + CF_CQL + " ("
+                                                     + "k text,"
+                                                     + "c text,"
+                                                     + "v text,"
+                                                     + "PRIMARY KEY (k, c))", KEYSPACE_CQL),
+                                    CFMetaData.compile("CREATE TABLE " + CF_CQL_WITH_STATIC + " ("
+                                                     + "pk text, "
+                                                     + "ck int, "
+                                                     + "st int static, "
+                                                     + "v1 int, "
+                                                     + "v2 int, "
+                                                     + "PRIMARY KEY(pk, ck))", KEYSPACE_CQL));
+        addData();
+    }
 
     private static String string(CellName name)
     {
@@ -65,7 +98,6 @@
         }
     }
 
-    @BeforeClass
     public static void addData()
     {
         cfs().clearUnsafe();
@@ -81,8 +113,8 @@
          */
         for (int i = 0; i < nbKeys; i++)
         {
-            Mutation rm = new Mutation(KS, bytes("k" + i));
-            ColumnFamily cf = rm.addOrGet(CF);
+            Mutation rm = new Mutation(KEYSPACE1, bytes("k" + i));
+            ColumnFamily cf = rm.addOrGet(CF_STANDARD);
 
             for (int j = 0; j < nbCols; j++)
                 cf.addColumn(Util.column("c" + j, "", 0));
@@ -93,7 +125,7 @@
 
     private static ColumnFamilyStore cfs()
     {
-        return Keyspace.open(KS).getColumnFamilyStore(CF);
+        return Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD);
     }
 
     private static String toString(List<Row> rows)
@@ -120,7 +152,7 @@
         SortedSet<CellName> s = new TreeSet<CellName>(cfs().metadata.comparator);
         for (String name : names)
             s.add(CellNames.simpleDense(bytes(name)));
-        return new SliceByNamesReadCommand(KS, bytes(key), CF, System.currentTimeMillis(), new NamesQueryFilter(s, true));
+        return new SliceByNamesReadCommand(KEYSPACE1, bytes(key), CF_STANDARD, System.currentTimeMillis(), new NamesQueryFilter(s, true));
     }
 
     private static ReadCommand sliceQuery(String key, String start, String end, int count)
@@ -132,7 +164,7 @@
     {
         SliceQueryFilter filter = new SliceQueryFilter(CellNames.simpleDense(bytes(start)), CellNames.simpleDense(bytes(end)), reversed, count);
         // Note: for MultiQueryTest, we need the same timestamp/expireBefore for all queries, so we just use 0 as it doesn't matter here.
-        return new SliceFromReadCommand(KS, bytes(key), CF, 0, filter);
+        return new SliceFromReadCommand(KEYSPACE1, bytes(key), CF_STANDARD, 0, filter);
     }
 
     private static RangeSliceCommand rangeNamesQuery(AbstractBounds<RowPosition> range, int count, String... names)
@@ -140,13 +172,13 @@
         SortedSet<CellName> s = new TreeSet<CellName>(cfs().metadata.comparator);
         for (String name : names)
             s.add(CellNames.simpleDense(bytes(name)));
-        return new RangeSliceCommand(KS, CF, System.currentTimeMillis(), new NamesQueryFilter(s, true), range, count);
+        return new RangeSliceCommand(KEYSPACE1, CF_STANDARD, System.currentTimeMillis(), new NamesQueryFilter(s, true), range, count);
     }
 
     private static RangeSliceCommand rangeSliceQuery(AbstractBounds<RowPosition> range, int count, String start, String end)
     {
         SliceQueryFilter filter = new SliceQueryFilter(CellNames.simpleDense(bytes(start)), CellNames.simpleDense(bytes(end)), false, Integer.MAX_VALUE);
-        return new RangeSliceCommand(KS, CF, System.currentTimeMillis(), filter, range, null, count, true, false);
+        return new RangeSliceCommand(KEYSPACE1, CF_STANDARD, System.currentTimeMillis(), filter, range, null, count, true, false);
     }
 
     private static void assertRow(Row r, String key, String... names)
@@ -354,4 +386,46 @@
             assertRow(page.get(0), "k0", ct.decompose("c" + i, ""));
         }
     }
+
+    @Test
+    public void pagingReversedQueriesWithStaticColumnsTest() throws Exception
+    {
+        // insert some rows into a single partition
+        for (int i=0; i < 5; i++)
+            executeInternal(String.format("INSERT INTO %s.%s (pk, ck, st, v1, v2) VALUES ('k0', %3$s, %3$s, %3$s, %3$s)",
+                                          KEYSPACE_CQL, CF_CQL_WITH_STATIC, i));
+
+        // query the table in reverse with page size = 1 & check that the returned rows contain the correct cells
+        CFMetaData cfm = Keyspace.open(KEYSPACE_CQL).getColumnFamilyStore(CF_CQL_WITH_STATIC).metadata;
+        queryAndVerifyCells(cfm, true, "k0");
+    }
+
+    private void queryAndVerifyCells(CFMetaData cfm, boolean reversed, String key) throws Exception
+    {
+        SliceQueryFilter filter = new SliceQueryFilter(ColumnSlice.ALL_COLUMNS_ARRAY, reversed, 100, 1);
+        QueryPager pager = QueryPagers.localPager(new SliceFromReadCommand(cfm.ksName, bytes(key), cfm.cfName, 0, filter));
+        CellName staticCellName = cfm.comparator.create(cfm.comparator.staticPrefix(),
+                                                        cfm.staticColumns().iterator().next());
+        for (int i=0; i<5; i++)
+        {
+            List<Row> page = pager.fetchPage(1);
+            assertEquals(1, page.size());
+            Row row = page.get(0);
+            assertCell(row.cf, staticCellName, 4);
+            int cellIndex = !reversed ? i : 4 - i;
+            assertCell(row.cf, Util.cellname(ByteBufferUtil.bytes(cellIndex), ByteBufferUtil.bytes("v1")), cellIndex);
+            assertCell(row.cf, Util.cellname(ByteBufferUtil.bytes(cellIndex), ByteBufferUtil.bytes("v2")), cellIndex);
+        }
+
+        // After processing the 5 rows there should be no more rows to return
+        List<Row> page = pager.fetchPage(1);
+        assertTrue(page.isEmpty());
+    }
+
+    private void assertCell(ColumnFamily cf, CellName cellName, int value)
+    {
+        Cell cell = cf.getColumn(cellName);
+        assertNotNull(cell);
+        assertEquals(value, ByteBufferUtil.toInt(cell.value()));
+    }
 }
diff --git a/test/unit/org/apache/cassandra/service/RemoveTest.java b/test/unit/org/apache/cassandra/service/RemoveTest.java
index 649a3f2..800c904 100644
--- a/test/unit/org/apache/cassandra/service/RemoveTest.java
+++ b/test/unit/org/apache/cassandra/service/RemoveTest.java
@@ -35,11 +35,9 @@
 import org.apache.cassandra.dht.RandomPartitioner;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.gms.Gossiper;
 import org.apache.cassandra.locator.TokenMetadata;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.sink.SinkManager;
 import org.apache.cassandra.utils.FBUtilities;
 
 import static org.junit.Assert.assertEquals;
@@ -47,7 +45,7 @@
 
 public class RemoveTest
 {
-    static final IPartitioner partitioner = new RandomPartitioner();
+    static final IPartitioner partitioner = RandomPartitioner.instance;
     StorageService ss = StorageService.instance;
     TokenMetadata tmd = ss.getTokenMetadata();
     static IPartitioner oldPartitioner;
@@ -62,14 +60,13 @@
     public static void setupClass() throws ConfigurationException
     {
         oldPartitioner = StorageService.instance.setPartitionerUnsafe(partitioner);
-        SchemaLoader.loadSchema();
+        SchemaLoader.prepareServer();
     }
 
     @AfterClass
     public static void tearDownClass()
     {
         StorageService.instance.setPartitionerUnsafe(oldPartitioner);
-        SchemaLoader.stopGossiper();
     }
 
     @Before
@@ -80,8 +77,7 @@
         // create a ring of 5 nodes
         Util.createInitialRing(ss, partitioner, endpointTokens, keyTokens, hosts, hostIds, 6);
 
-        MessagingService.instance().listen(FBUtilities.getBroadcastAddress());
-        Gossiper.instance.start(1);
+        MessagingService.instance().listen();
         removalhost = hosts.get(5);
         hosts.remove(removalhost);
         removalId = hostIds.get(5);
@@ -91,7 +87,7 @@
     @After
     public void tearDown()
     {
-        SinkManager.clear();
+        MessagingService.instance().clearMessageSinks();
         MessagingService.instance().clearCallbacksUnsafe();
         MessagingService.instance().shutdown();
     }
diff --git a/test/unit/org/apache/cassandra/service/ResponseResolverTest.java b/test/unit/org/apache/cassandra/service/ResponseResolverTest.java
index 54e584d..7e42825 100644
--- a/test/unit/org/apache/cassandra/service/ResponseResolverTest.java
+++ b/test/unit/org/apache/cassandra/service/ResponseResolverTest.java
@@ -27,8 +27,13 @@
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
+
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.ColumnSlice;
@@ -44,9 +49,20 @@
 
 public class ResponseResolverTest extends SchemaLoader
 {
-    private final String KEYSPACE = "Keyspace1";
-    private final String TABLE = "Standard1";
+    private final static String KEYSPACE = "Keyspace1";
+    private final static String TABLE = "Standard1";
+    private final static int MAX_RESPONSE_COUNT = 3;
 
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(MAX_RESPONSE_COUNT),
+                                    SchemaLoader.standardCFMD(KEYSPACE, TABLE));
+    }
+    
     @Test
     public void testSingleMessage_RowDigestResolver() throws DigestMismatchException, UnknownHostException
     {
@@ -55,7 +71,7 @@
         cf.addColumn(column("c1", "v1", 0));
         Row row = new Row(key, cf);
 
-        testReadResponses(new RowDigestResolver(KEYSPACE, key), row, makeReadResponse("127.0.0.1", row));
+        testReadResponses(new RowDigestResolver(KEYSPACE, key, MAX_RESPONSE_COUNT), row, makeReadResponse("127.0.0.1", row));
     }
 
     @Test
@@ -66,7 +82,7 @@
         cf.addColumn(column("c1", "v1", 0));
         Row row = new Row(key, cf);
 
-        testReadResponses(new RowDigestResolver(KEYSPACE, key),
+        testReadResponses(new RowDigestResolver(KEYSPACE, key, MAX_RESPONSE_COUNT),
                           row,
                           makeReadResponse("127.0.0.1", row),
                           makeReadResponse("127.0.0.2", row),
@@ -85,7 +101,7 @@
         cf2.addColumn(column("c1", "v2", 1));
         Row row2 = new Row(key, cf2);
 
-        testReadResponses(new RowDigestResolver(KEYSPACE, key),
+        testReadResponses(new RowDigestResolver(KEYSPACE, key, MAX_RESPONSE_COUNT),
                           row1,
                           makeReadResponse("127.0.0.1", row1),
                           makeReadResponse("127.0.0.2", row2),
@@ -100,7 +116,7 @@
         cf.addColumn(column("c1", "v1", 0));
         Row row = new Row(key, cf);
 
-        testReadResponsesMT(new RowDigestResolver(KEYSPACE, key),
+        testReadResponsesMT(new RowDigestResolver(KEYSPACE, key, MAX_RESPONSE_COUNT),
                             row,
                             makeReadResponse("127.0.0.1", row),
                             makeReadResponse("127.0.0.2", row),
@@ -118,7 +134,8 @@
         testReadResponses(new RowDataResolver(KEYSPACE,
                                               key,
                                               new SliceQueryFilter(ColumnSlice.ALL_COLUMNS_ARRAY, false, 10),
-                                              System.currentTimeMillis()),
+                                              System.currentTimeMillis(),
+                                              MAX_RESPONSE_COUNT),
                           row,
                           makeReadResponse("127.0.0.1", row));
     }
@@ -134,7 +151,8 @@
         testReadResponses(new RowDataResolver(KEYSPACE,
                                               key,
                                               new SliceQueryFilter(ColumnSlice.ALL_COLUMNS_ARRAY, false, 10),
-                                              System.currentTimeMillis()),
+                                              System.currentTimeMillis(),
+                                              MAX_RESPONSE_COUNT),
                           row,
                           makeReadResponse("127.0.0.1", row),
                           makeReadResponse("127.0.0.2", row),
@@ -152,7 +170,8 @@
         testReadResponsesMT(new RowDataResolver(KEYSPACE,
                                                 key,
                                                 new SliceQueryFilter(ColumnSlice.ALL_COLUMNS_ARRAY, false, 10),
-                                                System.currentTimeMillis()),
+                                                System.currentTimeMillis(),
+                                                MAX_RESPONSE_COUNT),
                             row,
                             makeReadResponse("127.0.0.1", row),
                             makeReadResponse("127.0.0.2", row),
diff --git a/test/unit/org/apache/cassandra/service/RowResolverTest.java b/test/unit/org/apache/cassandra/service/RowResolverTest.java
index 286d037..825944c 100644
--- a/test/unit/org/apache/cassandra/service/RowResolverTest.java
+++ b/test/unit/org/apache/cassandra/service/RowResolverTest.java
@@ -23,26 +23,43 @@
 
 import java.util.Arrays;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.ArrayBackedSortedColumns;
 import org.apache.cassandra.db.ColumnFamily;
 import org.apache.cassandra.db.DeletionInfo;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 
 import static org.junit.Assert.*;
 import static org.apache.cassandra.Util.column;
 import static org.apache.cassandra.db.KeyspaceTest.*;
 
-public class RowResolverTest extends SchemaLoader
+public class RowResolverTest
 {
+    public static final String KEYSPACE1 = "Keyspace1";
+    public static final String CF_STANDARD = "Standard1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD));
+    }
+    
     @Test
     public void testResolveSupersetNewer()
     {
-        ColumnFamily cf1 = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf1 = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf1.addColumn(column("c1", "v1", 0));
 
-        ColumnFamily cf2 = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf2 = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf2.addColumn(column("c1", "v2", 1));
 
         ColumnFamily resolved = RowDataResolver.resolveSuperset(Arrays.asList(cf1, cf2), System.currentTimeMillis());
@@ -54,10 +71,10 @@
     @Test
     public void testResolveSupersetDisjoint()
     {
-        ColumnFamily cf1 = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf1 = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf1.addColumn(column("c1", "v1", 0));
 
-        ColumnFamily cf2 = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf2 = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf2.addColumn(column("c2", "v2", 1));
 
         ColumnFamily resolved = RowDataResolver.resolveSuperset(Arrays.asList(cf1, cf2), System.currentTimeMillis());
@@ -69,7 +86,7 @@
     @Test
     public void testResolveSupersetNullOne()
     {
-        ColumnFamily cf2 = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf2 = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf2.addColumn(column("c2", "v2", 1));
 
         ColumnFamily resolved = RowDataResolver.resolveSuperset(Arrays.asList(null, cf2), System.currentTimeMillis());
@@ -81,7 +98,7 @@
     @Test
     public void testResolveSupersetNullTwo()
     {
-        ColumnFamily cf1 = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf1 = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf1.addColumn(column("c1", "v1", 0));
 
         ColumnFamily resolved = RowDataResolver.resolveSuperset(Arrays.asList(cf1, null), System.currentTimeMillis());
@@ -100,10 +117,10 @@
     public void testResolveDeleted()
     {
         // one CF with columns timestamped before a delete in another cf
-        ColumnFamily cf1 = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf1 = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf1.addColumn(column("one", "A", 0));
 
-        ColumnFamily cf2 = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf2 = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf2.delete(new DeletionInfo(1L, (int) (System.currentTimeMillis() / 1000)));
 
         ColumnFamily resolved = RowDataResolver.resolveSuperset(Arrays.asList(cf1, cf2), System.currentTimeMillis());
@@ -118,19 +135,19 @@
     {
         // deletes and columns with interleaved timestamp, with out of order return sequence
 
-        ColumnFamily cf1 = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf1 = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf1.delete(new DeletionInfo(0L, (int) (System.currentTimeMillis() / 1000)));
 
         // these columns created after the previous deletion
-        ColumnFamily cf2 = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf2 = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf2.addColumn(column("one", "A", 1));
         cf2.addColumn(column("two", "A", 1));
 
         //this column created after the next delete
-        ColumnFamily cf3 = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf3 = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf3.addColumn(column("two", "B", 3));
 
-        ColumnFamily cf4 = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf4 = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf4.delete(new DeletionInfo(2L, (int) (System.currentTimeMillis() / 1000)));
 
         ColumnFamily resolved = RowDataResolver.resolveSuperset(Arrays.asList(cf1, cf2, cf3, cf4), System.currentTimeMillis());
diff --git a/test/unit/org/apache/cassandra/service/SerializationsTest.java b/test/unit/org/apache/cassandra/service/SerializationsTest.java
index 49632f9..5d2b74d 100644
--- a/test/unit/org/apache/cassandra/service/SerializationsTest.java
+++ b/test/unit/org/apache/cassandra/service/SerializationsTest.java
@@ -19,20 +19,18 @@
 package org.apache.cassandra.service;
 
 import java.io.DataInputStream;
-import java.io.DataOutputStream;
 import java.io.IOException;
 import java.net.InetAddress;
 import java.util.Collections;
 import java.util.UUID;
 
 import org.junit.Test;
-
 import org.apache.cassandra.AbstractSerializationsTester;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.RandomPartitioner;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.io.util.DataOutputStreamAndChannel;
+import org.apache.cassandra.io.util.DataOutputStreamPlus;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.repair.NodePair;
@@ -55,7 +53,7 @@
 
     private void testRepairMessageWrite(String fileName, RepairMessage... messages) throws IOException
     {
-        try (DataOutputStreamAndChannel out = getOutput(fileName))
+        try (DataOutputStreamPlus out = getOutput(fileName))
         {
             for (RepairMessage message : messages)
             {
@@ -93,7 +91,7 @@
 
     private void testValidationCompleteWrite() throws IOException
     {
-        IPartitioner p = new RandomPartitioner();
+        IPartitioner p = RandomPartitioner.instance;
         // empty validation
         MerkleTree mt = new MerkleTree(p, FULL_RANGE, MerkleTree.RECOMMENDED_DEPTH, (int) Math.pow(2, 15));
         Validator v0 = new Validator(DESC, FBUtilities.getBroadcastAddress(),  -1);
diff --git a/test/unit/org/apache/cassandra/service/StartupChecksTest.java b/test/unit/org/apache/cassandra/service/StartupChecksTest.java
new file mode 100644
index 0000000..d32b1b1
--- /dev/null
+++ b/test/unit/org/apache/cassandra/service/StartupChecksTest.java
@@ -0,0 +1,139 @@
+/*
+ * 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.service;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+
+import org.junit.*;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.exceptions.StartupException;
+import org.apache.cassandra.io.util.FileUtils;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class StartupChecksTest
+{
+    public static final String INVALID_LEGACY_SSTABLE_ROOT_PROP = "invalid-legacy-sstable-root";
+    StartupChecks startupChecks;
+    Path sstableDir;
+
+    @BeforeClass
+    public static void setupServer()
+    {
+        SchemaLoader.prepareServer();
+    }
+
+    @Before
+    public void setup() throws IOException
+    {
+        for (ColumnFamilyStore cfs : Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStores())
+            cfs.clearUnsafe();
+        for (File dataDir : Directories.getKSChildDirectories(SystemKeyspace.NAME))
+            FileUtils.deleteRecursive(dataDir);
+
+        File dataDir = new File(DatabaseDescriptor.getAllDataFileLocations()[0]);
+        sstableDir = Paths.get(dataDir.getAbsolutePath(), "Keyspace1", "Standard1");
+        Files.createDirectories(sstableDir);
+
+        startupChecks = new StartupChecks();
+    }
+
+    @After
+    public void tearDown() throws IOException
+    {
+        FileUtils.deleteRecursive(sstableDir.toFile());
+    }
+
+    @Test
+    public void failStartupIfInvalidSSTablesFound() throws Exception
+    {
+        startupChecks = startupChecks.withTest(StartupChecks.checkSSTablesFormat);
+
+        copyInvalidLegacySSTables(sstableDir);
+
+        verifyFailure(startupChecks, "Detected unreadable sstables");
+
+        // we should ignore invalid sstables in a snapshots directory
+        FileUtils.deleteRecursive(sstableDir.toFile());
+        Path snapshotDir = sstableDir.resolve("snapshots");
+        Files.createDirectories(snapshotDir);
+        copyInvalidLegacySSTables(snapshotDir); startupChecks.verify();
+
+        // and in a backups directory
+        FileUtils.deleteRecursive(sstableDir.toFile());
+        Path backupDir = sstableDir.resolve("backups");
+        Files.createDirectories(backupDir);
+        copyInvalidLegacySSTables(backupDir);
+        startupChecks.verify();
+    }
+
+    @Test
+    public void compatibilityCheckIgnoresNonDbFiles() throws Exception
+    {
+        startupChecks = startupChecks.withTest(StartupChecks.checkSSTablesFormat);
+
+        copyLegacyNonSSTableFiles(sstableDir);
+        assertFalse(sstableDir.toFile().listFiles().length == 0);
+
+        startupChecks.verify();
+    }
+
+    private void copyLegacyNonSSTableFiles(Path targetDir) throws IOException
+    {
+
+        Path legacySSTableRoot = Paths.get(System.getProperty(INVALID_LEGACY_SSTABLE_ROOT_PROP),
+                                          "Keyspace1",
+                                          "Standard1");
+        for (String filename : new String[]{"Keyspace1-Standard1-ic-0-TOC.txt",
+                                            "Keyspace1-Standard1-ic-0-Digest.sha1",
+                                            "legacyleveled.json"})
+            Files.copy(Paths.get(legacySSTableRoot.toString(), filename), targetDir.resolve(filename));
+    }
+
+    private void copyInvalidLegacySSTables(Path targetDir) throws IOException
+    {
+        File legacySSTableRoot = Paths.get(System.getProperty(INVALID_LEGACY_SSTABLE_ROOT_PROP),
+                                           "Keyspace1",
+                                           "Standard1").toFile();
+        for (File f : legacySSTableRoot.listFiles())
+            Files.copy(f.toPath(), targetDir.resolve(f.getName()));
+
+    }
+
+    private void verifyFailure(StartupChecks tests, String message)
+    {
+        try
+        {
+            tests.verify();
+            fail("Expected a startup exception but none was thrown");
+        }
+        catch (StartupException e)
+        {
+            assertTrue(e.getMessage().contains(message));
+        }
+    }
+}
diff --git a/test/unit/org/apache/cassandra/service/StorageProxyTest.java b/test/unit/org/apache/cassandra/service/StorageProxyTest.java
index 6fbc80a..c8afac0 100644
--- a/test/unit/org/apache/cassandra/service/StorageProxyTest.java
+++ b/test/unit/org/apache/cassandra/service/StorageProxyTest.java
@@ -25,7 +25,6 @@
 import org.junit.Test;
 import static org.junit.Assert.assertEquals;
 
-import org.apache.cassandra.SchemaLoader;
 import static org.apache.cassandra.Util.token;
 import static org.apache.cassandra.Util.rp;
 
@@ -39,7 +38,7 @@
 import org.apache.cassandra.locator.TokenMetadata;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
-public class StorageProxyTest extends SchemaLoader
+public class StorageProxyTest
 {
     private static Range<RowPosition> range(RowPosition left, RowPosition right)
     {
diff --git a/test/unit/org/apache/cassandra/service/StorageServiceClientTest.java b/test/unit/org/apache/cassandra/service/StorageServiceClientTest.java
deleted file mode 100644
index f3ba754..0000000
--- a/test/unit/org/apache/cassandra/service/StorageServiceClientTest.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
-* 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.service;
-
-import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.junit.Test;
-import static org.junit.Assert.assertFalse;
-
-import java.io.File;
-
-public class StorageServiceClientTest
-{
-    @Test
-    public void testClientOnlyMode() throws ConfigurationException
-    {
-        SchemaLoader.mkdirs();
-        SchemaLoader.cleanup();
-        StorageService.instance.initClient(0);
-
-        // verify that no storage directories were created.
-        for (String path : DatabaseDescriptor.getAllDataFileLocations())
-        {
-            assertFalse(new File(path).exists());
-        }
-        StorageService.instance.stopClient();
-    }
-}
diff --git a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
index 564239b..a693a23 100644
--- a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
+++ b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
@@ -20,16 +20,15 @@
 package org.apache.cassandra.service;
 
 import java.io.File;
+import java.io.FileWriter;
 import java.io.IOException;
+import java.io.PrintWriter;
 import java.net.InetAddress;
 import java.util.*;
 
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.Multimap;
 
-import org.apache.cassandra.dht.BigIntegerToken;
-import org.apache.cassandra.dht.LongToken;
-import org.apache.cassandra.dht.Murmur3Partitioner;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -40,17 +39,23 @@
 import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.WindowsFailedSnapshotTracker;
 import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.dht.StringToken;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.dht.OrderPreservingPartitioner.StringToken;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.dht.Murmur3Partitioner.LongToken;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.locator.IEndpointSnitch;
 import org.apache.cassandra.locator.PropertyFileSnitch;
 import org.apache.cassandra.locator.TokenMetadata;
+import org.apache.cassandra.schema.LegacySchemaTables;
+import org.apache.cassandra.utils.FBUtilities;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assume.assumeTrue;
 
 @RunWith(OrderedJUnit4ClassRunner.class)
 public class StorageServiceServerTest
@@ -58,6 +63,7 @@
     @BeforeClass
     public static void setUp() throws ConfigurationException
     {
+        DatabaseDescriptor.setDaemonInitialized();
         IEndpointSnitch snitch = new PropertyFileSnitch();
         DatabaseDescriptor.setEndpointSnitch(snitch);
         Keyspace.setInitialized();
@@ -95,11 +101,80 @@
         StorageService.instance.takeSnapshot("snapshot");
     }
 
+    private void checkTempFilePresence(File f, boolean exist)
+    {
+        for (int i = 0; i < 5; i++)
+        {
+            File subdir = new File(f, Integer.toString(i));
+            subdir.mkdir();
+            for (int j = 0; j < 5; j++)
+            {
+                File subF = new File(subdir, Integer.toString(j));
+                assert(exist ? subF.exists() : !subF.exists());
+            }
+        }
+    }
+
+    @Test
+    public void testSnapshotFailureHandler() throws IOException
+    {
+        assumeTrue(FBUtilities.isWindows());
+
+        // Initial "run" of Cassandra, nothing in failed snapshot file
+        WindowsFailedSnapshotTracker.deleteOldSnapshots();
+
+        File f = new File(System.getenv("TEMP") + File.separator + Integer.toString(new Random().nextInt()));
+        f.mkdir();
+        f.deleteOnExit();
+        for (int i = 0; i < 5; i++)
+        {
+            File subdir = new File(f, Integer.toString(i));
+            subdir.mkdir();
+            for (int j = 0; j < 5; j++)
+                new File(subdir, Integer.toString(j)).createNewFile();
+        }
+
+        checkTempFilePresence(f, true);
+
+        // Confirm deletion is recursive
+        for (int i = 0; i < 5; i++)
+            WindowsFailedSnapshotTracker.handleFailedSnapshot(new File(f, Integer.toString(i)));
+
+        assert new File(WindowsFailedSnapshotTracker.TODELETEFILE).exists();
+
+        // Simulate shutdown and restart of C* node, closing out the list of failed snapshots.
+        WindowsFailedSnapshotTracker.resetForTests();
+
+        // Perform new run, mimicking behavior of C* at startup
+        WindowsFailedSnapshotTracker.deleteOldSnapshots();
+        checkTempFilePresence(f, false);
+
+        // Check to make sure we don't delete non-temp, non-datafile locations
+        WindowsFailedSnapshotTracker.resetForTests();
+        PrintWriter tempPrinter = new PrintWriter(new FileWriter(WindowsFailedSnapshotTracker.TODELETEFILE, true));
+        tempPrinter.println(".safeDir");
+        tempPrinter.close();
+
+        File protectedDir = new File(".safeDir");
+        protectedDir.mkdir();
+        File protectedFile = new File(protectedDir, ".safeFile");
+        protectedFile.createNewFile();
+
+        WindowsFailedSnapshotTracker.handleFailedSnapshot(protectedDir);
+        WindowsFailedSnapshotTracker.deleteOldSnapshots();
+
+        assert protectedDir.exists();
+        assert protectedFile.exists();
+
+        protectedFile.delete();
+        protectedDir.delete();
+    }
+
     @Test
     public void testColumnFamilySnapshot() throws IOException
     {
         // no need to insert extra data, even an "empty" database will have a little information in the system keyspace
-        StorageService.instance.takeColumnFamilySnapshot(Keyspace.SYSTEM_KS, SystemKeyspace.SCHEMA_KEYSPACES_CF, "cf_snapshot");
+        StorageService.instance.takeColumnFamilySnapshot(SystemKeyspace.NAME, LegacySchemaTables.KEYSPACES, "cf_snapshot");
     }
 
     @Test
@@ -466,7 +541,7 @@
     @Test
     public void testCreateRepairRangeFrom() throws Exception
     {
-        StorageService.instance.setPartitionerUnsafe(new Murmur3Partitioner());
+        StorageService.instance.setPartitionerUnsafe(Murmur3Partitioner.instance);
 
         TokenMetadata metadata = StorageService.instance.getTokenMetadata();
         metadata.clearUnsafe();
@@ -476,8 +551,6 @@
         metadata.updateNormalToken(new LongToken(3000L), InetAddress.getByName("127.0.0.3"));
         metadata.updateNormalToken(new LongToken(4000L), InetAddress.getByName("127.0.0.4"));
 
-        Map<String, String> configOptions = new HashMap<String, String>();
-        configOptions.put("replication_factor", "3");
         Collection<Range<Token>> repairRangeFrom = StorageService.instance.createRepairRangeFrom("1500", "3700");
         assert repairRangeFrom.size() == 3;
         assert repairRangeFrom.contains(new Range<Token>(new LongToken(1500L), new LongToken(2000L)));
@@ -508,10 +581,4 @@
         repairRangeFrom = StorageService.instance.createRepairRangeFrom("2000", "2000");
         assert repairRangeFrom.size() == 0;
     }
-
-    @Test(expected=IllegalArgumentException.class)
-    public void testIncrementalRepairWithSubrangesThrows() throws Exception
-    {
-        StorageService.instance.forceRepairRangeAsync("",  "", "", true,  true, false, "");
-    }
 }
diff --git a/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java b/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java
index 306afc0..02af9a7 100644
--- a/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java
+++ b/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java
@@ -19,42 +19,69 @@
 
 import java.net.InetAddress;
 import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedList;
 import java.util.List;
+import java.util.UUID;
 import java.util.concurrent.CancellationException;
 import java.util.concurrent.Future;
-import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.junit.BeforeClass;
+import org.junit.After;
 import org.junit.Test;
 
 import junit.framework.Assert;
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.streaming.messages.OutgoingFileMessage;
 import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.concurrent.Ref;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 
-public class StreamTransferTaskTest extends SchemaLoader
+public class StreamTransferTaskTest
 {
+    public static final String KEYSPACE1 = "StreamTransferTaskTest";
+    public static final String CF_STANDARD = "Standard1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD));
+    }
+
+    @After
+    public void tearDown()
+    {
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD);
+        cfs.clearUnsafe();
+    }
+
     @Test
     public void testScheduleTimeout() throws Exception
     {
-        String ks = "Keyspace1";
-        String cf = "Standard1";
-
         InetAddress peer = FBUtilities.getBroadcastAddress();
-        StreamSession session = new StreamSession(peer, peer, null, 0);
-        ColumnFamilyStore cfs = Keyspace.open(ks).getColumnFamilyStore(cf);
+        StreamSession session = new StreamSession(peer, peer, null, 0, true, false);
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD);
 
         // create two sstables
         for (int i = 0; i < 2; i++)
         {
-            insertData(ks, cf, i, 1);
+            SchemaLoader.insertData(KEYSPACE1, CF_STANDARD, i, 1);
             cfs.forceBlockingFlush();
         }
 
@@ -88,4 +115,68 @@
         // when all streaming are done, time out task should not be scheduled.
         assertNull(task.scheduleTimeout(1, 1, TimeUnit.SECONDS));
     }
+
+    @Test
+    public void testFailSessionDuringTransferShouldNotReleaseReferences() throws Exception
+    {
+        InetAddress peer = FBUtilities.getBroadcastAddress();
+        StreamCoordinator streamCoordinator = new StreamCoordinator(1, true, false, null);
+        StreamResultFuture future = StreamResultFuture.init(UUID.randomUUID(), "", Collections.<StreamEventHandler>emptyList(), streamCoordinator);
+        StreamSession session = new StreamSession(peer, peer, null, 0, true, false);
+        session.init(future);
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD);
+
+        // create two sstables
+        for (int i = 0; i < 2; i++)
+        {
+            SchemaLoader.insertData(KEYSPACE1, CF_STANDARD, i, 1);
+            cfs.forceBlockingFlush();
+        }
+
+        // create streaming task that streams those two sstables
+        StreamTransferTask task = new StreamTransferTask(session, cfs.metadata.cfId);
+        List<Ref<SSTableReader>> refs = new ArrayList<>(cfs.getSSTables().size());
+        for (SSTableReader sstable : cfs.getSSTables())
+        {
+            List<Range<Token>> ranges = new ArrayList<>();
+            ranges.add(new Range<>(sstable.first.getToken(), sstable.last.getToken()));
+            Ref<SSTableReader> ref = sstable.selfRef();
+            refs.add(ref);
+            task.addTransferFile(ref, 1, sstable.getPositionsForRanges(ranges), 0);
+        }
+        assertEquals(2, task.getTotalNumberOfFiles());
+
+        //add task to stream session, so it is aborted when stream session fails
+        session.transfers.put(UUID.randomUUID(), task);
+
+        //make a copy of outgoing file messages, since task is cleared when it's aborted
+        Collection<OutgoingFileMessage> files = new LinkedList<>(task.files.values());
+
+        //simulate start transfer
+        for (OutgoingFileMessage file : files)
+        {
+            file.startTransfer();
+        }
+
+        //fail stream session mid-transfer
+        session.onError(new Exception("Fake exception"));
+
+        //make sure reference was not released
+        for (Ref<SSTableReader> ref : refs)
+        {
+            assertEquals(1, ref.globalCount());
+        }
+
+        //simulate finish transfer
+        for (OutgoingFileMessage file : files)
+        {
+            file.finishTransfer();
+        }
+
+        //now reference should be released
+        for (Ref<SSTableReader> ref : refs)
+        {
+            assertEquals(0, ref.globalCount());
+        }
+    }
 }
diff --git a/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java b/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
index abff812..0af76c2 100644
--- a/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
+++ b/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
@@ -26,6 +26,7 @@
 import com.google.common.collect.Iterables;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -37,7 +38,9 @@
 import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.cql3.Operator;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.columniterator.IdentityQueryFilter;
@@ -47,11 +50,14 @@
 import org.apache.cassandra.db.context.CounterContext;
 import org.apache.cassandra.db.filter.IDiskAtomFilter;
 import org.apache.cassandra.db.filter.QueryFilter;
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.CounterColumnType;
+import org.apache.cassandra.db.marshal.IntegerType;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.io.sstable.SSTableUtils;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -66,16 +72,42 @@
 import static org.apache.cassandra.Util.column;
 
 @RunWith(OrderedJUnit4ClassRunner.class)
-public class StreamingTransferTest extends SchemaLoader
+public class StreamingTransferTest
 {
     private static final Logger logger = LoggerFactory.getLogger(StreamingTransferTest.class);
 
     public static final InetAddress LOCAL = FBUtilities.getBroadcastAddress();
+    public static final String KEYSPACE1 = "StreamingTransferTest1";
+    public static final String CF_STANDARD = "Standard1";
+    public static final String CF_COUNTER = "Counter1";
+    public static final String CF_STANDARDINT = "StandardInteger1";
+    public static final String CF_INDEX = "Indexed1";
+    public static final String KEYSPACE_CACHEKEY = "KeyStreamingTransferTestSpace";
+    public static final String CF_STANDARD2 = "Standard2";
+    public static final String CF_STANDARD3 = "Standard3";
+    public static final String KEYSPACE2 = "StreamingTransferTest2";
 
     @BeforeClass
-    public static void setup() throws Exception
+    public static void defineSchema() throws Exception
     {
+        SchemaLoader.prepareServer();
         StorageService.instance.initServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_COUNTER).defaultValidator(CounterColumnType.instance),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARDINT, IntegerType.instance),
+                                    SchemaLoader.indexCFMD(KEYSPACE1, CF_INDEX, true));
+        SchemaLoader.createKeyspace(KEYSPACE2,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1));
+        SchemaLoader.createKeyspace(KEYSPACE_CACHEKEY,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE_CACHEKEY, CF_STANDARD),
+                                    SchemaLoader.standardCFMD(KEYSPACE_CACHEKEY, CF_STANDARD2),
+                                    SchemaLoader.standardCFMD(KEYSPACE_CACHEKEY, CF_STANDARD3));
     }
 
     /**
@@ -114,7 +146,7 @@
         ranges.add(new Range<>(p.getToken(ByteBufferUtil.bytes("key2")), p.getMinimumToken()));
 
         StreamResultFuture futureResult = new StreamPlan("StreamingTransferTest")
-                                                  .requestRanges(LOCAL, LOCAL, "Keyspace2", ranges)
+                                                  .requestRanges(LOCAL, LOCAL, KEYSPACE2, ranges)
                                                   .execute();
 
         UUID planId = futureResult.planId;
@@ -139,7 +171,7 @@
     private List<String> createAndTransfer(ColumnFamilyStore cfs, Mutator mutator, boolean transferSSTables) throws Exception
     {
         // write a temporary SSTable, and unregister it
-        logger.debug("Mutating " + cfs.name);
+        logger.debug("Mutating {}", cfs.name);
         long timestamp = 1234;
         for (int i = 1; i <= 3; i++)
             mutator.mutate("key" + i, "col" + i, timestamp);
@@ -148,7 +180,7 @@
         assertEquals(1, cfs.getSSTables().size());
 
         // transfer the first and last key
-        logger.debug("Transferring " + cfs.name);
+        logger.debug("Transferring {}", cfs.name);
         int[] offs;
         if (transferSSTables)
         {
@@ -187,7 +219,7 @@
         for (int off : offs)
             keys.add("key" + off);
 
-        logger.debug("... everything looks good for " + cfs.name);
+        logger.debug("... everything looks good for {}", cfs.name);
         return keys;
     }
 
@@ -223,7 +255,7 @@
     {
         //after stream session is finished, message handlers may take several milliseconds to be closed
         outer:
-        for (int i = 0; i <= 10; i++)
+        for (int i = 0; i <= 100; i++)
         {
             for (MessagingService.SocketThread socketThread : MessagingService.instance().getSocketThreads())
                 if (!socketThread.connections.isEmpty())
@@ -250,7 +282,7 @@
 
     private void doTransferTable(boolean transferSSTables) throws Exception
     {
-        final Keyspace keyspace = Keyspace.open("Keyspace1");
+        final Keyspace keyspace = Keyspace.open(KEYSPACE1);
         final ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Indexed1");
 
         List<String> keys = createAndTransfer(cfs, new Mutator()
@@ -261,9 +293,9 @@
                 ColumnFamily cf = ArrayBackedSortedColumns.factory.create(keyspace.getName(), cfs.name);
                 cf.addColumn(column(col, "v", timestamp));
                 cf.addColumn(new BufferCell(cellname("birthdate"), ByteBufferUtil.bytes(val), timestamp));
-                Mutation rm = new Mutation("Keyspace1", ByteBufferUtil.bytes(key), cf);
-                logger.debug("Applying row to transfer " + rm);
-                rm.apply();
+                Mutation rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes(key), cf);
+                logger.debug("Applying row to transfer {}", rm);
+                rm.applyUnsafe();
             }
         }, transferSSTables);
 
@@ -289,7 +321,7 @@
     @Test
     public void testTransferRangeTombstones() throws Exception
     {
-        String ks = "Keyspace1";
+        String ks = KEYSPACE1;
         String cfname = "StandardInteger1";
         Keyspace keyspace = Keyspace.open(ks);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);
@@ -304,7 +336,7 @@
         cf.delete(new DeletionInfo(cellname(2), cellname(3), cf.getComparator(), 1, (int) (System.currentTimeMillis() / 1000)));
         cf.delete(new DeletionInfo(cellname(5), cellname(7), cf.getComparator(), 1, (int) (System.currentTimeMillis() / 1000)));
         cf.delete(new DeletionInfo(cellname(8), cellname(10), cf.getComparator(), 1, (int) (System.currentTimeMillis() / 1000)));
-        rm.apply();
+        rm.applyUnsafe();
 
         key = "key1";
         rm = new Mutation(ks, ByteBufferUtil.bytes(key));
@@ -313,7 +345,7 @@
         cf = rm.addOrGet(cfname);
         // add RangeTombstones
         cf.delete(new DeletionInfo(cellname(2), cellname(3), cf.getComparator(), 1, (int) (System.currentTimeMillis() / 1000)));
-        rm.apply();
+        rm.applyUnsafe();
 
         cfs.forceBlockingFlush();
 
@@ -366,7 +398,7 @@
     @Test
     public void testTransferTableCounter() throws Exception
     {
-        final Keyspace keyspace = Keyspace.open("Keyspace1");
+        final Keyspace keyspace = Keyspace.open(KEYSPACE1);
         final ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Counter1");
         final CounterContext cc = new CounterContext();
 
@@ -423,7 +455,7 @@
         content.add("test");
         content.add("test2");
         content.add("test3");
-        SSTableReader sstable = SSTableUtils.prepare().write(content);
+        SSTableReader sstable = new SSTableUtils(KEYSPACE1, CF_STANDARD).prepare().write(content);
         String keyspaceName = sstable.getKeyspaceName();
         String cfname = sstable.getColumnFamilyName();
 
@@ -462,7 +494,7 @@
     @Test
     public void testTransferOfMultipleColumnFamilies() throws Exception
     {
-        String keyspace = "KeyCacheSpace";
+        String keyspace = KEYSPACE_CACHEKEY;
         IPartitioner p = StorageService.getPartitioner();
         String[] columnFamilies = new String[] { "Standard1", "Standard2", "Standard3" };
         List<SSTableReader> ssTableReaders = new ArrayList<>();
@@ -511,7 +543,7 @@
     @Test
     public void testRandomSSTableTransfer() throws Exception
     {
-        final Keyspace keyspace = Keyspace.open("Keyspace1");
+        final Keyspace keyspace = Keyspace.open(KEYSPACE1);
         final ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
         Mutator mutator = new Mutator()
         {
@@ -520,9 +552,9 @@
                 ColumnFamily cf = ArrayBackedSortedColumns.factory.create(keyspace.getName(), cfs.name);
                 cf.addColumn(column(colName, "value", timestamp));
                 cf.addColumn(new BufferCell(cellname("birthdate"), ByteBufferUtil.bytes(new Date(timestamp).toString()), timestamp));
-                Mutation rm = new Mutation("Keyspace1", ByteBufferUtil.bytes(key), cf);
-                logger.debug("Applying row to transfer " + rm);
-                rm.apply();
+                Mutation rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes(key), cf);
+                logger.debug("Applying row to transfer {}", rm);
+                rm.applyUnsafe();
             }
         };
         // write a lot more data so the data is spread in more than 1 chunk.
diff --git a/test/unit/org/apache/cassandra/streaming/compress/CompressedInputStreamTest.java b/test/unit/org/apache/cassandra/streaming/compress/CompressedInputStreamTest.java
index c70b932..87d93fd 100644
--- a/test/unit/org/apache/cassandra/streaming/compress/CompressedInputStreamTest.java
+++ b/test/unit/org/apache/cassandra/streaming/compress/CompressedInputStreamTest.java
@@ -36,6 +36,7 @@
 import org.apache.cassandra.utils.Pair;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
 
 /**
  */
@@ -44,24 +45,33 @@
     @Test
     public void testCompressedRead() throws Exception
     {
-        testCompressedReadWith(new long[]{0L}, false);
-        testCompressedReadWith(new long[]{1L}, false);
-        testCompressedReadWith(new long[]{100L}, false);
+        testCompressedReadWith(new long[]{0L}, false, false);
+        testCompressedReadWith(new long[]{1L}, false, false);
+        testCompressedReadWith(new long[]{100L}, false, false);
 
-        testCompressedReadWith(new long[]{1L, 122L, 123L, 124L, 456L}, false);
+        testCompressedReadWith(new long[]{1L, 122L, 123L, 124L, 456L}, false, false);
     }
 
     @Test(expected = EOFException.class)
     public void testTruncatedRead() throws Exception
     {
-        testCompressedReadWith(new long[]{1L, 122L, 123L, 124L, 456L}, true);
+        testCompressedReadWith(new long[]{1L, 122L, 123L, 124L, 456L}, true, false);
+    }
+
+    /**
+     * Test that CompressedInputStream does not block if there's an exception while reading stream
+     */
+    @Test(timeout = 30000)
+    public void testException() throws Exception
+    {
+        testCompressedReadWith(new long[]{1L, 122L, 123L, 124L, 456L}, false, true);
     }
 
     /**
      * @param valuesToCheck array of longs of range(0-999)
      * @throws Exception
      */
-    private void testCompressedReadWith(long[] valuesToCheck, boolean testTruncate) throws Exception
+    private void testCompressedReadWith(long[] valuesToCheck, boolean testTruncate, boolean testException) throws Exception
     {
         assert valuesToCheck != null && valuesToCheck.length > 0;
 
@@ -78,7 +88,7 @@
                 index.put(l, writer.getFilePointer());
                 writer.stream.writeLong(l);
             }
-            writer.close();
+            writer.finish();
         }
 
         CompressionMetadata comp = CompressionMetadata.create(tmp.getAbsolutePath());
@@ -120,7 +130,13 @@
 
         // read buffer using CompressedInputStream
         CompressionInfo info = new CompressionInfo(chunks, param);
-        CompressedInputStream input = new CompressedInputStream(new ByteArrayInputStream(toRead), info, true);
+
+        if (testException)
+        {
+            testException(sections, info);
+            return;
+        }
+        CompressedInputStream input = new CompressedInputStream(new ByteArrayInputStream(toRead), info);
 
         try (DataInputStream in = new DataInputStream(input))
         {
@@ -132,4 +148,25 @@
             }
         }
     }
+
+    private static void testException(List<Pair<Long, Long>> sections, CompressionInfo info) throws IOException
+    {
+        CompressedInputStream input = new CompressedInputStream(new ByteArrayInputStream(new byte[0]), info);
+
+        try (DataInputStream in = new DataInputStream(input))
+        {
+            for (int i = 0; i < sections.size(); i++)
+            {
+                input.position(sections.get(i).left);
+                try {
+                    in.readLong();
+                    fail("Should have thrown IOException");
+                }
+                catch (IOException e)
+                {
+                    continue;
+                }
+            }
+        }
+    }
 }
diff --git a/test/unit/org/apache/cassandra/thrift/MultiSliceTest.java b/test/unit/org/apache/cassandra/thrift/MultiSliceTest.java
index 50e409e..9716876 100644
--- a/test/unit/org/apache/cassandra/thrift/MultiSliceTest.java
+++ b/test/unit/org/apache/cassandra/thrift/MultiSliceTest.java
@@ -28,25 +28,33 @@
 import junit.framework.Assert;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.service.EmbeddedCassandraService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.thrift.TException;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-public class MultiSliceTest extends SchemaLoader
+public class MultiSliceTest
 {
     private static CassandraServer server;
-    
+    public static final String KEYSPACE1 = "MultiSliceTest";
+    public static final String CF_STANDARD = "Standard1";
+
     @BeforeClass
-    public static void setup() throws IOException, TException 
+    public static void defineSchema() throws ConfigurationException, IOException, TException
     {
-        Schema.instance.clear(); // Schema are now written on disk and will be reloaded
+        SchemaLoader.prepareServer();
         new EmbeddedCassandraService().start();
-        ThriftSessionManager.instance.setCurrentSocket(new InetSocketAddress(9160));        
+        ThriftSessionManager.instance.setCurrentSocket(new InetSocketAddress(9160));
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD));
         server = new CassandraServer();
-        server.set_keyspace("Keyspace1");
+        server.set_keyspace(KEYSPACE1);
     }
 
     private static MultiSliceRequest makeMultiSliceRequest(ByteBuffer key)
diff --git a/test/unit/org/apache/cassandra/thrift/ThriftValidationTest.java b/test/unit/org/apache/cassandra/thrift/ThriftValidationTest.java
index 744fd89..c693f7c 100644
--- a/test/unit/org/apache/cassandra/thrift/ThriftValidationTest.java
+++ b/test/unit/org/apache/cassandra/thrift/ThriftValidationTest.java
@@ -20,41 +20,61 @@
  *
  */
 
-import org.apache.cassandra.db.marshal.LongType;
-import org.apache.cassandra.exceptions.*;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.*;
-import org.apache.cassandra.db.marshal.AsciiType;
+import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.locator.LocalStrategy;
 import org.apache.cassandra.locator.NetworkTopologyStrategy;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-
-import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
 
-public class ThriftValidationTest extends SchemaLoader
+public class ThriftValidationTest
 {
+    public static final String KEYSPACE1 = "MultiSliceTest";
+    public static final String CF_STANDARD = "Standard1";
+    public static final String CF_COUNTER = "Counter1";
+    public static final String CF_UUID = "UUIDKeys";
+    public static final String CF_STANDARDLONG3 = "StandardLong3";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_COUNTER).defaultValidator(CounterColumnType.instance),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_UUID).keyValidator(UUIDType.instance),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARDLONG3, IntegerType.instance));
+    }
+    
     @Test(expected=org.apache.cassandra.exceptions.InvalidRequestException.class)
     public void testValidateCommutativeWithStandard() throws org.apache.cassandra.exceptions.InvalidRequestException
     {
-        ThriftValidation.validateColumnFamily("Keyspace1", "Standard1", true);
+        ThriftValidation.validateColumnFamily(KEYSPACE1, "Standard1", true);
     }
 
     @Test
     public void testValidateCommutativeWithCounter() throws org.apache.cassandra.exceptions.InvalidRequestException
     {
-        ThriftValidation.validateColumnFamily("Keyspace1", "Counter1", true);
+        ThriftValidation.validateColumnFamily(KEYSPACE1, "Counter1", true);
     }
 
     @Test
     public void testColumnNameEqualToKeyAlias() throws org.apache.cassandra.exceptions.InvalidRequestException
     {
-        CFMetaData metaData = Schema.instance.getCFMetaData("Keyspace1", "Standard1");
+        CFMetaData metaData = Schema.instance.getCFMetaData(KEYSPACE1, "Standard1");
         CFMetaData newMetadata = metaData.copy();
 
         boolean gotException = false;
@@ -100,7 +120,7 @@
     @Test
     public void testColumnNameEqualToDefaultKeyAlias() throws org.apache.cassandra.exceptions.InvalidRequestException
     {
-        CFMetaData metaData = Schema.instance.getCFMetaData("Keyspace1", "UUIDKeys");
+        CFMetaData metaData = Schema.instance.getCFMetaData(KEYSPACE1, "UUIDKeys");
         ColumnDefinition definition = metaData.getColumnDefinition(ByteBufferUtil.bytes(CFMetaData.DEFAULT_KEY_ALIAS));
         assertNotNull(definition);
         assertEquals(ColumnDefinition.Kind.PARTITION_KEY, definition.kind);
@@ -119,7 +139,7 @@
     @Test
     public void testColumnNameEqualToDefaultColumnAlias() throws org.apache.cassandra.exceptions.InvalidRequestException
     {
-        CFMetaData metaData = Schema.instance.getCFMetaData("Keyspace1", "StandardLong3");
+        CFMetaData metaData = Schema.instance.getCFMetaData(KEYSPACE1, "StandardLong3");
         ColumnDefinition definition = metaData.getColumnDefinition(ByteBufferUtil.bytes(CFMetaData.DEFAULT_COLUMN_ALIAS + 1));
         assertNotNull(definition);
 
@@ -143,7 +163,7 @@
 
         try
         {
-            KSMetaData.fromThrift(ks_def).validate();
+            ThriftConversion.fromThrift(ks_def).validate();
         }
         catch (ConfigurationException e)
         {
@@ -158,7 +178,7 @@
 
         try
         {
-            KSMetaData.fromThrift(ks_def).validate();
+            ThriftConversion.fromThrift(ks_def).validate();
         }
         catch (ConfigurationException e)
         {
@@ -173,7 +193,7 @@
 
         try
         {
-            KSMetaData.fromThrift(ks_def).validate();
+            ThriftConversion.fromThrift(ks_def).validate();
         }
         catch (ConfigurationException e)
         {
diff --git a/test/unit/org/apache/cassandra/tools/BulkLoaderTest.java b/test/unit/org/apache/cassandra/tools/BulkLoaderTest.java
deleted file mode 100644
index dcdb7eb..0000000
--- a/test/unit/org/apache/cassandra/tools/BulkLoaderTest.java
+++ /dev/null
@@ -1,172 +0,0 @@
-/*
- * 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.tools;
-
-
-import java.io.File;
-
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
-import com.google.common.io.Files;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.cql3.QueryProcessor;
-import org.apache.cassandra.cql3.UntypedResultSet;
-import org.apache.cassandra.io.sstable.CQLSSTableWriter;
-import org.apache.cassandra.io.sstable.SSTableLoader;
-import org.apache.cassandra.service.EmbeddedCassandraService;
-import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.thrift.TFramedTransportFactory;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.OutputHandler;
-
-import static org.junit.Assert.assertEquals;
-
-public class BulkLoaderTest
-{
-
-    static EmbeddedCassandraService embeddedCassandraService = new EmbeddedCassandraService();
-
-    @BeforeClass
-    public static void setup() throws Exception
-    {
-        SchemaLoader.cleanupAndLeaveDirs();
-        embeddedCassandraService.start();
-
-
-        QueryProcessor.executeInternal("CREATE KEYSPACE cql_keyspace WITH replication = { 'class' : 'SimpleStrategy', 'replication_factor' : 1 }");
-    }
-
-
-    @Test
-    public void testClientWriter() throws Exception
-    {
-        String KS = "cql_keyspace";
-        String TABLE = "table2";
-
-        File tempdir = Files.createTempDir();
-        File dataDir = new File(tempdir.getAbsolutePath() + File.separator + KS + File.separator + TABLE);
-        assert dataDir.mkdirs();
-
-        String schema = "CREATE TABLE cql_keyspace.table2 ("
-                        + "  k int PRIMARY KEY,"
-                        + "  v1 text,"
-                        + "  v2 int"
-                        + ")";
-
-        QueryProcessor.executeInternal(schema);
-
-        String insert = "INSERT INTO cql_keyspace.table2 (k, v1, v2) VALUES (?, ?, ?)";
-        CQLSSTableWriter writer = CQLSSTableWriter.builder()
-                                                  .inDirectory(dataDir)
-                                                  .forTable(schema)
-                                                  .withPartitioner(StorageService.instance.getPartitioner())
-                                                  .using(insert).build();
-
-        writer.addRow(0, "test1", 24);
-        writer.addRow(1, "test2", null);
-        writer.addRow(2, "test3", 42);
-        writer.addRow(ImmutableMap.<String, Object>of("k", 3, "v2", 12));
-        writer.close();
-
-        BulkLoader.ExternalClient client = new BulkLoader.ExternalClient(Sets.newHashSet(FBUtilities.getLocalAddress()),
-                                                                         DatabaseDescriptor.getRpcPort(),
-                                                                         null, null, new TFramedTransportFactory(),
-                                                                         DatabaseDescriptor.getStoragePort(),
-                                                                         DatabaseDescriptor.getSSLStoragePort(), null);
-
-
-        SSTableLoader loader = new SSTableLoader(dataDir, client, new OutputHandler.SystemOutput(false, false));
-
-
-
-        loader.stream().get();
-
-        UntypedResultSet rs = QueryProcessor.executeInternal("SELECT * FROM cql_keyspace.table2;");
-        assertEquals(4, rs.size());
-    }
-
-
-    @Test
-    public void testClientWriterWithDroppedColumn() throws Exception
-    {
-        String KS = "cql_keyspace";
-        String TABLE = "table3";
-
-        File tempdir = Files.createTempDir();
-        File dataDir = new File(tempdir.getAbsolutePath() + File.separator + KS + File.separator + TABLE);
-        assert dataDir.mkdirs();
-
-        String schemaToDrop = "CREATE TABLE cql_keyspace.table3 ("
-                            + "  k int PRIMARY KEY,"
-                            + "  v1 text,"
-                            + "  v2 int,"
-                            + "  v3 list<int>,"
-                            + "  v4 text"
-                            + ")";
-
-        QueryProcessor.executeInternal(schemaToDrop);
-        QueryProcessor.executeInternal("ALTER TABLE cql_keyspace.table3 DROP v4");
-
-
-        String schema = "CREATE TABLE cql_keyspace.table3 ("
-                        + "  k int PRIMARY KEY,"
-                        + "  v1 text,"
-                        + "  v2 int,"
-                        + "  v3 list<int>"
-                        + ")";
-
-
-        String insert = "INSERT INTO cql_keyspace.table3 (k, v1, v2, v3) VALUES (?, ?, ?, ?)";
-        CQLSSTableWriter writer = CQLSSTableWriter.builder()
-                                                  .inDirectory(dataDir)
-                                                  .forTable(schema)
-                                                  .withPartitioner(StorageService.instance.getPartitioner())
-                                                  .using(insert).build();
-
-        writer.addRow(0, "test1", 24, Lists.newArrayList(4));
-        writer.addRow(1, "test2", null, Lists.newArrayList(4,4,5));
-        writer.addRow(2, "test3", 42, null);
-        writer.close();
-
-        BulkLoader.ExternalClient client = new BulkLoader.ExternalClient(Sets.newHashSet(FBUtilities.getLocalAddress()),
-                                                                         DatabaseDescriptor.getRpcPort(),
-                                                                         null, null, new TFramedTransportFactory(),
-                                                                         DatabaseDescriptor.getStoragePort(),
-                                                                         DatabaseDescriptor.getSSLStoragePort(), null);
-
-        SSTableLoader loader = new SSTableLoader(dataDir, client, new OutputHandler.SystemOutput(false, false));
-        
-
-        loader.stream().get();
-
-
-        CFMetaData cfMetaData = client.getCFMetaData(KS, TABLE);
-        assert cfMetaData != null;
-
-        UntypedResultSet rs = QueryProcessor.executeInternal("SELECT * FROM cql_keyspace.table3;");
-        assertEquals(3, rs.size());
-    }
-
-}
diff --git a/test/unit/org/apache/cassandra/tools/SSTableExportTest.java b/test/unit/org/apache/cassandra/tools/SSTableExportTest.java
index 16f4bda..bc73c83 100644
--- a/test/unit/org/apache/cassandra/tools/SSTableExportTest.java
+++ b/test/unit/org/apache/cassandra/tools/SSTableExportTest.java
@@ -18,51 +18,86 @@
 */
 package org.apache.cassandra.tools;
 
-import static org.apache.cassandra.Util.column;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.apache.cassandra.io.sstable.SSTableUtils.tempSSTableFile;
-import static org.apache.cassandra.utils.ByteBufferUtil.bytesToHex;
-import static org.apache.cassandra.utils.ByteBufferUtil.hexToBytes;
-
 import java.io.File;
 import java.io.FileReader;
 import java.io.IOException;
 import java.io.PrintStream;
 
+import org.junit.BeforeClass;
+import org.junit.Test;
+
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.db.*;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.db.ArrayBackedSortedColumns;
+import org.apache.cassandra.db.BufferCell;
+import org.apache.cassandra.db.BufferCounterCell;
+import org.apache.cassandra.db.BufferExpiringCell;
+import org.apache.cassandra.db.ColumnFamily;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DeletionInfo;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.filter.QueryFilter;
 import org.apache.cassandra.db.marshal.AsciiType;
 import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.CounterColumnType;
 import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.db.marshal.UUIDType;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.sstable.SSTableReader;
-import org.apache.cassandra.io.sstable.SSTableWriter;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.UUIDGen;
+import org.apache.thrift.TException;
 import org.json.simple.JSONArray;
 import org.json.simple.JSONObject;
 import org.json.simple.JSONValue;
 import org.json.simple.parser.ParseException;
-import org.junit.Test;
 
-public class SSTableExportTest extends SchemaLoader
+import static org.apache.cassandra.Util.column;
+import static org.apache.cassandra.io.sstable.SSTableUtils.tempSSTableFile;
+import static org.apache.cassandra.utils.ByteBufferUtil.bytesToHex;
+import static org.apache.cassandra.utils.ByteBufferUtil.hexToBytes;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+public class SSTableExportTest
 {
+    public static final String KEYSPACE1 = "SSTableExportTest";
+    public static final String CF_STANDARD = "Standard1";
+    public static final String CF_COUNTER = "Counter1";
+    public static final String CF_UUID = "UUIDKeys";
+    public static final String CF_VALSWITHQUOTES = "ValuesWithQuotes";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_COUNTER).defaultValidator(CounterColumnType.instance),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_UUID).keyValidator(UUIDType.instance),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_VALSWITHQUOTES).defaultValidator(UTF8Type.instance),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, "AsciiKeys").keyValidator(AsciiType.instance));
+    }
+
     public String asHex(String str)
     {
         return bytesToHex(ByteBufferUtil.bytes(str));
     }
-    
+
     @Test
     public void testEnumeratekeys() throws IOException
     {
-        File tempSS = tempSSTableFile("Keyspace1", "Standard1");
-        ColumnFamily cfamily = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
-        SSTableWriter writer = new SSTableWriter(tempSS.getPath(), 2, ActiveRepairService.UNREPAIRED_SSTABLE);
+        File tempSS = tempSSTableFile(KEYSPACE1, "Standard1");
+        ColumnFamily cfamily = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
+        SSTableWriter writer = SSTableWriter.create(tempSS.getPath(), 2, ActiveRepairService.UNREPAIRED_SSTABLE, 0);
 
         // Add rowA
         cfamily.addColumn(Util.cellname("colA"), ByteBufferUtil.bytes("valA"), System.currentTimeMillis());
@@ -74,7 +109,7 @@
         writer.append(Util.dk("rowB"), cfamily);
         cfamily.clear();
 
-        writer.closeAndOpenReader();
+        writer.finish(true);
 
         // Enumerate and verify
         File temp = File.createTempFile("Standard1", ".txt");
@@ -88,7 +123,7 @@
             char[] buf = new char[(int) temp.length()];
             file.read(buf);
             String output = new String(buf);
-    
+
             String sep = System.getProperty("line.separator");
             assert output.equals(asHex("rowA") + sep + asHex("rowB") + sep) : output;
         }
@@ -97,9 +132,9 @@
     @Test
     public void testExportSimpleCf() throws IOException, ParseException
     {
-        File tempSS = tempSSTableFile("Keyspace1", "Standard1");
-        ColumnFamily cfamily = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
-        SSTableWriter writer = new SSTableWriter(tempSS.getPath(), 2, ActiveRepairService.UNREPAIRED_SSTABLE);
+        File tempSS = tempSSTableFile(KEYSPACE1, "Standard1");
+        ColumnFamily cfamily = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
+        SSTableWriter writer = SSTableWriter.create(tempSS.getPath(), 2, ActiveRepairService.UNREPAIRED_SSTABLE, 0);
 
         int nowInSec = (int)(System.currentTimeMillis() / 1000) + 42; //live for 42 seconds
         // Add rowA
@@ -118,12 +153,11 @@
         writer.append(Util.dk("rowExclude"), cfamily);
         cfamily.clear();
 
-        SSTableReader reader = writer.closeAndOpenReader();
+        SSTableReader reader = writer.finish(true);
 
         // Export to JSON and verify
         File tempJson = File.createTempFile("Standard1", ".json");
-        SSTableExport.export(reader, new PrintStream(tempJson.getPath()), new String[]{asHex("rowExclude")},
-                CFMetaData.sparseCFMetaData("Keyspace1", "Standard1", BytesType.instance));
+        SSTableExport.export(reader, new PrintStream(tempJson.getPath()), new String[]{asHex("rowExclude")});
 
         JSONArray json = (JSONArray)JSONValue.parseWithException(new FileReader(tempJson));
         assertEquals("unexpected number of rows", 2, json.size());
@@ -153,10 +187,10 @@
     @Test
     public void testRoundTripStandardCf() throws IOException
     {
-        ColumnFamilyStore cfs = Keyspace.open("Keyspace1").getColumnFamilyStore("Standard1");
-        File tempSS = tempSSTableFile("Keyspace1", "Standard1");
-        ColumnFamily cfamily = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
-        SSTableWriter writer = new SSTableWriter(tempSS.getPath(), 2, ActiveRepairService.UNREPAIRED_SSTABLE);
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore("Standard1");
+        File tempSS = tempSSTableFile(KEYSPACE1, "Standard1");
+        ColumnFamily cfamily = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
+        SSTableWriter writer = SSTableWriter.create(tempSS.getPath(), 2, ActiveRepairService.UNREPAIRED_SSTABLE, 0);
 
         // Add rowA
         cfamily.addColumn(Util.cellname("name"), ByteBufferUtil.bytes("val"), System.currentTimeMillis());
@@ -168,16 +202,15 @@
         writer.append(Util.dk("rowExclude"), cfamily);
         cfamily.clear();
 
-        SSTableReader reader = writer.closeAndOpenReader();
+        SSTableReader reader = writer.finish(true);
 
         // Export to JSON and verify
         File tempJson = File.createTempFile("Standard1", ".json");
-        SSTableExport.export(reader, new PrintStream(tempJson.getPath()), new String[]{asHex("rowExclude")},
-                CFMetaData.sparseCFMetaData("Keyspace1", "Standard1", BytesType.instance));
+        SSTableExport.export(reader, new PrintStream(tempJson.getPath()), new String[]{asHex("rowExclude")});
 
         // Import JSON to another SSTable file
-        File tempSS2 = tempSSTableFile("Keyspace1", "Standard1");
-        new SSTableImport().importJson(tempJson.getPath(), "Keyspace1", "Standard1", tempSS2.getPath());
+        File tempSS2 = tempSSTableFile(KEYSPACE1, "Standard1");
+        new SSTableImport().importJson(tempJson.getPath(), KEYSPACE1, "Standard1", tempSS2.getPath());
 
         reader = SSTableReader.open(Descriptor.fromFilename(tempSS2.getPath()));
         QueryFilter qf = Util.namesQueryFilter(cfs, Util.dk("rowA"), "name");
@@ -195,21 +228,20 @@
     @Test
     public void testExportCounterCf() throws IOException, ParseException
     {
-        File tempSS = tempSSTableFile("Keyspace1", "Counter1");
-        ColumnFamily cfamily = ArrayBackedSortedColumns.factory.create("Keyspace1", "Counter1");
-        SSTableWriter writer = new SSTableWriter(tempSS.getPath(), 2, ActiveRepairService.UNREPAIRED_SSTABLE);
+        File tempSS = tempSSTableFile(KEYSPACE1, "Counter1");
+        ColumnFamily cfamily = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Counter1");
+        SSTableWriter writer = SSTableWriter.create(tempSS.getPath(), 2, ActiveRepairService.UNREPAIRED_SSTABLE, 0);
 
         // Add rowA
         cfamily.addColumn(BufferCounterCell.createLocal(Util.cellname("colA"), 42, System.currentTimeMillis(), Long.MIN_VALUE));
         writer.append(Util.dk("rowA"), cfamily);
         cfamily.clear();
 
-        SSTableReader reader = writer.closeAndOpenReader();
+        SSTableReader reader = writer.finish(true);
 
         // Export to JSON and verify
         File tempJson = File.createTempFile("Counter1", ".json");
-        SSTableExport.export(reader, new PrintStream(tempJson.getPath()), new String[0],
-                CFMetaData.sparseCFMetaData("Keyspace1", "Counter1", BytesType.instance));
+        SSTableExport.export(reader, new PrintStream(tempJson.getPath()), new String[0]);
         JSONArray json = (JSONArray)JSONValue.parseWithException(new FileReader(tempJson));
         assertEquals("unexpected number of rows", 1, json.size());
 
@@ -227,21 +259,20 @@
     @Test
     public void testEscapingDoubleQuotes() throws IOException, ParseException
     {
-        File tempSS = tempSSTableFile("Keyspace1", "ValuesWithQuotes");
-        ColumnFamily cfamily = ArrayBackedSortedColumns.factory.create("Keyspace1", "ValuesWithQuotes");
-        SSTableWriter writer = new SSTableWriter(tempSS.getPath(), 2, ActiveRepairService.UNREPAIRED_SSTABLE);
+        File tempSS = tempSSTableFile(KEYSPACE1, "ValuesWithQuotes");
+        ColumnFamily cfamily = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "ValuesWithQuotes");
+        SSTableWriter writer = SSTableWriter.create(tempSS.getPath(), 2, ActiveRepairService.UNREPAIRED_SSTABLE);
 
         // Add rowA
         cfamily.addColumn(new BufferCell(Util.cellname("data"), UTF8Type.instance.fromString("{\"foo\":\"bar\"}")));
         writer.append(Util.dk("rowA"), cfamily);
         cfamily.clear();
 
-        SSTableReader reader = writer.closeAndOpenReader();
+        SSTableReader reader = writer.finish(true);
 
         // Export to JSON and verify
         File tempJson = File.createTempFile("ValuesWithQuotes", ".json");
-        SSTableExport.export(reader, new PrintStream(tempJson.getPath()), new String[0],
-                CFMetaData.sparseCFMetaData("Keyspace1", "ValuesWithQuotes", BytesType.instance));
+        SSTableExport.export(reader, new PrintStream(tempJson.getPath()), new String[0]);
 
         JSONArray json = (JSONArray)JSONValue.parseWithException(new FileReader(tempJson));
         assertEquals("unexpected number of rows", 1, json.size());
@@ -259,9 +290,9 @@
     @Test
     public void testExportColumnsWithMetadata() throws IOException, ParseException
     {
-        File tempSS = tempSSTableFile("Keyspace1", "Standard1");
-        ColumnFamily cfamily = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
-        SSTableWriter writer = new SSTableWriter(tempSS.getPath(), 2, ActiveRepairService.UNREPAIRED_SSTABLE);
+        File tempSS = tempSSTableFile(KEYSPACE1, "Standard1");
+        ColumnFamily cfamily = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
+        SSTableWriter writer = SSTableWriter.create(tempSS.getPath(), 2, ActiveRepairService.UNREPAIRED_SSTABLE);
 
         // Add rowA
         cfamily.addColumn(Util.cellname("colName"), ByteBufferUtil.bytes("val"), System.currentTimeMillis());
@@ -269,11 +300,10 @@
         cfamily.delete(new DeletionInfo(0, 0));
         writer.append(Util.dk("rowA"), cfamily);
 
-        SSTableReader reader = writer.closeAndOpenReader();
+        SSTableReader reader = writer.finish(true);
         // Export to JSON and verify
         File tempJson = File.createTempFile("CFWithDeletionInfo", ".json");
-        SSTableExport.export(reader, new PrintStream(tempJson.getPath()), new String[0],
-                CFMetaData.sparseCFMetaData("Keyspace1", "Counter1", BytesType.instance));
+        SSTableExport.export(reader, new PrintStream(tempJson.getPath()), new String[0]);
 
         JSONArray json = (JSONArray)JSONValue.parseWithException(new FileReader(tempJson));
         assertEquals("unexpected number of rows", 1, json.size());
@@ -296,9 +326,10 @@
         JSONObject serializedDeletionInfo = (JSONObject) meta.get("deletionInfo");
         assertNotNull("expecing deletionInfo to be present", serializedDeletionInfo);
 
-        assertEquals("unexpected serialization format for topLevelDeletion",
-                     JSONValue.parse("{\"markedForDeleteAt\":0,\"localDeletionTime\":0}"),
-                     serializedDeletionInfo);
+        assertEquals(
+                "unexpected serialization format for topLevelDeletion",
+                JSONValue.parse("{\"markedForDeleteAt\":0,\"localDeletionTime\":0}"),
+                serializedDeletionInfo);
 
         // check the colums are what we put in
         JSONArray cols = (JSONArray) row.get("cells");
@@ -320,19 +351,18 @@
     @Test
     public void testColumnNameEqualToDefaultKeyAlias() throws IOException, ParseException
     {
-        File tempSS = tempSSTableFile("Keyspace1", "UUIDKeys");
-        ColumnFamily cfamily = ArrayBackedSortedColumns.factory.create("Keyspace1", "UUIDKeys");
-        SSTableWriter writer = new SSTableWriter(tempSS.getPath(), 2, ActiveRepairService.UNREPAIRED_SSTABLE);
+        File tempSS = tempSSTableFile(KEYSPACE1, "UUIDKeys");
+        ColumnFamily cfamily = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "UUIDKeys");
+        SSTableWriter writer = SSTableWriter.create(tempSS.getPath(), 2, ActiveRepairService.UNREPAIRED_SSTABLE);
 
         // Add a row
         cfamily.addColumn(column(CFMetaData.DEFAULT_KEY_ALIAS, "not a uuid", 1L));
         writer.append(Util.dk(ByteBufferUtil.bytes(UUIDGen.getTimeUUID())), cfamily);
 
-        SSTableReader reader = writer.closeAndOpenReader();
+        SSTableReader reader = writer.finish(true);
         // Export to JSON and verify
         File tempJson = File.createTempFile("CFWithColumnNameEqualToDefaultKeyAlias", ".json");
-        SSTableExport.export(reader, new PrintStream(tempJson.getPath()), new String[0],
-                CFMetaData.sparseCFMetaData("Keyspace1", "UUIDKeys", BytesType.instance));
+        SSTableExport.export(reader, new PrintStream(tempJson.getPath()), new String[0]);
 
         JSONArray json = (JSONArray)JSONValue.parseWithException(new FileReader(tempJson));
         assertEquals(1, json.size());
@@ -350,21 +380,20 @@
     @Test
     public void testAsciiKeyValidator() throws IOException, ParseException
     {
-        File tempSS = tempSSTableFile("Keyspace1", "AsciiKeys");
-        ColumnFamily cfamily = ArrayBackedSortedColumns.factory.create("Keyspace1", "AsciiKeys");
-        SSTableWriter writer = new SSTableWriter(tempSS.getPath(), 2, ActiveRepairService.UNREPAIRED_SSTABLE);
+        File tempSS = tempSSTableFile(KEYSPACE1, "AsciiKeys");
+        ColumnFamily cfamily = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "AsciiKeys");
+        SSTableWriter writer = SSTableWriter.create(tempSS.getPath(), 2, ActiveRepairService.UNREPAIRED_SSTABLE, 0);
 
         // Add a row
         cfamily.addColumn(column("column", "value", 1L));
         writer.append(Util.dk("key", AsciiType.instance), cfamily);
 
-        SSTableReader reader = writer.closeAndOpenReader();
+        SSTableReader reader = writer.finish(true);
         // Export to JSON and verify
         File tempJson = File.createTempFile("CFWithAsciiKeys", ".json");
         SSTableExport.export(reader,
                              new PrintStream(tempJson.getPath()),
-                             new String[0],
-                             CFMetaData.sparseCFMetaData("Keyspace1", "AsciiKeys", BytesType.instance));
+                             new String[0]);
 
         JSONArray json = (JSONArray)JSONValue.parseWithException(new FileReader(tempJson));
         assertEquals(1, json.size());
diff --git a/test/unit/org/apache/cassandra/tools/SSTableImportTest.java b/test/unit/org/apache/cassandra/tools/SSTableImportTest.java
index 77036e8..5eaf154 100644
--- a/test/unit/org/apache/cassandra/tools/SSTableImportTest.java
+++ b/test/unit/org/apache/cassandra/tools/SSTableImportTest.java
@@ -18,8 +18,8 @@
 */
 package org.apache.cassandra.tools;
 
-import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertEquals;
+import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
 import static org.junit.matchers.JUnitMatchers.hasItem;
 
@@ -31,6 +31,8 @@
 import java.net.URI;
 import java.net.URISyntaxException;
 
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.junit.BeforeClass;
 import org.hamcrest.Description;
 import org.hamcrest.Matcher;
 import org.junit.Test;
@@ -38,19 +40,48 @@
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.db.ArrayBackedSortedColumns;
+import org.apache.cassandra.db.BufferDeletedCell;
+import org.apache.cassandra.db.Cell;
+import org.apache.cassandra.db.ColumnFamily;
+import org.apache.cassandra.db.CounterCell;
+import org.apache.cassandra.db.DeletionInfo;
+import org.apache.cassandra.db.ExpiringCell;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.cql3.UntypedResultSet.Row;
-import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
 import org.apache.cassandra.db.filter.QueryFilter;
 import org.apache.cassandra.db.marshal.AsciiType;
 import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.CounterColumnType;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.locator.SimpleStrategy;
 
-public class SSTableImportTest extends SchemaLoader
+public class SSTableImportTest
 {
+    public static final String KEYSPACE1 = "SSTableImportTest";
+    public static final String CF_STANDARD = "Standard1";
+    public static final String CF_COUNTER = "Counter1";
+    public static final String CQL_TABLE = "table1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_COUNTER).defaultValidator(CounterColumnType.instance),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, "AsciiKeys").keyValidator(AsciiType.instance),
+                                    CFMetaData.compile("CREATE TABLE table1 (k int PRIMARY KEY, v1 text, v2 int)", KEYSPACE1));
+    }
+
     @Test(expected = IllegalArgumentException.class)
     public void testImportUnknownCf() throws IOException, URISyntaxException
     {
@@ -65,8 +96,8 @@
     {
         // Import JSON to temp SSTable file
         String jsonUrl = resourcePath("SimpleCF.json");
-        File tempSS = tempSSTableFile("Keyspace1", "Standard1");
-        new SSTableImport(true).importJson(jsonUrl, "Keyspace1", "Standard1", tempSS.getPath());
+        File tempSS = tempSSTableFile(KEYSPACE1, "Standard1");
+        new SSTableImport(true).importJson(jsonUrl, KEYSPACE1, "Standard1", tempSS.getPath());
 
         // Verify results
         SSTableReader reader = SSTableReader.open(Descriptor.fromFilename(tempSS.getPath()));
@@ -99,9 +130,9 @@
     public void testImportUnsortedMode() throws IOException, URISyntaxException
     {
         String jsonUrl = resourcePath("UnsortedCF.json");
-        File tempSS = tempSSTableFile("Keyspace1", "Standard1");
+        File tempSS = tempSSTableFile(KEYSPACE1, "Standard1");
 
-        new SSTableImport().importJson(jsonUrl, "Keyspace1", "Standard1", tempSS.getPath());
+        new SSTableImport().importJson(jsonUrl, KEYSPACE1, "Standard1", tempSS.getPath());
 
         SSTableReader reader = SSTableReader.open(Descriptor.fromFilename(tempSS.getPath()));
         QueryFilter qf = QueryFilter.getIdentityFilter(Util.dk("rowA"), "Standard1", System.currentTimeMillis());
@@ -123,8 +154,8 @@
     {
         // Import JSON to temp SSTable file
         String jsonUrl = resourcePath("SimpleCFWithDeletionInfo.json");
-        File tempSS = tempSSTableFile("Keyspace1", "Standard1");
-        new SSTableImport(true).importJson(jsonUrl, "Keyspace1", "Standard1", tempSS.getPath());
+        File tempSS = tempSSTableFile(KEYSPACE1, "Standard1");
+        new SSTableImport(true).importJson(jsonUrl, KEYSPACE1, "Standard1", tempSS.getPath());
 
         // Verify results
         SSTableReader reader = SSTableReader.open(Descriptor.fromFilename(tempSS.getPath()));
@@ -148,8 +179,8 @@
     {
         // Import JSON to temp SSTable file
         String jsonUrl = resourcePath("CounterCF.json");
-        File tempSS = tempSSTableFile("Keyspace1", "Counter1");
-        new SSTableImport(true).importJson(jsonUrl, "Keyspace1", "Counter1", tempSS.getPath());
+        File tempSS = tempSSTableFile(KEYSPACE1, "Counter1");
+        new SSTableImport(true).importJson(jsonUrl, KEYSPACE1, "Counter1", tempSS.getPath());
 
         // Verify results
         SSTableReader reader = SSTableReader.open(Descriptor.fromFilename(tempSS.getPath()));
@@ -168,9 +199,9 @@
     {
         // Import JSON to temp SSTable file
         String jsonUrl = resourcePath("SimpleCF.json");
-        File tempSS = tempSSTableFile("Keyspace1", "AsciiKeys");
+        File tempSS = tempSSTableFile(KEYSPACE1, "AsciiKeys");
         System.setProperty("skip.key.validator", "false");
-        new SSTableImport(true).importJson(jsonUrl, "Keyspace1", "AsciiKeys", tempSS.getPath());
+        new SSTableImport(true).importJson(jsonUrl, KEYSPACE1, "AsciiKeys", tempSS.getPath());
 
         // Verify results
         SSTableReader reader = SSTableReader.open(Descriptor.fromFilename(tempSS.getPath()));
@@ -189,10 +220,10 @@
     {
         // Import JSON to temp SSTable file
         String jsonUrl = resourcePath("SimpleCF.json");
-        File tempSS = tempSSTableFile("Keyspace1", "AsciiKeys");
+        File tempSS = tempSSTableFile(KEYSPACE1, "AsciiKeys");
         // To ignore current key validator
         System.setProperty("skip.key.validator", "true");
-        new SSTableImport(true).importJson(jsonUrl, "Keyspace1", "AsciiKeys", tempSS.getPath());
+        new SSTableImport(true).importJson(jsonUrl, KEYSPACE1, "AsciiKeys", tempSS.getPath());
 
         // Verify results
         SSTableReader reader = SSTableReader.open(Descriptor.fromFilename(tempSS.getPath()));
@@ -210,15 +241,13 @@
      * */
     public void shouldImportCqlTable() throws IOException, URISyntaxException
     {
-        String cql_keyspace = "cql_keyspace";
-        String cql_table = "table1";
         String jsonUrl = resourcePath("CQLTable.json");
-        File tempSS = tempSSTableFile(cql_keyspace, cql_table);
-        new SSTableImport(true).importJson(jsonUrl, cql_keyspace, cql_table, tempSS.getPath());
+        File tempSS = tempSSTableFile(KEYSPACE1, CQL_TABLE);
+        new SSTableImport(true).importJson(jsonUrl, KEYSPACE1, CQL_TABLE, tempSS.getPath());
         SSTableReader reader = SSTableReader.open(Descriptor.fromFilename(tempSS.getPath()));
-        Keyspace.open(cql_keyspace).getColumnFamilyStore(cql_table).addSSTable(reader);
+        Keyspace.open(KEYSPACE1).getColumnFamilyStore(CQL_TABLE).addSSTable(reader);
         
-        UntypedResultSet result = QueryProcessor.executeOnceInternal(String.format("SELECT * FROM %s.%s", cql_keyspace, cql_table));
+        UntypedResultSet result = QueryProcessor.executeOnceInternal(String.format("SELECT * FROM \"%s\".%s", KEYSPACE1, CQL_TABLE));
         assertThat(result.size(), is(2));
         assertThat(result, hasItem(withElements(1, "NY", 1980)));
         assertThat(result, hasItem(withElements(2, "CA", 2014)));
@@ -229,8 +258,8 @@
     public void shouldRejectEmptyCellNamesForNonCqlTables() throws IOException, URISyntaxException
     {
         String jsonUrl = resourcePath("CQLTable.json");
-        File tempSS = tempSSTableFile("Keyspace1", "Counter1");
-        new SSTableImport(true).importJson(jsonUrl, "Keyspace1", "Counter1", tempSS.getPath());
+        File tempSS = tempSSTableFile(KEYSPACE1, CF_COUNTER);
+        new SSTableImport(true).importJson(jsonUrl, KEYSPACE1, CF_COUNTER, tempSS.getPath());
     }
     
     private static Matcher<UntypedResultSet.Row> withElements(final int key, final String v1, final int v2) {
diff --git a/test/unit/org/apache/cassandra/transport/DataTypeTest.java b/test/unit/org/apache/cassandra/transport/DataTypeTest.java
new file mode 100644
index 0000000..dc2c4e2
--- /dev/null
+++ b/test/unit/org/apache/cassandra/transport/DataTypeTest.java
@@ -0,0 +1,108 @@
+/*
+ * 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.transport;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import org.junit.Test;
+
+import io.netty.buffer.ByteBuf;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.AsciiType;
+import org.apache.cassandra.db.marshal.LongType;
+
+import static org.junit.Assert.assertEquals;
+
+public class DataTypeTest
+{
+    @Test
+    public void TestSimpleDataTypeSerialization()
+    {
+        for (DataType type : DataType.values())
+        {
+            if (isComplexType(type))
+                continue;
+
+            Map<DataType, Object> options = Collections.singletonMap(type, (Object)type.toString());
+            for (int version = 1; version < 5; version++)
+                testEncodeDecode(type, options, version);
+        }
+    }
+
+    @Test
+    public void TestListDataTypeSerialization()
+    {
+        DataType type = DataType.LIST;
+        Map<DataType, Object> options =  Collections.singletonMap(type, (Object)LongType.instance);
+        for (int version = 1; version < 5; version++)
+            testEncodeDecode(type, options, version);
+    }
+
+    @Test
+    public void TestMapDataTypeSerialization()
+    {
+        DataType type = DataType.MAP;
+        List<AbstractType> value = new ArrayList<>();
+        value.add(LongType.instance);
+        value.add(AsciiType.instance);
+        Map<DataType, Object> options = Collections.singletonMap(type, (Object)value);
+        for (int version = 1; version < 5; version++)
+            testEncodeDecode(type, options, version);
+    }
+
+    private void testEncodeDecode(DataType type, Map<DataType, Object> options, int version)
+    {
+        ByteBuf dest = type.codec.encode(options, version);
+        Map<DataType, Object> results = type.codec.decode(dest, version);
+
+        for (DataType key : results.keySet())
+        {
+            int ssize = type.serializedValueSize(results.get(key), version);
+            int esize = version < type.getProtocolVersion() ? 2 + TypeSizes.encodedUTF8Length(results.get(key).toString()) : 0;
+            switch (type)
+            {
+                case LIST:
+                case SET:
+                    esize += 2;
+                    break;
+                case MAP:
+                    esize += 4;
+                    break;
+                case CUSTOM:
+                    esize = 8;
+                    break;
+            }
+            assertEquals(esize, ssize);
+
+            DataType expected = version < type.getProtocolVersion()
+                ? DataType.CUSTOM
+                : type;
+            assertEquals(expected, key);
+        }
+    }
+
+    private boolean isComplexType(DataType type)
+    {
+        return type.getId(Server.CURRENT_VERSION) >= 32;
+    }
+}
diff --git a/test/unit/org/apache/cassandra/transport/MessagePayloadTest.java b/test/unit/org/apache/cassandra/transport/MessagePayloadTest.java
new file mode 100644
index 0000000..1dd3c5d
--- /dev/null
+++ b/test/unit/org/apache/cassandra/transport/MessagePayloadTest.java
@@ -0,0 +1,371 @@
+/*
+ * 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.transport;
+
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.Map;
+
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.BatchQueryOptions;
+import org.apache.cassandra.cql3.CQLStatement;
+import org.apache.cassandra.cql3.QueryHandler;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.cql3.statements.BatchStatement;
+import org.apache.cassandra.cql3.statements.ParsedStatement;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.exceptions.RequestExecutionException;
+import org.apache.cassandra.exceptions.RequestValidationException;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.transport.messages.BatchMessage;
+import org.apache.cassandra.transport.messages.ExecuteMessage;
+import org.apache.cassandra.transport.messages.PrepareMessage;
+import org.apache.cassandra.transport.messages.QueryMessage;
+import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.MD5Digest;
+
+import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
+
+public class MessagePayloadTest extends CQLTester
+{
+    public static Map<String, ByteBuffer> requestPayload;
+    public static Map<String, ByteBuffer> responsePayload;
+
+    private static Field cqlQueryHandlerField;
+    private static boolean modifiersAccessible;
+
+    @BeforeClass
+    public static void makeCqlQueryHandlerAccessible()
+    {
+        try
+        {
+            DatabaseDescriptor.setPartitioner(ByteOrderedPartitioner.instance);
+
+            cqlQueryHandlerField = ClientState.class.getDeclaredField("cqlQueryHandler");
+            cqlQueryHandlerField.setAccessible(true);
+
+            Field modifiersField = Field.class.getDeclaredField("modifiers");
+            modifiersAccessible = modifiersField.isAccessible();
+            modifiersField.setAccessible(true);
+            modifiersField.setInt(cqlQueryHandlerField, cqlQueryHandlerField.getModifiers() & ~Modifier.FINAL);
+        }
+        catch (IllegalAccessException | NoSuchFieldException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @AfterClass
+    public static void resetCqlQueryHandlerField()
+    {
+        if (cqlQueryHandlerField == null)
+            return;
+        try
+        {
+            Field modifiersField = Field.class.getDeclaredField("modifiers");
+            modifiersField.setAccessible(true);
+            modifiersField.setInt(cqlQueryHandlerField, cqlQueryHandlerField.getModifiers() | Modifier.FINAL);
+
+            cqlQueryHandlerField.setAccessible(false);
+
+            modifiersField.setAccessible(modifiersAccessible);
+        }
+        catch (IllegalAccessException | NoSuchFieldException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @After
+    public void dropCreatedTable()
+    {
+        try
+        {
+            QueryProcessor.executeOnceInternal("DROP TABLE " + KEYSPACE + ".atable");
+        }
+        catch (Throwable t)
+        {
+            // ignore
+        }
+    }
+
+    @Test
+    public void testMessagePayload() throws Throwable
+    {
+        QueryHandler queryHandler = (QueryHandler) cqlQueryHandlerField.get(null);
+        cqlQueryHandlerField.set(null, new TestQueryHandler());
+        try
+        {
+            requireNetwork();
+
+            Assert.assertSame(TestQueryHandler.class, ClientState.getCQLQueryHandler().getClass());
+
+            SimpleClient client = new SimpleClient(nativeAddr.getHostAddress(), nativePort);
+            try
+            {
+                client.connect(false);
+
+                Map<String, ByteBuffer> reqMap;
+                Map<String, ByteBuffer> respMap;
+
+                QueryMessage queryMessage = new QueryMessage(
+                                                            "CREATE TABLE " + KEYSPACE + ".atable (pk int PRIMARY KEY, v text)",
+                                                            QueryOptions.DEFAULT
+                );
+                PrepareMessage prepareMessage = new PrepareMessage("SELECT * FROM " + KEYSPACE + ".atable");
+
+                reqMap = Collections.singletonMap("foo", bytes(42));
+                responsePayload = respMap = Collections.singletonMap("bar", bytes(42));
+                queryMessage.setCustomPayload(reqMap);
+                Message.Response queryResponse = client.execute(queryMessage);
+                payloadEquals(reqMap, requestPayload);
+                payloadEquals(respMap, queryResponse.getCustomPayload());
+
+                reqMap = Collections.singletonMap("foo", bytes(43));
+                responsePayload = respMap = Collections.singletonMap("bar", bytes(43));
+                prepareMessage.setCustomPayload(reqMap);
+                ResultMessage.Prepared prepareResponse = (ResultMessage.Prepared) client.execute(prepareMessage);
+                payloadEquals(reqMap, requestPayload);
+                payloadEquals(respMap, prepareResponse.getCustomPayload());
+
+                ExecuteMessage executeMessage = new ExecuteMessage(prepareResponse.statementId, QueryOptions.DEFAULT);
+                reqMap = Collections.singletonMap("foo", bytes(44));
+                responsePayload = respMap = Collections.singletonMap("bar", bytes(44));
+                executeMessage.setCustomPayload(reqMap);
+                Message.Response executeResponse = client.execute(executeMessage);
+                payloadEquals(reqMap, requestPayload);
+                payloadEquals(respMap, executeResponse.getCustomPayload());
+
+                BatchMessage batchMessage = new BatchMessage(BatchStatement.Type.UNLOGGED,
+                                                             Collections.<Object>singletonList("INSERT INTO " + KEYSPACE + ".atable (pk,v) VALUES (1, 'foo')"),
+                                                             Collections.singletonList(Collections.<ByteBuffer>emptyList()),
+                                                             QueryOptions.DEFAULT);
+                reqMap = Collections.singletonMap("foo", bytes(45));
+                responsePayload = respMap = Collections.singletonMap("bar", bytes(45));
+                batchMessage.setCustomPayload(reqMap);
+                Message.Response batchResponse = client.execute(batchMessage);
+                payloadEquals(reqMap, requestPayload);
+                payloadEquals(respMap, batchResponse.getCustomPayload());
+            }
+            finally
+            {
+                client.close();
+            }
+        }
+        finally
+        {
+            cqlQueryHandlerField.set(null, queryHandler);
+        }
+    }
+
+    @Test
+    public void testMessagePayloadVersion3() throws Throwable
+    {
+        QueryHandler queryHandler = (QueryHandler) cqlQueryHandlerField.get(null);
+        cqlQueryHandlerField.set(null, new TestQueryHandler());
+        try
+        {
+            requireNetwork();
+
+            Assert.assertSame(TestQueryHandler.class, ClientState.getCQLQueryHandler().getClass());
+
+            SimpleClient client = new SimpleClient(nativeAddr.getHostAddress(), nativePort, Server.VERSION_3);
+            try
+            {
+                client.connect(false);
+
+                Map<String, ByteBuffer> reqMap;
+
+                QueryMessage queryMessage = new QueryMessage(
+                                                            "CREATE TABLE " + KEYSPACE + ".atable (pk int PRIMARY KEY, v text)",
+                                                            QueryOptions.DEFAULT
+                );
+                PrepareMessage prepareMessage = new PrepareMessage("SELECT * FROM " + KEYSPACE + ".atable");
+
+                reqMap = Collections.singletonMap("foo", bytes(42));
+                responsePayload = Collections.singletonMap("bar", bytes(42));
+                queryMessage.setCustomPayload(reqMap);
+                try
+                {
+                    client.execute(queryMessage);
+                    Assert.fail();
+                }
+                catch (RuntimeException e)
+                {
+                    Assert.assertTrue(e.getCause() instanceof ProtocolException);
+                }
+                queryMessage.setCustomPayload(null);
+                client.execute(queryMessage);
+
+                reqMap = Collections.singletonMap("foo", bytes(43));
+                responsePayload = Collections.singletonMap("bar", bytes(43));
+                prepareMessage.setCustomPayload(reqMap);
+                try
+                {
+                    client.execute(prepareMessage);
+                    Assert.fail();
+                }
+                catch (RuntimeException e)
+                {
+                    Assert.assertTrue(e.getCause() instanceof ProtocolException);
+                }
+                prepareMessage.setCustomPayload(null);
+                ResultMessage.Prepared prepareResponse = (ResultMessage.Prepared) client.execute(prepareMessage);
+
+                ExecuteMessage executeMessage = new ExecuteMessage(prepareResponse.statementId, QueryOptions.DEFAULT);
+                reqMap = Collections.singletonMap("foo", bytes(44));
+                responsePayload = Collections.singletonMap("bar", bytes(44));
+                executeMessage.setCustomPayload(reqMap);
+                try
+                {
+                    client.execute(executeMessage);
+                    Assert.fail();
+                }
+                catch (RuntimeException e)
+                {
+                    Assert.assertTrue(e.getCause() instanceof ProtocolException);
+                }
+
+                BatchMessage batchMessage = new BatchMessage(BatchStatement.Type.UNLOGGED,
+                                                             Collections.<Object>singletonList("INSERT INTO " + KEYSPACE + ".atable (pk,v) VALUES (1, 'foo')"),
+                                                             Collections.singletonList(Collections.<ByteBuffer>emptyList()),
+                                                             QueryOptions.DEFAULT);
+                reqMap = Collections.singletonMap("foo", bytes(45));
+                responsePayload = Collections.singletonMap("bar", bytes(45));
+                batchMessage.setCustomPayload(reqMap);
+                try
+                {
+                    client.execute(batchMessage);
+                    Assert.fail();
+                }
+                catch (RuntimeException e)
+                {
+                    Assert.assertTrue(e.getCause() instanceof ProtocolException);
+                }
+            }
+            finally
+            {
+                client.close();
+            }
+        }
+        finally
+        {
+            cqlQueryHandlerField.set(null, queryHandler);
+        }
+    }
+
+    private static void payloadEquals(Map<String, ByteBuffer> map1, Map<String, ByteBuffer> map2)
+    {
+        Assert.assertNotNull(map1);
+        Assert.assertNotNull(map2);
+        Assert.assertEquals(map1.keySet(), map2.keySet());
+        for (Map.Entry<String, ByteBuffer> e : map1.entrySet())
+            Assert.assertEquals(e.getValue(), map2.get(e.getKey()));
+    }
+
+    public static class TestQueryHandler implements QueryHandler
+    {
+        public ParsedStatement.Prepared getPrepared(MD5Digest id)
+        {
+            return QueryProcessor.instance.getPrepared(id);
+        }
+
+        public ParsedStatement.Prepared getPreparedForThrift(Integer id)
+        {
+            return QueryProcessor.instance.getPreparedForThrift(id);
+        }
+
+        public ResultMessage.Prepared prepare(String query,
+                                              QueryState state,
+                                              Map<String, ByteBuffer> customPayload)
+                                                      throws RequestValidationException
+        {
+            if (customPayload != null)
+                requestPayload = customPayload;
+            ResultMessage.Prepared result = QueryProcessor.instance.prepare(query, state, customPayload);
+            if (customPayload != null)
+            {
+                result.setCustomPayload(responsePayload);
+                responsePayload = null;
+            }
+            return result;
+        }
+
+        public ResultMessage process(String query,
+                                     QueryState state,
+                                     QueryOptions options,
+                                     Map<String, ByteBuffer> customPayload)
+                                             throws RequestExecutionException, RequestValidationException
+        {
+            if (customPayload != null)
+                requestPayload = customPayload;
+            ResultMessage result = QueryProcessor.instance.process(query, state, options, customPayload);
+            if (customPayload != null)
+            {
+                result.setCustomPayload(responsePayload);
+                responsePayload = null;
+            }
+            return result;
+        }
+
+        public ResultMessage processBatch(BatchStatement statement,
+                                          QueryState state,
+                                          BatchQueryOptions options,
+                                          Map<String, ByteBuffer> customPayload)
+                                                  throws RequestExecutionException, RequestValidationException
+        {
+            if (customPayload != null)
+                requestPayload = customPayload;
+            ResultMessage result = QueryProcessor.instance.processBatch(statement, state, options, customPayload);
+            if (customPayload != null)
+            {
+                result.setCustomPayload(responsePayload);
+                responsePayload = null;
+            }
+            return result;
+        }
+
+        public ResultMessage processPrepared(CQLStatement statement,
+                                             QueryState state,
+                                             QueryOptions options,
+                                             Map<String, ByteBuffer> customPayload)
+                                                     throws RequestExecutionException, RequestValidationException
+        {
+            if (customPayload != null)
+                requestPayload = customPayload;
+            ResultMessage result = QueryProcessor.instance.processPrepared(statement, state, options, customPayload);
+            if (customPayload != null)
+            {
+                result.setCustomPayload(responsePayload);
+                responsePayload = null;
+            }
+            return result;
+        }
+    }
+}
diff --git a/test/unit/org/apache/cassandra/transport/ProtocolErrorTest.java b/test/unit/org/apache/cassandra/transport/ProtocolErrorTest.java
index 80d2b17..fc8c41c 100644
--- a/test/unit/org/apache/cassandra/transport/ProtocolErrorTest.java
+++ b/test/unit/org/apache/cassandra/transport/ProtocolErrorTest.java
@@ -38,7 +38,7 @@
         List<Object> results = new ArrayList<>();
         // should generate a protocol exception for using a protocol version higher than the current version
         byte[] frame = new byte[] {
-                (byte) RESPONSE.addToVersion(Server.CURRENT_VERSION + 1),  // direction & version
+                (byte) ((Server.CURRENT_VERSION + 1) & Frame.PROTOCOL_VERSION_MASK),  // direction & version
                 0x00,  // flags
                 0x01,  // stream ID
                 0x09,  // opcode
@@ -67,7 +67,7 @@
         // should generate a protocol exception for using a response frame with
         // a prepare op, ensure that it comes back with stream ID 1
         byte[] frame = new byte[] {
-                (byte) RESPONSE.addToVersion(2),  // direction & version
+                (byte) RESPONSE.addToVersion(Server.VERSION_2),  // direction & version
                 0x00,  // flags
                 0x01,  // stream ID
                 0x09,  // opcode
@@ -96,7 +96,7 @@
 
         List<Object> results = new ArrayList<>();
         byte[] frame = new byte[] {
-                (byte) (byte) REQUEST.addToVersion(2),  // direction & version
+                (byte) REQUEST.addToVersion(Server.VERSION_2),  // direction & version
                 0x00,  // flags
                 0x01,  // stream ID
                 0x09,  // opcode
@@ -113,4 +113,22 @@
             Assert.assertTrue(e.getMessage().contains("Request is too big"));
         }
     }
+
+    @Test
+    public void testErrorMessageWithNullString() throws Exception
+    {
+        // test for CASSANDRA-11167
+        ErrorMessage msg = ErrorMessage.fromException(new ServerError((String) null));
+        assert msg.toString().endsWith("null") : msg.toString();
+        int size = ErrorMessage.codec.encodedSize(msg, Server.CURRENT_VERSION);
+        ByteBuf buf = Unpooled.buffer(size);
+        ErrorMessage.codec.encode(msg, buf, Server.CURRENT_VERSION);
+
+        ByteBuf expected = Unpooled.wrappedBuffer(new byte[]{
+                0x00, 0x00, 0x00, 0x00,  // int error code
+                0x00, 0x00               // short message length
+        });
+
+        Assert.assertEquals(expected, buf);
+    }
 }
diff --git a/test/unit/org/apache/cassandra/transport/SerDeserTest.java b/test/unit/org/apache/cassandra/transport/SerDeserTest.java
index 649f7a2..352327e 100644
--- a/test/unit/org/apache/cassandra/transport/SerDeserTest.java
+++ b/test/unit/org/apache/cassandra/transport/SerDeserTest.java
@@ -36,6 +36,7 @@
 
 import static org.junit.Assert.assertEquals;
 import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
+import static org.junit.Assert.assertNotSame;
 
 /**
  * Serialization/deserialization tests for protocol objects and messages.
@@ -94,6 +95,7 @@
     {
         eventSerDeserTest(2);
         eventSerDeserTest(3);
+        eventSerDeserTest(4);
     }
 
     public void eventSerDeserTest(int version) throws Exception
@@ -122,6 +124,19 @@
             events.add(new SchemaChange(SchemaChange.Change.DROPPED, SchemaChange.Target.TYPE, "ks", "type"));
         }
 
+        if (version >= 4)
+        {
+            List<String> moreTypes = Arrays.asList("text", "bigint");
+
+            events.add(new SchemaChange(SchemaChange.Change.CREATED, SchemaChange.Target.FUNCTION, "ks", "func", Collections.<String>emptyList()));
+            events.add(new SchemaChange(SchemaChange.Change.UPDATED, SchemaChange.Target.FUNCTION, "ks", "func", moreTypes));
+            events.add(new SchemaChange(SchemaChange.Change.DROPPED, SchemaChange.Target.FUNCTION, "ks", "func", moreTypes));
+
+            events.add(new SchemaChange(SchemaChange.Change.CREATED, SchemaChange.Target.AGGREGATE, "ks", "aggr", Collections.<String>emptyList()));
+            events.add(new SchemaChange(SchemaChange.Change.UPDATED, SchemaChange.Target.AGGREGATE, "ks", "aggr", moreTypes));
+            events.add(new SchemaChange(SchemaChange.Change.DROPPED, SchemaChange.Target.AGGREGATE, "ks", "aggr", moreTypes));
+        }
+
         for (Event ev : events)
         {
             ByteBuf buf = Unpooled.buffer(ev.serializedSize(version));
@@ -213,4 +228,34 @@
         m.put("foo", 24L);
         assertEquals(m, mt.getSerializer().deserializeForNativeProtocol(fields[3], 3));
     }
+
+    @Test
+    public void preparedMetadataSerializationTest()
+    {
+        List<ColumnSpecification> columnNames = new ArrayList<>();
+        for (int i = 0; i < 3; i++)
+            columnNames.add(new ColumnSpecification("ks", "cf", new ColumnIdentifier("col" + i, false), Int32Type.instance));
+
+        ResultSet.PreparedMetadata meta = new ResultSet.PreparedMetadata(columnNames, new Short[]{2, 1});
+        ByteBuf buf = Unpooled.buffer(meta.codec.encodedSize(meta, Server.VERSION_4));
+        meta.codec.encode(meta, buf, Server.VERSION_4);
+        ResultSet.PreparedMetadata decodedMeta = meta.codec.decode(buf, Server.VERSION_4);
+
+        assertEquals(meta, decodedMeta);
+
+        // v3 encoding doesn't include partition key bind indexes
+        buf = Unpooled.buffer(meta.codec.encodedSize(meta, Server.VERSION_3));
+        meta.codec.encode(meta, buf, Server.VERSION_3);
+        decodedMeta = meta.codec.decode(buf, Server.VERSION_3);
+
+        assertNotSame(meta, decodedMeta);
+
+        // however, if there are no partition key indexes, they should be the same
+        ResultSet.PreparedMetadata metaWithoutIndexes = new ResultSet.PreparedMetadata(columnNames, null);
+        buf = Unpooled.buffer(metaWithoutIndexes.codec.encodedSize(metaWithoutIndexes, Server.VERSION_4));
+        metaWithoutIndexes.codec.encode(metaWithoutIndexes, buf, Server.VERSION_4);
+        ResultSet.PreparedMetadata decodedMetaWithoutIndexes = metaWithoutIndexes.codec.decode(buf, Server.VERSION_4);
+
+        assertEquals(decodedMeta, decodedMetaWithoutIndexes);
+    }
 }
diff --git a/test/unit/org/apache/cassandra/triggers/TriggersSchemaTest.java b/test/unit/org/apache/cassandra/triggers/TriggersSchemaTest.java
index c6a1ac5..58f743e 100644
--- a/test/unit/org/apache/cassandra/triggers/TriggersSchemaTest.java
+++ b/test/unit/org/apache/cassandra/triggers/TriggersSchemaTest.java
@@ -19,6 +19,7 @@
 
 import java.util.Collections;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
@@ -26,6 +27,7 @@
 import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.config.TriggerDefinition;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.service.MigrationManager;
 
@@ -33,13 +35,19 @@
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-public class TriggersSchemaTest extends SchemaLoader
+public class TriggersSchemaTest
 {
     String ksName = "ks" + System.nanoTime();
     String cfName = "cf" + System.nanoTime();
     String triggerName = "trigger_" + System.nanoTime();
     String triggerClass = "org.apache.cassandra.triggers.NoSuchTrigger.class";
 
+    @BeforeClass
+    public static void beforeTest() throws ConfigurationException
+    {
+        SchemaLoader.loadSchema();
+    }
+
     @Test
     public void newKsContainsCfWithTrigger() throws Exception
     {
@@ -95,7 +103,7 @@
         CFMetaData cfm2 = Schema.instance.getCFMetaData(ksName, cfName).copy();
         TriggerDefinition td = TriggerDefinition.create(triggerName, triggerClass);
         cfm2.addTriggerDefinition(td);
-        MigrationManager.announceColumnFamilyUpdate(cfm2, false);
+        MigrationManager.announceColumnFamilyUpdate(cfm2);
 
         CFMetaData cfm3 = Schema.instance.getCFMetaData(ksName, cfName);
         assertFalse(cfm3.getTriggers().isEmpty());
@@ -118,7 +126,7 @@
 
         CFMetaData cfm2 = Schema.instance.getCFMetaData(ksName, cfName).copy();
         cfm2.removeTrigger(triggerName);
-        MigrationManager.announceColumnFamilyUpdate(cfm2, false);
+        MigrationManager.announceColumnFamilyUpdate(cfm2);
 
         CFMetaData cfm3 = Schema.instance.getCFMetaData(ksName, cfName).copy();
         assertTrue(cfm3.getTriggers().isEmpty());
diff --git a/test/unit/org/apache/cassandra/triggers/TriggersTest.java b/test/unit/org/apache/cassandra/triggers/TriggersTest.java
index 74fde69..b0a5aca 100644
--- a/test/unit/org/apache/cassandra/triggers/TriggersTest.java
+++ b/test/unit/org/apache/cassandra/triggers/TriggersTest.java
@@ -22,31 +22,27 @@
 import java.util.Collection;
 import java.util.Collections;
 
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.*;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
-import org.apache.cassandra.db.ArrayBackedSortedColumns;
-import org.apache.cassandra.db.BufferCell;
-import org.apache.cassandra.db.ColumnFamily;
+import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.Mutation;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.RequestExecutionException;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.thrift.*;
 import org.apache.thrift.protocol.TBinaryProtocol;
 
+import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
+import static org.apache.cassandra.utils.ByteBufferUtil.toInt;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
-import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
-import static org.apache.cassandra.utils.ByteBufferUtil.toInt;
-
-public class TriggersTest extends SchemaLoader
+public class TriggersTest
 {
     private static boolean triggerCreated = false;
     private static ThriftServer thriftServer;
@@ -55,10 +51,16 @@
     private static String cfName = "test_table";
     private static String otherCf = "other_table";
 
+    @BeforeClass
+    public static void beforeTest() throws ConfigurationException
+    {
+        SchemaLoader.loadSchema();
+        StorageService.instance.initServer(0);
+    }
+
     @Before
     public void setup() throws Exception
     {
-        StorageService.instance.initServer(0);
         if (thriftServer == null || ! thriftServer.isRunning())
         {
             thriftServer = new ThriftServer(InetAddress.getLocalHost(), 9170, 50);
@@ -275,6 +277,29 @@
         }
     }
 
+    @Test(expected=RuntimeException.class)
+    public void ifTriggerThrowsErrorNoMutationsAreApplied() throws Exception
+    {
+        String cf = "cf" + System.nanoTime();
+        try
+        {
+            setupTableWithTrigger(cf, ErrorTrigger.class);
+            String cql = String.format("INSERT INTO %s.%s (k, v1) VALUES (11, 11)", ksName, cf);
+            QueryProcessor.process(cql, ConsistencyLevel.ONE);
+        }
+        catch (Exception e)
+        {
+            Throwable cause = e.getCause();
+            assertTrue((cause instanceof org.apache.cassandra.exceptions.InvalidRequestException));
+            assertTrue(cause.getMessage().equals(ErrorTrigger.MESSAGE));
+            throw e;
+        }
+        finally
+        {
+            assertUpdateNotExecuted(cf, 11);
+        }
+    }
+
     private void setupTableWithTrigger(String cf, Class<? extends ITrigger> triggerImpl)
     throws RequestExecutionException
     {
@@ -342,4 +367,14 @@
             return Collections.singletonList(new Mutation(ksName, key, extraUpdate));
         }
     }
+
+    public static class ErrorTrigger implements ITrigger
+    {
+        public static final String MESSAGE = "Thrown by ErrorTrigger";
+        public Collection<Mutation> augment(ByteBuffer partitionKey, ColumnFamily update)
+        {
+            throw new org.apache.cassandra.exceptions.InvalidRequestException(MESSAGE);
+        }
+    }
+
 }
diff --git a/test/unit/org/apache/cassandra/utils/BTreeTest.java b/test/unit/org/apache/cassandra/utils/BTreeTest.java
index a6d4528..e1bf388 100644
--- a/test/unit/org/apache/cassandra/utils/BTreeTest.java
+++ b/test/unit/org/apache/cassandra/utils/BTreeTest.java
@@ -17,22 +17,21 @@
  */
 package org.apache.cassandra.utils;
 
-import java.util.ArrayList;
-import java.util.Comparator;
-import java.util.List;
-import java.util.Random;
+import java.util.*;
 import java.util.concurrent.ThreadLocalRandom;
 
 import org.junit.Test;
 
-import junit.framework.Assert;
 import org.apache.cassandra.utils.btree.BTree;
 import org.apache.cassandra.utils.btree.BTreeSet;
 import org.apache.cassandra.utils.btree.UpdateFunction;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertTrue;
+
 public class BTreeTest
 {
-
     static Integer[] ints = new Integer[20];
     static
     {
@@ -114,13 +113,78 @@
         }
     }
 
+    /**
+     * Tests that the apply method of the <code>UpdateFunction</code> is only called once with each key update.
+     * (see CASSANDRA-8018).
+     */
+    @Test
+    public void testUpdate_UpdateFunctionCallBack()
+    {
+        Object[] btree = new Object[0];
+        CallsMonitor monitor = new CallsMonitor();
+
+        btree = BTree.update(btree, CMP, Arrays.asList(1), true, monitor);
+        assertArrayEquals(new Object[] {1, null}, btree);
+        assertEquals(1, monitor.getNumberOfCalls(1));
+
+        monitor.clear();
+        btree = BTree.update(btree, CMP, Arrays.asList(2), true, monitor);
+        assertArrayEquals(new Object[] {1, 2}, btree);
+        assertEquals(1, monitor.getNumberOfCalls(2));
+
+        // with existing value
+        monitor.clear();
+        btree = BTree.update(btree, CMP, Arrays.asList(1), true, monitor);
+        assertArrayEquals(new Object[] {1, 2}, btree);
+        assertEquals(1, monitor.getNumberOfCalls(1));
+
+        // with two non-existing values
+        monitor.clear();
+        btree = BTree.update(btree, CMP, Arrays.asList(3, 4), true, monitor);
+        assertArrayEquals(new Object[] {1, 2, 3, 4}, btree);
+        assertEquals(1, monitor.getNumberOfCalls(3));
+        assertEquals(1, monitor.getNumberOfCalls(4));
+
+        // with one existing value and one non existing value in disorder
+        monitor.clear();
+        btree = BTree.update(btree, CMP, Arrays.asList(5, 2), false, monitor);
+        assertArrayEquals(new Object[] {3, new Object[]{1, 2}, new Object[]{4, 5}}, btree);
+        assertEquals(1, monitor.getNumberOfCalls(2));
+        assertEquals(1, monitor.getNumberOfCalls(5));
+    }
+
+    /**
+     * Tests that the apply method of the <code>UpdateFunction</code> is only called once per value with each build call.
+     */
+    @Test
+    public void testBuilding_UpdateFunctionCallBack()
+    {
+        CallsMonitor monitor = new CallsMonitor();
+        Object[] btree = BTree.build(Arrays.asList(1), CMP, true, monitor);
+        assertArrayEquals(new Object[] {1, null}, btree);
+        assertEquals(1, monitor.getNumberOfCalls(1));
+
+        monitor.clear();
+        btree = BTree.build(Arrays.asList(1, 2), CMP, true, monitor);
+        assertArrayEquals(new Object[] {1, 2}, btree);
+        assertEquals(1, monitor.getNumberOfCalls(1));
+        assertEquals(1, monitor.getNumberOfCalls(2));
+
+        monitor.clear();
+        btree = BTree.build(Arrays.asList(3, 1, 2), CMP, false, monitor);
+        assertArrayEquals(new Object[] {1, 2, 3, null}, btree);
+        assertEquals(1, monitor.getNumberOfCalls(1));
+        assertEquals(1, monitor.getNumberOfCalls(2));
+        assertEquals(1, monitor.getNumberOfCalls(3));
+    }
+
     private static void checkResult(int count, Object[] btree)
     {
         BTreeSet<Integer> vs = new BTreeSet<>(btree, CMP);
         assert vs.size() == count;
         int i = 0;
         for (Integer j : vs)
-            Assert.assertEquals(j, ints[i++]);
+            assertEquals(j, ints[i++]);
     }
 
     @Test
@@ -137,7 +201,7 @@
         Object[] btree = BTree.build(ranges(range(0, 8)), cmp, true, UpdateFunction.NoOp.<String>instance());
         BTree.update(btree, cmp, ranges(range(0, 94)), false, new AbortAfterX(90));
         btree = BTree.update(btree, cmp, ranges(range(0, 94)), false, UpdateFunction.NoOp.<String>instance());
-        Assert.assertTrue(BTree.isWellFormed(btree, cmp));
+        assertTrue(BTree.isWellFormed(btree, cmp));
     }
 
     private static final class AbortAfterX implements UpdateFunction<String>
@@ -181,4 +245,44 @@
         }
         return r;
     }
+
+    /**
+     * <code>UpdateFunction</code> that count the number of call made to apply for each value.
+     */
+    public static final class CallsMonitor implements UpdateFunction<Integer>
+    {
+        private int[] numberOfCalls = new int[20];
+
+        public Integer apply(Integer replacing, Integer update)
+        {
+            numberOfCalls[update] = numberOfCalls[update] + 1;
+            return update;
+        }
+
+        public boolean abortEarly()
+        {
+            return false;
+        }
+
+        public void allocated(long heapSize)
+        {
+
+        }
+
+        public Integer apply(Integer integer)
+        {
+            numberOfCalls[integer] = numberOfCalls[integer] + 1;
+            return integer;
+        }
+
+        public int getNumberOfCalls(Integer key)
+        {
+            return numberOfCalls[key];
+        }
+
+        public void clear()
+        {
+            Arrays.fill(numberOfCalls, 0);
+        }
+    };
 }
diff --git a/test/unit/org/apache/cassandra/utils/BitSetTest.java b/test/unit/org/apache/cassandra/utils/BitSetTest.java
index e9f9211..9d82edf 100644
--- a/test/unit/org/apache/cassandra/utils/BitSetTest.java
+++ b/test/unit/org/apache/cassandra/utils/BitSetTest.java
@@ -20,16 +20,16 @@
 import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
 import java.io.IOException;
-import java.nio.ByteBuffer;
 import java.util.List;
 import java.util.Random;
 
 import com.google.common.collect.Lists;
-import org.junit.Test;
 
+import org.junit.Test;
 import org.junit.Assert;
 import org.apache.cassandra.io.util.DataOutputBuffer;
-import org.apache.cassandra.utils.KeyGenerator.WordGenerator;
+import org.apache.cassandra.utils.IFilter.FilterKey;
+import org.apache.cassandra.utils.KeyGenerator.RandomStringGenerator;
 import org.apache.cassandra.utils.obs.IBitSet;
 import org.apache.cassandra.utils.obs.OffHeapBitSet;
 import org.apache.cassandra.utils.obs.OpenBitSet;
@@ -44,17 +44,17 @@
     @Test
     public void compareBitSets()
     {
-        BloomFilter bf2 = (BloomFilter) FilterFactory.getFilter(KeyGenerator.WordGenerator.WORDS / 2, FilterTestHelper.MAX_FAILURE_RATE, false);
-        BloomFilter bf3 = (BloomFilter) FilterFactory.getFilter(KeyGenerator.WordGenerator.WORDS / 2, FilterTestHelper.MAX_FAILURE_RATE, true);
-        int skipEven = KeyGenerator.WordGenerator.WORDS % 2 == 0 ? 0 : 2;
-        WordGenerator gen1 = new KeyGenerator.WordGenerator(skipEven, 2);
+        BloomFilter bf2 = (BloomFilter) FilterFactory.getFilter(FilterTestHelper.ELEMENTS / 2, FilterTestHelper.MAX_FAILURE_RATE, false);
+        BloomFilter bf3 = (BloomFilter) FilterFactory.getFilter(FilterTestHelper.ELEMENTS / 2, FilterTestHelper.MAX_FAILURE_RATE, true);
+
+        RandomStringGenerator gen1 = new KeyGenerator.RandomStringGenerator(new Random().nextInt(), FilterTestHelper.ELEMENTS);
 
         // make sure both bitsets are empty.
         compare(bf2.bitset, bf3.bitset);
 
         while (gen1.hasNext())
         {
-            ByteBuffer key = gen1.next();
+            FilterKey key = FilterTestHelper.wrap(gen1.next());
             bf2.add(key);
             bf3.add(key);
         }
@@ -70,15 +70,19 @@
     @Test
     public void testOffHeapSerialization() throws IOException
     {
-        OffHeapBitSet bs = new OffHeapBitSet(100000);
-        populateAndReserialize(bs);
+        try (OffHeapBitSet bs = new OffHeapBitSet(100000))
+        {
+            populateAndReserialize(bs);
+        }
     }
 
     @Test
     public void testOffHeapCompatibility() throws IOException
     {
-        OpenBitSet bs = new OpenBitSet(100000);
-        populateAndReserialize(bs);
+        try (OpenBitSet bs = new OpenBitSet(100000)) 
+        {
+            populateAndReserialize(bs);
+        }
     }
 
     private void populateAndReserialize(IBitSet bs) throws IOException
@@ -90,11 +94,13 @@
         DataOutputBuffer out = new DataOutputBuffer();
         bs.serialize(out);
         DataInputStream in = new DataInputStream(new ByteArrayInputStream(out.getData()));
-        OffHeapBitSet newbs = OffHeapBitSet.deserialize(in);
-        compare(bs, newbs);
+        try (OffHeapBitSet newbs = OffHeapBitSet.deserialize(in))
+        {
+            compare(bs, newbs);
+        }
     }
 
-    private void compare(IBitSet bs, IBitSet newbs)
+    static void compare(IBitSet bs, IBitSet newbs)
     {
         assertEquals(bs.capacity(), newbs.capacity());
         for (long i = 0; i < bs.capacity(); i++)
@@ -102,25 +108,26 @@
     }
 
     @Test
-    public void testBitClear() throws IOException
+    public void testBitClear()
     {
         int size = Integer.MAX_VALUE / 4000;
-        OffHeapBitSet bitset = new OffHeapBitSet(size);
-        List<Integer> randomBits = Lists.newArrayList();
-        for (int i = 0; i < 10; i++)
-            randomBits.add(random.nextInt(size));
-
-        for (long randomBit : randomBits)
-            bitset.set(randomBit);
-
-        for (long randomBit : randomBits)
-            Assert.assertEquals(true, bitset.get(randomBit));
-
-        for (long randomBit : randomBits)
-            bitset.clear(randomBit);
-
-        for (long randomBit : randomBits)
-            Assert.assertEquals(false, bitset.get(randomBit));
-        bitset.close();
+        try (OffHeapBitSet bitset = new OffHeapBitSet(size))
+        {
+            List<Integer> randomBits = Lists.newArrayList();
+            for (int i = 0; i < 10; i++)
+                randomBits.add(random.nextInt(size));
+    
+            for (long randomBit : randomBits)
+                bitset.set(randomBit);
+    
+            for (long randomBit : randomBits)
+                Assert.assertEquals(true, bitset.get(randomBit));
+    
+            for (long randomBit : randomBits)
+                bitset.clear(randomBit);
+    
+            for (long randomBit : randomBits)
+                Assert.assertEquals(false, bitset.get(randomBit));
+        }
     }
 }
diff --git a/test/unit/org/apache/cassandra/utils/BloomFilterTest.java b/test/unit/org/apache/cassandra/utils/BloomFilterTest.java
index aee0880..0c8aec6 100644
--- a/test/unit/org/apache/cassandra/utils/BloomFilterTest.java
+++ b/test/unit/org/apache/cassandra/utils/BloomFilterTest.java
@@ -20,7 +20,6 @@
 
 import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
-import java.io.DataOutputStream;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
@@ -28,13 +27,18 @@
 import java.nio.ByteBuffer;
 import java.util.HashSet;
 import java.util.Iterator;
+import java.util.Random;
 import java.util.Set;
 
 import org.junit.*;
-
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.Murmur3Partitioner;
 import org.apache.cassandra.io.util.DataOutputBuffer;
-import org.apache.cassandra.io.util.DataOutputStreamAndChannel;
+import org.apache.cassandra.io.util.DataOutputStreamPlus;
 import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.BufferedDataOutputStreamPlus;
+import org.apache.cassandra.utils.IFilter.FilterKey;
+import org.apache.cassandra.utils.KeyGenerator.RandomStringGenerator;
 
 public class BloomFilterTest
 {
@@ -47,15 +51,15 @@
 
     public static IFilter testSerialize(IFilter f) throws IOException
     {
-        f.add(ByteBufferUtil.bytes("a"));
+        f.add(FilterTestHelper.bytes("a"));
         DataOutputBuffer out = new DataOutputBuffer();
         FilterFactory.serialize(f, out);
 
         ByteArrayInputStream in = new ByteArrayInputStream(out.getData(), 0, out.getLength());
         IFilter f2 = FilterFactory.deserialize(new DataInputStream(in), true);
 
-        assert f2.isPresent(ByteBufferUtil.bytes("a"));
-        assert !f2.isPresent(ByteBufferUtil.bytes("b"));
+        assert f2.isPresent(FilterTestHelper.bytes("a"));
+        assert !f2.isPresent(FilterTestHelper.bytes("b"));
         return f2;
     }
 
@@ -88,9 +92,9 @@
     @Test
     public void testOne()
     {
-        bf.add(ByteBufferUtil.bytes("a"));
-        assert bf.isPresent(ByteBufferUtil.bytes("a"));
-        assert !bf.isPresent(ByteBufferUtil.bytes("b"));
+        bf.add(FilterTestHelper.bytes("a"));
+        assert bf.isPresent(FilterTestHelper.bytes("a"));
+        assert !bf.isPresent(FilterTestHelper.bytes("b"));
     }
 
     @Test
@@ -134,7 +138,7 @@
         while (keys.hasNext())
         {
             hashes.clear();
-            ByteBuffer buf = keys.next();
+            FilterKey buf = FilterTestHelper.wrap(keys.next());
             BloomFilter bf = (BloomFilter) FilterFactory.getFilter(10, 1, false);
             for (long hashIndex : bf.getHashBuckets(buf, MAX_HASH_COUNT, 1024 * 1024))
             {
@@ -160,6 +164,32 @@
     }
 
     @Test
+    public void compareCachedKey()
+    {
+        BloomFilter bf1 = (BloomFilter) FilterFactory.getFilter(FilterTestHelper.ELEMENTS / 2, FilterTestHelper.MAX_FAILURE_RATE, false);
+        BloomFilter bf2 = (BloomFilter) FilterFactory.getFilter(FilterTestHelper.ELEMENTS / 2, FilterTestHelper.MAX_FAILURE_RATE, false);
+        BloomFilter bf3 = (BloomFilter) FilterFactory.getFilter(FilterTestHelper.ELEMENTS / 2, FilterTestHelper.MAX_FAILURE_RATE, false);
+
+        RandomStringGenerator gen1 = new KeyGenerator.RandomStringGenerator(new Random().nextInt(), FilterTestHelper.ELEMENTS);
+
+        // make sure all bitsets are empty.
+        BitSetTest.compare(bf1.bitset, bf2.bitset);
+        BitSetTest.compare(bf1.bitset, bf3.bitset);
+
+        while (gen1.hasNext())
+        {
+            ByteBuffer key = gen1.next();
+            FilterKey cached = FilterTestHelper.wrapCached(key);
+            bf1.add(FilterTestHelper.wrap(key));
+            bf2.add(cached);
+            bf3.add(cached);
+        }
+
+        BitSetTest.compare(bf1.bitset, bf2.bitset);
+        BitSetTest.compare(bf1.bitset, bf3.bitset);
+    }
+
+    @Test
     @Ignore
     public void testHugeBFSerialization() throws IOException
     {
@@ -167,16 +197,38 @@
 
         File file = FileUtils.createTempFile("bloomFilterTest-", ".dat");
         BloomFilter filter = (BloomFilter) FilterFactory.getFilter(((long)Integer.MAX_VALUE / 8) + 1, 0.01d, true);
-        filter.add(test);
-        DataOutputStreamAndChannel out = new DataOutputStreamAndChannel(new FileOutputStream(file));
+        filter.add(FilterTestHelper.wrap(test));
+        DataOutputStreamPlus out = new BufferedDataOutputStreamPlus(new FileOutputStream(file));
         FilterFactory.serialize(filter, out);
         filter.bitset.serialize(out);
         out.close();
         filter.close();
-        
+
         DataInputStream in = new DataInputStream(new FileInputStream(file));
         BloomFilter filter2 = (BloomFilter) FilterFactory.deserialize(in, true);
-        Assert.assertTrue(filter2.isPresent(test));
+        Assert.assertTrue(filter2.isPresent(FilterTestHelper.wrap(test)));
         FileUtils.closeQuietly(in);
     }
+
+    @Test
+    public void testMurmur3FilterHash()
+    {
+        IPartitioner partitioner = new Murmur3Partitioner();
+        Iterator<ByteBuffer> gen = new KeyGenerator.RandomStringGenerator(new Random().nextInt(), FilterTestHelper.ELEMENTS);
+        long[] expected = new long[2];
+        long[] actual = new long[2];
+        while (gen.hasNext())
+        {
+            expected[0] = 1;
+            expected[1] = 2;
+            actual[0] = 3;
+            actual[1] = 4;
+            ByteBuffer key = gen.next();
+            FilterKey expectedKey = FilterTestHelper.wrap(key);
+            FilterKey actualKey = partitioner.decorateKey(key);
+            actualKey.filterHash(actual);
+            expectedKey.filterHash(expected);
+            Assert.assertArrayEquals(expected, actual);
+        }
+    }
 }
diff --git a/test/unit/org/apache/cassandra/utils/ByteBufferUtilTest.java b/test/unit/org/apache/cassandra/utils/ByteBufferUtilTest.java
index 44cb20d..2cbac92 100644
--- a/test/unit/org/apache/cassandra/utils/ByteBufferUtilTest.java
+++ b/test/unit/org/apache/cassandra/utils/ByteBufferUtilTest.java
@@ -23,9 +23,7 @@
 
 import java.io.IOException;
 import java.io.DataInputStream;
-import java.io.DataOutputStream;
 import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
 import java.nio.ByteBuffer;
 import java.nio.charset.CharacterCodingException;
 import java.util.Arrays;
@@ -33,7 +31,6 @@
 import org.junit.Test;
 
 import org.apache.cassandra.io.util.DataOutputBuffer;
-import org.apache.cassandra.io.util.FastByteArrayOutputStream;
 
 public class ByteBufferUtilTest
 {
diff --git a/test/unit/org/apache/cassandra/utils/CLibraryTest.java b/test/unit/org/apache/cassandra/utils/CLibraryTest.java
new file mode 100644
index 0000000..be52bed
--- /dev/null
+++ b/test/unit/org/apache/cassandra/utils/CLibraryTest.java
@@ -0,0 +1,37 @@
+/**
+ * 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.utils;
+
+import java.io.File;
+
+import org.junit.Test;
+
+import org.apache.cassandra.io.util.FileUtils;
+
+public class CLibraryTest
+{
+    @Test
+    public void testSkipCache()
+    {
+        File file = FileUtils.createTempFile("testSkipCache", "1");
+
+        int fd = CLibrary.getfd(file.getPath());
+        CLibrary.trySkipCache(fd, 0, 0);
+    }
+}
diff --git a/test/unit/org/apache/cassandra/utils/CassandraVersionTest.java b/test/unit/org/apache/cassandra/utils/CassandraVersionTest.java
new file mode 100644
index 0000000..145b735
--- /dev/null
+++ b/test/unit/org/apache/cassandra/utils/CassandraVersionTest.java
@@ -0,0 +1,146 @@
+/*
+ * 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.utils;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertTrue;
+
+public class CassandraVersionTest
+{
+    @Test
+    public void testParsing()
+    {
+        CassandraVersion version;
+
+        version = new CassandraVersion("1.2.3");
+        assert version.major == 1 && version.minor == 2 && version.patch == 3;
+
+        version = new CassandraVersion("1.2.3-foo.2+Bar");
+        assert version.major == 1 && version.minor == 2 && version.patch == 3;
+
+        // CassandraVersion can parse 4th '.' as build number
+        version = new CassandraVersion("1.2.3.456");
+        assert version.major == 1 && version.minor == 2 && version.patch == 3;
+    }
+
+    @Test
+    public void testComparison()
+    {
+        CassandraVersion v1, v2;
+
+        v1 = new CassandraVersion("1.2.3");
+        v2 = new CassandraVersion("1.2.4");
+        assert v1.compareTo(v2) == -1;
+
+        v1 = new CassandraVersion("1.2.3");
+        v2 = new CassandraVersion("1.2.3");
+        assert v1.compareTo(v2) == 0;
+
+        v1 = new CassandraVersion("1.2.3");
+        v2 = new CassandraVersion("2.0.0");
+        assert v1.compareTo(v2) == -1;
+        assert v2.compareTo(v1) == 1;
+
+        v1 = new CassandraVersion("1.2.3");
+        v2 = new CassandraVersion("1.2.3-alpha");
+        assert v1.compareTo(v2) == 1;
+
+        v1 = new CassandraVersion("1.2.3");
+        v2 = new CassandraVersion("1.2.3+foo");
+        assert v1.compareTo(v2) == -1;
+
+        v1 = new CassandraVersion("1.2.3");
+        v2 = new CassandraVersion("1.2.3-alpha+foo");
+        assert v1.compareTo(v2) == 1;
+
+        v1 = new CassandraVersion("1.2.3-alpha+1");
+        v2 = new CassandraVersion("1.2.3-alpha+2");
+        assert v1.compareTo(v2) == -1;
+    }
+
+    @Test
+    public void testIsSupportedBy()
+    {
+        CassandraVersion v1, v2;
+
+        v1 = new CassandraVersion("3.0.2");
+        assert v1.isSupportedBy(v1);
+
+        v1 = new CassandraVersion("1.2.3");
+        v2 = new CassandraVersion("1.2.4");
+        assert v1.isSupportedBy(v2);
+        assert !v2.isSupportedBy(v1);
+
+        v1 = new CassandraVersion("1.2.3");
+        v2 = new CassandraVersion("1.3.3");
+        assert v1.isSupportedBy(v2);
+        assert !v2.isSupportedBy(v1);
+
+        v1 = new CassandraVersion("2.2.3");
+        v2 = new CassandraVersion("1.3.3");
+        assert !v1.isSupportedBy(v2);
+        assert !v2.isSupportedBy(v1);
+
+        v1 = new CassandraVersion("3.1.0");
+        v2 = new CassandraVersion("3.0.1");
+        assert !v1.isSupportedBy(v2);
+        assert v2.isSupportedBy(v1);
+    }
+
+    @Test
+    public void testInvalid()
+    {
+        assertThrows("1.0");
+        assertThrows("1.0.0a");
+        assertThrows("1.a.4");
+        assertThrows("1.0.0-foo&");
+    }
+
+    @Test
+    public void testSnapshot()
+    {
+        CassandraVersion prev, next;
+
+        prev = new CassandraVersion("2.1.5");
+        next = new CassandraVersion("2.1.5.123");
+        assertTrue(prev.compareTo(next) < 0);
+
+        prev = next;
+        next = new CassandraVersion("2.2.0-beta1-SNAPSHOT");
+        assertTrue(prev.compareTo(next) < 0);
+
+        prev = new CassandraVersion("2.2.0-beta1");
+        next = new CassandraVersion("2.2.0-rc1-SNAPSHOT");
+        assertTrue(prev.compareTo(next) < 0);
+
+        prev = next;
+        next = new CassandraVersion("2.2.0");
+        assertTrue(prev.compareTo(next) < 0);
+    }
+
+    private static void assertThrows(String str)
+    {
+        try
+        {
+            new CassandraVersion(str);
+            assert false;
+        }
+        catch (IllegalArgumentException e) {}
+    }
+}
diff --git a/test/unit/org/apache/cassandra/utils/CoalescingStrategiesTest.java b/test/unit/org/apache/cassandra/utils/CoalescingStrategiesTest.java
index 97d15fe..26b6b3a 100644
--- a/test/unit/org/apache/cassandra/utils/CoalescingStrategiesTest.java
+++ b/test/unit/org/apache/cassandra/utils/CoalescingStrategiesTest.java
@@ -17,10 +17,12 @@
  */
 package org.apache.cassandra.utils;
 
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.utils.CoalescingStrategies.Clock;
 import org.apache.cassandra.utils.CoalescingStrategies.Coalescable;
 import org.apache.cassandra.utils.CoalescingStrategies.CoalescingStrategy;
 import org.apache.cassandra.utils.CoalescingStrategies.Parker;
+import org.junit.BeforeClass;
 import org.junit.Before;
 import org.junit.Test;
 import org.slf4j.Logger;
@@ -101,6 +103,12 @@
     Semaphore queueParked = new Semaphore(0);
     Semaphore queueRelease = new Semaphore(0);
 
+    @BeforeClass
+    public static void initDD()
+    {
+        DatabaseDescriptor.forceStaticInitialization();
+    }
+
     @SuppressWarnings({ "serial" })
     @Before
     public void setUp() throws Exception
@@ -207,6 +215,38 @@
     }
 
     @Test
+    public void testFixedCoalescingStrategyEnough() throws Exception
+    {
+        int oldValue = DatabaseDescriptor.getOtcCoalescingEnoughCoalescedMessages();
+        DatabaseDescriptor.setOtcCoalescingEnoughCoalescedMessages(1);
+        try {
+            cs = newStrategy("FIXED", 200);
+
+            //Test that when a stream of messages continues arriving it keeps sending until all are drained
+            //It does this because it is already awake and sending messages
+            add(42);
+            add(42);
+            cs.coalesce(input, output, 128);
+            assertEquals(2, output.size());
+            assertNull(parker.parks.poll());
+
+            clear();
+
+            runBlocker(queueParked);
+            add(42);
+            add(42);
+            add(42);
+            release();
+            assertEquals(3, output.size());
+            assertNull(parker.parks.poll());
+        }
+        finally {
+            DatabaseDescriptor.setOtcCoalescingEnoughCoalescedMessages(oldValue);
+        }
+
+    }
+
+    @Test
     public void testDisabledCoalescingStrateg() throws Exception
     {
         cs = newStrategy("DISABLED", 200);
diff --git a/test/unit/org/apache/cassandra/utils/EncodedStreamsTest.java b/test/unit/org/apache/cassandra/utils/EncodedStreamsTest.java
index 2a67d42..c23ef53 100644
--- a/test/unit/org/apache/cassandra/utils/EncodedStreamsTest.java
+++ b/test/unit/org/apache/cassandra/utils/EncodedStreamsTest.java
@@ -26,25 +26,42 @@
 import java.io.IOException;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.ArrayBackedSortedColumns;
 import org.apache.cassandra.db.ColumnFamily;
 import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.CounterColumnType;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.utils.vint.EncodedDataInputStream;
 import org.apache.cassandra.utils.vint.EncodedDataOutputStream;
 
 import org.junit.Assert;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
-public class EncodedStreamsTest extends SchemaLoader
+public class EncodedStreamsTest
 {
-    private String keyspaceName = "Keyspace1";
-    private String standardCFName = "Standard1";
-    private String counterCFName = "Counter1";
-    private String superCFName = "Super1";
-
+    private static final String KEYSPACE1 = "Keyspace1";
+    private static final String CF_STANDARD = "Standard1";
+    private static final String CF_COUNTER = "Counter1";
     private int version = MessagingService.current_version;
 
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+    SchemaLoader.prepareServer();
+    SchemaLoader.createKeyspace(KEYSPACE1,
+                                SimpleStrategy.class,
+                                KSMetaData.optsWithRF(1),
+                                SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD),
+                                SchemaLoader.standardCFMD(KEYSPACE1, CF_COUNTER)
+                                            .defaultValidator(CounterColumnType.instance));
+    }
+
     @Test
     public void testStreams() throws IOException
     {
@@ -97,7 +114,7 @@
 
     private ColumnFamily createCF()
     {
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(keyspaceName, standardCFName);
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, CF_STANDARD);
         cf.addColumn(column("vijay", "try", 1));
         cf.addColumn(column("to", "be_nice", 1));
         return cf;
@@ -105,7 +122,7 @@
 
     private ColumnFamily createCounterCF()
     {
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(keyspaceName, counterCFName);
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, CF_COUNTER);
         cf.addCounter(cellname("vijay"), 1);
         cf.addCounter(cellname("wants"), 1000000);
         return cf;
diff --git a/test/unit/org/apache/cassandra/utils/EstimatedHistogramTest.java b/test/unit/org/apache/cassandra/utils/EstimatedHistogramTest.java
index eebaa25..b0e0e4f 100644
--- a/test/unit/org/apache/cassandra/utils/EstimatedHistogramTest.java
+++ b/test/unit/org/apache/cassandra/utils/EstimatedHistogramTest.java
@@ -28,12 +28,23 @@
     @Test
     public void testSimple()
     {
-        // 0 and 1 map to the same, first bucket
-        EstimatedHistogram histogram = new EstimatedHistogram();
-        histogram.add(0);
-        assertEquals(1, histogram.get(0));
-        histogram.add(1);
-        assertEquals(2, histogram.get(0));
+        {
+            // 0 and 1 map to the same, first bucket
+            EstimatedHistogram histogram = new EstimatedHistogram();
+            histogram.add(0);
+            assertEquals(1, histogram.get(0));
+            histogram.add(1);
+            assertEquals(2, histogram.get(0));
+        }
+        {
+            // 0 and 1 map to different buckets
+            EstimatedHistogram histogram = new EstimatedHistogram(90, true);
+            histogram.add(0);
+            assertEquals(1, histogram.get(0));
+            histogram.add(1);
+            assertEquals(1, histogram.get(0));
+            assertEquals(1, histogram.get(1));
+        }
     }
 
     @Test
@@ -55,6 +66,33 @@
     }
 
     @Test
+    public void testMean()
+    {
+        {
+            EstimatedHistogram histogram = new EstimatedHistogram();
+            for (int i = 0; i < 40; i++)
+                histogram.add(0);
+            for (int i = 0; i < 20; i++)
+                histogram.add(1);
+            for (int i = 0; i < 10; i++)
+                histogram.add(2);
+            assertEquals(70, histogram.count());
+            assertEquals(2, histogram.mean());
+        }
+        {
+            EstimatedHistogram histogram = new EstimatedHistogram(90, true);
+            for (int i = 0; i < 40; i++)
+                histogram.add(0);
+            for (int i = 0; i < 20; i++)
+                histogram.add(1);
+            for (int i = 0; i < 10; i++)
+                histogram.add(2);
+            assertEquals(70, histogram.count());
+            assertEquals(1, histogram.mean());
+        }
+    }
+
+    @Test
     public void testFindingCorrectBuckets()
     {
         EstimatedHistogram histogram = new EstimatedHistogram();
@@ -119,5 +157,14 @@
             assertEquals(17, histogram.percentile(0.60));
             assertEquals(20, histogram.percentile(0.80));
         }
+        {
+            EstimatedHistogram histogram = new EstimatedHistogram(90, true);
+            histogram.add(0);
+            histogram.add(0);
+            histogram.add(1);
+
+            assertEquals(0, histogram.percentile(0.5));
+            assertEquals(1, histogram.percentile(0.99));
+        }
     }
 }
diff --git a/test/unit/org/apache/cassandra/utils/FBUtilitiesTest.java b/test/unit/org/apache/cassandra/utils/FBUtilitiesTest.java
index c82bcc9..5b86252 100644
--- a/test/unit/org/apache/cassandra/utils/FBUtilitiesTest.java
+++ b/test/unit/org/apache/cassandra/utils/FBUtilitiesTest.java
@@ -21,6 +21,7 @@
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
+import java.net.InetAddress;
 import java.nio.ByteBuffer;
 import java.nio.charset.CharacterCodingException;
 import java.nio.charset.StandardCharsets;
@@ -30,6 +31,9 @@
 import java.util.Map;
 import java.util.TreeMap;
 
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+
 import static org.junit.Assert.assertEquals;
 
 public class FBUtilitiesTest
@@ -95,4 +99,32 @@
         ByteBuffer bytes = ByteBuffer.wrap(new byte[]{(byte)0xff, (byte)0xfe});
         ByteBufferUtil.string(bytes, StandardCharsets.UTF_8);
     }
+
+    @Test
+    public void testGetBroadcastRpcAddress() throws Exception
+    {
+        //When both rpc_address and broadcast_rpc_address are null, it should return the local address (from DD.applyAddressConfig)
+        FBUtilities.reset();
+        Config testConfig = DatabaseDescriptor.loadConfig();
+        testConfig.rpc_address = null;
+        testConfig.broadcast_rpc_address = null;
+        DatabaseDescriptor.applyAddressConfig(testConfig);
+        assertEquals(FBUtilities.getLocalAddress(), FBUtilities.getBroadcastRpcAddress());
+
+        //When rpc_address is defined and broadcast_rpc_address is null, it should return the rpc_address
+        FBUtilities.reset();
+        testConfig.rpc_address = "127.0.0.2";
+        testConfig.broadcast_rpc_address = null;
+        DatabaseDescriptor.applyAddressConfig(testConfig);
+        assertEquals(InetAddress.getByName("127.0.0.2"), FBUtilities.getBroadcastRpcAddress());
+
+        //When both rpc_address and broadcast_rpc_address are defined, it should return broadcast_rpc_address
+        FBUtilities.reset();
+        testConfig.rpc_address = "127.0.0.2";
+        testConfig.broadcast_rpc_address = "127.0.0.3";
+        DatabaseDescriptor.applyAddressConfig(testConfig);
+        assertEquals(InetAddress.getByName("127.0.0.3"), FBUtilities.getBroadcastRpcAddress());
+
+        FBUtilities.reset();
+    }
 }
diff --git a/test/unit/org/apache/cassandra/utils/FilterTestHelper.java b/test/unit/org/apache/cassandra/utils/FilterTestHelper.java
index cab7195..6d921cb 100644
--- a/test/unit/org/apache/cassandra/utils/FilterTestHelper.java
+++ b/test/unit/org/apache/cassandra/utils/FilterTestHelper.java
@@ -20,6 +20,11 @@
 
 import java.nio.ByteBuffer;
 
+import org.apache.cassandra.db.BufferDecoratedKey;
+import org.apache.cassandra.db.CachedHashDecoratedKey;
+import org.apache.cassandra.dht.Murmur3Partitioner.LongToken;
+import org.apache.cassandra.utils.IFilter.FilterKey;
+
 public class FilterTestHelper
 {
     // used by filter subclass tests
@@ -28,6 +33,21 @@
     public static final BloomCalculations.BloomSpecification spec = BloomCalculations.computeBloomSpec(15, MAX_FAILURE_RATE);
     static final int ELEMENTS = 10000;
 
+    static final FilterKey bytes(String s)
+    {
+        return new BufferDecoratedKey(new LongToken(0L), ByteBufferUtil.bytes(s));
+    }
+    
+    static final FilterKey wrap(ByteBuffer buf)
+    {
+        return new BufferDecoratedKey(new LongToken(0L), buf);
+    }
+
+    static final FilterKey wrapCached(ByteBuffer buf)
+    {
+        return new CachedHashDecoratedKey(new LongToken(0L), buf);
+    }
+
     static final ResetableIterator<ByteBuffer> intKeys()
     {
         return new KeyGenerator.IntGenerator(ELEMENTS);
@@ -49,13 +69,13 @@
 
         while (keys.hasNext())
         {
-            f.add(keys.next());
+            f.add(wrap(keys.next()));
         }
 
         int fp = 0;
         while (otherkeys.hasNext())
         {
-            if (f.isPresent(otherkeys.next()))
+            if (f.isPresent(wrap(otherkeys.next())))
             {
                 fp++;
             }
diff --git a/test/unit/org/apache/cassandra/utils/IntervalTreeTest.java b/test/unit/org/apache/cassandra/utils/IntervalTreeTest.java
index ea88092..1f66fb7 100644
--- a/test/unit/org/apache/cassandra/utils/IntervalTreeTest.java
+++ b/test/unit/org/apache/cassandra/utils/IntervalTreeTest.java
@@ -115,7 +115,7 @@
 
         IntervalTree<Integer, Void, Interval<Integer, Void>> it = IntervalTree.build(intervals);
 
-        Collections.sort(intervals, it.minOrdering);
+        Collections.sort(intervals, Interval.<Integer, Void>minOrdering());
 
         List<Interval<Integer, Void>> l = new ArrayList<Interval<Integer, Void>>();
         for (Interval<Integer, Void> i : it)
@@ -157,7 +157,7 @@
                 public String deserialize(DataInput in) throws IOException { return in.readUTF(); }
                 public long serializedSize(String v, TypeSizes s) { return v.length(); }
             },
-            (Constructor<Interval<Integer, String>>) (Object) Interval.class.getConstructor(Object.class, Object.class, Object.class)
+            (Constructor<Interval<Integer, String>>) (Constructor<?>) Interval.class.getConstructor(Object.class, Object.class, Object.class)
         );
 
         DataOutputBuffer out = new DataOutputBuffer();
diff --git a/test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java b/test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java
index 59a7e48..7142f97 100644
--- a/test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java
+++ b/test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java
@@ -22,14 +22,9 @@
 import org.apache.cassandra.io.FSReadError;
 import org.junit.Test;
 
-import java.io.File;
-import java.io.FileInputStream;
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.net.ServerSocket;
-import java.net.Socket;
 import java.net.SocketException;
-import java.util.ArrayList;
 
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -82,7 +77,7 @@
     }
 
     @Test
-    public void fileHandleTest() throws FileNotFoundException
+    public void fileHandleTest()
     {
         KillerForTests killerForTests = new KillerForTests();
         JVMStabilityInspector.Killer originalKiller = JVMStabilityInspector.replaceKiller(killerForTests);
diff --git a/test/unit/org/apache/cassandra/utils/MerkleTreeTest.java b/test/unit/org/apache/cassandra/utils/MerkleTreeTest.java
index 116e194..8d6e272 100644
--- a/test/unit/org/apache/cassandra/utils/MerkleTreeTest.java
+++ b/test/unit/org/apache/cassandra/utils/MerkleTreeTest.java
@@ -23,13 +23,13 @@
 
 import com.google.common.collect.AbstractIterator;
 import com.google.common.io.ByteArrayDataInput;
-import com.google.common.io.ByteArrayDataOutput;
 import com.google.common.io.ByteStreams;
 
 import org.junit.Before;
 import org.junit.Test;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.dht.*;
+import org.apache.cassandra.dht.RandomPartitioner.BigIntegerToken;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.utils.MerkleTree.Hashable;
@@ -62,7 +62,7 @@
     public void clear()
     {
         TOKEN_SCALE = new BigInteger("8");
-        partitioner = new RandomPartitioner();
+        partitioner = RandomPartitioner.instance;
         // TODO need to trickle TokenSerializer
         DatabaseDescriptor.setPartitioner(partitioner);
         mt = new MerkleTree(partitioner, fullRange(), RECOMMENDED_DEPTH, Integer.MAX_VALUE);
diff --git a/test/unit/org/apache/cassandra/utils/NanoTimeToCurrentTimeMillisTest.java b/test/unit/org/apache/cassandra/utils/NanoTimeToCurrentTimeMillisTest.java
index 5c025cf..1662e77 100644
--- a/test/unit/org/apache/cassandra/utils/NanoTimeToCurrentTimeMillisTest.java
+++ b/test/unit/org/apache/cassandra/utils/NanoTimeToCurrentTimeMillisTest.java
@@ -40,13 +40,19 @@
                 }
                 Thread.sleep(1);
             }
-            nowNanos = Math.max(now, System.nanoTime());
-            long convertedNow = NanoTimeToCurrentTimeMillis.convert(nowNanos);
-            assertTrue("convertedNow = " + convertedNow + " lastConverted = " + lastConverted + " in iteration " + ii, convertedNow >= (lastConverted - 1));
-            lastConverted = convertedNow;
-            //Seems to be off by as much as two milliseconds sadly
-            assertTrue("now = " + now + " convertedNow = " + convertedNow + " in iteration " + ii, (now - 2) <= convertedNow);
 
+            nowNanos = Math.max(nowNanos, System.nanoTime());
+            long convertedNow = NanoTimeToCurrentTimeMillis.convert(nowNanos);
+
+            int maxDiff = FBUtilities.isWindows()? 15 : 1;
+            assertTrue("convertedNow = " + convertedNow + " lastConverted = " + lastConverted + " in iteration " + ii,
+                       convertedNow >= (lastConverted - maxDiff));
+
+            maxDiff = FBUtilities.isWindows()? 25 : 2;
+            assertTrue("now = " + now + " convertedNow = " + convertedNow + " in iteration " + ii,
+                       (maxDiff - 2) <= convertedNow);
+
+            lastConverted = convertedNow;
         }
     }
 }
diff --git a/test/unit/org/apache/cassandra/utils/NoSpamLoggerTest.java b/test/unit/org/apache/cassandra/utils/NoSpamLoggerTest.java
index 0d6c8b1..0a5a005 100644
--- a/test/unit/org/apache/cassandra/utils/NoSpamLoggerTest.java
+++ b/test/unit/org/apache/cassandra/utils/NoSpamLoggerTest.java
@@ -32,169 +32,34 @@
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.slf4j.Logger;
-import org.slf4j.helpers.MarkerIgnoringBase;
+import org.slf4j.helpers.SubstituteLogger;
 
 
 public class NoSpamLoggerTest
 {
     Map<Level, Queue<Pair<String, Object[]>>> logged = new HashMap<>();
 
-   Logger mock = new MarkerIgnoringBase()
+   Logger mock = new SubstituteLogger(null)
    {
 
-       public boolean isTraceEnabled()
-       {
-           return false;
-       }
-
-       public void trace(String s)
-       {
-
-       }
-
-       public void trace(String s, Object o)
-       {
-
-       }
-
-       public void trace(String s, Object o, Object o1)
-       {
-
-       }
-
-       public void trace(String s, Object... objects)
-       {
-
-       }
-
-       public void trace(String s, Throwable throwable)
-       {
-
-       }
-
-       public boolean isDebugEnabled()
-       {
-           return false;
-       }
-
-       public void debug(String s)
-       {
-
-       }
-
-       public void debug(String s, Object o)
-       {
-
-       }
-
-       public void debug(String s, Object o, Object o1)
-       {
-
-       }
-
-       public void debug(String s, Object... objects)
-       {
-
-       }
-
-       public void debug(String s, Throwable throwable)
-       {
-
-       }
-
-       public boolean isInfoEnabled()
-       {
-           return false;
-       }
-
-       public void info(String s)
-       {
-
-       }
-
-       public void info(String s, Object o)
-       {
-
-       }
-
-       public void info(String s, Object o, Object o1)
-       {
-
-       }
-
        @Override
        public void info(String statement, Object... args)
        {
            logged.get(Level.INFO).offer(Pair.create(statement, args));
        }
 
-       public void info(String s, Throwable throwable)
-       {
-
-       }
-
-       public boolean isWarnEnabled()
-       {
-           return false;
-       }
-
-       public void warn(String s)
-       {
-
-       }
-
-       public void warn(String s, Object o)
-       {
-
-       }
-
        @Override
        public void warn(String statement, Object... args)
        {
            logged.get(Level.WARN).offer(Pair.create(statement, args));
        }
 
-       public void warn(String s, Object o, Object o1)
-       {
-
-       }
-
-       public void warn(String s, Throwable throwable)
-       {
-
-       }
-
-       public boolean isErrorEnabled()
-       {
-           return false;
-       }
-
-       public void error(String s)
-       {
-
-       }
-
-       public void error(String s, Object o)
-       {
-
-       }
-
-       public void error(String s, Object o, Object o1)
-       {
-
-       }
-
        @Override
        public void error(String statement, Object... args)
        {
            logged.get(Level.ERROR).offer(Pair.create(statement, args));
        }
 
-       public void error(String s, Throwable throwable)
-       {
-
-       }
-
        @Override
        public int hashCode()
        {
@@ -258,7 +123,7 @@
 
        now += 5;
 
-       NoSpamLogger.log(mock, l, 5, TimeUnit.NANOSECONDS, statement, param);
+       NoSpamLogger.log( mock, l, 5,  TimeUnit.NANOSECONDS, statement, param);
 
        assertEquals(2, logged.get(l).size());
    }
diff --git a/test/unit/org/apache/cassandra/utils/OverlapIteratorTest.java b/test/unit/org/apache/cassandra/utils/OverlapIteratorTest.java
new file mode 100644
index 0000000..5bbe267
--- /dev/null
+++ b/test/unit/org/apache/cassandra/utils/OverlapIteratorTest.java
@@ -0,0 +1,114 @@
+package org.apache.cassandra.utils;
+/*
+ *
+ * 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.
+ *
+ */
+
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.TreeSet;
+import java.util.concurrent.ThreadLocalRandom;
+
+import org.junit.Test;
+
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.ISerializer;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class OverlapIteratorTest
+{
+
+    private static List<Interval<Integer, Integer>> randomIntervals(int range, int increment, int count)
+    {
+        List<Integer> a = random(range, increment, count);
+        List<Integer> b = random(range, increment, count);
+        List<Interval<Integer, Integer>> r = new ArrayList<>();
+        for (int i = 0 ; i < count ; i++)
+        {
+            r.add(a.get(i) < b.get(i) ? Interval.create(a.get(i), b.get(i), i)
+                                      : Interval.create(b.get(i), a.get(i), i));
+        }
+        return r;
+    }
+
+    private static List<Integer> random(int range, int increment, int count)
+    {
+        List<Integer> random = new ArrayList<>();
+        for (int i = 0 ; i < count ; i++)
+        {
+            int base = i * increment;
+            random.add(ThreadLocalRandom.current().nextInt(base, base + range));
+        }
+        return random;
+    }
+
+    @Test
+    public void test()
+    {
+        for (int i = 0 ; i < 10 ; i++)
+        {
+            test(1000, 0, 1000);
+            test(100000, 100, 1000);
+            test(1000000, 0, 1000);
+        }
+    }
+
+    private void test(int range, int increment, int count)
+    {
+        compare(randomIntervals(range, increment, count), random(range, increment, count), 1);
+        compare(randomIntervals(range, increment, count), random(range, increment, count), 2);
+        compare(randomIntervals(range, increment, count), random(range, increment, count), 3);
+    }
+
+    private <I extends Comparable<I>, V> void compare(List<Interval<I, V>> intervals, List<I> points, int initCount)
+    {
+        Collections.sort(points);
+        IntervalTree<I, V, Interval<I, V>> tree = IntervalTree.build(intervals);
+        OverlapIterator<I, V> iter = new OverlapIterator<>(intervals);
+        int initPoint = points.size() / initCount;
+        int i = 0;
+        for (I point : points)
+        {
+            if (i++ == initPoint)
+                iter = new OverlapIterator<>(intervals);
+            iter.update(point);
+            TreeSet<V> act = new TreeSet<>(iter.overlaps);
+            TreeSet<V> exp = new TreeSet<>(tree.search(point));
+            TreeSet<V> extra = new TreeSet<>(act);
+            extra.removeAll(exp);
+            TreeSet<V> missing = new TreeSet<>(exp);
+            missing.removeAll(act);
+            assertTrue(extra.isEmpty());
+            assertTrue(missing.isEmpty());
+        }
+    }
+
+}
diff --git a/test/unit/org/apache/cassandra/utils/SemanticVersionTest.java b/test/unit/org/apache/cassandra/utils/SemanticVersionTest.java
deleted file mode 100644
index 4ead709..0000000
--- a/test/unit/org/apache/cassandra/utils/SemanticVersionTest.java
+++ /dev/null
@@ -1,113 +0,0 @@
-/*
- * 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.utils;
-
-import org.junit.Test;
-
-public class SemanticVersionTest
-{
-    @Test
-    public void testParsing()
-    {
-        SemanticVersion version;
-
-        version = new SemanticVersion("1.2.3");
-        assert version.major == 1 && version.minor == 2 && version.patch == 3;
-
-        version = new SemanticVersion("1.2.3-foo.2+Bar");
-        assert version.major == 1 && version.minor == 2 && version.patch == 3;
-    }
-
-    @Test
-    public void testComparison()
-    {
-        SemanticVersion v1, v2;
-
-        v1 = new SemanticVersion("1.2.3");
-        v2 = new SemanticVersion("1.2.4");
-        assert v1.compareTo(v2) == -1;
-
-        v1 = new SemanticVersion("1.2.3");
-        v2 = new SemanticVersion("1.2.3");
-        assert v1.compareTo(v2) == 0;
-
-        v1 = new SemanticVersion("1.2.3");
-        v2 = new SemanticVersion("2.0.0");
-        assert v1.compareTo(v2) == -1;
-
-        v1 = new SemanticVersion("1.2.3");
-        v2 = new SemanticVersion("1.2.3-alpha");
-        assert v1.compareTo(v2) == 1;
-
-        v1 = new SemanticVersion("1.2.3");
-        v2 = new SemanticVersion("1.2.3+foo");
-        assert v1.compareTo(v2) == -1;
-
-        v1 = new SemanticVersion("1.2.3");
-        v2 = new SemanticVersion("1.2.3-alpha+foo");
-        assert v1.compareTo(v2) == 1;
-    }
-
-    @Test
-    public void testIsSupportedBy()
-    {
-        SemanticVersion v1, v2;
-
-        v1 = new SemanticVersion("3.0.2");
-        assert v1.isSupportedBy(v1);
-
-        v1 = new SemanticVersion("1.2.3");
-        v2 = new SemanticVersion("1.2.4");
-        assert v1.isSupportedBy(v2);
-        assert !v2.isSupportedBy(v1);
-
-        v1 = new SemanticVersion("1.2.3");
-        v2 = new SemanticVersion("1.3.3");
-        assert v1.isSupportedBy(v2);
-        assert !v2.isSupportedBy(v1);
-
-        v1 = new SemanticVersion("2.2.3");
-        v2 = new SemanticVersion("1.3.3");
-        assert !v1.isSupportedBy(v2);
-        assert !v2.isSupportedBy(v1);
-
-        v1 = new SemanticVersion("3.1.0");
-        v2 = new SemanticVersion("3.0.1");
-        assert !v1.isSupportedBy(v2);
-        assert v2.isSupportedBy(v1);
-    }
-
-    @Test
-    public void testInvalid()
-    {
-        assertThrows("1.0");
-        assertThrows("1.0.0a");
-        assertThrows("1.a.4");
-        assertThrows("1.0.0-foo&");
-    }
-
-    private static void assertThrows(String str)
-    {
-        try
-        {
-            new SemanticVersion(str);
-            assert false;
-        }
-        catch (IllegalArgumentException e) {}
-    }
-}
diff --git a/test/unit/org/apache/cassandra/utils/SerializationsTest.java b/test/unit/org/apache/cassandra/utils/SerializationsTest.java
index b0a23fd..497b16d 100644
--- a/test/unit/org/apache/cassandra/utils/SerializationsTest.java
+++ b/test/unit/org/apache/cassandra/utils/SerializationsTest.java
@@ -19,13 +19,13 @@
 package org.apache.cassandra.utils;
 
 import org.apache.cassandra.AbstractSerializationsTester;
-import org.apache.cassandra.io.util.DataOutputStreamAndChannel;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.util.DataOutputStreamPlus;
 import org.apache.cassandra.service.StorageService;
-
+import org.junit.Assert;
 import org.junit.Test;
 
 import java.io.DataInputStream;
-import java.io.DataOutputStream;
 import java.io.IOException;
 
 public class SerializationsTest extends AbstractSerializationsTester
@@ -33,13 +33,16 @@
 
     private void testBloomFilterWrite(boolean offheap) throws IOException
     {
-        IFilter bf = FilterFactory.getFilter(1000000, 0.0001, offheap);
-        for (int i = 0; i < 100; i++)
-            bf.add(StorageService.getPartitioner().getTokenFactory().toByteArray(StorageService.getPartitioner().getRandomToken()));
-        DataOutputStreamAndChannel out = getOutput("utils.BloomFilter.bin");
-        FilterFactory.serialize(bf, out);
-        out.close();
-        bf.close();
+        IPartitioner partitioner = StorageService.getPartitioner();
+        try (IFilter bf = FilterFactory.getFilter(1000000, 0.0001, offheap))
+        {
+            for (int i = 0; i < 100; i++)
+                bf.add(partitioner.decorateKey(partitioner.getTokenFactory().toByteArray(partitioner.getRandomToken())));
+            try (DataOutputStreamPlus out = getOutput("utils.BloomFilter.bin"))
+            {
+                FilterFactory.serialize(bf, out);
+            }
+        }
     }
 
     @Test
@@ -48,11 +51,11 @@
         if (EXECUTE_WRITES)
             testBloomFilterWrite(true);
 
-        DataInputStream in = getInput("utils.BloomFilter.bin");
-        IFilter bf = FilterFactory.deserialize(in, true);
-        assert bf != null;
-        bf.close();
-        in.close();
+        try (DataInputStream in = getInput("utils.BloomFilter.bin");
+             IFilter filter = FilterFactory.deserialize(in, true))
+        {
+            Assert.assertNotNull(filter);
+        }
     }
 
     private void testEstimatedHistogramWrite() throws IOException
@@ -69,11 +72,12 @@
         data[offsets.length] = 100000;
         EstimatedHistogram hist2 = new EstimatedHistogram(offsets, data);
 
-        DataOutputStreamAndChannel out = getOutput("utils.EstimatedHistogram.bin");
-        EstimatedHistogram.serializer.serialize(hist0, out);
-        EstimatedHistogram.serializer.serialize(hist1, out);
-        EstimatedHistogram.serializer.serialize(hist2, out);
-        out.close();
+        try (DataOutputStreamPlus out = getOutput("utils.EstimatedHistogram.bin"))
+        {
+            EstimatedHistogram.serializer.serialize(hist0, out);
+            EstimatedHistogram.serializer.serialize(hist1, out);
+            EstimatedHistogram.serializer.serialize(hist2, out);
+        }
     }
 
     @Test
@@ -82,10 +86,11 @@
         if (EXECUTE_WRITES)
             testEstimatedHistogramWrite();
 
-        DataInputStream in = getInput("utils.EstimatedHistogram.bin");
-        assert EstimatedHistogram.serializer.deserialize(in) != null;
-        assert EstimatedHistogram.serializer.deserialize(in) != null;
-        assert EstimatedHistogram.serializer.deserialize(in) != null;
-        in.close();
+        try (DataInputStream in = getInput("utils.EstimatedHistogram.bin"))
+        {
+            Assert.assertNotNull(EstimatedHistogram.serializer.deserialize(in));
+            Assert.assertNotNull(EstimatedHistogram.serializer.deserialize(in));
+            Assert.assertNotNull(EstimatedHistogram.serializer.deserialize(in));
+        }
     }
 }
diff --git a/test/unit/org/apache/cassandra/utils/StreamingHistogramTest.java b/test/unit/org/apache/cassandra/utils/StreamingHistogramTest.java
index 13828eb..0e9b90b 100644
--- a/test/unit/org/apache/cassandra/utils/StreamingHistogramTest.java
+++ b/test/unit/org/apache/cassandra/utils/StreamingHistogramTest.java
@@ -20,9 +20,7 @@
 import org.junit.Test;
 
 import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
-import java.io.DataOutputStream;
 import java.util.*;
 
 import org.apache.cassandra.io.util.DataOutputBuffer;
diff --git a/test/unit/org/apache/cassandra/utils/TopKSamplerTest.java b/test/unit/org/apache/cassandra/utils/TopKSamplerTest.java
index b45b429..61e4c52 100644
--- a/test/unit/org/apache/cassandra/utils/TopKSamplerTest.java
+++ b/test/unit/org/apache/cassandra/utils/TopKSamplerTest.java
@@ -27,8 +27,6 @@
 
 import junit.framework.Assert;
 
-import org.apache.cassandra.concurrent.Stage;
-import org.apache.cassandra.concurrent.StageManager;
 import org.apache.cassandra.utils.TopKSampler.SamplerResult;
 import org.junit.Test;
 
diff --git a/test/unit/org/apache/cassandra/utils/concurrent/AbstractTransactionalTest.java b/test/unit/org/apache/cassandra/utils/concurrent/AbstractTransactionalTest.java
new file mode 100644
index 0000000..4e160c2
--- /dev/null
+++ b/test/unit/org/apache/cassandra/utils/concurrent/AbstractTransactionalTest.java
@@ -0,0 +1,136 @@
+/*
+* 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.utils.concurrent;
+
+import org.junit.Ignore;
+import org.junit.Test;
+
+import junit.framework.Assert;
+
+@Ignore
+public abstract class AbstractTransactionalTest
+{
+
+    protected abstract TestableTransaction newTest() throws Exception;
+
+    @Test
+    public void testNoPrepare() throws Exception
+    {
+        TestableTransaction txn;
+
+        txn = newTest();
+        txn.assertInProgress();
+        txn.testing.close();
+        txn.assertAborted();
+
+        txn = newTest();
+        txn.assertInProgress();
+        Assert.assertNull(txn.testing.abort(null));
+        txn.assertAborted();
+    }
+
+    @Test
+    public void testPrepare() throws Exception
+    {
+        TestableTransaction txn;
+        txn = newTest();
+        txn.assertInProgress();
+        txn.testing.prepareToCommit();
+        txn.assertPrepared();
+        txn.testing.close();
+        txn.assertAborted();
+
+        txn = newTest();
+        txn.assertInProgress();
+        txn.testing.prepareToCommit();
+        txn.assertPrepared();
+        Assert.assertNull(txn.testing.abort(null));
+        txn.assertAborted();
+    }
+
+    @Test
+    public void testCommit() throws Exception
+    {
+        TestableTransaction txn = newTest();
+        txn.assertInProgress();
+        txn.testing.prepareToCommit();
+        txn.assertPrepared();
+        Assert.assertNull(txn.testing.commit(null));
+        txn.assertCommitted();
+        txn.testing.close();
+        txn.assertCommitted();
+        Throwable t = txn.testing.abort(null);
+        Assert.assertTrue(t instanceof IllegalStateException);
+        txn.assertCommitted();
+    }
+
+    @Test
+    public void testThrowableReturn() throws Exception
+    {
+        TestableTransaction txn;
+        txn = newTest();
+        Throwable t = new RuntimeException();
+        txn.testing.prepareToCommit();
+        Assert.assertEquals(t, txn.testing.commit(t));
+        Assert.assertEquals(t, txn.testing.abort(t));
+        Assert.assertTrue(t.getSuppressed()[0] instanceof IllegalStateException);
+    }
+
+    @Test
+    public void testBadCommit() throws Exception
+    {
+        TestableTransaction txn;
+        txn = newTest();
+        try
+        {
+            txn.testing.commit(null);
+            Assert.assertTrue(false);
+        }
+        catch (IllegalStateException t)
+        {
+        }
+        txn.assertInProgress();
+        Assert.assertNull(txn.testing.abort(null));
+        txn.assertAborted();
+        try
+        {
+            txn.testing.commit(null);
+            Assert.assertTrue(false);
+        }
+        catch (IllegalStateException t)
+        {
+        }
+        txn.assertAborted();
+    }
+
+
+    public static abstract class TestableTransaction
+    {
+        final Transactional testing;
+        public TestableTransaction(Transactional transactional)
+        {
+            this.testing = transactional;
+        }
+
+        protected abstract void assertInProgress() throws Exception;
+        protected abstract void assertPrepared() throws Exception;
+        protected abstract void assertAborted() throws Exception;
+        protected abstract void assertCommitted() throws Exception;
+    }
+}
diff --git a/test/unit/org/apache/cassandra/utils/concurrent/AccumulatorTest.java b/test/unit/org/apache/cassandra/utils/concurrent/AccumulatorTest.java
new file mode 100644
index 0000000..2842374
--- /dev/null
+++ b/test/unit/org/apache/cassandra/utils/concurrent/AccumulatorTest.java
@@ -0,0 +1,106 @@
+/*
+* 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.utils.concurrent;
+
+import java.util.Iterator;
+
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+public class AccumulatorTest
+{
+    @Test
+    public void testAddMoreThanCapacity()
+    {
+        Accumulator<Integer> accu = new Accumulator(4);
+
+        accu.add(1);
+        accu.add(2);
+        accu.add(3);
+        accu.add(4);
+
+        try
+        {
+            accu.add(5);
+            fail();
+        }
+        catch (IllegalStateException e)
+        {
+            // Expected
+        }
+    }
+
+    @Test
+    public void testIsEmptyAndSize()
+    {
+        Accumulator<Integer> accu = new Accumulator(4);
+
+        assertTrue(accu.isEmpty());
+        assertEquals(0, accu.size());
+
+        accu.add(1);
+        accu.add(2);
+
+        assertTrue(!accu.isEmpty());
+        assertEquals(2, accu.size());
+
+        accu.add(3);
+        accu.add(4);
+
+        assertTrue(!accu.isEmpty());
+        assertEquals(4, accu.size());
+    }
+
+    @Test
+    public void testGetAndIterator()
+    {
+        Accumulator<String> accu = new Accumulator(4);
+
+        accu.add("3");
+        accu.add("2");
+        accu.add("4");
+
+        assertEquals("3", accu.get(0));
+        assertEquals("2", accu.get(1));
+        assertEquals("4", accu.get(2));
+
+        try
+        {
+            assertEquals(null, accu.get(3));
+            fail();
+        }
+        catch (IndexOutOfBoundsException e)
+        {
+            // Expected
+        }
+
+        accu.add("0");
+
+        assertEquals("0", accu.get(3));
+
+        Iterator<String> iter = accu.iterator();
+
+        assertEquals("3", iter.next());
+        assertEquals("2", iter.next());
+        assertEquals("4", iter.next());
+        assertEquals("0", iter.next());
+        assertFalse(iter.hasNext());
+    }
+}
diff --git a/test/unit/org/apache/cassandra/utils/progress/jmx/LegacyJMXProgressSupportTest.java b/test/unit/org/apache/cassandra/utils/progress/jmx/LegacyJMXProgressSupportTest.java
new file mode 100644
index 0000000..efa4a27
--- /dev/null
+++ b/test/unit/org/apache/cassandra/utils/progress/jmx/LegacyJMXProgressSupportTest.java
@@ -0,0 +1,118 @@
+/*
+ * 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.utils.progress.jmx;
+
+import java.util.UUID;
+
+import com.google.common.base.Optional;
+import org.junit.Test;
+
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.service.ActiveRepairService;
+import org.apache.cassandra.utils.progress.ProgressEvent;
+import org.apache.cassandra.utils.progress.ProgressEventType;
+
+import static org.junit.Assert.*;
+
+
+public class LegacyJMXProgressSupportTest
+{
+
+    @Test
+    public void testSessionSuccess()
+    {
+        int cmd = 321;
+        String message = String.format("Repair session %s for range %s finished", UUID.randomUUID(),
+                                       new Range<Token>(new Murmur3Partitioner.LongToken(3), new Murmur3Partitioner.LongToken(4)));
+        Optional<int[]> result = LegacyJMXProgressSupport.getLegacyUserdata(String.format("repair:%d", cmd),
+                                                            new ProgressEvent(ProgressEventType.PROGRESS, 2, 10, message));
+        assertTrue(result.isPresent());
+        assertArrayEquals(new int[]{ cmd, ActiveRepairService.Status.SESSION_SUCCESS.ordinal() }, result.get());
+    }
+
+    @Test
+    public void testSessionFailed()
+    {
+        int cmd = 321;
+        String message = String.format("Repair session %s for range %s failed with error %s", UUID.randomUUID(),
+                                       new Range<Token>(new Murmur3Partitioner.LongToken(3), new Murmur3Partitioner.LongToken(4)).toString(),
+                                       new RuntimeException("error"));
+        Optional<int[]> result = LegacyJMXProgressSupport.getLegacyUserdata(String.format("repair:%d", cmd),
+                                                                            new ProgressEvent(ProgressEventType.PROGRESS, 2, 10, message));
+        assertTrue(result.isPresent());
+        assertArrayEquals(new int[]{ cmd, ActiveRepairService.Status.SESSION_FAILED.ordinal() }, result.get());
+    }
+
+    @Test
+    public void testStarted()
+    {
+        int cmd = 321;
+        Optional<int[]> result = LegacyJMXProgressSupport.getLegacyUserdata(String.format("repair:%d", cmd),
+                                                                            new ProgressEvent(ProgressEventType.START,
+                                                                                              0, 100, "bla"));
+        assertTrue(result.isPresent());
+        assertArrayEquals(new int[]{ cmd, ActiveRepairService.Status.STARTED.ordinal() }, result.get());
+    }
+
+    @Test
+    public void testFinished()
+    {
+        int cmd = 321;
+        Optional<int[]> result = LegacyJMXProgressSupport.getLegacyUserdata(String.format("repair:%d", cmd),
+                                                                         new ProgressEvent(ProgressEventType.COMPLETE,
+                                                                                           2, 10, "bla"));
+        assertTrue(result.isPresent());
+        assertArrayEquals(new int[]{ cmd, ActiveRepairService.Status.FINISHED.ordinal() }, result.get());
+    }
+
+    /*
+    States not mapped to the legacy notification
+     */
+    @Test
+    public void testNone()
+    {
+        int cmd = 33;
+        Optional<int[]> result = LegacyJMXProgressSupport.getLegacyUserdata(String.format("repair:%d", cmd),
+                                                                         new ProgressEvent(ProgressEventType.ERROR, 2, 10, "bla"));
+        assertFalse(result.isPresent());
+
+        cmd = 33;
+        result = LegacyJMXProgressSupport.getLegacyUserdata(String.format("repair:%d", cmd),
+                                                                         new ProgressEvent(ProgressEventType.SUCCESS, 2, 10, "bla"));
+        assertFalse(result.isPresent());
+
+        cmd = 43;
+        result = LegacyJMXProgressSupport.getLegacyUserdata(String.format("repair:%d", cmd),
+                                                            new ProgressEvent(ProgressEventType.PROGRESS, 2, 10, "bla"));
+        assertFalse(result.isPresent());
+
+        cmd = 1;
+        result = LegacyJMXProgressSupport.getLegacyUserdata(String.format("repair:%d", cmd),
+                                                            new ProgressEvent(ProgressEventType.ABORT, 2, 10, "bla"));
+        assertFalse(result.isPresent());
+
+        cmd = 9;
+        result = LegacyJMXProgressSupport.getLegacyUserdata(String.format("repair:%d", cmd),
+                                                            new ProgressEvent(ProgressEventType.NOTIFICATION, 2, 10, "bla"));
+        assertFalse(result.isPresent());
+    }
+
+}
diff --git a/tools/bin/cassandra.in.bat b/tools/bin/cassandra.in.bat
index 889fa9b..0fdd31a 100644
--- a/tools/bin/cassandra.in.bat
+++ b/tools/bin/cassandra.in.bat
@@ -24,7 +24,7 @@
 REM ***** CLASSPATH library setting *****

 

 REM Ensure that any user defined CLASSPATH variables are not used on startup

-set CLASSPATH="%CASSANDRA_HOME%\conf"

+set CLASSPATH=%CASSANDRA_CONF%

 

 REM For each jar in the CASSANDRA_HOME lib directory call append to build the CLASSPATH variable.

 for %%i in ("%CASSANDRA_HOME%\lib\*.jar") do call :append "%%i"

diff --git a/tools/bin/sstablelevelreset.bat b/tools/bin/sstablelevelreset.bat
new file mode 100644
index 0000000..b15cb48
--- /dev/null
+++ b/tools/bin/sstablelevelreset.bat
@@ -0,0 +1,46 @@
+@REM  Licensed to the Apache Software Foundation (ASF) under one or more
+@REM  contributor license agreements.  See the NOTICE file distributed with
+@REM  this work for additional information regarding copyright ownership.
+@REM  The ASF licenses this file to You under the Apache License, Version 2.0
+@REM  (the "License"); you may not use this file except in compliance with
+@REM  the License.  You may obtain a copy of the License at
+@REM
+@REM      http://www.apache.org/licenses/LICENSE-2.0
+@REM
+@REM  Unless required by applicable law or agreed to in writing, software
+@REM  distributed under the License is distributed on an "AS IS" BASIS,
+@REM  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+@REM  See the License for the specific language governing permissions and
+@REM  limitations under the License.
+
+@echo off
+
+if "%OS%" == "Windows_NT" setlocal
+
+pushd "%~dp0"
+call cassandra.in.bat
+
+if NOT DEFINED JAVA_HOME goto :err
+set TOOL_MAIN=org.apache.cassandra.tools.SSTableLevelResetter
+
+REM ***** JAVA options *****
+set JAVA_OPTS=^
+ -Dlogback.configurationFile=logback-tools.xml
+
+"%JAVA_HOME%\bin\java" %JAVA_OPTS% %CASSANDRA_PARAMS% -cp %CLASSPATH% %TOOL_MAIN% %*
+goto finally
+
+:err
+echo JAVA_HOME environment variable must be set!
+set ERRORLEVEL=1
+pause
+
+:finally
+ENDLOCAL & set RC=%ERRORLEVEL%
+goto :exit_with_code
+
+:returncode
+exit /B %RC%
+
+:exit_with_code
+call :returncode %RC%
\ No newline at end of file
diff --git a/tools/bin/sstableofflinerelevel.bat b/tools/bin/sstableofflinerelevel.bat
new file mode 100644
index 0000000..3372bba
--- /dev/null
+++ b/tools/bin/sstableofflinerelevel.bat
@@ -0,0 +1,49 @@
+@REM  Licensed to the Apache Software Foundation (ASF) under one or more
+@REM  contributor license agreements.  See the NOTICE file distributed with
+@REM  this work for additional information regarding copyright ownership.
+@REM  The ASF licenses this file to You under the Apache License, Version 2.0
+@REM  (the "License"); you may not use this file except in compliance with
+@REM  the License.  You may obtain a copy of the License at
+@REM
+@REM      http://www.apache.org/licenses/LICENSE-2.0
+@REM
+@REM  Unless required by applicable law or agreed to in writing, software
+@REM  distributed under the License is distributed on an "AS IS" BASIS,
+@REM  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+@REM  See the License for the specific language governing permissions and
+@REM  limitations under the License.
+
+@echo off
+
+if "%OS%" == "Windows_NT" setlocal
+
+pushd "%~dp0"
+call cassandra.in.bat
+
+if NOT DEFINED JAVA_HOME goto :err
+if NOT DEFINED MAX_HEAP_SIZE set MAX_HEAP_SIZE="256M"
+set TOOL_MAIN=org.apache.cassandra.tools.SSTableOfflineRelevel
+
+REM ***** JAVA options *****
+set JAVA_OPTS=^
+ -ea ^
+ -Xmx%MAX_HEAP_SIZE% ^
+ -Dlogback.configurationFile=logback-tools.xml
+
+"%JAVA_HOME%\bin\java" %JAVA_OPTS% %CASSANDRA_PARAMS% -cp %CLASSPATH% %TOOL_MAIN% %*
+goto finally
+
+:err
+echo JAVA_HOME environment variable must be set!
+set ERRORLEVEL=1
+pause
+
+:finally
+ENDLOCAL & set RC=%ERRORLEVEL%
+goto :exit_with_code
+
+:returncode
+exit /B %RC%
+
+:exit_with_code
+call :returncode %RC%
\ No newline at end of file
diff --git a/tools/bin/sstablerepairedset.bat b/tools/bin/sstablerepairedset.bat
new file mode 100644
index 0000000..f7a4eb8
--- /dev/null
+++ b/tools/bin/sstablerepairedset.bat
@@ -0,0 +1,46 @@
+@REM  Licensed to the Apache Software Foundation (ASF) under one or more
+@REM  contributor license agreements.  See the NOTICE file distributed with
+@REM  this work for additional information regarding copyright ownership.
+@REM  The ASF licenses this file to You under the Apache License, Version 2.0
+@REM  (the "License"); you may not use this file except in compliance with
+@REM  the License.  You may obtain a copy of the License at
+@REM
+@REM      http://www.apache.org/licenses/LICENSE-2.0
+@REM
+@REM  Unless required by applicable law or agreed to in writing, software
+@REM  distributed under the License is distributed on an "AS IS" BASIS,
+@REM  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+@REM  See the License for the specific language governing permissions and
+@REM  limitations under the License.
+
+@echo off
+
+if "%OS%" == "Windows_NT" setlocal
+
+pushd "%~dp0"
+call cassandra.in.bat
+
+if NOT DEFINED JAVA_HOME goto :err
+set TOOL_MAIN=org.apache.cassandra.tools.SSTableRepairedAtSetter
+
+REM ***** JAVA options *****
+set JAVA_OPTS=^
+ -Dlogback.configurationFile=logback-tools.xml
+
+"%JAVA_HOME%\bin\java" %JAVA_OPTS% %CASSANDRA_PARAMS% -cp %CLASSPATH% %TOOL_MAIN% %*
+goto finally
+
+:err
+echo JAVA_HOME environment variable must be set!
+set ERRORLEVEL=1
+pause
+
+:finally
+ENDLOCAL & set RC=%ERRORLEVEL%
+goto :exit_with_code
+
+:returncode
+exit /B %RC%
+
+:exit_with_code
+call :returncode %RC%
\ No newline at end of file
diff --git a/tools/cqlstress-example.yaml b/tools/cqlstress-example.yaml
index 4dd5e4a..3c60c32 100644
--- a/tools/cqlstress-example.yaml
+++ b/tools/cqlstress-example.yaml
@@ -42,9 +42,10 @@
         uid timeuuid,
         value blob,
         PRIMARY KEY((name,choice), date, address, dbl, lval, ival, uid)
-  ) WITH COMPACT STORAGE 
-    AND compaction = { 'class':'LeveledCompactionStrategy' }
-    AND comment='A table of many types to test wide rows'
+  ) 
+    WITH compaction = { 'class':'LeveledCompactionStrategy' }
+#    AND compression = { 'sstable_compression' : '' }
+#    AND comment='A table of many types to test wide rows'
 
 #
 # Optional meta information on the generated columns in the above table
@@ -67,7 +68,7 @@
 columnspec:
   - name: name
     size: uniform(1..10)
-    population: uniform(1..1M)     # the range of unique values to select for the field (default is 100Billion)
+    population: uniform(1..10)     # the range of unique values to select for the field (default is 100Billion)
   - name: date
     cluster: uniform(20..40)
   - name: lval
diff --git a/tools/cqlstress-insanity-example.yaml b/tools/cqlstress-insanity-example.yaml
index ea4f97f..a286625 100644
--- a/tools/cqlstress-insanity-example.yaml
+++ b/tools/cqlstress-insanity-example.yaml
@@ -41,8 +41,6 @@
         fval float,
         ival int,
         uid timeuuid,
-        dates list<timestamp>,
-        inets set<inet>,
         value blob,
         PRIMARY KEY((name, choice), date)
   ) WITH compaction = { 'class':'LeveledCompactionStrategy' }
diff --git a/tools/lib/cassandra-driver-core-2.0.9.2.jar b/tools/lib/cassandra-driver-core-2.0.9.2.jar
deleted file mode 100644
index 3f82e77..0000000
--- a/tools/lib/cassandra-driver-core-2.0.9.2.jar
+++ /dev/null
Binary files differ
diff --git a/tools/lib/netty-3.9.0.Final.jar b/tools/lib/netty-3.9.0.Final.jar
deleted file mode 100644
index 872340e..0000000
--- a/tools/lib/netty-3.9.0.Final.jar
+++ /dev/null
Binary files differ
diff --git a/tools/stress/src/org/apache/cassandra/stress/Stress.java b/tools/stress/src/org/apache/cassandra/stress/Stress.java
index 256cefb..a4ec8a0 100644
--- a/tools/stress/src/org/apache/cassandra/stress/Stress.java
+++ b/tools/stress/src/org/apache/cassandra/stress/Stress.java
@@ -22,6 +22,8 @@
 import java.net.SocketException;
 
 import org.apache.cassandra.stress.settings.StressSettings;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.WindowsTimer;
 
 public final class Stress
 {
@@ -52,6 +54,9 @@
 
     public static void main(String[] arguments) throws Exception
     {
+        if (FBUtilities.isWindows())
+            WindowsTimer.startTimerPeriod(1);
+
         final StressSettings settings;
         try
         {
@@ -109,6 +114,8 @@
             stressAction.run();
         }
 
+        if (FBUtilities.isWindows())
+            WindowsTimer.endTimerPeriod(1);
         System.exit(0);
     }
 
diff --git a/tools/stress/src/org/apache/cassandra/stress/StressMetrics.java b/tools/stress/src/org/apache/cassandra/stress/StressMetrics.java
index 46ca488..a640058 100644
--- a/tools/stress/src/org/apache/cassandra/stress/StressMetrics.java
+++ b/tools/stress/src/org/apache/cassandra/stress/StressMetrics.java
@@ -31,7 +31,6 @@
 
 import org.apache.cassandra.stress.util.*;
 import org.apache.commons.lang3.time.DurationFormatUtils;
-
 import org.apache.cassandra.concurrent.NamedThreadFactory;
 import org.apache.cassandra.stress.settings.StressSettings;
 
diff --git a/tools/stress/src/org/apache/cassandra/stress/StressProfile.java b/tools/stress/src/org/apache/cassandra/stress/StressProfile.java
index 192b535..410f666 100644
--- a/tools/stress/src/org/apache/cassandra/stress/StressProfile.java
+++ b/tools/stress/src/org/apache/cassandra/stress/StressProfile.java
@@ -243,14 +243,21 @@
                     try
                     {
                         JavaDriverClient jclient = settings.getJavaDriverClient();
-                        ThriftClient tclient = settings.getThriftClient();
+                        ThriftClient tclient = null;
+
+                        if (settings.mode.api != ConnectionAPI.JAVA_DRIVER_NATIVE)
+                            tclient = settings.getThriftClient();
+
                         Map<String, PreparedStatement> stmts = new HashMap<>();
                         Map<String, Integer> tids = new HashMap<>();
                         Map<String, SchemaQuery.ArgSelect> args = new HashMap<>();
                         for (Map.Entry<String, StressYaml.QueryDef> e : queries.entrySet())
                         {
                             stmts.put(e.getKey().toLowerCase(), jclient.prepare(e.getValue().cql));
-                            tids.put(e.getKey().toLowerCase(), tclient.prepare_cql3_query(e.getValue().cql, Compression.NONE));
+
+                            if (tclient != null)
+                                tids.put(e.getKey().toLowerCase(), tclient.prepare_cql3_query(e.getValue().cql, Compression.NONE));
+
                             args.put(e.getKey().toLowerCase(), e.getValue().fields == null
                                                                      ? SchemaQuery.ArgSelect.MULTIROW
                                                                      : SchemaQuery.ArgSelect.valueOf(e.getValue().fields.toUpperCase()));
@@ -354,33 +361,38 @@
                     // guarantee the vast majority of actions occur in these bounds
                     double minBatchSize = selectchance.get().min() * partitions.get().minValue() * generator.minRowCount * (1d / visits.maxValue());
                     double maxBatchSize = selectchance.get().max() * partitions.get().maxValue() * generator.maxRowCount * (1d / visits.minValue());
-                    System.out.printf("Generating batches with [%d..%d] partitions and [%.0f..%.0f] rows (of [%.0f..%.0f] total rows in the partitions)\n",
+                    System.out.printf("Generating batches with [%d..%d] partitions and [%.0f..%.0f] rows (of [%.0f..%.0f] total rows in the partitions)%n",
                                       partitions.get().minValue(), partitions.get().maxValue(),
                                       minBatchSize, maxBatchSize,
                                       partitions.get().minValue() * generator.minRowCount,
                                       partitions.get().maxValue() * generator.maxRowCount);
                     if (generator.maxRowCount > 100 * 1000 * 1000)
-                        System.err.printf("WARNING: You have defined a schema that permits very large partitions (%.0f max rows (>100M))\n", generator.maxRowCount);
+                        System.err.printf("WARNING: You have defined a schema that permits very large partitions (%.0f max rows (>100M))%n", generator.maxRowCount);
                     if (batchType == BatchStatement.Type.LOGGED && maxBatchSize > 65535)
                     {
-                        System.err.printf("ERROR: You have defined a workload that generates batches with more than 65k rows (%.0f), but have required the use of LOGGED batches. There is a 65k row limit on a single batch.\n",
+                        System.err.printf("ERROR: You have defined a workload that generates batches with more than 65k rows (%.0f), but have required the use of LOGGED batches. There is a 65k row limit on a single batch.%n",
                                           selectchance.get().max() * partitions.get().maxValue() * generator.maxRowCount);
                         System.exit(1);
                     }
                     if (maxBatchSize > 100000)
-                        System.err.printf("WARNING: You have defined a schema that permits very large batches (%.0f max rows (>100K)). This may OOM this stress client, or the server.\n",
+                        System.err.printf("WARNING: You have defined a schema that permits very large batches (%.0f max rows (>100K)). This may OOM this stress client, or the server.%n",
                                           selectchance.get().max() * partitions.get().maxValue() * generator.maxRowCount);
 
                     JavaDriverClient client = settings.getJavaDriverClient();
                     String query = sb.toString();
-                    try
+
+                    if (settings.mode.api != ConnectionAPI.JAVA_DRIVER_NATIVE)
                     {
-                        thriftInsertId = settings.getThriftClient().prepare_cql3_query(query, Compression.NONE);
+                        try
+                        {
+                            thriftInsertId = settings.getThriftClient().prepare_cql3_query(query, Compression.NONE);
+                        }
+                        catch (TException e)
+                        {
+                            throw new RuntimeException(e);
+                        }
                     }
-                    catch (TException e)
-                    {
-                        throw new RuntimeException(e);
-                    }
+
                     insertStatement = client.prepare(query);
                 }
             }
@@ -519,6 +531,14 @@
                     return new UUIDs(name, config);
                 case TIMEUUID:
                     return new TimeUUIDs(name, config);
+                case TINYINT:
+                    return new TinyInts(name, config);
+                case SMALLINT:
+                    return new SmallInts(name, config);
+                case TIME:
+                    return new Times(name, config);
+                case DATE:
+                    return new LocalDates(name, config);
                 case SET:
                     return new Sets(name, getGenerator(name, type.getTypeArguments().get(0), config), config);
                 case LIST:
diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/DistributionBoundApache.java b/tools/stress/src/org/apache/cassandra/stress/generate/DistributionBoundApache.java
index 23ce3e9..db077a5 100644
--- a/tools/stress/src/org/apache/cassandra/stress/generate/DistributionBoundApache.java
+++ b/tools/stress/src/org/apache/cassandra/stress/generate/DistributionBoundApache.java
@@ -61,7 +61,7 @@
     private static long bound(long min, long max, double val)
     {
         long r = (long) val;
-        if ((r >= min) & (r <= max))
+        if ((r >= min) && (r <= max))
             return r;
         if (r < min)
             return min;
@@ -72,7 +72,7 @@
 
     private static double boundDouble(long min, long max, double r)
     {
-        if ((r >= min) & (r <= max))
+        if ((r >= min) && (r <= max))
             return r;
         if (r < min)
             return min;
diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/values/LocalDates.java b/tools/stress/src/org/apache/cassandra/stress/generate/values/LocalDates.java
new file mode 100644
index 0000000..f079d35
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generate/values/LocalDates.java
@@ -0,0 +1,43 @@
+/*
+ *
+ * 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.stress.generate.values;
+
+import com.datastax.driver.core.LocalDate;
+import org.apache.cassandra.db.marshal.SimpleDateType;
+import org.joda.time.DateTimeZone;
+import org.joda.time.format.DateTimeFormat;
+import org.joda.time.format.DateTimeFormatter;
+
+public class LocalDates extends Generator<Integer>
+{
+
+    public LocalDates(String name, GeneratorConfig config)
+    {
+        super(SimpleDateType.instance, config, name, Integer.class);
+    }
+
+    public Integer generate()
+    {
+        return (int)identityDistribution.next();
+    }
+
+
+}
diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/values/SmallInts.java b/tools/stress/src/org/apache/cassandra/stress/generate/values/SmallInts.java
new file mode 100644
index 0000000..702b6dc
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generate/values/SmallInts.java
@@ -0,0 +1,38 @@
+/*
+ *
+ * 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.stress.generate.values;
+
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.ShortType;
+
+public class SmallInts extends Generator<Short>
+{
+    public SmallInts(String name, GeneratorConfig config)
+    {
+        super(ShortType.instance, config, name, Short.class);
+    }
+
+    public Short generate()
+    {
+        long seed = identityDistribution.next();
+        return (short)seed;
+    }
+}
diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/values/Times.java b/tools/stress/src/org/apache/cassandra/stress/generate/values/Times.java
new file mode 100644
index 0000000..35bac86
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generate/values/Times.java
@@ -0,0 +1,37 @@
+/*
+ *
+ * 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.stress.generate.values;
+
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.TimeType;
+
+public class Times extends Generator<Long>
+{
+    public Times(String name, GeneratorConfig config)
+    {
+        super(TimeType.instance, config, name, Long.class);
+    }
+
+    public Long generate()
+    {
+        return identityDistribution.next();
+    }
+}
diff --git a/tools/stress/src/org/apache/cassandra/stress/generate/values/TinyInts.java b/tools/stress/src/org/apache/cassandra/stress/generate/values/TinyInts.java
new file mode 100644
index 0000000..0fe3f35
--- /dev/null
+++ b/tools/stress/src/org/apache/cassandra/stress/generate/values/TinyInts.java
@@ -0,0 +1,45 @@
+/*
+ *
+ * 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.stress.generate.values;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+import org.apache.cassandra.db.marshal.ByteType;
+import org.apache.cassandra.db.marshal.DecimalType;
+import org.apache.cassandra.db.marshal.IntegerType;
+import org.apache.cassandra.db.marshal.ShortType;
+import org.apache.cassandra.stress.generate.FasterRandom;
+
+public class TinyInts extends Generator<Byte>
+{
+    public TinyInts(String name, GeneratorConfig config)
+    {
+        super(ByteType.instance, config, name, Byte.class);
+    }
+
+    public Byte generate()
+    {
+        long seed = identityDistribution.next();
+        return (byte)seed;
+    }
+}
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/SampledOpDistributionFactory.java b/tools/stress/src/org/apache/cassandra/stress/operations/SampledOpDistributionFactory.java
index a20272a..194f84f 100644
--- a/tools/stress/src/org/apache/cassandra/stress/operations/SampledOpDistributionFactory.java
+++ b/tools/stress/src/org/apache/cassandra/stress/operations/SampledOpDistributionFactory.java
@@ -29,6 +29,8 @@
 import org.apache.commons.math3.util.Pair;
 
 import org.apache.cassandra.stress.Operation;
+import org.apache.cassandra.stress.generate.DistributionFactory;
+import org.apache.cassandra.stress.generate.PartitionGenerator;
 import org.apache.cassandra.stress.util.Timer;
 
 public abstract class SampledOpDistributionFactory<T> implements OpDistributionFactory
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/predefined/CqlCounterAdder.java b/tools/stress/src/org/apache/cassandra/stress/operations/predefined/CqlCounterAdder.java
index 456c821..bb8135c 100644
--- a/tools/stress/src/org/apache/cassandra/stress/operations/predefined/CqlCounterAdder.java
+++ b/tools/stress/src/org/apache/cassandra/stress/operations/predefined/CqlCounterAdder.java
@@ -46,14 +46,7 @@
     @Override
     protected String buildQuery()
     {
-        String counterCF = isCql2() ? type.table : "Counter3";
-
-        StringBuilder query = new StringBuilder("UPDATE ").append(wrapInQuotesIfRequired(counterCF));
-
-        if (isCql2())
-            query.append(" USING CONSISTENCY ").append(settings.command.consistencyLevel);
-
-        query.append(" SET ");
+        StringBuilder query = new StringBuilder("UPDATE counter1 SET ");
 
         // TODO : increment distribution subset of columns
         for (int i = 0; i < settings.columns.maxColumnsPerKey; i++)
@@ -61,7 +54,8 @@
             if (i > 0)
                 query.append(",");
 
-            query.append('C').append(i).append("=C").append(i).append("+?");
+            String name = wrapInQuotes(settings.columns.namestrs.get(i));
+            query.append(name).append("=").append(name).append("+?");
         }
         query.append(" WHERE KEY=?");
         return query.toString();
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/predefined/CqlCounterGetter.java b/tools/stress/src/org/apache/cassandra/stress/operations/predefined/CqlCounterGetter.java
index 8c1c65c..d91ab37 100644
--- a/tools/stress/src/org/apache/cassandra/stress/operations/predefined/CqlCounterGetter.java
+++ b/tools/stress/src/org/apache/cassandra/stress/operations/predefined/CqlCounterGetter.java
@@ -48,22 +48,7 @@
     @Override
     protected String buildQuery()
     {
-        StringBuilder query = new StringBuilder("SELECT ");
-
-        // TODO: obey slice/noslice option (instead of always slicing)
-        if (isCql2())
-            query.append("FIRST ").append(settings.columns.maxColumnsPerKey).append(" ''..''");
-        else
-            query.append("*");
-
-        String counterCF = isCql2() ? type.table : "Counter3";
-
-        query.append(" FROM ").append(wrapInQuotesIfRequired(counterCF));
-
-        if (isCql2())
-            query.append(" USING CONSISTENCY ").append(settings.command.consistencyLevel);
-
-        return query.append(" WHERE KEY=?").toString();
+        return "SELECT * FROM " + wrapInQuotes(type.table) + " WHERE KEY=?";
     }
 
     @Override
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/predefined/CqlInserter.java b/tools/stress/src/org/apache/cassandra/stress/operations/predefined/CqlInserter.java
index 88ee752..fdf5007 100644
--- a/tools/stress/src/org/apache/cassandra/stress/operations/predefined/CqlInserter.java
+++ b/tools/stress/src/org/apache/cassandra/stress/operations/predefined/CqlInserter.java
@@ -42,10 +42,9 @@
     @Override
     protected String buildQuery()
     {
-        StringBuilder query = new StringBuilder("UPDATE ").append(wrapInQuotesIfRequired(type.table));
-
-        if (isCql2())
-            query.append(" USING CONSISTENCY ").append(settings.command.consistencyLevel);
+        StringBuilder query = new StringBuilder("UPDATE ").append(wrapInQuotes(type.table));
+        if (settings.columns.timestamp != null)
+            query.append(" USING TIMESTAMP ").append(settings.columns.timestamp);
 
         query.append(" SET ");
 
@@ -54,7 +53,7 @@
             if (i > 0)
                 query.append(',');
 
-            query.append(wrapInQuotesIfRequired(settings.columns.namestrs.get(i))).append(" = ?");
+            query.append(wrapInQuotes(settings.columns.namestrs.get(i))).append(" = ?");
         }
 
         query.append(" WHERE KEY=?");
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/predefined/CqlOperation.java b/tools/stress/src/org/apache/cassandra/stress/operations/predefined/CqlOperation.java
index 9cea854..afdc0b1 100644
--- a/tools/stress/src/org/apache/cassandra/stress/operations/predefined/CqlOperation.java
+++ b/tools/stress/src/org/apache/cassandra/stress/operations/predefined/CqlOperation.java
@@ -33,7 +33,6 @@
 import org.apache.cassandra.stress.generate.SeedManager;
 import org.apache.cassandra.stress.settings.Command;
 import org.apache.cassandra.stress.settings.ConnectionStyle;
-import org.apache.cassandra.stress.settings.CqlVersion;
 import org.apache.cassandra.stress.settings.StressSettings;
 import org.apache.cassandra.stress.util.JavaDriverClient;
 import org.apache.cassandra.stress.util.ThriftClient;
@@ -261,10 +260,7 @@
 
     public ClientWrapper wrap(ThriftClient client)
     {
-        return isCql3()
-                ? new Cql3CassandraClientWrapper(client)
-                : new Cql2CassandraClientWrapper(client);
-
+        return new Cql3CassandraClientWrapper(client);
     }
 
     public ClientWrapper wrap(JavaDriverClient client)
@@ -295,7 +291,7 @@
         @Override
         public <V> V execute(String query, ByteBuffer key, List<Object> queryParams, ResultHandler<V> handler)
         {
-            String formattedQuery = formatCqlQuery(query, queryParams, isCql3());
+            String formattedQuery = formatCqlQuery(query, queryParams);
             return handler.javaDriverHandler().apply(client.execute(formattedQuery, ThriftConversion.fromThrift(settings.command.consistencyLevel)));
         }
 
@@ -327,7 +323,7 @@
         @Override
         public <V> V execute(String query, ByteBuffer key, List<Object> queryParams, ResultHandler<V> handler)
         {
-            String formattedQuery = formatCqlQuery(query, queryParams, isCql3());
+            String formattedQuery = formatCqlQuery(query, queryParams);
             return handler.thriftHandler().apply(client.execute(formattedQuery, ThriftConversion.fromThrift(settings.command.consistencyLevel)));
         }
 
@@ -360,7 +356,7 @@
         @Override
         public <V> V execute(String query, ByteBuffer key, List<Object> queryParams, ResultHandler<V> handler) throws TException
         {
-            String formattedQuery = formatCqlQuery(query, queryParams, true);
+            String formattedQuery = formatCqlQuery(query, queryParams);
             return handler.simpleNativeHandler().apply(
                     client.execute_cql3_query(formattedQuery, key, Compression.NONE, settings.command.consistencyLevel)
             );
@@ -382,40 +378,6 @@
         }
     }
 
-    // client wrapper for Cql2
-    private final class Cql2CassandraClientWrapper implements ClientWrapper
-    {
-        final ThriftClient client;
-        private Cql2CassandraClientWrapper(ThriftClient client)
-        {
-            this.client = client;
-        }
-
-        @Override
-        public <V> V execute(String query, ByteBuffer key, List<Object> queryParams, ResultHandler<V> handler) throws TException
-        {
-            String formattedQuery = formatCqlQuery(query, queryParams, false);
-            return handler.simpleNativeHandler().apply(
-                    client.execute_cql_query(formattedQuery, key, Compression.NONE)
-            );
-        }
-
-        @Override
-        public <V> V execute(Object preparedStatementId, ByteBuffer key, List<Object> queryParams, ResultHandler<V> handler) throws TException
-        {
-            Integer id = (Integer) preparedStatementId;
-            return handler.simpleNativeHandler().apply(
-                    client.execute_prepared_cql_query(id, key, toByteBufferParams(queryParams))
-            );
-        }
-
-        @Override
-        public Object createPreparedStatement(String cqlQuery) throws TException
-        {
-            return client.prepare_cql_query(cqlQuery, Compression.NONE);
-        }
-    }
-
     // interface for building functions to standardise results from each client
     protected static interface ResultHandler<V>
     {
@@ -625,11 +587,9 @@
 
     }
 
-    private static String getUnQuotedCqlBlob(ByteBuffer term, boolean isCQL3)
+    private static String getUnQuotedCqlBlob(ByteBuffer term)
     {
-        return isCQL3
-                ? "0x" + ByteBufferUtil.bytesToHex(term)
-                : ByteBufferUtil.bytesToHex(term);
+        return "0x" + ByteBufferUtil.bytesToHex(term);
     }
 
     /**
@@ -640,7 +600,7 @@
      * @param parms sequence of string query parameters
      * @return formatted CQL query string
      */
-    private static String formatCqlQuery(String query, List<Object> parms, boolean isCql3)
+    private static String formatCqlQuery(String query, List<Object> parms)
     {
         int marker, position = 0;
         StringBuilder result = new StringBuilder();
@@ -651,10 +611,11 @@
         for (Object parm : parms)
         {
             result.append(query.substring(position, marker));
+
             if (parm instanceof ByteBuffer)
-                result.append(getUnQuotedCqlBlob((ByteBuffer) parm, isCql3));
+                result.append(getUnQuotedCqlBlob((ByteBuffer) parm));
             else if (parm instanceof Long)
-                result.append(parm.toString());
+                result.append(parm);
             else throw new AssertionError();
 
             position = marker + 1;
@@ -682,11 +643,9 @@
         return r;
     }
 
-    protected String wrapInQuotesIfRequired(String string)
+    protected String wrapInQuotes(String string)
     {
-        return settings.mode.cqlVersion == CqlVersion.CQL3
-                ? "\"" + string + "\""
-                : string;
+        return "\"" + string + "\"";
     }
 
 }
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/predefined/CqlReader.java b/tools/stress/src/org/apache/cassandra/stress/operations/predefined/CqlReader.java
index 12cc241..0ee17e9 100644
--- a/tools/stress/src/org/apache/cassandra/stress/operations/predefined/CqlReader.java
+++ b/tools/stress/src/org/apache/cassandra/stress/operations/predefined/CqlReader.java
@@ -47,10 +47,7 @@
 
         if (settings.columns.slice)
         {
-            if (isCql2())
-                query.append("FIRST ").append(settings.columns.maxColumnsPerKey).append(" ''..''");
-            else
-                query.append("*");
+            query.append("*");
         }
         else
         {
@@ -58,14 +55,11 @@
             {
                 if (i > 0)
                     query.append(",");
-                query.append(wrapInQuotesIfRequired(settings.columns.namestrs.get(i)));
+                query.append(wrapInQuotes(settings.columns.namestrs.get(i)));
             }
         }
 
-        query.append(" FROM ").append(wrapInQuotesIfRequired(type.table));
-
-        if (isCql2())
-            query.append(" USING CONSISTENCY ").append(settings.command.consistencyLevel);
+        query.append(" FROM ").append(wrapInQuotes(type.table));
         query.append(" WHERE KEY=?");
         return query.toString();
     }
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/predefined/PredefinedOperation.java b/tools/stress/src/org/apache/cassandra/stress/operations/predefined/PredefinedOperation.java
index d5c3edc..89298ab 100644
--- a/tools/stress/src/org/apache/cassandra/stress/operations/predefined/PredefinedOperation.java
+++ b/tools/stress/src/org/apache/cassandra/stress/operations/predefined/PredefinedOperation.java
@@ -54,10 +54,6 @@
     {
         return settings.mode.cqlVersion == CqlVersion.CQL3;
     }
-    public boolean isCql2()
-    {
-        return settings.mode.cqlVersion == CqlVersion.CQL2;
-    }
     public Object getCqlCache()
     {
         return cqlCache;
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/userdefined/SchemaStatement.java b/tools/stress/src/org/apache/cassandra/stress/operations/userdefined/SchemaStatement.java
index e90de23..49891ec 100644
--- a/tools/stress/src/org/apache/cassandra/stress/operations/userdefined/SchemaStatement.java
+++ b/tools/stress/src/org/apache/cassandra/stress/operations/userdefined/SchemaStatement.java
@@ -28,15 +28,15 @@
 
 import com.datastax.driver.core.BoundStatement;
 import com.datastax.driver.core.ColumnDefinitions;
+import com.datastax.driver.core.DataType;
+import com.datastax.driver.core.LocalDate;
 import com.datastax.driver.core.PreparedStatement;
-import com.datastax.driver.core.ResultSet;
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.stress.Operation;
 import org.apache.cassandra.stress.generate.Row;
 import org.apache.cassandra.stress.settings.StressSettings;
 import org.apache.cassandra.stress.util.JavaDriverClient;
 import org.apache.cassandra.stress.util.Timer;
-import org.apache.cassandra.thrift.CqlResult;
 import org.apache.cassandra.transport.SimpleClient;
 
 public abstract class SchemaStatement extends Operation
@@ -47,6 +47,7 @@
     final ConsistencyLevel cl;
     final int[] argumentIndex;
     final Object[] bindBuffer;
+    final ColumnDefinitions definitions;
 
     public SchemaStatement(Timer timer, StressSettings settings, DataSpec spec,
                            PreparedStatement statement, Integer thriftId, ConsistencyLevel cl)
@@ -57,8 +58,9 @@
         this.cl = cl;
         argumentIndex = new int[statement.getVariables().size()];
         bindBuffer = new Object[argumentIndex.length];
+        definitions = statement.getVariables();
         int i = 0;
-        for (ColumnDefinitions.Definition definition : statement.getVariables())
+        for (ColumnDefinitions.Definition definition : definitions)
             argumentIndex[i++] = spec.partitionGenerator.indexOf(definition.getName());
 
         statement.setConsistencyLevel(JavaDriverClient.from(cl));
@@ -68,7 +70,13 @@
     {
         for (int i = 0 ; i < argumentIndex.length ; i++)
         {
-            bindBuffer[i] = row.get(argumentIndex[i]);
+            Object value = row.get(argumentIndex[i]);
+            if (definitions.getType(i).getName().equals(DataType.date().getName()))
+            {
+                // the java driver only accepts com.datastax.driver.core.LocalDate for CQL type "DATE"
+                value= LocalDate.fromDaysSinceEpoch((Integer) value);
+            }
+            bindBuffer[i] = value;
             if (bindBuffer[i] == null && !spec.partitionGenerator.permitNulls(argumentIndex[i]))
                 throw new IllegalStateException();
         }
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/userdefined/ValidatingSchemaQuery.java b/tools/stress/src/org/apache/cassandra/stress/operations/userdefined/ValidatingSchemaQuery.java
index 2cbdcb4..1b10fcf 100644
--- a/tools/stress/src/org/apache/cassandra/stress/operations/userdefined/ValidatingSchemaQuery.java
+++ b/tools/stress/src/org/apache/cassandra/stress/operations/userdefined/ValidatingSchemaQuery.java
@@ -131,8 +131,6 @@
                     valueIndex[i++] = spec.partitionGenerator.indexOf(definition.getName());
             }
 
-            List<Object[]> prev1 = new ArrayList<>();
-            List<Object[]> prev2 = new ArrayList<>();
             rowCount = 0;
             Iterator<com.datastax.driver.core.Row> results = rs.iterator();
             if (!statements[statementIndex].inclusiveStart && iter.hasNext())
@@ -148,24 +146,16 @@
 
                 rowCount++;
                 com.datastax.driver.core.Row actualRow = results.next();
-                Object[] vs1 = new Object[actualRow.getColumnDefinitions().size()];
-                Object[] vs2 = vs1.clone();
                 for (int i = 0 ; i < actualRow.getColumnDefinitions().size() ; i++)
                 {
                     Object expectedValue = expectedRow.get(valueIndex[i]);
                     Object actualValue = spec.partitionGenerator.convert(valueIndex[i], actualRow.getBytesUnsafe(i));
-                    vs1[i] = expectedValue;
-                    vs2[i] = actualValue;
                     if (!expectedValue.equals(actualValue))
                         return false;
                 }
-                prev1.add(vs1);
-                prev2.add(vs2);
             }
             partitionCount = Math.min(1, rowCount);
-            if (!rs.isExhausted())
-                return false;
-            return true;
+            return rs.isExhausted();
         }
     }
 
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/CqlVersion.java b/tools/stress/src/org/apache/cassandra/stress/settings/CqlVersion.java
index d7d09f6..1e963c7 100644
--- a/tools/stress/src/org/apache/cassandra/stress/settings/CqlVersion.java
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/CqlVersion.java
@@ -25,7 +25,6 @@
 {
 
     NOCQL(null),
-    CQL2("2.0.0"),
     CQL3("3.0.0");
 
     public final String connectVersion;
@@ -41,8 +40,6 @@
             return NOCQL;
         switch(version.charAt(0))
         {
-            case '2':
-                return CQL2;
             case '3':
                 return CQL3;
             default:
@@ -55,11 +52,6 @@
         return this != NOCQL;
     }
 
-    public boolean isCql2()
-    {
-        return this == CQL2;
-    }
-
     public boolean isCql3()
     {
         return this == CQL3;
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/GroupedOptions.java b/tools/stress/src/org/apache/cassandra/stress/settings/GroupedOptions.java
index 8bbba15..f3924c0 100644
--- a/tools/stress/src/org/apache/cassandra/stress/settings/GroupedOptions.java
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/GroupedOptions.java
@@ -87,7 +87,7 @@
             }
             firstRow = false;
 
-            StringBuilder sb = new StringBuilder("Usage: " + command);
+            StringBuilder sb = new StringBuilder("Usage: ").append(command);
             for (Option option : grouping.options())
             {
                 sb.append(" ");
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/Legacy.java b/tools/stress/src/org/apache/cassandra/stress/settings/Legacy.java
index 7f17893..70693af 100644
--- a/tools/stress/src/org/apache/cassandra/stress/settings/Legacy.java
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/Legacy.java
@@ -60,7 +60,6 @@
         availableOptions.addOption("i",  "progress-interval",    true,   "Progress Report Interval (seconds), default:10");
         availableOptions.addOption("g",  "keys-per-call",        true,   "Number of keys to get_range_slices or multiget per call, default:1000");
         availableOptions.addOption("l",  "replication-factor",   true,   "Replication Factor to use when creating needed column families, default:1");
-        availableOptions.addOption("L",  "enable-cql",           false,  "Perform queries using CQL2 (Cassandra Query Language v 2.0.0)");
         availableOptions.addOption("L3", "enable-cql3",          false,  "Perform queries using CQL3 (Cassandra Query Language v 3.0.0)");
         availableOptions.addOption("b",  "enable-native-protocol",  false,  "Use the binary native protocol (only work along with -L3)");
         availableOptions.addOption("P",  "use-prepared-statements", false, "Perform queries using prepared statements (only applicable to CQL).");
@@ -216,12 +215,12 @@
             {
                 StringBuilder rep = new StringBuilder();
                 if (cmd.hasOption("R"))
-                    rep.append("strategy=" + cmd.getOptionValue("R"));
+                    rep.append("strategy=").append(cmd.getOptionValue("R"));
                 if (cmd.hasOption("l"))
                 {
                     if (rep.length() > 0)
                         rep.append(",");
-                    rep.append("factor=" + cmd.getOptionValue("l"));
+                    rep.append("factor=").append(cmd.getOptionValue("l"));
                 }
                 if (cmd.hasOption("O"))
                 {
@@ -232,9 +231,7 @@
                 r.add("-schema", "replication(" + rep + ")");
             }
 
-            if (cmd.hasOption("L"))
-                r.add("-mode", cmd.hasOption("P") ? "prepared cql2" : "cql2");
-            else if (cmd.hasOption("L3"))
+            if (cmd.hasOption("L3"))
                 r.add("-mode", (cmd.hasOption("P") ? "prepared" : "") + (cmd.hasOption("b") ? "native" : "") +  "cql3");
             else
                 r.add("-mode", "thrift");
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/OptionAnyProbabilities.java b/tools/stress/src/org/apache/cassandra/stress/settings/OptionAnyProbabilities.java
index 28d8f96..1a444a0 100644
--- a/tools/stress/src/org/apache/cassandra/stress/settings/OptionAnyProbabilities.java
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/OptionAnyProbabilities.java
@@ -42,7 +42,7 @@
             String[] args = param.split("=");
             if (args.length == 2 && args[1].length() > 0 && args[0].length() > 0)
             {
-                if (options.put(args[0], Double.parseDouble(args[1])) != null)
+                if (options.put(args[0], Double.valueOf(args[1])) != null)
                     throw new IllegalArgumentException(args[0] + " set twice");
                 return true;
             }
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/OptionReplication.java b/tools/stress/src/org/apache/cassandra/stress/settings/OptionReplication.java
index 8d1a6fc..141c645 100644
--- a/tools/stress/src/org/apache/cassandra/stress/settings/OptionReplication.java
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/OptionReplication.java
@@ -83,6 +83,7 @@
                     break;
                 } catch (Exception ignore)
                 {
+                    // will throw below if strategy is still null
                 }
             }
             if (strategy == null)
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsMode.java b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsMode.java
index c95ed80..2c91b6d 100644
--- a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsMode.java
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsMode.java
@@ -95,18 +95,6 @@
             authProvider = null;
             authProviderClassname = null;
         }
-        else if (options instanceof Cql2ThriftOptions)
-        {
-            cqlVersion = CqlVersion.CQL2;
-            api = ConnectionAPI.THRIFT;
-            Cql2ThriftOptions opts = (Cql2ThriftOptions) options;
-            style = opts.usePrepared.setByUser() ? ConnectionStyle.CQL_PREPARED : ConnectionStyle.CQL;
-            compression = ProtocolOptions.Compression.NONE.name();
-            username = null;
-            password = null;
-            authProvider = null;
-            authProviderClassname = null;
-        }
         else if (options instanceof ThriftOptions)
         {
             ThriftOptions opts = (ThriftOptions) options;
@@ -181,19 +169,6 @@
         }
     }
 
-    private static final class Cql2ThriftOptions extends GroupedOptions
-    {
-        final OptionSimple api = new OptionSimple("cql2", "", null, "", true);
-        final OptionSimple mode = new OptionSimple("thrift", "", null, "", true);
-        final OptionSimple usePrepared = new OptionSimple("prepared", "", null, "", false);
-
-        @Override
-        public List<? extends Option> options()
-        {
-            return Arrays.asList(mode, api, usePrepared);
-        }
-    }
-
     private static final class ThriftOptions extends GroupedOptions
     {
         final OptionSimple api = new OptionSimple("thrift", "", null, "", true);
@@ -223,7 +198,7 @@
             return new SettingsMode(opts);
         }
 
-        GroupedOptions options = GroupedOptions.select(params, new ThriftOptions(), new Cql3NativeOptions(), new Cql3ThriftOptions(), new Cql3SimpleNativeOptions(), new Cql2ThriftOptions());
+        GroupedOptions options = GroupedOptions.select(params, new ThriftOptions(), new Cql3NativeOptions(), new Cql3SimpleNativeOptions());
         if (options == null)
         {
             printHelp();
@@ -235,7 +210,7 @@
 
     public static void printHelp()
     {
-        GroupedOptions.printOptions(System.out, "-mode", new ThriftOptions(), new Cql3NativeOptions(), new Cql3ThriftOptions(), new Cql3SimpleNativeOptions(), new Cql2ThriftOptions());
+        GroupedOptions.printOptions(System.out, "-mode", new ThriftOptions(), new Cql3NativeOptions(), new Cql3SimpleNativeOptions());
     }
 
     public static Runnable helpPrinter()
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsSchema.java b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsSchema.java
index 54f27bc..5c437a9 100644
--- a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsSchema.java
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsSchema.java
@@ -22,9 +22,13 @@
 
 import java.io.Serializable;
 import java.nio.ByteBuffer;
+import java.nio.charset.CharacterCodingException;
 import java.util.*;
 
+import com.datastax.driver.core.exceptions.AlreadyExistsException;
+import org.apache.cassandra.stress.util.JavaDriverClient;
 import org.apache.cassandra.thrift.*;
+import org.apache.cassandra.thrift.ConsistencyLevel;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 public class SettingsSchema implements Serializable
@@ -56,9 +60,159 @@
 
     public void createKeySpaces(StressSettings settings)
     {
-        createKeySpacesThrift(settings);
+        if (settings.mode.api != ConnectionAPI.JAVA_DRIVER_NATIVE)
+        {
+            createKeySpacesThrift(settings);
+        }
+        else
+        {
+            createKeySpacesNative(settings);
+        }
     }
 
+    /**
+     * Create Keyspace with Standard and Super/Counter column families
+     */
+    public void createKeySpacesNative(StressSettings settings)
+    {
+
+        JavaDriverClient client  = settings.getJavaDriverClient(false);
+
+        try
+        {
+            //Keyspace
+            client.execute(createKeyspaceStatementCQL3(), org.apache.cassandra.db.ConsistencyLevel.LOCAL_ONE);
+
+            client.execute("USE \""+keyspace+"\"", org.apache.cassandra.db.ConsistencyLevel.LOCAL_ONE);
+
+            //Add standard1 and counter1
+            client.execute(createStandard1StatementCQL3(settings), org.apache.cassandra.db.ConsistencyLevel.LOCAL_ONE);
+            client.execute(createCounter1StatementCQL3(settings), org.apache.cassandra.db.ConsistencyLevel.LOCAL_ONE);
+
+            System.out.println(String.format("Created keyspaces. Sleeping %ss for propagation.", settings.node.nodes.size()));
+            Thread.sleep(settings.node.nodes.size() * 1000L); // seconds
+        }
+        catch (AlreadyExistsException e)
+        {
+            //Ok.
+        }
+        catch (Exception e)
+        {
+            throw new RuntimeException("Encountered exception creating schema", e);
+        }
+    }
+
+    String createKeyspaceStatementCQL3()
+    {
+        StringBuilder b = new StringBuilder();
+
+        //Create Keyspace
+        b.append("CREATE KEYSPACE IF NOT EXISTS \"")
+         .append(keyspace)
+         .append("\" WITH replication = {'class': '")
+         .append(replicationStrategy)
+         .append("'");
+
+        if (replicationStrategyOptions.isEmpty())
+        {
+            b.append(", 'replication_factor': '1'}");
+        }
+        else
+        {
+            for(Map.Entry<String, String> entry : replicationStrategyOptions.entrySet())
+            {
+                b.append(", '").append(entry.getKey()).append("' : '").append(entry.getValue()).append("'");
+            }
+
+            b.append("}");
+        }
+
+        b.append(" AND durable_writes = true;\n");
+
+        return b.toString();
+    }
+
+    String createStandard1StatementCQL3(StressSettings settings)
+    {
+
+        StringBuilder b = new StringBuilder();
+
+        b.append("CREATE TABLE IF NOT EXISTS ")
+         .append("standard1 (key blob PRIMARY KEY ");
+
+        try
+        {
+            for (ByteBuffer name : settings.columns.names)
+                b.append("\n, \"").append(ByteBufferUtil.string(name)).append("\" blob");
+        }
+        catch (CharacterCodingException e)
+        {
+            throw new RuntimeException(e);
+        }
+
+        //Compression
+        b.append(") WITH COMPACT STORAGE AND compression = {");
+        if (compression != null)
+            b.append("'sstable_compression' : '").append(compression).append("'");
+
+        b.append("}");
+
+        //Compaction
+        if (compactionStrategy != null)
+        {
+            b.append(" AND compaction = { 'class' : '").append(compactionStrategy).append("'");
+
+            for (Map.Entry<String, String> entry : compactionStrategyOptions.entrySet())
+                b.append(", '").append(entry.getKey()).append("' : '").append(entry.getValue()).append("'");
+
+            b.append("}");
+        }
+
+        b.append(";\n");
+
+        return b.toString();
+    }
+
+    String createCounter1StatementCQL3(StressSettings settings)
+    {
+
+        StringBuilder b = new StringBuilder();
+
+        b.append("CREATE TABLE IF NOT EXISTS ")
+         .append("counter1 (key blob PRIMARY KEY,");
+
+        try
+        {
+            for (ByteBuffer name : settings.columns.names)
+                b.append("\n, \"").append(ByteBufferUtil.string(name)).append("\" counter");
+        }
+        catch (CharacterCodingException e)
+        {
+            throw new RuntimeException(e);
+        }
+
+        //Compression
+        b.append(") WITH COMPACT STORAGE AND compression = {");
+        if (compression != null)
+            b.append("'sstable_compression' : '").append(compression).append("'");
+
+        b.append("}");
+
+        //Compaction
+        if (compactionStrategy != null)
+        {
+            b.append(" AND compaction = { 'class' : '").append(compactionStrategy).append("'");
+
+            for (Map.Entry<String, String> entry : compactionStrategyOptions.entrySet())
+                b.append(", '").append(entry.getKey()).append("' : '").append(entry.getValue()).append("'");
+
+            b.append("}");
+        }
+
+        b.append(";\n");
+
+        return b.toString();
+    }
 
     /**
      * Create Keyspace with Standard and Super/Counter column families
@@ -113,16 +267,7 @@
         try
         {
             client.system_add_keyspace(ksdef);
-
-            /* CQL3 counter cf */
-            client.set_cql_version("3.0.0"); // just to create counter cf for cql3
-
             client.set_keyspace(keyspace);
-            client.execute_cql3_query(createCounterCFStatementForCQL3(settings), Compression.NONE, ConsistencyLevel.ONE);
-
-            if (settings.mode.cqlVersion.isCql())
-                client.set_cql_version(settings.mode.cqlVersion.connectVersion);
-            /* end */
 
             System.out.println(String.format("Created keyspaces. Sleeping %ss for propagation.", settings.node.nodes.size()));
             Thread.sleep(settings.node.nodes.size() * 1000L); // seconds
@@ -137,20 +282,6 @@
         }
     }
 
-    private ByteBuffer createCounterCFStatementForCQL3(StressSettings options)
-    {
-        StringBuilder counter3 = new StringBuilder("CREATE TABLE \"Counter3\" (KEY blob PRIMARY KEY, ");
-
-        for (int i = 0; i < options.columns.maxColumnsPerKey; i++)
-        {
-            counter3.append("c").append(i).append(" counter");
-            if (i != options.columns.maxColumnsPerKey - 1)
-                counter3.append(", ");
-        }
-        counter3.append(");");
-
-        return ByteBufferUtil.bytes(counter3.toString());
-    }
 
     // Option Declarations
 
diff --git a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsTransport.java b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsTransport.java
index 8b0ef6a..a253c07 100644
--- a/tools/stress/src/org/apache/cassandra/stress/settings/SettingsTransport.java
+++ b/tools/stress/src/org/apache/cassandra/stress/settings/SettingsTransport.java
@@ -65,6 +65,10 @@
             factoryOptions.put(SSLTransportFactory.TRUSTSTORE, options.trustStore.value());
         if (transportFactory.supportedOptions().contains(SSLTransportFactory.TRUSTSTORE_PASSWORD))
             factoryOptions.put(SSLTransportFactory.TRUSTSTORE_PASSWORD, options.trustStorePw.value());
+        if (transportFactory.supportedOptions().contains(SSLTransportFactory.KEYSTORE))
+            factoryOptions.put(SSLTransportFactory.KEYSTORE, options.keyStore.value());
+        if (transportFactory.supportedOptions().contains(SSLTransportFactory.KEYSTORE_PASSWORD))
+            factoryOptions.put(SSLTransportFactory.KEYSTORE_PASSWORD, options.keyStorePw.value());
         if (transportFactory.supportedOptions().contains(SSLTransportFactory.PROTOCOL))
             factoryOptions.put(SSLTransportFactory.PROTOCOL, options.protocol.value());
         if (transportFactory.supportedOptions().contains(SSLTransportFactory.CIPHER_SUITES))
@@ -102,6 +106,17 @@
             encOptions.enabled = true;
             encOptions.truststore = options.trustStore.value();
             encOptions.truststore_password = options.trustStorePw.value();
+            if (options.keyStore.present())
+            {
+                encOptions.keystore = options.keyStore.value();
+                encOptions.keystore_password = options.keyStorePw.value();
+            }
+            else
+            {
+                // mandatory for SSLFactory.createSSLContext(), see CASSANDRA-9325
+                encOptions.keystore = encOptions.truststore;
+                encOptions.keystore_password = encOptions.truststore_password;
+            }
             encOptions.algorithm = options.alg.value();
             encOptions.protocol = options.protocol.value();
             encOptions.cipher_suites = options.ciphers.value().split(",");
@@ -116,6 +131,8 @@
         final OptionSimple factory = new OptionSimple("factory=", ".*", TFramedTransportFactory.class.getName(), "Fully-qualified ITransportFactory class name for creating a connection. Note: For Thrift over SSL, use org.apache.cassandra.thrift.SSLTransportFactory.", false);
         final OptionSimple trustStore = new OptionSimple("truststore=", ".*", null, "SSL: full path to truststore", false);
         final OptionSimple trustStorePw = new OptionSimple("truststore-password=", ".*", null, "SSL: truststore password", false);
+        final OptionSimple keyStore = new OptionSimple("keystore=", ".*", null, "SSL: full path to keystore", false);
+        final OptionSimple keyStorePw = new OptionSimple("keystore-password=", ".*", null, "SSL: keystore password", false);
         final OptionSimple protocol = new OptionSimple("ssl-protocol=", ".*", "TLS", "SSL: connection protocol to use", false);
         final OptionSimple alg = new OptionSimple("ssl-alg=", ".*", "SunX509", "SSL: algorithm", false);
         final OptionSimple storeType = new OptionSimple("store-type=", ".*", "JKS", "SSL: keystore format", false);
@@ -124,7 +141,7 @@
         @Override
         public List<? extends Option> options()
         {
-            return Arrays.asList(factory, trustStore, trustStorePw, protocol, alg, storeType, ciphers);
+            return Arrays.asList(factory, trustStore, trustStorePw, keyStore, keyStorePw, protocol, alg, storeType, ciphers);
         }
     }
 
diff --git a/tools/stress/src/org/apache/cassandra/stress/util/JavaDriverClient.java b/tools/stress/src/org/apache/cassandra/stress/util/JavaDriverClient.java
index fa04dbc..30d0d4a 100644
--- a/tools/stress/src/org/apache/cassandra/stress/util/JavaDriverClient.java
+++ b/tools/stress/src/org/apache/cassandra/stress/util/JavaDriverClient.java
@@ -89,9 +89,13 @@
 
     public void connect(ProtocolOptions.Compression compression) throws Exception
     {
+        PoolingOptions poolingOpts = new PoolingOptions();
+        poolingOpts.setCoreConnectionsPerHost(HostDistance.LOCAL, 8);
         Cluster.Builder clusterBuilder = Cluster.builder()
                                                 .addContactPoint(host)
                                                 .withPort(port)
+                                                .withPoolingOptions(poolingOpts)
+                                                .withProtocolVersion(ProtocolVersion.NEWEST_SUPPORTED)
                                                 .withoutMetrics(); // The driver uses metrics 3 with conflict with our version
         if (whitelist != null)
             clusterBuilder.withLoadBalancingPolicy(whitelist);